From e9b70598fbdf32615416e613afffa4b730b957bb Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 15 Jan 2019 11:52:55 +1100 Subject: [PATCH 01/91] Create gossipsub crate - Basic template, borrowed from floodsub --- Cargo.toml | 2 + protocols/gossipsub/Cargo.toml | 25 + protocols/gossipsub/regen_structs_proto.sh | 13 + protocols/gossipsub/rpc.proto | 75 + protocols/gossipsub/src/handler.rs | 255 ++ protocols/gossipsub/src/layer.rs | 360 +++ protocols/gossipsub/src/lib.rs | 44 + protocols/gossipsub/src/mcache.rs | 311 +++ protocols/gossipsub/src/protocol.rs | 241 ++ protocols/gossipsub/src/rpc_proto.rs | 2828 ++++++++++++++++++++ 10 files changed, 4154 insertions(+) create mode 100644 protocols/gossipsub/Cargo.toml create mode 100755 protocols/gossipsub/regen_structs_proto.sh create mode 100644 protocols/gossipsub/rpc.proto create mode 100644 protocols/gossipsub/src/handler.rs create mode 100644 protocols/gossipsub/src/layer.rs create mode 100644 protocols/gossipsub/src/lib.rs create mode 100644 protocols/gossipsub/src/mcache.rs create mode 100644 protocols/gossipsub/src/protocol.rs create mode 100644 protocols/gossipsub/src/rpc_proto.rs diff --git a/Cargo.toml b/Cargo.toml index 9098bf03d14..78a973ce988 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -22,6 +22,7 @@ libp2p-mplex = { version = "0.3.0", path = "./muxers/mplex" } libp2p-identify = { version = "0.3.0", path = "./protocols/identify" } libp2p-kad = { version = "0.3.0", path = "./protocols/kad" } libp2p-floodsub = { version = "0.3.0", path = "./protocols/floodsub" } +libp2p-gossipsub = { version = "0.1.0", path = "./protocols/gossipsub" } libp2p-ping = { version = "0.3.0", path = "./protocols/ping" } libp2p-plaintext = { version = "0.3.0", path = "./protocols/plaintext" } libp2p-ratelimit = { version = "0.3.0", path = "./transports/ratelimit" } @@ -64,6 +65,7 @@ members = [ "muxers/mplex", "muxers/yamux", "protocols/floodsub", + "protocols/gossipsub", "protocols/identify", "protocols/kad", "protocols/noise", diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml new file mode 100644 index 00000000000..aeffd1bd267 --- /dev/null +++ b/protocols/gossipsub/Cargo.toml @@ -0,0 +1,25 @@ +[package] +name = "libp2p-gossipsub" +version = "0.1.0" +authors = ["Age Manning "] +license = "MIT" + +[dependencies] +libp2p-core = { path = "../../core" } +libp2p-floodsub = { path = "../floodsub" } +bs58 = "" +bytes = "" +cuckoofilter = "" +fnv = "" +futures = "" +protobuf = "" +rand = "" +smallvec = "" +tokio-codec = "" +tokio-io = "" +unsigned-varint = "" + +[dev-dependencies] +libp2p = { path = "../../" } +tokio = "0.1" +tokio-stdin-stdout = "0.1" diff --git a/protocols/gossipsub/regen_structs_proto.sh b/protocols/gossipsub/regen_structs_proto.sh new file mode 100755 index 00000000000..054ae5cdfec --- /dev/null +++ b/protocols/gossipsub/regen_structs_proto.sh @@ -0,0 +1,13 @@ +#!/bin/sh + +# This script regenerates the `src/rpc_proto.rs` file from `rpc.proto`. + +docker run --rm -v `pwd`:/usr/code:z -w /usr/code rust /bin/bash -c " \ + apt-get update; \ + apt-get install -y protobuf-compiler; \ + cargo install --version 2.0.2 protobuf-codegen; \ + protoc --rust_out . rpc.proto" + +sudo chown $USER:$USER *.rs + +mv -f rpc.rs ./src/rpc_proto.rs diff --git a/protocols/gossipsub/rpc.proto b/protocols/gossipsub/rpc.proto new file mode 100644 index 00000000000..65127cd688c --- /dev/null +++ b/protocols/gossipsub/rpc.proto @@ -0,0 +1,75 @@ +syntax = "proto2"; + +package floodsub.pb; + +message RPC { + repeated SubOpts subscriptions = 1; + repeated Message publish = 2; + + message SubOpts { + optional bool subscribe = 1; // subscribe or unsubscribe + optional string topicid = 2; + } + + optional ControlMessage control = 3; +} + +message Message { + optional bytes from = 1; + optional bytes data = 2; + optional bytes seqno = 3; + repeated string topicIDs = 4; +} + +message ControlMessage { + repeated ControlIHave ihave = 1; + repeated ControlIWant iwant = 2; + repeated ControlGraft graft = 3; + repeated ControlPrune prune = 4; +} + +message ControlIHave { + optional string topicID = 1; + repeated string messageIDs = 2; +} + +message ControlIWant { + repeated string messageIDs = 1; +} + +message ControlGraft { + optional string topicID = 1; +} + +message ControlPrune { + optional string topicID = 1; +} + +// topicID = hash(topicDescriptor); (not the topic.name) +message TopicDescriptor { + optional string name = 1; + optional AuthOpts auth = 2; + optional EncOpts enc = 3; + + message AuthOpts { + optional AuthMode mode = 1; + repeated bytes keys = 2; // root keys to trust + + enum AuthMode { + NONE = 0; // no authentication, anyone can publish + KEY = 1; // only messages signed by keys in the topic descriptor are accepted + WOT = 2; // web of trust, certificates can allow publisher set to grow + } + } + + message EncOpts { + optional EncMode mode = 1; + repeated bytes keyHashes = 2; // the hashes of the shared keys used (salted) + + enum EncMode { + NONE = 0; // no encryption, anyone can read + SHAREDKEY = 1; // messages are encrypted with shared key + WOT = 2; // web of trust, certificates can allow publisher set to grow + } + } +} diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs new file mode 100644 index 00000000000..dfdbec41628 --- /dev/null +++ b/protocols/gossipsub/src/handler.rs @@ -0,0 +1,255 @@ +// Copyright 2018 Parity Technologies (UK) 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::protocol::{GossipsubCodec, GossipsubConfig, GossipsubRpc}; +use futures::prelude::*; +use libp2p_core::{ + protocols_handler::ProtocolsHandlerUpgrErr, + upgrade::{InboundUpgrade, OutboundUpgrade}, + ProtocolsHandler, ProtocolsHandlerEvent, +}; +use smallvec::SmallVec; +use std::{fmt, io}; +use tokio_codec::Framed; +use tokio_io::{AsyncRead, AsyncWrite}; + +/// Protocol handler that handles communication with the remote for the gossipsub protocol. +/// +/// The handler will automatically open a substream with the remote for each request we make. +/// +/// It also handles requests made by the remote. +pub struct GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Configuration for the gossipsub protocol. + config: GossipsubConfig, + + /// If true, we are trying to shut down the existing gossipsub substream and should refuse any + /// incoming connection. + shutting_down: bool, + + /// The active substreams. + // TODO: add a limit to the number of allowed substreams + substreams: Vec>, + + /// Queue of values that we want to send to the remote. + send_queue: SmallVec<[GossipsubRpc; 16]>, +} + +/// State of an active substream, opened either by us or by the remote. +enum SubstreamState +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Waiting for a message from the remote. + WaitingInput(Framed), + /// Waiting to send a message to the remote. + PendingSend(Framed, GossipsubRpc), + /// Waiting to flush the substream so that the data arrives to the remote. + PendingFlush(Framed), + /// The substream is being closed. + Closing(Framed), +} + +impl SubstreamState +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Consumes this state and produces the substream. + fn into_substream(self) -> Framed { + match self { + SubstreamState::WaitingInput(substream) => substream, + SubstreamState::PendingSend(substream, _) => substream, + SubstreamState::PendingFlush(substream) => substream, + SubstreamState::Closing(substream) => substream, + } + } +} + +impl GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Builds a new `GossipsubHandler`. + pub fn new() -> Self { + GossipsubHandler { + config: GossipsubConfig::new(), + shutting_down: false, + substreams: Vec::new(), + send_queue: SmallVec::new(), + } + } +} + +impl ProtocolsHandler for GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + type InEvent = GossipsubRpc; + type OutEvent = GossipsubRpc; + type Error = io::Error; + type Substream = TSubstream; + type InboundProtocol = GossipsubConfig; + type OutboundProtocol = GossipsubConfig; + type OutboundOpenInfo = GossipsubRpc; + + #[inline] + fn listen_protocol(&self) -> Self::InboundProtocol { + self.config.clone() + } + + fn inject_fully_negotiated_inbound( + &mut self, + protocol: >::Output, + ) { + if self.shutting_down { + return (); + } + self.substreams.push(SubstreamState::WaitingInput(protocol)) + } + + fn inject_fully_negotiated_outbound( + &mut self, + protocol: >::Output, + message: Self::OutboundOpenInfo, + ) { + if self.shutting_down { + return (); + } + self.substreams + .push(SubstreamState::PendingSend(protocol, message)) + } + + #[inline] + fn inject_event(&mut self, message: GossipsubRpc) { + self.send_queue.push(message); + } + + #[inline] + fn inject_inbound_closed(&mut self) {} + + #[inline] + fn inject_dial_upgrade_error( + &mut self, + _: Self::OutboundOpenInfo, + _: ProtocolsHandlerUpgrErr< + >::Error, + >, + ) { + } + + #[inline] + fn connection_keep_alive(&self) -> bool { + !self.substreams.is_empty() + } + + #[inline] + fn shutdown(&mut self) { + self.shutting_down = true; + for n in (0..self.substreams.len()).rev() { + let mut substream = self.substreams.swap_remove(n); + self.substreams + .push(SubstreamState::Closing(substream.into_substream())); + } + } + + fn poll( + &mut self, + ) -> Poll< + ProtocolsHandlerEvent, + io::Error, + > { + if !self.send_queue.is_empty() { + let message = self.send_queue.remove(0); + return Ok(Async::Ready( + ProtocolsHandlerEvent::OutboundSubstreamRequest { + info: message, + upgrade: self.config.clone(), + }, + )); + } + + for n in (0..self.substreams.len()).rev() { + let mut substream = self.substreams.swap_remove(n); + loop { + substream = match substream { + SubstreamState::WaitingInput(mut substream) => match substream.poll() { + Ok(Async::Ready(Some(message))) => { + self.substreams + .push(SubstreamState::WaitingInput(substream)); + return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(message))); + } + Ok(Async::Ready(None)) => SubstreamState::Closing(substream), + Ok(Async::NotReady) => { + self.substreams + .push(SubstreamState::WaitingInput(substream)); + return Ok(Async::NotReady); + } + Err(_) => SubstreamState::Closing(substream), + }, + SubstreamState::PendingSend(mut substream, message) => { + match substream.start_send(message)? { + AsyncSink::Ready => SubstreamState::PendingFlush(substream), + AsyncSink::NotReady(message) => { + self.substreams + .push(SubstreamState::PendingSend(substream, message)); + return Ok(Async::NotReady); + } + } + } + SubstreamState::PendingFlush(mut substream) => { + match substream.poll_complete()? { + Async::Ready(()) => SubstreamState::Closing(substream), + Async::NotReady => { + self.substreams + .push(SubstreamState::PendingFlush(substream)); + return Ok(Async::NotReady); + } + } + } + SubstreamState::Closing(mut substream) => match substream.close() { + Ok(Async::Ready(())) => break, + Ok(Async::NotReady) => { + self.substreams.push(SubstreamState::Closing(substream)); + return Ok(Async::NotReady); + } + Err(_) => return Ok(Async::Ready(ProtocolsHandlerEvent::Shutdown)), + }, + } + } + } + + Ok(Async::NotReady) + } +} + +impl fmt::Debug for GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { + f.debug_struct("GossipsubHandler") + .field("shutting_down", &self.shutting_down) + .field("substreams", &self.substreams.len()) + .field("send_queue", &self.send_queue.len()) + .finish() + } +} diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs new file mode 100644 index 00000000000..f9ec78a8369 --- /dev/null +++ b/protocols/gossipsub/src/layer.rs @@ -0,0 +1,360 @@ +// Copyright 2018 Parity Technologies (UK) 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 cuckoofilter::CuckooFilter; +use futures::prelude::*; +use handler::GossipsubHandler; +use libp2p_core::swarm::{ + ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters, +}; +use libp2p_core::{protocols_handler::ProtocolsHandler, PeerId}; +use libp2p_floodsub::{Topic, TopicHash}; +use protocol::{ + GossipsubMessage, GossipsubRpc, GossipsubSubscription, GossipsubSubscriptionAction, +}; +use rand; +use smallvec::SmallVec; +use std::collections::hash_map::{DefaultHasher, HashMap}; +use std::{collections::VecDeque, iter, marker::PhantomData}; +use tokio_io::{AsyncRead, AsyncWrite}; + +/// Network behaviour that automatically identifies nodes periodically, and returns information +/// about them. +pub struct Gossipsub { + /// Events that need to be yielded to the outside when polling. + events: VecDeque>, + + /// Peer id of the local node. Used for the source of the messages that we publish. + local_peer_id: PeerId, + + /// List of peers the network is connected to, and the topics that they're subscribed to. + // TODO: filter out peers that don't support gossipsub so that we avoid hammering them with + // opened substream + connected_peers: HashMap>, + + // List of topics we're subscribed to. Necessary to filter out messages that we receive + // erroneously. + subscribed_topics: SmallVec<[Topic; 16]>, + + // We keep track of the messages we received (in the format `hash(source ID, seq_no)`) so that + // we don't dispatch the same message twice if we receive it twice on the network. + received: CuckooFilter, + + /// Marker to pin the generics. + marker: PhantomData, +} + +impl Gossipsub { + /// Creates a `Gossipsub`. + pub fn new(local_peer_id: PeerId) -> Self { + Gossipsub { + events: VecDeque::new(), + local_peer_id, + connected_peers: HashMap::new(), + subscribed_topics: SmallVec::new(), + received: CuckooFilter::new(), + marker: PhantomData, + } + } + + /// Subscribes to a topic. + /// + /// Returns true if the subscription worked. Returns false if we were already subscribed. + pub fn subscribe(&mut self, topic: Topic) -> bool { + if self + .subscribed_topics + .iter() + .any(|t| t.hash() == topic.hash()) + { + return false; + } + + for peer in self.connected_peers.keys() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic: topic.hash().clone(), + action: GossipsubSubscriptionAction::Subscribe, + }], + }, + }); + } + + self.subscribed_topics.push(topic); + true + } + + /// Unsubscribes from a topic. + /// + /// Note that this only requires a `TopicHash` and not a full `Topic`. + /// + /// Returns true if we were subscribed to this topic. + pub fn unsubscribe(&mut self, topic: impl AsRef) -> bool { + let topic = topic.as_ref(); + let pos = match self + .subscribed_topics + .iter() + .position(|t| t.hash() == topic) + { + Some(pos) => pos, + None => return false, + }; + + self.subscribed_topics.remove(pos); + + for peer in self.connected_peers.keys() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic: topic.clone(), + action: GossipsubSubscriptionAction::Unsubscribe, + }], + }, + }); + } + + true + } + + /// Publishes a message to the network. + /// + /// > **Note**: Doesn't do anything if we're not subscribed to the topic. + pub fn publish(&mut self, topic: impl Into, data: impl Into>) { + self.publish_many(iter::once(topic), data) + } + + /// Publishes a message with multiple topics to the network. + /// + /// > **Note**: Doesn't do anything if we're not subscribed to any of the topics. + pub fn publish_many( + &mut self, + topic: impl IntoIterator>, + data: impl Into>, + ) { + let message = GossipsubMessage { + source: self.local_peer_id.clone(), + data: data.into(), + // If the sequence numbers are predictable, then an attacker could flood the network + // with packets with the predetermined sequence numbers and absorb our legitimate + // messages. We therefore use a random number. + sequence_number: rand::random::<[u8; 20]>().to_vec(), + topics: topic.into_iter().map(|t| t.into().clone()).collect(), + }; + + // Don't publish the message if we're not subscribed ourselves to any of the topics. + if !self + .subscribed_topics + .iter() + .any(|t| message.topics.iter().any(|u| t.hash() == u)) + { + return; + } + + self.received.add(&message); + + // Send to peers we know are subscribed to the topic. + for (peer_id, sub_topic) in self.connected_peers.iter() { + if !sub_topic + .iter() + .any(|t| message.topics.iter().any(|u| t == u)) + { + continue; + } + + println!("peers subscribed? {:?}", peer_id); + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer_id.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![message.clone()], + }, + }); + } + } +} + +impl NetworkBehaviour for Gossipsub +where + TSubstream: AsyncRead + AsyncWrite, +{ + type ProtocolsHandler = GossipsubHandler; + type OutEvent = GossipsubEvent; + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + GossipsubHandler::new() + } + + fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { + // We need to send our subscriptions to the newly-connected node. + for topic in self.subscribed_topics.iter() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: id.clone(), + event: GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic: topic.hash().clone(), + action: GossipsubSubscriptionAction::Subscribe, + }], + }, + }); + } + + self.connected_peers.insert(id.clone(), SmallVec::new()); + } + + fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { + let was_in = self.connected_peers.remove(id); + debug_assert!(was_in.is_some()); + } + + fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { + // Update connected peers topics + for subscription in event.subscriptions { + let mut remote_peer_topics = self.connected_peers + .get_mut(&propagation_source) + .expect("connected_peers is kept in sync with the peers we are connected to; we are guaranteed to only receive events from connected peers; QED"); + match subscription.action { + GossipsubSubscriptionAction::Subscribe => { + if !remote_peer_topics.contains(&subscription.topic) { + remote_peer_topics.push(subscription.topic.clone()); + } + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Subscribed { + peer_id: propagation_source.clone(), + topic: subscription.topic, + }, + )); + } + GossipsubSubscriptionAction::Unsubscribe => { + if let Some(pos) = remote_peer_topics + .iter() + .position(|t| t == &subscription.topic) + { + remote_peer_topics.remove(pos); + } + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Unsubscribed { + peer_id: propagation_source.clone(), + topic: subscription.topic, + }, + )); + } + } + } + + // List of messages we're going to propagate on the network. + let mut rpcs_to_dispatch: Vec<(PeerId, GossipsubRpc)> = Vec::new(); + + for message in event.messages { + // Use `self.received` to skip the messages that we have already received in the past. + // Note that this can be a false positive. + if !self.received.test_and_add(&message) { + continue; + } + + // Add the message to be dispatched to the user. + if self + .subscribed_topics + .iter() + .any(|t| message.topics.iter().any(|u| t.hash() == u)) + { + let event = GossipsubEvent::Message(message.clone()); + self.events + .push_back(NetworkBehaviourAction::GenerateEvent(event)); + } + + // Propagate the message to everyone else who is subscribed to any of the topics. + for (peer_id, subscr_topics) in self.connected_peers.iter() { + if peer_id == &propagation_source { + continue; + } + + if !subscr_topics + .iter() + .any(|t| message.topics.iter().any(|u| t == u)) + { + continue; + } + + if let Some(pos) = rpcs_to_dispatch.iter().position(|(p, _)| p == peer_id) { + rpcs_to_dispatch[pos].1.messages.push(message.clone()); + } else { + rpcs_to_dispatch.push(( + peer_id.clone(), + GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![message.clone()], + }, + )); + } + } + } + + for (peer_id, rpc) in rpcs_to_dispatch { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id, + event: rpc, + }); + } + } + + fn poll( + &mut self, + _: &mut PollParameters, + ) -> Async< + NetworkBehaviourAction< + ::InEvent, + Self::OutEvent, + >, + > { + if let Some(event) = self.events.pop_front() { + return Async::Ready(event); + } + + Async::NotReady + } +} + +/// Event that can happen on the gossipsub behaviour. +#[derive(Debug)] +pub enum GossipsubEvent { + /// A message has been received. + Message(GossipsubMessage), + + /// A remote subscribed to a topic. + Subscribed { + /// Remote that has subscribed. + peer_id: PeerId, + /// The topic it has subscribed to. + topic: TopicHash, + }, + + /// A remote unsubscribed from a topic. + Unsubscribed { + /// Remote that has unsubscribed. + peer_id: PeerId, + /// The topic it has subscribed from. + topic: TopicHash, + }, +} diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs new file mode 100644 index 00000000000..c21de840abf --- /dev/null +++ b/protocols/gossipsub/src/lib.rs @@ -0,0 +1,44 @@ +// Copyright 2018 Parity Technologies (UK) 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. + +extern crate bs58; +extern crate bytes; +extern crate cuckoofilter; +extern crate fnv; +extern crate futures; +extern crate libp2p_core; +extern crate libp2p_floodsub; +extern crate protobuf; +extern crate rand; +extern crate smallvec; +extern crate tokio_codec; +extern crate tokio_io; +extern crate unsigned_varint; + +pub mod handler; +pub mod protocol; + +mod layer; +mod mcache; +mod rpc_proto; + +pub use self::handler::GossipsubHandler; +pub use self::layer::Gossipsub; +pub use self::protocol::*; diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs new file mode 100644 index 00000000000..5586ac3fd30 --- /dev/null +++ b/protocols/gossipsub/src/mcache.rs @@ -0,0 +1,311 @@ +extern crate fnv; + +use super::rpc_proto::{Message}; +use fnv::{FnvHashMap}; + +/// CacheEntry stored in the history +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct CacheEntry { + mid: String, + topics: Vec, +} + +/// MessageCache struct holding history of messages +#[derive(Debug, Clone, PartialEq)] +pub struct MessageCache { + msgs: FnvHashMap, + history: Vec>, + gossip : usize, +} + +/// Implementation of the MessageCache +impl MessageCache { + pub fn new(gossip: usize, history_capacity: usize) -> MessageCache { + MessageCache{ + gossip, + msgs: FnvHashMap::default(), + history: vec![Vec::new(); history_capacity], + } + } + + /// Put a message into the memory cache + pub fn put(&mut self, msg: Message) -> Result<(), MsgError> { + let message_id = msg_id(&msg)?; + let cache_entry = CacheEntry { + mid: message_id.clone(), + topics: msg.get_topicIDs().to_vec() + }; + + self.msgs.insert(message_id, msg); + + self.history[0].push(cache_entry); + Ok(()) + } + + /// Get a message with `message_id` + pub fn get(&self, message_id: &str) -> Option<&Message> { + self.msgs.get(message_id) + } + + /// Get a list of GossipIds for a given topic + pub fn get_gossip_ids(&self, topic: &str) -> Vec { + + self.history[..self.gossip] + .iter().fold(vec![], |mut current_entries, entries| { + // search for entries with desired topic + let mut found_entries: Vec = + entries.iter().filter_map(|entry| { + if entry.topics.iter().any(|t| *t == topic) { + Some(entry.mid.clone()) + } + else { + None + } + }).collect(); + + // generate the list + current_entries.append(&mut found_entries); + current_entries + }) + } + + /// Shift the history array down one and delete messages associated with the + /// last entry + pub fn shift(&mut self) { + let last_index = self.history.len()-1; + for entry in &self.history[last_index] { + self.msgs.remove(&entry.mid); + } + + // Pop the last value + self.history.pop(); + + // Insert an empty vec in position 0 + self.history.insert(0, Vec::new()); + + // TODO bench which one is quicker + // for i in (0..(self.history.len() - 1)).rev() { + // self.history[i+1] = self.history[i].clone(); + // } + // self.history[0] = Vec::new(); + } + +} + + +// Functions to be refactored later +/// Gets a unique message id. +/// Returns an error if the message has non-utf from or seqno values +fn msg_id(pmsg: &Message) -> Result { + let from = + String::from_utf8(pmsg.get_from().to_vec()).or(Err(MsgError::InvalidMessage))?; + let seqno = + String::from_utf8(pmsg.get_seqno().to_vec()).or(Err(MsgError::InvalidMessage))?; + Ok(from + &seqno) +} + +#[derive(Debug)] +pub enum MsgError { + InvalidMessage, +} + + +#[cfg(test)] +mod tests { + use super::*; + use super::super::protobuf; + + fn gen_testm(x: usize, topics: Vec) -> Message { + let u8x: u8 = x as u8; + let from: Vec = vec![u8x]; + let data: Vec = vec![u8x]; + let seqno: Vec = vec![u8x]; + let mut tids = protobuf::RepeatedField::new(); + + for topic in topics { + tids.push(topic); + } + + let mut m = Message::new(); + m.set_from(from.clone()); + m.set_data(data); + m.set_seqno(seqno); + m.set_topicIDs(tids); + + m + } + + #[test] + /// Test that the message cache can be created + fn test_new_cache() { + let x: usize = 3; + let mc = MessageCache::new(x, 5); + + assert_eq!(mc.gossip, x); + } + + #[test] + /// Test you can put one message and get one + fn test_put_get_one() { + let mut mc = MessageCache::new(10, 15); + + let m = gen_testm(10 as usize, vec![String::from("hello"), String::from("world")]); + + let res = mc.put(m.clone()); + assert_eq!(res.is_err(), false); + assert_eq!(res.ok(), Some(())); + + assert!(mc.history[0].len() == 1); + + let mid = msg_id(&m.clone()); + assert_eq!(mid.is_err(), false); + + let fetched = match mid.ok() { + Some(id) => mc.get(&id), + _ => None + }; + + assert_eq!(fetched.is_none(), false); + assert_eq!(fetched.is_some(), true); + + // Make sure it is the same fetched message + match fetched { + Some(x) => assert_eq!(*x, m), + _ => assert!(false) + } + } + + #[test] + /// Test attempting to 'get' with a wrong id + fn test_get_wrong() { + let mut mc = MessageCache::new(10, 15); + + // Build the message + let m = gen_testm(1 as usize, vec![String::from("hello"), String::from("world")]); + + let res = mc.put(m.clone()); + assert_eq!(res.is_err(), false); + assert_eq!(res.ok(), Some(())); + + let mid = msg_id(&m.clone()); + assert_eq!(mid.is_err(), false); + + // Try to get an incorrect ID + let wrong_string = String::from("wrongid"); + let fetched = mc.get(&wrong_string); + assert_eq!(fetched.is_none(), true); + } + + + #[test] + /// Test attempting to 'get' empty message cache + fn test_get_empty() { + let mc = MessageCache::new(10, 15); + + // Try to get an incorrect ID + let wrong_string = String::from("imempty"); + let fetched = mc.get(&wrong_string); + assert_eq!(fetched.is_none(), true); + } + + #[test] + /// Test adding a message with no topics + fn test_no_topic_put() { + let mut mc = MessageCache::new(3, 5); + + // Build the message + let m = gen_testm(1 as usize, vec![]); + + let res = mc.put(m.clone()); + assert_eq!(res.is_err(), false); + assert_eq!(res.ok(), Some(())); + + let mid = msg_id(&m.clone()); + let fetched = match mid.ok() { + Some(id) => mc.get(&id), + _ => None + }; + + // 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 = MessageCache::new(1, 5); + + // Build the message + for i in 0..10 { + let m = gen_testm(i as usize, vec![String::from("hello"), String::from("world")]); + let res = mc.put(m.clone()); + assert_eq!(res.is_err(), false); + } + + mc.shift(); + + // Ensure the shift occurred + assert!(mc.history[0].len() == 0); + assert!(mc.history[1].len() == 10); + + // Make sure no messages deleted + assert!(mc.msgs.len() == 10); + } + + #[test] + /// Test Shift with no additions + fn test_empty_shift() { + let mut mc = MessageCache::new(1, 5); + + // Build the message + for i in 0..10 { + let m = gen_testm(i as usize, vec![String::from("hello"), String::from("world")]); + let res = mc.put(m.clone()); + assert_eq!(res.is_err(), false); + } + + mc.shift(); + + // Ensure the shift occurred + assert!(mc.history[0].len() == 0); + assert!(mc.history[1].len() == 10); + + mc.shift(); + + assert!(mc.history[2].len() == 10); + assert!(mc.history[1].len() == 0); + assert!(mc.history[0].len() == 0); + } + + #[test] + /// Test shift to see if the last history messages are removed + fn test_remove_last_from_shift() { + + let mut mc = MessageCache::new(4, 5); + + for i in 0..10 { + let m = gen_testm(i as usize, vec![String::from("hello"), String::from("world")]); + let res = mc.put(m.clone()); + assert_eq!(res.is_err(), false); + } + + // Shift right until deleting messages + mc.shift(); + mc.shift(); + mc.shift(); + mc.shift(); + + assert_eq!(mc.history[mc.history.len() - 1].len(), 10); + + // Shift and delete the messages + mc.shift(); + assert_eq!(mc.history[mc.history.len() - 1].len(), 0); + assert_eq!(mc.history[0].len(), 0); + assert_eq!(mc.msgs.len(), 0); + } +} + + diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs new file mode 100644 index 00000000000..8f859a95bf4 --- /dev/null +++ b/protocols/gossipsub/src/protocol.rs @@ -0,0 +1,241 @@ +// Copyright 2018 Parity Technologies (UK) 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::rpc_proto; +use bytes::{BufMut, BytesMut}; +use futures::future; +use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; +use libp2p_floodsub::TopicHash; +use protobuf::Message as ProtobufMessage; +use std::{io, iter}; +use tokio_codec::{Decoder, Encoder, Framed}; +use tokio_io::{AsyncRead, AsyncWrite}; +use unsigned_varint::codec; + +/// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. +#[derive(Debug, Clone)] +pub struct GossipsubConfig {} + +impl GossipsubConfig { + /// Builds a new `GossipsubConfig`. + #[inline] + pub fn new() -> GossipsubConfig { + GossipsubConfig {} + } +} + +impl UpgradeInfo for GossipsubConfig { + type Info = &'static [u8]; + type InfoIter = iter::Once; + + #[inline] + fn protocol_info(&self) -> Self::InfoIter { + iter::once(b"/meshsub/1.0.0") + } +} + +impl InboundUpgrade for GossipsubConfig +where + TSocket: AsyncRead + AsyncWrite, +{ + type Output = Framed; + type Error = io::Error; + type Future = future::FutureResult; + + #[inline] + fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future { + future::ok(Framed::new( + socket, + GossipsubCodec { + length_prefix: Default::default(), + }, + )) + } +} + +impl OutboundUpgrade for GossipsubConfig +where + TSocket: AsyncRead + AsyncWrite, +{ + type Output = Framed; + type Error = io::Error; + type Future = future::FutureResult; + + #[inline] + fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future { + future::ok(Framed::new( + socket, + GossipsubCodec { + length_prefix: Default::default(), + }, + )) + } +} + +/// Implementation of `tokio_codec::Codec`. +pub struct GossipsubCodec { + /// The codec for encoding/decoding the length prefix of messages. + length_prefix: codec::UviBytes, +} + +impl Encoder for GossipsubCodec { + type Item = GossipsubRpc; + type Error = io::Error; + + fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { + let mut proto = rpc_proto::RPC::new(); + + for message in item.messages.into_iter() { + let mut msg = rpc_proto::Message::new(); + msg.set_from(message.source.into_bytes()); + msg.set_data(message.data); + msg.set_seqno(message.sequence_number); + msg.set_topicIDs( + message + .topics + .into_iter() + .map(TopicHash::into_string) + .collect(), + ); + proto.mut_publish().push(msg); + } + + for topic in item.subscriptions.into_iter() { + let mut subscription = rpc_proto::RPC_SubOpts::new(); + subscription.set_subscribe(topic.action == GossipsubSubscriptionAction::Subscribe); + subscription.set_topicid(topic.topic.into_string()); + proto.mut_subscriptions().push(subscription); + } + + let msg_size = proto.compute_size(); + // Reserve enough space for the data and the length. The length has a maximum of 32 bits, + // which means that 5 bytes is enough for the variable-length integer. + dst.reserve(msg_size as usize + 5); + + proto + .write_length_delimited_to_writer(&mut dst.by_ref().writer()) + .expect( + "there is no situation in which the protobuf message can be invalid, and \ + writing to a BytesMut never fails as we reserved enough space beforehand", + ); + Ok(()) + } +} + +impl Decoder for GossipsubCodec { + type Item = GossipsubRpc; + type Error = io::Error; + + fn decode(&mut self, src: &mut BytesMut) -> Result, Self::Error> { + let packet = match self.length_prefix.decode(src)? { + Some(p) => p, + None => return Ok(None), + }; + + let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; + + let mut messages = Vec::with_capacity(rpc.get_publish().len()); + for mut publish in rpc.take_publish().into_iter() { + messages.push(GossipsubMessage { + source: PeerId::from_bytes(publish.take_from()).map_err(|_| { + io::Error::new(io::ErrorKind::InvalidData, "Invalid peer ID in message") + })?, + data: publish.take_data(), + sequence_number: publish.take_seqno(), + topics: publish + .take_topicIDs() + .into_iter() + .map(|topic| TopicHash::from_raw(topic)) + .collect(), + }); + } + + Ok(Some(GossipsubRpc { + messages, + subscriptions: rpc + .take_subscriptions() + .into_iter() + .map(|mut sub| GossipsubSubscription { + action: if sub.get_subscribe() { + GossipsubSubscriptionAction::Subscribe + } else { + GossipsubSubscriptionAction::Unsubscribe + }, + topic: TopicHash::from_raw(sub.take_topicid()), + }) + .collect(), + })) + } +} + +/// An RPC received by the gossipsub system. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct GossipsubRpc { + /// List of messages that were part of this RPC query. + pub messages: Vec, + /// List of subscriptions. + pub subscriptions: Vec, +} + +/// A message received by the gossipsub system. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct GossipsubMessage { + /// Id of the peer that published this message. + pub source: PeerId, + + /// Content of the message. Its meaning is out of scope of this library. + pub data: Vec, + + /// An incrementing sequence number. + pub sequence_number: Vec, + + /// List of topics this message belongs to. + /// + /// Each message can belong to multiple topics at once. + pub topics: Vec, +} + +/// 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: 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, + /* + /// The remote has data + iHave, + /// iWant control action + iWant, + /// Graft control action + graft, + /// Prune control action + prune, + */ +} diff --git a/protocols/gossipsub/src/rpc_proto.rs b/protocols/gossipsub/src/rpc_proto.rs new file mode 100644 index 00000000000..4aff674e1cf --- /dev/null +++ b/protocols/gossipsub/src/rpc_proto.rs @@ -0,0 +1,2828 @@ +// This file is generated by rust-protobuf 2.0.2. Do not edit +// @generated + +// https://github.com/Manishearth/rust-clippy/issues/702 +#![allow(unknown_lints)] +#![allow(clippy)] + +#![cfg_attr(rustfmt, rustfmt_skip)] + +#![allow(box_pointers)] +#![allow(dead_code)] +#![allow(missing_docs)] +#![allow(non_camel_case_types)] +#![allow(non_snake_case)] +#![allow(non_upper_case_globals)] +#![allow(trivial_casts)] +#![allow(unsafe_code)] +#![allow(unused_imports)] +#![allow(unused_results)] + +use protobuf::Message as Message_imported_for_functions; +use protobuf::ProtobufEnum as ProtobufEnum_imported_for_functions; + +#[derive(PartialEq,Clone,Default)] +pub struct RPC { + // message fields + subscriptions: ::protobuf::RepeatedField, + publish: ::protobuf::RepeatedField, + control: ::protobuf::SingularPtrField, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl RPC { + pub fn new() -> RPC { + ::std::default::Default::default() + } + + // repeated .floodsub.pb.RPC.SubOpts subscriptions = 1; + + pub fn clear_subscriptions(&mut self) { + self.subscriptions.clear(); + } + + // Param is passed by value, moved + pub fn set_subscriptions(&mut self, v: ::protobuf::RepeatedField) { + self.subscriptions = v; + } + + // Mutable pointer to the field. + pub fn mut_subscriptions(&mut self) -> &mut ::protobuf::RepeatedField { + &mut self.subscriptions + } + + // Take field + pub fn take_subscriptions(&mut self) -> ::protobuf::RepeatedField { + ::std::mem::replace(&mut self.subscriptions, ::protobuf::RepeatedField::new()) + } + + pub fn get_subscriptions(&self) -> &[RPC_SubOpts] { + &self.subscriptions + } + + // repeated .floodsub.pb.Message publish = 2; + + pub fn clear_publish(&mut self) { + self.publish.clear(); + } + + // Param is passed by value, moved + pub fn set_publish(&mut self, v: ::protobuf::RepeatedField) { + self.publish = v; + } + + // Mutable pointer to the field. + pub fn mut_publish(&mut self) -> &mut ::protobuf::RepeatedField { + &mut self.publish + } + + // Take field + pub fn take_publish(&mut self) -> ::protobuf::RepeatedField { + ::std::mem::replace(&mut self.publish, ::protobuf::RepeatedField::new()) + } + + pub fn get_publish(&self) -> &[Message] { + &self.publish + } + + // optional .floodsub.pb.ControlMessage control = 3; + + pub fn clear_control(&mut self) { + self.control.clear(); + } + + pub fn has_control(&self) -> bool { + self.control.is_some() + } + + // Param is passed by value, moved + pub fn set_control(&mut self, v: ControlMessage) { + self.control = ::protobuf::SingularPtrField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_control(&mut self) -> &mut ControlMessage { + if self.control.is_none() { + self.control.set_default(); + } + self.control.as_mut().unwrap() + } + + // Take field + pub fn take_control(&mut self) -> ControlMessage { + self.control.take().unwrap_or_else(|| ControlMessage::new()) + } + + pub fn get_control(&self) -> &ControlMessage { + self.control.as_ref().unwrap_or_else(|| ControlMessage::default_instance()) + } +} + +impl ::protobuf::Message for RPC { + fn is_initialized(&self) -> bool { + for v in &self.subscriptions { + if !v.is_initialized() { + return false; + } + }; + for v in &self.publish { + if !v.is_initialized() { + return false; + } + }; + for v in &self.control { + if !v.is_initialized() { + return false; + } + }; + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.subscriptions)?; + }, + 2 => { + ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.publish)?; + }, + 3 => { + ::protobuf::rt::read_singular_message_into(wire_type, is, &mut self.control)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + for value in &self.subscriptions { + let len = value.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + }; + for value in &self.publish { + let len = value.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + }; + if let Some(ref v) = self.control.as_ref() { + let len = v.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + } + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + for v in &self.subscriptions { + os.write_tag(1, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + }; + for v in &self.publish { + os.write_tag(2, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + }; + if let Some(ref v) = self.control.as_ref() { + os.write_tag(3, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + } + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> RPC { + RPC::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "subscriptions", + |m: &RPC| { &m.subscriptions }, + |m: &mut RPC| { &mut m.subscriptions }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "publish", + |m: &RPC| { &m.publish }, + |m: &mut RPC| { &mut m.publish }, + )); + fields.push(::protobuf::reflect::accessor::make_singular_ptr_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "control", + |m: &RPC| { &m.control }, + |m: &mut RPC| { &mut m.control }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "RPC", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static RPC { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const RPC, + }; + unsafe { + instance.get(RPC::new) + } + } +} + +impl ::protobuf::Clear for RPC { + fn clear(&mut self) { + self.clear_subscriptions(); + self.clear_publish(); + self.clear_control(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for RPC { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for RPC { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct RPC_SubOpts { + // message fields + subscribe: ::std::option::Option, + topicid: ::protobuf::SingularField<::std::string::String>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl RPC_SubOpts { + pub fn new() -> RPC_SubOpts { + ::std::default::Default::default() + } + + // optional bool subscribe = 1; + + pub fn clear_subscribe(&mut self) { + self.subscribe = ::std::option::Option::None; + } + + pub fn has_subscribe(&self) -> bool { + self.subscribe.is_some() + } + + // Param is passed by value, moved + pub fn set_subscribe(&mut self, v: bool) { + self.subscribe = ::std::option::Option::Some(v); + } + + pub fn get_subscribe(&self) -> bool { + self.subscribe.unwrap_or(false) + } + + // optional string topicid = 2; + + pub fn clear_topicid(&mut self) { + self.topicid.clear(); + } + + pub fn has_topicid(&self) -> bool { + self.topicid.is_some() + } + + // Param is passed by value, moved + pub fn set_topicid(&mut self, v: ::std::string::String) { + self.topicid = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_topicid(&mut self) -> &mut ::std::string::String { + if self.topicid.is_none() { + self.topicid.set_default(); + } + self.topicid.as_mut().unwrap() + } + + // Take field + pub fn take_topicid(&mut self) -> ::std::string::String { + self.topicid.take().unwrap_or_else(|| ::std::string::String::new()) + } + + pub fn get_topicid(&self) -> &str { + match self.topicid.as_ref() { + Some(v) => &v, + None => "", + } + } +} + +impl ::protobuf::Message for RPC_SubOpts { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + if wire_type != ::protobuf::wire_format::WireTypeVarint { + return ::std::result::Result::Err(::protobuf::rt::unexpected_wire_type(wire_type)); + } + let tmp = is.read_bool()?; + self.subscribe = ::std::option::Option::Some(tmp); + }, + 2 => { + ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicid)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(v) = self.subscribe { + my_size += 2; + } + if let Some(ref v) = self.topicid.as_ref() { + my_size += ::protobuf::rt::string_size(2, &v); + } + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(v) = self.subscribe { + os.write_bool(1, v)?; + } + if let Some(ref v) = self.topicid.as_ref() { + os.write_string(2, &v)?; + } + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> RPC_SubOpts { + RPC_SubOpts::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeBool>( + "subscribe", + |m: &RPC_SubOpts| { &m.subscribe }, + |m: &mut RPC_SubOpts| { &mut m.subscribe }, + )); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "topicid", + |m: &RPC_SubOpts| { &m.topicid }, + |m: &mut RPC_SubOpts| { &mut m.topicid }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "RPC_SubOpts", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static RPC_SubOpts { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const RPC_SubOpts, + }; + unsafe { + instance.get(RPC_SubOpts::new) + } + } +} + +impl ::protobuf::Clear for RPC_SubOpts { + fn clear(&mut self) { + self.clear_subscribe(); + self.clear_topicid(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for RPC_SubOpts { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for RPC_SubOpts { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct Message { + // message fields + from: ::protobuf::SingularField<::std::vec::Vec>, + data: ::protobuf::SingularField<::std::vec::Vec>, + seqno: ::protobuf::SingularField<::std::vec::Vec>, + topicIDs: ::protobuf::RepeatedField<::std::string::String>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl Message { + pub fn new() -> Message { + ::std::default::Default::default() + } + + // optional bytes from = 1; + + pub fn clear_from(&mut self) { + self.from.clear(); + } + + pub fn has_from(&self) -> bool { + self.from.is_some() + } + + // Param is passed by value, moved + pub fn set_from(&mut self, v: ::std::vec::Vec) { + self.from = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_from(&mut self) -> &mut ::std::vec::Vec { + if self.from.is_none() { + self.from.set_default(); + } + self.from.as_mut().unwrap() + } + + // Take field + pub fn take_from(&mut self) -> ::std::vec::Vec { + self.from.take().unwrap_or_else(|| ::std::vec::Vec::new()) + } + + pub fn get_from(&self) -> &[u8] { + match self.from.as_ref() { + Some(v) => &v, + None => &[], + } + } + + // optional bytes data = 2; + + pub fn clear_data(&mut self) { + self.data.clear(); + } + + pub fn has_data(&self) -> bool { + self.data.is_some() + } + + // Param is passed by value, moved + pub fn set_data(&mut self, v: ::std::vec::Vec) { + self.data = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_data(&mut self) -> &mut ::std::vec::Vec { + if self.data.is_none() { + self.data.set_default(); + } + self.data.as_mut().unwrap() + } + + // Take field + pub fn take_data(&mut self) -> ::std::vec::Vec { + self.data.take().unwrap_or_else(|| ::std::vec::Vec::new()) + } + + pub fn get_data(&self) -> &[u8] { + match self.data.as_ref() { + Some(v) => &v, + None => &[], + } + } + + // optional bytes seqno = 3; + + pub fn clear_seqno(&mut self) { + self.seqno.clear(); + } + + pub fn has_seqno(&self) -> bool { + self.seqno.is_some() + } + + // Param is passed by value, moved + pub fn set_seqno(&mut self, v: ::std::vec::Vec) { + self.seqno = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_seqno(&mut self) -> &mut ::std::vec::Vec { + if self.seqno.is_none() { + self.seqno.set_default(); + } + self.seqno.as_mut().unwrap() + } + + // Take field + pub fn take_seqno(&mut self) -> ::std::vec::Vec { + self.seqno.take().unwrap_or_else(|| ::std::vec::Vec::new()) + } + + pub fn get_seqno(&self) -> &[u8] { + match self.seqno.as_ref() { + Some(v) => &v, + None => &[], + } + } + + // repeated string topicIDs = 4; + + pub fn clear_topicIDs(&mut self) { + self.topicIDs.clear(); + } + + // Param is passed by value, moved + pub fn set_topicIDs(&mut self, v: ::protobuf::RepeatedField<::std::string::String>) { + self.topicIDs = v; + } + + // Mutable pointer to the field. + pub fn mut_topicIDs(&mut self) -> &mut ::protobuf::RepeatedField<::std::string::String> { + &mut self.topicIDs + } + + // Take field + pub fn take_topicIDs(&mut self) -> ::protobuf::RepeatedField<::std::string::String> { + ::std::mem::replace(&mut self.topicIDs, ::protobuf::RepeatedField::new()) + } + + pub fn get_topicIDs(&self) -> &[::std::string::String] { + &self.topicIDs + } +} + +impl ::protobuf::Message for Message { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.from)?; + }, + 2 => { + ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.data)?; + }, + 3 => { + ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.seqno)?; + }, + 4 => { + ::protobuf::rt::read_repeated_string_into(wire_type, is, &mut self.topicIDs)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(ref v) = self.from.as_ref() { + my_size += ::protobuf::rt::bytes_size(1, &v); + } + if let Some(ref v) = self.data.as_ref() { + my_size += ::protobuf::rt::bytes_size(2, &v); + } + if let Some(ref v) = self.seqno.as_ref() { + my_size += ::protobuf::rt::bytes_size(3, &v); + } + for value in &self.topicIDs { + my_size += ::protobuf::rt::string_size(4, &value); + }; + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(ref v) = self.from.as_ref() { + os.write_bytes(1, &v)?; + } + if let Some(ref v) = self.data.as_ref() { + os.write_bytes(2, &v)?; + } + if let Some(ref v) = self.seqno.as_ref() { + os.write_bytes(3, &v)?; + } + for v in &self.topicIDs { + os.write_string(4, &v)?; + }; + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> Message { + Message::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( + "from", + |m: &Message| { &m.from }, + |m: &mut Message| { &mut m.from }, + )); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( + "data", + |m: &Message| { &m.data }, + |m: &mut Message| { &mut m.data }, + )); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( + "seqno", + |m: &Message| { &m.seqno }, + |m: &mut Message| { &mut m.seqno }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "topicIDs", + |m: &Message| { &m.topicIDs }, + |m: &mut Message| { &mut m.topicIDs }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "Message", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static Message { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const Message, + }; + unsafe { + instance.get(Message::new) + } + } +} + +impl ::protobuf::Clear for Message { + fn clear(&mut self) { + self.clear_from(); + self.clear_data(); + self.clear_seqno(); + self.clear_topicIDs(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for Message { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for Message { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct ControlMessage { + // message fields + ihave: ::protobuf::RepeatedField, + iwant: ::protobuf::RepeatedField, + graft: ::protobuf::RepeatedField, + prune: ::protobuf::RepeatedField, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl ControlMessage { + pub fn new() -> ControlMessage { + ::std::default::Default::default() + } + + // repeated .floodsub.pb.ControlIHave ihave = 1; + + pub fn clear_ihave(&mut self) { + self.ihave.clear(); + } + + // Param is passed by value, moved + pub fn set_ihave(&mut self, v: ::protobuf::RepeatedField) { + self.ihave = v; + } + + // Mutable pointer to the field. + pub fn mut_ihave(&mut self) -> &mut ::protobuf::RepeatedField { + &mut self.ihave + } + + // Take field + pub fn take_ihave(&mut self) -> ::protobuf::RepeatedField { + ::std::mem::replace(&mut self.ihave, ::protobuf::RepeatedField::new()) + } + + pub fn get_ihave(&self) -> &[ControlIHave] { + &self.ihave + } + + // repeated .floodsub.pb.ControlIWant iwant = 2; + + pub fn clear_iwant(&mut self) { + self.iwant.clear(); + } + + // Param is passed by value, moved + pub fn set_iwant(&mut self, v: ::protobuf::RepeatedField) { + self.iwant = v; + } + + // Mutable pointer to the field. + pub fn mut_iwant(&mut self) -> &mut ::protobuf::RepeatedField { + &mut self.iwant + } + + // Take field + pub fn take_iwant(&mut self) -> ::protobuf::RepeatedField { + ::std::mem::replace(&mut self.iwant, ::protobuf::RepeatedField::new()) + } + + pub fn get_iwant(&self) -> &[ControlIWant] { + &self.iwant + } + + // repeated .floodsub.pb.ControlGraft graft = 3; + + pub fn clear_graft(&mut self) { + self.graft.clear(); + } + + // Param is passed by value, moved + pub fn set_graft(&mut self, v: ::protobuf::RepeatedField) { + self.graft = v; + } + + // Mutable pointer to the field. + pub fn mut_graft(&mut self) -> &mut ::protobuf::RepeatedField { + &mut self.graft + } + + // Take field + pub fn take_graft(&mut self) -> ::protobuf::RepeatedField { + ::std::mem::replace(&mut self.graft, ::protobuf::RepeatedField::new()) + } + + pub fn get_graft(&self) -> &[ControlGraft] { + &self.graft + } + + // repeated .floodsub.pb.ControlPrune prune = 4; + + pub fn clear_prune(&mut self) { + self.prune.clear(); + } + + // Param is passed by value, moved + pub fn set_prune(&mut self, v: ::protobuf::RepeatedField) { + self.prune = v; + } + + // Mutable pointer to the field. + pub fn mut_prune(&mut self) -> &mut ::protobuf::RepeatedField { + &mut self.prune + } + + // Take field + pub fn take_prune(&mut self) -> ::protobuf::RepeatedField { + ::std::mem::replace(&mut self.prune, ::protobuf::RepeatedField::new()) + } + + pub fn get_prune(&self) -> &[ControlPrune] { + &self.prune + } +} + +impl ::protobuf::Message for ControlMessage { + fn is_initialized(&self) -> bool { + for v in &self.ihave { + if !v.is_initialized() { + return false; + } + }; + for v in &self.iwant { + if !v.is_initialized() { + return false; + } + }; + for v in &self.graft { + if !v.is_initialized() { + return false; + } + }; + for v in &self.prune { + if !v.is_initialized() { + return false; + } + }; + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.ihave)?; + }, + 2 => { + ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.iwant)?; + }, + 3 => { + ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.graft)?; + }, + 4 => { + ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.prune)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + for value in &self.ihave { + let len = value.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + }; + for value in &self.iwant { + let len = value.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + }; + for value in &self.graft { + let len = value.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + }; + for value in &self.prune { + let len = value.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + }; + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + for v in &self.ihave { + os.write_tag(1, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + }; + for v in &self.iwant { + os.write_tag(2, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + }; + for v in &self.graft { + os.write_tag(3, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + }; + for v in &self.prune { + os.write_tag(4, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + }; + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> ControlMessage { + ControlMessage::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "ihave", + |m: &ControlMessage| { &m.ihave }, + |m: &mut ControlMessage| { &mut m.ihave }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "iwant", + |m: &ControlMessage| { &m.iwant }, + |m: &mut ControlMessage| { &mut m.iwant }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "graft", + |m: &ControlMessage| { &m.graft }, + |m: &mut ControlMessage| { &mut m.graft }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "prune", + |m: &ControlMessage| { &m.prune }, + |m: &mut ControlMessage| { &mut m.prune }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "ControlMessage", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static ControlMessage { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ControlMessage, + }; + unsafe { + instance.get(ControlMessage::new) + } + } +} + +impl ::protobuf::Clear for ControlMessage { + fn clear(&mut self) { + self.clear_ihave(); + self.clear_iwant(); + self.clear_graft(); + self.clear_prune(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for ControlMessage { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for ControlMessage { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct ControlIHave { + // message fields + topicID: ::protobuf::SingularField<::std::string::String>, + messageIDs: ::protobuf::RepeatedField<::std::string::String>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl ControlIHave { + pub fn new() -> ControlIHave { + ::std::default::Default::default() + } + + // optional string topicID = 1; + + pub fn clear_topicID(&mut self) { + self.topicID.clear(); + } + + pub fn has_topicID(&self) -> bool { + self.topicID.is_some() + } + + // Param is passed by value, moved + pub fn set_topicID(&mut self, v: ::std::string::String) { + self.topicID = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_topicID(&mut self) -> &mut ::std::string::String { + if self.topicID.is_none() { + self.topicID.set_default(); + } + self.topicID.as_mut().unwrap() + } + + // Take field + pub fn take_topicID(&mut self) -> ::std::string::String { + self.topicID.take().unwrap_or_else(|| ::std::string::String::new()) + } + + pub fn get_topicID(&self) -> &str { + match self.topicID.as_ref() { + Some(v) => &v, + None => "", + } + } + + // repeated string messageIDs = 2; + + pub fn clear_messageIDs(&mut self) { + self.messageIDs.clear(); + } + + // Param is passed by value, moved + pub fn set_messageIDs(&mut self, v: ::protobuf::RepeatedField<::std::string::String>) { + self.messageIDs = v; + } + + // Mutable pointer to the field. + pub fn mut_messageIDs(&mut self) -> &mut ::protobuf::RepeatedField<::std::string::String> { + &mut self.messageIDs + } + + // Take field + pub fn take_messageIDs(&mut self) -> ::protobuf::RepeatedField<::std::string::String> { + ::std::mem::replace(&mut self.messageIDs, ::protobuf::RepeatedField::new()) + } + + pub fn get_messageIDs(&self) -> &[::std::string::String] { + &self.messageIDs + } +} + +impl ::protobuf::Message for ControlIHave { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicID)?; + }, + 2 => { + ::protobuf::rt::read_repeated_string_into(wire_type, is, &mut self.messageIDs)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(ref v) = self.topicID.as_ref() { + my_size += ::protobuf::rt::string_size(1, &v); + } + for value in &self.messageIDs { + my_size += ::protobuf::rt::string_size(2, &value); + }; + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(ref v) = self.topicID.as_ref() { + os.write_string(1, &v)?; + } + for v in &self.messageIDs { + os.write_string(2, &v)?; + }; + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> ControlIHave { + ControlIHave::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "topicID", + |m: &ControlIHave| { &m.topicID }, + |m: &mut ControlIHave| { &mut m.topicID }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "messageIDs", + |m: &ControlIHave| { &m.messageIDs }, + |m: &mut ControlIHave| { &mut m.messageIDs }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "ControlIHave", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static ControlIHave { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ControlIHave, + }; + unsafe { + instance.get(ControlIHave::new) + } + } +} + +impl ::protobuf::Clear for ControlIHave { + fn clear(&mut self) { + self.clear_topicID(); + self.clear_messageIDs(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for ControlIHave { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for ControlIHave { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct ControlIWant { + // message fields + messageIDs: ::protobuf::RepeatedField<::std::string::String>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl ControlIWant { + pub fn new() -> ControlIWant { + ::std::default::Default::default() + } + + // repeated string messageIDs = 1; + + pub fn clear_messageIDs(&mut self) { + self.messageIDs.clear(); + } + + // Param is passed by value, moved + pub fn set_messageIDs(&mut self, v: ::protobuf::RepeatedField<::std::string::String>) { + self.messageIDs = v; + } + + // Mutable pointer to the field. + pub fn mut_messageIDs(&mut self) -> &mut ::protobuf::RepeatedField<::std::string::String> { + &mut self.messageIDs + } + + // Take field + pub fn take_messageIDs(&mut self) -> ::protobuf::RepeatedField<::std::string::String> { + ::std::mem::replace(&mut self.messageIDs, ::protobuf::RepeatedField::new()) + } + + pub fn get_messageIDs(&self) -> &[::std::string::String] { + &self.messageIDs + } +} + +impl ::protobuf::Message for ControlIWant { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_repeated_string_into(wire_type, is, &mut self.messageIDs)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + for value in &self.messageIDs { + my_size += ::protobuf::rt::string_size(1, &value); + }; + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + for v in &self.messageIDs { + os.write_string(1, &v)?; + }; + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> ControlIWant { + ControlIWant::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "messageIDs", + |m: &ControlIWant| { &m.messageIDs }, + |m: &mut ControlIWant| { &mut m.messageIDs }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "ControlIWant", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static ControlIWant { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ControlIWant, + }; + unsafe { + instance.get(ControlIWant::new) + } + } +} + +impl ::protobuf::Clear for ControlIWant { + fn clear(&mut self) { + self.clear_messageIDs(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for ControlIWant { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for ControlIWant { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct ControlGraft { + // message fields + topicID: ::protobuf::SingularField<::std::string::String>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl ControlGraft { + pub fn new() -> ControlGraft { + ::std::default::Default::default() + } + + // optional string topicID = 1; + + pub fn clear_topicID(&mut self) { + self.topicID.clear(); + } + + pub fn has_topicID(&self) -> bool { + self.topicID.is_some() + } + + // Param is passed by value, moved + pub fn set_topicID(&mut self, v: ::std::string::String) { + self.topicID = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_topicID(&mut self) -> &mut ::std::string::String { + if self.topicID.is_none() { + self.topicID.set_default(); + } + self.topicID.as_mut().unwrap() + } + + // Take field + pub fn take_topicID(&mut self) -> ::std::string::String { + self.topicID.take().unwrap_or_else(|| ::std::string::String::new()) + } + + pub fn get_topicID(&self) -> &str { + match self.topicID.as_ref() { + Some(v) => &v, + None => "", + } + } +} + +impl ::protobuf::Message for ControlGraft { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicID)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(ref v) = self.topicID.as_ref() { + my_size += ::protobuf::rt::string_size(1, &v); + } + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(ref v) = self.topicID.as_ref() { + os.write_string(1, &v)?; + } + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> ControlGraft { + ControlGraft::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "topicID", + |m: &ControlGraft| { &m.topicID }, + |m: &mut ControlGraft| { &mut m.topicID }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "ControlGraft", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static ControlGraft { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ControlGraft, + }; + unsafe { + instance.get(ControlGraft::new) + } + } +} + +impl ::protobuf::Clear for ControlGraft { + fn clear(&mut self) { + self.clear_topicID(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for ControlGraft { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for ControlGraft { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct ControlPrune { + // message fields + topicID: ::protobuf::SingularField<::std::string::String>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl ControlPrune { + pub fn new() -> ControlPrune { + ::std::default::Default::default() + } + + // optional string topicID = 1; + + pub fn clear_topicID(&mut self) { + self.topicID.clear(); + } + + pub fn has_topicID(&self) -> bool { + self.topicID.is_some() + } + + // Param is passed by value, moved + pub fn set_topicID(&mut self, v: ::std::string::String) { + self.topicID = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_topicID(&mut self) -> &mut ::std::string::String { + if self.topicID.is_none() { + self.topicID.set_default(); + } + self.topicID.as_mut().unwrap() + } + + // Take field + pub fn take_topicID(&mut self) -> ::std::string::String { + self.topicID.take().unwrap_or_else(|| ::std::string::String::new()) + } + + pub fn get_topicID(&self) -> &str { + match self.topicID.as_ref() { + Some(v) => &v, + None => "", + } + } +} + +impl ::protobuf::Message for ControlPrune { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicID)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(ref v) = self.topicID.as_ref() { + my_size += ::protobuf::rt::string_size(1, &v); + } + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(ref v) = self.topicID.as_ref() { + os.write_string(1, &v)?; + } + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> ControlPrune { + ControlPrune::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "topicID", + |m: &ControlPrune| { &m.topicID }, + |m: &mut ControlPrune| { &mut m.topicID }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "ControlPrune", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static ControlPrune { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ControlPrune, + }; + unsafe { + instance.get(ControlPrune::new) + } + } +} + +impl ::protobuf::Clear for ControlPrune { + fn clear(&mut self) { + self.clear_topicID(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for ControlPrune { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for ControlPrune { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct TopicDescriptor { + // message fields + name: ::protobuf::SingularField<::std::string::String>, + auth: ::protobuf::SingularPtrField, + enc: ::protobuf::SingularPtrField, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl TopicDescriptor { + pub fn new() -> TopicDescriptor { + ::std::default::Default::default() + } + + // optional string name = 1; + + pub fn clear_name(&mut self) { + self.name.clear(); + } + + pub fn has_name(&self) -> bool { + self.name.is_some() + } + + // Param is passed by value, moved + pub fn set_name(&mut self, v: ::std::string::String) { + self.name = ::protobuf::SingularField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_name(&mut self) -> &mut ::std::string::String { + if self.name.is_none() { + self.name.set_default(); + } + self.name.as_mut().unwrap() + } + + // Take field + pub fn take_name(&mut self) -> ::std::string::String { + self.name.take().unwrap_or_else(|| ::std::string::String::new()) + } + + pub fn get_name(&self) -> &str { + match self.name.as_ref() { + Some(v) => &v, + None => "", + } + } + + // optional .floodsub.pb.TopicDescriptor.AuthOpts auth = 2; + + pub fn clear_auth(&mut self) { + self.auth.clear(); + } + + pub fn has_auth(&self) -> bool { + self.auth.is_some() + } + + // Param is passed by value, moved + pub fn set_auth(&mut self, v: TopicDescriptor_AuthOpts) { + self.auth = ::protobuf::SingularPtrField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_auth(&mut self) -> &mut TopicDescriptor_AuthOpts { + if self.auth.is_none() { + self.auth.set_default(); + } + self.auth.as_mut().unwrap() + } + + // Take field + pub fn take_auth(&mut self) -> TopicDescriptor_AuthOpts { + self.auth.take().unwrap_or_else(|| TopicDescriptor_AuthOpts::new()) + } + + pub fn get_auth(&self) -> &TopicDescriptor_AuthOpts { + self.auth.as_ref().unwrap_or_else(|| TopicDescriptor_AuthOpts::default_instance()) + } + + // optional .floodsub.pb.TopicDescriptor.EncOpts enc = 3; + + pub fn clear_enc(&mut self) { + self.enc.clear(); + } + + pub fn has_enc(&self) -> bool { + self.enc.is_some() + } + + // Param is passed by value, moved + pub fn set_enc(&mut self, v: TopicDescriptor_EncOpts) { + self.enc = ::protobuf::SingularPtrField::some(v); + } + + // Mutable pointer to the field. + // If field is not initialized, it is initialized with default value first. + pub fn mut_enc(&mut self) -> &mut TopicDescriptor_EncOpts { + if self.enc.is_none() { + self.enc.set_default(); + } + self.enc.as_mut().unwrap() + } + + // Take field + pub fn take_enc(&mut self) -> TopicDescriptor_EncOpts { + self.enc.take().unwrap_or_else(|| TopicDescriptor_EncOpts::new()) + } + + pub fn get_enc(&self) -> &TopicDescriptor_EncOpts { + self.enc.as_ref().unwrap_or_else(|| TopicDescriptor_EncOpts::default_instance()) + } +} + +impl ::protobuf::Message for TopicDescriptor { + fn is_initialized(&self) -> bool { + for v in &self.auth { + if !v.is_initialized() { + return false; + } + }; + for v in &self.enc { + if !v.is_initialized() { + return false; + } + }; + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.name)?; + }, + 2 => { + ::protobuf::rt::read_singular_message_into(wire_type, is, &mut self.auth)?; + }, + 3 => { + ::protobuf::rt::read_singular_message_into(wire_type, is, &mut self.enc)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(ref v) = self.name.as_ref() { + my_size += ::protobuf::rt::string_size(1, &v); + } + if let Some(ref v) = self.auth.as_ref() { + let len = v.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + } + if let Some(ref v) = self.enc.as_ref() { + let len = v.compute_size(); + my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; + } + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(ref v) = self.name.as_ref() { + os.write_string(1, &v)?; + } + if let Some(ref v) = self.auth.as_ref() { + os.write_tag(2, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + } + if let Some(ref v) = self.enc.as_ref() { + os.write_tag(3, ::protobuf::wire_format::WireTypeLengthDelimited)?; + os.write_raw_varint32(v.get_cached_size())?; + v.write_to_with_cached_sizes(os)?; + } + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> TopicDescriptor { + TopicDescriptor::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( + "name", + |m: &TopicDescriptor| { &m.name }, + |m: &mut TopicDescriptor| { &mut m.name }, + )); + fields.push(::protobuf::reflect::accessor::make_singular_ptr_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "auth", + |m: &TopicDescriptor| { &m.auth }, + |m: &mut TopicDescriptor| { &mut m.auth }, + )); + fields.push(::protobuf::reflect::accessor::make_singular_ptr_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( + "enc", + |m: &TopicDescriptor| { &m.enc }, + |m: &mut TopicDescriptor| { &mut m.enc }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "TopicDescriptor", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static TopicDescriptor { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const TopicDescriptor, + }; + unsafe { + instance.get(TopicDescriptor::new) + } + } +} + +impl ::protobuf::Clear for TopicDescriptor { + fn clear(&mut self) { + self.clear_name(); + self.clear_auth(); + self.clear_enc(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for TopicDescriptor { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for TopicDescriptor { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct TopicDescriptor_AuthOpts { + // message fields + mode: ::std::option::Option, + keys: ::protobuf::RepeatedField<::std::vec::Vec>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl TopicDescriptor_AuthOpts { + pub fn new() -> TopicDescriptor_AuthOpts { + ::std::default::Default::default() + } + + // optional .floodsub.pb.TopicDescriptor.AuthOpts.AuthMode mode = 1; + + pub fn clear_mode(&mut self) { + self.mode = ::std::option::Option::None; + } + + pub fn has_mode(&self) -> bool { + self.mode.is_some() + } + + // Param is passed by value, moved + pub fn set_mode(&mut self, v: TopicDescriptor_AuthOpts_AuthMode) { + self.mode = ::std::option::Option::Some(v); + } + + pub fn get_mode(&self) -> TopicDescriptor_AuthOpts_AuthMode { + self.mode.unwrap_or(TopicDescriptor_AuthOpts_AuthMode::NONE) + } + + // repeated bytes keys = 2; + + pub fn clear_keys(&mut self) { + self.keys.clear(); + } + + // Param is passed by value, moved + pub fn set_keys(&mut self, v: ::protobuf::RepeatedField<::std::vec::Vec>) { + self.keys = v; + } + + // Mutable pointer to the field. + pub fn mut_keys(&mut self) -> &mut ::protobuf::RepeatedField<::std::vec::Vec> { + &mut self.keys + } + + // Take field + pub fn take_keys(&mut self) -> ::protobuf::RepeatedField<::std::vec::Vec> { + ::std::mem::replace(&mut self.keys, ::protobuf::RepeatedField::new()) + } + + pub fn get_keys(&self) -> &[::std::vec::Vec] { + &self.keys + } +} + +impl ::protobuf::Message for TopicDescriptor_AuthOpts { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_proto2_enum_with_unknown_fields_into(wire_type, is, &mut self.mode, 1, &mut self.unknown_fields)? + }, + 2 => { + ::protobuf::rt::read_repeated_bytes_into(wire_type, is, &mut self.keys)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(v) = self.mode { + my_size += ::protobuf::rt::enum_size(1, v); + } + for value in &self.keys { + my_size += ::protobuf::rt::bytes_size(2, &value); + }; + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(v) = self.mode { + os.write_enum(1, v.value())?; + } + for v in &self.keys { + os.write_bytes(2, &v)?; + }; + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> TopicDescriptor_AuthOpts { + TopicDescriptor_AuthOpts::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeEnum>( + "mode", + |m: &TopicDescriptor_AuthOpts| { &m.mode }, + |m: &mut TopicDescriptor_AuthOpts| { &mut m.mode }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( + "keys", + |m: &TopicDescriptor_AuthOpts| { &m.keys }, + |m: &mut TopicDescriptor_AuthOpts| { &mut m.keys }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "TopicDescriptor_AuthOpts", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static TopicDescriptor_AuthOpts { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const TopicDescriptor_AuthOpts, + }; + unsafe { + instance.get(TopicDescriptor_AuthOpts::new) + } + } +} + +impl ::protobuf::Clear for TopicDescriptor_AuthOpts { + fn clear(&mut self) { + self.clear_mode(); + self.clear_keys(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for TopicDescriptor_AuthOpts { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_AuthOpts { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(Clone,PartialEq,Eq,Debug,Hash)] +pub enum TopicDescriptor_AuthOpts_AuthMode { + NONE = 0, + KEY = 1, + WOT = 2, +} + +impl ::protobuf::ProtobufEnum for TopicDescriptor_AuthOpts_AuthMode { + fn value(&self) -> i32 { + *self as i32 + } + + fn from_i32(value: i32) -> ::std::option::Option { + match value { + 0 => ::std::option::Option::Some(TopicDescriptor_AuthOpts_AuthMode::NONE), + 1 => ::std::option::Option::Some(TopicDescriptor_AuthOpts_AuthMode::KEY), + 2 => ::std::option::Option::Some(TopicDescriptor_AuthOpts_AuthMode::WOT), + _ => ::std::option::Option::None + } + } + + fn values() -> &'static [Self] { + static values: &'static [TopicDescriptor_AuthOpts_AuthMode] = &[ + TopicDescriptor_AuthOpts_AuthMode::NONE, + TopicDescriptor_AuthOpts_AuthMode::KEY, + TopicDescriptor_AuthOpts_AuthMode::WOT, + ]; + values + } + + fn enum_descriptor_static() -> &'static ::protobuf::reflect::EnumDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::EnumDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::EnumDescriptor, + }; + unsafe { + descriptor.get(|| { + ::protobuf::reflect::EnumDescriptor::new("TopicDescriptor_AuthOpts_AuthMode", file_descriptor_proto()) + }) + } + } +} + +impl ::std::marker::Copy for TopicDescriptor_AuthOpts_AuthMode { +} + +impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_AuthOpts_AuthMode { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Enum(self.descriptor()) + } +} + +#[derive(PartialEq,Clone,Default)] +pub struct TopicDescriptor_EncOpts { + // message fields + mode: ::std::option::Option, + keyHashes: ::protobuf::RepeatedField<::std::vec::Vec>, + // special fields + unknown_fields: ::protobuf::UnknownFields, + cached_size: ::protobuf::CachedSize, +} + +impl TopicDescriptor_EncOpts { + pub fn new() -> TopicDescriptor_EncOpts { + ::std::default::Default::default() + } + + // optional .floodsub.pb.TopicDescriptor.EncOpts.EncMode mode = 1; + + pub fn clear_mode(&mut self) { + self.mode = ::std::option::Option::None; + } + + pub fn has_mode(&self) -> bool { + self.mode.is_some() + } + + // Param is passed by value, moved + pub fn set_mode(&mut self, v: TopicDescriptor_EncOpts_EncMode) { + self.mode = ::std::option::Option::Some(v); + } + + pub fn get_mode(&self) -> TopicDescriptor_EncOpts_EncMode { + self.mode.unwrap_or(TopicDescriptor_EncOpts_EncMode::NONE) + } + + // repeated bytes keyHashes = 2; + + pub fn clear_keyHashes(&mut self) { + self.keyHashes.clear(); + } + + // Param is passed by value, moved + pub fn set_keyHashes(&mut self, v: ::protobuf::RepeatedField<::std::vec::Vec>) { + self.keyHashes = v; + } + + // Mutable pointer to the field. + pub fn mut_keyHashes(&mut self) -> &mut ::protobuf::RepeatedField<::std::vec::Vec> { + &mut self.keyHashes + } + + // Take field + pub fn take_keyHashes(&mut self) -> ::protobuf::RepeatedField<::std::vec::Vec> { + ::std::mem::replace(&mut self.keyHashes, ::protobuf::RepeatedField::new()) + } + + pub fn get_keyHashes(&self) -> &[::std::vec::Vec] { + &self.keyHashes + } +} + +impl ::protobuf::Message for TopicDescriptor_EncOpts { + fn is_initialized(&self) -> bool { + true + } + + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + while !is.eof()? { + let (field_number, wire_type) = is.read_tag_unpack()?; + match field_number { + 1 => { + ::protobuf::rt::read_proto2_enum_with_unknown_fields_into(wire_type, is, &mut self.mode, 1, &mut self.unknown_fields)? + }, + 2 => { + ::protobuf::rt::read_repeated_bytes_into(wire_type, is, &mut self.keyHashes)?; + }, + _ => { + ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; + }, + }; + } + ::std::result::Result::Ok(()) + } + + // Compute sizes of nested messages + #[allow(unused_variables)] + fn compute_size(&self) -> u32 { + let mut my_size = 0; + if let Some(v) = self.mode { + my_size += ::protobuf::rt::enum_size(1, v); + } + for value in &self.keyHashes { + my_size += ::protobuf::rt::bytes_size(2, &value); + }; + my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); + self.cached_size.set(my_size); + my_size + } + + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + if let Some(v) = self.mode { + os.write_enum(1, v.value())?; + } + for v in &self.keyHashes { + os.write_bytes(2, &v)?; + }; + os.write_unknown_fields(self.get_unknown_fields())?; + ::std::result::Result::Ok(()) + } + + fn get_cached_size(&self) -> u32 { + self.cached_size.get() + } + + fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { + &self.unknown_fields + } + + fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { + &mut self.unknown_fields + } + + fn as_any(&self) -> &::std::any::Any { + self as &::std::any::Any + } + fn as_any_mut(&mut self) -> &mut ::std::any::Any { + self as &mut ::std::any::Any + } + fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + Self::descriptor_static() + } + + fn new() -> TopicDescriptor_EncOpts { + TopicDescriptor_EncOpts::new() + } + + fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, + }; + unsafe { + descriptor.get(|| { + let mut fields = ::std::vec::Vec::new(); + fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeEnum>( + "mode", + |m: &TopicDescriptor_EncOpts| { &m.mode }, + |m: &mut TopicDescriptor_EncOpts| { &mut m.mode }, + )); + fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( + "keyHashes", + |m: &TopicDescriptor_EncOpts| { &m.keyHashes }, + |m: &mut TopicDescriptor_EncOpts| { &mut m.keyHashes }, + )); + ::protobuf::reflect::MessageDescriptor::new::( + "TopicDescriptor_EncOpts", + fields, + file_descriptor_proto() + ) + }) + } + } + + fn default_instance() -> &'static TopicDescriptor_EncOpts { + static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const TopicDescriptor_EncOpts, + }; + unsafe { + instance.get(TopicDescriptor_EncOpts::new) + } + } +} + +impl ::protobuf::Clear for TopicDescriptor_EncOpts { + fn clear(&mut self) { + self.clear_mode(); + self.clear_keyHashes(); + self.unknown_fields.clear(); + } +} + +impl ::std::fmt::Debug for TopicDescriptor_EncOpts { + fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + ::protobuf::text_format::fmt(self, f) + } +} + +impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_EncOpts { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Message(self) + } +} + +#[derive(Clone,PartialEq,Eq,Debug,Hash)] +pub enum TopicDescriptor_EncOpts_EncMode { + NONE = 0, + SHAREDKEY = 1, + WOT = 2, +} + +impl ::protobuf::ProtobufEnum for TopicDescriptor_EncOpts_EncMode { + fn value(&self) -> i32 { + *self as i32 + } + + fn from_i32(value: i32) -> ::std::option::Option { + match value { + 0 => ::std::option::Option::Some(TopicDescriptor_EncOpts_EncMode::NONE), + 1 => ::std::option::Option::Some(TopicDescriptor_EncOpts_EncMode::SHAREDKEY), + 2 => ::std::option::Option::Some(TopicDescriptor_EncOpts_EncMode::WOT), + _ => ::std::option::Option::None + } + } + + fn values() -> &'static [Self] { + static values: &'static [TopicDescriptor_EncOpts_EncMode] = &[ + TopicDescriptor_EncOpts_EncMode::NONE, + TopicDescriptor_EncOpts_EncMode::SHAREDKEY, + TopicDescriptor_EncOpts_EncMode::WOT, + ]; + values + } + + fn enum_descriptor_static() -> &'static ::protobuf::reflect::EnumDescriptor { + static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::EnumDescriptor> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::reflect::EnumDescriptor, + }; + unsafe { + descriptor.get(|| { + ::protobuf::reflect::EnumDescriptor::new("TopicDescriptor_EncOpts_EncMode", file_descriptor_proto()) + }) + } + } +} + +impl ::std::marker::Copy for TopicDescriptor_EncOpts_EncMode { +} + +impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_EncOpts_EncMode { + fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { + ::protobuf::reflect::ProtobufValueRef::Enum(self.descriptor()) + } +} + +static file_descriptor_proto_data: &'static [u8] = b"\ + \n\trpc.proto\x12\x0bfloodsub.pb\"\xef\x01\n\x03RPC\x12>\n\rsubscription\ + s\x18\x01\x20\x03(\x0b2\x18.floodsub.pb.RPC.SubOptsR\rsubscriptions\x12.\ + \n\x07publish\x18\x02\x20\x03(\x0b2\x14.floodsub.pb.MessageR\x07publish\ + \x125\n\x07control\x18\x03\x20\x01(\x0b2\x1b.floodsub.pb.ControlMessageR\ + \x07control\x1aA\n\x07SubOpts\x12\x1c\n\tsubscribe\x18\x01\x20\x01(\x08R\ + \tsubscribe\x12\x18\n\x07topicid\x18\x02\x20\x01(\tR\x07topicid\"c\n\x07\ + Message\x12\x12\n\x04from\x18\x01\x20\x01(\x0cR\x04from\x12\x12\n\x04dat\ + a\x18\x02\x20\x01(\x0cR\x04data\x12\x14\n\x05seqno\x18\x03\x20\x01(\x0cR\ + \x05seqno\x12\x1a\n\x08topicIDs\x18\x04\x20\x03(\tR\x08topicIDs\"\xd4\ + \x01\n\x0eControlMessage\x12/\n\x05ihave\x18\x01\x20\x03(\x0b2\x19.flood\ + sub.pb.ControlIHaveR\x05ihave\x12/\n\x05iwant\x18\x02\x20\x03(\x0b2\x19.\ + floodsub.pb.ControlIWantR\x05iwant\x12/\n\x05graft\x18\x03\x20\x03(\x0b2\ + \x19.floodsub.pb.ControlGraftR\x05graft\x12/\n\x05prune\x18\x04\x20\x03(\ + \x0b2\x19.floodsub.pb.ControlPruneR\x05prune\"H\n\x0cControlIHave\x12\ + \x18\n\x07topicID\x18\x01\x20\x01(\tR\x07topicID\x12\x1e\n\nmessageIDs\ + \x18\x02\x20\x03(\tR\nmessageIDs\".\n\x0cControlIWant\x12\x1e\n\nmessage\ + IDs\x18\x01\x20\x03(\tR\nmessageIDs\"(\n\x0cControlGraft\x12\x18\n\x07to\ + picID\x18\x01\x20\x01(\tR\x07topicID\"(\n\x0cControlPrune\x12\x18\n\x07t\ + opicID\x18\x01\x20\x01(\tR\x07topicID\"\xbe\x03\n\x0fTopicDescriptor\x12\ + \x12\n\x04name\x18\x01\x20\x01(\tR\x04name\x129\n\x04auth\x18\x02\x20\ + \x01(\x0b2%.floodsub.pb.TopicDescriptor.AuthOptsR\x04auth\x126\n\x03enc\ + \x18\x03\x20\x01(\x0b2$.floodsub.pb.TopicDescriptor.EncOptsR\x03enc\x1a\ + \x8a\x01\n\x08AuthOpts\x12B\n\x04mode\x18\x01\x20\x01(\x0e2..floodsub.pb\ + .TopicDescriptor.AuthOpts.AuthModeR\x04mode\x12\x12\n\x04keys\x18\x02\ + \x20\x03(\x0cR\x04keys\"&\n\x08AuthMode\x12\x08\n\x04NONE\x10\0\x12\x07\ + \n\x03KEY\x10\x01\x12\x07\n\x03WOT\x10\x02\x1a\x96\x01\n\x07EncOpts\x12@\ + \n\x04mode\x18\x01\x20\x01(\x0e2,.floodsub.pb.TopicDescriptor.EncOpts.En\ + cModeR\x04mode\x12\x1c\n\tkeyHashes\x18\x02\x20\x03(\x0cR\tkeyHashes\"+\ + \n\x07EncMode\x12\x08\n\x04NONE\x10\0\x12\r\n\tSHAREDKEY\x10\x01\x12\x07\ + \n\x03WOT\x10\x02J\xf6\x16\n\x06\x12\x04\0\0J\x01\n\x08\n\x01\x0c\x12\ + \x03\0\0\x12\n\x08\n\x01\x02\x12\x03\x02\x08\x13\n\n\n\x02\x04\0\x12\x04\ + \x04\0\x0e\x01\n\n\n\x03\x04\0\x01\x12\x03\x04\x08\x0b\n\x0b\n\x04\x04\0\ + \x02\0\x12\x03\x05\x08+\n\x0c\n\x05\x04\0\x02\0\x04\x12\x03\x05\x08\x10\ + \n\x0c\n\x05\x04\0\x02\0\x06\x12\x03\x05\x11\x18\n\x0c\n\x05\x04\0\x02\0\ + \x01\x12\x03\x05\x19&\n\x0c\n\x05\x04\0\x02\0\x03\x12\x03\x05)*\n\x0b\n\ + \x04\x04\0\x02\x01\x12\x03\x06\x08%\n\x0c\n\x05\x04\0\x02\x01\x04\x12\ + \x03\x06\x08\x10\n\x0c\n\x05\x04\0\x02\x01\x06\x12\x03\x06\x11\x18\n\x0c\ + \n\x05\x04\0\x02\x01\x01\x12\x03\x06\x19\x20\n\x0c\n\x05\x04\0\x02\x01\ + \x03\x12\x03\x06#$\n\x0c\n\x04\x04\0\x03\0\x12\x04\x08\x08\x0b\t\n\x0c\n\ + \x05\x04\0\x03\0\x01\x12\x03\x08\x10\x17\n(\n\x06\x04\0\x03\0\x02\0\x12\ + \x03\t\x10,\"\x19\x20subscribe\x20or\x20unsubcribe\n\n\x0e\n\x07\x04\0\ + \x03\0\x02\0\x04\x12\x03\t\x10\x18\n\x0e\n\x07\x04\0\x03\0\x02\0\x05\x12\ + \x03\t\x19\x1d\n\x0e\n\x07\x04\0\x03\0\x02\0\x01\x12\x03\t\x1e'\n\x0e\n\ + \x07\x04\0\x03\0\x02\0\x03\x12\x03\t*+\n\r\n\x06\x04\0\x03\0\x02\x01\x12\ + \x03\n\x10,\n\x0e\n\x07\x04\0\x03\0\x02\x01\x04\x12\x03\n\x10\x18\n\x0e\ + \n\x07\x04\0\x03\0\x02\x01\x05\x12\x03\n\x19\x1f\n\x0e\n\x07\x04\0\x03\0\ + \x02\x01\x01\x12\x03\n\x20'\n\x0e\n\x07\x04\0\x03\0\x02\x01\x03\x12\x03\ + \n*+\n\x0b\n\x04\x04\0\x02\x02\x12\x03\r\x08,\n\x0c\n\x05\x04\0\x02\x02\ + \x04\x12\x03\r\x08\x10\n\x0c\n\x05\x04\0\x02\x02\x06\x12\x03\r\x11\x1f\n\ + \x0c\n\x05\x04\0\x02\x02\x01\x12\x03\r\x20'\n\x0c\n\x05\x04\0\x02\x02\ + \x03\x12\x03\r*+\n\n\n\x02\x04\x01\x12\x04\x10\0\x15\x01\n\n\n\x03\x04\ + \x01\x01\x12\x03\x10\x08\x0f\n\x0b\n\x04\x04\x01\x02\0\x12\x03\x11\x08\ + \x20\n\x0c\n\x05\x04\x01\x02\0\x04\x12\x03\x11\x08\x10\n\x0c\n\x05\x04\ + \x01\x02\0\x05\x12\x03\x11\x11\x16\n\x0c\n\x05\x04\x01\x02\0\x01\x12\x03\ + \x11\x17\x1b\n\x0c\n\x05\x04\x01\x02\0\x03\x12\x03\x11\x1e\x1f\n\x0b\n\ + \x04\x04\x01\x02\x01\x12\x03\x12\x08\x20\n\x0c\n\x05\x04\x01\x02\x01\x04\ + \x12\x03\x12\x08\x10\n\x0c\n\x05\x04\x01\x02\x01\x05\x12\x03\x12\x11\x16\ + \n\x0c\n\x05\x04\x01\x02\x01\x01\x12\x03\x12\x17\x1b\n\x0c\n\x05\x04\x01\ + \x02\x01\x03\x12\x03\x12\x1e\x1f\n\x0b\n\x04\x04\x01\x02\x02\x12\x03\x13\ + \x08!\n\x0c\n\x05\x04\x01\x02\x02\x04\x12\x03\x13\x08\x10\n\x0c\n\x05\ + \x04\x01\x02\x02\x05\x12\x03\x13\x11\x16\n\x0c\n\x05\x04\x01\x02\x02\x01\ + \x12\x03\x13\x17\x1c\n\x0c\n\x05\x04\x01\x02\x02\x03\x12\x03\x13\x1f\x20\ + \n\x0b\n\x04\x04\x01\x02\x03\x12\x03\x14\x08%\n\x0c\n\x05\x04\x01\x02\ + \x03\x04\x12\x03\x14\x08\x10\n\x0c\n\x05\x04\x01\x02\x03\x05\x12\x03\x14\ + \x11\x17\n\x0c\n\x05\x04\x01\x02\x03\x01\x12\x03\x14\x18\x20\n\x0c\n\x05\ + \x04\x01\x02\x03\x03\x12\x03\x14#$\n\n\n\x02\x04\x02\x12\x04\x17\0\x1c\ + \x01\n\n\n\x03\x04\x02\x01\x12\x03\x17\x08\x16\n\x0b\n\x04\x04\x02\x02\0\ + \x12\x03\x18\x08(\n\x0c\n\x05\x04\x02\x02\0\x04\x12\x03\x18\x08\x10\n\ + \x0c\n\x05\x04\x02\x02\0\x06\x12\x03\x18\x11\x1d\n\x0c\n\x05\x04\x02\x02\ + \0\x01\x12\x03\x18\x1e#\n\x0c\n\x05\x04\x02\x02\0\x03\x12\x03\x18&'\n\ + \x0b\n\x04\x04\x02\x02\x01\x12\x03\x19\x08(\n\x0c\n\x05\x04\x02\x02\x01\ + \x04\x12\x03\x19\x08\x10\n\x0c\n\x05\x04\x02\x02\x01\x06\x12\x03\x19\x11\ + \x1d\n\x0c\n\x05\x04\x02\x02\x01\x01\x12\x03\x19\x1e#\n\x0c\n\x05\x04\ + \x02\x02\x01\x03\x12\x03\x19&'\n\x0b\n\x04\x04\x02\x02\x02\x12\x03\x1a\ + \x08(\n\x0c\n\x05\x04\x02\x02\x02\x04\x12\x03\x1a\x08\x10\n\x0c\n\x05\ + \x04\x02\x02\x02\x06\x12\x03\x1a\x11\x1d\n\x0c\n\x05\x04\x02\x02\x02\x01\ + \x12\x03\x1a\x1e#\n\x0c\n\x05\x04\x02\x02\x02\x03\x12\x03\x1a&'\n\x0b\n\ + \x04\x04\x02\x02\x03\x12\x03\x1b\x08(\n\x0c\n\x05\x04\x02\x02\x03\x04\ + \x12\x03\x1b\x08\x10\n\x0c\n\x05\x04\x02\x02\x03\x06\x12\x03\x1b\x11\x1d\ + \n\x0c\n\x05\x04\x02\x02\x03\x01\x12\x03\x1b\x1e#\n\x0c\n\x05\x04\x02\ + \x02\x03\x03\x12\x03\x1b&'\n\n\n\x02\x04\x03\x12\x04\x1e\0!\x01\n\n\n\ + \x03\x04\x03\x01\x12\x03\x1e\x08\x14\n\x0b\n\x04\x04\x03\x02\0\x12\x03\ + \x1f\x08$\n\x0c\n\x05\x04\x03\x02\0\x04\x12\x03\x1f\x08\x10\n\x0c\n\x05\ + \x04\x03\x02\0\x05\x12\x03\x1f\x11\x17\n\x0c\n\x05\x04\x03\x02\0\x01\x12\ + \x03\x1f\x18\x1f\n\x0c\n\x05\x04\x03\x02\0\x03\x12\x03\x1f\"#\n\x0b\n\ + \x04\x04\x03\x02\x01\x12\x03\x20\x08'\n\x0c\n\x05\x04\x03\x02\x01\x04\ + \x12\x03\x20\x08\x10\n\x0c\n\x05\x04\x03\x02\x01\x05\x12\x03\x20\x11\x17\ + \n\x0c\n\x05\x04\x03\x02\x01\x01\x12\x03\x20\x18\"\n\x0c\n\x05\x04\x03\ + \x02\x01\x03\x12\x03\x20%&\n\n\n\x02\x04\x04\x12\x04#\0%\x01\n\n\n\x03\ + \x04\x04\x01\x12\x03#\x08\x14\n\x0b\n\x04\x04\x04\x02\0\x12\x03$\x08'\n\ + \x0c\n\x05\x04\x04\x02\0\x04\x12\x03$\x08\x10\n\x0c\n\x05\x04\x04\x02\0\ + \x05\x12\x03$\x11\x17\n\x0c\n\x05\x04\x04\x02\0\x01\x12\x03$\x18\"\n\x0c\ + \n\x05\x04\x04\x02\0\x03\x12\x03$%&\n\n\n\x02\x04\x05\x12\x04'\0)\x01\n\ + \n\n\x03\x04\x05\x01\x12\x03'\x08\x14\n\x0b\n\x04\x04\x05\x02\0\x12\x03(\ + \x08$\n\x0c\n\x05\x04\x05\x02\0\x04\x12\x03(\x08\x10\n\x0c\n\x05\x04\x05\ + \x02\0\x05\x12\x03(\x11\x17\n\x0c\n\x05\x04\x05\x02\0\x01\x12\x03(\x18\ + \x1f\n\x0c\n\x05\x04\x05\x02\0\x03\x12\x03(\"#\n\n\n\x02\x04\x06\x12\x04\ + +\0-\x01\n\n\n\x03\x04\x06\x01\x12\x03+\x08\x14\n\x0b\n\x04\x04\x06\x02\ + \0\x12\x03,\x08$\n\x0c\n\x05\x04\x06\x02\0\x04\x12\x03,\x08\x10\n\x0c\n\ + \x05\x04\x06\x02\0\x05\x12\x03,\x11\x17\n\x0c\n\x05\x04\x06\x02\0\x01\ + \x12\x03,\x18\x1f\n\x0c\n\x05\x04\x06\x02\0\x03\x12\x03,\"#\nC\n\x02\x04\ + \x07\x12\x040\0J\x01\x1a7\x20topicID\x20=\x20hash(topicDescriptor);\x20(\ + not\x20the\x20topic.name)\n\n\n\n\x03\x04\x07\x01\x12\x030\x08\x17\n\x0b\ + \n\x04\x04\x07\x02\0\x12\x031\x08!\n\x0c\n\x05\x04\x07\x02\0\x04\x12\x03\ + 1\x08\x10\n\x0c\n\x05\x04\x07\x02\0\x05\x12\x031\x11\x17\n\x0c\n\x05\x04\ + \x07\x02\0\x01\x12\x031\x18\x1c\n\x0c\n\x05\x04\x07\x02\0\x03\x12\x031\ + \x1f\x20\n\x0b\n\x04\x04\x07\x02\x01\x12\x032\x08#\n\x0c\n\x05\x04\x07\ + \x02\x01\x04\x12\x032\x08\x10\n\x0c\n\x05\x04\x07\x02\x01\x06\x12\x032\ + \x11\x19\n\x0c\n\x05\x04\x07\x02\x01\x01\x12\x032\x1a\x1e\n\x0c\n\x05\ + \x04\x07\x02\x01\x03\x12\x032!\"\n\x0b\n\x04\x04\x07\x02\x02\x12\x033\ + \x08!\n\x0c\n\x05\x04\x07\x02\x02\x04\x12\x033\x08\x10\n\x0c\n\x05\x04\ + \x07\x02\x02\x06\x12\x033\x11\x18\n\x0c\n\x05\x04\x07\x02\x02\x01\x12\ + \x033\x19\x1c\n\x0c\n\x05\x04\x07\x02\x02\x03\x12\x033\x1f\x20\n\x0c\n\ + \x04\x04\x07\x03\0\x12\x045\x08>\t\n\x0c\n\x05\x04\x07\x03\0\x01\x12\x03\ + 5\x10\x18\n\r\n\x06\x04\x07\x03\0\x02\0\x12\x036\x10+\n\x0e\n\x07\x04\ + \x07\x03\0\x02\0\x04\x12\x036\x10\x18\n\x0e\n\x07\x04\x07\x03\0\x02\0\ + \x06\x12\x036\x19!\n\x0e\n\x07\x04\x07\x03\0\x02\0\x01\x12\x036\"&\n\x0e\ + \n\x07\x04\x07\x03\0\x02\0\x03\x12\x036)*\n#\n\x06\x04\x07\x03\0\x02\x01\ + \x12\x037\x10(\"\x14\x20root\x20keys\x20to\x20trust\n\n\x0e\n\x07\x04\ + \x07\x03\0\x02\x01\x04\x12\x037\x10\x18\n\x0e\n\x07\x04\x07\x03\0\x02\ + \x01\x05\x12\x037\x19\x1e\n\x0e\n\x07\x04\x07\x03\0\x02\x01\x01\x12\x037\ + \x1f#\n\x0e\n\x07\x04\x07\x03\0\x02\x01\x03\x12\x037&'\n\x0e\n\x06\x04\ + \x07\x03\0\x04\0\x12\x049\x10=\x11\n\x0e\n\x07\x04\x07\x03\0\x04\0\x01\ + \x12\x039\x15\x1d\n8\n\x08\x04\x07\x03\0\x04\0\x02\0\x12\x03:\x18!\"'\ + \x20no\x20authentication,\x20anyone\x20can\x20publish\n\n\x10\n\t\x04\ + \x07\x03\0\x04\0\x02\0\x01\x12\x03:\x18\x1c\n\x10\n\t\x04\x07\x03\0\x04\ + \0\x02\0\x02\x12\x03:\x1f\x20\nT\n\x08\x04\x07\x03\0\x04\0\x02\x01\x12\ + \x03;\x18\x20\"C\x20only\x20messages\x20signed\x20by\x20keys\x20in\x20th\ + e\x20topic\x20descriptor\x20are\x20accepted\n\n\x10\n\t\x04\x07\x03\0\ + \x04\0\x02\x01\x01\x12\x03;\x18\x1b\n\x10\n\t\x04\x07\x03\0\x04\0\x02\ + \x01\x02\x12\x03;\x1e\x1f\nM\n\x08\x04\x07\x03\0\x04\0\x02\x02\x12\x03<\ + \x18\x20\"<\x20web\x20of\x20trust,\x20certificates\x20can\x20allow\x20pu\ + blisher\x20set\x20to\x20grow\n\n\x10\n\t\x04\x07\x03\0\x04\0\x02\x02\x01\ + \x12\x03<\x18\x1b\n\x10\n\t\x04\x07\x03\0\x04\0\x02\x02\x02\x12\x03<\x1e\ + \x1f\n\x0c\n\x04\x04\x07\x03\x01\x12\x04@\x08I\t\n\x0c\n\x05\x04\x07\x03\ + \x01\x01\x12\x03@\x10\x17\n\r\n\x06\x04\x07\x03\x01\x02\0\x12\x03A\x10*\ + \n\x0e\n\x07\x04\x07\x03\x01\x02\0\x04\x12\x03A\x10\x18\n\x0e\n\x07\x04\ + \x07\x03\x01\x02\0\x06\x12\x03A\x19\x20\n\x0e\n\x07\x04\x07\x03\x01\x02\ + \0\x01\x12\x03A!%\n\x0e\n\x07\x04\x07\x03\x01\x02\0\x03\x12\x03A()\n<\n\ + \x06\x04\x07\x03\x01\x02\x01\x12\x03B\x10-\"-\x20the\x20hashes\x20of\x20\ + the\x20shared\x20keys\x20used\x20(salted)\n\n\x0e\n\x07\x04\x07\x03\x01\ + \x02\x01\x04\x12\x03B\x10\x18\n\x0e\n\x07\x04\x07\x03\x01\x02\x01\x05\ + \x12\x03B\x19\x1e\n\x0e\n\x07\x04\x07\x03\x01\x02\x01\x01\x12\x03B\x1f(\ + \n\x0e\n\x07\x04\x07\x03\x01\x02\x01\x03\x12\x03B+,\n\x0e\n\x06\x04\x07\ + \x03\x01\x04\0\x12\x04D\x10H\x11\n\x0e\n\x07\x04\x07\x03\x01\x04\0\x01\ + \x12\x03D\x15\x1c\n1\n\x08\x04\x07\x03\x01\x04\0\x02\0\x12\x03E\x18!\"\ + \x20\x20no\x20encryption,\x20anyone\x20can\x20read\n\n\x10\n\t\x04\x07\ + \x03\x01\x04\0\x02\0\x01\x12\x03E\x18\x1c\n\x10\n\t\x04\x07\x03\x01\x04\ + \0\x02\0\x02\x12\x03E\x1f\x20\n9\n\x08\x04\x07\x03\x01\x04\0\x02\x01\x12\ + \x03F\x18&\"(\x20messages\x20are\x20encrypted\x20with\x20shared\x20key\n\ + \n\x10\n\t\x04\x07\x03\x01\x04\0\x02\x01\x01\x12\x03F\x18!\n\x10\n\t\x04\ + \x07\x03\x01\x04\0\x02\x01\x02\x12\x03F$%\nM\n\x08\x04\x07\x03\x01\x04\0\ + \x02\x02\x12\x03G\x18\x20\"<\x20web\x20of\x20trust,\x20certificates\x20c\ + an\x20allow\x20publisher\x20set\x20to\x20grow\n\n\x10\n\t\x04\x07\x03\ + \x01\x04\0\x02\x02\x01\x12\x03G\x18\x1b\n\x10\n\t\x04\x07\x03\x01\x04\0\ + \x02\x02\x02\x12\x03G\x1e\x1f\ +"; + +static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy { + lock: ::protobuf::lazy::ONCE_INIT, + ptr: 0 as *const ::protobuf::descriptor::FileDescriptorProto, +}; + +fn parse_descriptor_proto() -> ::protobuf::descriptor::FileDescriptorProto { + ::protobuf::parse_from_bytes(file_descriptor_proto_data).unwrap() +} + +pub fn file_descriptor_proto() -> &'static ::protobuf::descriptor::FileDescriptorProto { + unsafe { + file_descriptor_proto_lazy.get(|| { + parse_descriptor_proto() + }) + } +} From e09cb3fbc733021e010a13a9c763a00d665ed1d3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 16 Jan 2019 17:55:02 +1100 Subject: [PATCH 02/91] Add a GossipsubConfig struct and set up basic structures in the Gossipsub struct --- protocols/gossipsub/src/layer.rs | 107 ++++++++++++++++++++++++------ protocols/gossipsub/src/mcache.rs | 85 +++++++++++++----------- 2 files changed, 131 insertions(+), 61 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index f9ec78a8369..db27831d86f 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -26,15 +26,41 @@ use libp2p_core::swarm::{ }; use libp2p_core::{protocols_handler::ProtocolsHandler, PeerId}; use libp2p_floodsub::{Topic, TopicHash}; +use mcache::MessageCache; use protocol::{ GossipsubMessage, GossipsubRpc, GossipsubSubscription, GossipsubSubscriptionAction, }; use rand; use smallvec::SmallVec; use std::collections::hash_map::{DefaultHasher, HashMap}; +use std::time::Duration; use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; +// potentially rename this struct - due to clashes +/// Configuration parameters that define the performance of the gossipsub network +pub struct GossipsubConfig { + /// Overlay network parameters + /// Number of heartbeats to keep in the memcache + history_length: usize, + /// Number of past heartbeats to gossip about + history_gossip: usize, + + /// Target number of peers for the mesh network + mesh_n: usize, + /// Minimum number of peers in mesh network before adding more + mesh_n_low: usize, + /// Maximum number of peers in mesh network before removing some + mesh_n_high: usize, + + /// Initial delay in each heartbeat + heartbeat_initial_delay: Duration, + /// Time between each heartbeat + heartbeat_interval: Duration, + /// Time to live for fanout peers + fanout_ttl: Duration, +} + /// Network behaviour that automatically identifies nodes periodically, and returns information /// about them. pub struct Gossipsub { @@ -44,32 +70,50 @@ pub struct Gossipsub { /// Peer id of the local node. Used for the source of the messages that we publish. local_peer_id: PeerId, + // These data structures may be combined in later revisions - kept for ease of iteration /// List of peers the network is connected to, and the topics that they're subscribed to. - // TODO: filter out peers that don't support gossipsub so that we avoid hammering them with - // opened substream - connected_peers: HashMap>, - - // List of topics we're subscribed to. Necessary to filter out messages that we receive - // erroneously. - subscribed_topics: SmallVec<[Topic; 16]>, + peers_topic: HashMap>, + /// Inverse hashmap of connected_peers - maps a topic to a tuple which contains a list of + /// gossipsub peers and floodsub peers. Used to efficiently look up peers per topic. + topic_peers: HashMap, Vec)>, + + /* use topic_peers instead of two hashmaps + /// Map of topics to connected gossipsub peers + gossipsub_peers: HashMap>, + /// Map of topics to connected floodsub peers + floodsub_peers: HashMap>, + */ + /// Overlay network of connected peers - Maps topics to connected gossipsub peers + mesh: HashMap>, + + /// Map of topics to list of peers that we publish to, but don't subscribe to + fanout: HashMap>, + + /// Message cache for the last few heartbeats + mcache: MessageCache, // We keep track of the messages we received (in the format `hash(source ID, seq_no)`) so that // we don't dispatch the same message twice if we receive it twice on the network. received: CuckooFilter, + subscribed_topics: SmallVec<[Topic; 16]>, /// Marker to pin the generics. marker: PhantomData, } impl Gossipsub { /// Creates a `Gossipsub`. - pub fn new(local_peer_id: PeerId) -> Self { + pub fn new(local_peer_id: PeerId, gs_config: GossipsubConfig) -> Self { Gossipsub { events: VecDeque::new(), local_peer_id, - connected_peers: HashMap::new(), - subscribed_topics: SmallVec::new(), + peers_topic: HashMap::new(), + topic_peers: HashMap::new(), + mesh: HashMap::new(), + fanout: HashMap::new(), + mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), received: CuckooFilter::new(), + subscribed_topics: SmallVec::new(), marker: PhantomData, } } @@ -78,6 +122,7 @@ impl Gossipsub { /// /// Returns true if the subscription worked. Returns false if we were already subscribed. pub fn subscribe(&mut self, topic: Topic) -> bool { + // TODO: Can simply check if topic is in the mesh if self .subscribed_topics .iter() @@ -86,7 +131,8 @@ impl Gossipsub { return false; } - for peer in self.connected_peers.keys() { + // send subscription request to all floodsub and gossipsub peers + for peer in self.peers_topic.keys() { self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { @@ -99,7 +145,11 @@ impl Gossipsub { }); } - self.subscribed_topics.push(topic); + self.subscribed_topics.push(topic.clone()); + + // call JOIN(topic) + self.join(topic); + true } @@ -109,11 +159,12 @@ impl Gossipsub { /// /// Returns true if we were subscribed to this topic. pub fn unsubscribe(&mut self, topic: impl AsRef) -> bool { - let topic = topic.as_ref(); + let topic_hash = topic.as_ref(); + // TODO: Check the mesh if we are subscribed let pos = match self .subscribed_topics .iter() - .position(|t| t.hash() == topic) + .position(|t| t.hash() == topic_hash) { Some(pos) => pos, None => return false, @@ -121,22 +172,27 @@ impl Gossipsub { self.subscribed_topics.remove(pos); - for peer in self.connected_peers.keys() { + // announce to all floodsub and gossipsub peers + for peer in self.peers_topic.keys() { self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { messages: Vec::new(), subscriptions: vec![GossipsubSubscription { - topic: topic.clone(), + topic: topic_hash.clone(), action: GossipsubSubscriptionAction::Unsubscribe, }], }, }); } + // call LEAVE(topic) + self.leave(&topic); + true } + //TODO: Update publish for gossipsub /// Publishes a message to the network. /// /// > **Note**: Doesn't do anything if we're not subscribed to the topic. @@ -162,19 +218,22 @@ impl Gossipsub { topics: topic.into_iter().map(|t| t.into().clone()).collect(), }; - // Don't publish the message if we're not subscribed ourselves to any of the topics. + // If we are not subscribed to the topic, forward to fanout peers + // TODO: Can check mesh if !self .subscribed_topics .iter() .any(|t| message.topics.iter().any(|u| t.hash() == u)) { + //TODO: Send to fanout peers if exist - add fanout logic + // loop through topics etc return; } self.received.add(&message); // Send to peers we know are subscribed to the topic. - for (peer_id, sub_topic) in self.connected_peers.iter() { + for (peer_id, sub_topic) in self.peers_topic.iter() { if !sub_topic .iter() .any(|t| message.topics.iter().any(|u| t == u)) @@ -192,6 +251,10 @@ impl Gossipsub { }); } } + + fn join(&mut self, topic: impl AsRef) {} + + fn leave(&mut self, topic: impl AsRef) {} } impl NetworkBehaviour for Gossipsub @@ -220,18 +283,18 @@ where }); } - self.connected_peers.insert(id.clone(), SmallVec::new()); + self.peers_topic.insert(id.clone(), SmallVec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { - let was_in = self.connected_peers.remove(id); + let was_in = self.peers_topic.remove(id); debug_assert!(was_in.is_some()); } fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { // Update connected peers topics for subscription in event.subscriptions { - let mut remote_peer_topics = self.connected_peers + let mut remote_peer_topics = self.peers_topic .get_mut(&propagation_source) .expect("connected_peers is kept in sync with the peers we are connected to; we are guaranteed to only receive events from connected peers; QED"); match subscription.action { @@ -285,7 +348,7 @@ where } // Propagate the message to everyone else who is subscribed to any of the topics. - for (peer_id, subscr_topics) in self.connected_peers.iter() { + for (peer_id, subscr_topics) in self.peers_topic.iter() { if peer_id == &propagation_source { continue; } diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 5586ac3fd30..67523b50bc3 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -1,7 +1,7 @@ extern crate fnv; -use super::rpc_proto::{Message}; -use fnv::{FnvHashMap}; +use super::rpc_proto::Message; +use fnv::FnvHashMap; /// CacheEntry stored in the history #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -15,13 +15,13 @@ pub struct CacheEntry { pub struct MessageCache { msgs: FnvHashMap, history: Vec>, - gossip : usize, + gossip: usize, } /// Implementation of the MessageCache impl MessageCache { pub fn new(gossip: usize, history_capacity: usize) -> MessageCache { - MessageCache{ + MessageCache { gossip, msgs: FnvHashMap::default(), history: vec![Vec::new(); history_capacity], @@ -33,7 +33,7 @@ impl MessageCache { let message_id = msg_id(&msg)?; let cache_entry = CacheEntry { mid: message_id.clone(), - topics: msg.get_topicIDs().to_vec() + topics: msg.get_topicIDs().to_vec(), }; self.msgs.insert(message_id, msg); @@ -49,30 +49,31 @@ impl MessageCache { /// Get a list of GossipIds for a given topic pub fn get_gossip_ids(&self, topic: &str) -> Vec { - self.history[..self.gossip] - .iter().fold(vec![], |mut current_entries, entries| { - // search for entries with desired topic - let mut found_entries: Vec = - entries.iter().filter_map(|entry| { + .iter() + .fold(vec![], |mut current_entries, entries| { + // search for entries with desired topic + let mut found_entries: Vec = entries + .iter() + .filter_map(|entry| { if entry.topics.iter().any(|t| *t == topic) { Some(entry.mid.clone()) - } - else { + } else { None } - }).collect(); - - // generate the list - current_entries.append(&mut found_entries); - current_entries }) + .collect(); + + // generate the list + current_entries.append(&mut found_entries); + current_entries + }) } /// Shift the history array down one and delete messages associated with the /// last entry pub fn shift(&mut self) { - let last_index = self.history.len()-1; + let last_index = self.history.len() - 1; for entry in &self.history[last_index] { self.msgs.remove(&entry.mid); } @@ -89,18 +90,14 @@ impl MessageCache { // } // self.history[0] = Vec::new(); } - } - // Functions to be refactored later /// Gets a unique message id. /// Returns an error if the message has non-utf from or seqno values fn msg_id(pmsg: &Message) -> Result { - let from = - String::from_utf8(pmsg.get_from().to_vec()).or(Err(MsgError::InvalidMessage))?; - let seqno = - String::from_utf8(pmsg.get_seqno().to_vec()).or(Err(MsgError::InvalidMessage))?; + let from = String::from_utf8(pmsg.get_from().to_vec()).or(Err(MsgError::InvalidMessage))?; + let seqno = String::from_utf8(pmsg.get_seqno().to_vec()).or(Err(MsgError::InvalidMessage))?; Ok(from + &seqno) } @@ -109,11 +106,10 @@ pub enum MsgError { InvalidMessage, } - #[cfg(test)] mod tests { - use super::*; use super::super::protobuf; + use super::*; fn gen_testm(x: usize, topics: Vec) -> Message { let u8x: u8 = x as u8; @@ -149,7 +145,10 @@ mod tests { fn test_put_get_one() { let mut mc = MessageCache::new(10, 15); - let m = gen_testm(10 as usize, vec![String::from("hello"), String::from("world")]); + let m = gen_testm( + 10 as usize, + vec![String::from("hello"), String::from("world")], + ); let res = mc.put(m.clone()); assert_eq!(res.is_err(), false); @@ -162,7 +161,7 @@ mod tests { let fetched = match mid.ok() { Some(id) => mc.get(&id), - _ => None + _ => None, }; assert_eq!(fetched.is_none(), false); @@ -171,7 +170,7 @@ mod tests { // Make sure it is the same fetched message match fetched { Some(x) => assert_eq!(*x, m), - _ => assert!(false) + _ => assert!(false), } } @@ -181,7 +180,10 @@ mod tests { let mut mc = MessageCache::new(10, 15); // Build the message - let m = gen_testm(1 as usize, vec![String::from("hello"), String::from("world")]); + let m = gen_testm( + 1 as usize, + vec![String::from("hello"), String::from("world")], + ); let res = mc.put(m.clone()); assert_eq!(res.is_err(), false); @@ -196,7 +198,6 @@ mod tests { assert_eq!(fetched.is_none(), true); } - #[test] /// Test attempting to 'get' empty message cache fn test_get_empty() { @@ -223,13 +224,13 @@ mod tests { let mid = msg_id(&m.clone()); let fetched = match mid.ok() { Some(id) => mc.get(&id), - _ => None + _ => None, }; // Make sure it is the same fetched message match fetched { Some(x) => assert_eq!(*x, m), - _ => assert!(false) + _ => assert!(false), } } @@ -240,7 +241,10 @@ mod tests { // Build the message for i in 0..10 { - let m = gen_testm(i as usize, vec![String::from("hello"), String::from("world")]); + let m = gen_testm( + i as usize, + vec![String::from("hello"), String::from("world")], + ); let res = mc.put(m.clone()); assert_eq!(res.is_err(), false); } @@ -262,7 +266,10 @@ mod tests { // Build the message for i in 0..10 { - let m = gen_testm(i as usize, vec![String::from("hello"), String::from("world")]); + let m = gen_testm( + i as usize, + vec![String::from("hello"), String::from("world")], + ); let res = mc.put(m.clone()); assert_eq!(res.is_err(), false); } @@ -283,11 +290,13 @@ mod tests { #[test] /// Test shift to see if the last history messages are removed fn test_remove_last_from_shift() { - let mut mc = MessageCache::new(4, 5); for i in 0..10 { - let m = gen_testm(i as usize, vec![String::from("hello"), String::from("world")]); + let m = gen_testm( + i as usize, + vec![String::from("hello"), String::from("world")], + ); let res = mc.put(m.clone()); assert_eq!(res.is_err(), false); } @@ -307,5 +316,3 @@ mod tests { assert_eq!(mc.msgs.len(), 0); } } - - From 65f3211bc0275b73f3e060ff1cf023ce43e1cd19 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 16 Jan 2019 23:16:55 +1100 Subject: [PATCH 03/91] Begin implementation of join. Adds get_random_peers helper function and adds tests --- protocols/gossipsub/src/layer.rs | 173 ++++++++++++++++++++++++++++--- 1 file changed, 159 insertions(+), 14 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index db27831d86f..2dd52c1fd43 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -31,14 +31,16 @@ use protocol::{ GossipsubMessage, GossipsubRpc, GossipsubSubscription, GossipsubSubscriptionAction, }; use rand; +use rand::{seq::SliceRandom, thread_rng}; use smallvec::SmallVec; use std::collections::hash_map::{DefaultHasher, HashMap}; -use std::time::Duration; +use std::time::{Duration, Instant}; use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; // potentially rename this struct - due to clashes /// Configuration parameters that define the performance of the gossipsub network +#[derive(Clone)] pub struct GossipsubConfig { /// Overlay network parameters /// Number of heartbeats to keep in the memcache @@ -46,11 +48,11 @@ pub struct GossipsubConfig { /// Number of past heartbeats to gossip about history_gossip: usize, - /// Target number of peers for the mesh network + /// Target number of peers for the mesh network (D in the spec) mesh_n: usize, - /// Minimum number of peers in mesh network before adding more + /// Minimum number of peers in mesh network before adding more (D_lo in the spec) mesh_n_low: usize, - /// Maximum number of peers in mesh network before removing some + /// Maximum number of peers in mesh network before removing some (D_high in the spec) mesh_n_high: usize, /// Initial delay in each heartbeat @@ -61,9 +63,59 @@ pub struct GossipsubConfig { fanout_ttl: Duration, } +impl Default for GossipsubConfig { + fn default() -> GossipsubConfig { + GossipsubConfig { + history_length: 5, + history_gossip: 3, + mesh_n: 6, + mesh_n_low: 4, + mesh_n_high: 12, + heartbeat_initial_delay: Duration::from_millis(100), + heartbeat_interval: Duration::from_secs(1), + fanout_ttl: Duration::from_secs(60), + } + } +} + +impl GossipsubConfig { + pub fn new( + history_length: usize, + history_gossip: usize, + mesh_n: usize, + mesh_n_low: usize, + mesh_n_high: usize, + heartbeat_initial_delay: Duration, + heartbeat_interval: Duration, + fanout_ttl: Duration, + ) -> GossipsubConfig { + assert!( + history_length >= history_gossip, + "The history_length must be greater than or equal to the history_gossip length" + ); + assert!( + mesh_n_low <= mesh_n && mesh_n <= mesh_n_high, + "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" + ); + GossipsubConfig { + history_length, + history_gossip, + mesh_n, + mesh_n_low, + mesh_n_high, + heartbeat_initial_delay, + heartbeat_interval, + fanout_ttl, + } + } +} + /// Network behaviour that automatically identifies nodes periodically, and returns information /// about them. pub struct Gossipsub { + /// Configuration providing gossipsub performance parameters + config: GossipsubConfig, + /// Events that need to be yielded to the outside when polling. events: VecDeque>, @@ -72,7 +124,7 @@ pub struct Gossipsub { // These data structures may be combined in later revisions - kept for ease of iteration /// List of peers the network is connected to, and the topics that they're subscribed to. - peers_topic: HashMap>, + peer_topics: HashMap>, /// Inverse hashmap of connected_peers - maps a topic to a tuple which contains a list of /// gossipsub peers and floodsub peers. Used to efficiently look up peers per topic. topic_peers: HashMap, Vec)>, @@ -89,6 +141,9 @@ pub struct Gossipsub { /// Map of topics to list of peers that we publish to, but don't subscribe to fanout: HashMap>, + /// The last publish time for fanout topics + fanout_last_pub: HashMap, + /// Message cache for the last few heartbeats mcache: MessageCache, @@ -105,12 +160,14 @@ impl Gossipsub { /// Creates a `Gossipsub`. pub fn new(local_peer_id: PeerId, gs_config: GossipsubConfig) -> Self { Gossipsub { + config: gs_config.clone(), events: VecDeque::new(), local_peer_id, - peers_topic: HashMap::new(), + peer_topics: HashMap::new(), topic_peers: HashMap::new(), mesh: HashMap::new(), fanout: HashMap::new(), + fanout_last_pub: HashMap::new(), mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), received: CuckooFilter::new(), subscribed_topics: SmallVec::new(), @@ -132,7 +189,7 @@ impl Gossipsub { } // send subscription request to all floodsub and gossipsub peers - for peer in self.peers_topic.keys() { + for peer in self.peer_topics.keys() { self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { @@ -173,7 +230,7 @@ impl Gossipsub { self.subscribed_topics.remove(pos); // announce to all floodsub and gossipsub peers - for peer in self.peers_topic.keys() { + for peer in self.peer_topics.keys() { self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { @@ -233,7 +290,7 @@ impl Gossipsub { self.received.add(&message); // Send to peers we know are subscribed to the topic. - for (peer_id, sub_topic) in self.peers_topic.iter() { + for (peer_id, sub_topic) in self.peer_topics.iter() { if !sub_topic .iter() .any(|t| message.topics.iter().any(|u| t == u)) @@ -252,9 +309,58 @@ impl Gossipsub { } } - fn join(&mut self, topic: impl AsRef) {} + fn join(&mut self, topic: impl AsRef) { + let topic_hash = topic.as_ref(); + + // if we are already in the mesh, return + if let Some(_) = self.mesh.get(topic_hash) { + return; + } + + let mut peers = vec![]; + + // check if we have peers in fanout[topic] and remove them if we do + if let Some((_, peers)) = self.fanout.remove_entry(topic_hash) { + // add them to the mesh + self.mesh.insert(topic_hash.clone(), peers.clone()); + // remove the last published time + self.fanout_last_pub.remove(topic_hash); + } else { + // no peers in fanout[topic] - select mesh_n at random + let mesh_n = self.config.mesh_n; + peers = self.get_random_peers(&topic, mesh_n); + // put them in the mesh + self.mesh.insert(topic_hash.clone(), peers.clone()); + } + + for peer_id in peers { + //TODO: Send graft message + //TODO: tagPeer + + } + } fn leave(&mut self, topic: impl AsRef) {} + + /// Helper function to get a set of `n` random gossipsub peers for a topic + fn get_random_peers(&mut self, topic: impl AsRef, n: usize) -> Vec { + let topic = topic.as_ref(); + let mut gossip_peers = match self.topic_peers.get(topic) { + Some((gossip_peers, _)) => gossip_peers.clone(), + None => Vec::new(), + }; + + // if we have less than needed, return them + if gossip_peers.len() <= n { + return gossip_peers.to_vec(); + } + + // we have more peers than needed, shuffle them and return n of them + let mut rng = thread_rng(); + gossip_peers.partial_shuffle(&mut rng, n); + + return gossip_peers[..n].to_vec(); + } } impl NetworkBehaviour for Gossipsub @@ -283,18 +389,18 @@ where }); } - self.peers_topic.insert(id.clone(), SmallVec::new()); + self.peer_topics.insert(id.clone(), SmallVec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { - let was_in = self.peers_topic.remove(id); + let was_in = self.peer_topics.remove(id); debug_assert!(was_in.is_some()); } fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { // Update connected peers topics for subscription in event.subscriptions { - let mut remote_peer_topics = self.peers_topic + let mut remote_peer_topics = self.peer_topics .get_mut(&propagation_source) .expect("connected_peers is kept in sync with the peers we are connected to; we are guaranteed to only receive events from connected peers; QED"); match subscription.action { @@ -348,7 +454,7 @@ where } // Propagate the message to everyone else who is subscribed to any of the topics. - for (peer_id, subscr_topics) in self.peers_topic.iter() { + for (peer_id, subscr_topics) in self.peer_topics.iter() { if peer_id == &propagation_source { continue; } @@ -421,3 +527,42 @@ pub enum GossipsubEvent { topic: TopicHash, }, } + +#[cfg(test)] +mod tests { + use super::*; + use libp2p_floodsub::TopicBuilder; + + #[test] + /// Test Gossipsub.get_random_peers() function + fn test_get_random_peers() { + // generate a default GossipsubConfig + let gs_config = GossipsubConfig::default(); + // create a gossipsub struct + let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); + + // create a topic and fill it with some peers + let topic = TopicBuilder::new("Test").build(); + let topic_hash = topic.hash().clone(); + let mut peers = vec![]; + for _ in 0..20 { + peers.push(PeerId::random()) + } + + gs.topic_peers.insert(topic_hash, (peers.clone(), vec![])); + + println!("Peers: {:?}", peers); + + let random_peers = gs.get_random_peers(&topic, 5); + assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); + let random_peers = gs.get_random_peers(&topic, 30); + assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); + assert!(random_peers == peers, "Expected no shuffling"); + let random_peers = gs.get_random_peers(&topic, 20); + assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); + assert!(random_peers == peers, "Expected no shuffling"); + let random_peers = gs.get_random_peers(&topic, 0); + assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); + } + +} From bcef5ff76ffc60ab95946afcb8ca85495b1813a0 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 17 Jan 2019 11:35:52 +1100 Subject: [PATCH 04/91] Implements gossipsub leave() --- protocols/gossipsub/src/layer.rs | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 2dd52c1fd43..4ea56c686dd 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -309,11 +309,12 @@ impl Gossipsub { } } + /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT msgs fn join(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); // if we are already in the mesh, return - if let Some(_) = self.mesh.get(topic_hash) { + if self.mesh.contains_key(topic_hash) { return; } @@ -340,7 +341,19 @@ impl Gossipsub { } } - fn leave(&mut self, topic: impl AsRef) {} + /// Gossipsub LEAVE(topic) - Notifies mesh[topic] peers with PRUNE msgs + fn leave(&mut self, topic: impl AsRef) { + let topic_hash = topic.as_ref(); + + // if our mesh contains the topic, send prune to peers and delete it from the mesh + if let Some((_, peers)) = self.mesh.remove_entry(topic_hash) { + for peer_id in peers { + // prune peers + //TODO: Send prune message + //TODO: untag Peer + } + } + } /// Helper function to get a set of `n` random gossipsub peers for a topic fn get_random_peers(&mut self, topic: impl AsRef, n: usize) -> Vec { From fb0160f6284e28d3a978182960fba0eeab28cd71 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 17 Jan 2019 13:11:55 +1100 Subject: [PATCH 05/91] Update publishMany to incorporate gossipsub mesh and fanout logic --- protocols/gossipsub/src/layer.rs | 97 +++++++++++++++++++++----------- 1 file changed, 63 insertions(+), 34 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 4ea56c686dd..73f39571dd8 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -249,17 +249,12 @@ impl Gossipsub { true } - //TODO: Update publish for gossipsub /// Publishes a message to the network. - /// - /// > **Note**: Doesn't do anything if we're not subscribed to the topic. pub fn publish(&mut self, topic: impl Into, data: impl Into>) { self.publish_many(iter::once(topic), data) } /// Publishes a message with multiple topics to the network. - /// - /// > **Note**: Doesn't do anything if we're not subscribed to any of the topics. pub fn publish_many( &mut self, topic: impl IntoIterator>, @@ -275,29 +270,64 @@ impl Gossipsub { topics: topic.into_iter().map(|t| t.into().clone()).collect(), }; - // If we are not subscribed to the topic, forward to fanout peers - // TODO: Can check mesh - if !self - .subscribed_topics - .iter() - .any(|t| message.topics.iter().any(|u| t.hash() == u)) - { - //TODO: Send to fanout peers if exist - add fanout logic - // loop through topics etc - return; + // build a list of peers to forward the message to + let mut recipient_peers: Vec = vec![]; + + for t in message.topics.iter() { + // floodsub peers in the topic - add them to recipient_peers + if let Some((_, floodsub_peers)) = self.topic_peers.get(t) { + for peer_id in floodsub_peers { + if !recipient_peers.contains(peer_id) { + recipient_peers.push(peer_id.clone()); + } + } + } + + // gossipsub peers in the mesh + match self.mesh.get(t) { + // we are in the mesh, add the gossip peers to recipient_peers + Some(gossip_peers) => { + for peer_id in gossip_peers { + if !recipient_peers.contains(peer_id) { + recipient_peers.push(peer_id.clone()); + } + } + } + // not in the mesh, use fanout peers + None => { + if self.fanout.contains_key(t) { + // we have fanout peers. Add them to recipient_peers + if let Some(fanout_peers) = self.fanout.get(t) { + for peer_id in fanout_peers { + if !recipient_peers.contains(peer_id) { + recipient_peers.push(peer_id.clone()); + } + } + } + } else { + // TODO: Ensure fanout key never contains an empty set + // we have no fanout peers, select mesh_n of them and add them to the fanout + let mesh_n = self.config.mesh_n; + let new_peers = self.get_random_peers(t, mesh_n); + // add the new peers to the fanout and recipient peers + self.fanout.insert(t.clone(), new_peers.clone()); + for peer_id in new_peers { + if !recipient_peers.contains(&peer_id) { + recipient_peers.push(peer_id.clone()); + } + } + } + // we are publishing to fanout peers - update the time we published + self.fanout_last_pub.insert(t.clone(), Instant::now()); + } + } } + // add published message to our received cache self.received.add(&message); // Send to peers we know are subscribed to the topic. - for (peer_id, sub_topic) in self.peer_topics.iter() { - if !sub_topic - .iter() - .any(|t| message.topics.iter().any(|u| t == u)) - { - continue; - } - + for peer_id in recipient_peers { println!("peers subscribed? {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), @@ -329,7 +359,7 @@ impl Gossipsub { } else { // no peers in fanout[topic] - select mesh_n at random let mesh_n = self.config.mesh_n; - peers = self.get_random_peers(&topic, mesh_n); + peers = self.get_random_peers(topic_hash, mesh_n); // put them in the mesh self.mesh.insert(topic_hash.clone(), peers.clone()); } @@ -356,9 +386,8 @@ impl Gossipsub { } /// Helper function to get a set of `n` random gossipsub peers for a topic - fn get_random_peers(&mut self, topic: impl AsRef, n: usize) -> Vec { - let topic = topic.as_ref(); - let mut gossip_peers = match self.topic_peers.get(topic) { + fn get_random_peers(&self, topic_hash: &TopicHash, n: usize) -> Vec { + let mut gossip_peers = match self.topic_peers.get(topic_hash) { Some((gossip_peers, _)) => gossip_peers.clone(), None => Vec::new(), }; @@ -555,26 +584,26 @@ mod tests { let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); // create a topic and fill it with some peers - let topic = TopicBuilder::new("Test").build(); - let topic_hash = topic.hash().clone(); + let topic_hash = TopicBuilder::new("Test").build().hash().clone(); let mut peers = vec![]; for _ in 0..20 { peers.push(PeerId::random()) } - gs.topic_peers.insert(topic_hash, (peers.clone(), vec![])); + gs.topic_peers + .insert(topic_hash.clone(), (peers.clone(), vec![])); println!("Peers: {:?}", peers); - let random_peers = gs.get_random_peers(&topic, 5); + let random_peers = gs.get_random_peers(&topic_hash, 5); assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); - let random_peers = gs.get_random_peers(&topic, 30); + let random_peers = gs.get_random_peers(&topic_hash, 30); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic, 20); + let random_peers = gs.get_random_peers(&topic_hash, 20); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic, 0); + let random_peers = gs.get_random_peers(&topic_hash, 0); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); } From e0933ddf059a660fd93327c80fb7aef647e2cf4d Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 17 Jan 2019 16:13:55 +1100 Subject: [PATCH 06/91] Use the gossipsub mesh for determining peer subscription --- protocols/gossipsub/src/layer.rs | 26 ++++++++------------------ 1 file changed, 8 insertions(+), 18 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 73f39571dd8..1d93eebfaeb 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -179,16 +179,12 @@ impl Gossipsub { /// /// Returns true if the subscription worked. Returns false if we were already subscribed. pub fn subscribe(&mut self, topic: Topic) -> bool { - // TODO: Can simply check if topic is in the mesh - if self - .subscribed_topics - .iter() - .any(|t| t.hash() == topic.hash()) - { + if self.mesh.get(&topic.hash()).is_some() { return false; } // send subscription request to all floodsub and gossipsub peers + // TODO: Consolidate hashmap of peers for peer in self.peer_topics.keys() { self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), @@ -202,9 +198,8 @@ impl Gossipsub { }); } - self.subscribed_topics.push(topic.clone()); - // call JOIN(topic) + // this will add new peers to the mesh for the topic self.join(topic); true @@ -217,17 +212,11 @@ impl Gossipsub { /// Returns true if we were subscribed to this topic. pub fn unsubscribe(&mut self, topic: impl AsRef) -> bool { let topic_hash = topic.as_ref(); - // TODO: Check the mesh if we are subscribed - let pos = match self - .subscribed_topics - .iter() - .position(|t| t.hash() == topic_hash) - { - Some(pos) => pos, - None => return false, - }; - self.subscribed_topics.remove(pos); + if self.mesh.get(topic_hash).is_none() { + // we are not subscribed + return false; + } // announce to all floodsub and gossipsub peers for peer in self.peer_topics.keys() { @@ -244,6 +233,7 @@ impl Gossipsub { } // call LEAVE(topic) + // this will remove the topic from the mesh self.leave(&topic); true From 6de264e9a74907981eaa888a447d631686fcf32f Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 17 Jan 2019 17:24:20 +1100 Subject: [PATCH 07/91] Remove subscribed_topics field from the Gossipsub struct --- protocols/gossipsub/src/layer.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 1d93eebfaeb..d1e4f60c327 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -151,7 +151,6 @@ pub struct Gossipsub { // we don't dispatch the same message twice if we receive it twice on the network. received: CuckooFilter, - subscribed_topics: SmallVec<[Topic; 16]>, /// Marker to pin the generics. marker: PhantomData, } @@ -170,7 +169,6 @@ impl Gossipsub { fanout_last_pub: HashMap::new(), mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), received: CuckooFilter::new(), - subscribed_topics: SmallVec::new(), marker: PhantomData, } } @@ -408,23 +406,26 @@ where fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { // We need to send our subscriptions to the newly-connected node. - for topic in self.subscribed_topics.iter() { + for topic in self.mesh.keys() { + //TODO: Build a list of subscriptions self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: id.clone(), event: GossipsubRpc { messages: Vec::new(), subscriptions: vec![GossipsubSubscription { - topic: topic.hash().clone(), + topic: topic.clone(), action: GossipsubSubscriptionAction::Subscribe, }], }, }); } + // TODO: Handle the peer addition self.peer_topics.insert(id.clone(), SmallVec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { + // TODO: Handle peer diconnection let was_in = self.peer_topics.remove(id); debug_assert!(was_in.is_some()); } @@ -440,6 +441,7 @@ where if !remote_peer_topics.contains(&subscription.topic) { remote_peer_topics.push(subscription.topic.clone()); } + // generates a subscription event to be polled self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Subscribed { peer_id: propagation_source.clone(), @@ -467,6 +469,7 @@ where // List of messages we're going to propagate on the network. let mut rpcs_to_dispatch: Vec<(PeerId, GossipsubRpc)> = Vec::new(); + //TODO: Update for gossipsub for message in event.messages { // Use `self.received` to skip the messages that we have already received in the past. // Note that this can be a false positive. @@ -475,6 +478,7 @@ where } // Add the message to be dispatched to the user. + /* if self .subscribed_topics .iter() @@ -484,7 +488,9 @@ where self.events .push_back(NetworkBehaviourAction::GenerateEvent(event)); } + */ + //TODO: Update for gossipsub // Propagate the message to everyone else who is subscribed to any of the topics. for (peer_id, subscr_topics) in self.peer_topics.iter() { if peer_id == &propagation_source { From e3c3d0fbcd01728ffa94fc8a5542b1623c9eb47a Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 17 Jan 2019 22:02:17 +1100 Subject: [PATCH 08/91] Rename gossipsubconfig to ProtocolConfig --- protocols/gossipsub/src/handler.rs | 4 ++-- protocols/gossipsub/src/protocol.rs | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index dfdbec41628..b183967cdcd 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::protocol::{GossipsubCodec, GossipsubConfig, GossipsubRpc}; +use crate::protocol::{GossipsubCodec, GossipsubRpc, ProtocolConfig}; use futures::prelude::*; use libp2p_core::{ protocols_handler::ProtocolsHandlerUpgrErr, @@ -40,7 +40,7 @@ where TSubstream: AsyncRead + AsyncWrite, { /// Configuration for the gossipsub protocol. - config: GossipsubConfig, + config: ProtocolConfig, /// If true, we are trying to shut down the existing gossipsub substream and should refuse any /// incoming connection. diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 8f859a95bf4..c5ca13c0a57 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -31,17 +31,17 @@ use unsigned_varint::codec; /// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. #[derive(Debug, Clone)] -pub struct GossipsubConfig {} +pub struct ProtocolConfig {} -impl GossipsubConfig { - /// Builds a new `GossipsubConfig`. +impl ProtocolConfig { + /// Builds a new `ProtocolConfig`. #[inline] - pub fn new() -> GossipsubConfig { - GossipsubConfig {} + pub fn new() -> ProtocolConfig { + ProtocolConfig {} } } -impl UpgradeInfo for GossipsubConfig { +impl UpgradeInfo for ProtocolConfig { type Info = &'static [u8]; type InfoIter = iter::Once; @@ -51,7 +51,7 @@ impl UpgradeInfo for GossipsubConfig { } } -impl InboundUpgrade for GossipsubConfig +impl InboundUpgrade for ProtocolConfig where TSocket: AsyncRead + AsyncWrite, { @@ -70,7 +70,7 @@ where } } -impl OutboundUpgrade for GossipsubConfig +impl OutboundUpgrade for ProtocolConfig where TSocket: AsyncRead + AsyncWrite, { From 47dbb6926935e96e27b265311a808d9db9449f3a Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 18 Jan 2019 00:15:09 +1100 Subject: [PATCH 09/91] Implement the gossipsub control messages into the Codec's Encode/Decode and modifies GossipsubRpc --- protocols/gossipsub/rpc.proto | 2 +- protocols/gossipsub/src/handler.rs | 6 +- protocols/gossipsub/src/layer.rs | 8 +- protocols/gossipsub/src/protocol.rs | 154 ++++++++++++++++++++++++++-- 4 files changed, 155 insertions(+), 15 deletions(-) diff --git a/protocols/gossipsub/rpc.proto b/protocols/gossipsub/rpc.proto index 65127cd688c..2e65c1d8460 100644 --- a/protocols/gossipsub/rpc.proto +++ b/protocols/gossipsub/rpc.proto @@ -1,6 +1,6 @@ syntax = "proto2"; -package floodsub.pb; +package gossipsub.pb; message RPC { repeated SubOpts subscriptions = 1; diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index b183967cdcd..d84802fdf3d 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -91,7 +91,7 @@ where /// Builds a new `GossipsubHandler`. pub fn new() -> Self { GossipsubHandler { - config: GossipsubConfig::new(), + config: ProtocolConfig::new(), shutting_down: false, substreams: Vec::new(), send_queue: SmallVec::new(), @@ -107,8 +107,8 @@ where type OutEvent = GossipsubRpc; type Error = io::Error; type Substream = TSubstream; - type InboundProtocol = GossipsubConfig; - type OutboundProtocol = GossipsubConfig; + type InboundProtocol = ProtocolConfig; + type OutboundProtocol = ProtocolConfig; type OutboundOpenInfo = GossipsubRpc; #[inline] diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index d1e4f60c327..1b485355c04 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -28,7 +28,8 @@ use libp2p_core::{protocols_handler::ProtocolsHandler, PeerId}; use libp2p_floodsub::{Topic, TopicHash}; use mcache::MessageCache; use protocol::{ - GossipsubMessage, GossipsubRpc, GossipsubSubscription, GossipsubSubscriptionAction, + GossipsubControl, GossipsubMessage, GossipsubRpc, GossipsubSubscription, + GossipsubSubscriptionAction, }; use rand; use rand::{seq::SliceRandom, thread_rng}; @@ -192,6 +193,7 @@ impl Gossipsub { topic: topic.hash().clone(), action: GossipsubSubscriptionAction::Subscribe, }], + control_msg: GossipsubControl::new(), }, }); } @@ -226,6 +228,7 @@ impl Gossipsub { topic: topic_hash.clone(), action: GossipsubSubscriptionAction::Unsubscribe, }], + control_msg: GossipsubControl::new(), }, }); } @@ -322,6 +325,7 @@ impl Gossipsub { event: GossipsubRpc { subscriptions: Vec::new(), messages: vec![message.clone()], + control_msg: GossipsubControl::new(), }, }); } @@ -416,6 +420,7 @@ where topic: topic.clone(), action: GossipsubSubscriptionAction::Subscribe, }], + control_msg: GossipsubControl::new(), }, }); } @@ -512,6 +517,7 @@ where GossipsubRpc { subscriptions: Vec::new(), messages: vec![message.clone()], + control_msg: GossipsubControl::new(), }, )); } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index c5ca13c0a57..e1f0e21bdd4 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -124,6 +124,44 @@ impl Encoder for GossipsubCodec { proto.mut_subscriptions().push(subscription); } + // gossipsub control messages + let mut control_msg = rpc_proto::ControlMessage::new(); + + // collect all ihave messages + for ihave in item.control_msg.ihave_msgs { + let mut rpc_ihave = rpc_proto::ControlIHave::new(); + rpc_ihave.set_topicID(ihave.topic.into_string()); + for msg_id in ihave.message_ids { + rpc_ihave.mut_messageIDs().push(msg_id); + } + control_msg.mut_ihave().push(rpc_ihave); + } + + // collect all iwant messages + for iwant in item.control_msg.iwant_msgs { + let mut rpc_iwant = rpc_proto::ControlIWant::new(); + for msg_id in iwant.message_ids { + rpc_iwant.mut_messageIDs().push(msg_id); + } + control_msg.mut_iwant().push(rpc_iwant); + } + + // collect all graft messages + for graft in item.control_msg.graft_msgs { + let mut rpc_graft = rpc_proto::ControlGraft::new(); + rpc_graft.set_topicID(graft.topic.into_string()); + control_msg.mut_graft().push(rpc_graft); + } + + // collect all prune messages + for prune in item.control_msg.prune_msgs { + let mut rpc_prune = rpc_proto::ControlPrune::new(); + rpc_prune.set_topicID(prune.topic.into_string()); + control_msg.mut_prune().push(rpc_prune); + } + + proto.set_control(control_msg); + let msg_size = proto.compute_size(); // Reserve enough space for the data and the length. The length has a maximum of 32 bits, // which means that 5 bytes is enough for the variable-length integer. @@ -167,6 +205,57 @@ impl Decoder for GossipsubCodec { }); } + // Collect the gossipsub control messages + let ihave_msgs = rpc + .take_control() + .take_ihave() + .into_iter() + .map(|mut ihave| { + IHave { + topic: TopicHash::from_raw(ihave.take_topicID()), + // TODO: Potentially format the message ids better + message_ids: ihave.take_messageIDs().into_vec(), + } + }) + .collect(); + + let iwant_msgs = rpc + .take_control() + .take_iwant() + .into_iter() + .map(|mut iwant| { + IWant { + // TODO: Potentially format the message ids better + message_ids: iwant.take_messageIDs().into_vec(), + } + }) + .collect(); + + let graft_msgs = rpc + .take_control() + .take_graft() + .into_iter() + .map(|mut graft| Graft { + topic: TopicHash::from_raw(graft.take_topicID()), + }) + .collect(); + + let prune_msgs = rpc + .take_control() + .take_prune() + .into_iter() + .map(|mut prune| Prune { + topic: TopicHash::from_raw(prune.take_topicID()), + }) + .collect(); + + let control_msg = GossipsubControl { + ihave_msgs, + iwant_msgs, + graft_msgs, + prune_msgs, + }; + Ok(Some(GossipsubRpc { messages, subscriptions: rpc @@ -181,6 +270,7 @@ impl Decoder for GossipsubCodec { topic: TopicHash::from_raw(sub.take_topicid()), }) .collect(), + control_msg, })) } } @@ -192,6 +282,8 @@ pub struct GossipsubRpc { pub messages: Vec, /// List of subscriptions. pub subscriptions: Vec, + /// Gossipsub control message. + pub control_msg: GossipsubControl, } /// A message received by the gossipsub system. @@ -228,14 +320,56 @@ pub enum GossipsubSubscriptionAction { Subscribe, /// The remote wants to unsubscribe from the given topic. Unsubscribe, - /* - /// The remote has data - iHave, - /// iWant control action - iWant, - /// Graft control action - graft, - /// Prune control action - prune, - */ +} + +/// A Control message received by the gossipsub system. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct GossipsubControl { + /// List of IHave control messages + pub ihave_msgs: Vec, + /// List of IWant control messages + pub iwant_msgs: Vec, + /// List of Graft control messages + pub graft_msgs: Vec, + /// List of Prune control messages + pub prune_msgs: Vec, +} + +impl GossipsubControl { + /// Creates an empty `GossipsubControl` + pub fn new() -> Self { + GossipsubControl { + ihave_msgs: Vec::new(), + iwant_msgs: Vec::new(), + graft_msgs: Vec::new(), + prune_msgs: Vec::new(), + } + } +} + +/// The node broadcasts known messages, IHave. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct IHave { + /// The topic of the message. + topic: TopicHash, + /// A list of known message ids (peer_id + sequence _number) as a string. + message_ids: Vec, +} +/// The node requests specific message ids (peer_id + sequence _number), IWant. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct IWant { + message_ids: Vec, +} + +/// The node has been added to the mesh, Graft. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct Graft { + /// The mesh topic the peer should be added to. + topic: TopicHash, +} +/// The node has left the topic and removed from the mesh, Prune. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct Prune { + /// The mesh topic the peer should be removed from. + topic: TopicHash, } From e05f363f726b813e0f3346e9001913698b807ce9 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 21 Jan 2019 11:59:03 +1100 Subject: [PATCH 10/91] Modify GossipsubActions to enums for succinctness. --- protocols/gossipsub/src/handler.rs | 2 + protocols/gossipsub/src/protocol.rs | 169 ++++++++++++---------------- 2 files changed, 71 insertions(+), 100 deletions(-) diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index d84802fdf3d..d466ae8c83c 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -18,6 +18,8 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +//TODO: Remove this handler and replace with OneShotHandler once PR #868 is merged + use crate::protocol::{GossipsubCodec, GossipsubRpc, ProtocolConfig}; use futures::prelude::*; use libp2p_core::{ diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index e1f0e21bdd4..96eeaf222dd 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -127,37 +127,35 @@ impl Encoder for GossipsubCodec { // gossipsub control messages let mut control_msg = rpc_proto::ControlMessage::new(); - // collect all ihave messages - for ihave in item.control_msg.ihave_msgs { - let mut rpc_ihave = rpc_proto::ControlIHave::new(); - rpc_ihave.set_topicID(ihave.topic.into_string()); - for msg_id in ihave.message_ids { - rpc_ihave.mut_messageIDs().push(msg_id); - } - control_msg.mut_ihave().push(rpc_ihave); - } - - // collect all iwant messages - for iwant in item.control_msg.iwant_msgs { - let mut rpc_iwant = rpc_proto::ControlIWant::new(); - for msg_id in iwant.message_ids { - rpc_iwant.mut_messageIDs().push(msg_id); + for action in item.control_msgs { + match action { + // collect all ihave messages + GossipsubControlAction::IHave { topic, message_ids } => { + let mut rpc_ihave = rpc_proto::ControlIHave::new(); + rpc_ihave.set_topicID(topic.into_string()); + for msg_id in message_ids { + rpc_ihave.mut_messageIDs().push(msg_id); + } + control_msg.mut_ihave().push(rpc_ihave); + } + GossipsubControlAction::IWant { message_ids } => { + let mut rpc_iwant = rpc_proto::ControlIWant::new(); + for msg_id in message_ids { + rpc_iwant.mut_messageIDs().push(msg_id); + } + control_msg.mut_iwant().push(rpc_iwant); + } + GossipsubControlAction::Graft { topic } => { + let mut rpc_graft = rpc_proto::ControlGraft::new(); + rpc_graft.set_topicID(topic.into_string()); + control_msg.mut_graft().push(rpc_graft); + } + GossipsubControlAction::Prune { topic } => { + let mut rpc_prune = rpc_proto::ControlPrune::new(); + rpc_prune.set_topicID(topic.into_string()); + control_msg.mut_prune().push(rpc_prune); + } } - control_msg.mut_iwant().push(rpc_iwant); - } - - // collect all graft messages - for graft in item.control_msg.graft_msgs { - let mut rpc_graft = rpc_proto::ControlGraft::new(); - rpc_graft.set_topicID(graft.topic.into_string()); - control_msg.mut_graft().push(rpc_graft); - } - - // collect all prune messages - for prune in item.control_msg.prune_msgs { - let mut rpc_prune = rpc_proto::ControlPrune::new(); - rpc_prune.set_topicID(prune.topic.into_string()); - control_msg.mut_prune().push(rpc_prune); } proto.set_control(control_msg); @@ -205,13 +203,14 @@ impl Decoder for GossipsubCodec { }); } + let mut rpc_control = rpc.take_control(); + let mut control_msgs = vec![]; // Collect the gossipsub control messages - let ihave_msgs = rpc - .take_control() + let ihave_msgs: Vec = rpc_control .take_ihave() .into_iter() .map(|mut ihave| { - IHave { + GossipsubControlAction::IHave { topic: TopicHash::from_raw(ihave.take_topicID()), // TODO: Potentially format the message ids better message_ids: ihave.take_messageIDs().into_vec(), @@ -219,42 +218,37 @@ impl Decoder for GossipsubCodec { }) .collect(); - let iwant_msgs = rpc - .take_control() + let iwant_msgs: Vec = rpc_control .take_iwant() .into_iter() .map(|mut iwant| { - IWant { + GossipsubControlAction::IWant { // TODO: Potentially format the message ids better message_ids: iwant.take_messageIDs().into_vec(), } }) .collect(); - let graft_msgs = rpc - .take_control() + let graft_msgs: Vec = rpc_control .take_graft() .into_iter() - .map(|mut graft| Graft { + .map(|mut graft| GossipsubControlAction::Graft { topic: TopicHash::from_raw(graft.take_topicID()), }) .collect(); - let prune_msgs = rpc - .take_control() + let prune_msgs: Vec = rpc_control .take_prune() .into_iter() - .map(|mut prune| Prune { + .map(|mut prune| GossipsubControlAction::Prune { topic: TopicHash::from_raw(prune.take_topicID()), }) .collect(); - let control_msg = GossipsubControl { - ihave_msgs, - iwant_msgs, - graft_msgs, - prune_msgs, - }; + control_msgs.extend(ihave_msgs); + control_msgs.extend(iwant_msgs); + control_msgs.extend(graft_msgs); + control_msgs.extend(prune_msgs); Ok(Some(GossipsubRpc { messages, @@ -270,7 +264,7 @@ impl Decoder for GossipsubCodec { topic: TopicHash::from_raw(sub.take_topicid()), }) .collect(), - control_msg, + control_msgs, })) } } @@ -282,8 +276,8 @@ pub struct GossipsubRpc { pub messages: Vec, /// List of subscriptions. pub subscriptions: Vec, - /// Gossipsub control message. - pub control_msg: GossipsubControl, + /// List of Gossipsub control messages. + pub control_msgs: Vec, } /// A message received by the gossipsub system. @@ -295,7 +289,7 @@ pub struct GossipsubMessage { /// Content of the message. Its meaning is out of scope of this library. pub data: Vec, - /// An incrementing sequence number. + /// A random sequence number. pub sequence_number: Vec, /// List of topics this message belongs to. @@ -324,52 +318,27 @@ pub enum GossipsubSubscriptionAction { /// A Control message received by the gossipsub system. #[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct GossipsubControl { - /// List of IHave control messages - pub ihave_msgs: Vec, - /// List of IWant control messages - pub iwant_msgs: Vec, - /// List of Graft control messages - pub graft_msgs: Vec, - /// List of Prune control messages - pub prune_msgs: Vec, -} - -impl GossipsubControl { - /// Creates an empty `GossipsubControl` - pub fn new() -> Self { - GossipsubControl { - ihave_msgs: Vec::new(), - iwant_msgs: Vec::new(), - graft_msgs: Vec::new(), - prune_msgs: Vec::new(), - } - } -} - -/// The node broadcasts known messages, IHave. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct IHave { - /// The topic of the message. - topic: TopicHash, - /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, -} -/// The node requests specific message ids (peer_id + sequence _number), IWant. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct IWant { - message_ids: Vec, -} - -/// The node has been added to the mesh, Graft. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct Graft { - /// The mesh topic the peer should be added to. - topic: TopicHash, -} -/// The node has left the topic and removed from the mesh, Prune. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct Prune { - /// The mesh topic the peer should be removed from. - topic: TopicHash, +pub enum GossipsubControlAction { + /// Node broadcasts known messages per topic - IHave control message. + IHave { + /// The topic of the message. + topic: TopicHash, + /// A list of known message ids (peer_id + sequence _number) as a string. + message_ids: Vec, + }, + /// The node requests specific message ids (peer_id + sequence _number) - IWant control message. + IWant { + /// A list of known message ids (peer_id + sequence _number) as a string. + message_ids: Vec, + }, + /// The node has been added to the mesh - Graft control message. + Graft { + /// The mesh topic the peer should be added to. + topic: TopicHash, + }, + /// The node has been removed from the mesh - Prune control message. + Prune { + /// The mesh topic the peer should be removed from. + topic: TopicHash, + }, } From 53c364bb60533d065083008cbbacdb95845fa06d Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 21 Jan 2019 15:37:54 +1100 Subject: [PATCH 11/91] Modify the memcache to store Gossipsub messages --- protocols/gossipsub/src/mcache.rs | 140 ++++++++++-------------------- 1 file changed, 48 insertions(+), 92 deletions(-) diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 67523b50bc3..3e9d6d751b3 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -1,19 +1,20 @@ extern crate fnv; -use super::rpc_proto::Message; use fnv::FnvHashMap; +use libp2p_floodsub::TopicHash; +use protocol::GossipsubMessage; /// CacheEntry stored in the history #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct CacheEntry { mid: String, - topics: Vec, + topics: Vec, } /// MessageCache struct holding history of messages #[derive(Debug, Clone, PartialEq)] pub struct MessageCache { - msgs: FnvHashMap, + msgs: FnvHashMap, history: Vec>, gossip: usize, } @@ -29,26 +30,25 @@ impl MessageCache { } /// Put a message into the memory cache - pub fn put(&mut self, msg: Message) -> Result<(), MsgError> { - let message_id = msg_id(&msg)?; + pub fn put(&mut self, msg: GossipsubMessage) { + let message_id = msg.msg_id(); let cache_entry = CacheEntry { mid: message_id.clone(), - topics: msg.get_topicIDs().to_vec(), + topics: msg.topics.clone(), }; self.msgs.insert(message_id, msg); self.history[0].push(cache_entry); - Ok(()) } /// Get a message with `message_id` - pub fn get(&self, message_id: &str) -> Option<&Message> { + pub fn get(&self, message_id: &str) -> Option<&GossipsubMessage> { self.msgs.get(message_id) } /// Get a list of GossipIds for a given topic - pub fn get_gossip_ids(&self, topic: &str) -> Vec { + pub fn get_gossip_ids(&self, topic: &TopicHash) -> Vec { self.history[..self.gossip] .iter() .fold(vec![], |mut current_entries, entries| { @@ -56,7 +56,7 @@ impl MessageCache { let mut found_entries: Vec = entries .iter() .filter_map(|entry| { - if entry.topics.iter().any(|t| *t == topic) { + if entry.topics.iter().any(|t| t == topic) { Some(entry.mid.clone()) } else { None @@ -92,42 +92,24 @@ impl MessageCache { } } -// Functions to be refactored later -/// Gets a unique message id. -/// Returns an error if the message has non-utf from or seqno values -fn msg_id(pmsg: &Message) -> Result { - let from = String::from_utf8(pmsg.get_from().to_vec()).or(Err(MsgError::InvalidMessage))?; - let seqno = String::from_utf8(pmsg.get_seqno().to_vec()).or(Err(MsgError::InvalidMessage))?; - Ok(from + &seqno) -} - -#[derive(Debug)] -pub enum MsgError { - InvalidMessage, -} - #[cfg(test)] mod tests { - use super::super::protobuf; use super::*; + use libp2p_core::PeerId; + use libp2p_floodsub::{TopicBuilder, TopicHash}; - fn gen_testm(x: usize, topics: Vec) -> Message { + fn gen_testm(x: usize, topics: Vec) -> GossipsubMessage { let u8x: u8 = x as u8; - let from: Vec = vec![u8x]; + let source = PeerId::random(); let data: Vec = vec![u8x]; - let seqno: Vec = vec![u8x]; - let mut tids = protobuf::RepeatedField::new(); - - for topic in topics { - tids.push(topic); - } - - let mut m = Message::new(); - m.set_from(from.clone()); - m.set_data(data); - m.set_seqno(seqno); - m.set_topicIDs(tids); + let sequence_number: Vec = vec![u8x]; + let m = GossipsubMessage { + source, + data, + sequence_number, + topics, + }; m } @@ -145,24 +127,16 @@ mod tests { fn test_put_get_one() { let mut mc = MessageCache::new(10, 15); - let m = gen_testm( - 10 as usize, - vec![String::from("hello"), String::from("world")], - ); + let topic1_hash = TopicBuilder::new("topic1").build().hash().clone(); + let topic2_hash = TopicBuilder::new("topic2").build().hash().clone(); - let res = mc.put(m.clone()); - assert_eq!(res.is_err(), false); - assert_eq!(res.ok(), Some(())); + let m = gen_testm(10 as usize, vec![topic1_hash, topic2_hash]); - assert!(mc.history[0].len() == 1); + mc.put(m.clone()); - let mid = msg_id(&m.clone()); - assert_eq!(mid.is_err(), false); + assert!(mc.history[0].len() == 1); - let fetched = match mid.ok() { - Some(id) => mc.get(&id), - _ => None, - }; + let fetched = mc.get(&m.msg_id()); assert_eq!(fetched.is_none(), false); assert_eq!(fetched.is_some(), true); @@ -179,22 +153,16 @@ mod tests { fn test_get_wrong() { let mut mc = MessageCache::new(10, 15); - // Build the message - let m = gen_testm( - 1 as usize, - vec![String::from("hello"), String::from("world")], - ); + let topic1_hash = TopicBuilder::new("topic1").build().hash().clone(); + let topic2_hash = TopicBuilder::new("topic2").build().hash().clone(); - let res = mc.put(m.clone()); - assert_eq!(res.is_err(), false); - assert_eq!(res.ok(), Some(())); + let m = gen_testm(10 as usize, vec![topic1_hash, topic2_hash]); - let mid = msg_id(&m.clone()); - assert_eq!(mid.is_err(), false); + mc.put(m.clone()); // Try to get an incorrect ID - let wrong_string = String::from("wrongid"); - let fetched = mc.get(&wrong_string); + let wrong_msg_id = String::from("wrongid"); + let fetched = mc.get(&wrong_msg_id); assert_eq!(fetched.is_none(), true); } @@ -216,16 +184,9 @@ mod tests { // Build the message let m = gen_testm(1 as usize, vec![]); + mc.put(m.clone()); - let res = mc.put(m.clone()); - assert_eq!(res.is_err(), false); - assert_eq!(res.ok(), Some(())); - - let mid = msg_id(&m.clone()); - let fetched = match mid.ok() { - Some(id) => mc.get(&id), - _ => None, - }; + let fetched = mc.get(&m.msg_id()); // Make sure it is the same fetched message match fetched { @@ -239,14 +200,12 @@ mod tests { fn test_shift() { let mut mc = MessageCache::new(1, 5); + let topic1_hash = TopicBuilder::new("topic1").build().hash().clone(); + let topic2_hash = TopicBuilder::new("topic2").build().hash().clone(); // Build the message for i in 0..10 { - let m = gen_testm( - i as usize, - vec![String::from("hello"), String::from("world")], - ); - let res = mc.put(m.clone()); - assert_eq!(res.is_err(), false); + let m = gen_testm(i as usize, vec![topic1_hash.clone(), topic2_hash.clone()]); + mc.put(m.clone()); } mc.shift(); @@ -264,14 +223,12 @@ mod tests { fn test_empty_shift() { let mut mc = MessageCache::new(1, 5); + let topic1_hash = TopicBuilder::new("topic1").build().hash().clone(); + let topic2_hash = TopicBuilder::new("topic2").build().hash().clone(); // Build the message for i in 0..10 { - let m = gen_testm( - i as usize, - vec![String::from("hello"), String::from("world")], - ); - let res = mc.put(m.clone()); - assert_eq!(res.is_err(), false); + let m = gen_testm(i as usize, vec![topic1_hash.clone(), topic2_hash.clone()]); + mc.put(m.clone()); } mc.shift(); @@ -292,13 +249,12 @@ mod tests { fn test_remove_last_from_shift() { let mut mc = MessageCache::new(4, 5); + let topic1_hash = TopicBuilder::new("topic1").build().hash().clone(); + let topic2_hash = TopicBuilder::new("topic2").build().hash().clone(); + // Build the message for i in 0..10 { - let m = gen_testm( - i as usize, - vec![String::from("hello"), String::from("world")], - ); - let res = mc.put(m.clone()); - assert_eq!(res.is_err(), false); + let m = gen_testm(i as usize, vec![topic1_hash.clone(), topic2_hash.clone()]); + mc.put(m.clone()); } // Shift right until deleting messages From cb5e20b57b0ac05de653caa9e3cb036ff6343635 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 21 Jan 2019 15:44:36 +1100 Subject: [PATCH 12/91] Implement control message handling. --- protocols/gossipsub/Cargo.toml | 1 + protocols/gossipsub/src/layer.rs | 171 +++++++++++++++++++++++----- protocols/gossipsub/src/lib.rs | 1 + protocols/gossipsub/src/protocol.rs | 18 +++ 4 files changed, 165 insertions(+), 26 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index aeffd1bd267..12f8dddf5ba 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -9,6 +9,7 @@ libp2p-core = { path = "../../core" } libp2p-floodsub = { path = "../floodsub" } bs58 = "" bytes = "" +byteorder = "" cuckoofilter = "" fnv = "" futures = "" diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 1b485355c04..c698e031d37 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -28,7 +28,7 @@ use libp2p_core::{protocols_handler::ProtocolsHandler, PeerId}; use libp2p_floodsub::{Topic, TopicHash}; use mcache::MessageCache; use protocol::{ - GossipsubControl, GossipsubMessage, GossipsubRpc, GossipsubSubscription, + GossipsubControlAction, GossipsubMessage, GossipsubRpc, GossipsubSubscription, GossipsubSubscriptionAction, }; use rand; @@ -40,27 +40,27 @@ use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; // potentially rename this struct - due to clashes -/// Configuration parameters that define the performance of the gossipsub network +/// Configuration parameters that define the performance of the gossipsub network. #[derive(Clone)] pub struct GossipsubConfig { - /// Overlay network parameters - /// Number of heartbeats to keep in the memcache + /// Overlay network parameters. + /// Number of heartbeats to keep in the `memcache`. history_length: usize, - /// Number of past heartbeats to gossip about + /// Number of past heartbeats to gossip about. history_gossip: usize, - /// Target number of peers for the mesh network (D in the spec) + /// Target number of peers for the mesh network (D in the spec). mesh_n: usize, - /// Minimum number of peers in mesh network before adding more (D_lo in the spec) + /// Minimum number of peers in mesh network before adding more (D_lo in the spec). mesh_n_low: usize, - /// Maximum number of peers in mesh network before removing some (D_high in the spec) + /// Maximum number of peers in mesh network before removing some (D_high in the spec). mesh_n_high: usize, - /// Initial delay in each heartbeat + /// Initial delay in each heartbeat. heartbeat_initial_delay: Duration, - /// Time between each heartbeat + /// Time between each heartbeat. heartbeat_interval: Duration, - /// Time to live for fanout peers + /// Time to live for fanout peers. fanout_ttl: Duration, } @@ -193,7 +193,7 @@ impl Gossipsub { topic: topic.hash().clone(), action: GossipsubSubscriptionAction::Subscribe, }], - control_msg: GossipsubControl::new(), + control_msgs: Vec::new(), }, }); } @@ -228,7 +228,7 @@ impl Gossipsub { topic: topic_hash.clone(), action: GossipsubSubscriptionAction::Unsubscribe, }], - control_msg: GossipsubControl::new(), + control_msgs: Vec::new(), }, }); } @@ -257,7 +257,10 @@ impl Gossipsub { // If the sequence numbers are predictable, then an attacker could flood the network // with packets with the predetermined sequence numbers and absorb our legitimate // messages. We therefore use a random number. - sequence_number: rand::random::<[u8; 20]>().to_vec(), + // TODO: Check if the random sequence numbers causes issues with other clients. + // To be interoperable with the go-implementation this is treated as a 64-bit + // big-endian uint. + sequence_number: rand::random::<[u8; 8]>().to_vec(), topics: topic.into_iter().map(|t| t.into().clone()).collect(), }; @@ -315,7 +318,8 @@ impl Gossipsub { } // add published message to our received cache - self.received.add(&message); + // TODO: Add to memcache + self.received.add(&message.msg_id()); // Send to peers we know are subscribed to the topic. for peer_id in recipient_peers { @@ -325,13 +329,13 @@ impl Gossipsub { event: GossipsubRpc { subscriptions: Vec::new(), messages: vec![message.clone()], - control_msg: GossipsubControl::new(), + control_msgs: Vec::new(), }, }); } } - /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT msgs + /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. fn join(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); @@ -357,26 +361,143 @@ impl Gossipsub { } for peer_id in peers { - //TODO: Send graft message + // Send a GRAFT control message + println!("Graft message sent to peer: {:?}", peer_id); + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer_id.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: vec![GossipsubControlAction::Graft { + topic: topic_hash.clone(), + }], + }, + }); //TODO: tagPeer - } } - /// Gossipsub LEAVE(topic) - Notifies mesh[topic] peers with PRUNE msgs + /// Gossipsub LEAVE(topic) - Notifies mesh[topic] peers with PRUNE messages. fn leave(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); // if our mesh contains the topic, send prune to peers and delete it from the mesh if let Some((_, peers)) = self.mesh.remove_entry(topic_hash) { for peer_id in peers { - // prune peers - //TODO: Send prune message + // Send a PRUNE control message + println!("Prune message sent to peer: {:?}", peer_id); + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer_id.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: vec![GossipsubControlAction::Prune { + topic: topic_hash.clone(), + }], + }, + }); //TODO: untag Peer } } } + /// Handles an IHAVE control message. Checks our cache of messages. If the message is unknown, + /// requests it with an IWANT control message. + fn handle_ihave(&mut self, peer_id: PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { + // use a hashmap to avoid duplicates efficiently + let mut iwant_msg_ids = HashMap::new(); + + for (topic, msg_ids) in ihave_msgs { + // only process the message if we are subscribed + if !self.mesh.contains_key(&topic) { + return; // continue + } + + for msg_id in msg_ids { + if !self.received.contains(&msg_id) { + // have not seen this message, request it + iwant_msg_ids.insert(msg_id, true); + } + } + } + + if !iwant_msg_ids.is_empty() { + // Send the list of IWANT control messages + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer_id.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: vec![GossipsubControlAction::IWant { + message_ids: iwant_msg_ids.keys().map(|msg_id| msg_id.clone()).collect(), + }], + }, + }); + } + } + + /// Handles an IWANT control message. Checks our cache of messages. If the message exists it is + /// forwarded to the requesting peer. + fn handle_iwant(&mut self, peer_id: PeerId, iwant_msgs: Vec) { + // build a hashmap of available messages + let mut cached_messages = HashMap::new(); + + for msg_id in iwant_msgs { + // if we have it, add it do the cached_messages mapping + if let Some(msg) = self.mcache.get(&msg_id) { + cached_messages.insert(msg_id.clone(), msg.clone()); + } + } + + if !cached_messages.is_empty() { + // Send the messages to the peer + let message_list = cached_messages.values().map(|msg| msg.clone()).collect(); + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer_id.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: message_list, + control_msgs: Vec::new(), + }, + }); + } + } + + /// Handles a GRAFT control message. If subscribed to the topic, adds the peer to mesh, if not responds + /// with PRUNE message. + //TODO: Update this to handle multiple messages per peer. + fn handle_graft(&mut self, peer_id: PeerId, topic: &TopicHash) { + if let Some(peers) = self.mesh.get_mut(topic) { + // if we are subscribed, add peer to the mesh + println!("GRAFT: Mesh link added from {:?}", peer_id); + peers.push(peer_id); + //TODO: tagPeer + } else { + // We are not subscribed, send a PRUNE message to the peer. + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer_id.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: vec![GossipsubControlAction::Prune { + topic: topic.clone(), + }], + }, + }); + } + } + + /// Handles a PRUNE control message. Removes peer from the mesh. + //TODO: Update this to handle multiple messages per peer. + fn handle_prune(&mut self, peer_id: PeerId, topic: &TopicHash) { + if let Some(peers) = self.mesh.get_mut(topic) { + // remove the peer if it exists in the mesh + if let Some(pos) = peers.iter().position(|p| p == &peer_id) { + peers.remove(pos); + } + } + } + /// Helper function to get a set of `n` random gossipsub peers for a topic fn get_random_peers(&self, topic_hash: &TopicHash, n: usize) -> Vec { let mut gossip_peers = match self.topic_peers.get(topic_hash) { @@ -420,7 +541,7 @@ where topic: topic.clone(), action: GossipsubSubscriptionAction::Subscribe, }], - control_msg: GossipsubControl::new(), + control_msgs: Vec::new(), }, }); } @@ -517,7 +638,7 @@ where GossipsubRpc { subscriptions: Vec::new(), messages: vec![message.clone()], - control_msg: GossipsubControl::new(), + control_msgs: Vec::new(), }, )); } @@ -595,8 +716,6 @@ mod tests { gs.topic_peers .insert(topic_hash.clone(), (peers.clone(), vec![])); - println!("Peers: {:?}", peers); - let random_peers = gs.get_random_peers(&topic_hash, 5); assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); let random_peers = gs.get_random_peers(&topic_hash, 30); diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index c21de840abf..9f421e517e8 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -19,6 +19,7 @@ // DEALINGS IN THE SOFTWARE. extern crate bs58; +extern crate byteorder; extern crate bytes; extern crate cuckoofilter; extern crate fnv; diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 96eeaf222dd..4f8076f129f 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -19,6 +19,7 @@ // DEALINGS IN THE SOFTWARE. use crate::rpc_proto; +use byteorder::{BigEndian, ByteOrder}; use bytes::{BufMut, BytesMut}; use futures::future; use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; @@ -298,6 +299,23 @@ pub struct GossipsubMessage { pub topics: Vec, } +impl GossipsubMessage { + /// Converts message into a message_id. + // To be compatible with the go implementation + pub fn msg_id(&self) -> String { + let mut source_string = self.source.to_base58(); + // the sequence number is a big endian uint64 (as per go implementation) + // avoid a potential panic by setting the seqno to 0 if it is not long enough. + // TODO: Check that this doesn't introduce a vulnerability or issue + let mut seqno = 0; + if self.sequence_number.len() >= 8 { + seqno = BigEndian::read_u64(&self.sequence_number); + } + source_string.push_str(&seqno.to_string()); + source_string + } +} + /// A subscription received by the gossipsub system. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct GossipsubSubscription { From e865ed5fcb4cfd486f3157349f95cba96e9a9af3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 21 Jan 2019 16:08:23 +1100 Subject: [PATCH 13/91] Update control message handling to handle multiple messages. --- protocols/gossipsub/src/layer.rs | 55 ++++++++++++++++++++------------ 1 file changed, 34 insertions(+), 21 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index c698e031d37..4d4dc2855ef 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -463,37 +463,50 @@ impl Gossipsub { } } - /// Handles a GRAFT control message. If subscribed to the topic, adds the peer to mesh, if not responds - /// with PRUNE message. - //TODO: Update this to handle multiple messages per peer. - fn handle_graft(&mut self, peer_id: PeerId, topic: &TopicHash) { - if let Some(peers) = self.mesh.get_mut(topic) { - // if we are subscribed, add peer to the mesh - println!("GRAFT: Mesh link added from {:?}", peer_id); - peers.push(peer_id); - //TODO: tagPeer - } else { - // We are not subscribed, send a PRUNE message to the peer. + /// Handles GRAFT control messages. If subscribed to the topic, adds the peer to mesh, if not, responds + /// with PRUNE messages. + fn handle_graft(&mut self, peer_id: PeerId, topics: Vec) { + let mut to_prune_topics = HashMap::new(); + for topic in topics { + if let Some(peers) = self.mesh.get_mut(&topic) { + // if we are subscribed, add peer to the mesh + println!("GRAFT: Mesh link added from {:?}", peer_id); + peers.push(peer_id.clone()); + //TODO: tagPeer + } else { + to_prune_topics.insert(topic.clone(), ()); + } + } + + if !to_prune_topics.is_empty() { + // build the prune messages to send + let prune_messages = to_prune_topics + .keys() + .map(|topic| GossipsubControlAction::Prune { + topic: topic.clone(), + }) + .collect(); + // Send the prune messages to the peer self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), - control_msgs: vec![GossipsubControlAction::Prune { - topic: topic.clone(), - }], + control_msgs: prune_messages, }, }); } } - /// Handles a PRUNE control message. Removes peer from the mesh. - //TODO: Update this to handle multiple messages per peer. - fn handle_prune(&mut self, peer_id: PeerId, topic: &TopicHash) { - if let Some(peers) = self.mesh.get_mut(topic) { - // remove the peer if it exists in the mesh - if let Some(pos) = peers.iter().position(|p| p == &peer_id) { - peers.remove(pos); + /// Handles PRUNE control messages. Removes peer from the mesh. + fn handle_prune(&mut self, peer_id: PeerId, topics: Vec) { + for topic in topics { + if let Some(peers) = self.mesh.get_mut(&topic) { + // remove the peer if it exists in the mesh + if let Some(pos) = peers.iter().position(|p| p == &peer_id) { + peers.remove(pos); + //TODO: untagPeer + } } } } From 80673d1330aa19374730570011745b66ea92317b Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 21 Jan 2019 18:26:40 +1100 Subject: [PATCH 14/91] Handle received gossipsub messages using pre-built handlers. --- protocols/gossipsub/src/layer.rs | 241 +++++++++++++++++----------- protocols/gossipsub/src/protocol.rs | 16 +- 2 files changed, 151 insertions(+), 106 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 4d4dc2855ef..a749763b99a 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -265,6 +265,7 @@ impl Gossipsub { }; // build a list of peers to forward the message to + // TODO: Potentially use a hashmap - efficiency in checking for duplicates let mut recipient_peers: Vec = vec![]; for t in message.topics.iter() { @@ -369,7 +370,7 @@ impl Gossipsub { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: vec![GossipsubControlAction::Graft { - topic: topic_hash.clone(), + topic_hash: topic_hash.clone(), }], }, }); @@ -392,7 +393,7 @@ impl Gossipsub { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: vec![GossipsubControlAction::Prune { - topic: topic_hash.clone(), + topic_hash: topic_hash.clone(), }], }, }); @@ -403,7 +404,7 @@ impl Gossipsub { /// Handles an IHAVE control message. Checks our cache of messages. If the message is unknown, /// requests it with an IWANT control message. - fn handle_ihave(&mut self, peer_id: PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { + fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { // use a hashmap to avoid duplicates efficiently let mut iwant_msg_ids = HashMap::new(); @@ -438,7 +439,7 @@ impl Gossipsub { /// Handles an IWANT control message. Checks our cache of messages. If the message exists it is /// forwarded to the requesting peer. - fn handle_iwant(&mut self, peer_id: PeerId, iwant_msgs: Vec) { + fn handle_iwant(&mut self, peer_id: &PeerId, iwant_msgs: Vec) { // build a hashmap of available messages let mut cached_messages = HashMap::new(); @@ -465,7 +466,7 @@ impl Gossipsub { /// Handles GRAFT control messages. If subscribed to the topic, adds the peer to mesh, if not, responds /// with PRUNE messages. - fn handle_graft(&mut self, peer_id: PeerId, topics: Vec) { + fn handle_graft(&mut self, peer_id: &PeerId, topics: Vec) { let mut to_prune_topics = HashMap::new(); for topic in topics { if let Some(peers) = self.mesh.get_mut(&topic) { @@ -483,7 +484,7 @@ impl Gossipsub { let prune_messages = to_prune_topics .keys() .map(|topic| GossipsubControlAction::Prune { - topic: topic.clone(), + topic_hash: topic.clone(), }) .collect(); // Send the prune messages to the peer @@ -499,11 +500,11 @@ impl Gossipsub { } /// Handles PRUNE control messages. Removes peer from the mesh. - fn handle_prune(&mut self, peer_id: PeerId, topics: Vec) { + fn handle_prune(&mut self, peer_id: &PeerId, topics: Vec) { for topic in topics { if let Some(peers) = self.mesh.get_mut(&topic) { // remove the peer if it exists in the mesh - if let Some(pos) = peers.iter().position(|p| p == &peer_id) { + if let Some(pos) = peers.iter().position(|p| p == peer_id) { peers.remove(pos); //TODO: untagPeer } @@ -511,6 +512,101 @@ impl Gossipsub { } } + /// Handles a newly received GossipsubMessage. + /// Forwards the message to all floodsub peers and peers in the mesh. + fn handle_received_message(&mut self, msg: GossipsubMessage, propagation_source: &PeerId) { + // if we have seen this message, ignore it + // there's a 3% chance this is a false positive + // TODO: Check this has no significant emergent behaviour + if !self.received.test_and_add(&msg.msg_id()) { + return; + } + + // dispatch the message to the user + if self.mesh.keys().any(|t| msg.topics.iter().any(|u| t == u)) { + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Message(msg.clone()), + )); + } + + // peers to forward the message to + let mut recipient_peers = HashMap::new(); + + // add floodsub and mesh peers + for topic in &msg.topics { + // floodsub + if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { + for peer_id in floodsub_peers { + if peer_id != propagation_source { + recipient_peers.insert(peer_id.clone(), ()); + } + } + } + + // mesh + if let Some(mesh_peers) = self.mesh.get(&topic) { + for peer_id in mesh_peers { + if peer_id != propagation_source { + recipient_peers.insert(peer_id.clone(), ()); + } + } + } + } + + // forward the message to peers + if !recipient_peers.is_empty() { + for peer in recipient_peers.keys() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![msg.clone()], + control_msgs: Vec::new(), + }, + }); + } + } + } + + /// Handles received subscription. + fn handle_received_subscriptions( + &mut self, + subscription: GossipsubSubscription, + propagation_source: &PeerId, + ) { + let remote_peer_topics = self.peer_topics + .get_mut(&propagation_source) + .expect("connected_peers is kept in sync with the peers we are connected to; we are guaranteed to only receive events from connected peers; QED"); + match subscription.action { + GossipsubSubscriptionAction::Subscribe => { + if !remote_peer_topics.contains(&subscription.topic) { + remote_peer_topics.push(subscription.topic.clone()); + } + // generates a subscription event to be polled + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Subscribed { + peer_id: propagation_source.clone(), + topic: subscription.topic, + }, + )); + } + GossipsubSubscriptionAction::Unsubscribe => { + if let Some(pos) = remote_peer_topics + .iter() + .position(|t| t == &subscription.topic) + { + remote_peer_topics.remove(pos); + } + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Unsubscribed { + peer_id: propagation_source.clone(), + topic: subscription.topic, + }, + )); + } + } + } + /// Helper function to get a set of `n` random gossipsub peers for a topic fn get_random_peers(&self, topic_hash: &TopicHash, n: usize) -> Vec { let mut gossip_peers = match self.topic_peers.get(topic_hash) { @@ -544,125 +640,74 @@ where fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { // We need to send our subscriptions to the newly-connected node. + let mut subscriptions = vec![]; for topic in self.mesh.keys() { - //TODO: Build a list of subscriptions + subscriptions.push(GossipsubSubscription { + topic: topic.clone(), + action: GossipsubSubscriptionAction::Subscribe, + }); + } + + if !subscriptions.is_empty() { + // send our subscriptions to the peer self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: id.clone(), event: GossipsubRpc { messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic: topic.clone(), - action: GossipsubSubscriptionAction::Subscribe, - }], + subscriptions, control_msgs: Vec::new(), }, }); } - // TODO: Handle the peer addition + // TODO: Handle the peer addition - Specifically add the gossipsub and floodsub peers. self.peer_topics.insert(id.clone(), SmallVec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { - // TODO: Handle peer diconnection + // TODO: Handle peer disconnection - floodsub and gossipsub peers let was_in = self.peer_topics.remove(id); debug_assert!(was_in.is_some()); } fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { + // Handle subscriptions // Update connected peers topics for subscription in event.subscriptions { - let mut remote_peer_topics = self.peer_topics - .get_mut(&propagation_source) - .expect("connected_peers is kept in sync with the peers we are connected to; we are guaranteed to only receive events from connected peers; QED"); - match subscription.action { - GossipsubSubscriptionAction::Subscribe => { - if !remote_peer_topics.contains(&subscription.topic) { - remote_peer_topics.push(subscription.topic.clone()); - } - // generates a subscription event to be polled - self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Subscribed { - peer_id: propagation_source.clone(), - topic: subscription.topic, - }, - )); - } - GossipsubSubscriptionAction::Unsubscribe => { - if let Some(pos) = remote_peer_topics - .iter() - .position(|t| t == &subscription.topic) - { - remote_peer_topics.remove(pos); - } - self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Unsubscribed { - peer_id: propagation_source.clone(), - topic: subscription.topic, - }, - )); - } - } + self.handle_received_subscriptions(subscription, &propagation_source); } - // List of messages we're going to propagate on the network. - let mut rpcs_to_dispatch: Vec<(PeerId, GossipsubRpc)> = Vec::new(); - - //TODO: Update for gossipsub + // Handle messages for message in event.messages { - // Use `self.received` to skip the messages that we have already received in the past. - // Note that this can be a false positive. - if !self.received.test_and_add(&message) { - continue; - } - - // Add the message to be dispatched to the user. - /* - if self - .subscribed_topics - .iter() - .any(|t| message.topics.iter().any(|u| t.hash() == u)) - { - let event = GossipsubEvent::Message(message.clone()); - self.events - .push_back(NetworkBehaviourAction::GenerateEvent(event)); - } - */ + self.handle_received_message(message, &propagation_source); + } - //TODO: Update for gossipsub - // Propagate the message to everyone else who is subscribed to any of the topics. - for (peer_id, subscr_topics) in self.peer_topics.iter() { - if peer_id == &propagation_source { - continue; + // Handle control messages + // group some control messages, this minimises SendEvents (code is simplified to handle each event at a time however) + // TODO: Decide if the grouping is necessary + let mut ihave_msgs = vec![]; + let mut graft_msgs = vec![]; + let mut prune_msgs = vec![]; + for control_msg in event.control_msgs { + match control_msg { + GossipsubControlAction::IHave { topic, message_ids } => { + ihave_msgs.push((topic, message_ids)); } - - if !subscr_topics - .iter() - .any(|t| message.topics.iter().any(|u| t == u)) - { - continue; - } - - if let Some(pos) = rpcs_to_dispatch.iter().position(|(p, _)| p == peer_id) { - rpcs_to_dispatch[pos].1.messages.push(message.clone()); - } else { - rpcs_to_dispatch.push(( - peer_id.clone(), - GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![message.clone()], - control_msgs: Vec::new(), - }, - )); + GossipsubControlAction::IWant { message_ids } => { + self.handle_iwant(&propagation_source, message_ids) } + GossipsubControlAction::Graft { topic_hash } => graft_msgs.push(topic_hash), + GossipsubControlAction::Prune { topic_hash } => prune_msgs.push(topic_hash), } } - - for (peer_id, rpc) in rpcs_to_dispatch { - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id, - event: rpc, - }); + if !ihave_msgs.is_empty() { + self.handle_ihave(&propagation_source, ihave_msgs); + } + if !graft_msgs.is_empty() { + self.handle_graft(&propagation_source, graft_msgs); + } + if !prune_msgs.is_empty() { + self.handle_prune(&propagation_source, prune_msgs); } } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 4f8076f129f..34856f6abac 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -146,14 +146,14 @@ impl Encoder for GossipsubCodec { } control_msg.mut_iwant().push(rpc_iwant); } - GossipsubControlAction::Graft { topic } => { + GossipsubControlAction::Graft { topic_hash } => { let mut rpc_graft = rpc_proto::ControlGraft::new(); - rpc_graft.set_topicID(topic.into_string()); + rpc_graft.set_topicID(topic_hash.into_string()); control_msg.mut_graft().push(rpc_graft); } - GossipsubControlAction::Prune { topic } => { + GossipsubControlAction::Prune { topic_hash } => { let mut rpc_prune = rpc_proto::ControlPrune::new(); - rpc_prune.set_topicID(topic.into_string()); + rpc_prune.set_topicID(topic_hash.into_string()); control_msg.mut_prune().push(rpc_prune); } } @@ -234,7 +234,7 @@ impl Decoder for GossipsubCodec { .take_graft() .into_iter() .map(|mut graft| GossipsubControlAction::Graft { - topic: TopicHash::from_raw(graft.take_topicID()), + topic_hash: TopicHash::from_raw(graft.take_topicID()), }) .collect(); @@ -242,7 +242,7 @@ impl Decoder for GossipsubCodec { .take_prune() .into_iter() .map(|mut prune| GossipsubControlAction::Prune { - topic: TopicHash::from_raw(prune.take_topicID()), + topic_hash: TopicHash::from_raw(prune.take_topicID()), }) .collect(); @@ -352,11 +352,11 @@ pub enum GossipsubControlAction { /// The node has been added to the mesh - Graft control message. Graft { /// The mesh topic the peer should be added to. - topic: TopicHash, + 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: TopicHash, + topic_hash: TopicHash, }, } From 1aa63ada843e46343b0e57e0d034f26a2f206699 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 21 Jan 2019 20:29:04 +1100 Subject: [PATCH 15/91] Remove excess connected peer hashmap --- protocols/gossipsub/src/layer.rs | 178 ++++++++++++++++++------------- 1 file changed, 104 insertions(+), 74 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index a749763b99a..f5349980456 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -17,6 +17,9 @@ // 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. +// + +// TODO: Implement control message piggybacking use cuckoofilter::CuckooFilter; use futures::prelude::*; @@ -123,19 +126,9 @@ pub struct Gossipsub { /// Peer id of the local node. Used for the source of the messages that we publish. local_peer_id: PeerId, - // These data structures may be combined in later revisions - kept for ease of iteration - /// List of peers the network is connected to, and the topics that they're subscribed to. - peer_topics: HashMap>, - /// Inverse hashmap of connected_peers - maps a topic to a tuple which contains a list of - /// gossipsub peers and floodsub peers. Used to efficiently look up peers per topic. + /// A map of all connected_peers - A map of topic hash to a tuple containing a list of gossipsub peers and floodsub peers respectively. topic_peers: HashMap, Vec)>, - /* use topic_peers instead of two hashmaps - /// Map of topics to connected gossipsub peers - gossipsub_peers: HashMap>, - /// Map of topics to connected floodsub peers - floodsub_peers: HashMap>, - */ /// Overlay network of connected peers - Maps topics to connected gossipsub peers mesh: HashMap>, @@ -148,7 +141,7 @@ pub struct Gossipsub { /// Message cache for the last few heartbeats mcache: MessageCache, - // We keep track of the messages we received (in the format `hash(source ID, seq_no)`) so that + // We keep track of the messages we received (in the format `string(source ID, seq_no)`) so that // we don't dispatch the same message twice if we receive it twice on the network. received: CuckooFilter, @@ -163,7 +156,6 @@ impl Gossipsub { config: gs_config.clone(), events: VecDeque::new(), local_peer_id, - peer_topics: HashMap::new(), topic_peers: HashMap::new(), mesh: HashMap::new(), fanout: HashMap::new(), @@ -183,19 +175,20 @@ impl Gossipsub { } // send subscription request to all floodsub and gossipsub peers - // TODO: Consolidate hashmap of peers - for peer in self.peer_topics.keys() { - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer.clone(), - event: GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic: topic.hash().clone(), - action: GossipsubSubscriptionAction::Subscribe, - }], - control_msgs: Vec::new(), - }, - }); + for (flood_peers, gossip_peers) in self.topic_peers.values() { + for peer in flood_peers.iter().chain(gossip_peers) { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic: topic.hash().clone(), + action: GossipsubSubscriptionAction::Subscribe, + }], + control_msgs: Vec::new(), + }, + }); + } } // call JOIN(topic) @@ -219,18 +212,20 @@ impl Gossipsub { } // announce to all floodsub and gossipsub peers - for peer in self.peer_topics.keys() { - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer.clone(), - event: GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic: topic_hash.clone(), - action: GossipsubSubscriptionAction::Unsubscribe, - }], - control_msgs: Vec::new(), - }, - }); + for (flood_peers, gossip_peers) in self.topic_peers.values() { + for peer in flood_peers.iter().chain(gossip_peers) { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic: topic_hash.clone(), + action: GossipsubSubscriptionAction::Unsubscribe, + }], + control_msgs: Vec::new(), + }, + }); + } } // call LEAVE(topic) @@ -568,41 +563,67 @@ impl Gossipsub { } } - /// Handles received subscription. + /// Handles received subscriptions. fn handle_received_subscriptions( &mut self, - subscription: GossipsubSubscription, + subscriptions: Vec, propagation_source: &PeerId, + node_type: NodeType, ) { - let remote_peer_topics = self.peer_topics - .get_mut(&propagation_source) - .expect("connected_peers is kept in sync with the peers we are connected to; we are guaranteed to only receive events from connected peers; QED"); - match subscription.action { - GossipsubSubscriptionAction::Subscribe => { - if !remote_peer_topics.contains(&subscription.topic) { - remote_peer_topics.push(subscription.topic.clone()); + for subscription in subscriptions { + // get the peers from the mapping, or insert empty lists if topic doesn't exist + let (flood_peers, gossip_peers) = self + .topic_peers + .entry(subscription.topic.clone()) + .or_insert((vec![], vec![])); + + match subscription.action { + GossipsubSubscriptionAction::Subscribe => { + match node_type { + NodeType::Floodsub => { + if !flood_peers.contains(&propagation_source) { + flood_peers.push(propagation_source.clone()); + } + } + NodeType::Gossipsub => { + if !gossip_peers.contains(&propagation_source) { + gossip_peers.push(propagation_source.clone()); + } + } + } + // generates a subscription event to be polled + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Subscribed { + peer_id: propagation_source.clone(), + topic: subscription.topic, + }, + )); } - // generates a subscription event to be polled - self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Subscribed { - peer_id: propagation_source.clone(), - topic: subscription.topic, - }, - )); - } - GossipsubSubscriptionAction::Unsubscribe => { - if let Some(pos) = remote_peer_topics - .iter() - .position(|t| t == &subscription.topic) - { - remote_peer_topics.remove(pos); + GossipsubSubscriptionAction::Unsubscribe => { + match node_type { + NodeType::Floodsub => { + if let Some(pos) = + flood_peers.iter().position(|p| p == propagation_source) + { + flood_peers.remove(pos); + } + } + NodeType::Gossipsub => { + if let Some(pos) = + gossip_peers.iter().position(|p| p == propagation_source) + { + gossip_peers.remove(pos); + } + } + } + + self.events.push_back(NetworkBehaviourAction::GenerateEvent( + GossipsubEvent::Unsubscribed { + peer_id: propagation_source.clone(), + topic: subscription.topic, + }, + )); } - self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Unsubscribed { - peer_id: propagation_source.clone(), - topic: subscription.topic, - }, - )); } } } @@ -661,21 +682,23 @@ where } // TODO: Handle the peer addition - Specifically add the gossipsub and floodsub peers. - self.peer_topics.insert(id.clone(), SmallVec::new()); + // self.peer_topics.insert(id.clone(), SmallVec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { // TODO: Handle peer disconnection - floodsub and gossipsub peers - let was_in = self.peer_topics.remove(id); - debug_assert!(was_in.is_some()); + //let was_in = self.peer_topics.remove(id); + //debug_assert!(was_in.is_some()); } fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { // Handle subscriptions // Update connected peers topics - for subscription in event.subscriptions { - self.handle_received_subscriptions(subscription, &propagation_source); - } + self.handle_received_subscriptions( + event.subscriptions, + &propagation_source, + NodeType::Gossipsub, + ); // Handle messages for message in event.messages { @@ -727,7 +750,6 @@ where Async::NotReady } } - /// Event that can happen on the gossipsub behaviour. #[derive(Debug)] pub enum GossipsubEvent { @@ -751,6 +773,14 @@ pub enum GossipsubEvent { }, } +/// The type of node in the pubsub system. +pub enum NodeType { + /// A gossipsub node. + Gossipsub, + /// A Floodsub node. + Floodsub, +} + #[cfg(test)] mod tests { use super::*; From db345bb77b407468bd591058a93750cdd873fcd1 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 22 Jan 2019 18:11:29 +1100 Subject: [PATCH 16/91] Add extra peer mapping and consistent topic naming. --- protocols/gossipsub/src/layer.rs | 148 ++++++++++++++++++++++------ protocols/gossipsub/src/protocol.rs | 26 ++--- 2 files changed, 134 insertions(+), 40 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index f5349980456..417c30fc7a8 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -126,9 +126,14 @@ pub struct Gossipsub { /// Peer id of the local node. Used for the source of the messages that we publish. local_peer_id: PeerId, - /// A map of all connected_peers - A map of topic hash to a tuple containing a list of gossipsub peers and floodsub peers respectively. + /// A map of all connected peers - A map of topic hash to a tuple containing a list of gossipsub peers and floodsub peers respectively. topic_peers: HashMap, Vec)>, + /// A map of all connected peers to a tuple containing their subscribed topics and NodeType + /// respectively. + // This is used to efficiently keep track of all currently connected nodes and their type + peer_topics: HashMap, NodeType)>, + /// Overlay network of connected peers - Maps topics to connected gossipsub peers mesh: HashMap>, @@ -157,6 +162,7 @@ impl Gossipsub { events: VecDeque::new(), local_peer_id, topic_peers: HashMap::new(), + peer_topics: HashMap::new(), mesh: HashMap::new(), fanout: HashMap::new(), fanout_last_pub: HashMap::new(), @@ -182,7 +188,7 @@ impl Gossipsub { event: GossipsubRpc { messages: Vec::new(), subscriptions: vec![GossipsubSubscription { - topic: topic.hash().clone(), + topic_hash: topic.hash().clone(), action: GossipsubSubscriptionAction::Subscribe, }], control_msgs: Vec::new(), @@ -219,7 +225,7 @@ impl Gossipsub { event: GossipsubRpc { messages: Vec::new(), subscriptions: vec![GossipsubSubscription { - topic: topic_hash.clone(), + topic_hash: topic_hash.clone(), action: GossipsubSubscriptionAction::Unsubscribe, }], control_msgs: Vec::new(), @@ -463,14 +469,14 @@ impl Gossipsub { /// with PRUNE messages. fn handle_graft(&mut self, peer_id: &PeerId, topics: Vec) { let mut to_prune_topics = HashMap::new(); - for topic in topics { - if let Some(peers) = self.mesh.get_mut(&topic) { + for topic_hash in topics { + if let Some(peers) = self.mesh.get_mut(&topic_hash) { // if we are subscribed, add peer to the mesh println!("GRAFT: Mesh link added from {:?}", peer_id); peers.push(peer_id.clone()); //TODO: tagPeer } else { - to_prune_topics.insert(topic.clone(), ()); + to_prune_topics.insert(topic_hash.clone(), ()); } } @@ -478,8 +484,8 @@ impl Gossipsub { // build the prune messages to send let prune_messages = to_prune_topics .keys() - .map(|topic| GossipsubControlAction::Prune { - topic_hash: topic.clone(), + .map(|topic_hash| GossipsubControlAction::Prune { + topic_hash: topic_hash.clone(), }) .collect(); // Send the prune messages to the peer @@ -496,8 +502,8 @@ impl Gossipsub { /// Handles PRUNE control messages. Removes peer from the mesh. fn handle_prune(&mut self, peer_id: &PeerId, topics: Vec) { - for topic in topics { - if let Some(peers) = self.mesh.get_mut(&topic) { + for topic_hash in topics { + if let Some(peers) = self.mesh.get_mut(&topic_hash) { // remove the peer if it exists in the mesh if let Some(pos) = peers.iter().position(|p| p == peer_id) { peers.remove(pos); @@ -568,13 +574,23 @@ impl Gossipsub { &mut self, subscriptions: Vec, propagation_source: &PeerId, - node_type: NodeType, ) { + let (subscribed_topics, node_type) = match self.peer_topics.get_mut(&propagation_source) { + Some((topics, node_type)) => (topics, node_type), + None => { + println!( + "ERROR: Subscription by unknown peer: {:?}", + &propagation_source + ); + return; + } + }; + for subscription in subscriptions { // get the peers from the mapping, or insert empty lists if topic doesn't exist let (flood_peers, gossip_peers) = self .topic_peers - .entry(subscription.topic.clone()) + .entry(subscription.topic_hash.clone()) .or_insert((vec![], vec![])); match subscription.action { @@ -591,11 +607,15 @@ impl Gossipsub { } } } + // add to the peer_topics mapping + if !subscribed_topics.contains(&subscription.topic_hash) { + subscribed_topics.push(subscription.topic_hash.clone()); + } // generates a subscription event to be polled self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Subscribed { peer_id: propagation_source.clone(), - topic: subscription.topic, + topic: subscription.topic_hash, }, )); } @@ -616,11 +636,18 @@ impl Gossipsub { } } } - + // remove topic from the peer_topics mapping + if let Some(pos) = subscribed_topics + .iter() + .position(|t| t == &subscription.topic_hash) + { + subscribed_topics.remove(pos); + } + // generate a subscription even to be polled self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Unsubscribed { peer_id: propagation_source.clone(), - topic: subscription.topic, + topic: subscription.topic_hash, }, )); } @@ -662,9 +689,9 @@ where fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { // We need to send our subscriptions to the newly-connected node. let mut subscriptions = vec![]; - for topic in self.mesh.keys() { + for topic_hash in self.mesh.keys() { subscriptions.push(GossipsubSubscription { - topic: topic.clone(), + topic_hash: topic_hash.clone(), action: GossipsubSubscriptionAction::Subscribe, }); } @@ -681,24 +708,84 @@ where }); } - // TODO: Handle the peer addition - Specifically add the gossipsub and floodsub peers. - // self.peer_topics.insert(id.clone(), SmallVec::new()); + // TODO: Handle the peer addition - Specifically handle floodsub peers. + // For the time being assume all gossipsub peers + self.peer_topics + .insert(id.clone(), (SmallVec::new(), NodeType::Gossipsub)); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { - // TODO: Handle peer disconnection - floodsub and gossipsub peers - //let was_in = self.peer_topics.remove(id); - //debug_assert!(was_in.is_some()); + // TODO: Handle peer disconnection - specifically floodsub peers + // TODO: Refactor + // remove from mesh, topic_peers and peer_topic + { + let (topics, node_type) = match self.peer_topics.get(&id) { + Some((topics, node_type)) => (topics, node_type), + None => { + println!("ERROR: Disconnected node, not in connected nodes"); + return; + } + }; + + // remove peer from all mappings + for topic in topics { + // check the mesh for the topic + if let Some(mesh_peers) = self.mesh.get_mut(&topic) { + // check if the peer is in the mesh and remove it + if let Some(pos) = mesh_peers.iter().position(|p| p == id) { + mesh_peers.remove(pos); + //TODO: untagPeer + } + } + + // remove from topic_peers + if let Some((floodsub_peers, gossip_peers)) = self.topic_peers.get_mut(&topic) { + match node_type { + NodeType::Gossipsub => { + if let Some(pos) = gossip_peers.iter().position(|p| p == id) { + gossip_peers.remove(pos); + //TODO: untagPeer + } + // debugging purposes + else { + println!( + "ERROR: Disconnected node: {:?} not in topic_peers peer list", + &id + ); + } + } + NodeType::Floodsub => { + if let Some(pos) = floodsub_peers.iter().position(|p| p == id) { + floodsub_peers.remove(pos); + //TODO: untagPeer + } + // debugging purposes + else { + println!( + "ERROR: Disconnected node: {:?} not in topic_peers peer list", + &id + ); + } + } + } + } else { + println!( + "ERROR: Disconnected node: {:?} with topic: {:?} not in topic_peers", + &id, &topic + ); + } + } + } + + // remove peer from peer_topics + let was_in = self.peer_topics.remove(id); + debug_assert!(was_in.is_some()); } fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { // Handle subscriptions // Update connected peers topics - self.handle_received_subscriptions( - event.subscriptions, - &propagation_source, - NodeType::Gossipsub, - ); + self.handle_received_subscriptions(event.subscriptions, &propagation_source); // Handle messages for message in event.messages { @@ -713,8 +800,11 @@ where let mut prune_msgs = vec![]; for control_msg in event.control_msgs { match control_msg { - GossipsubControlAction::IHave { topic, message_ids } => { - ihave_msgs.push((topic, message_ids)); + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => { + ihave_msgs.push((topic_hash, message_ids)); } GossipsubControlAction::IWant { message_ids } => { self.handle_iwant(&propagation_source, message_ids) diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 34856f6abac..7f2d876b4f3 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -118,11 +118,12 @@ impl Encoder for GossipsubCodec { proto.mut_publish().push(msg); } - for topic in item.subscriptions.into_iter() { - let mut subscription = rpc_proto::RPC_SubOpts::new(); - subscription.set_subscribe(topic.action == GossipsubSubscriptionAction::Subscribe); - subscription.set_topicid(topic.topic.into_string()); - proto.mut_subscriptions().push(subscription); + for subscription in item.subscriptions.into_iter() { + let mut rpc_subscription = rpc_proto::RPC_SubOpts::new(); + rpc_subscription + .set_subscribe(subscription.action == GossipsubSubscriptionAction::Subscribe); + rpc_subscription.set_topicid(subscription.topic_hash.into_string()); + proto.mut_subscriptions().push(rpc_subscription); } // gossipsub control messages @@ -131,9 +132,12 @@ impl Encoder for GossipsubCodec { for action in item.control_msgs { match action { // collect all ihave messages - GossipsubControlAction::IHave { topic, message_ids } => { + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => { let mut rpc_ihave = rpc_proto::ControlIHave::new(); - rpc_ihave.set_topicID(topic.into_string()); + rpc_ihave.set_topicID(topic_hash.into_string()); for msg_id in message_ids { rpc_ihave.mut_messageIDs().push(msg_id); } @@ -212,7 +216,7 @@ impl Decoder for GossipsubCodec { .into_iter() .map(|mut ihave| { GossipsubControlAction::IHave { - topic: TopicHash::from_raw(ihave.take_topicID()), + topic_hash: TopicHash::from_raw(ihave.take_topicID()), // TODO: Potentially format the message ids better message_ids: ihave.take_messageIDs().into_vec(), } @@ -262,7 +266,7 @@ impl Decoder for GossipsubCodec { } else { GossipsubSubscriptionAction::Unsubscribe }, - topic: TopicHash::from_raw(sub.take_topicid()), + topic_hash: TopicHash::from_raw(sub.take_topicid()), }) .collect(), control_msgs, @@ -322,7 +326,7 @@ pub struct GossipsubSubscription { /// Action to perform. pub action: GossipsubSubscriptionAction, /// The topic from which to subscribe or unsubscribe. - pub topic: TopicHash, + pub topic_hash: TopicHash, } /// Action that a subscription wants to perform. @@ -340,7 +344,7 @@ pub enum GossipsubControlAction { /// Node broadcasts known messages per topic - IHave control message. IHave { /// The topic of the message. - topic: TopicHash, + topic_hash: TopicHash, /// A list of known message ids (peer_id + sequence _number) as a string. message_ids: Vec, }, From 5531342bf8568b2b2eed561c60401e13a7c800d3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 23 Jan 2019 18:21:53 +1100 Subject: [PATCH 17/91] Implement heartbeat, emit_gossip and send_graft_prune. --- protocols/gossipsub/src/layer.rs | 213 +++++++++++++++++++++++++++++-- 1 file changed, 204 insertions(+), 9 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 417c30fc7a8..e0e20c85627 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -59,6 +59,9 @@ pub struct GossipsubConfig { /// Maximum number of peers in mesh network before removing some (D_high in the spec). mesh_n_high: usize, + /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). + gossip_lazy: usize, + /// Initial delay in each heartbeat. heartbeat_initial_delay: Duration, /// Time between each heartbeat. @@ -75,6 +78,7 @@ impl Default for GossipsubConfig { mesh_n: 6, mesh_n_low: 4, mesh_n_high: 12, + gossip_lazy: 6, // default to mesh_n heartbeat_initial_delay: Duration::from_millis(100), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), @@ -89,6 +93,7 @@ impl GossipsubConfig { mesh_n: usize, mesh_n_low: usize, mesh_n_high: usize, + gossip_lazy: usize, heartbeat_initial_delay: Duration, heartbeat_interval: Duration, fanout_ttl: Duration, @@ -107,6 +112,7 @@ impl GossipsubConfig { mesh_n, mesh_n_low, mesh_n_high, + gossip_lazy, heartbeat_initial_delay, heartbeat_interval, fanout_ttl, @@ -304,7 +310,7 @@ impl Gossipsub { // TODO: Ensure fanout key never contains an empty set // we have no fanout peers, select mesh_n of them and add them to the fanout let mesh_n = self.config.mesh_n; - let new_peers = self.get_random_peers(t, mesh_n); + let new_peers = self.get_random_peers(t, mesh_n, { |_| true }); // add the new peers to the fanout and recipient peers self.fanout.insert(t.clone(), new_peers.clone()); for peer_id in new_peers { @@ -357,7 +363,7 @@ impl Gossipsub { } else { // no peers in fanout[topic] - select mesh_n at random let mesh_n = self.config.mesh_n; - peers = self.get_random_peers(topic_hash, mesh_n); + peers = self.get_random_peers(topic_hash, mesh_n, { |_| true }); // put them in the mesh self.mesh.insert(topic_hash.clone(), peers.clone()); } @@ -655,10 +661,199 @@ impl Gossipsub { } } - /// Helper function to get a set of `n` random gossipsub peers for a topic - fn get_random_peers(&self, topic_hash: &TopicHash, n: usize) -> Vec { + /// Heartbeat function which shifts the memcache and updates the mesh + fn heartbeat(&mut self) { + //TODO: Clean up any state from last heartbeat. + + let mut to_graft = HashMap::new(); + let mut to_prune = HashMap::new(); + + // maintain the mesh for each topic + for (topic_hash, peers) in self.mesh.clone().iter_mut() { + // too little peers - add some + if peers.len() < self.config.mesh_n_low { + // not enough peers - get mesh_n - current_length more + let desired_peers = self.config.mesh_n - peers.len(); + let peer_list = self + .get_random_peers(topic_hash, desired_peers, { |peer| !peers.contains(peer) }); + for peer in peer_list { + peers.push(peer.clone()); + // TODO: tagPeer + let current_topic = to_graft.entry(peer).or_insert(Vec::new()); + current_topic.push(topic_hash.clone()); + } + // update the mesh + self.mesh.insert(topic_hash.clone(), peers.clone()); + } + + // too many peers - remove some + if peers.len() > self.config.mesh_n_high { + let excess_peer_no = peers.len() - self.config.mesh_n; + // shuffle the peers + let mut rng = thread_rng(); + peers.shuffle(&mut rng); + // remove the first excess_peer_no peers adding them to to_prune + for _ in 0..excess_peer_no { + let peer = peers + .pop() + .expect("There should always be enough peers to remove"); + let current_topic = to_prune.entry(peer).or_insert(vec![]); + current_topic.push(topic_hash.clone()); + //TODO: untagPeer + } + // update the mesh + self.mesh.insert(topic_hash.clone(), peers.clone()); + } + + // emit gossip + self.emit_gossip(topic_hash.clone(), peers.clone()); + } + + // remove expired fanout topics + { + let fanout = &mut self.fanout; // help the borrow checker + let fanout_ttl = self.config.fanout_ttl; + self.fanout_last_pub.retain(|topic_hash, last_pub_time| { + if *last_pub_time + fanout_ttl < Instant::now() { + fanout.remove(&topic_hash); + return false; + } + true + }); + } + + // maintain fanout + // check if our peers are still apart of the topic + for (topic_hash, peers) in self.fanout.clone().iter_mut() { + peers.retain(|peer| { + // is the peer still subscribed to the topic? + if !self + .peer_topics + .get(peer) + .expect("Peer should exist") + .0 + .contains(&topic_hash) + { + return false; + } + true + }); + + // not enough peers + if peers.len() < self.config.mesh_n { + let needed_peers = self.config.mesh_n - peers.len(); + let mut new_peers = + self.get_random_peers(topic_hash, needed_peers, |peer| !peers.contains(peer)); + peers.append(&mut new_peers); + } + // update the entry + self.fanout.insert(topic_hash.clone(), peers.to_vec()); + + self.emit_gossip(topic_hash.clone(), peers.clone()); + } + + // send graft/prunes + self.send_graft_prune(to_graft, to_prune); + + // shift the memcache + self.mcache.shift(); + } + + /// Emits gossip - Send IHAVE messages to a random set of gossip peers that are not in the mesh, but are subscribed to the `topic`. + fn emit_gossip(&mut self, topic_hash: TopicHash, peers: Vec) { + let message_ids = self.mcache.get_gossip_ids(&topic_hash); + if message_ids.is_empty() { + return; + } + + // get gossip_lazy random peers + let to_msg_peers = self.get_random_peers(&topic_hash, self.config.gossip_lazy, |peer| { + !peers.contains(peer) + }); + for peer in to_msg_peers { + // send an IHAVE message + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer, + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: vec![GossipsubControlAction::IHave { + topic_hash: topic_hash.clone(), + message_ids: message_ids.clone(), + }], + }, + }); + } + } + + /// Handles multiple GRAFT/PRUNE messages and coalesces them into chunked gossip control + /// messages. + fn send_graft_prune( + &mut self, + to_graft: HashMap>, + mut to_prune: HashMap>, + ) { + // handle the grafts and overlapping prunes + for (peer, topics) in to_graft.iter() { + let mut grafts: Vec = topics + .iter() + .map(|topic_hash| { + return GossipsubControlAction::Graft { + topic_hash: topic_hash.clone(), + }; + }) + .collect(); + let mut prunes: Vec = to_prune + .remove(&peer) + .unwrap_or_else(|| vec![]) + .iter() + .map(|topic_hash| GossipsubControlAction::Prune { + topic_hash: topic_hash.clone(), + }) + .collect(); + grafts.append(&mut prunes); + + // send the control messages + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: grafts, + }, + }); + } + + // handle the remaining prunes + for (peer, topics) in to_prune.iter() { + let remaining_prunes = topics + .iter() + .map(|topic_hash| GossipsubControlAction::Prune { + topic_hash: topic_hash.clone(), + }) + .collect(); + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: remaining_prunes, + }, + }); + } + } + + /// Helper function to get a set of `n` random gossipsub peers for a `topic_hash` + /// filtered by the function `f`. + fn get_random_peers( + &self, + topic_hash: &TopicHash, + n: usize, + mut f: impl FnMut(&PeerId) -> bool, + ) -> Vec { let mut gossip_peers = match self.topic_peers.get(topic_hash) { - Some((gossip_peers, _)) => gossip_peers.clone(), + // if they exist, filter the peers by `f` + Some((gossip_peers, _)) => gossip_peers.iter().cloned().filter(|p| f(p)).collect(), None => Vec::new(), }; @@ -894,15 +1089,15 @@ mod tests { gs.topic_peers .insert(topic_hash.clone(), (peers.clone(), vec![])); - let random_peers = gs.get_random_peers(&topic_hash, 5); + let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| true }); assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); - let random_peers = gs.get_random_peers(&topic_hash, 30); + let random_peers = gs.get_random_peers(&topic_hash, 30, { |_| true }); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic_hash, 20); + let random_peers = gs.get_random_peers(&topic_hash, 20, { |_| true }); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic_hash, 0); + let random_peers = gs.get_random_peers(&topic_hash, 0, { |_| true }); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); } From a29ae3780d9f711d464dd96fb5e1f7dbb46659d7 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 23 Jan 2019 19:12:53 +1100 Subject: [PATCH 18/91] Group logic in forwarding messages. Add messages to memcache. --- protocols/gossipsub/src/layer.rs | 159 ++++++++++++++----------------- 1 file changed, 72 insertions(+), 87 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index e0e20c85627..7a8e1803ca5 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -271,66 +271,43 @@ impl Gossipsub { topics: topic.into_iter().map(|t| t.into().clone()).collect(), }; - // build a list of peers to forward the message to - // TODO: Potentially use a hashmap - efficiency in checking for duplicates - let mut recipient_peers: Vec = vec![]; + // forward the message to mesh and floodsub peers + let local_peer_id = self.local_peer_id.clone(); + self.forward_msg(message.clone(), local_peer_id); - for t in message.topics.iter() { - // floodsub peers in the topic - add them to recipient_peers - if let Some((_, floodsub_peers)) = self.topic_peers.get(t) { - for peer_id in floodsub_peers { - if !recipient_peers.contains(peer_id) { - recipient_peers.push(peer_id.clone()); - } - } - } - - // gossipsub peers in the mesh - match self.mesh.get(t) { - // we are in the mesh, add the gossip peers to recipient_peers - Some(gossip_peers) => { - for peer_id in gossip_peers { - if !recipient_peers.contains(peer_id) { - recipient_peers.push(peer_id.clone()); - } + let mut recipient_peers = HashMap::new(); + for topic_hash in &message.topics { + // if not subscribed to the topic, use fanout peers + if self.mesh.get(&topic_hash).is_none() { + // build a list of peers to forward the message to + // if we have fanout peers add them to the map + if let Some(fanout_peers) = self.fanout.get(&topic_hash) { + for peer in fanout_peers { + recipient_peers.insert(peer.clone(), ()); } } - // not in the mesh, use fanout peers - None => { - if self.fanout.contains_key(t) { - // we have fanout peers. Add them to recipient_peers - if let Some(fanout_peers) = self.fanout.get(t) { - for peer_id in fanout_peers { - if !recipient_peers.contains(peer_id) { - recipient_peers.push(peer_id.clone()); - } - } - } - } else { - // TODO: Ensure fanout key never contains an empty set - // we have no fanout peers, select mesh_n of them and add them to the fanout - let mesh_n = self.config.mesh_n; - let new_peers = self.get_random_peers(t, mesh_n, { |_| true }); - // add the new peers to the fanout and recipient peers - self.fanout.insert(t.clone(), new_peers.clone()); - for peer_id in new_peers { - if !recipient_peers.contains(&peer_id) { - recipient_peers.push(peer_id.clone()); - } - } - } - // we are publishing to fanout peers - update the time we published - self.fanout_last_pub.insert(t.clone(), Instant::now()); + } else { + // TODO: Ensure fanout key never contains an empty set + // we have no fanout peers, select mesh_n of them and add them to the fanout + let mesh_n = self.config.mesh_n; + let new_peers = self.get_random_peers(&topic_hash, mesh_n, { |_| true }); + // add the new peers to the fanout and recipient peers + self.fanout.insert(topic_hash.clone(), new_peers.clone()); + for peer in new_peers { + recipient_peers.insert(peer.clone(), ()); } } + // we are publishing to fanout peers - update the time we published + self.fanout_last_pub + .insert(topic_hash.clone(), Instant::now()); } - // add published message to our received cache - // TODO: Add to memcache + // add published message to our received caches + self.mcache.put(message.clone()); self.received.add(&message.msg_id()); // Send to peers we know are subscribed to the topic. - for peer_id in recipient_peers { + for peer_id in recipient_peers.keys() { println!("peers subscribed? {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), @@ -343,6 +320,46 @@ impl Gossipsub { } } + /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. + fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { + let mut recipient_peers = HashMap::new(); + + // add floodsub and mesh peers + for topic in &message.topics { + // floodsub + if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { + for peer_id in floodsub_peers { + if *peer_id != source { + recipient_peers.insert(peer_id.clone(), ()); + } + } + } + + // mesh + if let Some(mesh_peers) = self.mesh.get(&topic) { + for peer_id in mesh_peers { + if *peer_id != source { + recipient_peers.insert(peer_id.clone(), ()); + } + } + } + } + + // forward the message to peers + if !recipient_peers.is_empty() { + for peer in recipient_peers.keys() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![message.clone()], + control_msgs: Vec::new(), + }, + }); + } + } + } + /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. fn join(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); @@ -529,6 +546,9 @@ impl Gossipsub { return; } + // add to the memcache + self.mcache.put(msg.clone()); + // dispatch the message to the user if self.mesh.keys().any(|t| msg.topics.iter().any(|u| t == u)) { self.events.push_back(NetworkBehaviourAction::GenerateEvent( @@ -536,43 +556,8 @@ impl Gossipsub { )); } - // peers to forward the message to - let mut recipient_peers = HashMap::new(); - - // add floodsub and mesh peers - for topic in &msg.topics { - // floodsub - if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { - for peer_id in floodsub_peers { - if peer_id != propagation_source { - recipient_peers.insert(peer_id.clone(), ()); - } - } - } - - // mesh - if let Some(mesh_peers) = self.mesh.get(&topic) { - for peer_id in mesh_peers { - if peer_id != propagation_source { - recipient_peers.insert(peer_id.clone(), ()); - } - } - } - } - - // forward the message to peers - if !recipient_peers.is_empty() { - for peer in recipient_peers.keys() { - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![msg.clone()], - control_msgs: Vec::new(), - }, - }); - } - } + // forward the message to floodsub and mesh peers + self.forward_msg(msg, propagation_source.clone()); } /// Handles received subscriptions. From a85ee9a1b9c836a04b59c05810633cc8f059fd7e Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 23 Jan 2019 21:41:05 +1100 Subject: [PATCH 19/91] Add heartbeat timer and move location of helper function. --- protocols/gossipsub/Cargo.toml | 1 + protocols/gossipsub/src/layer.rs | 114 +++++++++++++++++-------------- protocols/gossipsub/src/lib.rs | 1 + 3 files changed, 65 insertions(+), 51 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 12f8dddf5ba..48ab9fcfbaa 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -18,6 +18,7 @@ rand = "" smallvec = "" tokio-codec = "" tokio-io = "" +tokio-timer = "" unsigned-varint = "" [dev-dependencies] diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 7a8e1803ca5..5c34191fcbe 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -41,6 +41,7 @@ use std::collections::hash_map::{DefaultHasher, HashMap}; use std::time::{Duration, Instant}; use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; +use tokio_timer::Interval; // potentially rename this struct - due to clashes /// Configuration parameters that define the performance of the gossipsub network. @@ -62,8 +63,9 @@ pub struct GossipsubConfig { /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). gossip_lazy: usize, - /// Initial delay in each heartbeat. - heartbeat_initial_delay: Duration, + // Not applicable right now + // /// Initial delay in each heartbeat. + // heartbeat_initial_delay: Duration, /// Time between each heartbeat. heartbeat_interval: Duration, /// Time to live for fanout peers. @@ -79,7 +81,7 @@ impl Default for GossipsubConfig { mesh_n_low: 4, mesh_n_high: 12, gossip_lazy: 6, // default to mesh_n - heartbeat_initial_delay: Duration::from_millis(100), + // heartbeat_initial_delay: Duration::from_millis(100), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), } @@ -94,7 +96,7 @@ impl GossipsubConfig { mesh_n_low: usize, mesh_n_high: usize, gossip_lazy: usize, - heartbeat_initial_delay: Duration, + // heartbeat_initial_delay: Duration, heartbeat_interval: Duration, fanout_ttl: Duration, ) -> GossipsubConfig { @@ -113,7 +115,7 @@ impl GossipsubConfig { mesh_n_low, mesh_n_high, gossip_lazy, - heartbeat_initial_delay, + // heartbeat_initial_delay, heartbeat_interval, fanout_ttl, } @@ -123,7 +125,7 @@ impl GossipsubConfig { /// Network behaviour that automatically identifies nodes periodically, and returns information /// about them. pub struct Gossipsub { - /// Configuration providing gossipsub performance parameters + /// Configuration providing gossipsub performance parameters. config: GossipsubConfig, /// Events that need to be yielded to the outside when polling. @@ -140,28 +142,31 @@ pub struct Gossipsub { // This is used to efficiently keep track of all currently connected nodes and their type peer_topics: HashMap, NodeType)>, - /// Overlay network of connected peers - Maps topics to connected gossipsub peers + /// Overlay network of connected peers - Maps topics to connected gossipsub peers. mesh: HashMap>, - /// Map of topics to list of peers that we publish to, but don't subscribe to + /// Map of topics to list of peers that we publish to, but don't subscribe to. fanout: HashMap>, - /// The last publish time for fanout topics + /// The last publish time for fanout topics. fanout_last_pub: HashMap, - /// Message cache for the last few heartbeats + /// Message cache for the last few heartbeats. mcache: MessageCache, // We keep track of the messages we received (in the format `string(source ID, seq_no)`) so that // we don't dispatch the same message twice if we receive it twice on the network. received: CuckooFilter, + /// Heartbeat interval stream. + heartbeat: Interval, + /// Marker to pin the generics. marker: PhantomData, } impl Gossipsub { - /// Creates a `Gossipsub`. + /// Creates a `Gossipsub` struct given a set of parameters specified by `gs_config`. pub fn new(local_peer_id: PeerId, gs_config: GossipsubConfig) -> Self { Gossipsub { config: gs_config.clone(), @@ -174,6 +179,7 @@ impl Gossipsub { fanout_last_pub: HashMap::new(), mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), received: CuckooFilter::new(), + heartbeat: Interval::new_interval(gs_config.heartbeat_interval), marker: PhantomData, } } @@ -320,46 +326,6 @@ impl Gossipsub { } } - /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. - fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { - let mut recipient_peers = HashMap::new(); - - // add floodsub and mesh peers - for topic in &message.topics { - // floodsub - if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { - for peer_id in floodsub_peers { - if *peer_id != source { - recipient_peers.insert(peer_id.clone(), ()); - } - } - } - - // mesh - if let Some(mesh_peers) = self.mesh.get(&topic) { - for peer_id in mesh_peers { - if *peer_id != source { - recipient_peers.insert(peer_id.clone(), ()); - } - } - } - } - - // forward the message to peers - if !recipient_peers.is_empty() { - for peer in recipient_peers.keys() { - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![message.clone()], - control_msgs: Vec::new(), - }, - }); - } - } - } - /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. fn join(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); @@ -828,6 +794,46 @@ impl Gossipsub { } } + /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. + fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { + let mut recipient_peers = HashMap::new(); + + // add floodsub and mesh peers + for topic in &message.topics { + // floodsub + if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { + for peer_id in floodsub_peers { + if *peer_id != source { + recipient_peers.insert(peer_id.clone(), ()); + } + } + } + + // mesh + if let Some(mesh_peers) = self.mesh.get(&topic) { + for peer_id in mesh_peers { + if *peer_id != source { + recipient_peers.insert(peer_id.clone(), ()); + } + } + } + } + + // forward the message to peers + if !recipient_peers.is_empty() { + for peer in recipient_peers.keys() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer.clone(), + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![message.clone()], + control_msgs: Vec::new(), + }, + }); + } + } + } + /// Helper function to get a set of `n` random gossipsub peers for a `topic_hash` /// filtered by the function `f`. fn get_random_peers( @@ -1017,6 +1023,12 @@ where return Async::Ready(event); } + match self.heartbeat.poll() { + // heartbeat ready + Ok(Async::Ready(Some(_))) => self.heartbeat(), + _ => {} + }; + Async::NotReady } } diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 9f421e517e8..738aa5e101e 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -31,6 +31,7 @@ extern crate rand; extern crate smallvec; extern crate tokio_codec; extern crate tokio_io; +extern crate tokio_timer; extern crate unsigned_varint; pub mod handler; From e0142b64981f05d06bf82b9ae3936a4b2ae1c41e Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 23 Jan 2019 22:54:55 +1100 Subject: [PATCH 20/91] Add gossipsub the libp2p workspace, makes layer structs public --- protocols/gossipsub/src/lib.rs | 2 +- src/lib.rs | 91 ++++++++++++++++++++++++---------- 2 files changed, 66 insertions(+), 27 deletions(-) diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 738aa5e101e..cb4f8e67bf1 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -42,5 +42,5 @@ mod mcache; mod rpc_proto; pub use self::handler::GossipsubHandler; -pub use self::layer::Gossipsub; +pub use self::layer::{Gossipsub, GossipsubConfig, GossipsubEvent}; pub use self::protocol::*; diff --git a/src/lib.rs b/src/lib.rs index 4bcd9e4fd75..85cab110a3b 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -136,8 +136,8 @@ pub extern crate bytes; pub extern crate futures; pub extern crate multiaddr; pub extern crate multihash; -pub extern crate tokio_io; pub extern crate tokio_codec; +pub extern crate tokio_io; extern crate libp2p_core_derive; extern crate tokio_executor; @@ -148,16 +148,18 @@ pub use libp2p_core as core; #[doc(inline)] pub use libp2p_dns as dns; #[doc(inline)] -pub use libp2p_identify as identify; +pub use libp2p_floodsub as floodsub; #[doc(inline)] -pub use libp2p_kad as kad; +pub use libp2p_gossipsub as gossipsub; #[doc(inline)] -pub use libp2p_floodsub as floodsub; +pub use libp2p_identify as identify; #[doc(inline)] -pub use libp2p_mplex as mplex; +pub use libp2p_kad as kad; #[cfg(not(any(target_os = "emscripten", target_os = "unknown")))] #[doc(inline)] pub use libp2p_mdns as mdns; +#[doc(inline)] +pub use libp2p_mplex as mplex; #[cfg(not(any(target_os = "emscripten", target_os = "unknown")))] #[doc(inline)] pub use libp2p_noise as noise; @@ -185,14 +187,14 @@ mod transport_ext; pub mod simple; pub use self::core::{ - Transport, PeerId, Swarm, transport::TransportError, - upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt} + upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt}, + PeerId, Swarm, Transport, }; -pub use libp2p_core_derive::NetworkBehaviour; pub use self::multiaddr::Multiaddr; pub use self::simple::SimpleProtocol; pub use self::transport_ext::TransportExt; +pub use libp2p_core_derive::NetworkBehaviour; use futures::prelude::*; use std::{error, time::Duration}; @@ -202,10 +204,21 @@ use std::{error, time::Duration}; /// > **Note**: This `Transport` is not suitable for production usage, as its implementation /// > reserves the right to support additional protocols or remove deprecated protocols. #[inline] -pub fn build_development_transport(local_private_key: secio::SecioKeyPair) - -> impl Transport + Send + Sync), Error = impl error::Error + Send, Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone -{ - build_tcp_ws_secio_mplex_yamux(local_private_key) +pub fn build_development_transport( + local_private_key: secio::SecioKeyPair, +) -> impl Transport< + Output = ( + PeerId, + impl core::muxing::StreamMuxer + + Send + + Sync, + ), + Error = impl error::Error + Send, + Listener = impl Send, + Dial = impl Send, + ListenerUpgrade = impl Send, +> + Clone { + build_tcp_ws_secio_mplex_yamux(local_private_key) } /// Builds an implementation of `Transport` that is suitable for usage with the `Swarm`. @@ -214,18 +227,32 @@ pub fn build_development_transport(local_private_key: secio::SecioKeyPair) /// and mplex or yamux as the multiplexing layer. /// /// > **Note**: If you ever need to express the type of this `Transport`. -pub fn build_tcp_ws_secio_mplex_yamux(local_private_key: secio::SecioKeyPair) - -> impl Transport + Send + Sync), Error = impl error::Error + Send, Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone -{ +pub fn build_tcp_ws_secio_mplex_yamux( + local_private_key: secio::SecioKeyPair, +) -> impl Transport< + Output = ( + PeerId, + impl core::muxing::StreamMuxer + + Send + + Sync, + ), + Error = impl error::Error + Send, + Listener = impl Send, + Dial = impl Send, + ListenerUpgrade = impl Send, +> + Clone { CommonTransport::new() .with_upgrade(secio::SecioConfig::new(local_private_key)) .and_then(move |out, endpoint| { let peer_id = out.remote_key.into_peer_id(); let peer_id2 = peer_id.clone(); - let upgrade = core::upgrade::SelectUpgrade::new(yamux::Config::default(), mplex::MplexConfig::new()) - // TODO: use a single `.map` instead of two maps - .map_inbound(move |muxer| (peer_id, muxer)) - .map_outbound(move |muxer| (peer_id2, muxer)); + let upgrade = core::upgrade::SelectUpgrade::new( + yamux::Config::default(), + mplex::MplexConfig::new(), + ) + // TODO: use a single `.map` instead of two maps + .map_inbound(move |muxer| (peer_id, muxer)) + .map_outbound(move |muxer| (peer_id2, muxer)); core::upgrade::apply(out.stream, upgrade, endpoint) .map(|(id, muxer)| (id, core::muxing::StreamMuxerBox::new(muxer))) @@ -240,12 +267,21 @@ pub fn build_tcp_ws_secio_mplex_yamux(local_private_key: secio::SecioKeyPair) #[derive(Debug, Clone)] struct CommonTransport { // The actual implementation of everything. - inner: CommonTransportInner + inner: CommonTransportInner, } -#[cfg(all(not(any(target_os = "emscripten", target_os = "unknown")), feature = "libp2p-websocket"))] -type InnerImplementation = core::transport::OrTransport, websocket::WsConfig>>; -#[cfg(all(not(any(target_os = "emscripten", target_os = "unknown")), not(feature = "libp2p-websocket")))] +#[cfg(all( + not(any(target_os = "emscripten", target_os = "unknown")), + feature = "libp2p-websocket" +))] +type InnerImplementation = core::transport::OrTransport< + dns::DnsConfig, + websocket::WsConfig>, +>; +#[cfg(all( + not(any(target_os = "emscripten", target_os = "unknown")), + not(feature = "libp2p-websocket") +))] type InnerImplementation = dns::DnsConfig; #[cfg(any(target_os = "emscripten", target_os = "unknown"))] type InnerImplementation = websocket::BrowserWsConfig; @@ -269,7 +305,7 @@ impl CommonTransport { }; CommonTransport { - inner: CommonTransportInner { inner: transport } + inner: CommonTransportInner { inner: transport }, } } @@ -279,7 +315,7 @@ impl CommonTransport { pub fn new() -> CommonTransport { let inner = websocket::BrowserWsConfig::new(); CommonTransport { - inner: CommonTransportInner { inner } + inner: CommonTransportInner { inner }, } } } @@ -292,7 +328,10 @@ impl Transport for CommonTransport { type Dial = ::Dial; #[inline] - fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError> { + fn listen_on( + self, + addr: Multiaddr, + ) -> Result<(Self::Listener, Multiaddr), TransportError> { self.inner.inner.listen_on(addr) } From 8cdaf6bffb2e384af361bd1820da46759592b26f Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 24 Jan 2019 15:58:24 +1100 Subject: [PATCH 21/91] Add logging to gossipsub - Adds the log crate and implements logging macros - Specifies versions for external crates --- protocols/gossipsub/Cargo.toml | 29 ++++--- protocols/gossipsub/src/layer.rs | 145 ++++++++++++++++++++++++++----- protocols/gossipsub/src/lib.rs | 2 + 3 files changed, 142 insertions(+), 34 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 48ab9fcfbaa..fc7371f688b 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -7,21 +7,24 @@ license = "MIT" [dependencies] libp2p-core = { path = "../../core" } libp2p-floodsub = { path = "../floodsub" } -bs58 = "" -bytes = "" -byteorder = "" -cuckoofilter = "" -fnv = "" -futures = "" -protobuf = "" -rand = "" -smallvec = "" -tokio-codec = "" -tokio-io = "" -tokio-timer = "" -unsigned-varint = "" +bs58 = "0.2.2" +bytes = "0.4.11" +byteorder = "1.3.1" +cuckoofilter = "0.3.2" +fnv = "1.0.6" +futures = "0.1.25" +protobuf = "2.2.5" +rand = "0.6.4" +smallvec = "0.6.8" +tokio-codec = "0.1.1" +tokio-io = "0.1.11" +tokio-timer = "0.2.8" +unsigned-varint = "0.2.1" +log = "0.4.6" + [dev-dependencies] libp2p = { path = "../../" } tokio = "0.1" tokio-stdin-stdout = "0.1" +env_logger = "0.6.0" diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 5c34191fcbe..105aa28331e 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -188,13 +188,16 @@ impl Gossipsub { /// /// Returns true if the subscription worked. Returns false if we were already subscribed. pub fn subscribe(&mut self, topic: Topic) -> bool { + debug!("Subscribing to topic: {:?}", topic); if self.mesh.get(&topic.hash()).is_some() { + debug!("Topic: {:?} is already in the mesh.", topic); return false; } // send subscription request to all floodsub and gossipsub peers for (flood_peers, gossip_peers) in self.topic_peers.values() { for peer in flood_peers.iter().chain(gossip_peers) { + debug!("Sending SUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { @@ -211,8 +214,8 @@ impl Gossipsub { // call JOIN(topic) // this will add new peers to the mesh for the topic - self.join(topic); - + self.join(topic.clone()); + info!("Subscribed to topic: {:?}", topic); true } @@ -223,8 +226,10 @@ impl Gossipsub { /// Returns true if we were subscribed to this topic. pub fn unsubscribe(&mut self, topic: impl AsRef) -> bool { let topic_hash = topic.as_ref(); + debug!("Unsubscribing from topic: {:?}", topic_hash); if self.mesh.get(topic_hash).is_none() { + debug!("Already unsubscribed to topic: {:?}", topic_hash); // we are not subscribed return false; } @@ -232,6 +237,7 @@ impl Gossipsub { // announce to all floodsub and gossipsub peers for (flood_peers, gossip_peers) in self.topic_peers.values() { for peer in flood_peers.iter().chain(gossip_peers) { + debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { @@ -250,6 +256,7 @@ impl Gossipsub { // this will remove the topic from the mesh self.leave(&topic); + info!("Unsubscribed from topic: {:?}", topic_hash); true } @@ -277,6 +284,8 @@ impl Gossipsub { topics: topic.into_iter().map(|t| t.into().clone()).collect(), }; + debug!("Publishing message: {:?}", message.msg_id()); + // forward the message to mesh and floodsub peers let local_peer_id = self.local_peer_id.clone(); self.forward_msg(message.clone(), local_peer_id); @@ -285,6 +294,7 @@ impl Gossipsub { for topic_hash in &message.topics { // if not subscribed to the topic, use fanout peers if self.mesh.get(&topic_hash).is_none() { + debug!("Topic: {:?} not in the mesh", topic_hash); // build a list of peers to forward the message to // if we have fanout peers add them to the map if let Some(fanout_peers) = self.fanout.get(&topic_hash) { @@ -300,6 +310,7 @@ impl Gossipsub { // add the new peers to the fanout and recipient peers self.fanout.insert(topic_hash.clone(), new_peers.clone()); for peer in new_peers { + debug!("Peer added to fanout: {:?}", peer); recipient_peers.insert(peer.clone(), ()); } } @@ -314,7 +325,7 @@ impl Gossipsub { // Send to peers we know are subscribed to the topic. for peer_id in recipient_peers.keys() { - println!("peers subscribed? {:?}", peer_id); + debug!("Sending message to peer: {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { @@ -324,14 +335,17 @@ impl Gossipsub { }, }); } + info!("Published message: {:?}", message.msg_id()); } /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. fn join(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); + debug!("Running JOIN for topic: {:?}", topic_hash); // if we are already in the mesh, return if self.mesh.contains_key(topic_hash) { + info!("JOIN: The topic is already in the mesh, ignoring JOIN"); return; } @@ -339,6 +353,10 @@ impl Gossipsub { // check if we have peers in fanout[topic] and remove them if we do if let Some((_, peers)) = self.fanout.remove_entry(topic_hash) { + debug!( + "JOIN: Removing peers from the fanout for topic: {:?}", + topic_hash + ); // add them to the mesh self.mesh.insert(topic_hash.clone(), peers.clone()); // remove the last published time @@ -348,12 +366,16 @@ impl Gossipsub { let mesh_n = self.config.mesh_n; peers = self.get_random_peers(topic_hash, mesh_n, { |_| true }); // put them in the mesh + debug!( + "JOIN: Inserting {:?} random peers into the mesh", + peers.len() + ); self.mesh.insert(topic_hash.clone(), peers.clone()); } for peer_id in peers { // Send a GRAFT control message - println!("Graft message sent to peer: {:?}", peer_id); + info!("JOIN: Sending Graft message to peer: {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { @@ -366,19 +388,21 @@ impl Gossipsub { }); //TODO: tagPeer } + debug!("Completed JOIN for topic: {:?}", topic_hash); } /// Gossipsub LEAVE(topic) - Notifies mesh[topic] peers with PRUNE messages. fn leave(&mut self, topic: impl AsRef) { let topic_hash = topic.as_ref(); + debug!("Running LEAVE for topic {:?}", topic_hash); // if our mesh contains the topic, send prune to peers and delete it from the mesh if let Some((_, peers)) = self.mesh.remove_entry(topic_hash) { - for peer_id in peers { + for peer in peers { // Send a PRUNE control message - println!("Prune message sent to peer: {:?}", peer_id); + info!("LEAVE: Sending PRUNE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer_id.clone(), + peer_id: peer.clone(), event: GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), @@ -390,17 +414,23 @@ impl Gossipsub { //TODO: untag Peer } } + debug!("Completed LEAVE for topic: {:?}", topic_hash); } /// Handles an IHAVE control message. Checks our cache of messages. If the message is unknown, /// requests it with an IWANT control message. fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { + debug!("Handling IHAVE for peer: {:?}", peer_id); // use a hashmap to avoid duplicates efficiently let mut iwant_msg_ids = HashMap::new(); for (topic, msg_ids) in ihave_msgs { // only process the message if we are subscribed if !self.mesh.contains_key(&topic) { + info!( + "IHAVE: Ignoring IHAVE - Not subscribed to topic: {:?}", + topic + ); return; // continue } @@ -414,22 +444,25 @@ impl Gossipsub { if !iwant_msg_ids.is_empty() { // Send the list of IWANT control messages + info!("IHAVE: Sending IWANT message"); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: vec![GossipsubControlAction::IWant { - message_ids: iwant_msg_ids.keys().map(|msg_id| msg_id.clone()).collect(), + message_ids: iwant_msg_ids.keys().cloned().collect(), }], }, }); } + debug!("Completed IHAVE handling for peer: {:?}", peer_id); } /// Handles an IWANT control message. Checks our cache of messages. If the message exists it is /// forwarded to the requesting peer. fn handle_iwant(&mut self, peer_id: &PeerId, iwant_msgs: Vec) { + debug!("Handling IWANT for peer: {:?}", peer_id); // build a hashmap of available messages let mut cached_messages = HashMap::new(); @@ -441,8 +474,9 @@ impl Gossipsub { } if !cached_messages.is_empty() { + info!("IWANT: Sending cached messages to peer: {:?}", peer_id); // Send the messages to the peer - let message_list = cached_messages.values().map(|msg| msg.clone()).collect(); + let message_list = cached_messages.values().cloned().collect(); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { @@ -452,16 +486,22 @@ impl Gossipsub { }, }); } + debug!("Completed IWANT handling for peer: {:?}", peer_id); } /// Handles GRAFT control messages. If subscribed to the topic, adds the peer to mesh, if not, responds /// with PRUNE messages. fn handle_graft(&mut self, peer_id: &PeerId, topics: Vec) { + debug!("Handling GRAFT message for peer: {:?}", peer_id); + let mut to_prune_topics = HashMap::new(); for topic_hash in topics { if let Some(peers) = self.mesh.get_mut(&topic_hash) { // if we are subscribed, add peer to the mesh - println!("GRAFT: Mesh link added from {:?}", peer_id); + info!( + "GRAFT: Mesh link added for peer: {:?} in topic: {:?}", + peer_id, topic_hash + ); peers.push(peer_id.clone()); //TODO: tagPeer } else { @@ -473,11 +513,15 @@ impl Gossipsub { // build the prune messages to send let prune_messages = to_prune_topics .keys() - .map(|topic_hash| GossipsubControlAction::Prune { - topic_hash: topic_hash.clone(), + .map(|t| GossipsubControlAction::Prune { + topic_hash: t.clone(), }) .collect(); // Send the prune messages to the peer + info!( + "GRAFT: Not subscribed to topics - Sending PRUNE to peer: {:?}", + peer_id + ); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { @@ -487,28 +531,42 @@ impl Gossipsub { }, }); } + debug!("Completed GRAFT handling for peer: {:?}", peer_id); } /// Handles PRUNE control messages. Removes peer from the mesh. fn handle_prune(&mut self, peer_id: &PeerId, topics: Vec) { + debug!("Handling PRUNE message for peer: {:?}", peer_id); for topic_hash in topics { if let Some(peers) = self.mesh.get_mut(&topic_hash) { // remove the peer if it exists in the mesh - if let Some(pos) = peers.iter().position(|p| p == peer_id) { - peers.remove(pos); - //TODO: untagPeer - } + info!( + "PRUNE: Removing peer: {:?} from the mesh for topic: {:?}", + peer_id, topic_hash + ); + peers.retain(|p| p != peer_id); + //TODO: untagPeer } } + debug!("Completed PRUNE handling for peer: {:?}", peer_id); } /// Handles a newly received GossipsubMessage. /// Forwards the message to all floodsub peers and peers in the mesh. fn handle_received_message(&mut self, msg: GossipsubMessage, propagation_source: &PeerId) { + debug!( + "Handling message: {:?} from peer: {:?}", + msg.msg_id(), + propagation_source + ); // if we have seen this message, ignore it // there's a 3% chance this is a false positive // TODO: Check this has no significant emergent behaviour if !self.received.test_and_add(&msg.msg_id()) { + info!( + "Message already received, ignoring. Message: {:?}", + msg.msg_id() + ); return; } @@ -517,13 +575,15 @@ impl Gossipsub { // dispatch the message to the user if self.mesh.keys().any(|t| msg.topics.iter().any(|u| t == u)) { + debug!("Sending received message to poll"); self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Message(msg.clone()), )); } // forward the message to floodsub and mesh peers - self.forward_msg(msg, propagation_source.clone()); + self.forward_msg(msg.clone(), propagation_source.clone()); + debug!("Completed message handling for message: {:?}", msg.msg_id()); } /// Handles received subscriptions. @@ -532,13 +592,14 @@ impl Gossipsub { subscriptions: Vec, propagation_source: &PeerId, ) { + debug!( + "Handling subscriptions from source: {:?}", + propagation_source + ); let (subscribed_topics, node_type) = match self.peer_topics.get_mut(&propagation_source) { Some((topics, node_type)) => (topics, node_type), None => { - println!( - "ERROR: Subscription by unknown peer: {:?}", - &propagation_source - ); + error!("Subscription by unknown peer: {:?}", &propagation_source); return; } }; @@ -555,17 +616,23 @@ impl Gossipsub { match node_type { NodeType::Floodsub => { if !flood_peers.contains(&propagation_source) { + debug!("SUBSCRIPTION: topic_peer: Adding floodsub peer: {:?} to topic: {:?}", propagation_source, subscription.topic_hash); flood_peers.push(propagation_source.clone()); } } NodeType::Gossipsub => { if !gossip_peers.contains(&propagation_source) { + debug!("SUBSCRIPTION: topic_peer: Adding gossip peer: {:?} to topic: {:?}", propagation_source, subscription.topic_hash); gossip_peers.push(propagation_source.clone()); } } } // add to the peer_topics mapping if !subscribed_topics.contains(&subscription.topic_hash) { + info!( + "SUBSCRIPTION: Adding peer: {:?} to topic: {:?}", + propagation_source, subscription.topic_hash + ); subscribed_topics.push(subscription.topic_hash.clone()); } // generates a subscription event to be polled @@ -582,6 +649,10 @@ impl Gossipsub { if let Some(pos) = flood_peers.iter().position(|p| p == propagation_source) { + info!( + "SUBSCRIPTION: Removing floodsub peer: {:?} from topic: {:?}", + propagation_source, subscription.topic_hash + ); flood_peers.remove(pos); } } @@ -589,6 +660,10 @@ impl Gossipsub { if let Some(pos) = gossip_peers.iter().position(|p| p == propagation_source) { + info!( + "SUBSCRIPTION: Removing gossip peer: {:?} from topic: {:?}", + propagation_source, subscription.topic_hash + ); gossip_peers.remove(pos); } } @@ -610,10 +685,16 @@ impl Gossipsub { } } } + debug!( + "Completed handling subscriptions from source: {:?}", + propagation_source + ); } /// Heartbeat function which shifts the memcache and updates the mesh fn heartbeat(&mut self) { + debug!("Starting heartbeat"); + //TODO: Clean up any state from last heartbeat. let mut to_graft = HashMap::new(); @@ -623,6 +704,7 @@ impl Gossipsub { for (topic_hash, peers) in self.mesh.clone().iter_mut() { // too little peers - add some if peers.len() < self.config.mesh_n_low { + debug!("HEARTBEAT: Mesh low"); // not enough peers - get mesh_n - current_length more let desired_peers = self.config.mesh_n - peers.len(); let peer_list = self @@ -639,6 +721,7 @@ impl Gossipsub { // too many peers - remove some if peers.len() > self.config.mesh_n_high { + debug!("HEARTBEAT: Mesh high"); let excess_peer_no = peers.len() - self.config.mesh_n; // shuffle the peers let mut rng = thread_rng(); @@ -666,6 +749,10 @@ impl Gossipsub { let fanout_ttl = self.config.fanout_ttl; self.fanout_last_pub.retain(|topic_hash, last_pub_time| { if *last_pub_time + fanout_ttl < Instant::now() { + debug!( + "HEARTBEAT: Fanout topic removed due to timeout. Topic: {:?}", + topic_hash + ); fanout.remove(&topic_hash); return false; } @@ -685,6 +772,10 @@ impl Gossipsub { .0 .contains(&topic_hash) { + debug!( + "HEARTBEAT: Peer removed from fanout for topic: {:?}", + topic_hash + ); return false; } true @@ -692,6 +783,7 @@ impl Gossipsub { // not enough peers if peers.len() < self.config.mesh_n { + debug!("HEARTBEAT: Fanout low peers"); let needed_peers = self.config.mesh_n - peers.len(); let mut new_peers = self.get_random_peers(topic_hash, needed_peers, |peer| !peers.contains(peer)); @@ -707,11 +799,14 @@ impl Gossipsub { self.send_graft_prune(to_graft, to_prune); // shift the memcache + debug!("HEARTBEAT: Memcache shifted"); self.mcache.shift(); + debug!("Completed Heartbeat"); } /// Emits gossip - Send IHAVE messages to a random set of gossip peers that are not in the mesh, but are subscribed to the `topic`. fn emit_gossip(&mut self, topic_hash: TopicHash, peers: Vec) { + debug!("Started gossip"); let message_ids = self.mcache.get_gossip_ids(&topic_hash); if message_ids.is_empty() { return; @@ -735,6 +830,7 @@ impl Gossipsub { }, }); } + debug!("Completed gossip"); } /// Handles multiple GRAFT/PRUNE messages and coalesces them into chunked gossip control @@ -792,10 +888,12 @@ impl Gossipsub { }, }); } + debug!("Completed gossip"); } /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { + debug!("Forwarding message: {:?}", message.msg_id()); let mut recipient_peers = HashMap::new(); // add floodsub and mesh peers @@ -822,6 +920,7 @@ impl Gossipsub { // forward the message to peers if !recipient_peers.is_empty() { for peer in recipient_peers.keys() { + debug!("Sending message: {:?} to peer {:?}", message.msg_id(), peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { @@ -832,6 +931,7 @@ impl Gossipsub { }); } } + debug!("Completed forwarding message"); } /// Helper function to get a set of `n` random gossipsub peers for a `topic_hash` @@ -850,6 +950,7 @@ impl Gossipsub { // if we have less than needed, return them if gossip_peers.len() <= n { + debug!("RANDOM PEERS: Got {:?} peers", gossip_peers.len()); return gossip_peers.to_vec(); } @@ -857,6 +958,8 @@ impl Gossipsub { let mut rng = thread_rng(); gossip_peers.partial_shuffle(&mut rng, n); + debug!("RANDOM PEERS: Got {:?} peers", gossip_peers.len()); + return gossip_peers[..n].to_vec(); } } diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index cb4f8e67bf1..96fcffedc2c 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -28,6 +28,8 @@ extern crate libp2p_core; extern crate libp2p_floodsub; extern crate protobuf; extern crate rand; +#[macro_use] +extern crate log; extern crate smallvec; extern crate tokio_codec; extern crate tokio_io; From deccbe5da2a6d6b8c7cfd2f6e0143b3e6383b95d Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 24 Jan 2019 16:00:43 +1100 Subject: [PATCH 22/91] Add example chat for debugging purposes --- protocols/gossipsub/examples/chat.rs | 88 ++++++++++++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 protocols/gossipsub/examples/chat.rs diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs new file mode 100644 index 00000000000..8c59cc606a6 --- /dev/null +++ b/protocols/gossipsub/examples/chat.rs @@ -0,0 +1,88 @@ +extern crate env_logger; +extern crate futures; +extern crate libp2p; +extern crate log; +extern crate tokio; + +use env_logger::{Builder, Env}; +use futures::prelude::*; +use libp2p::gossipsub::GossipsubEvent; +use libp2p::{ + gossipsub, secio, + tokio_codec::{FramedRead, LinesCodec}, +}; + +fn main() { + Builder::from_env(Env::default().default_filter_or("debug")).init(); + + // Create a random PeerId + let local_key = secio::SecioKeyPair::ed25519_generated().unwrap(); + let local_pub_key = local_key.to_public_key(); + println!("Local peer id: {:?}", local_pub_key.clone().into_peer_id()); + + // Set up a an encrypted DNS-enabled TCP Transport over the Mplex and Yamux protocols + let transport = libp2p::build_development_transport(local_key); + + // Create a Floodsub/Gossipsub topic + let topic = libp2p::floodsub::TopicBuilder::new("test").build(); + + // Create a Swarm to manage peers and events + let mut swarm = { + // set default parameters for gossipsub + let gossipsub_config = 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( + transport, + gossipsub, + libp2p::core::topology::MemoryTopology::empty(local_pub_key), + ) + }; + + // 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(); + println!("Listening on {:?}", addr); + + // Reach out to another node if specified + if let Some(to_dial) = std::env::args().nth(1) { + let dialing = to_dial.clone(); + match to_dial.parse() { + Ok(to_dial) => match libp2p::Swarm::dial_addr(&mut swarm, to_dial) { + Ok(_) => println!("Dialed {:?}", dialing), + Err(e) => println!("Dial {:?} failed: {:?}", dialing, e), + }, + Err(err) => println!("Failed to parse address to dial: {:?}", err), + } + } + + // Read full lines from stdin + let stdin = tokio_stdin_stdout::stdin(0); + let mut framed_stdin = FramedRead::new(stdin, LinesCodec::new()); + + // Kick it off + tokio::run(futures::future::poll_fn(move || -> Result<_, ()> { + loop { + match framed_stdin.poll().expect("Error while polling stdin") { + Async::Ready(Some(line)) => swarm.publish(&topic, line.as_bytes()), + Async::Ready(None) => panic!("Stdin closed"), + Async::NotReady => break, + }; + } + + loop { + match swarm.poll().expect("Error while polling swarm") { + Async::Ready(Some(gossip_event)) => match gossip_event { + GossipsubEvent::Message(message) => { + println!("Got message: {:?}", String::from_utf8_lossy(&message.data)) + } + _ => {} + }, + Async::Ready(None) | Async::NotReady => break, + } + } + + Ok(Async::NotReady) + })); +} From e69a772c16309a6dfb3985031b126ee371c2627f Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 24 Jan 2019 16:25:20 +1100 Subject: [PATCH 23/91] Implement #868 for gossipsub. --- protocols/gossipsub/src/handler.rs | 257 ---------------------------- protocols/gossipsub/src/layer.rs | 42 ++++- protocols/gossipsub/src/lib.rs | 2 - protocols/gossipsub/src/protocol.rs | 245 ++++++++++++++------------ 4 files changed, 168 insertions(+), 378 deletions(-) delete mode 100644 protocols/gossipsub/src/handler.rs diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs deleted file mode 100644 index d466ae8c83c..00000000000 --- a/protocols/gossipsub/src/handler.rs +++ /dev/null @@ -1,257 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -//TODO: Remove this handler and replace with OneShotHandler once PR #868 is merged - -use crate::protocol::{GossipsubCodec, GossipsubRpc, ProtocolConfig}; -use futures::prelude::*; -use libp2p_core::{ - protocols_handler::ProtocolsHandlerUpgrErr, - upgrade::{InboundUpgrade, OutboundUpgrade}, - ProtocolsHandler, ProtocolsHandlerEvent, -}; -use smallvec::SmallVec; -use std::{fmt, io}; -use tokio_codec::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; - -/// Protocol handler that handles communication with the remote for the gossipsub protocol. -/// -/// The handler will automatically open a substream with the remote for each request we make. -/// -/// It also handles requests made by the remote. -pub struct GossipsubHandler -where - TSubstream: AsyncRead + AsyncWrite, -{ - /// Configuration for the gossipsub protocol. - config: ProtocolConfig, - - /// If true, we are trying to shut down the existing gossipsub substream and should refuse any - /// incoming connection. - shutting_down: bool, - - /// The active substreams. - // TODO: add a limit to the number of allowed substreams - substreams: Vec>, - - /// Queue of values that we want to send to the remote. - send_queue: SmallVec<[GossipsubRpc; 16]>, -} - -/// State of an active substream, opened either by us or by the remote. -enum SubstreamState -where - TSubstream: AsyncRead + AsyncWrite, -{ - /// Waiting for a message from the remote. - WaitingInput(Framed), - /// Waiting to send a message to the remote. - PendingSend(Framed, GossipsubRpc), - /// Waiting to flush the substream so that the data arrives to the remote. - PendingFlush(Framed), - /// The substream is being closed. - Closing(Framed), -} - -impl SubstreamState -where - TSubstream: AsyncRead + AsyncWrite, -{ - /// Consumes this state and produces the substream. - fn into_substream(self) -> Framed { - match self { - SubstreamState::WaitingInput(substream) => substream, - SubstreamState::PendingSend(substream, _) => substream, - SubstreamState::PendingFlush(substream) => substream, - SubstreamState::Closing(substream) => substream, - } - } -} - -impl GossipsubHandler -where - TSubstream: AsyncRead + AsyncWrite, -{ - /// Builds a new `GossipsubHandler`. - pub fn new() -> Self { - GossipsubHandler { - config: ProtocolConfig::new(), - shutting_down: false, - substreams: Vec::new(), - send_queue: SmallVec::new(), - } - } -} - -impl ProtocolsHandler for GossipsubHandler -where - TSubstream: AsyncRead + AsyncWrite, -{ - type InEvent = GossipsubRpc; - type OutEvent = GossipsubRpc; - type Error = io::Error; - type Substream = TSubstream; - type InboundProtocol = ProtocolConfig; - type OutboundProtocol = ProtocolConfig; - type OutboundOpenInfo = GossipsubRpc; - - #[inline] - fn listen_protocol(&self) -> Self::InboundProtocol { - self.config.clone() - } - - fn inject_fully_negotiated_inbound( - &mut self, - protocol: >::Output, - ) { - if self.shutting_down { - return (); - } - self.substreams.push(SubstreamState::WaitingInput(protocol)) - } - - fn inject_fully_negotiated_outbound( - &mut self, - protocol: >::Output, - message: Self::OutboundOpenInfo, - ) { - if self.shutting_down { - return (); - } - self.substreams - .push(SubstreamState::PendingSend(protocol, message)) - } - - #[inline] - fn inject_event(&mut self, message: GossipsubRpc) { - self.send_queue.push(message); - } - - #[inline] - fn inject_inbound_closed(&mut self) {} - - #[inline] - fn inject_dial_upgrade_error( - &mut self, - _: Self::OutboundOpenInfo, - _: ProtocolsHandlerUpgrErr< - >::Error, - >, - ) { - } - - #[inline] - fn connection_keep_alive(&self) -> bool { - !self.substreams.is_empty() - } - - #[inline] - fn shutdown(&mut self) { - self.shutting_down = true; - for n in (0..self.substreams.len()).rev() { - let mut substream = self.substreams.swap_remove(n); - self.substreams - .push(SubstreamState::Closing(substream.into_substream())); - } - } - - fn poll( - &mut self, - ) -> Poll< - ProtocolsHandlerEvent, - io::Error, - > { - if !self.send_queue.is_empty() { - let message = self.send_queue.remove(0); - return Ok(Async::Ready( - ProtocolsHandlerEvent::OutboundSubstreamRequest { - info: message, - upgrade: self.config.clone(), - }, - )); - } - - for n in (0..self.substreams.len()).rev() { - let mut substream = self.substreams.swap_remove(n); - loop { - substream = match substream { - SubstreamState::WaitingInput(mut substream) => match substream.poll() { - Ok(Async::Ready(Some(message))) => { - self.substreams - .push(SubstreamState::WaitingInput(substream)); - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(message))); - } - Ok(Async::Ready(None)) => SubstreamState::Closing(substream), - Ok(Async::NotReady) => { - self.substreams - .push(SubstreamState::WaitingInput(substream)); - return Ok(Async::NotReady); - } - Err(_) => SubstreamState::Closing(substream), - }, - SubstreamState::PendingSend(mut substream, message) => { - match substream.start_send(message)? { - AsyncSink::Ready => SubstreamState::PendingFlush(substream), - AsyncSink::NotReady(message) => { - self.substreams - .push(SubstreamState::PendingSend(substream, message)); - return Ok(Async::NotReady); - } - } - } - SubstreamState::PendingFlush(mut substream) => { - match substream.poll_complete()? { - Async::Ready(()) => SubstreamState::Closing(substream), - Async::NotReady => { - self.substreams - .push(SubstreamState::PendingFlush(substream)); - return Ok(Async::NotReady); - } - } - } - SubstreamState::Closing(mut substream) => match substream.close() { - Ok(Async::Ready(())) => break, - Ok(Async::NotReady) => { - self.substreams.push(SubstreamState::Closing(substream)); - return Ok(Async::NotReady); - } - Err(_) => return Ok(Async::Ready(ProtocolsHandlerEvent::Shutdown)), - }, - } - } - } - - Ok(Async::NotReady) - } -} - -impl fmt::Debug for GossipsubHandler -where - TSubstream: AsyncRead + AsyncWrite, -{ - fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { - f.debug_struct("GossipsubHandler") - .field("shutting_down", &self.shutting_down) - .field("substreams", &self.substreams.len()) - .field("send_queue", &self.send_queue.len()) - .finish() - } -} diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 105aa28331e..916f9ebcaec 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -23,16 +23,18 @@ use cuckoofilter::CuckooFilter; use futures::prelude::*; -use handler::GossipsubHandler; use libp2p_core::swarm::{ ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters, }; -use libp2p_core::{protocols_handler::ProtocolsHandler, PeerId}; +use libp2p_core::{ + protocols_handler::{OneShotHandler, ProtocolsHandler}, + PeerId, +}; use libp2p_floodsub::{Topic, TopicHash}; use mcache::MessageCache; use protocol::{ GossipsubControlAction, GossipsubMessage, GossipsubRpc, GossipsubSubscription, - GossipsubSubscriptionAction, + GossipsubSubscriptionAction, ProtocolConfig, }; use rand; use rand::{seq::SliceRandom, thread_rng}; @@ -968,11 +970,11 @@ impl NetworkBehaviour for Gossipsub; + type ProtocolsHandler = OneShotHandler; type OutEvent = GossipsubEvent; fn new_handler(&mut self) -> Self::ProtocolsHandler { - GossipsubHandler::new() + Default::default() } fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { @@ -1071,7 +1073,12 @@ where debug_assert!(was_in.is_some()); } - fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { + fn inject_node_event(&mut self, propagation_source: PeerId, event: InnerMessage) { + // ignore successful sends event + let event = match event { + InnerMessage::Rx(event) => event, + InnerMessage::Sent => return, + }; // Handle subscriptions // Update connected peers topics self.handle_received_subscriptions(event.subscriptions, &propagation_source); @@ -1135,6 +1142,29 @@ where Async::NotReady } } + +/// Transmission between the `OneShotHandler` and the `GossipsubRpc`. +pub enum InnerMessage { + /// We received an RPC from a remote. + Rx(GossipsubRpc), + /// We successfully sent an RPC request. + Sent, +} + +impl From for InnerMessage { + #[inline] + fn from(rpc: GossipsubRpc) -> InnerMessage { + InnerMessage::Rx(rpc) + } +} + +impl From<()> for InnerMessage { + #[inline] + fn from(_: ()) -> InnerMessage { + InnerMessage::Sent + } +} + /// Event that can happen on the gossipsub behaviour. #[derive(Debug)] pub enum GossipsubEvent { diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 96fcffedc2c..a5f283f6bd1 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -36,13 +36,11 @@ extern crate tokio_io; extern crate tokio_timer; extern crate unsigned_varint; -pub mod handler; pub mod protocol; mod layer; mod mcache; mod rpc_proto; -pub use self::handler::GossipsubHandler; pub use self::layer::{Gossipsub, GossipsubConfig, GossipsubEvent}; pub use self::protocol::*; diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 7f2d876b4f3..6101655e875 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -20,18 +20,18 @@ use crate::rpc_proto; use byteorder::{BigEndian, ByteOrder}; -use bytes::{BufMut, BytesMut}; -use futures::future; +use bytes::BytesMut; +use futures::{future, stream, Future, Stream}; use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; use libp2p_floodsub::TopicHash; use protobuf::Message as ProtobufMessage; use std::{io, iter}; -use tokio_codec::{Decoder, Encoder, Framed}; +use tokio_codec::{Decoder, FramedRead}; use tokio_io::{AsyncRead, AsyncWrite}; use unsigned_varint::codec; /// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Default)] pub struct ProtocolConfig {} impl ProtocolConfig { @@ -54,39 +54,35 @@ impl UpgradeInfo for ProtocolConfig { impl InboundUpgrade for ProtocolConfig where - TSocket: AsyncRead + AsyncWrite, + TSocket: AsyncRead, { - type Output = Framed; + type Output = GossipsubRpc; type Error = io::Error; - type Future = future::FutureResult; + type Future = future::MapErr< + future::AndThen< + stream::StreamFuture>, + Result)>, + fn( + (Option, FramedRead), + ) + -> Result)>, + >, + fn((io::Error, FramedRead)) -> io::Error, + >; #[inline] fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future { - future::ok(Framed::new( + FramedRead::new( socket, GossipsubCodec { length_prefix: Default::default(), }, - )) - } -} - -impl OutboundUpgrade for ProtocolConfig -where - TSocket: AsyncRead + AsyncWrite, -{ - type Output = Framed; - type Error = io::Error; - type Future = future::FutureResult; - - #[inline] - fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future { - future::ok(Framed::new( - socket, - GossipsubCodec { - length_prefix: Default::default(), - }, - )) + ) + .into_future() + .and_then:: _, _>(|(val, socket)| { + val.ok_or_else(move || (io::ErrorKind::UnexpectedEof.into(), socket)) + }) + .map_err(|(err, _)| err) } } @@ -96,90 +92,6 @@ pub struct GossipsubCodec { length_prefix: codec::UviBytes, } -impl Encoder for GossipsubCodec { - type Item = GossipsubRpc; - type Error = io::Error; - - fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { - let mut proto = rpc_proto::RPC::new(); - - for message in item.messages.into_iter() { - let mut msg = rpc_proto::Message::new(); - msg.set_from(message.source.into_bytes()); - msg.set_data(message.data); - msg.set_seqno(message.sequence_number); - msg.set_topicIDs( - message - .topics - .into_iter() - .map(TopicHash::into_string) - .collect(), - ); - proto.mut_publish().push(msg); - } - - for subscription in item.subscriptions.into_iter() { - let mut rpc_subscription = rpc_proto::RPC_SubOpts::new(); - rpc_subscription - .set_subscribe(subscription.action == GossipsubSubscriptionAction::Subscribe); - rpc_subscription.set_topicid(subscription.topic_hash.into_string()); - proto.mut_subscriptions().push(rpc_subscription); - } - - // gossipsub control messages - let mut control_msg = rpc_proto::ControlMessage::new(); - - for action in item.control_msgs { - match action { - // collect all ihave messages - GossipsubControlAction::IHave { - topic_hash, - message_ids, - } => { - let mut rpc_ihave = rpc_proto::ControlIHave::new(); - rpc_ihave.set_topicID(topic_hash.into_string()); - for msg_id in message_ids { - rpc_ihave.mut_messageIDs().push(msg_id); - } - control_msg.mut_ihave().push(rpc_ihave); - } - GossipsubControlAction::IWant { message_ids } => { - let mut rpc_iwant = rpc_proto::ControlIWant::new(); - for msg_id in message_ids { - rpc_iwant.mut_messageIDs().push(msg_id); - } - control_msg.mut_iwant().push(rpc_iwant); - } - GossipsubControlAction::Graft { topic_hash } => { - let mut rpc_graft = rpc_proto::ControlGraft::new(); - rpc_graft.set_topicID(topic_hash.into_string()); - control_msg.mut_graft().push(rpc_graft); - } - GossipsubControlAction::Prune { topic_hash } => { - let mut rpc_prune = rpc_proto::ControlPrune::new(); - rpc_prune.set_topicID(topic_hash.into_string()); - control_msg.mut_prune().push(rpc_prune); - } - } - } - - proto.set_control(control_msg); - - let msg_size = proto.compute_size(); - // Reserve enough space for the data and the length. The length has a maximum of 32 bits, - // which means that 5 bytes is enough for the variable-length integer. - dst.reserve(msg_size as usize + 5); - - proto - .write_length_delimited_to_writer(&mut dst.by_ref().writer()) - .expect( - "there is no situation in which the protobuf message can be invalid, and \ - writing to a BytesMut never fails as we reserved enough space beforehand", - ); - Ok(()) - } -} - impl Decoder for GossipsubCodec { type Item = GossipsubRpc; type Error = io::Error; @@ -203,7 +115,7 @@ impl Decoder for GossipsubCodec { topics: publish .take_topicIDs() .into_iter() - .map(|topic| TopicHash::from_raw(topic)) + .map(TopicHash::from_raw) .collect(), }); } @@ -285,6 +197,113 @@ pub struct GossipsubRpc { pub control_msgs: Vec, } +impl UpgradeInfo for GossipsubRpc { + type Info = &'static [u8]; + type InfoIter = iter::Once; + + #[inline] + fn protocol_info(&self) -> Self::InfoIter { + iter::once(b"/meshsub/1.0.0") + } +} + +impl OutboundUpgrade for GossipsubRpc +where + TSocket: AsyncWrite, +{ + type Output = (); + type Error = io::Error; + type Future = future::Map< + future::AndThen< + tokio_io::io::WriteAll>, + tokio_io::io::Shutdown, + fn((TSocket, Vec)) -> tokio_io::io::Shutdown, + >, + fn(TSocket) -> (), + >; + + #[inline] + fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future { + let bytes = self.into_length_delimited_bytes(); + tokio_io::io::write_all(socket, bytes) + .and_then:: _, _>(|(socket, _)| tokio_io::io::shutdown(socket)) + .map(|_| ()) + } +} + +impl GossipsubRpc { + /// Turns this `GossipsubRpc` into a message that can be sent to a substream. + fn into_length_delimited_bytes(self) -> Vec { + let mut proto = rpc_proto::RPC::new(); + + for message in self.messages.into_iter() { + let mut msg = rpc_proto::Message::new(); + msg.set_from(message.source.into_bytes()); + msg.set_data(message.data); + msg.set_seqno(message.sequence_number); + msg.set_topicIDs( + message + .topics + .into_iter() + .map(TopicHash::into_string) + .collect(), + ); + proto.mut_publish().push(msg); + } + + for subscription in self.subscriptions.into_iter() { + let mut rpc_subscription = rpc_proto::RPC_SubOpts::new(); + rpc_subscription + .set_subscribe(subscription.action == GossipsubSubscriptionAction::Subscribe); + rpc_subscription.set_topicid(subscription.topic_hash.into_string()); + proto.mut_subscriptions().push(rpc_subscription); + } + + // gossipsub control messages + let mut control_msg = rpc_proto::ControlMessage::new(); + + for action in self.control_msgs { + match action { + // collect all ihave messages + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => { + let mut rpc_ihave = rpc_proto::ControlIHave::new(); + rpc_ihave.set_topicID(topic_hash.into_string()); + for msg_id in message_ids { + rpc_ihave.mut_messageIDs().push(msg_id); + } + control_msg.mut_ihave().push(rpc_ihave); + } + GossipsubControlAction::IWant { message_ids } => { + let mut rpc_iwant = rpc_proto::ControlIWant::new(); + for msg_id in message_ids { + rpc_iwant.mut_messageIDs().push(msg_id); + } + control_msg.mut_iwant().push(rpc_iwant); + } + GossipsubControlAction::Graft { topic_hash } => { + let mut rpc_graft = rpc_proto::ControlGraft::new(); + rpc_graft.set_topicID(topic_hash.into_string()); + control_msg.mut_graft().push(rpc_graft); + } + GossipsubControlAction::Prune { topic_hash } => { + let mut rpc_prune = rpc_proto::ControlPrune::new(); + rpc_prune.set_topicID(topic_hash.into_string()); + control_msg.mut_prune().push(rpc_prune); + } + } + } + + proto.set_control(control_msg); + + proto + .write_length_delimited_to_bytes() + .expect("there is no situation in which the protobuf message can be invalid") + } +} + /// A message received by the gossipsub system. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct GossipsubMessage { From 3dad3ea44408f31edb184654392d174a07cbe1ab Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 28 Jan 2019 15:08:03 +1100 Subject: [PATCH 24/91] Add rust documentation to gossipsub crate. - Adds basic documentation, overview and examples to the gossipsub crate. --- protocols/gossipsub/src/lib.rs | 119 +++++++++++++++++++++++++++------ 1 file changed, 100 insertions(+), 19 deletions(-) diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index a5f283f6bd1..be4be1f81b1 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -1,22 +1,103 @@ -// Copyright 2018 Parity Technologies (UK) 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. +//! Gossipsub is a P2P pubsub (publish/subscription) routing layer designed to extend upon flooodsub +//! and meshsub routing protocols. +//! +//! # Overview +//! +//! *Note: The gossipsub protocol specifications (https://github.com/libp2p/specs/tree/master/pubsub/gossipsub) provide an outline for the routing protocol. They should be consulted for further detail.* +//! +//! Gossipsub is a blend of meshsub for data and randomsub for mesh metadata. It provides bounded +//! degree and amplification factor with the meshsub construction and augments it using gossip +//! propagation of metadata with the randomsub technique. +//! +//! The router maintains an overlay mesh network of peers on which to efficiently send messages and metadata. +//! Peers use control messages to broadcast and request known messages and subscribe/unsubscribe from topics in the mesh network. +//! +//! # Important Discrepancies +//! +//! This section outlines the current implementation's potential discrepancies from that of +//! other implementations, due to undefined elements in the current specification. +//! +//! - **Topics** - In gossipsub, topics are utf-8 string's that are `base58` encoded. These are of type +//! `TopicHash`. The current go implementation uses raw utf-8 strings. +//! - **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 integers. They are chosen at random in this implementation of gossipsub, but are sequential in the current go implementation. +//! +//! # Using Gossipsub +//! +//! ## GossipsubConfig +//! +//! The [`GossipsubConfig`] struct specifies various network performance/tuning configuration parameters. Specifically it specifies: +//! +//! [`GossipsubConfig`]: struct.GossipsubConfig.html + +//! - `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). +//! +//! 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`]. +//! +//! [`Gossipsub`]: struct.Gossipsub.html + +//! ## Example +//! +//! An example of initialising a gossipsub compatible swarm: +//! +//! ```ignore +//! #extern crate libp2p; +//! #extern crate futures; +//! #extern crate tokio; +//! #use libp2p::gossipsub::GossipsubEvent; +//! #use libp2p::{gossipsub, secio, +//! # tokio_codec::{FramedRead, LinesCodec}, +//! #}; +//! let local_key = secio::SecioKeyPair::ed25519_generated().unwrap(); +//! let local_pub_key = local_key.to_public_key(); +//! +//! // Set up an encrypted TCP Transport over the Mplex and Yamux protocols +//! let transport = libp2p::build_development_transport(local_key); +//! +//! // Create a Floodsub/Gossipsub topic +//! let topic = libp2p::floodsub::TopicBuilder::new("example").build(); +//! +//! // Create a Swarm to manage peers and events +//! let mut swarm = { +//! // set default parameters for gossipsub +//! let gossipsub_config = 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( +//! transport, +//! gossipsub, +//! libp2p::core::topology::MemoryTopology::empty(local_pub_key), +//! ) +//! }; +//! +//! // 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(); +//! println!("Listening on {:?}", addr); +//! ``` extern crate bs58; extern crate byteorder; From 77047c3da5fb372726dffdd0da8857065bc8ea04 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 28 Jan 2019 15:13:42 +1100 Subject: [PATCH 25/91] Re-introduce the initial heartbeat time config. This commit also adds the inject_connected test. --- protocols/gossipsub/examples/chat.rs | 17 ++- protocols/gossipsub/src/layer.rs | 149 +++++++++++++++++++++++---- 2 files changed, 146 insertions(+), 20 deletions(-) diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index 8c59cc606a6..a083a80a179 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -11,6 +11,7 @@ use libp2p::{ gossipsub, secio, tokio_codec::{FramedRead, LinesCodec}, }; +use std::time::Duration; fn main() { Builder::from_env(Env::default().default_filter_or("debug")).init(); @@ -20,7 +21,7 @@ fn main() { let local_pub_key = local_key.to_public_key(); println!("Local peer id: {:?}", local_pub_key.clone().into_peer_id()); - // Set up a an encrypted DNS-enabled TCP Transport over the Mplex and Yamux protocols + // Set up an encrypted TCP Transport over the Mplex and Yamux protocols let transport = libp2p::build_development_transport(local_key); // Create a Floodsub/Gossipsub topic @@ -29,7 +30,19 @@ fn main() { // 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 = gossipsub::GossipsubConfig::default(); + // set custom gossipsub + let gossipsub_config = gossipsub::GossipsubConfig::new( + 5, + 3, + 6, + 4, + 12, + 6, + Duration::from_secs(10), + Duration::from_secs(10), + Duration::from_secs(60), + ); // build a gossipsub network behaviour let mut gossipsub = gossipsub::Gossipsub::new(local_pub_key.clone().into_peer_id(), gossipsub_config); diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 916f9ebcaec..96f16dfa4ac 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -65,9 +65,8 @@ pub struct GossipsubConfig { /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). gossip_lazy: usize, - // Not applicable right now - // /// Initial delay in each heartbeat. - // heartbeat_initial_delay: Duration, + /// Initial delay in each heartbeat. + heartbeat_initial_delay: Duration, /// Time between each heartbeat. heartbeat_interval: Duration, /// Time to live for fanout peers. @@ -83,7 +82,7 @@ impl Default for GossipsubConfig { mesh_n_low: 4, mesh_n_high: 12, gossip_lazy: 6, // default to mesh_n - // heartbeat_initial_delay: Duration::from_millis(100), + heartbeat_initial_delay: Duration::from_secs(5), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), } @@ -98,7 +97,7 @@ impl GossipsubConfig { mesh_n_low: usize, mesh_n_high: usize, gossip_lazy: usize, - // heartbeat_initial_delay: Duration, + heartbeat_initial_delay: Duration, heartbeat_interval: Duration, fanout_ttl: Duration, ) -> GossipsubConfig { @@ -117,7 +116,7 @@ impl GossipsubConfig { mesh_n_low, mesh_n_high, gossip_lazy, - // heartbeat_initial_delay, + heartbeat_initial_delay, heartbeat_interval, fanout_ttl, } @@ -181,7 +180,10 @@ impl Gossipsub { fanout_last_pub: HashMap::new(), mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), received: CuckooFilter::new(), - heartbeat: Interval::new_interval(gs_config.heartbeat_interval), + heartbeat: Interval::new( + Instant::now() + gs_config.heartbeat_initial_delay, + gs_config.heartbeat_interval, + ), marker: PhantomData, } } @@ -706,7 +708,12 @@ impl Gossipsub { for (topic_hash, peers) in self.mesh.clone().iter_mut() { // too little peers - add some if peers.len() < self.config.mesh_n_low { - debug!("HEARTBEAT: Mesh low"); + debug!( + "HEARTBEAT: Mesh low. Topic: {:?} Contains: {:?} needs: {:?}", + topic_hash.clone().into_string(), + peers.len(), + self.config.mesh_n_low + ); // not enough peers - get mesh_n - current_length more let desired_peers = self.config.mesh_n - peers.len(); let peer_list = self @@ -723,7 +730,12 @@ impl Gossipsub { // too many peers - remove some if peers.len() > self.config.mesh_n_high { - debug!("HEARTBEAT: Mesh high"); + debug!( + "HEARTBEAT: Mesh high. Topic: {:?} Contains: {:?} needs: {:?}", + topic_hash, + peers.len(), + self.config.mesh_n_high + ); let excess_peer_no = peers.len() - self.config.mesh_n; // shuffle the peers let mut rng = thread_rng(); @@ -785,7 +797,11 @@ impl Gossipsub { // not enough peers if peers.len() < self.config.mesh_n { - debug!("HEARTBEAT: Fanout low peers"); + debug!( + "HEARTBEAT: Fanout low. Contains: {:?} needs: {:?}", + peers.len(), + self.config.mesh_n + ); let needed_peers = self.config.mesh_n - peers.len(); let mut new_peers = self.get_random_peers(topic_hash, needed_peers, |peer| !peers.contains(peer)); @@ -798,10 +814,11 @@ impl Gossipsub { } // send graft/prunes - self.send_graft_prune(to_graft, to_prune); + if !to_graft.is_empty() | !to_prune.is_empty() { + self.send_graft_prune(to_graft, to_prune); + } // shift the memcache - debug!("HEARTBEAT: Memcache shifted"); self.mcache.shift(); debug!("Completed Heartbeat"); } @@ -890,7 +907,6 @@ impl Gossipsub { }, }); } - debug!("Completed gossip"); } /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. @@ -1133,11 +1149,13 @@ where return Async::Ready(event); } - match self.heartbeat.poll() { - // heartbeat ready - Ok(Async::Ready(Some(_))) => self.heartbeat(), - _ => {} - }; + loop { + match self.heartbeat.poll() { + // heartbeat ready + Ok(Async::Ready(Some(_))) => self.heartbeat(), + _ => break, + }; + } Async::NotReady } @@ -1199,8 +1217,103 @@ pub enum NodeType { #[cfg(test)] mod tests { use super::*; + use libp2p_core::topology::MemoryTopology; use libp2p_floodsub::TopicBuilder; + // helper functions for testing + + fn build_and_inject_nodes( + peer_no: usize, + topics: Vec, + ) -> (Vec, Gossipsub) { + // generate a default GossipsubConfig + let gs_config = GossipsubConfig::default(); + // create a gossipsub struct + let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); + + // subscribe to the topics + for t in topics { + let topic = TopicBuilder::new(t).build(); + gs.subscribe(topic.clone()); + } + + // build and connect peer_no random peers + let mut peers = vec![]; + let dummy_connected_point = ConnectedPoint::Dialer { + address: "/ip4/0.0.0.0/tcp/0".parse().unwrap(), + }; + for _ in 0..20 { + let peer = PeerId::random(); + peers.push(peer.clone()); + as NetworkBehaviour>::inject_connected(&mut gs, + peer, + dummy_connected_point.clone(), + ); + } + + return (peers.clone(), gs); + } + + #[test] + /// Test the gossipsub NetworkBehaviour peer connection logic. + fn test_inject_connected() { + let (peers, gs) = + build_and_inject_nodes(20, vec![String::from("topic1"), String::from("topic2")]); + + // check that our subscriptions are sent to each of the peers + // collect all the SendEvents + let send_events: Vec<&NetworkBehaviourAction> = gs + .events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::SendEvent { + peer_id: _, + event: _, + } => true, + _ => false, + }) + .collect(); + + // check that there are two subscriptions sent to each peer + for sevent in send_events.clone() { + match sevent { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + assert!( + event.subscriptions.len() == 2, + "There should be two subscriptions sent to each peer (1 for each topic)." + ); + } + _ => {} + }; + } + + // check that there are 20 send events created + assert!( + send_events.len() == 20, + "There should be a subscription event sent to each peer." + ); + + // verify internal structures + + // should add the new peers to `peer_topics` with an empty vec as a gossipsub node + for peer in peers { + let known_topics = &gs.peer_topics.get(&peer).unwrap().0; + let node_type = &gs.peer_topics.get(&peer).unwrap().1; + assert!( + known_topics == &SmallVec::<[TopicHash; 16]>::new(), + "The topics for each node should be empty" + ); + // TODO: Update this for handling floodsub nodes + assert!( + match node_type { + NodeType::Gossipsub => true, + _ => false, + }, + "All peers should be added as a gossipsub node" + ); + } + } + #[test] /// Test Gossipsub.get_random_peers() function fn test_get_random_peers() { From c7b3ca58977d34233a801297d0be21c72030b12e Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 28 Jan 2019 21:02:51 +1100 Subject: [PATCH 26/91] Add subscribe tests. - Modifies `handle_received_subscriptions` to take a reference of subscriptions - Adds `test_subscribe` - Adds `test_handle_received_subscriptions` - Adds tests for the filter in `get_random_peers` --- protocols/gossipsub/src/layer.rs | 177 +++++++++++++++++++++++++++++-- 1 file changed, 170 insertions(+), 7 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 96f16dfa4ac..e98186dcaf5 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -593,7 +593,7 @@ impl Gossipsub { /// Handles received subscriptions. fn handle_received_subscriptions( &mut self, - subscriptions: Vec, + subscriptions: &Vec, propagation_source: &PeerId, ) { debug!( @@ -643,7 +643,7 @@ impl Gossipsub { self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Subscribed { peer_id: propagation_source.clone(), - topic: subscription.topic_hash, + topic: subscription.topic_hash.clone(), }, )); } @@ -683,7 +683,7 @@ impl Gossipsub { self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Unsubscribed { peer_id: propagation_source.clone(), - topic: subscription.topic_hash, + topic: subscription.topic_hash.clone(), }, )); } @@ -1097,7 +1097,7 @@ where }; // Handle subscriptions // Update connected peers topics - self.handle_received_subscriptions(event.subscriptions, &propagation_source); + self.handle_received_subscriptions(&event.subscriptions, &propagation_source); // Handle messages for message in event.messages { @@ -1225,16 +1225,22 @@ mod tests { fn build_and_inject_nodes( peer_no: usize, topics: Vec, - ) -> (Vec, Gossipsub) { + ) -> ( + Gossipsub, + Vec, + Vec, + ) { // generate a default GossipsubConfig let gs_config = GossipsubConfig::default(); // create a gossipsub struct let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); + let mut topic_hashes = vec![]; // subscribe to the topics for t in topics { let topic = TopicBuilder::new(t).build(); gs.subscribe(topic.clone()); + topic_hashes.push(topic.hash().clone()); } // build and connect peer_no random peers @@ -1251,13 +1257,79 @@ mod tests { ); } - return (peers.clone(), gs); + return (gs, peers, topic_hashes); + } + + #[test] + /// Test local node subscribing to a topic + fn test_subscribe() { + // The node should: + // - Create an empty vector in mesh[topic] + // - Send subscription request to all peers + // - run JOIN(topic) + + let subscribe_topic = vec![String::from("test_subscribe")]; + let (gs, _, topic_hashes) = build_and_inject_nodes(20, subscribe_topic); + + assert!( + *gs.mesh.get(&topic_hashes[0]).expect("Mesh should exist") == Vec::::new(), + "Subscribe should add a new entry to the mesh[topic] hashmap" + ); + + // collect all the SendEvents + let subscriptions = + gs.events + .iter() + .fold(vec![], |mut collected_subscriptions, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for s in &event.subscriptions { + match s.action { + GossipsubSubscriptionAction::Subscribe => { + collected_subscriptions.push(s.clone()) + } + _ => {} + }; + } + collected_subscriptions + } + _ => collected_subscriptions, + }); + + // we sent a subscribe to all known peers + assert!( + subscriptions.len() == 20, + "Should send a subscription to all known peers" + ); + + // there should be mesh_n GRAFT messages (test JOIN more thoroughly in separate test) + let graft_messages = gs + .events + .iter() + .fold(vec![], |mut collected_grafts, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for c in &event.control_msgs { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} + }; + } + collected_grafts + } + _ => collected_grafts, + }); + + assert!( + graft_messages.len() == 6, + "There should be 6 grafts messages sent to peers" + ); } #[test] /// Test the gossipsub NetworkBehaviour peer connection logic. fn test_inject_connected() { - let (peers, gs) = + let (gs, peers, _) = build_and_inject_nodes(20, vec![String::from("topic1"), String::from("topic2")]); // check that our subscriptions are sent to each of the peers @@ -1314,6 +1386,92 @@ mod tests { } } + #[test] + /// Test subscription handling + fn test_handle_received_subscriptions() { + // For every subscription: + // SUBSCRIBE: - Add subscribed topic to peer_topics for peer. + // - Add peer to topics_peer. + // UNSUBSCRIBE - Remove topic from peer_topics for peer. + // - Remove peer from topic_peers. + + let topics = vec!["topic1", "topic2", "topic3", "topic4"] + .iter() + .map(|&t| String::from(t)) + .collect(); + let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics); + + // The first peer sends 3 subscriptions and 1 unsubscription + let mut subscriptions = topic_hashes[..3] + .iter() + .map(|topic_hash| GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: topic_hash.clone(), + }) + .collect::>(); + + subscriptions.push(GossipsubSubscription { + action: GossipsubSubscriptionAction::Unsubscribe, + topic_hash: topic_hashes[topic_hashes.len() - 1].clone(), + }); + + let unknown_peer = PeerId::random(); + // process the subscriptions + // first and second peers send subscriptions + gs.handle_received_subscriptions(&subscriptions, &peers[0]); + gs.handle_received_subscriptions(&subscriptions, &peers[1]); + // unknown peer sends the same subscriptions + gs.handle_received_subscriptions(&subscriptions, &unknown_peer); + + // verify the result + + let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); + assert!( + peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), + "First peer should be subscribed to three topics" + ); + let peer_topics = gs.peer_topics.get(&peers[1]).unwrap().0.clone(); + assert!( + peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), + "Second peer should be subscribed to three topics" + ); + + assert!( + gs.peer_topics.get(&unknown_peer).is_none(), + "Unknown peer should not have been added" + ); + + for topic_hash in topic_hashes[..3].iter() { + let topic_peers = gs.topic_peers.get(topic_hash).unwrap().1.clone(); // only gossipsub at the moment + assert!( + topic_peers == peers[..2].to_vec(), + "Two peers should be added to the first three topics" + ); + } + + // Peer 0 unsubscribes from the first topic + + gs.handle_received_subscriptions( + &vec![GossipsubSubscription { + action: GossipsubSubscriptionAction::Unsubscribe, + topic_hash: topic_hashes[0].clone(), + }], + &peers[0], + ); + + let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); + assert!( + peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[1..3].to_vec()), + "Peer should be subscribed to two topics" + ); + + let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().1.clone(); // only gossipsub at the moment + assert!( + topic_peers == peers[1..2].to_vec(), + "Only the second peers should be in the first topic" + ); + } + #[test] /// Test Gossipsub.get_random_peers() function fn test_get_random_peers() { @@ -1342,6 +1500,11 @@ mod tests { assert!(random_peers == peers, "Expected no shuffling"); let random_peers = gs.get_random_peers(&topic_hash, 0, { |_| true }); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); + // test the filter + let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| false }); + assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); + let random_peers = gs.get_random_peers(&topic_hash, 10, { |peer| peers.contains(peer) }); + assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); } } From 2b74f28fbafef289e8a5e69940271729b38217ef Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 29 Jan 2019 13:10:50 +1100 Subject: [PATCH 27/91] Add Bug fixes and further testing for gossipsub. - Corrects the tuple use of topic_hashes - Corrects JOIN logic around fanout and adding peers to the mesh - Adds test_unsubscribe - Adds test_join --- protocols/gossipsub/src/layer.rs | 279 ++++++++++++++++++++++++++----- 1 file changed, 239 insertions(+), 40 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index e98186dcaf5..045e2847d83 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -199,7 +199,7 @@ impl Gossipsub { } // send subscription request to all floodsub and gossipsub peers - for (flood_peers, gossip_peers) in self.topic_peers.values() { + for (gossip_peers, flood_peers) in self.topic_peers.get(&topic.hash()) { for peer in flood_peers.iter().chain(gossip_peers) { debug!("Sending SUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -218,7 +218,7 @@ impl Gossipsub { // call JOIN(topic) // this will add new peers to the mesh for the topic - self.join(topic.clone()); + self.join(topic.hash()); info!("Subscribed to topic: {:?}", topic); true } @@ -238,8 +238,8 @@ impl Gossipsub { return false; } - // announce to all floodsub and gossipsub peers - for (flood_peers, gossip_peers) in self.topic_peers.values() { + // announce to all floodsub and gossipsub peers, in the topic + for (gossip_peers, flood_peers) in self.topic_peers.get(topic_hash) { for peer in flood_peers.iter().chain(gossip_peers) { debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -258,7 +258,7 @@ impl Gossipsub { // call LEAVE(topic) // this will remove the topic from the mesh - self.leave(&topic); + self.leave(&topic_hash); info!("Unsubscribed from topic: {:?}", topic_hash); true @@ -343,8 +343,7 @@ impl Gossipsub { } /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. - fn join(&mut self, topic: impl AsRef) { - let topic_hash = topic.as_ref(); + fn join(&mut self, topic_hash: &TopicHash) { debug!("Running JOIN for topic: {:?}", topic_hash); // if we are already in the mesh, return @@ -353,31 +352,50 @@ impl Gossipsub { return; } - let mut peers = vec![]; + let mut added_peers = vec![]; - // check if we have peers in fanout[topic] and remove them if we do + // check if we have mesh_n peers in fanout[topic] and add them to the mesh if we do, + // removing the fanout entry. if let Some((_, peers)) = self.fanout.remove_entry(topic_hash) { debug!( "JOIN: Removing peers from the fanout for topic: {:?}", topic_hash ); - // add them to the mesh - self.mesh.insert(topic_hash.clone(), peers.clone()); + // add up to mesh_n of them them to the mesh + // Note: These aren't randomly added, currently FIFO + let mut add_peers = self.config.mesh_n.clone(); + if peers.len() < self.config.mesh_n { + add_peers = peers.len(); + } + debug!( + "JOIN: Adding {:?} peers from the fanout for topic: {:?}", + add_peers, topic_hash + ); + added_peers.append(&mut peers.clone()[..add_peers].to_vec()); + self.mesh + .insert(topic_hash.clone(), peers[..add_peers].to_vec()); // remove the last published time self.fanout_last_pub.remove(topic_hash); - } else { - // no peers in fanout[topic] - select mesh_n at random - let mesh_n = self.config.mesh_n; - peers = self.get_random_peers(topic_hash, mesh_n, { |_| true }); - // put them in the mesh + } + + // check if we need to get more peers, which we randomly select + if added_peers.len() < self.config.mesh_n { + // get the peers + let mut new_peers = + self.get_random_peers(topic_hash, self.config.mesh_n - added_peers.len(), { + |_| true + }); + added_peers.append(&mut new_peers.clone()); + // add them to the mesh debug!( "JOIN: Inserting {:?} random peers into the mesh", - peers.len() + new_peers.len() ); - self.mesh.insert(topic_hash.clone(), peers.clone()); + let mesh_peers = self.mesh.entry(topic_hash.clone()).or_insert(vec![]); + mesh_peers.append(&mut new_peers); } - for peer_id in peers { + for peer_id in added_peers { // Send a GRAFT control message info!("JOIN: Sending Graft message to peer: {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -396,8 +414,7 @@ impl Gossipsub { } /// Gossipsub LEAVE(topic) - Notifies mesh[topic] peers with PRUNE messages. - fn leave(&mut self, topic: impl AsRef) { - let topic_hash = topic.as_ref(); + fn leave(&mut self, topic_hash: &TopicHash) { debug!("Running LEAVE for topic {:?}", topic_hash); // if our mesh contains the topic, send prune to peers and delete it from the mesh @@ -610,7 +627,7 @@ impl Gossipsub { for subscription in subscriptions { // get the peers from the mapping, or insert empty lists if topic doesn't exist - let (flood_peers, gossip_peers) = self + let (gossip_peers, flood_peers) = self .topic_peers .entry(subscription.topic_hash.clone()) .or_insert((vec![], vec![])); @@ -976,7 +993,7 @@ impl Gossipsub { let mut rng = thread_rng(); gossip_peers.partial_shuffle(&mut rng, n); - debug!("RANDOM PEERS: Got {:?} peers", gossip_peers.len()); + debug!("RANDOM PEERS: Got {:?} peers", n); return gossip_peers[..n].to_vec(); } @@ -1046,7 +1063,7 @@ where } // remove from topic_peers - if let Some((floodsub_peers, gossip_peers)) = self.topic_peers.get_mut(&topic) { + if let Some((gossip_peers, flood_peers)) = self.topic_peers.get_mut(&topic) { match node_type { NodeType::Gossipsub => { if let Some(pos) = gossip_peers.iter().position(|p| p == id) { @@ -1062,8 +1079,8 @@ where } } NodeType::Floodsub => { - if let Some(pos) = floodsub_peers.iter().position(|p| p == id) { - floodsub_peers.remove(pos); + if let Some(pos) = flood_peers.iter().position(|p| p == id) { + flood_peers.remove(pos); //TODO: untagPeer } // debugging purposes @@ -1222,9 +1239,12 @@ mod tests { // helper functions for testing + // This function generates `peer_no` random PeerId's, subscribes to `topics` and subscribes the + // injected nodes to all topics if `to_subscribe` is set. All nodes are considered gossipsub nodes. fn build_and_inject_nodes( peer_no: usize, topics: Vec, + to_subscribe: bool, ) -> ( Gossipsub, Vec, @@ -1236,6 +1256,7 @@ mod tests { let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); let mut topic_hashes = vec![]; + // subscribe to the topics for t in topics { let topic = TopicBuilder::new(t).build(); @@ -1248,13 +1269,27 @@ mod tests { let dummy_connected_point = ConnectedPoint::Dialer { address: "/ip4/0.0.0.0/tcp/0".parse().unwrap(), }; - for _ in 0..20 { + + for _ in 0..peer_no { let peer = PeerId::random(); peers.push(peer.clone()); as NetworkBehaviour>::inject_connected(&mut gs, - peer, + peer.clone(), dummy_connected_point.clone(), ); + if to_subscribe { + gs.handle_received_subscriptions( + &topic_hashes + .iter() + .cloned() + .map(|t| GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: t, + }) + .collect(), + &peer, + ); + }; } return (gs, peers, topic_hashes); @@ -1269,14 +1304,14 @@ mod tests { // - run JOIN(topic) let subscribe_topic = vec![String::from("test_subscribe")]; - let (gs, _, topic_hashes) = build_and_inject_nodes(20, subscribe_topic); + let (gs, _, topic_hashes) = build_and_inject_nodes(20, subscribe_topic, true); assert!( *gs.mesh.get(&topic_hashes[0]).expect("Mesh should exist") == Vec::::new(), "Subscribe should add a new entry to the mesh[topic] hashmap" ); - // collect all the SendEvents + // collect all the subscriptions let subscriptions = gs.events .iter() @@ -1300,8 +1335,122 @@ mod tests { subscriptions.len() == 20, "Should send a subscription to all known peers" ); + } + + #[test] + /// Test unsubscribe. + fn test_unsubscribe() { + // Unsubscribe should: + // - Remove the mesh entry for topic + // - Send UNSUBSCRIBE to all known peers + // - Call Leave + + let topic_strings = vec![String::from("topic1"), String::from("topic2")]; + let topics = topic_strings + .iter() + .map(|t| TopicBuilder::new(t.clone()).build()) + .collect::>(); + + // subscribe to topic_strings + let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); + + for topic_hash in &topic_hashes { + assert!( + gs.topic_peers.get(&topic_hash).is_some(), + "Topic_peers contain a topic entry" + ); + assert!( + gs.mesh.get(&topic_hash).is_some(), + "mesh should contain a topic entry" + ); + } + + // unsubscribe from both topics + assert!( + gs.unsubscribe(topics[0].clone()), + "should be able to unsubscribe successfully from each topic", + ); + assert!( + gs.unsubscribe(topics[1].clone()), + "should be able to unsubscribe successfully from each topic", + ); + + let subscriptions = + gs.events + .iter() + .fold(vec![], |mut collected_subscriptions, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for s in &event.subscriptions { + match s.action { + GossipsubSubscriptionAction::Unsubscribe => { + collected_subscriptions.push(s.clone()) + } + _ => {} + }; + } + collected_subscriptions + } + _ => collected_subscriptions, + }); + + // we sent a unsubscribe to all known peers, for two topics + assert!( + subscriptions.len() == 40, + "Should send an unsubscribe event to all known peers" + ); + + // check we clean up internal structures + for topic_hash in &topic_hashes { + assert!( + gs.mesh.get(&topic_hash).is_none(), + "All topics should have been removed from the mesh" + ); + } + } + + #[test] + /// Test JOIN(topic) functionality. + fn test_join() { + // The Join function should: + // - Remove peers from fanout[topic] + // - Add any fanout[topic] peers to the mesh (up to mesh_n) + // - Fill up to mesh_n peers from known gossipsub peers in the topic + // - Send GRAFT messages to all nodes added to the mesh + + // This test is not an isolated unit test, rather it uses higher level, + // subscribe/unsubscribe to perform the test. + + let topic_strings = vec![String::from("topic1"), String::from("topic2")]; + let topics = topic_strings + .iter() + .map(|t| TopicBuilder::new(t.clone()).build()) + .collect::>(); + + let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); + + // unsubscribe, then call join to invoke functionality + assert!( + gs.unsubscribe(topics[0].clone()), + "should be able to unsubscribe successfully" + ); + assert!( + gs.unsubscribe(topics[1].clone()), + "should be able to unsubscribe successfully" + ); + + // re-subscribe - there should be peers associated with the topic + assert!( + gs.subscribe(topics[0].clone()), + "should be able to subscribe successfully" + ); + + // should have added mesh_n nodes to the mesh + assert!( + gs.mesh.get(&topic_hashes[0]).unwrap().len() == 6, + "Should have added 6 nodes to the mesh" + ); - // there should be mesh_n GRAFT messages (test JOIN more thoroughly in separate test) + // there should be mesh_n GRAFT messages. let graft_messages = gs .events .iter() @@ -1324,13 +1473,65 @@ mod tests { graft_messages.len() == 6, "There should be 6 grafts messages sent to peers" ); + + // verify fanout nodes + // add 3 random peers to the fanout[topic1] + gs.fanout.insert(topic_hashes[1].clone(), vec![]); + let new_peers = vec![]; + for _ in 0..3 { + let mut fanout_peers = gs.fanout.get_mut(&topic_hashes[1]).unwrap(); + fanout_peers.push(PeerId::random()); + } + + // subscribe to topic1 + gs.subscribe(topics[1].clone()); + + // the three new peers should have been added, along with 3 more from the pool. + assert!( + gs.mesh.get(&topic_hashes[1]).unwrap().len() == 6, + "Should have added 6 nodes to the mesh" + ); + let mesh_peers = gs.mesh.get(&topic_hashes[1]).unwrap(); + for new_peer in new_peers { + assert!( + mesh_peers.contains(new_peer), + "Fanout peer should be included in the mesh" + ); + } + + // there should now be 12 graft messages to be sent + let graft_messages = gs + .events + .iter() + .fold(vec![], |mut collected_grafts, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for c in &event.control_msgs { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} + }; + } + collected_grafts + } + _ => collected_grafts, + }); + + assert!( + graft_messages.len() == 12, + "There should be 6 grafts messages sent to peers" + ); } #[test] /// Test the gossipsub NetworkBehaviour peer connection logic. fn test_inject_connected() { - let (gs, peers, _) = - build_and_inject_nodes(20, vec![String::from("topic1"), String::from("topic2")]); + let (gs, peers, topic_hashes) = build_and_inject_nodes( + 20, + vec![String::from("topic1"), String::from("topic2")], + true, + ); // check that our subscriptions are sent to each of the peers // collect all the SendEvents @@ -1365,15 +1566,13 @@ mod tests { "There should be a subscription event sent to each peer." ); - // verify internal structures - // should add the new peers to `peer_topics` with an empty vec as a gossipsub node for peer in peers { let known_topics = &gs.peer_topics.get(&peer).unwrap().0; let node_type = &gs.peer_topics.get(&peer).unwrap().1; assert!( - known_topics == &SmallVec::<[TopicHash; 16]>::new(), - "The topics for each node should be empty" + known_topics == &SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes.clone()), + "The topics for each node should all topics" ); // TODO: Update this for handling floodsub nodes assert!( @@ -1399,7 +1598,7 @@ mod tests { .iter() .map(|&t| String::from(t)) .collect(); - let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics); + let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics, false); // The first peer sends 3 subscriptions and 1 unsubscription let mut subscriptions = topic_hashes[..3] @@ -1442,7 +1641,7 @@ mod tests { ); for topic_hash in topic_hashes[..3].iter() { - let topic_peers = gs.topic_peers.get(topic_hash).unwrap().1.clone(); // only gossipsub at the moment + let topic_peers = gs.topic_peers.get(topic_hash).unwrap().0.clone(); // only gossipsub at the moment assert!( topic_peers == peers[..2].to_vec(), "Two peers should be added to the first three topics" @@ -1465,7 +1664,7 @@ mod tests { "Peer should be subscribed to two topics" ); - let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().1.clone(); // only gossipsub at the moment + let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().0.clone(); // only gossipsub at the moment assert!( topic_peers == peers[1..2].to_vec(), "Only the second peers should be in the first topic" From 0dd9cd29b645b8064c9edd489fc818231a362fca Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 29 Jan 2019 13:13:40 +1100 Subject: [PATCH 28/91] Rename GossipsubMessage::msg_id -> id --- protocols/gossipsub/src/layer.rs | 38 ++++++++++++++--------------- protocols/gossipsub/src/mcache.rs | 10 ++++---- protocols/gossipsub/src/protocol.rs | 2 +- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 045e2847d83..4dd552be56f 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -288,7 +288,7 @@ impl Gossipsub { topics: topic.into_iter().map(|t| t.into().clone()).collect(), }; - debug!("Publishing message: {:?}", message.msg_id()); + debug!("Publishing message: {:?}", message.id()); // forward the message to mesh and floodsub peers let local_peer_id = self.local_peer_id.clone(); @@ -325,7 +325,7 @@ impl Gossipsub { // add published message to our received caches self.mcache.put(message.clone()); - self.received.add(&message.msg_id()); + self.received.add(&message.id()); // Send to peers we know are subscribed to the topic. for peer_id in recipient_peers.keys() { @@ -339,7 +339,7 @@ impl Gossipsub { }, }); } - info!("Published message: {:?}", message.msg_id()); + info!("Published message: {:?}", message.id()); } /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. @@ -443,9 +443,9 @@ impl Gossipsub { fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { debug!("Handling IHAVE for peer: {:?}", peer_id); // use a hashmap to avoid duplicates efficiently - let mut iwant_msg_ids = HashMap::new(); + let mut iwant_ids = HashMap::new(); - for (topic, msg_ids) in ihave_msgs { + for (topic, ids) in ihave_msgs { // only process the message if we are subscribed if !self.mesh.contains_key(&topic) { info!( @@ -455,15 +455,15 @@ impl Gossipsub { return; // continue } - for msg_id in msg_ids { - if !self.received.contains(&msg_id) { + for id in ids { + if !self.received.contains(&id) { // have not seen this message, request it - iwant_msg_ids.insert(msg_id, true); + iwant_ids.insert(id, true); } } } - if !iwant_msg_ids.is_empty() { + if !iwant_ids.is_empty() { // Send the list of IWANT control messages info!("IHAVE: Sending IWANT message"); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -472,7 +472,7 @@ impl Gossipsub { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: vec![GossipsubControlAction::IWant { - message_ids: iwant_msg_ids.keys().cloned().collect(), + message_ids: iwant_ids.keys().cloned().collect(), }], }, }); @@ -487,10 +487,10 @@ impl Gossipsub { // build a hashmap of available messages let mut cached_messages = HashMap::new(); - for msg_id in iwant_msgs { + for id in iwant_msgs { // if we have it, add it do the cached_messages mapping - if let Some(msg) = self.mcache.get(&msg_id) { - cached_messages.insert(msg_id.clone(), msg.clone()); + if let Some(msg) = self.mcache.get(&id) { + cached_messages.insert(id.clone(), msg.clone()); } } @@ -577,16 +577,16 @@ impl Gossipsub { fn handle_received_message(&mut self, msg: GossipsubMessage, propagation_source: &PeerId) { debug!( "Handling message: {:?} from peer: {:?}", - msg.msg_id(), + msg.id(), propagation_source ); // if we have seen this message, ignore it // there's a 3% chance this is a false positive // TODO: Check this has no significant emergent behaviour - if !self.received.test_and_add(&msg.msg_id()) { + if !self.received.test_and_add(&msg.id()) { info!( "Message already received, ignoring. Message: {:?}", - msg.msg_id() + msg.id() ); return; } @@ -604,7 +604,7 @@ impl Gossipsub { // forward the message to floodsub and mesh peers self.forward_msg(msg.clone(), propagation_source.clone()); - debug!("Completed message handling for message: {:?}", msg.msg_id()); + debug!("Completed message handling for message: {:?}", msg.id()); } /// Handles received subscriptions. @@ -928,7 +928,7 @@ impl Gossipsub { /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { - debug!("Forwarding message: {:?}", message.msg_id()); + debug!("Forwarding message: {:?}", message.id()); let mut recipient_peers = HashMap::new(); // add floodsub and mesh peers @@ -955,7 +955,7 @@ impl Gossipsub { // forward the message to peers if !recipient_peers.is_empty() { for peer in recipient_peers.keys() { - debug!("Sending message: {:?} to peer {:?}", message.msg_id(), peer); + debug!("Sending message: {:?} to peer {:?}", message.id(), peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: GossipsubRpc { diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 3e9d6d751b3..e77ab5f25cc 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -31,7 +31,7 @@ impl MessageCache { /// Put a message into the memory cache pub fn put(&mut self, msg: GossipsubMessage) { - let message_id = msg.msg_id(); + let message_id = msg.id(); let cache_entry = CacheEntry { mid: message_id.clone(), topics: msg.topics.clone(), @@ -136,7 +136,7 @@ mod tests { assert!(mc.history[0].len() == 1); - let fetched = mc.get(&m.msg_id()); + let fetched = mc.get(&m.id()); assert_eq!(fetched.is_none(), false); assert_eq!(fetched.is_some(), true); @@ -161,8 +161,8 @@ mod tests { mc.put(m.clone()); // Try to get an incorrect ID - let wrong_msg_id = String::from("wrongid"); - let fetched = mc.get(&wrong_msg_id); + let wrong_id = String::from("wrongid"); + let fetched = mc.get(&wrong_id); assert_eq!(fetched.is_none(), true); } @@ -186,7 +186,7 @@ mod tests { let m = gen_testm(1 as usize, vec![]); mc.put(m.clone()); - let fetched = mc.get(&m.msg_id()); + let fetched = mc.get(&m.id()); // Make sure it is the same fetched message match fetched { diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 6101655e875..57579bb2700 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -325,7 +325,7 @@ pub struct GossipsubMessage { impl GossipsubMessage { /// Converts message into a message_id. // To be compatible with the go implementation - pub fn msg_id(&self) -> String { + pub fn id(&self) -> String { let mut source_string = self.source.to_base58(); // the sequence number is a big endian uint64 (as per go implementation) // avoid a potential panic by setting the seqno to 0 if it is not long enough. From 2442d5049317385aa0cb2d78d60ab0e7f5c31613 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 29 Jan 2019 16:44:26 +1100 Subject: [PATCH 29/91] Add bug fix for handling disconnected peers. --- protocols/gossipsub/src/layer.rs | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 4dd552be56f..d9616c394a4 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -792,21 +792,21 @@ impl Gossipsub { } // maintain fanout - // check if our peers are still apart of the topic + // check if our peers are still a part of the topic for (topic_hash, peers) in self.fanout.clone().iter_mut() { peers.retain(|peer| { // is the peer still subscribed to the topic? - if !self - .peer_topics - .get(peer) - .expect("Peer should exist") - .0 - .contains(&topic_hash) - { - debug!( - "HEARTBEAT: Peer removed from fanout for topic: {:?}", - topic_hash - ); + if let Some(topics) = self.peer_topics.get(peer) { + if !topics.0.contains(&topic_hash) { + debug!( + "HEARTBEAT: Peer removed from fanout for topic: {:?}", + topic_hash + ); + return false; + } + } + // remove if the peer has disconnected + else { return false; } true From abfa4dad180c35007947feb37b22c7702b4f1778 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 29 Jan 2019 16:45:49 +1100 Subject: [PATCH 30/91] Implements (partially) #889 for Gossipsub. --- protocols/gossipsub/examples/chat.rs | 13 ++++--------- protocols/gossipsub/src/layer.rs | 16 +++++++++++----- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index a083a80a179..d770cc69c95 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -18,8 +18,8 @@ fn main() { // Create a random PeerId let local_key = secio::SecioKeyPair::ed25519_generated().unwrap(); - let local_pub_key = local_key.to_public_key(); - println!("Local peer id: {:?}", local_pub_key.clone().into_peer_id()); + let local_peer_id = local_key.to_peer_id(); + println!("Local peer id: {:?}", local_peer_id); // Set up an encrypted TCP Transport over the Mplex and Yamux protocols let transport = libp2p::build_development_transport(local_key); @@ -44,14 +44,9 @@ fn main() { Duration::from_secs(60), ); // build a gossipsub network behaviour - let mut gossipsub = - gossipsub::Gossipsub::new(local_pub_key.clone().into_peer_id(), gossipsub_config); + let mut gossipsub = gossipsub::Gossipsub::new(local_peer_id.clone(), gossipsub_config); gossipsub.subscribe(topic.clone()); - libp2p::Swarm::new( - transport, - gossipsub, - libp2p::core::topology::MemoryTopology::empty(local_pub_key), - ) + libp2p::Swarm::new(transport, gossipsub, local_peer_id) }; // Listen on all interfaces and whatever port the OS assigns diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index d9616c394a4..8fa7c47cd36 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -28,7 +28,7 @@ use libp2p_core::swarm::{ }; use libp2p_core::{ protocols_handler::{OneShotHandler, ProtocolsHandler}, - PeerId, + Multiaddr, PeerId, }; use libp2p_floodsub::{Topic, TopicHash}; use mcache::MessageCache; @@ -999,7 +999,7 @@ impl Gossipsub { } } -impl NetworkBehaviour for Gossipsub +impl NetworkBehaviour for Gossipsub where TSubstream: AsyncRead + AsyncWrite, { @@ -1010,6 +1010,10 @@ where Default::default() } + fn addresses_of_peer(&self, _: &PeerId) -> Vec { + Vec::new() + } + fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { // We need to send our subscriptions to the newly-connected node. let mut subscriptions = vec![]; @@ -1104,6 +1108,8 @@ where // remove peer from peer_topics let was_in = self.peer_topics.remove(id); debug_assert!(was_in.is_some()); + + //TODO: Reconnect due to inactivity } fn inject_node_event(&mut self, propagation_source: PeerId, event: InnerMessage) { @@ -1155,7 +1161,7 @@ where fn poll( &mut self, - _: &mut PollParameters, + _: &mut PollParameters, ) -> Async< NetworkBehaviourAction< ::InEvent, @@ -1234,7 +1240,6 @@ pub enum NodeType { #[cfg(test)] mod tests { use super::*; - use libp2p_core::topology::MemoryTopology; use libp2p_floodsub::TopicBuilder; // helper functions for testing @@ -1273,7 +1278,8 @@ mod tests { for _ in 0..peer_no { let peer = PeerId::random(); peers.push(peer.clone()); - as NetworkBehaviour>::inject_connected(&mut gs, + as NetworkBehaviour>::inject_connected( + &mut gs, peer.clone(), dummy_connected_point.clone(), ); From f04d242f49000ec06faba46aefa681f6cb076e36 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Wed, 30 Jan 2019 22:09:47 -0600 Subject: [PATCH 31/91] handle_iwant event count tests --- protocols/gossipsub/src/layer.rs | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 8fa7c47cd36..bf1ebfd0dd1 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -1712,4 +1712,34 @@ mod tests { assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); } + #[test] + fn test_handle_iwant_msg_exists() { + let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + + let message = GossipsubMessage { + source: peers[11].clone(), + data: vec![1,2,3,4], + sequence_number: vec![2,4,3], + topics: Vec::new(), + }; + let msg_id = message.id(); + gs.mcache.put(message.clone()); + + let eventsBefore = gs.events.len(); + gs.handle_iwant(&peers[7], vec![msg_id.clone()]); + let eventsAfter = gs.events.len(); + + assert_eq!(eventsBefore + 1, eventsAfter); + } + + #[test] + fn test_handle_iwant_msg_not_exists() { + let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + + let eventsBefore = gs.events.len(); + gs.handle_iwant(&peers[7], vec![String::from("unknown id")]); + let eventsAfter = gs.events.len(); + + assert_eq!(eventsBefore, eventsAfter); + } } From 3c8ebe56599cb47484752e1e56908683655369e3 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Thu, 31 Jan 2019 20:59:46 -0600 Subject: [PATCH 32/91] handle_ihave event count tests --- protocols/gossipsub/src/layer.rs | 51 +++++++++++++++++++++++++++++--- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index bf1ebfd0dd1..ebff2dbea9a 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -1713,7 +1713,8 @@ mod tests { } #[test] - fn test_handle_iwant_msg_exists() { + // tests that an event is created when a peer asks for a message in our cache + fn test_handle_iwant_msg_cached() { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); let message = GossipsubMessage { @@ -1729,17 +1730,59 @@ mod tests { gs.handle_iwant(&peers[7], vec![msg_id.clone()]); let eventsAfter = gs.events.len(); - assert_eq!(eventsBefore + 1, eventsAfter); + assert_eq!(eventsBefore + 1, eventsAfter, "Expected event count to increase"); } #[test] - fn test_handle_iwant_msg_not_exists() { + // tests that an event is not created when a peers asks for a message not in our cache + fn test_handle_iwant_msg_not_cached() { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); let eventsBefore = gs.events.len(); gs.handle_iwant(&peers[7], vec![String::from("unknown id")]); let eventsAfter = gs.events.len(); - assert_eq!(eventsBefore, eventsAfter); + assert_eq!(eventsBefore, eventsAfter, "Expected event count to stay the same"); + } + + #[test] + // tests that an event is created when a peer shares that it has a message we want + fn test_handle_ihave_subscribed_and_msg_not_cached() { + let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); + + let eventsBefore = gs.events.len(); + gs.handle_ihave(&peers[7], vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])]); + let eventsAfter = gs.events.len(); + + assert_eq!(eventsBefore + 1, eventsAfter, "Expected event count to increase") + } + + #[test] + // tests that an event is not created when a peer shares that it has a message that + // we already have + fn test_handle_ihave_subscribed_and_msg_cached() { + let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); + + let msg_id = String::from("known id"); + gs.received.add(&msg_id); + + let eventsBefore = gs.events.len(); + gs.handle_ihave(&peers[7], vec![(topic_hashes[0].clone(), vec![msg_id])]); + let eventsAfter = gs.events.len(); + + assert_eq!(eventsBefore, eventsAfter, "Expected event count to stay the same") + } + + #[test] + // test that an event is not created when a peer shares that it has a message in + // a topic that we are not subscribed to + fn test_handle_ihave_not_subscribed() { + let (mut gs, peers, _) = build_and_inject_nodes(20, vec![], true); + + let eventsBefore = gs.events.len(); + gs.handle_ihave(&peers[7], vec![(TopicHash::from_raw(String::from("unsubscribed topic")), vec![String::from("irrelevant id")])]); + let eventsAfter = gs.events.len(); + + assert_eq!(eventsBefore, eventsAfter, "Expected event count to stay the same") } } From 12e07031882f18545b4981cc9b9cbdb320a474fd Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 1 Feb 2019 14:11:09 +1100 Subject: [PATCH 33/91] Move layer.rs tests into separate file. --- protocols/gossipsub/src/layer.rs | 552 +---------------------- protocols/gossipsub/src/layer/tests.rs | 579 +++++++++++++++++++++++++ 2 files changed, 581 insertions(+), 550 deletions(-) create mode 100644 protocols/gossipsub/src/layer/tests.rs diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index ebff2dbea9a..9eb1fdde687 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -45,6 +45,8 @@ use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; use tokio_timer::Interval; +mod tests; + // potentially rename this struct - due to clashes /// Configuration parameters that define the performance of the gossipsub network. #[derive(Clone)] @@ -1236,553 +1238,3 @@ pub enum NodeType { /// A Floodsub node. Floodsub, } - -#[cfg(test)] -mod tests { - use super::*; - use libp2p_floodsub::TopicBuilder; - - // helper functions for testing - - // This function generates `peer_no` random PeerId's, subscribes to `topics` and subscribes the - // injected nodes to all topics if `to_subscribe` is set. All nodes are considered gossipsub nodes. - fn build_and_inject_nodes( - peer_no: usize, - topics: Vec, - to_subscribe: bool, - ) -> ( - Gossipsub, - Vec, - Vec, - ) { - // generate a default GossipsubConfig - let gs_config = GossipsubConfig::default(); - // create a gossipsub struct - let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); - - let mut topic_hashes = vec![]; - - // subscribe to the topics - for t in topics { - let topic = TopicBuilder::new(t).build(); - gs.subscribe(topic.clone()); - topic_hashes.push(topic.hash().clone()); - } - - // build and connect peer_no random peers - let mut peers = vec![]; - let dummy_connected_point = ConnectedPoint::Dialer { - address: "/ip4/0.0.0.0/tcp/0".parse().unwrap(), - }; - - for _ in 0..peer_no { - let peer = PeerId::random(); - peers.push(peer.clone()); - as NetworkBehaviour>::inject_connected( - &mut gs, - peer.clone(), - dummy_connected_point.clone(), - ); - if to_subscribe { - gs.handle_received_subscriptions( - &topic_hashes - .iter() - .cloned() - .map(|t| GossipsubSubscription { - action: GossipsubSubscriptionAction::Subscribe, - topic_hash: t, - }) - .collect(), - &peer, - ); - }; - } - - return (gs, peers, topic_hashes); - } - - #[test] - /// Test local node subscribing to a topic - fn test_subscribe() { - // The node should: - // - Create an empty vector in mesh[topic] - // - Send subscription request to all peers - // - run JOIN(topic) - - let subscribe_topic = vec![String::from("test_subscribe")]; - let (gs, _, topic_hashes) = build_and_inject_nodes(20, subscribe_topic, true); - - assert!( - *gs.mesh.get(&topic_hashes[0]).expect("Mesh should exist") == Vec::::new(), - "Subscribe should add a new entry to the mesh[topic] hashmap" - ); - - // collect all the subscriptions - let subscriptions = - gs.events - .iter() - .fold(vec![], |mut collected_subscriptions, e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - for s in &event.subscriptions { - match s.action { - GossipsubSubscriptionAction::Subscribe => { - collected_subscriptions.push(s.clone()) - } - _ => {} - }; - } - collected_subscriptions - } - _ => collected_subscriptions, - }); - - // we sent a subscribe to all known peers - assert!( - subscriptions.len() == 20, - "Should send a subscription to all known peers" - ); - } - - #[test] - /// Test unsubscribe. - fn test_unsubscribe() { - // Unsubscribe should: - // - Remove the mesh entry for topic - // - Send UNSUBSCRIBE to all known peers - // - Call Leave - - let topic_strings = vec![String::from("topic1"), String::from("topic2")]; - let topics = topic_strings - .iter() - .map(|t| TopicBuilder::new(t.clone()).build()) - .collect::>(); - - // subscribe to topic_strings - let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); - - for topic_hash in &topic_hashes { - assert!( - gs.topic_peers.get(&topic_hash).is_some(), - "Topic_peers contain a topic entry" - ); - assert!( - gs.mesh.get(&topic_hash).is_some(), - "mesh should contain a topic entry" - ); - } - - // unsubscribe from both topics - assert!( - gs.unsubscribe(topics[0].clone()), - "should be able to unsubscribe successfully from each topic", - ); - assert!( - gs.unsubscribe(topics[1].clone()), - "should be able to unsubscribe successfully from each topic", - ); - - let subscriptions = - gs.events - .iter() - .fold(vec![], |mut collected_subscriptions, e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - for s in &event.subscriptions { - match s.action { - GossipsubSubscriptionAction::Unsubscribe => { - collected_subscriptions.push(s.clone()) - } - _ => {} - }; - } - collected_subscriptions - } - _ => collected_subscriptions, - }); - - // we sent a unsubscribe to all known peers, for two topics - assert!( - subscriptions.len() == 40, - "Should send an unsubscribe event to all known peers" - ); - - // check we clean up internal structures - for topic_hash in &topic_hashes { - assert!( - gs.mesh.get(&topic_hash).is_none(), - "All topics should have been removed from the mesh" - ); - } - } - - #[test] - /// Test JOIN(topic) functionality. - fn test_join() { - // The Join function should: - // - Remove peers from fanout[topic] - // - Add any fanout[topic] peers to the mesh (up to mesh_n) - // - Fill up to mesh_n peers from known gossipsub peers in the topic - // - Send GRAFT messages to all nodes added to the mesh - - // This test is not an isolated unit test, rather it uses higher level, - // subscribe/unsubscribe to perform the test. - - let topic_strings = vec![String::from("topic1"), String::from("topic2")]; - let topics = topic_strings - .iter() - .map(|t| TopicBuilder::new(t.clone()).build()) - .collect::>(); - - let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); - - // unsubscribe, then call join to invoke functionality - assert!( - gs.unsubscribe(topics[0].clone()), - "should be able to unsubscribe successfully" - ); - assert!( - gs.unsubscribe(topics[1].clone()), - "should be able to unsubscribe successfully" - ); - - // re-subscribe - there should be peers associated with the topic - assert!( - gs.subscribe(topics[0].clone()), - "should be able to subscribe successfully" - ); - - // should have added mesh_n nodes to the mesh - assert!( - gs.mesh.get(&topic_hashes[0]).unwrap().len() == 6, - "Should have added 6 nodes to the mesh" - ); - - // there should be mesh_n GRAFT messages. - let graft_messages = gs - .events - .iter() - .fold(vec![], |mut collected_grafts, e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - for c in &event.control_msgs { - match c { - GossipsubControlAction::Graft { topic_hash: _ } => { - collected_grafts.push(c.clone()) - } - _ => {} - }; - } - collected_grafts - } - _ => collected_grafts, - }); - - assert!( - graft_messages.len() == 6, - "There should be 6 grafts messages sent to peers" - ); - - // verify fanout nodes - // add 3 random peers to the fanout[topic1] - gs.fanout.insert(topic_hashes[1].clone(), vec![]); - let new_peers = vec![]; - for _ in 0..3 { - let mut fanout_peers = gs.fanout.get_mut(&topic_hashes[1]).unwrap(); - fanout_peers.push(PeerId::random()); - } - - // subscribe to topic1 - gs.subscribe(topics[1].clone()); - - // the three new peers should have been added, along with 3 more from the pool. - assert!( - gs.mesh.get(&topic_hashes[1]).unwrap().len() == 6, - "Should have added 6 nodes to the mesh" - ); - let mesh_peers = gs.mesh.get(&topic_hashes[1]).unwrap(); - for new_peer in new_peers { - assert!( - mesh_peers.contains(new_peer), - "Fanout peer should be included in the mesh" - ); - } - - // there should now be 12 graft messages to be sent - let graft_messages = gs - .events - .iter() - .fold(vec![], |mut collected_grafts, e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - for c in &event.control_msgs { - match c { - GossipsubControlAction::Graft { topic_hash: _ } => { - collected_grafts.push(c.clone()) - } - _ => {} - }; - } - collected_grafts - } - _ => collected_grafts, - }); - - assert!( - graft_messages.len() == 12, - "There should be 6 grafts messages sent to peers" - ); - } - - #[test] - /// Test the gossipsub NetworkBehaviour peer connection logic. - fn test_inject_connected() { - let (gs, peers, topic_hashes) = build_and_inject_nodes( - 20, - vec![String::from("topic1"), String::from("topic2")], - true, - ); - - // check that our subscriptions are sent to each of the peers - // collect all the SendEvents - let send_events: Vec<&NetworkBehaviourAction> = gs - .events - .iter() - .filter(|e| match e { - NetworkBehaviourAction::SendEvent { - peer_id: _, - event: _, - } => true, - _ => false, - }) - .collect(); - - // check that there are two subscriptions sent to each peer - for sevent in send_events.clone() { - match sevent { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - assert!( - event.subscriptions.len() == 2, - "There should be two subscriptions sent to each peer (1 for each topic)." - ); - } - _ => {} - }; - } - - // check that there are 20 send events created - assert!( - send_events.len() == 20, - "There should be a subscription event sent to each peer." - ); - - // should add the new peers to `peer_topics` with an empty vec as a gossipsub node - for peer in peers { - let known_topics = &gs.peer_topics.get(&peer).unwrap().0; - let node_type = &gs.peer_topics.get(&peer).unwrap().1; - assert!( - known_topics == &SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes.clone()), - "The topics for each node should all topics" - ); - // TODO: Update this for handling floodsub nodes - assert!( - match node_type { - NodeType::Gossipsub => true, - _ => false, - }, - "All peers should be added as a gossipsub node" - ); - } - } - - #[test] - /// Test subscription handling - fn test_handle_received_subscriptions() { - // For every subscription: - // SUBSCRIBE: - Add subscribed topic to peer_topics for peer. - // - Add peer to topics_peer. - // UNSUBSCRIBE - Remove topic from peer_topics for peer. - // - Remove peer from topic_peers. - - let topics = vec!["topic1", "topic2", "topic3", "topic4"] - .iter() - .map(|&t| String::from(t)) - .collect(); - let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics, false); - - // The first peer sends 3 subscriptions and 1 unsubscription - let mut subscriptions = topic_hashes[..3] - .iter() - .map(|topic_hash| GossipsubSubscription { - action: GossipsubSubscriptionAction::Subscribe, - topic_hash: topic_hash.clone(), - }) - .collect::>(); - - subscriptions.push(GossipsubSubscription { - action: GossipsubSubscriptionAction::Unsubscribe, - topic_hash: topic_hashes[topic_hashes.len() - 1].clone(), - }); - - let unknown_peer = PeerId::random(); - // process the subscriptions - // first and second peers send subscriptions - gs.handle_received_subscriptions(&subscriptions, &peers[0]); - gs.handle_received_subscriptions(&subscriptions, &peers[1]); - // unknown peer sends the same subscriptions - gs.handle_received_subscriptions(&subscriptions, &unknown_peer); - - // verify the result - - let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); - assert!( - peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), - "First peer should be subscribed to three topics" - ); - let peer_topics = gs.peer_topics.get(&peers[1]).unwrap().0.clone(); - assert!( - peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), - "Second peer should be subscribed to three topics" - ); - - assert!( - gs.peer_topics.get(&unknown_peer).is_none(), - "Unknown peer should not have been added" - ); - - for topic_hash in topic_hashes[..3].iter() { - let topic_peers = gs.topic_peers.get(topic_hash).unwrap().0.clone(); // only gossipsub at the moment - assert!( - topic_peers == peers[..2].to_vec(), - "Two peers should be added to the first three topics" - ); - } - - // Peer 0 unsubscribes from the first topic - - gs.handle_received_subscriptions( - &vec![GossipsubSubscription { - action: GossipsubSubscriptionAction::Unsubscribe, - topic_hash: topic_hashes[0].clone(), - }], - &peers[0], - ); - - let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); - assert!( - peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[1..3].to_vec()), - "Peer should be subscribed to two topics" - ); - - let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().0.clone(); // only gossipsub at the moment - assert!( - topic_peers == peers[1..2].to_vec(), - "Only the second peers should be in the first topic" - ); - } - - #[test] - /// Test Gossipsub.get_random_peers() function - fn test_get_random_peers() { - // generate a default GossipsubConfig - let gs_config = GossipsubConfig::default(); - // create a gossipsub struct - let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); - - // create a topic and fill it with some peers - let topic_hash = TopicBuilder::new("Test").build().hash().clone(); - let mut peers = vec![]; - for _ in 0..20 { - peers.push(PeerId::random()) - } - - gs.topic_peers - .insert(topic_hash.clone(), (peers.clone(), vec![])); - - let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| true }); - assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); - let random_peers = gs.get_random_peers(&topic_hash, 30, { |_| true }); - assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); - assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic_hash, 20, { |_| true }); - assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); - assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic_hash, 0, { |_| true }); - assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); - // test the filter - let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| false }); - assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); - let random_peers = gs.get_random_peers(&topic_hash, 10, { |peer| peers.contains(peer) }); - assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); - } - - #[test] - // tests that an event is created when a peer asks for a message in our cache - fn test_handle_iwant_msg_cached() { - let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); - - let message = GossipsubMessage { - source: peers[11].clone(), - data: vec![1,2,3,4], - sequence_number: vec![2,4,3], - topics: Vec::new(), - }; - let msg_id = message.id(); - gs.mcache.put(message.clone()); - - let eventsBefore = gs.events.len(); - gs.handle_iwant(&peers[7], vec![msg_id.clone()]); - let eventsAfter = gs.events.len(); - - assert_eq!(eventsBefore + 1, eventsAfter, "Expected event count to increase"); - } - - #[test] - // tests that an event is not created when a peers asks for a message not in our cache - fn test_handle_iwant_msg_not_cached() { - let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); - - let eventsBefore = gs.events.len(); - gs.handle_iwant(&peers[7], vec![String::from("unknown id")]); - let eventsAfter = gs.events.len(); - - assert_eq!(eventsBefore, eventsAfter, "Expected event count to stay the same"); - } - - #[test] - // tests that an event is created when a peer shares that it has a message we want - fn test_handle_ihave_subscribed_and_msg_not_cached() { - let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); - - let eventsBefore = gs.events.len(); - gs.handle_ihave(&peers[7], vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])]); - let eventsAfter = gs.events.len(); - - assert_eq!(eventsBefore + 1, eventsAfter, "Expected event count to increase") - } - - #[test] - // tests that an event is not created when a peer shares that it has a message that - // we already have - fn test_handle_ihave_subscribed_and_msg_cached() { - let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); - - let msg_id = String::from("known id"); - gs.received.add(&msg_id); - - let eventsBefore = gs.events.len(); - gs.handle_ihave(&peers[7], vec![(topic_hashes[0].clone(), vec![msg_id])]); - let eventsAfter = gs.events.len(); - - assert_eq!(eventsBefore, eventsAfter, "Expected event count to stay the same") - } - - #[test] - // test that an event is not created when a peer shares that it has a message in - // a topic that we are not subscribed to - fn test_handle_ihave_not_subscribed() { - let (mut gs, peers, _) = build_and_inject_nodes(20, vec![], true); - - let eventsBefore = gs.events.len(); - gs.handle_ihave(&peers[7], vec![(TopicHash::from_raw(String::from("unsubscribed topic")), vec![String::from("irrelevant id")])]); - let eventsAfter = gs.events.len(); - - assert_eq!(eventsBefore, eventsAfter, "Expected event count to stay the same") - } -} diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs new file mode 100644 index 00000000000..cb53dde04d7 --- /dev/null +++ b/protocols/gossipsub/src/layer/tests.rs @@ -0,0 +1,579 @@ +// collection of tests for the gossipsub network behaviour + +#[cfg(test)] +mod tests { + use super::*; + use libp2p_floodsub::TopicBuilder; + + // helper functions for testing + + // This function generates `peer_no` random PeerId's, subscribes to `topics` and subscribes the + // injected nodes to all topics if `to_subscribe` is set. All nodes are considered gossipsub nodes. + fn build_and_inject_nodes( + peer_no: usize, + topics: Vec, + to_subscribe: bool, + ) -> ( + Gossipsub, + Vec, + Vec, + ) { + // generate a default GossipsubConfig + let gs_config = GossipsubConfig::default(); + // create a gossipsub struct + let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); + + let mut topic_hashes = vec![]; + + // subscribe to the topics + for t in topics { + let topic = TopicBuilder::new(t).build(); + gs.subscribe(topic.clone()); + topic_hashes.push(topic.hash().clone()); + } + + // build and connect peer_no random peers + let mut peers = vec![]; + let dummy_connected_point = ConnectedPoint::Dialer { + address: "/ip4/0.0.0.0/tcp/0".parse().unwrap(), + }; + + for _ in 0..peer_no { + let peer = PeerId::random(); + peers.push(peer.clone()); + as NetworkBehaviour>::inject_connected( + &mut gs, + peer.clone(), + dummy_connected_point.clone(), + ); + if to_subscribe { + gs.handle_received_subscriptions( + &topic_hashes + .iter() + .cloned() + .map(|t| GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: t, + }) + .collect(), + &peer, + ); + }; + } + + return (gs, peers, topic_hashes); + } + + #[test] + /// Test local node subscribing to a topic + fn test_subscribe() { + // The node should: + // - Create an empty vector in mesh[topic] + // - Send subscription request to all peers + // - run JOIN(topic) + + let subscribe_topic = vec![String::from("test_subscribe")]; + let (gs, _, topic_hashes) = build_and_inject_nodes(20, subscribe_topic, true); + + assert!( + *gs.mesh.get(&topic_hashes[0]).expect("Mesh should exist") == Vec::::new(), + "Subscribe should add a new entry to the mesh[topic] hashmap" + ); + + // collect all the subscriptions + let subscriptions = + gs.events + .iter() + .fold(vec![], |mut collected_subscriptions, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for s in &event.subscriptions { + match s.action { + GossipsubSubscriptionAction::Subscribe => { + collected_subscriptions.push(s.clone()) + } + _ => {} + }; + } + collected_subscriptions + } + _ => collected_subscriptions, + }); + + // we sent a subscribe to all known peers + assert!( + subscriptions.len() == 20, + "Should send a subscription to all known peers" + ); + } + + #[test] + /// Test unsubscribe. + fn test_unsubscribe() { + // Unsubscribe should: + // - Remove the mesh entry for topic + // - Send UNSUBSCRIBE to all known peers + // - Call Leave + + let topic_strings = vec![String::from("topic1"), String::from("topic2")]; + let topics = topic_strings + .iter() + .map(|t| TopicBuilder::new(t.clone()).build()) + .collect::>(); + + // subscribe to topic_strings + let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); + + for topic_hash in &topic_hashes { + assert!( + gs.topic_peers.get(&topic_hash).is_some(), + "Topic_peers contain a topic entry" + ); + assert!( + gs.mesh.get(&topic_hash).is_some(), + "mesh should contain a topic entry" + ); + } + + // unsubscribe from both topics + assert!( + gs.unsubscribe(topics[0].clone()), + "should be able to unsubscribe successfully from each topic", + ); + assert!( + gs.unsubscribe(topics[1].clone()), + "should be able to unsubscribe successfully from each topic", + ); + + let subscriptions = + gs.events + .iter() + .fold(vec![], |mut collected_subscriptions, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for s in &event.subscriptions { + match s.action { + GossipsubSubscriptionAction::Unsubscribe => { + collected_subscriptions.push(s.clone()) + } + _ => {} + }; + } + collected_subscriptions + } + _ => collected_subscriptions, + }); + + // we sent a unsubscribe to all known peers, for two topics + assert!( + subscriptions.len() == 40, + "Should send an unsubscribe event to all known peers" + ); + + // check we clean up internal structures + for topic_hash in &topic_hashes { + assert!( + gs.mesh.get(&topic_hash).is_none(), + "All topics should have been removed from the mesh" + ); + } + } + + #[test] + /// Test JOIN(topic) functionality. + fn test_join() { + // The Join function should: + // - Remove peers from fanout[topic] + // - Add any fanout[topic] peers to the mesh (up to mesh_n) + // - Fill up to mesh_n peers from known gossipsub peers in the topic + // - Send GRAFT messages to all nodes added to the mesh + + // This test is not an isolated unit test, rather it uses higher level, + // subscribe/unsubscribe to perform the test. + + let topic_strings = vec![String::from("topic1"), String::from("topic2")]; + let topics = topic_strings + .iter() + .map(|t| TopicBuilder::new(t.clone()).build()) + .collect::>(); + + let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); + + // unsubscribe, then call join to invoke functionality + assert!( + gs.unsubscribe(topics[0].clone()), + "should be able to unsubscribe successfully" + ); + assert!( + gs.unsubscribe(topics[1].clone()), + "should be able to unsubscribe successfully" + ); + + // re-subscribe - there should be peers associated with the topic + assert!( + gs.subscribe(topics[0].clone()), + "should be able to subscribe successfully" + ); + + // should have added mesh_n nodes to the mesh + assert!( + gs.mesh.get(&topic_hashes[0]).unwrap().len() == 6, + "Should have added 6 nodes to the mesh" + ); + + // there should be mesh_n GRAFT messages. + let graft_messages = gs + .events + .iter() + .fold(vec![], |mut collected_grafts, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for c in &event.control_msgs { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} + }; + } + collected_grafts + } + _ => collected_grafts, + }); + + assert!( + graft_messages.len() == 6, + "There should be 6 grafts messages sent to peers" + ); + + // verify fanout nodes + // add 3 random peers to the fanout[topic1] + gs.fanout.insert(topic_hashes[1].clone(), vec![]); + let new_peers = vec![]; + for _ in 0..3 { + let mut fanout_peers = gs.fanout.get_mut(&topic_hashes[1]).unwrap(); + fanout_peers.push(PeerId::random()); + } + + // subscribe to topic1 + gs.subscribe(topics[1].clone()); + + // the three new peers should have been added, along with 3 more from the pool. + assert!( + gs.mesh.get(&topic_hashes[1]).unwrap().len() == 6, + "Should have added 6 nodes to the mesh" + ); + let mesh_peers = gs.mesh.get(&topic_hashes[1]).unwrap(); + for new_peer in new_peers { + assert!( + mesh_peers.contains(new_peer), + "Fanout peer should be included in the mesh" + ); + } + + // there should now be 12 graft messages to be sent + let graft_messages = gs + .events + .iter() + .fold(vec![], |mut collected_grafts, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for c in &event.control_msgs { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} + }; + } + collected_grafts + } + _ => collected_grafts, + }); + + assert!( + graft_messages.len() == 12, + "There should be 6 grafts messages sent to peers" + ); + } + + #[test] + /// Test the gossipsub NetworkBehaviour peer connection logic. + fn test_inject_connected() { + let (gs, peers, topic_hashes) = build_and_inject_nodes( + 20, + vec![String::from("topic1"), String::from("topic2")], + true, + ); + + // check that our subscriptions are sent to each of the peers + // collect all the SendEvents + let send_events: Vec<&NetworkBehaviourAction> = gs + .events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::SendEvent { + peer_id: _, + event: _, + } => true, + _ => false, + }) + .collect(); + + // check that there are two subscriptions sent to each peer + for sevent in send_events.clone() { + match sevent { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + assert!( + event.subscriptions.len() == 2, + "There should be two subscriptions sent to each peer (1 for each topic)." + ); + } + _ => {} + }; + } + + // check that there are 20 send events created + assert!( + send_events.len() == 20, + "There should be a subscription event sent to each peer." + ); + + // should add the new peers to `peer_topics` with an empty vec as a gossipsub node + for peer in peers { + let known_topics = &gs.peer_topics.get(&peer).unwrap().0; + let node_type = &gs.peer_topics.get(&peer).unwrap().1; + assert!( + known_topics == &SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes.clone()), + "The topics for each node should all topics" + ); + // TODO: Update this for handling floodsub nodes + assert!( + match node_type { + NodeType::Gossipsub => true, + _ => false, + }, + "All peers should be added as a gossipsub node" + ); + } + } + + #[test] + /// Test subscription handling + fn test_handle_received_subscriptions() { + // For every subscription: + // SUBSCRIBE: - Add subscribed topic to peer_topics for peer. + // - Add peer to topics_peer. + // UNSUBSCRIBE - Remove topic from peer_topics for peer. + // - Remove peer from topic_peers. + + let topics = vec!["topic1", "topic2", "topic3", "topic4"] + .iter() + .map(|&t| String::from(t)) + .collect(); + let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics, false); + + // The first peer sends 3 subscriptions and 1 unsubscription + let mut subscriptions = topic_hashes[..3] + .iter() + .map(|topic_hash| GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: topic_hash.clone(), + }) + .collect::>(); + + subscriptions.push(GossipsubSubscription { + action: GossipsubSubscriptionAction::Unsubscribe, + topic_hash: topic_hashes[topic_hashes.len() - 1].clone(), + }); + + let unknown_peer = PeerId::random(); + // process the subscriptions + // first and second peers send subscriptions + gs.handle_received_subscriptions(&subscriptions, &peers[0]); + gs.handle_received_subscriptions(&subscriptions, &peers[1]); + // unknown peer sends the same subscriptions + gs.handle_received_subscriptions(&subscriptions, &unknown_peer); + + // verify the result + + let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); + assert!( + peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), + "First peer should be subscribed to three topics" + ); + let peer_topics = gs.peer_topics.get(&peers[1]).unwrap().0.clone(); + assert!( + peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), + "Second peer should be subscribed to three topics" + ); + + assert!( + gs.peer_topics.get(&unknown_peer).is_none(), + "Unknown peer should not have been added" + ); + + for topic_hash in topic_hashes[..3].iter() { + let topic_peers = gs.topic_peers.get(topic_hash).unwrap().0.clone(); // only gossipsub at the moment + assert!( + topic_peers == peers[..2].to_vec(), + "Two peers should be added to the first three topics" + ); + } + + // Peer 0 unsubscribes from the first topic + + gs.handle_received_subscriptions( + &vec![GossipsubSubscription { + action: GossipsubSubscriptionAction::Unsubscribe, + topic_hash: topic_hashes[0].clone(), + }], + &peers[0], + ); + + let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); + assert!( + peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[1..3].to_vec()), + "Peer should be subscribed to two topics" + ); + + let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().0.clone(); // only gossipsub at the moment + assert!( + topic_peers == peers[1..2].to_vec(), + "Only the second peers should be in the first topic" + ); + } + + #[test] + /// Test Gossipsub.get_random_peers() function + fn test_get_random_peers() { + // generate a default GossipsubConfig + let gs_config = GossipsubConfig::default(); + // create a gossipsub struct + let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); + + // create a topic and fill it with some peers + let topic_hash = TopicBuilder::new("Test").build().hash().clone(); + let mut peers = vec![]; + for _ in 0..20 { + peers.push(PeerId::random()) + } + + gs.topic_peers + .insert(topic_hash.clone(), (peers.clone(), vec![])); + + let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| true }); + assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); + let random_peers = gs.get_random_peers(&topic_hash, 30, { |_| true }); + assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); + assert!(random_peers == peers, "Expected no shuffling"); + let random_peers = gs.get_random_peers(&topic_hash, 20, { |_| true }); + assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); + assert!(random_peers == peers, "Expected no shuffling"); + let random_peers = gs.get_random_peers(&topic_hash, 0, { |_| true }); + assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); + // test the filter + let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| false }); + assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); + let random_peers = gs.get_random_peers(&topic_hash, 10, { |peer| peers.contains(peer) }); + assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); + } + + #[test] + // tests that an event is created when a peer asks for a message in our cache + fn test_handle_iwant_msg_cached() { + let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + + let message = GossipsubMessage { + source: peers[11].clone(), + data: vec![1, 2, 3, 4], + sequence_number: vec![2, 4, 3], + topics: Vec::new(), + }; + let msg_id = message.id(); + gs.mcache.put(message.clone()); + + let eventsBefore = gs.events.len(); + gs.handle_iwant(&peers[7], vec![msg_id.clone()]); + let eventsAfter = gs.events.len(); + + assert_eq!( + eventsBefore + 1, + eventsAfter, + "Expected event count to increase" + ); + } + + #[test] + // tests that an event is not created when a peers asks for a message not in our cache + fn test_handle_iwant_msg_not_cached() { + let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + + let eventsBefore = gs.events.len(); + gs.handle_iwant(&peers[7], vec![String::from("unknown id")]); + let eventsAfter = gs.events.len(); + + assert_eq!( + eventsBefore, eventsAfter, + "Expected event count to stay the same" + ); + } + + #[test] + // tests that an event is created when a peer shares that it has a message we want + fn test_handle_ihave_subscribed_and_msg_not_cached() { + let (mut gs, peers, topic_hashes) = + build_and_inject_nodes(20, vec![String::from("topic1")], true); + + let eventsBefore = gs.events.len(); + gs.handle_ihave( + &peers[7], + vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])], + ); + let eventsAfter = gs.events.len(); + + assert_eq!( + eventsBefore + 1, + eventsAfter, + "Expected event count to increase" + ) + } + + #[test] + // tests that an event is not created when a peer shares that it has a message that + // we already have + fn test_handle_ihave_subscribed_and_msg_cached() { + let (mut gs, peers, topic_hashes) = + build_and_inject_nodes(20, vec![String::from("topic1")], true); + + let msg_id = String::from("known id"); + gs.received.add(&msg_id); + + let eventsBefore = gs.events.len(); + gs.handle_ihave(&peers[7], vec![(topic_hashes[0].clone(), vec![msg_id])]); + let eventsAfter = gs.events.len(); + + assert_eq!( + eventsBefore, eventsAfter, + "Expected event count to stay the same" + ) + } + + #[test] + // test that an event is not created when a peer shares that it has a message in + // a topic that we are not subscribed to + fn test_handle_ihave_not_subscribed() { + let (mut gs, peers, _) = build_and_inject_nodes(20, vec![], true); + + let eventsBefore = gs.events.len(); + gs.handle_ihave( + &peers[7], + vec![( + TopicHash::from_raw(String::from("unsubscribed topic")), + vec![String::from("irrelevant id")], + )], + ); + let eventsAfter = gs.events.len(); + + assert_eq!( + eventsBefore, eventsAfter, + "Expected event count to stay the same" + ) + } +} From 6a126e6aa423ff0f5a3bd5a636ddf79db8277ece Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 1 Feb 2019 15:14:00 +1100 Subject: [PATCH 34/91] Implement clippy suggestions for gossipsub. --- protocols/gossipsub/src/layer.rs | 43 +++++++++++++------------- protocols/gossipsub/src/layer/tests.rs | 36 ++++++++++----------- protocols/gossipsub/src/protocol.rs | 9 +++--- protocols/gossipsub/src/rpc_proto.rs | 2 +- 4 files changed, 45 insertions(+), 45 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 9eb1fdde687..c98c5b86759 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -200,8 +200,8 @@ impl Gossipsub { return false; } - // send subscription request to all floodsub and gossipsub peers - for (gossip_peers, flood_peers) in self.topic_peers.get(&topic.hash()) { + // send subscription request to all floodsub and gossipsub peers in the topic + if let Some((gossip_peers, flood_peers)) = self.topic_peers.get(&topic.hash()) { for peer in flood_peers.iter().chain(gossip_peers) { debug!("Sending SUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -241,7 +241,7 @@ impl Gossipsub { } // announce to all floodsub and gossipsub peers, in the topic - for (gossip_peers, flood_peers) in self.topic_peers.get(topic_hash) { + if let Some((gossip_peers, flood_peers)) = self.topic_peers.get(topic_hash) { for peer in flood_peers.iter().chain(gossip_peers) { debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -365,10 +365,12 @@ impl Gossipsub { ); // add up to mesh_n of them them to the mesh // Note: These aren't randomly added, currently FIFO - let mut add_peers = self.config.mesh_n.clone(); - if peers.len() < self.config.mesh_n { - add_peers = peers.len(); - } + let mut add_peers = if peers.len() < self.config.mesh_n { + peers.len() + } else { + self.config.mesh_n + }; + debug!( "JOIN: Adding {:?} peers from the fanout for topic: {:?}", add_peers, topic_hash @@ -393,7 +395,10 @@ impl Gossipsub { "JOIN: Inserting {:?} random peers into the mesh", new_peers.len() ); - let mesh_peers = self.mesh.entry(topic_hash.clone()).or_insert(vec![]); + let mesh_peers = self + .mesh + .entry(topic_hash.clone()) + .or_insert_with(|| vec![]); mesh_peers.append(&mut new_peers); } @@ -612,7 +617,7 @@ impl Gossipsub { /// Handles received subscriptions. fn handle_received_subscriptions( &mut self, - subscriptions: &Vec, + subscriptions: &[GossipsubSubscription], propagation_source: &PeerId, ) { debug!( @@ -740,7 +745,7 @@ impl Gossipsub { for peer in peer_list { peers.push(peer.clone()); // TODO: tagPeer - let current_topic = to_graft.entry(peer).or_insert(Vec::new()); + let current_topic = to_graft.entry(peer).or_insert_with(|| vec![]); current_topic.push(topic_hash.clone()); } // update the mesh @@ -764,7 +769,7 @@ impl Gossipsub { let peer = peers .pop() .expect("There should always be enough peers to remove"); - let current_topic = to_prune.entry(peer).or_insert(vec![]); + let current_topic = to_prune.entry(peer).or_insert_with(|| vec![]); current_topic.push(topic_hash.clone()); //TODO: untagPeer } @@ -882,10 +887,8 @@ impl Gossipsub { for (peer, topics) in to_graft.iter() { let mut grafts: Vec = topics .iter() - .map(|topic_hash| { - return GossipsubControlAction::Graft { - topic_hash: topic_hash.clone(), - }; + .map(|topic_hash| GossipsubControlAction::Graft { + topic_hash: topic_hash.clone(), }) .collect(); let mut prunes: Vec = to_prune @@ -997,7 +1000,7 @@ impl Gossipsub { debug!("RANDOM PEERS: Got {:?} peers", n); - return gossip_peers[..n].to_vec(); + gossip_peers[..n].to_vec() } } @@ -1174,12 +1177,8 @@ where return Async::Ready(event); } - loop { - match self.heartbeat.poll() { - // heartbeat ready - Ok(Async::Ready(Some(_))) => self.heartbeat(), - _ => break, - }; + while let Ok(Async::Ready(Some(_))) = self.heartbeat.poll() { + self.heartbeat(); } Async::NotReady diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index cb53dde04d7..5da81052328 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -2,7 +2,7 @@ #[cfg(test)] mod tests { - use super::*; + use super::super::*; use libp2p_floodsub::TopicBuilder; // helper functions for testing @@ -489,13 +489,13 @@ mod tests { let msg_id = message.id(); gs.mcache.put(message.clone()); - let eventsBefore = gs.events.len(); + let events_before = gs.events.len(); gs.handle_iwant(&peers[7], vec![msg_id.clone()]); - let eventsAfter = gs.events.len(); + let events_after = gs.events.len(); assert_eq!( - eventsBefore + 1, - eventsAfter, + events_before + 1, + events_after, "Expected event count to increase" ); } @@ -505,12 +505,12 @@ mod tests { fn test_handle_iwant_msg_not_cached() { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); - let eventsBefore = gs.events.len(); + let events_before = gs.events.len(); gs.handle_iwant(&peers[7], vec![String::from("unknown id")]); - let eventsAfter = gs.events.len(); + let events_after = gs.events.len(); assert_eq!( - eventsBefore, eventsAfter, + events_before, events_after, "Expected event count to stay the same" ); } @@ -521,16 +521,16 @@ mod tests { let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); - let eventsBefore = gs.events.len(); + let events_before = gs.events.len(); gs.handle_ihave( &peers[7], vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])], ); - let eventsAfter = gs.events.len(); + let events_after = gs.events.len(); assert_eq!( - eventsBefore + 1, - eventsAfter, + events_before + 1, + events_after, "Expected event count to increase" ) } @@ -545,12 +545,12 @@ mod tests { let msg_id = String::from("known id"); gs.received.add(&msg_id); - let eventsBefore = gs.events.len(); + let events_before = gs.events.len(); gs.handle_ihave(&peers[7], vec![(topic_hashes[0].clone(), vec![msg_id])]); - let eventsAfter = gs.events.len(); + let events_after = gs.events.len(); assert_eq!( - eventsBefore, eventsAfter, + events_before, events_after, "Expected event count to stay the same" ) } @@ -561,7 +561,7 @@ mod tests { fn test_handle_ihave_not_subscribed() { let (mut gs, peers, _) = build_and_inject_nodes(20, vec![], true); - let eventsBefore = gs.events.len(); + let events_before = gs.events.len(); gs.handle_ihave( &peers[7], vec![( @@ -569,10 +569,10 @@ mod tests { vec![String::from("irrelevant id")], )], ); - let eventsAfter = gs.events.len(); + let events_after = gs.events.len(); assert_eq!( - eventsBefore, eventsAfter, + events_before, events_after, "Expected event count to stay the same" ) } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 57579bb2700..caf63b135b6 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -330,10 +330,11 @@ impl GossipsubMessage { // the sequence number is a big endian uint64 (as per go implementation) // avoid a potential panic by setting the seqno to 0 if it is not long enough. // TODO: Check that this doesn't introduce a vulnerability or issue - let mut seqno = 0; - if self.sequence_number.len() >= 8 { - seqno = BigEndian::read_u64(&self.sequence_number); - } + let seqno = if self.sequence_number.len() >= 8 { + BigEndian::read_u64(&self.sequence_number) + } else { + 0 + }; source_string.push_str(&seqno.to_string()); source_string } diff --git a/protocols/gossipsub/src/rpc_proto.rs b/protocols/gossipsub/src/rpc_proto.rs index 4aff674e1cf..0fe3053f690 100644 --- a/protocols/gossipsub/src/rpc_proto.rs +++ b/protocols/gossipsub/src/rpc_proto.rs @@ -3,7 +3,7 @@ // https://github.com/Manishearth/rust-clippy/issues/702 #![allow(unknown_lints)] -#![allow(clippy)] +#![allow(clippy::all)] #![cfg_attr(rustfmt, rustfmt_skip)] From f8277fad081fc4efa31c4c669a2eb37345313d9f Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 1 Feb 2019 17:34:32 +1100 Subject: [PATCH 35/91] Modify control message tests for specific types. --- protocols/gossipsub/src/layer.rs | 2 +- protocols/gossipsub/src/layer/tests.rs | 91 +++++++++++++++++++++----- 2 files changed, 77 insertions(+), 16 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index c98c5b86759..0275ccd4317 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -617,7 +617,7 @@ impl Gossipsub { /// Handles received subscriptions. fn handle_received_subscriptions( &mut self, - subscriptions: &[GossipsubSubscription], + subscriptions: &Vec, propagation_source: &PeerId, ) { debug!( diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 5da81052328..a3dfbe63436 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -475,31 +475,85 @@ mod tests { assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); } + /// Tests that the correct message is sent when a peer asks for a message in our cache. #[test] - // tests that an event is created when a peer asks for a message in our cache fn test_handle_iwant_msg_cached() { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); let message = GossipsubMessage { source: peers[11].clone(), data: vec![1, 2, 3, 4], - sequence_number: vec![2, 4, 3], + sequence_number: vec![0, 0, 0, 0, 0, 0, 0, 1], topics: Vec::new(), }; let msg_id = message.id(); gs.mcache.put(message.clone()); - let events_before = gs.events.len(); gs.handle_iwant(&peers[7], vec![msg_id.clone()]); - let events_after = gs.events.len(); - assert_eq!( - events_before + 1, - events_after, - "Expected event count to increase" + // the messages we are sending + let sent_messages = gs + .events + .iter() + .fold(vec![], |mut collected_messages, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for c in &event.messages { + collected_messages.push(c.clone()) + } + collected_messages + } + _ => collected_messages, + }); + + assert!( + sent_messages.iter().any(|msg| msg.id() == msg_id), + "Expected the cached message to be sent to an IWANT peer" ); } + /// Tests that messages are sent correctly depending on the shifting of the message cache. + #[test] + fn test_handle_iwant_msg_cached_shifted() { + let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + + // perform 10 memshifts and check that it leaves the cache + for shift in 1..10 { + let message = GossipsubMessage { + source: peers[11].clone(), + data: vec![1, 2, 3, 4], + sequence_number: vec![0, 0, 0, 0, 0, 0, 0, shift], + topics: Vec::new(), + }; + let msg_id = message.id(); + gs.mcache.put(message.clone()); + for _ in 0..shift { + gs.mcache.shift(); + } + + gs.handle_iwant(&peers[7], vec![msg_id.clone()]); + + // is the message is being sent? + let message_exists = gs.events.iter().any(|e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + event.messages.iter().any(|msg| msg.id() == msg_id) + } + _ => false, + }); + // default history_length is 5, expect no messages after shift > 5 + if shift < 5 { + assert!( + message_exists, + "Expected the cached message to be sent to an IWANT peer before 5 shifts" + ); + } else { + assert!( + !message_exists, + "Expected the cached message to not be sent to an IWANT peer after 5 shifts" + ); + } + } + } + #[test] // tests that an event is not created when a peers asks for a message not in our cache fn test_handle_iwant_msg_not_cached() { @@ -521,18 +575,25 @@ mod tests { let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); - let events_before = gs.events.len(); gs.handle_ihave( &peers[7], vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])], ); - let events_after = gs.events.len(); + // check that we sent an IWANT request for `unknown id` + let iwant_exists = gs.events.iter().any(|e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + event.control_msgs.iter().any(|e| match e { + GossipsubControlAction::IWant { message_ids } => { + message_ids.iter().any(|m| *m == String::from("unknown id")) + } + _ => false, + }) + } + _ => false, + }); - assert_eq!( - events_before + 1, - events_after, - "Expected event count to increase" - ) + assert!(iwant_exists + "Expected to send an IWANT control message for unkown message id"); } #[test] From 9165964f46708d6c59a017b90a05230c2dd3edaa Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 2 Feb 2019 14:32:03 +1100 Subject: [PATCH 36/91] Implement builder pattern for GossipsubConfig. As suggested by @twittner - The builder pattern for building GossipsubConfig struct is implemented. --- protocols/gossipsub/examples/chat.rs | 16 +- protocols/gossipsub/src/gossipsub_config.rs | 167 ++++++++++++++++++++ protocols/gossipsub/src/layer.rs | 81 +--------- protocols/gossipsub/src/lib.rs | 6 +- 4 files changed, 177 insertions(+), 93 deletions(-) create mode 100644 protocols/gossipsub/src/gossipsub_config.rs diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index d770cc69c95..587d4a3a2a9 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -30,19 +30,11 @@ fn main() { // 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 = gossipsub::GossipsubConfig::default(); // set custom gossipsub - let gossipsub_config = gossipsub::GossipsubConfig::new( - 5, - 3, - 6, - 4, - 12, - 6, - Duration::from_secs(10), - Duration::from_secs(10), - Duration::from_secs(60), - ); + let gossipsub_config = gossipsub::GossipsubConfigBuilder::new() + .heartbeat_interval(Duration::from_secs(10)) + .build(); // build a gossipsub network behaviour let mut gossipsub = gossipsub::Gossipsub::new(local_peer_id.clone(), gossipsub_config); gossipsub.subscribe(topic.clone()); diff --git a/protocols/gossipsub/src/gossipsub_config.rs b/protocols/gossipsub/src/gossipsub_config.rs new file mode 100644 index 00000000000..1c60d30a94e --- /dev/null +++ b/protocols/gossipsub/src/gossipsub_config.rs @@ -0,0 +1,167 @@ +use std::time::Duration; + +/// Configuration parameters that define the performance of the gossipsub network. +#[derive(Clone)] +pub struct GossipsubConfig { + /// Overlay network parameters. + /// Number of heartbeats to keep in the `memcache`. + pub history_length: usize, + /// Number of past heartbeats to gossip about. + pub history_gossip: usize, + + /// Target number of peers for the mesh network (D in the spec). + pub mesh_n: usize, + /// Minimum number of peers in mesh network before adding more (D_lo in the spec). + pub mesh_n_low: usize, + /// Maximum number of peers in mesh network before removing some (D_high in the spec). + pub mesh_n_high: usize, + + /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). + pub gossip_lazy: usize, + + /// Initial delay in each heartbeat. + pub heartbeat_initial_delay: Duration, + /// Time between each heartbeat. + pub heartbeat_interval: Duration, + /// Time to live for fanout peers. + pub fanout_ttl: Duration, +} + +impl Default for GossipsubConfig { + fn default() -> GossipsubConfig { + GossipsubConfig { + history_length: 5, + history_gossip: 3, + mesh_n: 6, + mesh_n_low: 4, + 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), + } + } +} + +pub struct GossipsubConfigBuilder { + history_length: usize, + /// Number of past heartbeats to gossip about. + history_gossip: usize, + + /// Target number of peers for the mesh network (D in the spec). + mesh_n: usize, + /// Minimum number of peers in mesh network before adding more (D_lo in the spec). + mesh_n_low: usize, + /// Maximum number of peers in mesh network before removing some (D_high in the spec). + mesh_n_high: usize, + + /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). + gossip_lazy: usize, + + /// Initial delay in each heartbeat. + heartbeat_initial_delay: Duration, + /// Time between each heartbeat. + heartbeat_interval: Duration, + /// Time to live for fanout peers. + fanout_ttl: Duration, +} + +impl Default for GossipsubConfigBuilder { + fn default() -> GossipsubConfigBuilder { + GossipsubConfigBuilder { + history_length: 5, + history_gossip: 3, + mesh_n: 6, + mesh_n_low: 4, + 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), + } + } +} + +impl GossipsubConfigBuilder { + // set default values + pub fn new() -> GossipsubConfigBuilder { + GossipsubConfigBuilder::default() + } + + pub fn history_length(&mut self, history_length: usize) -> &mut Self { + assert!( + history_length >= self.history_gossip, + "The history_length must be greater than or equal to the history_gossip length" + ); + self.history_length = history_length; + self + } + + pub fn history_gossip(&mut self, history_gossip: usize) -> &mut Self { + assert!( + self.history_length >= history_gossip, + "The history_length must be greater than or equal to the history_gossip length" + ); + self.history_gossip = history_gossip; + self + } + + pub fn mesh_n(&mut self, mesh_n: usize) -> &mut Self { + assert!( + self.mesh_n_low <= mesh_n && mesh_n <= self.mesh_n_high, + "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" + ); + self.mesh_n = mesh_n; + self + } + + pub fn mesh_n_low(&mut self, mesh_n_low: usize) -> &mut Self { + assert!( + mesh_n_low <= self.mesh_n && self.mesh_n <= self.mesh_n_high, + "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" + ); + self.mesh_n_low = mesh_n_low; + self + } + + pub fn mesh_n_high(&mut self, mesh_n_high: usize) -> &mut Self { + assert!( + self.mesh_n_low <= self.mesh_n && self.mesh_n <= mesh_n_high, + "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" + ); + self.mesh_n_high = mesh_n_high; + self + } + + pub fn gossip_lazy(&mut self, gossip_lazy: usize) -> &mut Self { + self.gossip_lazy = gossip_lazy; + self + } + + pub fn heartbeat_initial_delay(&mut self, heartbeat_initial_delay: Duration) -> &mut Self { + self.heartbeat_initial_delay = heartbeat_initial_delay; + self + } + pub fn heartbeat_interval(&mut self, heartbeat_interval: Duration) -> &mut Self { + self.heartbeat_interval = heartbeat_interval; + self + } + pub fn fanout_ttl(&mut self, fanout_ttl: Duration) -> &mut Self { + self.fanout_ttl = fanout_ttl; + self + } + + pub fn build(&self) -> GossipsubConfig { + GossipsubConfig { + history_length: self.history_length, + history_gossip: self.history_gossip, + mesh_n: self.mesh_n, + mesh_n_low: self.mesh_n_low, + mesh_n_high: self.mesh_n_high, + gossip_lazy: self.gossip_lazy, + heartbeat_initial_delay: self.heartbeat_initial_delay, + heartbeat_interval: self.heartbeat_interval, + fanout_ttl: self.fanout_ttl, + } + } +} diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 0275ccd4317..02aad46a130 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -23,6 +23,7 @@ use cuckoofilter::CuckooFilter; use futures::prelude::*; +use gossipsub_config::GossipsubConfig; use libp2p_core::swarm::{ ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters, }; @@ -40,91 +41,13 @@ use rand; use rand::{seq::SliceRandom, thread_rng}; use smallvec::SmallVec; use std::collections::hash_map::{DefaultHasher, HashMap}; -use std::time::{Duration, Instant}; +use std::time::Instant; use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; use tokio_timer::Interval; mod tests; -// potentially rename this struct - due to clashes -/// Configuration parameters that define the performance of the gossipsub network. -#[derive(Clone)] -pub struct GossipsubConfig { - /// Overlay network parameters. - /// Number of heartbeats to keep in the `memcache`. - history_length: usize, - /// Number of past heartbeats to gossip about. - history_gossip: usize, - - /// Target number of peers for the mesh network (D in the spec). - mesh_n: usize, - /// Minimum number of peers in mesh network before adding more (D_lo in the spec). - mesh_n_low: usize, - /// Maximum number of peers in mesh network before removing some (D_high in the spec). - mesh_n_high: usize, - - /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). - gossip_lazy: usize, - - /// Initial delay in each heartbeat. - heartbeat_initial_delay: Duration, - /// Time between each heartbeat. - heartbeat_interval: Duration, - /// Time to live for fanout peers. - fanout_ttl: Duration, -} - -impl Default for GossipsubConfig { - fn default() -> GossipsubConfig { - GossipsubConfig { - history_length: 5, - history_gossip: 3, - mesh_n: 6, - mesh_n_low: 4, - 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), - } - } -} - -impl GossipsubConfig { - pub fn new( - history_length: usize, - history_gossip: usize, - mesh_n: usize, - mesh_n_low: usize, - mesh_n_high: usize, - gossip_lazy: usize, - heartbeat_initial_delay: Duration, - heartbeat_interval: Duration, - fanout_ttl: Duration, - ) -> GossipsubConfig { - assert!( - history_length >= history_gossip, - "The history_length must be greater than or equal to the history_gossip length" - ); - assert!( - mesh_n_low <= mesh_n && mesh_n <= mesh_n_high, - "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" - ); - GossipsubConfig { - history_length, - history_gossip, - mesh_n, - mesh_n_low, - mesh_n_high, - gossip_lazy, - heartbeat_initial_delay, - heartbeat_interval, - fanout_ttl, - } - } -} - /// Network behaviour that automatically identifies nodes periodically, and returns information /// about them. pub struct Gossipsub { diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index be4be1f81b1..04bb34bbf74 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -119,9 +119,11 @@ extern crate unsigned_varint; pub mod protocol; +mod gossipsub_config; mod layer; mod mcache; mod rpc_proto; -pub use self::layer::{Gossipsub, GossipsubConfig, GossipsubEvent}; -pub use self::protocol::*; +pub use self::gossipsub_config::{GossipsubConfig, GossipsubConfigBuilder}; +pub use self::layer::{Gossipsub, GossipsubEvent}; +pub use self::protocol::{GossipsubMessage, GossipsubRpc}; From a46ecc7bc990037a17946f847b117eb94d83c5b2 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 2 Feb 2019 15:11:00 +1100 Subject: [PATCH 37/91] Package version updates as suggested by @twittner. --- protocols/gossipsub/Cargo.toml | 4 ++-- protocols/gossipsub/regen_structs_proto.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index fc7371f688b..6bde6727470 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -13,8 +13,8 @@ byteorder = "1.3.1" cuckoofilter = "0.3.2" fnv = "1.0.6" futures = "0.1.25" -protobuf = "2.2.5" -rand = "0.6.4" +protobuf = "2.3.0" +rand = "0.6.5" smallvec = "0.6.8" tokio-codec = "0.1.1" tokio-io = "0.1.11" diff --git a/protocols/gossipsub/regen_structs_proto.sh b/protocols/gossipsub/regen_structs_proto.sh index 054ae5cdfec..fa7048178ae 100755 --- a/protocols/gossipsub/regen_structs_proto.sh +++ b/protocols/gossipsub/regen_structs_proto.sh @@ -5,7 +5,7 @@ docker run --rm -v `pwd`:/usr/code:z -w /usr/code rust /bin/bash -c " \ apt-get update; \ apt-get install -y protobuf-compiler; \ - cargo install --version 2.0.2 protobuf-codegen; \ + cargo install --version 2.3.0 protobuf-codegen; \ protoc --rust_out . rpc.proto" sudo chown $USER:$USER *.rs From ec522f2f42bef8d18d4ef34e85981de234b1b65c Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 2 Feb 2019 15:12:31 +1100 Subject: [PATCH 38/91] Correct line lengths in gossipsub. --- protocols/gossipsub/src/lib.rs | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 04bb34bbf74..d79de3cdb6e 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -1,32 +1,38 @@ -//! Gossipsub is a P2P pubsub (publish/subscription) routing layer designed to extend upon flooodsub -//! and meshsub routing protocols. +//! Gossipsub is a P2P pubsub (publish/subscription) routing layer designed to extend upon +//! flooodsub and meshsub routing protocols. //! //! # Overview //! -//! *Note: The gossipsub protocol specifications (https://github.com/libp2p/specs/tree/master/pubsub/gossipsub) provide an outline for the routing protocol. They should be consulted for further detail.* +//! *Note: The gossipsub protocol specifications +//! (https://github.com/libp2p/specs/tree/master/pubsub/gossipsub) provide an outline for the +//! routing protocol. They should be consulted for further detail.* //! //! Gossipsub is a blend of meshsub for data and randomsub for mesh metadata. It provides bounded //! degree and amplification factor with the meshsub construction and augments it using gossip //! propagation of metadata with the randomsub technique. //! -//! The router maintains an overlay mesh network of peers on which to efficiently send messages and metadata. -//! Peers use control messages to broadcast and request known messages and subscribe/unsubscribe from topics in the mesh network. +//! The router maintains an overlay mesh network of peers on which to efficiently send messages and +//! metadata. Peers use control messages to broadcast and request known messages and +//! subscribe/unsubscribe from topics in the mesh network. //! //! # Important Discrepancies //! -//! This section outlines the current implementation's potential discrepancies from that of -//! other implementations, due to undefined elements in the current specification. +//! This section outlines the current implementation's potential discrepancies from that of other +//! implementations, due to undefined elements in the current specification. //! -//! - **Topics** - In gossipsub, topics are utf-8 string's that are `base58` encoded. These are of type -//! `TopicHash`. The current go implementation uses raw utf-8 strings. -//! - **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 integers. They are chosen at random in this implementation of gossipsub, but are sequential in the current go implementation. +//! - **Topics** - In gossipsub, topics are utf-8 string's that are `base58` encoded. These are of +//! type `TopicHash`. The current go implementation uses raw utf-8 strings. - **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 integers. They are chosen at random in +//! this implementation of gossipsub, but are sequential in the current go implementation. //! //! # Using Gossipsub //! //! ## GossipsubConfig //! -//! The [`GossipsubConfig`] struct specifies various network performance/tuning configuration parameters. Specifically it specifies: +//! The [`GossipsubConfig`] struct specifies various network performance/tuning configuration +//! parameters. Specifically it specifies: //! //! [`GossipsubConfig`]: struct.GossipsubConfig.html From e5d85c9bd48142be1e4f14cf038890bb1770ddd9 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 2 Feb 2019 15:31:20 +1100 Subject: [PATCH 39/91] Correct braces in found by @twittner. --- protocols/gossipsub/src/layer.rs | 31 +++++++++++++++---------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 02aad46a130..045f3441e89 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -226,26 +226,25 @@ impl Gossipsub { debug!("Topic: {:?} not in the mesh", topic_hash); // build a list of peers to forward the message to // if we have fanout peers add them to the map - if let Some(fanout_peers) = self.fanout.get(&topic_hash) { - for peer in fanout_peers { + if self.fanout.contains_key(&topic_hash) { + for peer in self.fanout.get(&topic_hash).expect("Topic must exist") { + recipient_peers.insert(peer.clone(), ()); + } + } else { + // we have no fanout peers, select mesh_n of them and add them to the fanout + let mesh_n = self.config.mesh_n; + let new_peers = self.get_random_peers(&topic_hash, mesh_n, { |_| true }); + // add the new peers to the fanout and recipient peers + self.fanout.insert(topic_hash.clone(), new_peers.clone()); + for peer in new_peers { + debug!("Peer added to fanout: {:?}", peer); recipient_peers.insert(peer.clone(), ()); } } - } else { - // TODO: Ensure fanout key never contains an empty set - // we have no fanout peers, select mesh_n of them and add them to the fanout - let mesh_n = self.config.mesh_n; - let new_peers = self.get_random_peers(&topic_hash, mesh_n, { |_| true }); - // add the new peers to the fanout and recipient peers - self.fanout.insert(topic_hash.clone(), new_peers.clone()); - for peer in new_peers { - debug!("Peer added to fanout: {:?}", peer); - recipient_peers.insert(peer.clone(), ()); - } + // we are publishing to fanout peers - update the time we published + self.fanout_last_pub + .insert(topic_hash.clone(), Instant::now()); } - // we are publishing to fanout peers - update the time we published - self.fanout_last_pub - .insert(topic_hash.clone(), Instant::now()); } // add published message to our received caches From 4a08459b11ecb8b91aaa093b93e12b0a0020b8f3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 2 Feb 2019 16:51:42 +1100 Subject: [PATCH 40/91] Implement @twittner's suggestions. - Uses `HashSet` where applicable - Update `FnvHashMap` to standard `HashMap` - Uses `min` function in code simplification. --- protocols/gossipsub/src/layer.rs | 30 +++++++++++++----------------- protocols/gossipsub/src/mcache.rs | 6 +++--- 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 045f3441e89..c8b8bd5bc09 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -41,6 +41,7 @@ use rand; use rand::{seq::SliceRandom, thread_rng}; use smallvec::SmallVec; use std::collections::hash_map::{DefaultHasher, HashMap}; +use std::collections::HashSet; use std::time::Instant; use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; @@ -287,12 +288,7 @@ impl Gossipsub { ); // add up to mesh_n of them them to the mesh // Note: These aren't randomly added, currently FIFO - let mut add_peers = if peers.len() < self.config.mesh_n { - peers.len() - } else { - self.config.mesh_n - }; - + let add_peers = std::cmp::min(peers.len(), self.config.mesh_n); debug!( "JOIN: Adding {:?} peers from the fanout for topic: {:?}", add_peers, topic_hash @@ -372,7 +368,7 @@ impl Gossipsub { fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { debug!("Handling IHAVE for peer: {:?}", peer_id); // use a hashmap to avoid duplicates efficiently - let mut iwant_ids = HashMap::new(); + let mut iwant_ids = HashSet::new(); for (topic, ids) in ihave_msgs { // only process the message if we are subscribed @@ -387,7 +383,7 @@ impl Gossipsub { for id in ids { if !self.received.contains(&id) { // have not seen this message, request it - iwant_ids.insert(id, true); + iwant_ids.insert(id); } } } @@ -401,7 +397,7 @@ impl Gossipsub { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: vec![GossipsubControlAction::IWant { - message_ids: iwant_ids.keys().cloned().collect(), + message_ids: iwant_ids.iter().cloned().collect(), }], }, }); @@ -444,7 +440,7 @@ impl Gossipsub { fn handle_graft(&mut self, peer_id: &PeerId, topics: Vec) { debug!("Handling GRAFT message for peer: {:?}", peer_id); - let mut to_prune_topics = HashMap::new(); + let mut to_prune_topics = HashSet::new(); for topic_hash in topics { if let Some(peers) = self.mesh.get_mut(&topic_hash) { // if we are subscribed, add peer to the mesh @@ -455,14 +451,14 @@ impl Gossipsub { peers.push(peer_id.clone()); //TODO: tagPeer } else { - to_prune_topics.insert(topic_hash.clone(), ()); + to_prune_topics.insert(topic_hash.clone()); } } if !to_prune_topics.is_empty() { // build the prune messages to send let prune_messages = to_prune_topics - .keys() + .iter() .map(|t| GossipsubControlAction::Prune { topic_hash: t.clone(), }) @@ -856,7 +852,7 @@ impl Gossipsub { /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { debug!("Forwarding message: {:?}", message.id()); - let mut recipient_peers = HashMap::new(); + let mut recipient_peers = HashSet::new(); // add floodsub and mesh peers for topic in &message.topics { @@ -864,7 +860,7 @@ impl Gossipsub { if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { for peer_id in floodsub_peers { if *peer_id != source { - recipient_peers.insert(peer_id.clone(), ()); + recipient_peers.insert(peer_id.clone()); } } } @@ -873,7 +869,7 @@ impl Gossipsub { if let Some(mesh_peers) = self.mesh.get(&topic) { for peer_id in mesh_peers { if *peer_id != source { - recipient_peers.insert(peer_id.clone(), ()); + recipient_peers.insert(peer_id.clone()); } } } @@ -881,7 +877,7 @@ impl Gossipsub { // forward the message to peers if !recipient_peers.is_empty() { - for peer in recipient_peers.keys() { + for peer in recipient_peers.iter() { debug!("Sending message: {:?} to peer {:?}", message.id(), peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), @@ -937,7 +933,7 @@ where Default::default() } - fn addresses_of_peer(&self, _: &PeerId) -> Vec { + fn addresses_of_peer(&mut self, _: &PeerId) -> Vec { Vec::new() } diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index e77ab5f25cc..0c63566903f 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -1,8 +1,8 @@ extern crate fnv; -use fnv::FnvHashMap; use libp2p_floodsub::TopicHash; use protocol::GossipsubMessage; +use std::collections::HashMap; /// CacheEntry stored in the history #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -14,7 +14,7 @@ pub struct CacheEntry { /// MessageCache struct holding history of messages #[derive(Debug, Clone, PartialEq)] pub struct MessageCache { - msgs: FnvHashMap, + msgs: HashMap, history: Vec>, gossip: usize, } @@ -24,7 +24,7 @@ impl MessageCache { pub fn new(gossip: usize, history_capacity: usize) -> MessageCache { MessageCache { gossip, - msgs: FnvHashMap::default(), + msgs: HashMap::default(), history: vec![Vec::new(); history_capacity], } } From 0a51e3fb0aeb909af3f6973140db8d1946d25842 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 2 Feb 2019 17:16:06 +1100 Subject: [PATCH 41/91] Add NodeList struct to clarify topic_peers. --- protocols/gossipsub/src/layer.rs | 78 +++++++++++++++++--------- protocols/gossipsub/src/layer/tests.rs | 18 ++++-- 2 files changed, 66 insertions(+), 30 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index c8b8bd5bc09..ee068af3a98 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -61,8 +61,8 @@ pub struct Gossipsub { /// Peer id of the local node. Used for the source of the messages that we publish. local_peer_id: PeerId, - /// A map of all connected peers - A map of topic hash to a tuple containing a list of gossipsub peers and floodsub peers respectively. - topic_peers: HashMap, Vec)>, + /// A map of all connected peers - A map of topic hash to PeerList which contains a list of gossipsub and floodsub peers. + topic_peers: HashMap, /// A map of all connected peers to a tuple containing their subscribed topics and NodeType /// respectively. @@ -125,8 +125,8 @@ impl Gossipsub { } // send subscription request to all floodsub and gossipsub peers in the topic - if let Some((gossip_peers, flood_peers)) = self.topic_peers.get(&topic.hash()) { - for peer in flood_peers.iter().chain(gossip_peers) { + if let Some(peer_list) = self.topic_peers.get(&topic.hash()) { + for peer in peer_list.floodsub.iter().chain(peer_list.gossipsub.iter()) { debug!("Sending SUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), @@ -165,8 +165,8 @@ impl Gossipsub { } // announce to all floodsub and gossipsub peers, in the topic - if let Some((gossip_peers, flood_peers)) = self.topic_peers.get(topic_hash) { - for peer in flood_peers.iter().chain(gossip_peers) { + if let Some(peer_list) = self.topic_peers.get(topic_hash) { + for peer in peer_list.floodsub.iter().chain(peer_list.gossipsub.iter()) { debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), @@ -552,24 +552,24 @@ impl Gossipsub { for subscription in subscriptions { // get the peers from the mapping, or insert empty lists if topic doesn't exist - let (gossip_peers, flood_peers) = self + let peer_list = self .topic_peers .entry(subscription.topic_hash.clone()) - .or_insert((vec![], vec![])); + .or_insert(PeerList::new()); match subscription.action { GossipsubSubscriptionAction::Subscribe => { match node_type { NodeType::Floodsub => { - if !flood_peers.contains(&propagation_source) { + if !peer_list.floodsub.contains(&propagation_source) { debug!("SUBSCRIPTION: topic_peer: Adding floodsub peer: {:?} to topic: {:?}", propagation_source, subscription.topic_hash); - flood_peers.push(propagation_source.clone()); + peer_list.floodsub.push(propagation_source.clone()); } } NodeType::Gossipsub => { - if !gossip_peers.contains(&propagation_source) { + if !peer_list.gossipsub.contains(&propagation_source) { debug!("SUBSCRIPTION: topic_peer: Adding gossip peer: {:?} to topic: {:?}", propagation_source, subscription.topic_hash); - gossip_peers.push(propagation_source.clone()); + peer_list.gossipsub.push(propagation_source.clone()); } } } @@ -592,25 +592,29 @@ impl Gossipsub { GossipsubSubscriptionAction::Unsubscribe => { match node_type { NodeType::Floodsub => { - if let Some(pos) = - flood_peers.iter().position(|p| p == propagation_source) + if let Some(pos) = peer_list + .floodsub + .iter() + .position(|p| p == propagation_source) { info!( "SUBSCRIPTION: Removing floodsub peer: {:?} from topic: {:?}", propagation_source, subscription.topic_hash ); - flood_peers.remove(pos); + peer_list.floodsub.remove(pos); } } NodeType::Gossipsub => { - if let Some(pos) = - gossip_peers.iter().position(|p| p == propagation_source) + if let Some(pos) = peer_list + .gossipsub + .iter() + .position(|p| p == propagation_source) { info!( "SUBSCRIPTION: Removing gossip peer: {:?} from topic: {:?}", propagation_source, subscription.topic_hash ); - gossip_peers.remove(pos); + peer_list.gossipsub.remove(pos); } } } @@ -857,8 +861,8 @@ impl Gossipsub { // add floodsub and mesh peers for topic in &message.topics { // floodsub - if let Some((_, floodsub_peers)) = self.topic_peers.get(&topic) { - for peer_id in floodsub_peers { + if let Some(peer_list) = self.topic_peers.get(&topic) { + for peer_id in &peer_list.floodsub { if *peer_id != source { recipient_peers.insert(peer_id.clone()); } @@ -902,7 +906,12 @@ impl Gossipsub { ) -> Vec { let mut gossip_peers = match self.topic_peers.get(topic_hash) { // if they exist, filter the peers by `f` - Some((gossip_peers, _)) => gossip_peers.iter().cloned().filter(|p| f(p)).collect(), + Some(peer_list) => peer_list + .gossipsub + .iter() + .cloned() + .filter(|p| f(p)) + .collect(), None => Vec::new(), }; @@ -990,11 +999,11 @@ where } // remove from topic_peers - if let Some((gossip_peers, flood_peers)) = self.topic_peers.get_mut(&topic) { + if let Some(peer_list) = self.topic_peers.get_mut(&topic) { match node_type { NodeType::Gossipsub => { - if let Some(pos) = gossip_peers.iter().position(|p| p == id) { - gossip_peers.remove(pos); + if let Some(pos) = peer_list.gossipsub.iter().position(|p| p == id) { + peer_list.gossipsub.remove(pos); //TODO: untagPeer } // debugging purposes @@ -1006,8 +1015,8 @@ where } } NodeType::Floodsub => { - if let Some(pos) = flood_peers.iter().position(|p| p == id) { - flood_peers.remove(pos); + if let Some(pos) = peer_list.floodsub.iter().position(|p| p == id) { + peer_list.floodsub.remove(pos); //TODO: untagPeer } // debugging purposes @@ -1125,6 +1134,23 @@ impl From<()> for InnerMessage { } } +/// Struct that contains lists of gossipsub and floodsub peers. +struct PeerList { + /// List of gossipsub peers. + gossipsub: Vec, + /// List of floodsub peers. + floodsub: Vec, +} + +impl PeerList { + pub fn new() -> Self { + PeerList { + gossipsub: vec![], + floodsub: vec![], + } + } +} + /// Event that can happen on the gossipsub behaviour. #[derive(Debug)] pub enum GossipsubEvent { diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index a3dfbe63436..3a47d1c03ca 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -410,7 +410,7 @@ mod tests { ); for topic_hash in topic_hashes[..3].iter() { - let topic_peers = gs.topic_peers.get(topic_hash).unwrap().0.clone(); // only gossipsub at the moment + let topic_peers = gs.topic_peers.get(topic_hash).unwrap().gossipsub.clone(); // only gossipsub at the moment assert!( topic_peers == peers[..2].to_vec(), "Two peers should be added to the first three topics" @@ -433,7 +433,12 @@ mod tests { "Peer should be subscribed to two topics" ); - let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().0.clone(); // only gossipsub at the moment + let topic_peers = gs + .topic_peers + .get(&topic_hashes[0]) + .unwrap() + .gossipsub + .clone(); // only gossipsub at the moment assert!( topic_peers == peers[1..2].to_vec(), "Only the second peers should be in the first topic" @@ -455,8 +460,13 @@ mod tests { peers.push(PeerId::random()) } - gs.topic_peers - .insert(topic_hash.clone(), (peers.clone(), vec![])); + gs.topic_peers.insert( + topic_hash.clone(), + PeerList { + gossipsub: peers.clone(), + floodsub: vec![], + }, + ); let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| true }); assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); From 2e855f47df23ac277af34d99544569fb3c4fb57c Mon Sep 17 00:00:00 2001 From: Toralf Wittner Date: Sat, 2 Feb 2019 17:20:01 +1100 Subject: [PATCH 42/91] Cleaner handling of messagelist Co-Authored-By: AgeManning --- protocols/gossipsub/src/layer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index ee068af3a98..39a41706283 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -422,7 +422,7 @@ impl Gossipsub { if !cached_messages.is_empty() { info!("IWANT: Sending cached messages to peer: {:?}", peer_id); // Send the messages to the peer - let message_list = cached_messages.values().cloned().collect(); + let message_list = cached_messages.into_iter().map(|entry| entry.1).collect(); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), event: GossipsubRpc { From 8d32626b6888fb0179ac7071bd23809ead1d23c3 Mon Sep 17 00:00:00 2001 From: Toralf Wittner Date: Sat, 2 Feb 2019 17:20:59 +1100 Subject: [PATCH 43/91] Cleaner handling of added peers. Co-Authored-By: AgeManning --- protocols/gossipsub/src/layer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 39a41706283..6d56e677a92 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -293,7 +293,7 @@ impl Gossipsub { "JOIN: Adding {:?} peers from the fanout for topic: {:?}", add_peers, topic_hash ); - added_peers.append(&mut peers.clone()[..add_peers].to_vec()); + added_peers.extend_from_slice(&peers[.. add_peers]); self.mesh .insert(topic_hash.clone(), peers[..add_peers].to_vec()); // remove the last published time From e78f4170e2cfd7f7f90006bc8ac547550ee5a081 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Sun, 3 Feb 2019 22:37:46 -0600 Subject: [PATCH 44/91] handle_prune peer removed test --- protocols/gossipsub/src/layer/tests.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 3a47d1c03ca..4bf6938ebd6 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -647,4 +647,18 @@ mod tests { "Expected event count to stay the same" ) } + + #[test] + // tests that a peer is removed from our mesh + fn test_handle_prune_in_mesh(){ + let (mut gs, peers, topic_hashes) = + build_and_inject_nodes(20, vec![String::from("topic1")], true); + + // insert peer into our mesh for 'topic1' + gs.mesh.insert(topic_hashes[0].clone(), peers.clone()); + assert!(gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7])); + + gs.handle_prune(&peers[7], topic_hashes.clone()); + assert!(!gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7])); + } } From 30f98dc5e535c306dd38e2f300ff26cb64be8845 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Tue, 5 Feb 2019 20:38:38 -0600 Subject: [PATCH 45/91] basic grafting tests --- protocols/gossipsub/src/layer/tests.rs | 55 ++++++++++++++++++++++++-- 1 file changed, 52 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 4bf6938ebd6..81615564c95 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -648,17 +648,66 @@ mod tests { ) } + #[test] + // tests that a peer is added to our mesh when we are both subscribed + // to the same topic + fn test_handle_graft_is_subscribed() { + let (mut gs, peers, topic_hashes) = + build_and_inject_nodes(20, vec![String::from("topic1")], true); + + assert!( + !gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), + "Expected peer to not be in mesh" + ); + + gs.handle_graft(&peers[7], topic_hashes.clone()); + + assert!( + gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), + "Expected peer to have been added to mesh" + ); + } + + #[test] + // tests that a peer is not added to our mesh when they are subscribed to + // a topic that we are not + fn test_handle_graft_is_not_subscribed() { + let (mut gs, peers, topic_hashes) = + build_and_inject_nodes(20, vec![String::from("topic1")], true); + + assert!( + !gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), + "Expected peer to not be in mesh" + ); + + gs.handle_graft( + &peers[7], + vec![TopicHash::from_raw(String::from("unsubscribed topic"))] + ); + + assert!( + !gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), + "Expected peer to have been added to mesh" + ); + } + #[test] // tests that a peer is removed from our mesh - fn test_handle_prune_in_mesh(){ + fn test_handle_prune_peer_in_mesh(){ let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); // insert peer into our mesh for 'topic1' gs.mesh.insert(topic_hashes[0].clone(), peers.clone()); - assert!(gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7])); + assert!( + gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), + "Expected peer to be in mesh" + ); gs.handle_prune(&peers[7], topic_hashes.clone()); - assert!(!gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7])); + assert!( + !gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), + "Expected peer to be removed from mesh" + ); } } From 27414e30969cdb9fec684eca2f1248d1bdb9d932 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Wed, 6 Feb 2019 18:32:37 -0600 Subject: [PATCH 46/91] multiple topic grafting test --- protocols/gossipsub/src/layer/tests.rs | 44 ++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 81615564c95..d4524455a60 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -691,6 +691,50 @@ mod tests { ); } + #[test] + // tests multiple topics in a single graft message + fn test_handle_graft_multiple_topics() { + let topics: Vec = vec!["topic1", "topic2", "topic3", "topic4"] + .iter().map(|&t| String::from(t)).collect(); + + let (mut gs, peers, topic_hashes) = + build_and_inject_nodes(20, topics.clone(), true); + + // sanity check: mesh does not already contain peer + for topic_hash in topic_hashes.clone() { + assert!( + !gs.mesh.get(&topic_hash).unwrap().contains(&peers[7]), + "Expected peer to not be in mesh for any topic" + ); + } + + + let mut their_topics = topic_hashes.clone(); + // their_topics = [topic1, topic2, topic3] + // our_topics = [topic1, topic2, topic4] + their_topics.pop(); + gs.leave(&their_topics[2]); + + gs.handle_graft(&peers[7], their_topics.clone()); + + for i in 0..2 { + assert!( + gs.mesh.get(&topic_hashes[i]).unwrap().contains(&peers[7]), + "Expected peer to be in the mesh for the first 2 topics" + ); + } + + assert!( + gs.mesh.get(&topic_hashes[2]).is_none(), + "Expected the second topic to not be in the mesh" + ); + + assert!( + !gs.mesh.get(&topic_hashes[3]).unwrap().contains(&peers[7]), + "Expected peer to not be in the mesh for the fourth topic" + ); + } + #[test] // tests that a peer is removed from our mesh fn test_handle_prune_peer_in_mesh(){ From 4ed865fd7e24b82289d995515e903cd3f593f855 Mon Sep 17 00:00:00 2001 From: Toralf Wittner Date: Fri, 22 Feb 2019 10:53:31 +1100 Subject: [PATCH 47/91] Convert &vec to slice. Co-Authored-By: AgeManning --- protocols/gossipsub/src/layer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 6d56e677a92..b5c368cd6c7 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -535,7 +535,7 @@ impl Gossipsub { /// Handles received subscriptions. fn handle_received_subscriptions( &mut self, - subscriptions: &Vec, + subscriptions: &[GossipsubSubscription], propagation_source: &PeerId, ) { debug!( From 67ec8cc8a37802657d34cc5fd7deeabeb5233587 Mon Sep 17 00:00:00 2001 From: Toralf Wittner Date: Fri, 22 Feb 2019 10:54:09 +1100 Subject: [PATCH 48/91] Convert to lazy insert. Co-Authored-By: AgeManning --- protocols/gossipsub/src/layer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index b5c368cd6c7..01900037474 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -555,7 +555,7 @@ impl Gossipsub { let peer_list = self .topic_peers .entry(subscription.topic_hash.clone()) - .or_insert(PeerList::new()); + .or_insert_with(PeerList::new); match subscription.action { GossipsubSubscriptionAction::Subscribe => { From 156604358bb6ea25db0aa5a3d4513adcd2813f9f Mon Sep 17 00:00:00 2001 From: Toralf Wittner Date: Fri, 22 Feb 2019 10:55:19 +1100 Subject: [PATCH 49/91] Cleaner topic handling. Co-Authored-By: AgeManning --- protocols/gossipsub/src/layer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 01900037474..6e35f90a6dc 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -211,7 +211,7 @@ impl Gossipsub { // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. sequence_number: rand::random::<[u8; 8]>().to_vec(), - topics: topic.into_iter().map(|t| t.into().clone()).collect(), + topics: topic.into_iter().map(Into::into).collect(), }; debug!("Publishing message: {:?}", message.id()); From 866e5070f3e59f043af52a89198ff6fe72a68651 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Thu, 21 Feb 2019 22:34:54 -0600 Subject: [PATCH 50/91] control pool piggybacking using HashMap.drain() in control_pool_flush going to squash this --- protocols/gossipsub/src/layer.rs | 98 +++++++++++++++++++------------- 1 file changed, 57 insertions(+), 41 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 6e35f90a6dc..23ad38ff59d 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -58,6 +58,9 @@ pub struct Gossipsub { /// Events that need to be yielded to the outside when polling. events: VecDeque>, + // pool non-urgent control messages between heartbeats + control_pool: HashMap>, + /// Peer id of the local node. Used for the source of the messages that we publish. local_peer_id: PeerId, @@ -98,6 +101,7 @@ impl Gossipsub { Gossipsub { config: gs_config.clone(), events: VecDeque::new(), + control_pool: HashMap::new(), local_peer_id, topic_peers: HashMap::new(), peer_topics: HashMap::new(), @@ -323,16 +327,12 @@ impl Gossipsub { for peer_id in added_peers { // Send a GRAFT control message info!("JOIN: Sending Graft message to peer: {:?}", peer_id); - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer_id.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: vec![GossipsubControlAction::Graft { - topic_hash: topic_hash.clone(), - }], - }, - }); + self.control_pool_add( + peer_id.clone(), + GossipsubControlAction::Graft { + topic_hash: topic_hash.clone(), + } + ); //TODO: tagPeer } debug!("Completed JOIN for topic: {:?}", topic_hash); @@ -347,16 +347,12 @@ impl Gossipsub { for peer in peers { // Send a PRUNE control message info!("LEAVE: Sending PRUNE to peer: {:?}", peer); - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: vec![GossipsubControlAction::Prune { - topic_hash: topic_hash.clone(), - }], - }, - }); + self.control_pool_add( + peer.clone(), + GossipsubControlAction::Prune { + topic_hash: topic_hash.clone(), + } + ); //TODO: untag Peer } } @@ -391,16 +387,12 @@ impl Gossipsub { if !iwant_ids.is_empty() { // Send the list of IWANT control messages info!("IHAVE: Sending IWANT message"); - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer_id.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: vec![GossipsubControlAction::IWant { - message_ids: iwant_ids.iter().cloned().collect(), - }], - }, - }); + self.control_pool_add( + peer_id.clone(), + GossipsubControlAction::IWant { + message_ids: iwant_ids.iter().cloned().collect(), + } + ); } debug!("Completed IHAVE handling for peer: {:?}", peer_id); } @@ -764,6 +756,9 @@ impl Gossipsub { self.send_graft_prune(to_graft, to_prune); } + // piggyback pooled control messages + flush_control_pool(); + // shift the memcache self.mcache.shift(); debug!("Completed Heartbeat"); @@ -783,17 +778,13 @@ impl Gossipsub { }); for peer in to_msg_peers { // send an IHAVE message - self.events.push_back(NetworkBehaviourAction::SendEvent { - peer_id: peer, - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: vec![GossipsubControlAction::IHave { - topic_hash: topic_hash.clone(), - message_ids: message_ids.clone(), - }], - }, - }); + self.control_pool_add( + peer.clone(), + GossipsubControlAction::IHave { + topic_hash: topic_hash.clone(), + message_ids: message_ids.clone(), + } + ); } debug!("Completed gossip"); } @@ -929,6 +920,31 @@ impl Gossipsub { gossip_peers[..n].to_vec() } + + // adds a control action to control_pool + fn control_pool_add(&mut self, peer: PeerId, control: GossipsubControlAction) { + if !self.control_pool.contains_key(&peer) { + self.control_pool.insert(peer.clone(), Vec::new()); + } + + if let Some(controls) = self.control_pool.get_mut(&peer) { + controls.push(control.clone()); + } + } + + // takes each control action mapping and turns it into a message + fn flush_control_pool(&mut self) { + for (peer, controls) in self.control_pool.drain() { + self.events.push_back(NetworkBehaviourAction::SendEvent { + peer_id: peer, + event: GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: controls + } + }); + } + } } impl NetworkBehaviour for Gossipsub From faa7b038f66a0137db8536cf66d1b368d077d840 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 26 Feb 2019 21:08:17 +1100 Subject: [PATCH 51/91] Add Debug derives to gossipsub and correct tests. --- protocols/gossipsub/src/gossipsub_config.rs | 2 +- protocols/gossipsub/src/layer.rs | 5 ++++- protocols/gossipsub/src/layer/tests.rs | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/src/gossipsub_config.rs b/protocols/gossipsub/src/gossipsub_config.rs index 1c60d30a94e..a8e387a9146 100644 --- a/protocols/gossipsub/src/gossipsub_config.rs +++ b/protocols/gossipsub/src/gossipsub_config.rs @@ -1,7 +1,7 @@ use std::time::Duration; /// Configuration parameters that define the performance of the gossipsub network. -#[derive(Clone)] +#[derive(Debug, Clone)] pub struct GossipsubConfig { /// Overlay network parameters. /// Number of heartbeats to keep in the `memcache`. diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 6e35f90a6dc..fa481ec787e 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -293,7 +293,7 @@ impl Gossipsub { "JOIN: Adding {:?} peers from the fanout for topic: {:?}", add_peers, topic_hash ); - added_peers.extend_from_slice(&peers[.. add_peers]); + added_peers.extend_from_slice(&peers[..add_peers]); self.mesh .insert(topic_hash.clone(), peers[..add_peers].to_vec()); // remove the last published time @@ -1113,6 +1113,7 @@ where } /// Transmission between the `OneShotHandler` and the `GossipsubRpc`. +#[derive(Debug)] pub enum InnerMessage { /// We received an RPC from a remote. Rx(GossipsubRpc), @@ -1135,6 +1136,7 @@ impl From<()> for InnerMessage { } /// Struct that contains lists of gossipsub and floodsub peers. +#[derive(Debug, Clone)] struct PeerList { /// List of gossipsub peers. gossipsub: Vec, @@ -1175,6 +1177,7 @@ pub enum GossipsubEvent { } /// The type of node in the pubsub system. +#[derive(Debug)] pub enum NodeType { /// A gossipsub node. Gossipsub, diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 3a47d1c03ca..cf0aebb887f 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -55,7 +55,7 @@ mod tests { action: GossipsubSubscriptionAction::Subscribe, topic_hash: t, }) - .collect(), + .collect::>(), &peer, ); }; From d5459e72fbe8c88ddbce1fc2f8a6ed5e8e1e4466 Mon Sep 17 00:00:00 2001 From: Grant Wuerker Date: Fri, 1 Mar 2019 21:02:11 -0600 Subject: [PATCH 52/91] changes from PR squash this all tests passing, but still some that need to be reconsidered test reform --- protocols/gossipsub/src/layer.rs | 14 +++--- protocols/gossipsub/src/layer/tests.rs | 59 ++++++++++++-------------- 2 files changed, 33 insertions(+), 40 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index a6e587073d0..0719f91ea97 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -757,7 +757,7 @@ impl Gossipsub { } // piggyback pooled control messages - flush_control_pool(); + self.flush_control_pool(); // shift the memcache self.mcache.shift(); @@ -923,13 +923,11 @@ impl Gossipsub { // adds a control action to control_pool fn control_pool_add(&mut self, peer: PeerId, control: GossipsubControlAction) { - if !self.control_pool.contains_key(&peer) { - self.control_pool.insert(peer.clone(), Vec::new()); - } - - if let Some(controls) = self.control_pool.get_mut(&peer) { - controls.push(control.clone()); - } + (*self + .control_pool + .entry(peer.clone()) + .or_insert_with(|| Vec::new())) + .push(control.clone()); } // takes each control action mapping and turns it into a message diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 501a7450cfa..b6e1940d2db 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -221,25 +221,22 @@ mod tests { // there should be mesh_n GRAFT messages. let graft_messages = gs - .events + .control_pool .iter() - .fold(vec![], |mut collected_grafts, e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - for c in &event.control_msgs { - match c { - GossipsubControlAction::Graft { topic_hash: _ } => { - collected_grafts.push(c.clone()) - } - _ => {} - }; + .fold(vec![], |mut collected_grafts, (_, controls)| { + for c in controls.iter() { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} } - collected_grafts } - _ => collected_grafts, + collected_grafts }); - assert!( - graft_messages.len() == 6, + assert_eq!( + graft_messages.len(), 6, "There should be 6 grafts messages sent to peers" ); @@ -270,26 +267,23 @@ mod tests { // there should now be 12 graft messages to be sent let graft_messages = gs - .events + .control_pool .iter() - .fold(vec![], |mut collected_grafts, e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - for c in &event.control_msgs { - match c { - GossipsubControlAction::Graft { topic_hash: _ } => { - collected_grafts.push(c.clone()) - } - _ => {} - }; + .fold(vec![], |mut collected_grafts, (_, controls)| { + for c in controls.iter() { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} } - collected_grafts } - _ => collected_grafts, + collected_grafts }); assert!( graft_messages.len() == 12, - "There should be 6 grafts messages sent to peers" + "There should be 12 grafts messages sent to peers" ); } @@ -589,18 +583,19 @@ mod tests { &peers[7], vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])], ); + // check that we sent an IWANT request for `unknown id` - let iwant_exists = gs.events.iter().any(|e| match e { - NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - event.control_msgs.iter().any(|e| match e { + let iwant_exists = match gs.control_pool.get(&peers[7]) { + Some(controls) => { + controls.iter().any(|c| match c { GossipsubControlAction::IWant { message_ids } => { message_ids.iter().any(|m| *m == String::from("unknown id")) } _ => false, }) } - _ => false, - }); + _ => false + }; assert!(iwant_exists "Expected to send an IWANT control message for unkown message id"); From ed523b8671122eac8df4e30e0fa9e6bb9428dca0 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 11 Mar 2019 17:38:06 +1100 Subject: [PATCH 53/91] Implements Arc for GossipsubRpc events --- protocols/gossipsub/src/layer.rs | 134 ++++++++++++++++--------- protocols/gossipsub/src/layer/tests.rs | 81 ++++++++------- 2 files changed, 126 insertions(+), 89 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 0719f91ea97..471c97959a9 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -42,6 +42,7 @@ use rand::{seq::SliceRandom, thread_rng}; use smallvec::SmallVec; use std::collections::hash_map::{DefaultHasher, HashMap}; use std::collections::HashSet; +use std::sync::Arc; use std::time::Instant; use std::{collections::VecDeque, iter, marker::PhantomData}; use tokio_io::{AsyncRead, AsyncWrite}; @@ -56,7 +57,7 @@ pub struct Gossipsub { config: GossipsubConfig, /// Events that need to be yielded to the outside when polling. - events: VecDeque>, + events: VecDeque, GossipsubEvent>>, // pool non-urgent control messages between heartbeats control_pool: HashMap>, @@ -130,18 +131,20 @@ impl Gossipsub { // send subscription request to all floodsub and gossipsub peers in the topic if let Some(peer_list) = self.topic_peers.get(&topic.hash()) { + let event = Arc::new(GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic_hash: topic.hash().clone(), + action: GossipsubSubscriptionAction::Subscribe, + }], + control_msgs: Vec::new(), + }); + for peer in peer_list.floodsub.iter().chain(peer_list.gossipsub.iter()) { debug!("Sending SUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), - event: GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic_hash: topic.hash().clone(), - action: GossipsubSubscriptionAction::Subscribe, - }], - control_msgs: Vec::new(), - }, + event: event.clone(), }); } } @@ -170,18 +173,19 @@ impl Gossipsub { // announce to all floodsub and gossipsub peers, in the topic if let Some(peer_list) = self.topic_peers.get(topic_hash) { + let event = Arc::new(GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic_hash: topic_hash.clone(), + action: GossipsubSubscriptionAction::Unsubscribe, + }], + control_msgs: Vec::new(), + }); for peer in peer_list.floodsub.iter().chain(peer_list.gossipsub.iter()) { debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), - event: GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic_hash: topic_hash.clone(), - action: GossipsubSubscriptionAction::Unsubscribe, - }], - control_msgs: Vec::new(), - }, + event: event.clone(), }); } } @@ -256,16 +260,17 @@ impl Gossipsub { self.mcache.put(message.clone()); self.received.add(&message.id()); + let event = Arc::new(GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![message.clone()], + control_msgs: Vec::new(), + }); // Send to peers we know are subscribed to the topic. for peer_id in recipient_peers.keys() { debug!("Sending message to peer: {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![message.clone()], - control_msgs: Vec::new(), - }, + event: event.clone(), }); } info!("Published message: {:?}", message.id()); @@ -307,11 +312,11 @@ impl Gossipsub { // check if we need to get more peers, which we randomly select if added_peers.len() < self.config.mesh_n { // get the peers - let mut new_peers = + let new_peers = self.get_random_peers(topic_hash, self.config.mesh_n - added_peers.len(), { |_| true }); - added_peers.append(&mut new_peers.clone()); + added_peers.extend_from_slice(&new_peers); // add them to the mesh debug!( "JOIN: Inserting {:?} random peers into the mesh", @@ -321,7 +326,7 @@ impl Gossipsub { .mesh .entry(topic_hash.clone()) .or_insert_with(|| vec![]); - mesh_peers.append(&mut new_peers); + mesh_peers.extend_from_slice(&new_peers); } for peer_id in added_peers { @@ -331,7 +336,7 @@ impl Gossipsub { peer_id.clone(), GossipsubControlAction::Graft { topic_hash: topic_hash.clone(), - } + }, ); //TODO: tagPeer } @@ -351,7 +356,7 @@ impl Gossipsub { peer.clone(), GossipsubControlAction::Prune { topic_hash: topic_hash.clone(), - } + }, ); //TODO: untag Peer } @@ -391,7 +396,7 @@ impl Gossipsub { peer_id.clone(), GossipsubControlAction::IWant { message_ids: iwant_ids.iter().cloned().collect(), - } + }, ); } debug!("Completed IHAVE handling for peer: {:?}", peer_id); @@ -417,11 +422,11 @@ impl Gossipsub { let message_list = cached_messages.into_iter().map(|entry| entry.1).collect(); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), - event: GossipsubRpc { + event: Arc::new(GossipsubRpc { subscriptions: Vec::new(), messages: message_list, control_msgs: Vec::new(), - }, + }), }); } debug!("Completed IWANT handling for peer: {:?}", peer_id); @@ -462,11 +467,11 @@ impl Gossipsub { ); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), - event: GossipsubRpc { + event: Arc::new(GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: prune_messages, - }, + }), }); } debug!("Completed GRAFT handling for peer: {:?}", peer_id); @@ -783,7 +788,7 @@ impl Gossipsub { GossipsubControlAction::IHave { topic_hash: topic_hash.clone(), message_ids: message_ids.clone(), - } + }, ); } debug!("Completed gossip"); @@ -817,11 +822,11 @@ impl Gossipsub { // send the control messages self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), - event: GossipsubRpc { + event: Arc::new(GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: grafts, - }, + }), }); } @@ -835,11 +840,11 @@ impl Gossipsub { .collect(); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), - event: GossipsubRpc { + event: Arc::new(GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), control_msgs: remaining_prunes, - }, + }), }); } } @@ -872,15 +877,17 @@ impl Gossipsub { // forward the message to peers if !recipient_peers.is_empty() { + let event = Arc::new(GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![message.clone()], + control_msgs: Vec::new(), + }); + for peer in recipient_peers.iter() { debug!("Sending message: {:?} to peer {:?}", message.id(), peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), - event: GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![message.clone()], - control_msgs: Vec::new(), - }, + event: event.clone(), }); } } @@ -935,11 +942,11 @@ impl Gossipsub { for (peer, controls) in self.control_pool.drain() { self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer, - event: GossipsubRpc { + event: Arc::new(GossipsubRpc { subscriptions: Vec::new(), messages: Vec::new(), - control_msgs: controls - } + control_msgs: controls, + }), }); } } @@ -974,11 +981,11 @@ where // send our subscriptions to the peer self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: id.clone(), - event: GossipsubRpc { + event: Arc::new(GossipsubRpc { messages: Vec::new(), subscriptions, control_msgs: Vec::new(), - }, + }), }); } @@ -1115,7 +1122,38 @@ where >, > { if let Some(event) = self.events.pop_front() { - return Async::Ready(event); + // clone send event reference if others references are present + match event { + NetworkBehaviourAction::SendEvent { + peer_id, + event: send_event, + } => match Arc::try_unwrap(send_event) { + Ok(event) => { + return Async::Ready(NetworkBehaviourAction::SendEvent { + peer_id, + event: event, + }); + } + Err(event) => { + return Async::Ready(NetworkBehaviourAction::SendEvent { + peer_id, + event: (*event).clone(), + }); + } + }, + NetworkBehaviourAction::GenerateEvent(e) => { + return Async::Ready(NetworkBehaviourAction::GenerateEvent(e)); + } + NetworkBehaviourAction::DialAddress { address } => { + return Async::Ready(NetworkBehaviourAction::DialAddress { address }); + } + NetworkBehaviourAction::DialPeer { peer_id } => { + return Async::Ready(NetworkBehaviourAction::DialPeer { peer_id }); + } + NetworkBehaviourAction::ReportObservedAddr { address } => { + return Async::Ready(NetworkBehaviourAction::ReportObservedAddr { address }); + } + } } while let Ok(Async::Ready(Some(_))) = self.heartbeat.poll() { diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index b6e1940d2db..0c6555751ea 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -220,23 +220,24 @@ mod tests { ); // there should be mesh_n GRAFT messages. - let graft_messages = gs - .control_pool - .iter() - .fold(vec![], |mut collected_grafts, (_, controls)| { - for c in controls.iter() { - match c { - GossipsubControlAction::Graft { topic_hash: _ } => { - collected_grafts.push(c.clone()) + let graft_messages = + gs.control_pool + .iter() + .fold(vec![], |mut collected_grafts, (_, controls)| { + for c in controls.iter() { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} } - _ => {} } - } - collected_grafts - }); + collected_grafts + }); assert_eq!( - graft_messages.len(), 6, + graft_messages.len(), + 6, "There should be 6 grafts messages sent to peers" ); @@ -266,20 +267,20 @@ mod tests { } // there should now be 12 graft messages to be sent - let graft_messages = gs - .control_pool - .iter() - .fold(vec![], |mut collected_grafts, (_, controls)| { - for c in controls.iter() { - match c { - GossipsubControlAction::Graft { topic_hash: _ } => { - collected_grafts.push(c.clone()) + let graft_messages = + gs.control_pool + .iter() + .fold(vec![], |mut collected_grafts, (_, controls)| { + for c in controls.iter() { + match c { + GossipsubControlAction::Graft { topic_hash: _ } => { + collected_grafts.push(c.clone()) + } + _ => {} } - _ => {} } - } - collected_grafts - }); + collected_grafts + }); assert!( graft_messages.len() == 12, @@ -298,7 +299,7 @@ mod tests { // check that our subscriptions are sent to each of the peers // collect all the SendEvents - let send_events: Vec<&NetworkBehaviourAction> = gs + let send_events: Vec<&NetworkBehaviourAction, GossipsubEvent>> = gs .events .iter() .filter(|e| match e { @@ -586,15 +587,13 @@ mod tests { // check that we sent an IWANT request for `unknown id` let iwant_exists = match gs.control_pool.get(&peers[7]) { - Some(controls) => { - controls.iter().any(|c| match c { - GossipsubControlAction::IWant { message_ids } => { - message_ids.iter().any(|m| *m == String::from("unknown id")) - } - _ => false, - }) - } - _ => false + Some(controls) => controls.iter().any(|c| match c { + GossipsubControlAction::IWant { message_ids } => { + message_ids.iter().any(|m| *m == String::from("unknown id")) + } + _ => false, + }), + _ => false, }; assert!(iwant_exists @@ -677,7 +676,7 @@ mod tests { gs.handle_graft( &peers[7], - vec![TopicHash::from_raw(String::from("unsubscribed topic"))] + vec![TopicHash::from_raw(String::from("unsubscribed topic"))], ); assert!( @@ -690,10 +689,11 @@ mod tests { // tests multiple topics in a single graft message fn test_handle_graft_multiple_topics() { let topics: Vec = vec!["topic1", "topic2", "topic3", "topic4"] - .iter().map(|&t| String::from(t)).collect(); + .iter() + .map(|&t| String::from(t)) + .collect(); - let (mut gs, peers, topic_hashes) = - build_and_inject_nodes(20, topics.clone(), true); + let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics.clone(), true); // sanity check: mesh does not already contain peer for topic_hash in topic_hashes.clone() { @@ -703,7 +703,6 @@ mod tests { ); } - let mut their_topics = topic_hashes.clone(); // their_topics = [topic1, topic2, topic3] // our_topics = [topic1, topic2, topic4] @@ -732,7 +731,7 @@ mod tests { #[test] // tests that a peer is removed from our mesh - fn test_handle_prune_peer_in_mesh(){ + fn test_handle_prune_peer_in_mesh() { let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); From 97e433eefd6ad090988a0752e71eec556bc75cef Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 11 Mar 2019 19:25:50 +1100 Subject: [PATCH 54/91] Remove support for floodsub nodes --- protocols/gossipsub/src/layer.rs | 167 ++++++------------------- protocols/gossipsub/src/layer/tests.rs | 34 ++--- 2 files changed, 46 insertions(+), 155 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 471c97959a9..261774b7283 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -65,13 +65,11 @@ pub struct Gossipsub { /// Peer id of the local node. Used for the source of the messages that we publish. local_peer_id: PeerId, - /// A map of all connected peers - A map of topic hash to PeerList which contains a list of gossipsub and floodsub peers. - topic_peers: HashMap, + /// A map of all connected peers - A map of topic hash to a list of gossipsub peer Ids. + topic_peers: HashMap>, - /// A map of all connected peers to a tuple containing their subscribed topics and NodeType - /// respectively. - // This is used to efficiently keep track of all currently connected nodes and their type - peer_topics: HashMap, NodeType)>, + /// A map of all connected peers to their subscribed topics. + peer_topics: HashMap>, /// Overlay network of connected peers - Maps topics to connected gossipsub peers. mesh: HashMap>, @@ -129,7 +127,7 @@ impl Gossipsub { return false; } - // send subscription request to all floodsub and gossipsub peers in the topic + // send subscription request to all peers in the topic if let Some(peer_list) = self.topic_peers.get(&topic.hash()) { let event = Arc::new(GossipsubRpc { messages: Vec::new(), @@ -140,7 +138,7 @@ impl Gossipsub { control_msgs: Vec::new(), }); - for peer in peer_list.floodsub.iter().chain(peer_list.gossipsub.iter()) { + for peer in peer_list { debug!("Sending SUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), @@ -171,7 +169,7 @@ impl Gossipsub { return false; } - // announce to all floodsub and gossipsub peers, in the topic + // announce to all peers in the topic if let Some(peer_list) = self.topic_peers.get(topic_hash) { let event = Arc::new(GossipsubRpc { messages: Vec::new(), @@ -181,7 +179,7 @@ impl Gossipsub { }], control_msgs: Vec::new(), }); - for peer in peer_list.floodsub.iter().chain(peer_list.gossipsub.iter()) { + for peer in peer_list { debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), @@ -539,8 +537,8 @@ impl Gossipsub { "Handling subscriptions from source: {:?}", propagation_source ); - let (subscribed_topics, node_type) = match self.peer_topics.get_mut(&propagation_source) { - Some((topics, node_type)) => (topics, node_type), + let subscribed_topics = match self.peer_topics.get_mut(&propagation_source) { + Some(topics) => topics, None => { error!("Subscription by unknown peer: {:?}", &propagation_source); return; @@ -552,24 +550,18 @@ impl Gossipsub { let peer_list = self .topic_peers .entry(subscription.topic_hash.clone()) - .or_insert_with(PeerList::new); + .or_insert_with(Vec::new); match subscription.action { GossipsubSubscriptionAction::Subscribe => { - match node_type { - NodeType::Floodsub => { - if !peer_list.floodsub.contains(&propagation_source) { - debug!("SUBSCRIPTION: topic_peer: Adding floodsub peer: {:?} to topic: {:?}", propagation_source, subscription.topic_hash); - peer_list.floodsub.push(propagation_source.clone()); - } - } - NodeType::Gossipsub => { - if !peer_list.gossipsub.contains(&propagation_source) { - debug!("SUBSCRIPTION: topic_peer: Adding gossip peer: {:?} to topic: {:?}", propagation_source, subscription.topic_hash); - peer_list.gossipsub.push(propagation_source.clone()); - } - } + if !peer_list.contains(&propagation_source) { + debug!( + "SUBSCRIPTION: topic_peer: Adding gossip peer: {:?} to topic: {:?}", + propagation_source, subscription.topic_hash + ); + peer_list.push(propagation_source.clone()); } + // add to the peer_topics mapping if !subscribed_topics.contains(&subscription.topic_hash) { info!( @@ -587,33 +579,12 @@ impl Gossipsub { )); } GossipsubSubscriptionAction::Unsubscribe => { - match node_type { - NodeType::Floodsub => { - if let Some(pos) = peer_list - .floodsub - .iter() - .position(|p| p == propagation_source) - { - info!( - "SUBSCRIPTION: Removing floodsub peer: {:?} from topic: {:?}", - propagation_source, subscription.topic_hash - ); - peer_list.floodsub.remove(pos); - } - } - NodeType::Gossipsub => { - if let Some(pos) = peer_list - .gossipsub - .iter() - .position(|p| p == propagation_source) - { - info!( - "SUBSCRIPTION: Removing gossip peer: {:?} from topic: {:?}", - propagation_source, subscription.topic_hash - ); - peer_list.gossipsub.remove(pos); - } - } + if let Some(pos) = peer_list.iter().position(|p| p == propagation_source) { + info!( + "SUBSCRIPTION: Removing gossip peer: {:?} from topic: {:?}", + propagation_source, subscription.topic_hash + ); + peer_list.remove(pos); } // remove topic from the peer_topics mapping if let Some(pos) = subscribed_topics @@ -723,7 +694,7 @@ impl Gossipsub { peers.retain(|peer| { // is the peer still subscribed to the topic? if let Some(topics) = self.peer_topics.get(peer) { - if !topics.0.contains(&topic_hash) { + if !topics.contains(&topic_hash) { debug!( "HEARTBEAT: Peer removed from fanout for topic: {:?}", topic_hash @@ -856,15 +827,6 @@ impl Gossipsub { // add floodsub and mesh peers for topic in &message.topics { - // floodsub - if let Some(peer_list) = self.topic_peers.get(&topic) { - for peer_id in &peer_list.floodsub { - if *peer_id != source { - recipient_peers.insert(peer_id.clone()); - } - } - } - // mesh if let Some(mesh_peers) = self.mesh.get(&topic) { for peer_id in mesh_peers { @@ -904,12 +866,7 @@ impl Gossipsub { ) -> Vec { let mut gossip_peers = match self.topic_peers.get(topic_hash) { // if they exist, filter the peers by `f` - Some(peer_list) => peer_list - .gossipsub - .iter() - .cloned() - .filter(|p| f(p)) - .collect(), + Some(peer_list) => peer_list.iter().cloned().filter(|p| f(p)).collect(), None => Vec::new(), }; @@ -991,17 +948,15 @@ where // TODO: Handle the peer addition - Specifically handle floodsub peers. // For the time being assume all gossipsub peers - self.peer_topics - .insert(id.clone(), (SmallVec::new(), NodeType::Gossipsub)); + self.peer_topics.insert(id.clone(), SmallVec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { - // TODO: Handle peer disconnection - specifically floodsub peers // TODO: Refactor // remove from mesh, topic_peers and peer_topic { - let (topics, node_type) = match self.peer_topics.get(&id) { - Some((topics, node_type)) => (topics, node_type), + let topics = match self.peer_topics.get(&id) { + Some(topics) => (topics), None => { println!("ERROR: Disconnected node, not in connected nodes"); return; @@ -1021,33 +976,16 @@ where // remove from topic_peers if let Some(peer_list) = self.topic_peers.get_mut(&topic) { - match node_type { - NodeType::Gossipsub => { - if let Some(pos) = peer_list.gossipsub.iter().position(|p| p == id) { - peer_list.gossipsub.remove(pos); - //TODO: untagPeer - } - // debugging purposes - else { - println!( - "ERROR: Disconnected node: {:?} not in topic_peers peer list", - &id - ); - } - } - NodeType::Floodsub => { - if let Some(pos) = peer_list.floodsub.iter().position(|p| p == id) { - peer_list.floodsub.remove(pos); - //TODO: untagPeer - } - // debugging purposes - else { - println!( - "ERROR: Disconnected node: {:?} not in topic_peers peer list", - &id - ); - } - } + if let Some(pos) = peer_list.iter().position(|p| p == id) { + peer_list.remove(pos); + //TODO: untagPeer + } + // debugging purposes + else { + println!( + "ERROR: Disconnected node: {:?} not in topic_peers peer list", + &id + ); } } else { println!( @@ -1187,24 +1125,6 @@ impl From<()> for InnerMessage { } } -/// Struct that contains lists of gossipsub and floodsub peers. -#[derive(Debug, Clone)] -struct PeerList { - /// List of gossipsub peers. - gossipsub: Vec, - /// List of floodsub peers. - floodsub: Vec, -} - -impl PeerList { - pub fn new() -> Self { - PeerList { - gossipsub: vec![], - floodsub: vec![], - } - } -} - /// Event that can happen on the gossipsub behaviour. #[derive(Debug)] pub enum GossipsubEvent { @@ -1227,12 +1147,3 @@ pub enum GossipsubEvent { topic: TopicHash, }, } - -/// The type of node in the pubsub system. -#[derive(Debug)] -pub enum NodeType { - /// A gossipsub node. - Gossipsub, - /// A Floodsub node. - Floodsub, -} diff --git a/protocols/gossipsub/src/layer/tests.rs b/protocols/gossipsub/src/layer/tests.rs index 0c6555751ea..f61dc7a3fcc 100644 --- a/protocols/gossipsub/src/layer/tests.rs +++ b/protocols/gossipsub/src/layer/tests.rs @@ -332,20 +332,11 @@ mod tests { // should add the new peers to `peer_topics` with an empty vec as a gossipsub node for peer in peers { - let known_topics = &gs.peer_topics.get(&peer).unwrap().0; - let node_type = &gs.peer_topics.get(&peer).unwrap().1; + let known_topics = gs.peer_topics.get(&peer).unwrap(); assert!( known_topics == &SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes.clone()), "The topics for each node should all topics" ); - // TODO: Update this for handling floodsub nodes - assert!( - match node_type { - NodeType::Gossipsub => true, - _ => false, - }, - "All peers should be added as a gossipsub node" - ); } } @@ -388,12 +379,12 @@ mod tests { // verify the result - let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); + let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().clone(); assert!( peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), "First peer should be subscribed to three topics" ); - let peer_topics = gs.peer_topics.get(&peers[1]).unwrap().0.clone(); + let peer_topics = gs.peer_topics.get(&peers[1]).unwrap().clone(); assert!( peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), "Second peer should be subscribed to three topics" @@ -405,7 +396,7 @@ mod tests { ); for topic_hash in topic_hashes[..3].iter() { - let topic_peers = gs.topic_peers.get(topic_hash).unwrap().gossipsub.clone(); // only gossipsub at the moment + let topic_peers = gs.topic_peers.get(topic_hash).unwrap().clone(); assert!( topic_peers == peers[..2].to_vec(), "Two peers should be added to the first three topics" @@ -422,18 +413,13 @@ mod tests { &peers[0], ); - let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().0.clone(); + let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().clone(); assert!( peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[1..3].to_vec()), "Peer should be subscribed to two topics" ); - let topic_peers = gs - .topic_peers - .get(&topic_hashes[0]) - .unwrap() - .gossipsub - .clone(); // only gossipsub at the moment + let topic_peers = gs.topic_peers.get(&topic_hashes[0]).unwrap().clone(); // only gossipsub at the moment assert!( topic_peers == peers[1..2].to_vec(), "Only the second peers should be in the first topic" @@ -455,13 +441,7 @@ mod tests { peers.push(PeerId::random()) } - gs.topic_peers.insert( - topic_hash.clone(), - PeerList { - gossipsub: peers.clone(), - floodsub: vec![], - }, - ); + gs.topic_peers.insert(topic_hash.clone(), peers.clone()); let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| true }); assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); From 2d9b1255392971f14f1bb411b52e14718710fb42 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 26 Mar 2019 13:38:23 +1100 Subject: [PATCH 55/91] Reconnected to disconnected peers, to mitigate timeout --- protocols/gossipsub/src/layer.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 261774b7283..c59c93f8fd4 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -1000,7 +1000,10 @@ where let was_in = self.peer_topics.remove(id); debug_assert!(was_in.is_some()); - //TODO: Reconnect due to inactivity + // Some peers disconnect due to inactivity. Try and reconnect. + self.events.push_back(NetworkBehaviourAction::DialPeer { + peer_id: id.clone(), + }); } fn inject_node_event(&mut self, propagation_source: PeerId, event: InnerMessage) { From 987fec350b5c2eb55e5f0139615adab49ce493ff Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 26 Mar 2019 14:48:18 +1100 Subject: [PATCH 56/91] Use ReadOne WriteOne with configurable max gossip sizes --- protocols/gossipsub/examples/chat.rs | 7 +- protocols/gossipsub/src/gossipsub_config.rs | 11 + protocols/gossipsub/src/layer.rs | 2 +- protocols/gossipsub/src/protocol.rs | 260 +++++++++----------- src/lib.rs | 2 +- 5 files changed, 140 insertions(+), 142 deletions(-) diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index 587d4a3a2a9..841bd7ccaff 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -8,8 +8,9 @@ use env_logger::{Builder, Env}; use futures::prelude::*; use libp2p::gossipsub::GossipsubEvent; use libp2p::{ - gossipsub, secio, + gossipsub, identity, tokio_codec::{FramedRead, LinesCodec}, + PeerId, }; use std::time::Duration; @@ -17,8 +18,8 @@ fn main() { Builder::from_env(Env::default().default_filter_or("debug")).init(); // Create a random PeerId - let local_key = secio::SecioKeyPair::ed25519_generated().unwrap(); - let local_peer_id = local_key.to_peer_id(); + let local_key = identity::Keypair::generate_ed25519(); + let local_peer_id = PeerId::from(local_key.public()); println!("Local peer id: {:?}", local_peer_id); // Set up an encrypted TCP Transport over the Mplex and Yamux protocols diff --git a/protocols/gossipsub/src/gossipsub_config.rs b/protocols/gossipsub/src/gossipsub_config.rs index a8e387a9146..2b31c1fbe23 100644 --- a/protocols/gossipsub/src/gossipsub_config.rs +++ b/protocols/gossipsub/src/gossipsub_config.rs @@ -25,6 +25,8 @@ pub struct GossipsubConfig { pub heartbeat_interval: Duration, /// Time to live for fanout peers. pub fanout_ttl: Duration, + /// The maximum byte size for each gossip. + pub max_gossip_size: usize, } impl Default for GossipsubConfig { @@ -39,6 +41,7 @@ impl Default for GossipsubConfig { heartbeat_initial_delay: Duration::from_secs(5), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), + max_gossip_size: 2048, } } } @@ -64,6 +67,8 @@ pub struct GossipsubConfigBuilder { heartbeat_interval: Duration, /// Time to live for fanout peers. fanout_ttl: Duration, + /// The maximum byte size for each gossip. + pub max_gossip_size: usize, } impl Default for GossipsubConfigBuilder { @@ -78,6 +83,7 @@ impl Default for GossipsubConfigBuilder { heartbeat_initial_delay: Duration::from_secs(5), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), + max_gossip_size: 2048, } } } @@ -150,6 +156,10 @@ impl GossipsubConfigBuilder { self.fanout_ttl = fanout_ttl; self } + pub fn max_gossip_size(&mut self, max_gossip_size: usize) -> &mut Self { + self.max_gossip_size = max_gossip_size; + self + } pub fn build(&self) -> GossipsubConfig { GossipsubConfig { @@ -162,6 +172,7 @@ impl GossipsubConfigBuilder { heartbeat_initial_delay: self.heartbeat_initial_delay, heartbeat_interval: self.heartbeat_interval, fanout_ttl: self.fanout_ttl, + max_gossip_size: self.max_gossip_size, } } } diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index c59c93f8fd4..445dda1bb9d 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -917,7 +917,7 @@ where type OutEvent = GossipsubEvent; fn new_handler(&mut self) -> Self::ProtocolsHandler { - Default::default() + OneShotHandler::new(ProtocolConfig::new(self.config.max_gossip_size)) } fn addresses_of_peer(&mut self, _: &PeerId) -> Vec { diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index caf63b135b6..a148982855c 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -20,25 +20,32 @@ use crate::rpc_proto; use byteorder::{BigEndian, ByteOrder}; -use bytes::BytesMut; -use futures::{future, stream, Future, Stream}; -use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; +use libp2p_core::{upgrade, InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; use libp2p_floodsub::TopicHash; -use protobuf::Message as ProtobufMessage; +use protobuf::{Message as ProtobufMessage, ProtobufError}; use std::{io, iter}; -use tokio_codec::{Decoder, FramedRead}; use tokio_io::{AsyncRead, AsyncWrite}; -use unsigned_varint::codec; /// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. -#[derive(Debug, Clone, Default)] -pub struct ProtocolConfig {} +#[derive(Debug, Clone)] +pub struct ProtocolConfig { + max_transmit_size: usize, +} + +impl Default for ProtocolConfig { + fn default() -> Self { + Self { + max_transmit_size: 2048, + } + } +} impl ProtocolConfig { /// Builds a new `ProtocolConfig`. #[inline] - pub fn new() -> ProtocolConfig { - ProtocolConfig {} + /// Sets the maximum gossip transmission size. + pub fn new(max_transmit_size: usize) -> ProtocolConfig { + ProtocolConfig { max_transmit_size } } } @@ -57,132 +64,121 @@ where TSocket: AsyncRead, { type Output = GossipsubRpc; - type Error = io::Error; - type Future = future::MapErr< - future::AndThen< - stream::StreamFuture>, - Result)>, - fn( - (Option, FramedRead), - ) - -> Result)>, - >, - fn((io::Error, FramedRead)) -> io::Error, + type Error = GossipsubDecodeError; + type Future = upgrade::ReadOneThen< + upgrade::Negotiated, + (), + fn(Vec, ()) -> Result, >; - #[inline] - fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future { - FramedRead::new( - socket, - GossipsubCodec { - length_prefix: Default::default(), - }, - ) - .into_future() - .and_then:: _, _>(|(val, socket)| { - val.ok_or_else(move || (io::ErrorKind::UnexpectedEof.into(), socket)) - }) - .map_err(|(err, _)| err) - } -} + fn upgrade_inbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { + upgrade::read_one_then(socket, self.max_transmit_size, (), |packet, ()| { + let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; + + let mut messages = Vec::with_capacity(rpc.get_publish().len()); + for mut publish in rpc.take_publish().into_iter() { + messages.push(GossipsubMessage { + source: PeerId::from_bytes(publish.take_from()) + .map_err(|_| GossipsubDecodeError::InvalidPeerId)?, + data: publish.take_data(), + sequence_number: publish.take_seqno(), + topics: publish + .take_topicIDs() + .into_iter() + .map(TopicHash::from_raw) + .collect(), + }); + } -/// Implementation of `tokio_codec::Codec`. -pub struct GossipsubCodec { - /// The codec for encoding/decoding the length prefix of messages. - length_prefix: codec::UviBytes, -} + let mut rpc_control = rpc.take_control(); + let mut control_msgs = vec![]; + // Collect the gossipsub control messages + let ihave_msgs: Vec = rpc_control + .take_ihave() + .into_iter() + .map(|mut ihave| { + GossipsubControlAction::IHave { + topic_hash: TopicHash::from_raw(ihave.take_topicID()), + // TODO: Potentially format the message ids better + message_ids: ihave.take_messageIDs().into_vec(), + } + }) + .collect(); -impl Decoder for GossipsubCodec { - type Item = GossipsubRpc; - type Error = io::Error; + let iwant_msgs: Vec = rpc_control + .take_iwant() + .into_iter() + .map(|mut iwant| { + GossipsubControlAction::IWant { + // TODO: Potentially format the message ids better + message_ids: iwant.take_messageIDs().into_vec(), + } + }) + .collect(); - fn decode(&mut self, src: &mut BytesMut) -> Result, Self::Error> { - let packet = match self.length_prefix.decode(src)? { - Some(p) => p, - None => return Ok(None), - }; + let graft_msgs: Vec = rpc_control + .take_graft() + .into_iter() + .map(|mut graft| GossipsubControlAction::Graft { + topic_hash: TopicHash::from_raw(graft.take_topicID()), + }) + .collect(); - let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; - - let mut messages = Vec::with_capacity(rpc.get_publish().len()); - for mut publish in rpc.take_publish().into_iter() { - messages.push(GossipsubMessage { - source: PeerId::from_bytes(publish.take_from()).map_err(|_| { - io::Error::new(io::ErrorKind::InvalidData, "Invalid peer ID in message") - })?, - data: publish.take_data(), - sequence_number: publish.take_seqno(), - topics: publish - .take_topicIDs() - .into_iter() - .map(TopicHash::from_raw) - .collect(), - }); - } + let prune_msgs: Vec = rpc_control + .take_prune() + .into_iter() + .map(|mut prune| GossipsubControlAction::Prune { + topic_hash: TopicHash::from_raw(prune.take_topicID()), + }) + .collect(); - let mut rpc_control = rpc.take_control(); - let mut control_msgs = vec![]; - // Collect the gossipsub control messages - let ihave_msgs: Vec = rpc_control - .take_ihave() - .into_iter() - .map(|mut ihave| { - GossipsubControlAction::IHave { - topic_hash: TopicHash::from_raw(ihave.take_topicID()), - // TODO: Potentially format the message ids better - message_ids: ihave.take_messageIDs().into_vec(), - } - }) - .collect(); - - let iwant_msgs: Vec = rpc_control - .take_iwant() - .into_iter() - .map(|mut iwant| { - GossipsubControlAction::IWant { - // TODO: Potentially format the message ids better - message_ids: iwant.take_messageIDs().into_vec(), - } - }) - .collect(); + control_msgs.extend(ihave_msgs); + control_msgs.extend(iwant_msgs); + control_msgs.extend(graft_msgs); + control_msgs.extend(prune_msgs); - let graft_msgs: Vec = rpc_control - .take_graft() - .into_iter() - .map(|mut graft| GossipsubControlAction::Graft { - topic_hash: TopicHash::from_raw(graft.take_topicID()), + Ok(GossipsubRpc { + messages, + subscriptions: rpc + .take_subscriptions() + .into_iter() + .map(|mut sub| GossipsubSubscription { + action: if sub.get_subscribe() { + GossipsubSubscriptionAction::Subscribe + } else { + GossipsubSubscriptionAction::Unsubscribe + }, + topic_hash: TopicHash::from_raw(sub.take_topicid()), + }) + .collect(), + control_msgs, }) - .collect(); + }) + } +} - let prune_msgs: Vec = rpc_control - .take_prune() - .into_iter() - .map(|mut prune| GossipsubControlAction::Prune { - topic_hash: TopicHash::from_raw(prune.take_topicID()), - }) - .collect(); +/// Reach attempt interrupt errors. +#[derive(Debug)] +pub enum GossipsubDecodeError { + /// Error when reading the packet from the socket. + ReadError(upgrade::ReadOneError), + /// Error when decoding the raw buffer into a protobuf. + ProtobufError(ProtobufError), + /// Error when parsing the `PeerId` in the message. + InvalidPeerId, +} - control_msgs.extend(ihave_msgs); - control_msgs.extend(iwant_msgs); - control_msgs.extend(graft_msgs); - control_msgs.extend(prune_msgs); +impl From for GossipsubDecodeError { + #[inline] + fn from(err: upgrade::ReadOneError) -> Self { + GossipsubDecodeError::ReadError(err) + } +} - Ok(Some(GossipsubRpc { - messages, - subscriptions: rpc - .take_subscriptions() - .into_iter() - .map(|mut sub| GossipsubSubscription { - action: if sub.get_subscribe() { - GossipsubSubscriptionAction::Subscribe - } else { - GossipsubSubscriptionAction::Unsubscribe - }, - topic_hash: TopicHash::from_raw(sub.take_topicid()), - }) - .collect(), - control_msgs, - })) +impl From for GossipsubDecodeError { + #[inline] + fn from(err: ProtobufError) -> Self { + GossipsubDecodeError::ProtobufError(err) } } @@ -213,21 +209,11 @@ where { type Output = (); type Error = io::Error; - type Future = future::Map< - future::AndThen< - tokio_io::io::WriteAll>, - tokio_io::io::Shutdown, - fn((TSocket, Vec)) -> tokio_io::io::Shutdown, - >, - fn(TSocket) -> (), - >; - + type Future = upgrade::WriteOne>; #[inline] - fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { let bytes = self.into_length_delimited_bytes(); - tokio_io::io::write_all(socket, bytes) - .and_then:: _, _>(|(socket, _)| tokio_io::io::shutdown(socket)) - .map(|_| ()) + upgrade::write_one(socket, bytes) } } diff --git a/src/lib.rs b/src/lib.rs index 866fb79d2f9..8e25c1bec7e 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -210,7 +210,7 @@ pub use self::core::{ identity, transport::TransportError, upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt}, - PeerId, PeerId, Swarm, Swarm, Transport, Transport, + PeerId, Swarm, Transport, }; pub use self::multiaddr::{multiaddr as build_multiaddr, Multiaddr}; pub use self::simple::SimpleProtocol; From b3c32d9a821ae6cc89079499cc6e8a6bab0bffc3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 26 Mar 2019 16:34:03 +1100 Subject: [PATCH 57/91] Remove length delimination from RPC encoding --- protocols/gossipsub/src/protocol.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index a148982855c..92ec468c8f1 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -210,16 +210,17 @@ where type Output = (); type Error = io::Error; type Future = upgrade::WriteOne>; + #[inline] fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { - let bytes = self.into_length_delimited_bytes(); + let bytes = self.into_bytes(); upgrade::write_one(socket, bytes) } } impl GossipsubRpc { /// Turns this `GossipsubRpc` into a message that can be sent to a substream. - fn into_length_delimited_bytes(self) -> Vec { + fn into_bytes(self) -> Vec { let mut proto = rpc_proto::RPC::new(); for message in self.messages.into_iter() { @@ -285,7 +286,7 @@ impl GossipsubRpc { proto.set_control(control_msg); proto - .write_length_delimited_to_bytes() + .write_to_bytes() .expect("there is no situation in which the protobuf message can be invalid") } } From 666d1f1fd0acce7f7326499c018f04ad03b97bdd Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 2 Apr 2019 16:06:10 +1100 Subject: [PATCH 58/91] Prevent peer duplication in mesh --- protocols/gossipsub/src/layer.rs | 27 ++++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 445dda1bb9d..9282fa010b5 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -19,8 +19,6 @@ // DEALINGS IN THE SOFTWARE. // -// TODO: Implement control message piggybacking - use cuckoofilter::CuckooFilter; use futures::prelude::*; use gossipsub_config::GossipsubConfig; @@ -533,7 +531,7 @@ impl Gossipsub { subscriptions: &[GossipsubSubscription], propagation_source: &PeerId, ) { - debug!( + trace!( "Handling subscriptions from source: {:?}", propagation_source ); @@ -603,7 +601,7 @@ impl Gossipsub { } } } - debug!( + trace!( "Completed handling subscriptions from source: {:?}", propagation_source ); @@ -633,11 +631,15 @@ impl Gossipsub { let peer_list = self .get_random_peers(topic_hash, desired_peers, { |peer| !peers.contains(peer) }); for peer in peer_list { - peers.push(peer.clone()); + // exclude potential duplicates + if !peers.contains(&peer) { + peers.push(peer.clone()); + } // TODO: tagPeer let current_topic = to_graft.entry(peer).or_insert_with(|| vec![]); current_topic.push(topic_hash.clone()); } + debug!("Updating mesh, new mesh: {:?}", peers); // update the mesh self.mesh.insert(topic_hash.clone(), peers.clone()); } @@ -719,7 +721,12 @@ impl Gossipsub { let needed_peers = self.config.mesh_n - peers.len(); let mut new_peers = self.get_random_peers(topic_hash, needed_peers, |peer| !peers.contains(peer)); - peers.append(&mut new_peers); + // check for duplicates before adding + for new_peer in new_peers { + if !peers.contains(&new_peer) { + peers.push(new_peer); + } + } } // update the entry self.fanout.insert(topic_hash.clone(), peers.to_vec()); @@ -925,6 +932,7 @@ where } fn inject_connected(&mut self, id: PeerId, _: ConnectedPoint) { + info!("New peer connected: {:?}", id); // We need to send our subscriptions to the newly-connected node. let mut subscriptions = vec![]; for topic_hash in self.mesh.keys() { @@ -954,11 +962,12 @@ where fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { // TODO: Refactor // remove from mesh, topic_peers and peer_topic + debug!("Peer disconnected: {:?}", id); { let topics = match self.peer_topics.get(&id) { Some(topics) => (topics), None => { - println!("ERROR: Disconnected node, not in connected nodes"); + error!("ERROR: Disconnected node, not in connected nodes"); return; } }; @@ -982,13 +991,13 @@ where } // debugging purposes else { - println!( + warn!( "ERROR: Disconnected node: {:?} not in topic_peers peer list", &id ); } } else { - println!( + warn!( "ERROR: Disconnected node: {:?} with topic: {:?} not in topic_peers", &id, &topic ); From fd6b1ab0b1be9c32c2fba8e82eec0cd6fa350e1e Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 2 Apr 2019 16:18:51 +1100 Subject: [PATCH 59/91] Allow oneshot handler's inactivity_timeout to be configurable --- core/src/protocols_handler/one_shot.rs | 69 +++++++++++++++++--------- 1 file changed, 46 insertions(+), 23 deletions(-) diff --git a/core/src/protocols_handler/one_shot.rs b/core/src/protocols_handler/one_shot.rs index f4832fddecc..3f8ec501f09 100644 --- a/core/src/protocols_handler/one_shot.rs +++ b/core/src/protocols_handler/one_shot.rs @@ -18,7 +18,9 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr}; +use crate::protocols_handler::{ + KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, +}; use crate::upgrade::{InboundUpgrade, OutboundUpgrade}; use futures::prelude::*; use smallvec::SmallVec; @@ -30,12 +32,14 @@ use tokio_io::{AsyncRead, AsyncWrite}; /// This struct is meant to be a helper for other implementations to use. // TODO: Debug pub struct OneShotHandler -where TOutProto: OutboundUpgrade +where + TOutProto: OutboundUpgrade, { /// The upgrade for inbound substreams. listen_protocol: TInProto, /// If `Some`, something bad happened and we should shut down the handler with an error. - pending_error: Option>::Error>>, + pending_error: + Option>::Error>>, /// Queue of events to produce in `poll()`. events_out: SmallVec<[TOutEvent; 4]>, /// Queue of outbound substreams to open. @@ -54,11 +58,12 @@ where TOutProto: OutboundUpgrade impl OneShotHandler -where TOutProto: OutboundUpgrade +where + TOutProto: OutboundUpgrade, { /// Creates a `OneShotHandler`. #[inline] - pub fn new(listen_protocol: TInProto) -> Self { + pub fn new(listen_protocol: TInProto, inactive_timeout: Duration) -> Self { OneShotHandler { listen_protocol, pending_error: None, @@ -67,7 +72,7 @@ where TOutProto: OutboundUpgrade dial_negotiated: 0, max_dial_negotiated: 8, keep_alive: KeepAlive::Forever, - inactive_timeout: Duration::from_secs(10), // TODO: allow configuring + inactive_timeout, marker: PhantomData, } } @@ -104,19 +109,20 @@ where TOutProto: OutboundUpgrade } } -impl Default for - OneShotHandler -where TOutProto: OutboundUpgrade, - TInProto: Default +impl Default + for OneShotHandler +where + TOutProto: OutboundUpgrade, + TInProto: Default, { #[inline] fn default() -> Self { - OneShotHandler::new(Default::default()) + OneShotHandler::new(Default::default(), Duration::from_secs(10)) } } -impl ProtocolsHandler for - OneShotHandler +impl ProtocolsHandler + for OneShotHandler where TSubstream: AsyncRead + AsyncWrite, TInProto: InboundUpgrade + Clone, @@ -127,7 +133,9 @@ where { type InEvent = TOutProto; type OutEvent = TOutEvent; - type Error = ProtocolsHandlerUpgrErr<>::Error>; + type Error = ProtocolsHandlerUpgrErr< + >::Error, + >; type Substream = TSubstream; type InboundProtocol = TInProto; type OutboundProtocol = TOutProto; @@ -141,7 +149,7 @@ where #[inline] fn inject_fully_negotiated_inbound( &mut self, - out: >::Output + out: >::Output, ) { // If we're shutting down the connection for inactivity, reset the timeout. if !self.keep_alive.is_forever() { @@ -155,7 +163,7 @@ where fn inject_fully_negotiated_outbound( &mut self, out: >::Output, - _: Self::OutboundOpenInfo + _: Self::OutboundOpenInfo, ) { self.dial_negotiated -= 1; @@ -172,7 +180,13 @@ where } #[inline] - fn inject_dial_upgrade_error(&mut self, _: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>::Error>) { + fn inject_dial_upgrade_error( + &mut self, + _: Self::OutboundOpenInfo, + error: ProtocolsHandlerUpgrErr< + >::Error, + >, + ) { if self.pending_error.is_none() { self.pending_error = Some(error); } @@ -183,13 +197,20 @@ where self.keep_alive } - fn poll(&mut self) -> Poll, Self::Error> { + fn poll( + &mut self, + ) -> Poll< + ProtocolsHandlerEvent, + Self::Error, + > { if let Some(err) = self.pending_error.take() { return Err(err); } if !self.events_out.is_empty() { - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(self.events_out.remove(0)))); + return Ok(Async::Ready(ProtocolsHandlerEvent::Custom( + self.events_out.remove(0), + ))); } else { self.events_out.shrink_to_fit(); } @@ -197,10 +218,12 @@ where if !self.dial_queue.is_empty() { if self.dial_negotiated < self.max_dial_negotiated { self.dial_negotiated += 1; - return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { - upgrade: self.dial_queue.remove(0), - info: (), - })); + return Ok(Async::Ready( + ProtocolsHandlerEvent::OutboundSubstreamRequest { + upgrade: self.dial_queue.remove(0), + info: (), + }, + )); } } else { self.dial_queue.shrink_to_fit(); From 25ce0fa1092b40855e9ddd6a5f25cfac9cfd4585 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 30 Apr 2019 15:14:55 +1000 Subject: [PATCH 60/91] Correct peer duplication in mesh bug --- core/src/nodes/raw_swarm.rs | 965 +++++++++++++++++++----------- core/src/protocols_handler/mod.rs | 51 +- protocols/gossipsub/src/layer.rs | 30 +- 3 files changed, 660 insertions(+), 386 deletions(-) diff --git a/core/src/nodes/raw_swarm.rs b/core/src/nodes/raw_swarm.rs index 5045242d9b3..3325cd4d7f7 100644 --- a/core/src/nodes/raw_swarm.rs +++ b/core/src/nodes/raw_swarm.rs @@ -20,32 +20,25 @@ use crate::muxing::StreamMuxer; use crate::{ - Endpoint, Multiaddr, PeerId, + nodes::listeners::{ListenersEvent, ListenersStream}, nodes::{ collection::{ - CollectionEvent, - CollectionNodeAccept, - CollectionReachEvent, - CollectionStream, - ReachAttemptId - }, - handled_node::{ - HandledNodeError, - NodeHandler + CollectionEvent, CollectionNodeAccept, CollectionReachEvent, CollectionStream, + ReachAttemptId, }, + handled_node::{HandledNodeError, NodeHandler}, handled_node_tasks::IntoNodeHandler, - node::Substream + node::Substream, }, - nodes::listeners::{ListenersEvent, ListenersStream}, transport::Transport, transport::TransportError, + Endpoint, Multiaddr, PeerId, }; use fnv::FnvHashMap; -use futures::{prelude::*, future}; +use futures::{future, prelude::*}; use std::{ collections::hash_map::{Entry, OccupiedEntry}, - error, - fmt, + error, fmt, hash::Hash, num::NonZeroUsize, }; @@ -61,18 +54,26 @@ where listeners: ListenersStream, /// The nodes currently active. - active_nodes: CollectionStream, THandlerErr, (), TPeerId>, + active_nodes: CollectionStream< + TInEvent, + TOutEvent, + THandler, + InternalReachErr, + THandlerErr, + (), + TPeerId, + >, /// The reach attempts of the swarm. /// This needs to be a separate struct in order to handle multiple mutable borrows issues. reach_attempts: ReachAttempts, - /// Max numer of incoming connections. + /// Max number of incoming connections. incoming_limit: Option, } -impl fmt::Debug for - RawSwarm +impl fmt::Debug + for RawSwarm where TTrans: Transport + fmt::Debug, TPeerId: fmt::Debug + Eq + Hash, @@ -130,8 +131,15 @@ struct OutReachAttempt { } /// Event that can happen on the `RawSwarm`. -pub enum RawSwarmEvent<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a, TPeerId: 'a = PeerId> -where +pub enum RawSwarmEvent< + 'a, + TTrans: 'a, + TInEvent: 'a, + TOutEvent: 'a, + THandler: 'a, + THandlerErr: 'a, + TPeerId: 'a = PeerId, +> where TTrans: Transport, { /// One of the listeners gracefully closed. @@ -145,7 +153,9 @@ where }, /// A new connection arrived on a listener. - IncomingConnection(IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>), + IncomingConnection( + IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, + ), /// A new connection was arriving on a listener, but an error happened when negotiating it. /// @@ -226,8 +236,8 @@ where }, } -impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> fmt::Debug for - RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> +impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> fmt::Debug + for RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where TOutEvent: fmt::Debug, TTrans: Transport, @@ -237,65 +247,91 @@ where { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { match *self { - RawSwarmEvent::ListenerClosed { ref listen_addr, ref result, .. } => { - f.debug_struct("ListenerClosed") - .field("listen_addr", listen_addr) - .field("result", result) - .finish() - } - RawSwarmEvent::IncomingConnection( IncomingConnectionEvent { ref listen_addr, ref send_back_addr, .. } ) => { - f.debug_struct("IncomingConnection") - .field("listen_addr", listen_addr) - .field("send_back_addr", send_back_addr) - .finish() - } - RawSwarmEvent::IncomingConnectionError { ref listen_addr, ref send_back_addr, ref error} => { - f.debug_struct("IncomingConnectionError") - .field("listen_addr", listen_addr) - .field("send_back_addr", send_back_addr) - .field("error", error) - .finish() - } - RawSwarmEvent::Connected { ref peer_id, ref endpoint } => { - f.debug_struct("Connected") - .field("peer_id", peer_id) - .field("endpoint", endpoint) - .finish() - } - RawSwarmEvent::Replaced { ref peer_id, ref closed_endpoint, ref endpoint } => { - f.debug_struct("Replaced") - .field("peer_id", peer_id) - .field("closed_endpoint", closed_endpoint) - .field("endpoint", endpoint) - .finish() - } - RawSwarmEvent::NodeClosed { ref peer_id, ref endpoint, ref error } => { - f.debug_struct("NodeClosed") - .field("peer_id", peer_id) - .field("endpoint", endpoint) - .field("error", error) - .finish() - } - RawSwarmEvent::DialError { ref new_state, ref peer_id, ref multiaddr, ref error } => { - f.debug_struct("DialError") - .field("new_state", new_state) - .field("peer_id", peer_id) - .field("multiaddr", multiaddr) - .field("error", error) - .finish() - } - RawSwarmEvent::UnknownPeerDialError { ref multiaddr, ref error, .. } => { - f.debug_struct("UnknownPeerDialError") - .field("multiaddr", multiaddr) - .field("error", error) - .finish() - } - RawSwarmEvent::NodeEvent { ref peer_id, ref event } => { - f.debug_struct("NodeEvent") - .field("peer_id", peer_id) - .field("event", event) - .finish() - } + RawSwarmEvent::ListenerClosed { + ref listen_addr, + ref result, + .. + } => f + .debug_struct("ListenerClosed") + .field("listen_addr", listen_addr) + .field("result", result) + .finish(), + RawSwarmEvent::IncomingConnection(IncomingConnectionEvent { + ref listen_addr, + ref send_back_addr, + .. + }) => f + .debug_struct("IncomingConnection") + .field("listen_addr", listen_addr) + .field("send_back_addr", send_back_addr) + .finish(), + RawSwarmEvent::IncomingConnectionError { + ref listen_addr, + ref send_back_addr, + ref error, + } => f + .debug_struct("IncomingConnectionError") + .field("listen_addr", listen_addr) + .field("send_back_addr", send_back_addr) + .field("error", error) + .finish(), + RawSwarmEvent::Connected { + ref peer_id, + ref endpoint, + } => f + .debug_struct("Connected") + .field("peer_id", peer_id) + .field("endpoint", endpoint) + .finish(), + RawSwarmEvent::Replaced { + ref peer_id, + ref closed_endpoint, + ref endpoint, + } => f + .debug_struct("Replaced") + .field("peer_id", peer_id) + .field("closed_endpoint", closed_endpoint) + .field("endpoint", endpoint) + .finish(), + RawSwarmEvent::NodeClosed { + ref peer_id, + ref endpoint, + ref error, + } => f + .debug_struct("NodeClosed") + .field("peer_id", peer_id) + .field("endpoint", endpoint) + .field("error", error) + .finish(), + RawSwarmEvent::DialError { + ref new_state, + ref peer_id, + ref multiaddr, + ref error, + } => f + .debug_struct("DialError") + .field("new_state", new_state) + .field("peer_id", peer_id) + .field("multiaddr", multiaddr) + .field("error", error) + .finish(), + RawSwarmEvent::UnknownPeerDialError { + ref multiaddr, + ref error, + .. + } => f + .debug_struct("UnknownPeerDialError") + .field("multiaddr", multiaddr) + .field("error", error) + .finish(), + RawSwarmEvent::NodeEvent { + ref peer_id, + ref event, + } => f + .debug_struct("NodeEvent") + .field("peer_id", peer_id) + .field("event", event) + .finish(), } } } @@ -325,10 +361,8 @@ where InternalReachErr::Transport(err) => write!(f, "{}", err), InternalReachErr::PeerIdMismatch { obtained } => { write!(f, "Peer ID mismatch, obtained: {:?}", obtained) - }, - InternalReachErr::FoundLocalPeerId => { - write!(f, "Remote has the same PeerId as us") } + InternalReachErr::FoundLocalPeerId => write!(f, "Remote has the same PeerId as us"), } } } @@ -372,7 +406,7 @@ pub enum RawSwarmReachError { PeerIdMismatch { /// The peer id that the node reports. obtained: TPeerId, - } + }, } impl fmt::Display for RawSwarmReachError @@ -385,7 +419,7 @@ where RawSwarmReachError::Transport(err) => write!(f, "{}", err), RawSwarmReachError::PeerIdMismatch { obtained } => { write!(f, "Peer ID mismatch, obtained: {:?}", obtained) - }, + } } } } @@ -413,20 +447,20 @@ pub enum UnknownPeerDialErr { } impl fmt::Display for UnknownPeerDialErr -where TTransErr: fmt::Display +where + TTransErr: fmt::Display, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { UnknownPeerDialErr::Transport(err) => write!(f, "{}", err), - UnknownPeerDialErr::FoundLocalPeerId => { - write!(f, "Unknown peer has same PeerId as us") - }, + UnknownPeerDialErr::FoundLocalPeerId => write!(f, "Unknown peer has same PeerId as us"), } } } impl error::Error for UnknownPeerDialErr -where TTransErr: error::Error + 'static +where + TTransErr: error::Error + 'static, { fn source(&self) -> Option<&(dyn error::Error + 'static)> { match self { @@ -450,23 +484,23 @@ pub enum IncomingError { } impl fmt::Display for IncomingError -where TTransErr: fmt::Display +where + TTransErr: fmt::Display, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { IncomingError::Transport(err) => write!(f, "{}", err), - IncomingError::DeniedLowerPriority => { - write!(f, "Denied because of lower priority") - }, + IncomingError::DeniedLowerPriority => write!(f, "Denied because of lower priority"), IncomingError::FoundLocalPeerId => { write!(f, "Incoming connection has same PeerId as us") - }, + } } } } impl error::Error for IncomingError -where TTransErr: error::Error + 'static +where + TTransErr: error::Error + 'static, { fn source(&self) -> Option<&(dyn error::Error + 'static)> { match self { @@ -479,7 +513,8 @@ where TTransErr: error::Error + 'static /// A new connection arrived on a listener. pub struct IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> -where TTrans: Transport +where + TTrans: Transport, { /// The produced upgrade. upgrade: TTrans::ListenerUpgrade, @@ -490,7 +525,15 @@ where TTrans: Transport /// Address used to send back data to the remote. send_back_addr: Multiaddr, /// Reference to the `active_nodes` field of the swarm. - active_nodes: &'a mut CollectionStream, THandlerErr, (), TPeerId>, + active_nodes: &'a mut CollectionStream< + TInEvent, + TOutEvent, + THandler, + InternalReachErr, + THandlerErr, + (), + TPeerId, + >, /// Reference to the `other_reach_attempts` field of the swarm. other_reach_attempts: &'a mut Vec<(ReachAttemptId, ConnectedPoint)>, } @@ -502,7 +545,13 @@ where TTrans::Error: Send + 'static, TTrans::ListenerUpgrade: Send + 'static, THandler: IntoNodeHandler + Send + 'static, - THandler::Handler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, + THandler::Handler: NodeHandler< + Substream = Substream, + InEvent = TInEvent, + OutEvent = TOutEvent, + Error = THandlerErr, + > + Send + + 'static, ::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandlerErr: error::Error + Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, @@ -520,12 +569,14 @@ where /// Same as `accept`, but accepts a closure that turns a `IncomingInfo` into a handler. pub fn accept_with_builder(self, builder: TBuilder) - where TBuilder: FnOnce(IncomingInfo<'_>) -> THandler + where + TBuilder: FnOnce(IncomingInfo<'_>) -> THandler, { let connected_point = self.to_connected_point(); let handler = builder(self.info()); let local_peer_id = self.local_peer_id; - let upgrade = self.upgrade + let upgrade = self + .upgrade .map_err(|err| InternalReachErr::Transport(TransportError::Other(err))) .and_then(move |(peer_id, muxer)| { if peer_id == local_peer_id { @@ -535,16 +586,14 @@ where } }); let id = self.active_nodes.add_reach_attempt(upgrade, handler); - self.other_reach_attempts.push(( - id, - connected_point, - )); + self.other_reach_attempts.push((id, connected_point)); } } impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> -where TTrans: Transport +where + TTrans: Transport, { /// Returns the `IncomingInfo` corresponding to this incoming connection. #[inline] @@ -661,7 +710,13 @@ where TTrans: Transport + Clone, TMuxer: StreamMuxer, THandler: IntoNodeHandler + Send + 'static, - THandler::Handler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, + THandler::Handler: NodeHandler< + Substream = Substream, + InEvent = TInEvent, + OutEvent = TOutEvent, + Error = THandlerErr, + > + Send + + 'static, ::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandlerErr: error::Error + Send + 'static, TPeerId: fmt::Debug + Eq + Hash + Clone + AsRef<[u8]> + Send + 'static, @@ -685,9 +740,11 @@ where /// Creates a new node event stream with incoming connections limit. #[inline] - pub fn new_with_incoming_limit(transport: TTrans, - local_peer_id: TPeerId, incoming_limit: Option) -> Self - { + pub fn new_with_incoming_limit( + transport: TTrans, + local_peer_id: TPeerId, + incoming_limit: Option, + ) -> Self { RawSwarm { incoming_limit, listeners: ListenersStream::new(transport), @@ -709,7 +766,10 @@ where /// Start listening on the given multiaddress. #[inline] - pub fn listen_on(&mut self, addr: Multiaddr) -> Result> { + pub fn listen_on( + &mut self, + addr: Multiaddr, + ) -> Result> { self.listeners.listen_on(addr) } @@ -737,8 +797,9 @@ where &'a self, observed_addr: &'a Multiaddr, ) -> impl Iterator + 'a - where TMuxer: 'a, - THandler: 'a, + where + TMuxer: 'a, + THandler: 'a, { self.listeners() .flat_map(move |server| self.transport().nat_traversal(server, observed_addr)) @@ -755,7 +816,11 @@ where /// Dials a multiaddress without knowing the peer ID we're going to obtain. /// /// The second parameter is the handler to use if we manage to reach a node. - pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), TransportError> + pub fn dial( + &mut self, + addr: Multiaddr, + handler: THandler, + ) -> Result<(), TransportError> where TTrans: Transport, TTrans::Error: Send + 'static, @@ -767,7 +832,10 @@ where TOutEvent: Send + 'static, { let local_peer_id = self.reach_attempts.local_peer_id.clone(); - let future = self.transport().clone().dial(addr.clone())? + let future = self + .transport() + .clone() + .dial(addr.clone())? .map_err(|err| InternalReachErr::Transport(TransportError::Other(err))) .and_then(move |(peer_id, muxer)| { if peer_id == local_peer_id { @@ -779,7 +847,9 @@ where let connected_point = ConnectedPoint::Dialer { address: addr }; let reach_id = self.active_nodes.add_reach_attempt(future, handler); - self.reach_attempts.other_reach_attempts.push((reach_id, connected_point)); + self.reach_attempts + .other_reach_attempts + .push((reach_id, connected_point)); Ok(()) } @@ -791,7 +861,8 @@ where #[deprecated(note = "Use incoming_negotiated().count() instead")] #[inline] pub fn num_incoming_negotiated(&self) -> usize { - self.reach_attempts.other_reach_attempts + self.reach_attempts + .other_reach_attempts .iter() .filter(|&(_, endpoint)| endpoint.is_listener()) .count() @@ -804,30 +875,33 @@ where self.reach_attempts .other_reach_attempts .iter() - .filter_map(|&(_, ref endpoint)| { - match endpoint { - ConnectedPoint::Listener { listen_addr, send_back_addr } => { - Some(IncomingInfo { - listen_addr, - send_back_addr, - }) - }, - ConnectedPoint::Dialer { .. } => None, - } + .filter_map(|&(_, ref endpoint)| match endpoint { + ConnectedPoint::Listener { + listen_addr, + send_back_addr, + } => Some(IncomingInfo { + listen_addr, + send_back_addr, + }), + ConnectedPoint::Dialer { .. } => None, }) } /// Sends an event to all nodes. #[inline] pub fn broadcast_event(&mut self, event: &TInEvent) - where TInEvent: Clone, + where + TInEvent: Clone, { self.active_nodes.broadcast_event(event) } /// Grants access to a struct that represents a peer. #[inline] - pub fn peer(&mut self, peer_id: TPeerId) -> Peer<'_, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> { + pub fn peer( + &mut self, + peer_id: TPeerId, + ) -> Peer<'_, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> { if peer_id == self.reach_attempts.local_peer_id { return Peer::LocalNode; } @@ -847,9 +921,18 @@ where // The state of `connected_points` always follows `self.active_nodes`. debug_assert!(!self.reach_attempts.connected_points.contains_key(&peer_id)); - if self.reach_attempts.out_reach_attempts.get_mut(&peer_id).is_some() { + if self + .reach_attempts + .out_reach_attempts + .get_mut(&peer_id) + .is_some() + { return Peer::PendingConnect(PeerPendingConnect { - attempt: match self.reach_attempts.out_reach_attempts.entry(peer_id.clone()) { + attempt: match self + .reach_attempts + .out_reach_attempts + .entry(peer_id.clone()) + { Entry::Occupied(e) => e, Entry::Vacant(_) => panic!("we checked for Some just above"), }, @@ -868,8 +951,13 @@ where /// /// It is a logic error to call this method if we already have an outgoing attempt to the /// given peer. - fn start_dial_out(&mut self, peer_id: TPeerId, handler: THandler, first: Multiaddr, rest: Vec) - where + fn start_dial_out( + &mut self, + peer_id: TPeerId, + handler: THandler, + first: Multiaddr, + rest: Vec, + ) where TTrans: Transport, TTrans::Dial: Send + 'static, TTrans::Error: Send + 'static, @@ -888,15 +976,17 @@ where if actual_peer_id == expected_peer_id { Ok((actual_peer_id, muxer)) } else { - Err(InternalReachErr::PeerIdMismatch { obtained: actual_peer_id }) + Err(InternalReachErr::PeerIdMismatch { + obtained: actual_peer_id, + }) } }); self.active_nodes.add_reach_attempt(fut, handler) - }, + } Err(err) => { let fut = future::err(InternalReachErr::Transport(err)); self.active_nodes.add_reach_attempt(fut, handler) - }, + } }; let former = self.reach_attempts.out_reach_attempts.insert( @@ -912,7 +1002,9 @@ where } /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll(&mut self) -> Async> + pub fn poll( + &mut self, + ) -> Async> where TTrans: Transport, TTrans::Error: Send + 'static, @@ -924,44 +1016,49 @@ where TInEvent: Send + 'static, TOutEvent: Send + 'static, THandler: IntoNodeHandler + Send + 'static, - THandler::Handler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, + THandler::Handler: NodeHandler< + Substream = Substream, + InEvent = TInEvent, + OutEvent = TOutEvent, + Error = THandlerErr, + > + Send + + 'static, ::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandlerErr: error::Error + Send + 'static, { // Start by polling the listeners for events, but only - // if numer of incoming connection does not exceed the limit. + // if number of incoming connection does not exceed the limit. match self.incoming_limit { - Some(x) if self.incoming_negotiated().count() >= (x as usize) - => (), - _ => { - match self.listeners.poll() { - Async::NotReady => (), - Async::Ready(ListenersEvent::Incoming { - upgrade, listen_addr, send_back_addr }) => - { - let event = IncomingConnectionEvent { - upgrade, - local_peer_id: - self.reach_attempts.local_peer_id.clone(), - listen_addr, - send_back_addr, - active_nodes: &mut self.active_nodes, - other_reach_attempts: &mut self.reach_attempts.other_reach_attempts, - }; - return Async::Ready(RawSwarmEvent::IncomingConnection(event)); - }, - Async::Ready(ListenersEvent::Closed { - listen_addr, listener, result }) => - { - return Async::Ready(RawSwarmEvent::ListenerClosed { - listen_addr, - listener, - result, - }); - } - + Some(x) if self.incoming_negotiated().count() >= (x as usize) => (), + _ => match self.listeners.poll() { + Async::NotReady => (), + Async::Ready(ListenersEvent::Incoming { + upgrade, + listen_addr, + send_back_addr, + }) => { + let event = IncomingConnectionEvent { + upgrade, + local_peer_id: self.reach_attempts.local_peer_id.clone(), + listen_addr, + send_back_addr, + active_nodes: &mut self.active_nodes, + other_reach_attempts: &mut self.reach_attempts.other_reach_attempts, + }; + return Async::Ready(RawSwarmEvent::IncomingConnection(event)); } - } + Async::Ready(ListenersEvent::Closed { + listen_addr, + listener, + result, + }) => { + return Async::Ready(RawSwarmEvent::ListenerClosed { + listen_addr, + listener, + result, + }); + } + }, } // Poll the existing nodes. @@ -978,17 +1075,18 @@ where action = a; out_event = e; } - Async::Ready(CollectionEvent::NodeClosed { - peer_id, - error, - .. - }) => { - let endpoint = self.reach_attempts.connected_points.remove(&peer_id) - .expect("We insert into connected_points whenever a connection is \ - opened and remove only when a connection is closed; the \ - underlying API is guaranteed to always deliver a connection \ - closed message after it has been opened, and no two closed \ - messages; QED"); + Async::Ready(CollectionEvent::NodeClosed { peer_id, error, .. }) => { + let endpoint = self + .reach_attempts + .connected_points + .remove(&peer_id) + .expect( + "We insert into connected_points whenever a connection is \ + opened and remove only when a connection is closed; the \ + underlying API is guaranteed to always deliver a connection \ + closed message after it has been opened, and no two closed \ + messages; QED", + ); action = Default::default(); out_event = RawSwarmEvent::NodeClosed { peer_id, @@ -998,7 +1096,10 @@ where } Async::Ready(CollectionEvent::NodeEvent { peer, event }) => { action = Default::default(); - out_event = RawSwarmEvent::NodeEvent { peer_id: peer.id().clone(), event }; + out_event = RawSwarmEvent::NodeEvent { + peer_id: peer.id().clone(), + event, + }; } } @@ -1008,15 +1109,18 @@ where if let Some((peer_id, interrupt)) = action.take_over { // TODO: improve proof or remove; this is too complicated right now - let interrupted = self.active_nodes - .interrupt(interrupt) - .expect("take_over is guaranteed to be gathered from `out_reach_attempts`; + let interrupted = self.active_nodes.interrupt(interrupt).expect( + "take_over is guaranteed to be gathered from `out_reach_attempts`; we insert in out_reach_attempts only when we call \ active_nodes.add_reach_attempt, and we remove only when we call \ interrupt or when a reach attempt succeeds or errors; therefore the \ out_reach_attempts should always be in sync with the actual \ - attempts; QED"); - self.active_nodes.peer_mut(&peer_id).unwrap().take_over(interrupted); + attempts; QED", + ); + self.active_nodes + .peer_mut(&peer_id) + .unwrap() + .take_over(interrupted); } Async::Ready(out_event) @@ -1050,8 +1154,20 @@ impl Default for ActionItem { /// > panics will likely happen. fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>( reach_attempts: &mut ReachAttempts, - event: CollectionReachEvent<'_, TInEvent, TOutEvent, THandler, InternalReachErr, THandlerErr, (), TPeerId>, -) -> (ActionItem, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>) + event: CollectionReachEvent< + '_, + TInEvent, + TOutEvent, + THandler, + InternalReachErr, + THandlerErr, + (), + TPeerId, + >, +) -> ( + ActionItem, + RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, +) where TTrans: Transport + Clone, TMuxer: StreamMuxer + Send + Sync + 'static, @@ -1074,23 +1190,34 @@ where // If we already have an active connection to this peer, a priority system comes into play. // If we have a lower peer ID than the incoming one, we drop an incoming connection. if event.would_replace() && has_dial_prio { - if let Some(ConnectedPoint::Dialer { .. }) = reach_attempts.connected_points.get(event.peer_id()) { - if let ConnectedPoint::Listener { listen_addr, send_back_addr } = opened_endpoint { - return (Default::default(), RawSwarmEvent::IncomingConnectionError { - listen_addr, - send_back_addr, - error: IncomingError::DeniedLowerPriority, - }); + if let Some(ConnectedPoint::Dialer { .. }) = + reach_attempts.connected_points.get(event.peer_id()) + { + if let ConnectedPoint::Listener { + listen_addr, + send_back_addr, + } = opened_endpoint + { + return ( + Default::default(), + RawSwarmEvent::IncomingConnectionError { + listen_addr, + send_back_addr, + error: IncomingError::DeniedLowerPriority, + }, + ); } } } // Set the endpoint for this peer. - let closed_endpoint = reach_attempts.connected_points.insert(event.peer_id().clone(), opened_endpoint.clone()); + let closed_endpoint = reach_attempts + .connected_points + .insert(event.peer_id().clone(), opened_endpoint.clone()); // If we have dial priority, we keep the current outgoing attempt because it may already // have succeeded without us knowing. It is possible that the remote has already closed - // its ougoing attempt because it sees our outgoing attempt as a success. + // its outgoing attempt because it sees our outgoing attempt as a success. // However we cancel any further multiaddress to attempt in any situation. let action = if has_dial_prio { if let Some(attempt) = reach_attempts.out_reach_attempts.get_mut(&event.peer_id()) { @@ -1103,7 +1230,7 @@ where debug_assert_ne!(attempt.id, event.reach_attempt_id()); ActionItem { take_over: Some((event.peer_id().clone(), attempt.id)), - .. Default::default() + ..Default::default() } } else { ActionItem::default() @@ -1112,27 +1239,38 @@ where let (outcome, peer_id) = event.accept(()); if let CollectionNodeAccept::ReplacedExisting(()) = outcome { - let closed_endpoint = closed_endpoint - .expect("We insert into connected_points whenever a connection is opened and \ - remove only when a connection is closed; the underlying API is \ - guaranteed to always deliver a connection closed message after it has \ - been opened, and no two closed messages; QED"); - return (action, RawSwarmEvent::Replaced { - peer_id, - endpoint: opened_endpoint, - closed_endpoint, - }); + let closed_endpoint = closed_endpoint.expect( + "We insert into connected_points whenever a connection is opened and \ + remove only when a connection is closed; the underlying API is \ + guaranteed to always deliver a connection closed message after it has \ + been opened, and no two closed messages; QED", + ); + return ( + action, + RawSwarmEvent::Replaced { + peer_id, + endpoint: opened_endpoint, + closed_endpoint, + }, + ); } else { - return (action, RawSwarmEvent::Connected { peer_id, endpoint: opened_endpoint }); + return ( + action, + RawSwarmEvent::Connected { + peer_id, + endpoint: opened_endpoint, + }, + ); } } // Otherwise, try for outgoing attempts. - let is_outgoing_and_ok = if let Some(attempt) = reach_attempts.out_reach_attempts.get(event.peer_id()) { - attempt.id == event.reach_attempt_id() - } else { - false - }; + let is_outgoing_and_ok = + if let Some(attempt) = reach_attempts.out_reach_attempts.get(event.peer_id()) { + attempt.id == event.reach_attempt_id() + } else { + false + }; // We only remove the attempt from `out_reach_attempts` if it both matches the reach id // and the expected peer id. @@ -1145,33 +1283,46 @@ where address: attempt.cur_attempted, }; - let closed_endpoint = reach_attempts.connected_points + let closed_endpoint = reach_attempts + .connected_points .insert(event.peer_id().clone(), opened_endpoint.clone()); let (outcome, peer_id) = event.accept(()); if let CollectionNodeAccept::ReplacedExisting(()) = outcome { - let closed_endpoint = closed_endpoint - .expect("We insert into connected_points whenever a connection is opened and \ - remove only when a connection is closed; the underlying API is guaranteed \ - to always deliver a connection closed message after it has been opened, \ - and no two closed messages; QED"); - return (Default::default(), RawSwarmEvent::Replaced { - peer_id, - endpoint: opened_endpoint, - closed_endpoint, - }); + let closed_endpoint = closed_endpoint.expect( + "We insert into connected_points whenever a connection is opened and \ + remove only when a connection is closed; the underlying API is guaranteed \ + to always deliver a connection closed message after it has been opened, \ + and no two closed messages; QED", + ); + return ( + Default::default(), + RawSwarmEvent::Replaced { + peer_id, + endpoint: opened_endpoint, + closed_endpoint, + }, + ); } else { - return (Default::default(), RawSwarmEvent::Connected { peer_id, endpoint: opened_endpoint }); + return ( + Default::default(), + RawSwarmEvent::Connected { + peer_id, + endpoint: opened_endpoint, + }, + ); } } // We didn't find any entry in neither the outgoing connections not ingoing connections. // TODO: improve proof or remove; this is too complicated right now - panic!("The API of collection guarantees that the id sent back in NodeReached (which is where \ - we call handle_node_reached) is one that was passed to add_reach_attempt. Whenever we \ - call add_reach_attempt, we also insert at the same time an entry either in \ - out_reach_attempts or in other_reach_attempts. It is therefore guaranteed that we \ - find back this ID in either of these two sets"); + panic!( + "The API of collection guarantees that the id sent back in NodeReached (which is where \ + we call handle_node_reached) is one that was passed to add_reach_attempt. Whenever we \ + call add_reach_attempt, we also insert at the same time an entry either in \ + out_reach_attempts or in other_reach_attempts. It is therefore guaranteed that we \ + find back this ID in either of these two sets" + ); } /// Returns true if `local` has dialing priority over `other`. @@ -1197,7 +1348,10 @@ fn handle_reach_error<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TP reach_id: ReachAttemptId, error: InternalReachErr, handler: THandler, -) -> (ActionItem, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>) +) -> ( + ActionItem, + RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, +) where TTrans: Transport, TPeerId: Eq + Hash + Clone, @@ -1210,7 +1364,9 @@ where .find(|(_, a)| a.id == reach_id) .map(|(p, _)| p.clone()); if let Some(peer_id) = out_reach_peer_id { - let attempt = reach_attempts.out_reach_attempts.remove(&peer_id) + let attempt = reach_attempts + .out_reach_attempts + .remove(&peer_id) .expect("out_reach_peer_id is a key that is grabbed from out_reach_attempts"); let num_remain = attempt.next_attempts.len(); @@ -1231,8 +1387,13 @@ where let mut attempt = attempt; let next_attempt = attempt.next_attempts.remove(0); ActionItem { - start_dial_out: Some((peer_id.clone(), handler, next_attempt, attempt.next_attempts)), - .. Default::default() + start_dial_out: Some(( + peer_id.clone(), + handler, + next_attempt, + attempt.next_attempts, + )), + ..Default::default() } } else { Default::default() @@ -1242,19 +1403,22 @@ where InternalReachErr::Transport(err) => RawSwarmReachError::Transport(err), InternalReachErr::PeerIdMismatch { obtained } => { RawSwarmReachError::PeerIdMismatch { obtained } - }, - InternalReachErr::FoundLocalPeerId => { - unreachable!("We only generate FoundLocalPeerId within dial() or accept(); neither \ - of these methods add an entry to out_reach_attempts; QED") - }, + } + InternalReachErr::FoundLocalPeerId => unreachable!( + "We only generate FoundLocalPeerId within dial() or accept(); neither \ + of these methods add an entry to out_reach_attempts; QED" + ), }; - return (action, RawSwarmEvent::DialError { - new_state, - peer_id, - multiaddr: failed_addr, - error, - }); + return ( + action, + RawSwarmEvent::DialError { + new_state, + peer_id, + multiaddr: failed_addr, + error, + }, + ); } // If this is not an outgoing reach attempt, check the incoming reach attempts. @@ -1269,55 +1433,75 @@ where let error = match error { InternalReachErr::Transport(err) => UnknownPeerDialErr::Transport(err), InternalReachErr::FoundLocalPeerId => UnknownPeerDialErr::FoundLocalPeerId, - InternalReachErr::PeerIdMismatch { .. } => { - unreachable!("We only generate PeerIdMismatch within start_dial_out(), - which doesn't add any entry in other_reach_attempts; QED") - }, + InternalReachErr::PeerIdMismatch { .. } => unreachable!( + "We only generate PeerIdMismatch within start_dial_out(), + which doesn't add any entry in other_reach_attempts; QED" + ), }; - return (Default::default(), RawSwarmEvent::UnknownPeerDialError { - multiaddr: address, - error, - handler, - }); + return ( + Default::default(), + RawSwarmEvent::UnknownPeerDialError { + multiaddr: address, + error, + handler, + }, + ); } - ConnectedPoint::Listener { listen_addr, send_back_addr } => { + ConnectedPoint::Listener { + listen_addr, + send_back_addr, + } => { let error = match error { InternalReachErr::Transport(err) => IncomingError::Transport(err), InternalReachErr::FoundLocalPeerId => IncomingError::FoundLocalPeerId, - InternalReachErr::PeerIdMismatch { .. } => { - unreachable!("We only generate PeerIdMismatch within start_dial_out(), - which doesn't add any entry in other_reach_attempts; QED") - }, + InternalReachErr::PeerIdMismatch { .. } => unreachable!( + "We only generate PeerIdMismatch within start_dial_out(), + which doesn't add any entry in other_reach_attempts; QED" + ), }; - return (Default::default(), RawSwarmEvent::IncomingConnectionError { - listen_addr, - send_back_addr, - error - }); + return ( + Default::default(), + RawSwarmEvent::IncomingConnectionError { + listen_addr, + send_back_addr, + error, + }, + ); } } } // The id was neither in the outbound list nor the inbound list. // TODO: improve proof or remove; this is too complicated right now - panic!("The API of collection guarantees that the id sent back in ReachError events \ - (which is where we call handle_reach_error) is one that was passed to \ - add_reach_attempt. Whenever we call add_reach_attempt, we also insert \ - at the same time an entry either in out_reach_attempts or in \ - other_reach_attempts. It is therefore guaranteed that we find back this ID in \ - either of these two sets"); + panic!( + "The API of collection guarantees that the id sent back in ReachError events \ + (which is where we call handle_reach_error) is one that was passed to \ + add_reach_attempt. Whenever we call add_reach_attempt, we also insert \ + at the same time an entry either in out_reach_attempts or in \ + other_reach_attempts. It is therefore guaranteed that we find back this ID in \ + either of these two sets" + ); } /// State of a peer in the system. -pub enum Peer<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a, TPeerId: 'a> -where +pub enum Peer< + 'a, + TTrans: 'a, + TInEvent: 'a, + TOutEvent: 'a, + THandler: 'a, + THandlerErr: 'a, + TPeerId: 'a, +> where TTrans: Transport, { /// We are connected to this peer. Connected(PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>), /// We are currently attempting to connect to this peer. - PendingConnect(PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>), + PendingConnect( + PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, + ), /// We are not connected to this peer at all. /// @@ -1329,34 +1513,32 @@ where LocalNode, } -impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> fmt::Debug for - Peer<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> +impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> fmt::Debug + for Peer<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where TTrans: Transport, TPeerId: Eq + Hash + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { match *self { - Peer::Connected( PeerConnected { ref peer_id, ref connected_points, .. }) => { - f.debug_struct("Connected") - .field("peer_id", peer_id) - .field("connected_points", connected_points) - .finish() - } - Peer::PendingConnect( PeerPendingConnect { ref attempt, .. } ) => { - f.debug_struct("PendingConnect") - .field("attempt", attempt) - .finish() - } - Peer::NotConnected(PeerNotConnected { ref peer_id, .. }) => { - f.debug_struct("NotConnected") - .field("peer_id", peer_id) - .finish() - } - Peer::LocalNode => { - f.debug_struct("LocalNode") - .finish() - } + Peer::Connected(PeerConnected { + ref peer_id, + ref connected_points, + .. + }) => f + .debug_struct("Connected") + .field("peer_id", peer_id) + .field("connected_points", connected_points) + .finish(), + Peer::PendingConnect(PeerPendingConnect { ref attempt, .. }) => f + .debug_struct("PendingConnect") + .field("attempt", attempt) + .finish(), + Peer::NotConnected(PeerNotConnected { ref peer_id, .. }) => f + .debug_struct("NotConnected") + .field("peer_id", peer_id) + .finish(), + Peer::LocalNode => f.debug_struct("LocalNode").finish(), } } } @@ -1374,14 +1556,23 @@ where TInEvent: Send + 'static, TOutEvent: Send + 'static, THandler: IntoNodeHandler + Send + 'static, - THandler::Handler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, + THandler::Handler: NodeHandler< + Substream = Substream, + InEvent = TInEvent, + OutEvent = TOutEvent, + Error = THandlerErr, + > + Send + + 'static, ::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandlerErr: error::Error + Send + 'static, TPeerId: fmt::Debug + Eq + Hash + Clone + AsRef<[u8]> + Send + 'static, { /// If we are connected, returns the `PeerConnected`. #[inline] - pub fn into_connected(self) -> Option> { + pub fn into_connected( + self, + ) -> Option> + { match self { Peer::Connected(peer) => Some(peer), _ => None, @@ -1390,7 +1581,10 @@ where /// If a connection is pending, returns the `PeerPendingConnect`. #[inline] - pub fn into_pending_connect(self) -> Option> { + pub fn into_pending_connect( + self, + ) -> Option> + { match self { Peer::PendingConnect(peer) => Some(peer), _ => None, @@ -1399,7 +1593,10 @@ where /// If we are not connected, returns the `PeerNotConnected`. #[inline] - pub fn into_not_connected(self) -> Option> { + pub fn into_not_connected( + self, + ) -> Option> + { match self { Peer::NotConnected(peer) => Some(peer), _ => None, @@ -1413,9 +1610,14 @@ where /// /// Returns an error if we are `LocalNode`. #[inline] - pub fn or_connect(self, addr: Multiaddr, handler: THandler) - -> Result, Self> - { + pub fn or_connect( + self, + addr: Multiaddr, + handler: THandler, + ) -> Result< + PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, + Self, + > { self.or_connect_with(move |_| addr, handler) } @@ -1427,8 +1629,14 @@ where /// /// Returns an error if we are `LocalNode`. #[inline] - pub fn or_connect_with(self, addr: TFn, handler: THandler) - -> Result, Self> + pub fn or_connect_with( + self, + addr: TFn, + handler: THandler, + ) -> Result< + PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, + Self, + > where TFn: FnOnce(&TPeerId) -> Multiaddr, { @@ -1437,23 +1645,34 @@ where Peer::PendingConnect(peer) => Ok(PeerPotentialConnect::PendingConnect(peer)), Peer::NotConnected(peer) => { let addr = addr(&peer.peer_id); - Ok(PeerPotentialConnect::PendingConnect(peer.connect(addr, handler))) - }, + Ok(PeerPotentialConnect::PendingConnect( + peer.connect(addr, handler), + )) + } Peer::LocalNode => Err(Peer::LocalNode), } } } /// Peer we are potentially going to connect to. -pub enum PeerPotentialConnect<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a, TPeerId: 'a> -where - TTrans: Transport +pub enum PeerPotentialConnect< + 'a, + TTrans: 'a, + TInEvent: 'a, + TOutEvent: 'a, + THandler: 'a, + THandlerErr: 'a, + TPeerId: 'a, +> where + TTrans: Transport, { /// We are connected to this peer. Connected(PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>), /// We are currently attempting to connect to this peer. - PendingConnect(PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>), + PendingConnect( + PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, + ), } impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> @@ -1474,7 +1693,10 @@ where /// If we are connected, returns the `PeerConnected`. #[inline] - pub fn into_connected(self) -> Option> { + pub fn into_connected( + self, + ) -> Option> + { match self { PeerPotentialConnect::Connected(peer) => Some(peer), _ => None, @@ -1483,7 +1705,10 @@ where /// If a connection is pending, returns the `PeerPendingConnect`. #[inline] - pub fn into_pending_connect(self) -> Option> { + pub fn into_pending_connect( + self, + ) -> Option> + { match self { PeerPotentialConnect::PendingConnect(peer) => Some(peer), _ => None, @@ -1493,10 +1718,19 @@ where /// Access to a peer we are connected to. pub struct PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> -where TTrans: Transport, +where + TTrans: Transport, { /// Reference to the `active_nodes` of the parent. - active_nodes: &'a mut CollectionStream, THandlerErr, (), TPeerId>, + active_nodes: &'a mut CollectionStream< + TInEvent, + TOutEvent, + THandler, + InternalReachErr, + THandlerErr, + (), + TPeerId, + >, /// Reference to the `connected_points` field of the parent. connected_points: &'a mut FnvHashMap, /// Reference to the `out_reach_attempts` field of the parent. @@ -1504,7 +1738,8 @@ where TTrans: Transport, peer_id: TPeerId, } -impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> +impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> + PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where TTrans: Transport, TPeerId: Eq + Hash + Clone, @@ -1522,7 +1757,8 @@ where } self.connected_points.remove(&self.peer_id); - self.active_nodes.peer_mut(&self.peer_id) + self.active_nodes + .peer_mut(&self.peer_id) .expect("A PeerConnected is always created with a PeerId in active_nodes; QED") .close(); } @@ -1530,17 +1766,19 @@ where /// Returns the endpoint we're connected to. #[inline] pub fn endpoint(&self) -> &ConnectedPoint { - self.connected_points.get(&self.peer_id) - .expect("We insert into connected_points whenever a connection is opened and remove \ - only when a connection is closed; the underlying API is guaranteed to always \ - deliver a connection closed message after it has been opened, and no two \ - closed messages; QED") + self.connected_points.get(&self.peer_id).expect( + "We insert into connected_points whenever a connection is opened and remove \ + only when a connection is closed; the underlying API is guaranteed to always \ + deliver a connection closed message after it has been opened, and no two \ + closed messages; QED", + ) } /// Sends an event to the node. #[inline] pub fn send_event(&mut self, event: TInEvent) { - self.active_nodes.peer_mut(&self.peer_id) + self.active_nodes + .peer_mut(&self.peer_id) .expect("A PeerConnected is always created with a PeerId in active_nodes; QED") .send_event(event) } @@ -1550,10 +1788,18 @@ where #[derive(Debug)] pub struct PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where - TTrans: Transport + TTrans: Transport, { attempt: OccupiedEntry<'a, TPeerId, OutReachAttempt>, - active_nodes: &'a mut CollectionStream, THandlerErr, (), TPeerId>, + active_nodes: &'a mut CollectionStream< + TInEvent, + TOutEvent, + THandler, + InternalReachErr, + THandlerErr, + (), + TPeerId, + >, } impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> @@ -1570,11 +1816,13 @@ where let attempt = self.attempt.remove(); if self.active_nodes.interrupt(attempt.id).is_err() { // TODO: improve proof or remove; this is too complicated right now - panic!("We retreived this attempt.id from out_reach_attempts. We insert in \ - out_reach_attempts only at the same time as we call add_reach_attempt. \ - Whenever we receive a NodeReached, NodeReplaced or ReachError event, which \ - invalidate the attempt.id, we also remove the corresponding entry in \ - out_reach_attempts."); + panic!( + "We retreived this attempt.id from out_reach_attempts. We insert in \ + out_reach_attempts only at the same time as we call add_reach_attempt. \ + Whenever we receive a NodeReached, NodeReplaced or ReachError event, which \ + invalidate the attempt.id, we also remove the corresponding entry in \ + out_reach_attempts." + ); } } @@ -1620,8 +1868,8 @@ where nodes: &'a mut RawSwarm, } -impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> fmt::Debug for - PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> +impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> fmt::Debug + for PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where TTrans: Transport, TPeerId: fmt::Debug, @@ -1643,7 +1891,13 @@ where TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, THandler: IntoNodeHandler + Send + 'static, - THandler::Handler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, + THandler::Handler: NodeHandler< + Substream = Substream, + InEvent = TInEvent, + OutEvent = TOutEvent, + Error = THandlerErr, + > + Send + + 'static, ::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandlerErr: error::Error + Send + 'static, TInEvent: Send + 'static, @@ -1654,8 +1908,11 @@ where /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// the whole connection is immediately closed. #[inline] - pub fn connect(self, addr: Multiaddr, handler: THandler) - -> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> + pub fn connect( + self, + addr: Multiaddr, + handler: THandler, + ) -> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where TPeerId: fmt::Debug + Eq + Hash + Clone + AsRef<[u8]> + Send + 'static, { @@ -1671,8 +1928,14 @@ where /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// the whole connection is immediately closed. #[inline] - pub fn connect_iter(self, addrs: TIter, handler: THandler) - -> Result, Self> + pub fn connect_iter( + self, + addrs: TIter, + handler: THandler, + ) -> Result< + PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId>, + Self, + > where TIter: IntoIterator, TPeerId: fmt::Debug + Eq + Hash + Clone + AsRef<[u8]> + Send + 'static, @@ -1680,25 +1943,35 @@ where let mut addrs = addrs.into_iter(); let first = match addrs.next() { Some(f) => f, - None => return Err(self) + None => return Err(self), }; let rest = addrs.collect(); Ok(self.connect_inner(handler, first, rest)) } /// Inner implementation of `connect`. - fn connect_inner(self, handler: THandler, first: Multiaddr, rest: Vec) - -> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> + fn connect_inner( + self, + handler: THandler, + first: Multiaddr, + rest: Vec, + ) -> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TPeerId> where TPeerId: fmt::Debug + Eq + Hash + Clone + AsRef<[u8]> + Send + 'static, { - self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest); + self.nodes + .start_dial_out(self.peer_id.clone(), handler, first, rest); PeerPendingConnect { - attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) { + attempt: match self + .nodes + .reach_attempts + .out_reach_attempts + .entry(self.peer_id) + { Entry::Occupied(e) => e, Entry::Vacant(_) => { panic!("We called out_reach_attempts.insert with this peer id just above") - }, + } }, active_nodes: &mut self.nodes.active_nodes, } diff --git a/core/src/protocols_handler/mod.rs b/core/src/protocols_handler/mod.rs index 952dd4c25b6..b433c729293 100644 --- a/core/src/protocols_handler/mod.rs +++ b/core/src/protocols_handler/mod.rs @@ -33,12 +33,8 @@ //! > connection with a remote. In order to handle a protocol that requires knowledge of //! > the network as a whole, see the `NetworkBehaviour` trait. +use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError}; use crate::PeerId; -use crate::upgrade::{ - InboundUpgrade, - OutboundUpgrade, - UpgradeError, -}; use futures::prelude::*; use std::{cmp::Ordering, error, fmt, time::Duration, time::Instant}; use tokio_io::{AsyncRead, AsyncWrite}; @@ -46,7 +42,9 @@ use tokio_io::{AsyncRead, AsyncWrite}; pub use self::dummy::DummyProtocolsHandler; pub use self::map_in::MapInEvent; pub use self::map_out::MapOutEvent; -pub use self::node_handler::{NodeHandlerWrapper, NodeHandlerWrapperBuilder, NodeHandlerWrapperError}; +pub use self::node_handler::{ + NodeHandlerWrapper, NodeHandlerWrapperBuilder, NodeHandlerWrapperError, +}; pub use self::one_shot::OneShotHandler; pub use self::select::{IntoProtocolsHandlerSelect, ProtocolsHandlerSelect}; @@ -79,7 +77,7 @@ mod select; /// /// # Shutting down /// -/// Implementors of this trait should keep in mind that the connection can be closed at any time. +/// Implementers of this trait should keep in mind that the connection can be closed at any time. /// When a connection is closed (either by us or by the remote) `shutdown()` is called and the /// handler continues to be processed until it produces `ProtocolsHandlerEvent::Shutdown`. Only /// then the handler is destroyed. @@ -87,7 +85,7 @@ mod select; /// This makes it possible for the handler to finish delivering events even after knowing that it /// is shutting down. /// -/// Implementors of this trait should keep in mind that when `shutdown()` is called, the connection +/// Implementers of this trait should keep in mind that when `shutdown()` is called, the connection /// might already be closed or unresponsive. They should therefore not rely on being able to /// deliver messages. /// @@ -121,20 +119,26 @@ pub trait ProtocolsHandler { /// This method is called when a substream has been successfully opened and negotiated. fn inject_fully_negotiated_inbound( &mut self, - protocol: >::Output + protocol: >::Output, ); fn inject_fully_negotiated_outbound( &mut self, protocol: >::Output, - info: Self::OutboundOpenInfo + info: Self::OutboundOpenInfo, ); /// Injects an event coming from the outside in the handler. fn inject_event(&mut self, event: Self::InEvent); /// Indicates to the handler that upgrading a substream to the given protocol has failed. - fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>::Error>); + fn inject_dial_upgrade_error( + &mut self, + info: Self::OutboundOpenInfo, + error: ProtocolsHandlerUpgrErr< + >::Error, + >, + ); /// Returns until when the connection should be kept alive. /// @@ -155,7 +159,12 @@ pub trait ProtocolsHandler { /// Should behave like `Stream::poll()`. /// /// Returning an error will close the connection to the remote. - fn poll(&mut self) -> Poll, Self::Error>; + fn poll( + &mut self, + ) -> Poll< + ProtocolsHandlerEvent, + Self::Error, + >; /// Adds a closure that turns the input event into something else. #[inline] @@ -252,10 +261,7 @@ impl /// If this is an `OutboundSubstreamRequest`, maps the protocol (`TConnectionUpgrade`) to something else. #[inline] - pub fn map_protocol( - self, - map: F, - ) -> ProtocolsHandlerEvent + pub fn map_protocol(self, map: F) -> ProtocolsHandlerEvent where F: FnOnce(TConnectionUpgrade) -> I, { @@ -309,13 +315,11 @@ where match self { ProtocolsHandlerUpgrErr::Timeout => { write!(f, "Timeout error while opening a substream") - }, - ProtocolsHandlerUpgrErr::Timer => { - write!(f, "Timer error while opening a substream") - }, + } + ProtocolsHandlerUpgrErr::Timer => write!(f, "Timer error while opening a substream"), ProtocolsHandlerUpgrErr::MuxerDeniedSubstream => { write!(f, "Remote muxer denied our attempt to open a substream") - }, + } ProtocolsHandlerUpgrErr::Upgrade(err) => write!(f, "{}", err), } } @@ -323,7 +327,7 @@ where impl error::Error for ProtocolsHandlerUpgrErr where - TUpgrErr: error::Error + 'static + TUpgrErr: error::Error + 'static, { fn source(&self) -> Option<&(dyn error::Error + 'static)> { match self { @@ -367,7 +371,8 @@ pub trait IntoProtocolsHandler { } impl IntoProtocolsHandler for T -where T: ProtocolsHandler +where + T: ProtocolsHandler, { type Handler = Self; diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index c9a08594b71..1f8ca9ab375 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -436,12 +436,15 @@ impl Gossipsub { let mut to_prune_topics = HashSet::new(); for topic_hash in topics { if let Some(peers) = self.mesh.get_mut(&topic_hash) { - // if we are subscribed, add peer to the mesh + // if we are subscribed, add peer to the mesh, if not already added info!( "GRAFT: Mesh link added for peer: {:?} in topic: {:?}", peer_id, topic_hash ); - peers.push(peer_id.clone()); + // ensure peer is not already added + if !peers.contains(peer_id) { + peers.push(peer_id.clone()); + } //TODO: tagPeer } else { to_prune_topics.insert(topic_hash.clone()); @@ -628,20 +631,17 @@ impl Gossipsub { ); // not enough peers - get mesh_n - current_length more let desired_peers = self.config.mesh_n - peers.len(); - let peer_list = self + let mut peer_list = self .get_random_peers(topic_hash, desired_peers, { |peer| !peers.contains(peer) }); - for peer in peer_list { - // exclude potential duplicates - if !peers.contains(&peer) { - peers.push(peer.clone()); - } + for peer in &peer_list { // TODO: tagPeer - let current_topic = to_graft.entry(peer).or_insert_with(|| vec![]); + let current_topic = to_graft.entry(peer.clone()).or_insert_with(|| vec![]); current_topic.push(topic_hash.clone()); } - debug!("Updating mesh, new mesh: {:?}", peers); // update the mesh - self.mesh.insert(topic_hash.clone(), peers.clone()); + peer_list.extend(peers.clone()); + debug!("Updating mesh, new mesh: {:?}", peer_list); + self.mesh.insert(topic_hash.clone(), peer_list); } // too many peers - remove some @@ -721,12 +721,8 @@ impl Gossipsub { let needed_peers = self.config.mesh_n - peers.len(); let mut new_peers = self.get_random_peers(topic_hash, needed_peers, |peer| !peers.contains(peer)); - // check for duplicates before adding - for new_peer in new_peers { - if !peers.contains(&new_peer) { - peers.push(new_peer); - } - } + new_peers.extend(peers.clone()); + self.fanout.insert(topic_hash.clone(), new_peers); } // update the entry self.fanout.insert(topic_hash.clone(), peers.to_vec()); From 9ccb186779b42ac6e66c30a1afcd7b4ec486a369 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 30 Apr 2019 15:25:59 +1000 Subject: [PATCH 61/91] Remove auto-reconnect to allow for user-level disconnects --- protocols/gossipsub/src/layer.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/protocols/gossipsub/src/layer.rs b/protocols/gossipsub/src/layer.rs index 1f8ca9ab375..3d428eb8923 100644 --- a/protocols/gossipsub/src/layer.rs +++ b/protocols/gossipsub/src/layer.rs @@ -1007,11 +1007,6 @@ where // remove peer from peer_topics let was_in = self.peer_topics.remove(id); debug_assert!(was_in.is_some()); - - // Some peers disconnect due to inactivity. Try and reconnect. - self.events.push_back(NetworkBehaviourAction::DialPeer { - peer_id: id.clone(), - }); } fn inject_node_event(&mut self, propagation_source: PeerId, event: InnerMessage) { From 9c56b84dbe6657fa7300b6c97595d9acc8c7a504 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 29 Jul 2019 00:19:50 +1000 Subject: [PATCH 62/91] Single long-lived inbound/outbound streams to match go implementation --- protocols/gossipsub/Cargo.toml | 2 +- protocols/gossipsub/examples/chat.rs | 3 +- protocols/gossipsub/src/behaviour.rs | 80 ++--- protocols/gossipsub/src/gossipsub_config.rs | 47 +-- protocols/gossipsub/src/handler.rs | 327 ++++++++++++++++++++ protocols/gossipsub/src/lib.rs | 5 +- protocols/gossipsub/src/protocol.rs | 321 +++++++++---------- 7 files changed, 527 insertions(+), 258 deletions(-) create mode 100644 protocols/gossipsub/src/handler.rs diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index ac1fa20c219..60f56a5350b 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -21,7 +21,7 @@ smallvec = "0.6.8" tokio-codec = "0.1.1" tokio-io = "0.1.11" tokio-timer = "0.2.8" -unsigned-varint = "0.2.1" +unsigned-varint = "0.2.2" log = "0.4.6" [dev-dependencies] diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index 841bd7ccaff..1f16d57300c 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -30,8 +30,9 @@ fn main() { // Create a Swarm to manage peers and events let mut swarm = { - // set default parameters for gossipsub + // to set default parameters for gossipsub use: // let gossipsub_config = gossipsub::GossipsubConfig::default(); + // set custom gossipsub let gossipsub_config = gossipsub::GossipsubConfigBuilder::new() .heartbeat_interval(Duration::from_secs(10)) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 5efb68435e9..a1a3368fa28 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -1,38 +1,14 @@ -// Copyright 2018 Parity Technologies (UK) 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::gossipsub_config::GossipsubConfig; +use crate::handler::GossipsubHandler; use crate::mcache::MessageCache; use crate::protocol::{ - GossipsubControlAction, GossipsubMessage, GossipsubRpc, GossipsubSubscription, - GossipsubSubscriptionAction, ProtocolConfig, + GossipsubControlAction, GossipsubMessage, GossipsubSubscription, GossipsubSubscriptionAction, }; use cuckoofilter::CuckooFilter; use futures::prelude::*; use libp2p_core::{ConnectedPoint, Multiaddr, PeerId}; use libp2p_floodsub::{Topic, TopicHash}; -use libp2p_swarm::{ - NetworkBehaviour, NetworkBehaviourAction, OneShotHandler, PollParameters, ProtocolsHandler, - SubstreamProtocol, -}; +use libp2p_swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler}; use log::{debug, error, info, trace, warn}; use rand; use rand::{seq::SliceRandom, thread_rng}; @@ -56,7 +32,7 @@ pub struct Gossipsub { /// Events that need to be yielded to the outside when polling. events: VecDeque, GossipsubEvent>>, - // pool non-urgent control messages between heartbeats + /// Pools non-urgent control messages between heartbeats control_pool: HashMap>, /// Peer id of the local node. Used for the source of the messages that we publish. @@ -534,7 +510,8 @@ impl Gossipsub { propagation_source: &PeerId, ) { trace!( - "Handling subscriptions from source: {:?}", + "Handling subscriptions: {:?}, from source: {:?}", + subscriptions, propagation_source ); let subscribed_topics = match self.peer_topics.get_mut(&propagation_source) { @@ -915,13 +892,13 @@ impl NetworkBehaviour for Gossipsub where TSubstream: AsyncRead + AsyncWrite, { - type ProtocolsHandler = OneShotHandler; + type ProtocolsHandler = GossipsubHandler; type OutEvent = GossipsubEvent; fn new_handler(&mut self) -> Self::ProtocolsHandler { - OneShotHandler::new( - SubstreamProtocol::new(ProtocolConfig::new(self.config.max_gossip_size)), - self.config.inactivity_timeout, + GossipsubHandler::new( + self.config.protocol_id.clone(), + self.config.max_transmit_size, ) } @@ -1008,12 +985,7 @@ where debug_assert!(was_in.is_some()); } - fn inject_node_event(&mut self, propagation_source: PeerId, event: InnerMessage) { - // ignore successful sends event - let event = match event { - InnerMessage::Rx(event) => event, - InnerMessage::Sent => return, - }; + fn inject_node_event(&mut self, propagation_source: PeerId, event: GossipsubRpc) { // Handle subscriptions // Update connected peers topics self.handle_received_subscriptions(&event.subscriptions, &propagation_source); @@ -1107,27 +1079,15 @@ where } } -/// Transmission between the `OneShotHandler` and the `GossipsubRpc`. -#[derive(Debug)] -pub enum InnerMessage { - /// We received an RPC from a remote. - Rx(GossipsubRpc), - /// We successfully sent an RPC request. - Sent, -} - -impl From for InnerMessage { - #[inline] - fn from(rpc: GossipsubRpc) -> InnerMessage { - InnerMessage::Rx(rpc) - } -} - -impl From<()> for InnerMessage { - #[inline] - fn from(_: ()) -> InnerMessage { - InnerMessage::Sent - } +/// An RPC received/sent. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct GossipsubRpc { + /// List of messages that were part of this RPC query. + pub messages: Vec, + /// List of subscriptions. + pub subscriptions: Vec, + /// List of Gossipsub control messages. + pub control_msgs: Vec, } /// Event that can happen on the gossipsub behaviour. diff --git a/protocols/gossipsub/src/gossipsub_config.rs b/protocols/gossipsub/src/gossipsub_config.rs index b2dcc01caef..0cccdb482f6 100644 --- a/protocols/gossipsub/src/gossipsub_config.rs +++ b/protocols/gossipsub/src/gossipsub_config.rs @@ -1,11 +1,16 @@ +use std::borrow::Cow; use std::time::Duration; /// Configuration parameters that define the performance of the gossipsub network. #[derive(Debug, Clone)] pub struct GossipsubConfig { + /// The protocol id to negotiate this protocol. + pub protocol_id: Cow<'static, [u8]>, + /// Overlay network parameters. /// Number of heartbeats to keep in the `memcache`. pub history_length: usize, + /// Number of past heartbeats to gossip about. pub history_gossip: usize, @@ -21,19 +26,21 @@ pub struct GossipsubConfig { /// Initial delay in each heartbeat. pub heartbeat_initial_delay: Duration, + /// Time between each heartbeat. pub heartbeat_interval: Duration, + /// Time to live for fanout peers. pub fanout_ttl: Duration, + /// The maximum byte size for each gossip. - pub max_gossip_size: usize, - /// Timeout before the protocol handler terminates the stream. - pub inactivity_timeout: Duration, + pub max_transmit_size: usize, } 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, @@ -43,13 +50,15 @@ impl Default for GossipsubConfig { heartbeat_initial_delay: Duration::from_secs(5), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), - max_gossip_size: 2048, - inactivity_timeout: Duration::from_secs(60), + max_transmit_size: 2048, } } } pub struct GossipsubConfigBuilder { + /// The protocol id to negotiate this protocol. + protocol_id: Cow<'static, [u8]>, + history_length: usize, /// Number of past heartbeats to gossip about. history_gossip: usize, @@ -70,15 +79,14 @@ pub struct GossipsubConfigBuilder { heartbeat_interval: Duration, /// Time to live for fanout peers. fanout_ttl: Duration, - /// The maximum byte size for each gossip. - max_gossip_size: usize, - /// The inactivity time before a peer is disconnected. - inactivity_timeout: Duration, + /// The maximum byte size for each message. + max_transmit_size: usize, } impl Default for GossipsubConfigBuilder { fn default() -> GossipsubConfigBuilder { GossipsubConfigBuilder { + protocol_id: Cow::Borrowed(b"/meshsub/1.0.0"), history_length: 5, history_gossip: 3, mesh_n: 6, @@ -88,8 +96,7 @@ impl Default for GossipsubConfigBuilder { heartbeat_initial_delay: Duration::from_secs(5), heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), - max_gossip_size: 2048, - inactivity_timeout: Duration::from_secs(60), + max_transmit_size: 2048, } } } @@ -100,6 +107,11 @@ impl GossipsubConfigBuilder { GossipsubConfigBuilder::default() } + pub fn protocol_id(&mut self, protocol_id: impl Into>) -> &mut Self { + self.protocol_id = protocol_id.into(); + self + } + pub fn history_length(&mut self, history_length: usize) -> &mut Self { assert!( history_length >= self.history_gossip, @@ -162,18 +174,14 @@ impl GossipsubConfigBuilder { self.fanout_ttl = fanout_ttl; self } - pub fn max_gossip_size(&mut self, max_gossip_size: usize) -> &mut Self { - self.max_gossip_size = max_gossip_size; - self - } - - pub fn inactivity_timeout(&mut self, inactivity_timeout: Duration) -> &mut Self { - self.inactivity_timeout = inactivity_timeout; + pub fn max_transmit_size(&mut self, max_transmit_size: usize) -> &mut Self { + self.max_transmit_size = max_transmit_size; self } pub fn build(&self) -> GossipsubConfig { GossipsubConfig { + protocol_id: self.protocol_id.clone(), history_length: self.history_length, history_gossip: self.history_gossip, mesh_n: self.mesh_n, @@ -183,8 +191,7 @@ impl GossipsubConfigBuilder { heartbeat_initial_delay: self.heartbeat_initial_delay, heartbeat_interval: self.heartbeat_interval, fanout_ttl: self.fanout_ttl, - max_gossip_size: self.max_gossip_size, - inactivity_timeout: self.inactivity_timeout, + max_transmit_size: self.max_transmit_size, } } } diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs new file mode 100644 index 00000000000..241235816b2 --- /dev/null +++ b/protocols/gossipsub/src/handler.rs @@ -0,0 +1,327 @@ +use crate::behaviour::GossipsubRpc; +use crate::protocol::{GossipsubCodec, ProtocolConfig}; +use futures::prelude::*; +use libp2p_core::upgrade::{InboundUpgrade, Negotiated, OutboundUpgrade}; +use libp2p_swarm::protocols_handler::{ + KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, +}; + +use log::{error, trace, warn}; +use smallvec::SmallVec; +use std::borrow::Cow; +use std::io; +use tokio_codec::Framed; +use tokio_io::{AsyncRead, AsyncWrite}; + +/// Protocol Handler that manages a single long-lived substream with a peer. +pub struct GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Upgrade configuration for the gossipsub protocol. + listen_protocol: SubstreamProtocol, + + /// The single long-lived outbound substream. + outbound_substream: Option>, + + /// The single long-lived inbound substream. + inbound_substream: Option>, + + /// Queue of values that we want to send to the remote. + send_queue: SmallVec<[GossipsubRpc; 16]>, +} + +/// State of the inbound substream, opened either by us or by the remote. +enum InboundSubstreamState +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Waiting for a message from the remote. The idle state for an inbound substream. + WaitingInput(Framed, GossipsubCodec>), + /// The substream is being closed. + Closing(Framed, GossipsubCodec>), + /// An error occurred during processing. + Poisoned, +} + +/// State of the outbound substream, opened either by us or by the remote. +enum OutboundSubstreamState +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Waiting for the user to send a message. The idle state for an outbound substream. + WaitingOutput(Framed, GossipsubCodec>), + /// Waiting to send a message to the remote. + PendingSend(Framed, GossipsubCodec>, GossipsubRpc), + /// Waiting to flush the substream so that the data arrives to the remote. + PendingFlush(Framed, GossipsubCodec>), + /// The substream is being closed. Used by either substream. + _Closing(Framed, GossipsubCodec>), + /// An error occurred during processing. + Poisoned, +} + +/* +impl SubstreamState +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Consumes this state and produces the substream. + fn into_substream(self) -> Framed, GossipsubCodec> { + match self { + SubstreamState::WaitingInput(substream) => substream, + SubstreamState::PendingSend(substream, _) => substream, + SubstreamState::PendingFlush(substream) => substream, + SubstreamState::Closing(substream) => substream, + } + } +} +*/ + +impl GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + /// Builds a new `GossipsubHandler`. + pub fn new(protocol_id: impl Into>, max_transmit_size: usize) -> Self { + GossipsubHandler { + listen_protocol: SubstreamProtocol::new(ProtocolConfig::new( + protocol_id, + max_transmit_size, + )), + inbound_substream: None, + outbound_substream: None, + send_queue: SmallVec::new(), + } + } +} + +impl Default for GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + fn default() -> Self { + GossipsubHandler { + listen_protocol: SubstreamProtocol::new(ProtocolConfig::default()), + inbound_substream: None, + outbound_substream: None, + send_queue: SmallVec::new(), + } + } +} + +impl ProtocolsHandler for GossipsubHandler +where + TSubstream: AsyncRead + AsyncWrite, +{ + type InEvent = GossipsubRpc; + type OutEvent = GossipsubRpc; + type Error = io::Error; + type Substream = TSubstream; + type InboundProtocol = ProtocolConfig; + type OutboundProtocol = ProtocolConfig; + type OutboundOpenInfo = GossipsubRpc; + + #[inline] + fn listen_protocol(&self) -> SubstreamProtocol { + self.listen_protocol.clone() + } + + fn inject_fully_negotiated_inbound( + &mut self, + substream: >::Output, + ) { + // new inbound substream. Replace the current one, if it exists. + trace!("New inbound substream request"); + self.inbound_substream = Some(InboundSubstreamState::WaitingInput(substream)); + } + + fn inject_fully_negotiated_outbound( + &mut self, + substream: >::Output, + message: Self::OutboundOpenInfo, + ) { + // 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_none() { + error!("Established an outbound substream with one already available"); + return; + } + + self.outbound_substream = Some(OutboundSubstreamState::PendingSend(substream, message)); + } + + #[inline] + fn inject_event(&mut self, message: GossipsubRpc) { + self.send_queue.push(message); + } + + #[inline] + fn inject_dial_upgrade_error( + &mut self, + _: Self::OutboundOpenInfo, + _: ProtocolsHandlerUpgrErr< + >::Error, + >, + ) { + // ignore upgrade errors for now. + // If a peer doesn't support this protocol, this will just ignore them, but not disconnect + // them. + } + + #[inline] + //TODO: Implement a manual shutdown. + fn connection_keep_alive(&self) -> KeepAlive { + if self.inbound_substream.is_none() && self.outbound_substream.is_none() { + KeepAlive::No + } else { + KeepAlive::Yes + } + } + + fn poll( + &mut self, + ) -> Poll< + ProtocolsHandlerEvent, + io::Error, + > { + // determine if we need to create the stream + if !self.send_queue.is_empty() && self.outbound_substream.is_none() { + let message = self.send_queue.remove(0); + self.send_queue.shrink_to_fit(); + return Ok(Async::Ready( + ProtocolsHandlerEvent::OutboundSubstreamRequest { + protocol: self.listen_protocol.clone(), + info: message, + }, + )); + } + + loop { + match std::mem::replace( + &mut self.inbound_substream, + Some(InboundSubstreamState::Poisoned), + ) { + // inbound idle state + Some(InboundSubstreamState::WaitingInput(mut substream)) => { + match substream.poll() { + Ok(Async::Ready(Some(message))) => { + self.inbound_substream = + Some(InboundSubstreamState::WaitingInput(substream)); + return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(message))); + } + // peer closed the stream + Ok(Async::Ready(None)) => { + self.inbound_substream = + Some(InboundSubstreamState::Closing(substream)); + } + Ok(Async::NotReady) => { + self.inbound_substream = + Some(InboundSubstreamState::WaitingInput(substream)); + break; + } + Err(_) => { + self.inbound_substream = Some(InboundSubstreamState::Closing(substream)) + } + } + } + Some(InboundSubstreamState::Closing(mut substream)) => match substream.close() { + Ok(Async::Ready(())) => { + self.inbound_substream = None; + break; + } + Ok(Async::NotReady) => { + self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); + break; + } + Err(_) => { + return Err(io::Error::new( + io::ErrorKind::BrokenPipe, + "Failed to close stream", + )) + } + }, + None => { + self.inbound_substream = None; + break; + } + Some(InboundSubstreamState::Poisoned) => { + panic!("Error occurred during inbound stream processing") + } + } + } + + loop { + match std::mem::replace( + &mut self.outbound_substream, + Some(OutboundSubstreamState::Poisoned), + ) { + // outbound idle state + Some(OutboundSubstreamState::WaitingOutput(substream)) => { + if !self.send_queue.is_empty() { + let message = self.send_queue.remove(0); + self.send_queue.shrink_to_fit(); + self.outbound_substream = + Some(OutboundSubstreamState::PendingSend(substream, message)); + } else { + self.outbound_substream = + Some(OutboundSubstreamState::WaitingOutput(substream)); + break; + } + } + Some(OutboundSubstreamState::PendingSend(mut substream, message)) => { + match substream.start_send(message)? { + AsyncSink::Ready => { + self.outbound_substream = + Some(OutboundSubstreamState::PendingFlush(substream)) + } + AsyncSink::NotReady(message) => { + self.outbound_substream = + Some(OutboundSubstreamState::PendingSend(substream, message)); + break; + } + } + } + Some(OutboundSubstreamState::PendingFlush(mut substream)) => { + match substream.poll_complete()? { + Async::Ready(()) => { + self.outbound_substream = + Some(OutboundSubstreamState::WaitingOutput(substream)) + } + Async::NotReady => { + self.outbound_substream = + Some(OutboundSubstreamState::PendingFlush(substream)); + break; + } + } + } + // Currently never used - manual shutdown may implement this in the future + Some(OutboundSubstreamState::_Closing(mut substream)) => match substream.close() { + Ok(Async::Ready(())) => { + self.outbound_substream = None; + break; + } + Ok(Async::NotReady) => { + self.outbound_substream = Some(OutboundSubstreamState::_Closing(substream)); + break; + } + Err(_) => { + return Err(io::Error::new( + io::ErrorKind::BrokenPipe, + "Failed to close outbound substream", + )) + } + }, + None => { + self.outbound_substream = None; + break; + } + Some(OutboundSubstreamState::Poisoned) => { + panic!("Error occurred during outbound stream processing") + } + } + } + + Ok(Async::NotReady) + } +} diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 7161b7c0318..83821be63d6 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -109,9 +109,10 @@ pub mod protocol; mod behaviour; mod gossipsub_config; +mod handler; mod mcache; mod rpc_proto; -pub use self::behaviour::{Gossipsub, GossipsubEvent}; +pub use self::behaviour::{Gossipsub, GossipsubEvent, GossipsubRpc}; pub use self::gossipsub_config::{GossipsubConfig, GossipsubConfigBuilder}; -pub use self::protocol::{GossipsubMessage, GossipsubRpc}; +pub use self::protocol::GossipsubMessage; diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 92ec468c8f1..84ae76d7aea 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -1,40 +1,29 @@ -// Copyright 2018 Parity Technologies (UK) 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::behaviour::GossipsubRpc; use crate::rpc_proto; use byteorder::{BigEndian, ByteOrder}; +use bytes::Bytes; +use bytes::BytesMut; +use futures::future; use libp2p_core::{upgrade, InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; use libp2p_floodsub::TopicHash; -use protobuf::{Message as ProtobufMessage, ProtobufError}; +use protobuf::Message as ProtobufMessage; +use std::borrow::Cow; use std::{io, iter}; +use tokio_codec::{Decoder, Encoder, Framed}; use tokio_io::{AsyncRead, AsyncWrite}; +use unsigned_varint::codec; /// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. #[derive(Debug, Clone)] pub struct ProtocolConfig { + protocol_id: Cow<'static, [u8]>, max_transmit_size: usize, } impl Default for ProtocolConfig { fn default() -> Self { Self { + protocol_id: Cow::Borrowed(b"/meshsub/1.0.0"), max_transmit_size: 2048, } } @@ -44,186 +33,74 @@ impl ProtocolConfig { /// Builds a new `ProtocolConfig`. #[inline] /// Sets the maximum gossip transmission size. - pub fn new(max_transmit_size: usize) -> ProtocolConfig { - ProtocolConfig { max_transmit_size } + pub fn new( + protocol_id: impl Into>, + max_transmit_size: usize, + ) -> ProtocolConfig { + ProtocolConfig { + protocol_id: protocol_id.into(), + max_transmit_size, + } } } impl UpgradeInfo for ProtocolConfig { - type Info = &'static [u8]; + type Info = Cow<'static, [u8]>; type InfoIter = iter::Once; #[inline] fn protocol_info(&self) -> Self::InfoIter { - iter::once(b"/meshsub/1.0.0") + iter::once(self.protocol_id.clone()) } } impl InboundUpgrade for ProtocolConfig where - TSocket: AsyncRead, + TSocket: AsyncRead + AsyncWrite, { - type Output = GossipsubRpc; - type Error = GossipsubDecodeError; - type Future = upgrade::ReadOneThen< - upgrade::Negotiated, - (), - fn(Vec, ()) -> Result, - >; + type Output = Framed, GossipsubCodec>; + type Error = io::Error; + type Future = future::FutureResult; + #[inline] fn upgrade_inbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { - upgrade::read_one_then(socket, self.max_transmit_size, (), |packet, ()| { - let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; - - let mut messages = Vec::with_capacity(rpc.get_publish().len()); - for mut publish in rpc.take_publish().into_iter() { - messages.push(GossipsubMessage { - source: PeerId::from_bytes(publish.take_from()) - .map_err(|_| GossipsubDecodeError::InvalidPeerId)?, - data: publish.take_data(), - sequence_number: publish.take_seqno(), - topics: publish - .take_topicIDs() - .into_iter() - .map(TopicHash::from_raw) - .collect(), - }); - } - - let mut rpc_control = rpc.take_control(); - let mut control_msgs = vec![]; - // Collect the gossipsub control messages - let ihave_msgs: Vec = rpc_control - .take_ihave() - .into_iter() - .map(|mut ihave| { - GossipsubControlAction::IHave { - topic_hash: TopicHash::from_raw(ihave.take_topicID()), - // TODO: Potentially format the message ids better - message_ids: ihave.take_messageIDs().into_vec(), - } - }) - .collect(); - - let iwant_msgs: Vec = rpc_control - .take_iwant() - .into_iter() - .map(|mut iwant| { - GossipsubControlAction::IWant { - // TODO: Potentially format the message ids better - message_ids: iwant.take_messageIDs().into_vec(), - } - }) - .collect(); - - let graft_msgs: Vec = rpc_control - .take_graft() - .into_iter() - .map(|mut graft| GossipsubControlAction::Graft { - topic_hash: TopicHash::from_raw(graft.take_topicID()), - }) - .collect(); - - let prune_msgs: Vec = rpc_control - .take_prune() - .into_iter() - .map(|mut prune| GossipsubControlAction::Prune { - topic_hash: TopicHash::from_raw(prune.take_topicID()), - }) - .collect(); - - control_msgs.extend(ihave_msgs); - control_msgs.extend(iwant_msgs); - control_msgs.extend(graft_msgs); - control_msgs.extend(prune_msgs); - - Ok(GossipsubRpc { - messages, - subscriptions: rpc - .take_subscriptions() - .into_iter() - .map(|mut sub| GossipsubSubscription { - action: if sub.get_subscribe() { - GossipsubSubscriptionAction::Subscribe - } else { - GossipsubSubscriptionAction::Unsubscribe - }, - topic_hash: TopicHash::from_raw(sub.take_topicid()), - }) - .collect(), - control_msgs, - }) - }) + let mut length_codec = codec::UviBytes::default(); + length_codec.set_max_len(self.max_transmit_size); + future::ok(Framed::new(socket, GossipsubCodec { length_codec })) } } -/// Reach attempt interrupt errors. -#[derive(Debug)] -pub enum GossipsubDecodeError { - /// Error when reading the packet from the socket. - ReadError(upgrade::ReadOneError), - /// Error when decoding the raw buffer into a protobuf. - ProtobufError(ProtobufError), - /// Error when parsing the `PeerId` in the message. - InvalidPeerId, -} - -impl From for GossipsubDecodeError { - #[inline] - fn from(err: upgrade::ReadOneError) -> Self { - GossipsubDecodeError::ReadError(err) - } -} +impl OutboundUpgrade for ProtocolConfig +where + TSocket: AsyncWrite + AsyncRead, +{ + type Output = Framed, GossipsubCodec>; + type Error = io::Error; + type Future = future::FutureResult; -impl From for GossipsubDecodeError { #[inline] - fn from(err: ProtobufError) -> Self { - GossipsubDecodeError::ProtobufError(err) + fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { + let mut length_codec = codec::UviBytes::default(); + length_codec.set_max_len(self.max_transmit_size); + future::ok(Framed::new(socket, GossipsubCodec { length_codec })) } } -/// An RPC received by the gossipsub system. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct GossipsubRpc { - /// List of messages that were part of this RPC query. - pub messages: Vec, - /// List of subscriptions. - pub subscriptions: Vec, - /// List of Gossipsub control messages. - pub control_msgs: Vec, -} - -impl UpgradeInfo for GossipsubRpc { - type Info = &'static [u8]; - type InfoIter = iter::Once; +/* Gossip codec for the framing */ - #[inline] - fn protocol_info(&self) -> Self::InfoIter { - iter::once(b"/meshsub/1.0.0") - } +pub struct GossipsubCodec { + /// Codec to encode/decode the Unsigned varint length prefix of the frames. + length_codec: codec::UviBytes, } -impl OutboundUpgrade for GossipsubRpc -where - TSocket: AsyncWrite, -{ - type Output = (); +impl Encoder for GossipsubCodec { + type Item = GossipsubRpc; type Error = io::Error; - type Future = upgrade::WriteOne>; - - #[inline] - fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { - let bytes = self.into_bytes(); - upgrade::write_one(socket, bytes) - } -} -impl GossipsubRpc { - /// Turns this `GossipsubRpc` into a message that can be sent to a substream. - fn into_bytes(self) -> Vec { + fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { let mut proto = rpc_proto::RPC::new(); - for message in self.messages.into_iter() { + for message in item.messages.into_iter() { let mut msg = rpc_proto::Message::new(); msg.set_from(message.source.into_bytes()); msg.set_data(message.data); @@ -238,7 +115,7 @@ impl GossipsubRpc { proto.mut_publish().push(msg); } - for subscription in self.subscriptions.into_iter() { + for subscription in item.subscriptions.into_iter() { let mut rpc_subscription = rpc_proto::RPC_SubOpts::new(); rpc_subscription .set_subscribe(subscription.action == GossipsubSubscriptionAction::Subscribe); @@ -249,7 +126,7 @@ impl GossipsubRpc { // gossipsub control messages let mut control_msg = rpc_proto::ControlMessage::new(); - for action in self.control_msgs { + for action in item.control_msgs { match action { // collect all ihave messages GossipsubControlAction::IHave { @@ -285,9 +162,105 @@ impl GossipsubRpc { proto.set_control(control_msg); - proto + let bytes = proto .write_to_bytes() - .expect("there is no situation in which the protobuf message can be invalid") + .expect("there is no situation in which the protobuf message can be invalid"); + + // length prefix the protobuf message, ensuring the max limit is not hit + self.length_codec.encode(Bytes::from(bytes), dst) + } +} + +impl Decoder for GossipsubCodec { + type Item = GossipsubRpc; + type Error = io::Error; + + fn decode(&mut self, src: &mut BytesMut) -> Result, Self::Error> { + let packet = match self.length_codec.decode(src)? { + Some(p) => p, + None => return Ok(None), + }; + + let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; + + let mut messages = Vec::with_capacity(rpc.get_publish().len()); + for mut publish in rpc.take_publish().into_iter() { + messages.push(GossipsubMessage { + source: PeerId::from_bytes(publish.take_from()) + .map_err(|_| io::Error::new(io::ErrorKind::InvalidData, "Invalid Peer Id"))?, + data: publish.take_data(), + sequence_number: publish.take_seqno(), + topics: publish + .take_topicIDs() + .into_iter() + .map(TopicHash::from_raw) + .collect(), + }); + } + + let mut rpc_control = rpc.take_control(); + let mut control_msgs = vec![]; + // Collect the gossipsub control messages + let ihave_msgs: Vec = rpc_control + .take_ihave() + .into_iter() + .map(|mut ihave| { + GossipsubControlAction::IHave { + topic_hash: TopicHash::from_raw(ihave.take_topicID()), + // TODO: Potentially format the message ids better + message_ids: ihave.take_messageIDs().into_vec(), + } + }) + .collect(); + + let iwant_msgs: Vec = rpc_control + .take_iwant() + .into_iter() + .map(|mut iwant| { + GossipsubControlAction::IWant { + // TODO: Potentially format the message ids better + message_ids: iwant.take_messageIDs().into_vec(), + } + }) + .collect(); + + let graft_msgs: Vec = rpc_control + .take_graft() + .into_iter() + .map(|mut graft| GossipsubControlAction::Graft { + topic_hash: TopicHash::from_raw(graft.take_topicID()), + }) + .collect(); + + let prune_msgs: Vec = rpc_control + .take_prune() + .into_iter() + .map(|mut prune| GossipsubControlAction::Prune { + topic_hash: TopicHash::from_raw(prune.take_topicID()), + }) + .collect(); + + control_msgs.extend(ihave_msgs); + control_msgs.extend(iwant_msgs); + control_msgs.extend(graft_msgs); + control_msgs.extend(prune_msgs); + + Ok(Some(GossipsubRpc { + messages, + subscriptions: rpc + .take_subscriptions() + .into_iter() + .map(|mut sub| GossipsubSubscription { + action: if sub.get_subscribe() { + GossipsubSubscriptionAction::Subscribe + } else { + GossipsubSubscriptionAction::Unsubscribe + }, + topic_hash: TopicHash::from_raw(sub.take_topicid()), + }) + .collect(), + control_msgs, + })) } } From 6169fbab3b4e0afecfcd362c38563f9891f91bde Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 29 Jul 2019 23:55:35 +1000 Subject: [PATCH 63/91] Allow gossipsub topics to be optionally hashable --- protocols/gossipsub/Cargo.toml | 2 + protocols/gossipsub/src/behaviour.rs | 42 ++++++----- .../src/{gossipsub_config.rs => config.rs} | 25 ++++++- protocols/gossipsub/src/handler.rs | 22 +----- protocols/gossipsub/src/lib.rs | 6 +- protocols/gossipsub/src/mcache.rs | 2 +- protocols/gossipsub/src/protocol.rs | 2 +- protocols/gossipsub/src/topic.rs | 73 +++++++++++++++++++ 8 files changed, 132 insertions(+), 42 deletions(-) rename protocols/gossipsub/src/{gossipsub_config.rs => config.rs} (91%) create mode 100644 protocols/gossipsub/src/topic.rs diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 60f56a5350b..21390e33eab 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -23,6 +23,8 @@ tokio-io = "0.1.11" tokio-timer = "0.2.8" unsigned-varint = "0.2.2" log = "0.4.6" +sha2 = "0.8.0" +base64 = "0.10.1" [dev-dependencies] libp2p = { path = "../../" } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index a1a3368fa28..98a295c2287 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -1,13 +1,13 @@ -use crate::gossipsub_config::GossipsubConfig; +use crate::config::GossipsubConfig; use crate::handler::GossipsubHandler; use crate::mcache::MessageCache; use crate::protocol::{ GossipsubControlAction, GossipsubMessage, GossipsubSubscription, GossipsubSubscriptionAction, }; +use crate::topic::{Topic, TopicHash}; use cuckoofilter::CuckooFilter; use futures::prelude::*; use libp2p_core::{ConnectedPoint, Multiaddr, PeerId}; -use libp2p_floodsub::{Topic, TopicHash}; use libp2p_swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler}; use log::{debug, error, info, trace, warn}; use rand; @@ -94,18 +94,19 @@ impl Gossipsub { /// /// Returns true if the subscription worked. Returns false if we were already subscribed. pub fn subscribe(&mut self, topic: Topic) -> bool { - debug!("Subscribing to topic: {:?}", topic); - if self.mesh.get(&topic.hash()).is_some() { - debug!("Topic: {:?} is already in the mesh.", topic); + debug!("Subscribing to topic: {}", topic); + let topic_hash = self.topic_hash(topic.clone()); + if self.mesh.get(&topic_hash).is_some() { + debug!("Topic: {} is already in the mesh.", topic); return false; } // send subscription request to all peers in the topic - if let Some(peer_list) = self.topic_peers.get(&topic.hash()) { + if let Some(peer_list) = self.topic_peers.get(&topic_hash) { let event = Arc::new(GossipsubRpc { messages: Vec::new(), subscriptions: vec![GossipsubSubscription { - topic_hash: topic.hash().clone(), + topic_hash: topic_hash.clone(), action: GossipsubSubscriptionAction::Subscribe, }], control_msgs: Vec::new(), @@ -122,19 +123,17 @@ impl Gossipsub { // call JOIN(topic) // this will add new peers to the mesh for the topic - self.join(topic.hash()); - info!("Subscribed to topic: {:?}", topic); + self.join(&topic_hash); + info!("Subscribed to topic: {}", topic); true } /// Unsubscribes from a topic. /// - /// Note that this only requires a `TopicHash` and not a full `Topic`. - /// /// Returns true if we were subscribed to this topic. - pub fn unsubscribe(&mut self, topic: impl AsRef) -> bool { - let topic_hash = topic.as_ref(); - debug!("Unsubscribing from topic: {:?}", topic_hash); + pub fn unsubscribe(&mut self, topic: Topic) -> bool { + debug!("Unsubscribing from topic: {}", topic); + let topic_hash = &self.topic_hash(topic); if self.mesh.get(topic_hash).is_none() { debug!("Already unsubscribed to topic: {:?}", topic_hash); @@ -170,14 +169,14 @@ impl Gossipsub { } /// Publishes a message to the network. - pub fn publish(&mut self, topic: impl Into, data: impl Into>) { + pub fn publish(&mut self, topic: Topic, data: impl Into>) { self.publish_many(iter::once(topic), data) } /// Publishes a message with multiple topics to the network. pub fn publish_many( &mut self, - topic: impl IntoIterator>, + topic: impl IntoIterator, data: impl Into>, ) { let message = GossipsubMessage { @@ -190,7 +189,7 @@ impl Gossipsub { // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. sequence_number: rand::random::<[u8; 8]>().to_vec(), - topics: topic.into_iter().map(Into::into).collect(), + topics: topic.into_iter().map(|t| self.topic_hash(t)).collect(), }; debug!("Publishing message: {:?}", message.id()); @@ -873,6 +872,15 @@ impl Gossipsub { .push(control.clone()); } + /// Produces a `TopicHash` for a topic given the gossipsub configuration. + fn topic_hash(&self, topic: Topic) -> TopicHash { + if self.config.hash_topics { + topic.sha256_hash() + } else { + topic.no_hash() + } + } + // takes each control action mapping and turns it into a message fn flush_control_pool(&mut self) { for (peer, controls) in self.control_pool.drain() { diff --git a/protocols/gossipsub/src/gossipsub_config.rs b/protocols/gossipsub/src/config.rs similarity index 91% rename from protocols/gossipsub/src/gossipsub_config.rs rename to protocols/gossipsub/src/config.rs index 0cccdb482f6..e17781e95ba 100644 --- a/protocols/gossipsub/src/gossipsub_config.rs +++ b/protocols/gossipsub/src/config.rs @@ -7,7 +7,7 @@ pub struct GossipsubConfig { /// The protocol id to negotiate this protocol. pub protocol_id: Cow<'static, [u8]>, - /// Overlay network parameters. + // Overlay network parameters. /// Number of heartbeats to keep in the `memcache`. pub history_length: usize, @@ -16,8 +16,10 @@ pub struct GossipsubConfig { /// Target number of peers for the mesh network (D in the spec). pub mesh_n: usize, + /// Minimum number of peers in mesh network before adding more (D_lo in the spec). pub mesh_n_low: usize, + /// Maximum number of peers in mesh network before removing some (D_high in the spec). pub mesh_n_high: usize, @@ -35,6 +37,9 @@ pub struct GossipsubConfig { /// The maximum byte size for each gossip. pub max_transmit_size: usize, + + /// Flag determining if gossipsub topics are hashed or sent as plain strings. + pub hash_topics: bool, } impl Default for GossipsubConfig { @@ -51,6 +56,7 @@ impl Default for GossipsubConfig { heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, + hash_topics: false, // default compatibility with floodsub } } } @@ -59,14 +65,18 @@ pub struct GossipsubConfigBuilder { /// The protocol id to negotiate this protocol. protocol_id: Cow<'static, [u8]>, + /// Number of heartbeats to keep in the `memcache`. history_length: usize, + /// Number of past heartbeats to gossip about. history_gossip: usize, /// Target number of peers for the mesh network (D in the spec). mesh_n: usize, + /// Minimum number of peers in mesh network before adding more (D_lo in the spec). mesh_n_low: usize, + /// Maximum number of peers in mesh network before removing some (D_high in the spec). mesh_n_high: usize, @@ -75,12 +85,18 @@ pub struct GossipsubConfigBuilder { /// Initial delay in each heartbeat. heartbeat_initial_delay: Duration, + /// Time between each heartbeat. heartbeat_interval: Duration, + /// Time to live for fanout peers. fanout_ttl: Duration, + /// The maximum byte size for each message. max_transmit_size: usize, + + /// Flag determining if gossipsub topics are hashed or sent as plain strings. + pub hash_topics: bool, } impl Default for GossipsubConfigBuilder { @@ -97,6 +113,7 @@ impl Default for GossipsubConfigBuilder { heartbeat_interval: Duration::from_secs(1), fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, + hash_topics: false, } } } @@ -179,6 +196,11 @@ impl GossipsubConfigBuilder { self } + pub fn hash_topics(&mut self, hash_topics: bool) -> &mut Self { + self.hash_topics = hash_topics; + self + } + pub fn build(&self) -> GossipsubConfig { GossipsubConfig { protocol_id: self.protocol_id.clone(), @@ -192,6 +214,7 @@ impl GossipsubConfigBuilder { heartbeat_interval: self.heartbeat_interval, fanout_ttl: self.fanout_ttl, max_transmit_size: self.max_transmit_size, + hash_topics: self.hash_topics, } } } diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 241235816b2..49cd8946890 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -5,8 +5,7 @@ use libp2p_core::upgrade::{InboundUpgrade, Negotiated, OutboundUpgrade}; use libp2p_swarm::protocols_handler::{ KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, }; - -use log::{error, trace, warn}; +use log::{error, trace}; use smallvec::SmallVec; use std::borrow::Cow; use std::io; @@ -61,23 +60,6 @@ where Poisoned, } -/* -impl SubstreamState -where - TSubstream: AsyncRead + AsyncWrite, -{ - /// Consumes this state and produces the substream. - fn into_substream(self) -> Framed, GossipsubCodec> { - match self { - SubstreamState::WaitingInput(substream) => substream, - SubstreamState::PendingSend(substream, _) => substream, - SubstreamState::PendingFlush(substream) => substream, - SubstreamState::Closing(substream) => substream, - } - } -} -*/ - impl GossipsubHandler where TSubstream: AsyncRead + AsyncWrite, @@ -164,7 +146,7 @@ where >::Error, >, ) { - // ignore upgrade errors for now. + // Ignore upgrade errors for now. // If a peer doesn't support this protocol, this will just ignore them, but not disconnect // them. } diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 83821be63d6..609a3a730d6 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -108,11 +108,13 @@ pub mod protocol; mod behaviour; -mod gossipsub_config; +mod config; mod handler; mod mcache; mod rpc_proto; +mod topic; pub use self::behaviour::{Gossipsub, GossipsubEvent, GossipsubRpc}; -pub use self::gossipsub_config::{GossipsubConfig, GossipsubConfigBuilder}; +pub use self::config::{GossipsubConfig, GossipsubConfigBuilder}; pub use self::protocol::GossipsubMessage; +pub use self::topic::{Topic, TopicHash}; diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 6f396b3d91e..be412035ae3 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -1,7 +1,7 @@ extern crate fnv; use crate::protocol::GossipsubMessage; -use libp2p_floodsub::TopicHash; +use crate::topic::TopicHash; use std::collections::HashMap; /// CacheEntry stored in the history diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 84ae76d7aea..bf641335288 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -1,11 +1,11 @@ use crate::behaviour::GossipsubRpc; use crate::rpc_proto; +use crate::topic::TopicHash; use byteorder::{BigEndian, ByteOrder}; use bytes::Bytes; use bytes::BytesMut; use futures::future; use libp2p_core::{upgrade, InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; -use libp2p_floodsub::TopicHash; use protobuf::Message as ProtobufMessage; use std::borrow::Cow; use std::{io, iter}; diff --git a/protocols/gossipsub/src/topic.rs b/protocols/gossipsub/src/topic.rs new file mode 100644 index 00000000000..5930a5b6778 --- /dev/null +++ b/protocols/gossipsub/src/topic.rs @@ -0,0 +1,73 @@ +use crate::rpc_proto; +use base64::encode; +use protobuf::Message; +use sha2::{Digest, Sha256}; +use std::fmt; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct TopicHash { + /// The topic hash. Stored as a string to align with the protobuf API. + hash: String, +} + +impl TopicHash { + pub fn from_raw(hash: impl Into) -> TopicHash { + TopicHash { hash: hash.into() } + } + + pub fn into_string(self) -> String { + self.hash + } + + pub fn as_str(&self) -> &str { + &self.hash + } +} + +/// A gossipsub topic. +#[derive(Debug, Clone)] +pub struct Topic { + topic: String, +} + +impl Topic { + pub fn new(topic: String) -> Self { + Topic { topic } + } + + /// Creates a `TopicHash` by SHA256 hashing the topic then base64 encoding the + /// hash. + pub fn sha256_hash(&self) -> TopicHash { + let mut topic_descripter = rpc_proto::TopicDescriptor::new(); + topic_descripter.set_name(self.topic.clone()); + let hash = encode( + Sha256::digest( + &topic_descripter + .write_to_bytes() + .expect("Message is always valid"), + ) + .as_slice(), + ); + + TopicHash { hash } + } + + /// Creates a `TopicHash` as a raw string. + pub fn no_hash(&self) -> TopicHash { + TopicHash { + hash: self.topic.clone(), + } + } +} + +impl fmt::Display for Topic { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", self.topic) + } +} + +impl fmt::Display for TopicHash { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", self.hash) + } +} From 7a9de4ceb3c633f3229be245722b8ef73df8c00b Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sun, 25 Aug 2019 06:56:59 +1000 Subject: [PATCH 64/91] Improves gossipsub stream handling - Corrects the handler's keep alive. - Correct the chat example. - Instantly add peers to the mesh on subscription if the mesh is low. --- protocols/gossipsub/examples/chat.rs | 34 ++++++++++++++++++++++++++-- protocols/gossipsub/src/behaviour.rs | 20 ++++++++++++++-- protocols/gossipsub/src/handler.rs | 17 ++++++++++---- 3 files changed, 62 insertions(+), 9 deletions(-) diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index 1f16d57300c..ce078319806 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -1,3 +1,33 @@ +//! A basic chat application with logs demonstrating libp2p and the gossipsub protocol. +//! +//! Using two terminal windows, start two instances. Type a message in either terminal and hit return: the +//! message is sent and printed in the other terminal. Close with Ctrl-c. +//! +//! You can of course open more terminal windows and add more participants. +//! Dialing any of the other peers will propagate the new participant to all +//! chat members and everyone will receive all messages. +//! +//! # If they don't automatically connect +//! +//! If the nodes don't automatically connect, take note of the listening address of the first +//! instance and start the second with this address as the first argument. In the first terminal +//! window, run: +//! +//! ```sh +//! cargo run --example chat +//! ``` +//! +//! 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: +//! +//! ```sh +//! cargo run --example chat -- /ip4/127.0.0.1/tcp/24915 +//! ``` +//! +//! The two nodes then connect. + extern crate env_logger; extern crate futures; extern crate libp2p; @@ -6,7 +36,7 @@ extern crate tokio; use env_logger::{Builder, Env}; use futures::prelude::*; -use libp2p::gossipsub::GossipsubEvent; +use libp2p::gossipsub::{GossipsubEvent, Topic}; use libp2p::{ gossipsub, identity, tokio_codec::{FramedRead, LinesCodec}, @@ -26,7 +56,7 @@ fn main() { let transport = libp2p::build_development_transport(local_key); // Create a Floodsub/Gossipsub topic - let topic = libp2p::floodsub::TopicBuilder::new("test").build(); + let topic = Topic::new("test".into()); // Create a Swarm to manage peers and events let mut swarm = { diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 98a295c2287..d2bc9dbd28f 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -169,8 +169,8 @@ impl Gossipsub { } /// Publishes a message to the network. - pub fn publish(&mut self, topic: Topic, data: impl Into>) { - self.publish_many(iter::once(topic), data) + pub fn publish(&mut self, topic: &Topic, data: impl Into>) { + self.publish_many(iter::once(topic.clone()), data) } /// Publishes a message with multiple topics to the network. @@ -546,6 +546,17 @@ impl Gossipsub { ); subscribed_topics.push(subscription.topic_hash.clone()); } + + // if the mesh needs peers add the peer to the mesh + if let Some(peers) = self.mesh.get_mut(&subscription.topic_hash) { + if peers.len() < self.config.mesh_n_low { + debug!( + "SUBSCRIPTION: Adding peer {:?} to the mesh", + propagation_source, + ); + } + peers.push(propagation_source.clone()); + } // generates a subscription event to be polled self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Subscribed { @@ -569,6 +580,11 @@ impl Gossipsub { { subscribed_topics.remove(pos); } + // remove the peer from the mesh if it exists + if let Some(peers) = self.mesh.get_mut(&subscription.topic_hash) { + peers.retain(|peer| peer != propagation_source); + } + // generate a subscription even to be polled self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Unsubscribed { diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 49cd8946890..24b970fcaf6 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -28,6 +28,9 @@ where /// Queue of values that we want to send to the remote. send_queue: SmallVec<[GossipsubRpc; 16]>, + + /// Flag determining whether to maintain the connection to the peer. + keep_alive: KeepAlive, } /// State of the inbound substream, opened either by us or by the remote. @@ -74,6 +77,7 @@ where inbound_substream: None, outbound_substream: None, send_queue: SmallVec::new(), + keep_alive: KeepAlive::Yes, } } } @@ -88,6 +92,7 @@ where inbound_substream: None, outbound_substream: None, send_queue: SmallVec::new(), + keep_alive: KeepAlive::Yes, } } } @@ -154,11 +159,7 @@ where #[inline] //TODO: Implement a manual shutdown. fn connection_keep_alive(&self) -> KeepAlive { - if self.inbound_substream.is_none() && self.outbound_substream.is_none() { - KeepAlive::No - } else { - KeepAlive::Yes - } + self.keep_alive } fn poll( @@ -210,6 +211,9 @@ where Some(InboundSubstreamState::Closing(mut substream)) => match substream.close() { Ok(Async::Ready(())) => { self.inbound_substream = None; + if self.outbound_substream.is_none() { + self.keep_alive = KeepAlive::No; + } break; } Ok(Async::NotReady) => { @@ -281,6 +285,9 @@ where Some(OutboundSubstreamState::_Closing(mut substream)) => match substream.close() { Ok(Async::Ready(())) => { self.outbound_substream = None; + if self.inbound_substream.is_none() { + self.keep_alive = KeepAlive::No; + } break; } Ok(Async::NotReady) => { From 01632efeb9d1a82d1878b0fcf8fb3b8c400548f8 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 4 Sep 2019 01:21:36 +1000 Subject: [PATCH 65/91] Allows message validation in gossipsub --- protocols/gossipsub/examples/chat.rs | 10 +++--- protocols/gossipsub/src/behaviour.rs | 49 ++++++++++++++++++++-------- protocols/gossipsub/src/config.rs | 15 +++++++++ protocols/gossipsub/src/lib.rs | 25 ++++++++++---- 4 files changed, 75 insertions(+), 24 deletions(-) diff --git a/protocols/gossipsub/examples/chat.rs b/protocols/gossipsub/examples/chat.rs index ce078319806..2a8f78e3a2d 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/protocols/gossipsub/examples/chat.rs @@ -56,7 +56,7 @@ fn main() { let transport = libp2p::build_development_transport(local_key); // Create a Floodsub/Gossipsub topic - let topic = Topic::new("test".into()); + let topic = Topic::new("test-net".into()); // Create a Swarm to manage peers and events let mut swarm = { @@ -106,9 +106,11 @@ fn main() { loop { match swarm.poll().expect("Error while polling swarm") { Async::Ready(Some(gossip_event)) => match gossip_event { - GossipsubEvent::Message(message) => { - println!("Got message: {:?}", String::from_utf8_lossy(&message.data)) - } + GossipsubEvent::Message(peer_id, message) => println!( + "Got message: {:?} from peer {:?}", + String::from_utf8_lossy(&message.data), + peer_id + ), _ => {} }, Async::Ready(None) | Async::NotReady => break, diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index d2bc9dbd28f..9651654f4ba 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -196,7 +196,7 @@ impl Gossipsub { // forward the message to mesh and floodsub peers let local_peer_id = self.local_peer_id.clone(); - self.forward_msg(message.clone(), local_peer_id); + self.forward_msg(message.clone(), &local_peer_id); let mut recipient_peers = HashMap::new(); for topic_hash in &message.topics { @@ -246,6 +246,25 @@ impl Gossipsub { info!("Published message: {:?}", message.id()); } + /// This function should be called when `config.propagate_messages` is false to order to + /// propagate messages. Messages are stored in the Memcache and validation is expected to be + /// fast enough that the messages should still exist in the cache. + /// + /// Calling this function will propagate a message stored in the cache, if it still exists. + pub fn propagate_message(&mut self, message_id: &str, propagation_source: &PeerId) { + let message = match self.mcache.get(message_id) { + Some(message) => message.clone(), + None => { + warn!( + "Message not in cache. Ignoring forwarding. Message Id: {}", + message_id + ); + return; + } + }; + self.forward_msg(message, propagation_source) + } + /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. fn join(&mut self, topic_hash: &TopicHash) { debug!("Running JOIN for topic: {:?}", topic_hash); @@ -468,7 +487,7 @@ impl Gossipsub { } /// Handles a newly received GossipsubMessage. - /// Forwards the message to all floodsub peers and peers in the mesh. + /// Forwards the message to all peers in the mesh. fn handle_received_message(&mut self, msg: GossipsubMessage, propagation_source: &PeerId) { debug!( "Handling message: {:?} from peer: {:?}", @@ -491,15 +510,18 @@ impl Gossipsub { // dispatch the message to the user if self.mesh.keys().any(|t| msg.topics.iter().any(|u| t == u)) { - debug!("Sending received message to poll"); + debug!("Sending received message to user"); self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Message(msg.clone()), + GossipsubEvent::Message(propagation_source.clone(), msg.clone()), )); } - // forward the message to floodsub and mesh peers - self.forward_msg(msg.clone(), propagation_source.clone()); - debug!("Completed message handling for message: {:?}", msg.id()); + // forward the message to mesh peers, if no validation is required + if self.config.propagate_messages { + let message_id = msg.id(); + self.forward_msg(msg, propagation_source); + debug!("Completed message handling for message: {:?}", message_id); + } } /// Handles received subscriptions. @@ -814,17 +836,17 @@ impl Gossipsub { } } - /// Helper function to publish and forward messages to floodsub[topic] and mesh[topic] peers. - fn forward_msg(&mut self, message: GossipsubMessage, source: PeerId) { + /// Helper function which forwards a message to mesh[topic] peers. + fn forward_msg(&mut self, message: GossipsubMessage, source: &PeerId) { debug!("Forwarding message: {:?}", message.id()); let mut recipient_peers = HashSet::new(); - // add floodsub and mesh peers + // add mesh peers for topic in &message.topics { // mesh if let Some(mesh_peers) = self.mesh.get(&topic) { for peer_id in mesh_peers { - if *peer_id != source { + if peer_id != source { recipient_peers.insert(peer_id.clone()); } } @@ -1117,8 +1139,9 @@ pub struct GossipsubRpc { /// Event that can happen on the gossipsub behaviour. #[derive(Debug)] pub enum GossipsubEvent { - /// A message has been received. - Message(GossipsubMessage), + /// A message has been received. This contains the PeerId that we received the message from + /// and the actual message. + Message(PeerId, GossipsubMessage), /// A remote subscribed to a topic. Subscribed { diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index e17781e95ba..ed6fc14253f 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -40,6 +40,10 @@ pub struct GossipsubConfig { /// Flag determining if gossipsub topics are hashed or sent as plain strings. pub hash_topics: bool, + + /// Forward all received messages without validation. If this is set to true, the user must + /// manually call `propagate_message()` on the behaviour to forward message once validated. + pub propagate_messages: bool, } impl Default for GossipsubConfig { @@ -57,6 +61,7 @@ impl Default for GossipsubConfig { fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, hash_topics: false, // default compatibility with floodsub + propagate_messages: true, } } } @@ -97,6 +102,9 @@ pub struct GossipsubConfigBuilder { /// Flag determining if gossipsub topics are hashed or sent as plain strings. pub hash_topics: bool, + + /// Forward all received messages without validation. + pub propagate_messages: bool, } impl Default for GossipsubConfigBuilder { @@ -114,6 +122,7 @@ impl Default for GossipsubConfigBuilder { fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, hash_topics: false, + propagate_messages: true, } } } @@ -201,6 +210,11 @@ impl GossipsubConfigBuilder { self } + pub fn propagate_messages(&mut self, propagate_messages: bool) -> &mut Self { + self.propagate_messages = propagate_messages; + self + } + pub fn build(&self) -> GossipsubConfig { GossipsubConfig { protocol_id: self.protocol_id.clone(), @@ -215,6 +229,7 @@ impl GossipsubConfigBuilder { fanout_ttl: self.fanout_ttl, max_transmit_size: self.max_transmit_size, hash_topics: self.hash_topics, + propagate_messages: self.propagate_messages, } } } diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 609a3a730d6..cf447f6bf98 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -20,12 +20,14 @@ //! This section outlines the current implementation's potential discrepancies from that of other //! implementations, due to undefined elements in the current specification. //! -//! - **Topics** - In gossipsub, topics are utf-8 string's that are `base58` encoded. These are of -//! type `TopicHash`. The current go implementation uses raw utf-8 strings. - **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 integers. They are chosen at random in -//! this implementation of gossipsub, but are sequential in the current go implementation. +//! - **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 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 +//! integers. They are chosen at random in this implementation of gossipsub, but are sequential in +//! the current go implementation. //! //! # Using Gossipsub //! @@ -35,7 +37,8 @@ //! parameters. Specifically it specifies: //! //! [`GossipsubConfig`]: struct.GossipsubConfig.html - +//! +//! - `protocol_id` - The protocol id that this implementation will accept connections on. //! - `history_length` - The number of heartbeats which past messages are kept in cache (default: 5). //! - `history_gossip` - The number of past heartbeats that the node will send gossip metadata //! about (default: 3). @@ -51,6 +54,14 @@ //! - `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. +//! - `propagate_messages` - 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, propagation_source)` once validated, to +//! propagate the message to peers. //! //! This struct implements the `Default` trait and can be initialised via //! `GossipsubConfig::default()`. From 697e46a62c6af55c1dae0e78e744119b26b5e44c Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 11 Sep 2019 01:45:22 +1000 Subject: [PATCH 66/91] Replaces Cuckoofilter with LRUCache The false positive rate was unacceptable for rejecting messages. --- protocols/gossipsub/Cargo.toml | 2 +- protocols/gossipsub/src/behaviour.rs | 19 ++++++++----------- protocols/gossipsub/src/handler.rs | 2 +- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 46a9e192191..8eb44dc0cc8 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -11,7 +11,6 @@ libp2p-core = { path = "../../core" } bs58 = "0.2.2" bytes = "0.4.11" byteorder = "1.3.1" -cuckoofilter = "0.3.2" fnv = "1.0.6" futures = "0.1.25" protobuf = "2.3.0" @@ -24,6 +23,7 @@ unsigned-varint = "0.2.2" log = "0.4.6" sha2 = "0.8.0" base64 = "0.10.1" +lru = "0.1.17" [dev-dependencies] libp2p = { path = "../../" } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 9651654f4ba..4e7d24d9892 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -5,15 +5,15 @@ use crate::protocol::{ GossipsubControlAction, GossipsubMessage, GossipsubSubscription, GossipsubSubscriptionAction, }; use crate::topic::{Topic, TopicHash}; -use cuckoofilter::CuckooFilter; use futures::prelude::*; use libp2p_core::{ConnectedPoint, Multiaddr, PeerId}; use libp2p_swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler}; use log::{debug, error, info, trace, warn}; +use lru::LruCache; use rand; use rand::{seq::SliceRandom, thread_rng}; use smallvec::SmallVec; -use std::collections::hash_map::{DefaultHasher, HashMap}; +use std::collections::hash_map::HashMap; use std::collections::HashSet; use std::sync::Arc; use std::time::Instant; @@ -58,7 +58,7 @@ pub struct Gossipsub { // We keep track of the messages we received (in the format `string(source ID, seq_no)`) so that // we don't dispatch the same message twice if we receive it twice on the network. - received: CuckooFilter, + received: LruCache, /// Heartbeat interval stream. heartbeat: Interval, @@ -81,7 +81,7 @@ impl Gossipsub { fanout: HashMap::new(), fanout_last_pub: HashMap::new(), mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), - received: CuckooFilter::new(), + received: LruCache::new(256), // keep track of the last 256 messages heartbeat: Interval::new( Instant::now() + gs_config.heartbeat_initial_delay, gs_config.heartbeat_interval, @@ -228,7 +228,7 @@ impl Gossipsub { // add published message to our received caches self.mcache.put(message.clone()); - self.received.add(&message.id()); + self.received.put(message.id(), ()); let event = Arc::new(GossipsubRpc { subscriptions: Vec::new(), @@ -497,7 +497,7 @@ impl Gossipsub { // if we have seen this message, ignore it // there's a 3% chance this is a false positive // TODO: Check this has no significant emergent behaviour - if !self.received.test_and_add(&msg.id()) { + if self.received.put(msg.id(), ()).is_some() { info!( "Message already received, ignoring. Message: {:?}", msg.id() @@ -906,7 +906,7 @@ impl Gossipsub { (*self .control_pool .entry(peer.clone()) - .or_insert_with(|| Vec::new())) + .or_insert_with(Vec::new)) .push(control.clone()); } @@ -1090,10 +1090,7 @@ where event: send_event, } => match Arc::try_unwrap(send_event) { Ok(event) => { - return Async::Ready(NetworkBehaviourAction::SendEvent { - peer_id, - event: event, - }); + return Async::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }); } Err(event) => { return Async::Ready(NetworkBehaviourAction::SendEvent { diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 24b970fcaf6..674618bcb6f 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -130,7 +130,7 @@ where ) { // 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_none() { + if self.outbound_substream.is_some() { error!("Established an outbound substream with one already available"); return; } From 264cf333732b4c1df52f2df00255ef5939edd9d3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 25 Oct 2019 01:15:16 +1100 Subject: [PATCH 67/91] Renames configuration parameter and corrects logic --- protocols/gossipsub/src/behaviour.rs | 4 ++-- protocols/gossipsub/src/config.rs | 22 ++++++++++++---------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 4e7d24d9892..eaa8a29d2a0 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -246,7 +246,7 @@ impl Gossipsub { info!("Published message: {:?}", message.id()); } - /// This function should be called when `config.propagate_messages` is false to order to + /// This function should be called when `config.manual_propagation` is `true` to order to /// propagate messages. Messages are stored in the Memcache and validation is expected to be /// fast enough that the messages should still exist in the cache. /// @@ -517,7 +517,7 @@ impl Gossipsub { } // forward the message to mesh peers, if no validation is required - if self.config.propagate_messages { + if !self.config.manual_propagation { let message_id = msg.id(); self.forward_msg(msg, propagation_source); debug!("Completed message handling for message: {:?}", message_id); diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index ed6fc14253f..10bf052ab1e 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -41,9 +41,11 @@ pub struct GossipsubConfig { /// Flag determining if gossipsub topics are hashed or sent as plain strings. pub hash_topics: bool, - /// Forward all received messages without validation. If this is set to true, the user must - /// manually call `propagate_message()` on the behaviour to forward message once validated. - pub propagate_messages: bool, + /// 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 `propagate_message()` on the behaviour to forward message + /// once validated. + pub manual_propagation: bool, } impl Default for GossipsubConfig { @@ -61,7 +63,7 @@ impl Default for GossipsubConfig { fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, hash_topics: false, // default compatibility with floodsub - propagate_messages: true, + manual_propagation: false, } } } @@ -103,8 +105,8 @@ pub struct GossipsubConfigBuilder { /// Flag determining if gossipsub topics are hashed or sent as plain strings. pub hash_topics: bool, - /// Forward all received messages without validation. - pub propagate_messages: bool, + /// Manually propagate messages to peers. + pub manual_propagation: bool, } impl Default for GossipsubConfigBuilder { @@ -122,7 +124,7 @@ impl Default for GossipsubConfigBuilder { fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, hash_topics: false, - propagate_messages: true, + manual_propagation: true, } } } @@ -210,8 +212,8 @@ impl GossipsubConfigBuilder { self } - pub fn propagate_messages(&mut self, propagate_messages: bool) -> &mut Self { - self.propagate_messages = propagate_messages; + pub fn manual_propagation(&mut self, manual_propagation: bool) -> &mut Self { + self.manual_propagation = manual_propagation; self } @@ -229,7 +231,7 @@ impl GossipsubConfigBuilder { fanout_ttl: self.fanout_ttl, max_transmit_size: self.max_transmit_size, hash_topics: self.hash_topics, - propagate_messages: self.propagate_messages, + manual_propagation: self.manual_propagation, } } } From bf7ad5cdd5609f0c1f7cf316fe47f9464b04a6c2 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 29 Oct 2019 12:56:33 +1100 Subject: [PATCH 68/91] Removes peer from fanout on disconnection --- protocols/gossipsub/src/behaviour.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index eaa8a29d2a0..8948e6500f9 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -982,7 +982,7 @@ where fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { // TODO: Refactor - // remove from mesh, topic_peers and peer_topic + // remove from mesh, topic_peers, peer_topic and fanout debug!("Peer disconnected: {:?}", id); { let topics = match self.peer_topics.get(&id) { @@ -1023,6 +1023,13 @@ where &id, &topic ); } + + // remove from fanout + if let Some(fanout_peers) = self.fanout.get_mut(&topic) { + if let Some(pos) = fanout_peers.iter().position(|p| p == id) { + fanout_peers.remove(pos); + } + } } } From 6c5988fb47cedf8d7f99b03d3d0bafd40e5747b8 Mon Sep 17 00:00:00 2001 From: pawan Date: Wed, 6 Nov 2019 15:47:23 +0530 Subject: [PATCH 69/91] Add publish and fanout tests --- protocols/gossipsub/src/behaviour/tests.rs | 117 +++++++++++++++++++++ 1 file changed, 117 insertions(+) diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 53125f85a62..b8a0884e883 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -287,6 +287,123 @@ mod tests { ); } + /// Test local node publish to subscribed topic + #[test] + fn test_publish() { + // node should: + // - Send publish message to all peers + // - Insert message into gs.mcache and gs.received + + let publish_topic = String::from("test_publish"); + let (mut gs, _, topic_hashes) = + build_and_inject_nodes(20, vec![publish_topic.clone()], true); + + assert!( + gs.mesh.get(&topic_hashes[0]).is_some(), + "Subscribe should add a new entry to the mesh[topic] hashmap" + ); + + // publish on topic + let publish_data = vec![0; 42]; + gs.publish(&Topic::new(publish_topic), publish_data); + + // Collect all publish messages + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for s in &event.messages { + collected_publish.push(s.clone()); + } + collected_publish + } + _ => collected_publish, + }); + + let msg_id = publishes.first().expect("Should contain > 0 entries").id(); + + assert!( + publishes.len() == 20, + "Should send a publish message to all known peers" + ); + + assert!( + gs.mcache.get(&msg_id).is_some(), + "Message cache should contain published message" + ); + assert!( + gs.received.get(&msg_id).is_some(), + "Received cache should contain published message" + ); + } + + /// Test local node publish to unsubscribed topic + #[test] + fn test_fanout() { + // node should: + // - Populate fanout peers + // - Send publish message to fanout peers + // - Insert message into gs.mcache and gs.received + let fanout_topic = String::from("test_fanout"); + let (mut gs, _, topic_hashes) = + build_and_inject_nodes(20, vec![fanout_topic.clone()], true); + + assert!( + gs.mesh.get(&topic_hashes[0]).is_some(), + "Subscribe should add a new entry to the mesh[topic] hashmap" + ); + // Unsubscribe from topic + assert!( + gs.unsubscribe(Topic::new(fanout_topic.clone())), + "should be able to unsubscribe successfully from topic" + ); + + // Publish on unsubscribed topic + let publish_data = vec![0; 42]; + gs.publish(&Topic::new(fanout_topic.clone()), publish_data); + + assert_eq!( + gs.fanout + .get(&TopicHash::from_raw(fanout_topic.clone())) + .unwrap() + .len(), + gs.config.mesh_n, + "Fanout should contain `mesh_n` peers for fanout topic" + ); + + // Collect all publish messages + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::SendEvent { peer_id: _, event } => { + for s in &event.messages { + collected_publish.push(s.clone()); + } + collected_publish + } + _ => collected_publish, + }); + + let msg_id = publishes.first().expect("Should contain > 0 entries").id(); + + assert_eq!( + publishes.len(), + gs.config.mesh_n, + "Should send a publish message to `mesh_n` fanout peers" + ); + + assert!( + gs.mcache.get(&msg_id).is_some(), + "Message cache should contain published message" + ); + assert!( + gs.received.get(&msg_id).is_some(), + "Received cache should contain published message" + ); + } + #[test] /// Test the gossipsub NetworkBehaviour peer connection logic. fn test_inject_connected() { From 3843c04bfdb810ee1e53bad45a59608977348951 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Sat, 23 Nov 2019 18:34:15 +1100 Subject: [PATCH 70/91] Apply @mxinden suggestions --- .../chat.rs => examples/gossipsub-chat.rs | 27 +++++++-- protocols/gossipsub/regen_structs_proto.sh | 12 +--- protocols/gossipsub/src/behaviour.rs | 57 +++++++++++-------- protocols/gossipsub/src/lib.rs | 45 +++++++-------- protocols/gossipsub/src/mcache.rs | 34 ++++------- protocols/gossipsub/src/protocol.rs | 19 +++---- 6 files changed, 99 insertions(+), 95 deletions(-) rename protocols/gossipsub/examples/chat.rs => examples/gossipsub-chat.rs (77%) diff --git a/protocols/gossipsub/examples/chat.rs b/examples/gossipsub-chat.rs similarity index 77% rename from protocols/gossipsub/examples/chat.rs rename to examples/gossipsub-chat.rs index 2a8f78e3a2d..b57835a2222 100644 --- a/protocols/gossipsub/examples/chat.rs +++ b/examples/gossipsub-chat.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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 basic chat application with logs demonstrating libp2p and the gossipsub protocol. //! //! Using two terminal windows, start two instances. Type a message in either terminal and hit return: the @@ -7,9 +27,7 @@ //! Dialing any of the other peers will propagate the new participant to all //! chat members and everyone will receive all messages. //! -//! # If they don't automatically connect -//! -//! If the nodes don't automatically connect, take note of the listening address of the first +//! In order to get the nodes to connect, take note of the listening address of the first //! instance and start the second with this address as the first argument. In the first terminal //! window, run: //! @@ -26,12 +44,11 @@ //! cargo run --example chat -- /ip4/127.0.0.1/tcp/24915 //! ``` //! -//! The two nodes then connect. +//! The two nodes should then connect. extern crate env_logger; extern crate futures; extern crate libp2p; -extern crate log; extern crate tokio; use env_logger::{Builder, Env}; diff --git a/protocols/gossipsub/regen_structs_proto.sh b/protocols/gossipsub/regen_structs_proto.sh index 06f56a15d72..95a6bda2674 100755 --- a/protocols/gossipsub/regen_structs_proto.sh +++ b/protocols/gossipsub/regen_structs_proto.sh @@ -1,13 +1,3 @@ #!/bin/sh -# This script regenerates the `src/rpc_proto.rs` file from `rpc.proto`. - -docker run --rm -v `pwd`:/usr/code:z -w /usr/code rust /bin/bash -c " \ - apt-get update; \ - apt-get install -y protobuf-compiler; \ - cargo install --version 2.8.1 protobuf-codegen; \ - protoc --rust_out . rpc.proto" - -sudo chown $USER:$USER *.rs - -mv -f rpc.rs ./src/rpc_proto.rs +../../scripts/protobuf/gen.sh src/rpc.proto diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 8948e6500f9..c45d2618275 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -90,7 +90,7 @@ impl Gossipsub { } } - /// Subscribes to a topic. + /// Subscribe to a topic. /// /// Returns true if the subscription worked. Returns false if we were already subscribed. pub fn subscribe(&mut self, topic: Topic) -> bool { @@ -102,15 +102,20 @@ impl Gossipsub { } // send subscription request to all peers in the topic + let mut fixed_event = None; // initialise the event once if needed if let Some(peer_list) = self.topic_peers.get(&topic_hash) { - let event = Arc::new(GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic_hash: topic_hash.clone(), - action: GossipsubSubscriptionAction::Subscribe, - }], - control_msgs: Vec::new(), - }); + if fixed_event.is_none() { + fixed_event = Some(Arc::new(GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic_hash: topic_hash.clone(), + action: GossipsubSubscriptionAction::Subscribe, + }], + control_msgs: Vec::new(), + })); + } + + let event = fixed_event.expect("event has been initialised"); for peer in peer_list { debug!("Sending SUBSCRIBE to peer: {:?}", peer); @@ -142,15 +147,21 @@ impl Gossipsub { } // announce to all peers in the topic + let mut fixed_event = None; // initialise the event once if needed if let Some(peer_list) = self.topic_peers.get(topic_hash) { - let event = Arc::new(GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic_hash: topic_hash.clone(), - action: GossipsubSubscriptionAction::Unsubscribe, - }], - control_msgs: Vec::new(), - }); + if fixed_event.is_none() { + fixed_event = Some(Arc::new(GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic_hash: topic_hash.clone(), + action: GossipsubSubscriptionAction::Unsubscribe, + }], + control_msgs: Vec::new(), + })); + } + + let event = fixed_event.expect("event has been initialised"); + for peer in peer_list { debug!("Sending UNSUBSCRIBE to peer: {:?}", peer); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -198,7 +209,7 @@ impl Gossipsub { let local_peer_id = self.local_peer_id.clone(); self.forward_msg(message.clone(), &local_peer_id); - let mut recipient_peers = HashMap::new(); + let mut recipient_peers = HashSet::new(); for topic_hash in &message.topics { // if not subscribed to the topic, use fanout peers if self.mesh.get(&topic_hash).is_none() { @@ -207,7 +218,7 @@ impl Gossipsub { // if we have fanout peers add them to the map if self.fanout.contains_key(&topic_hash) { for peer in self.fanout.get(&topic_hash).expect("Topic must exist") { - recipient_peers.insert(peer.clone(), ()); + recipient_peers.insert(peer.clone()); } } else { // we have no fanout peers, select mesh_n of them and add them to the fanout @@ -217,7 +228,7 @@ impl Gossipsub { self.fanout.insert(topic_hash.clone(), new_peers.clone()); for peer in new_peers { debug!("Peer added to fanout: {:?}", peer); - recipient_peers.insert(peer.clone(), ()); + recipient_peers.insert(peer.clone()); } } // we are publishing to fanout peers - update the time we published @@ -236,7 +247,7 @@ impl Gossipsub { control_msgs: Vec::new(), }); // Send to peers we know are subscribed to the topic. - for peer_id in recipient_peers.keys() { + for peer_id in recipient_peers.iter() { debug!("Sending message to peer: {:?}", peer_id); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer_id.clone(), @@ -246,8 +257,8 @@ impl Gossipsub { info!("Published message: {:?}", message.id()); } - /// This function should be called when `config.manual_propagation` is `true` to order to - /// propagate messages. Messages are stored in the Memcache and validation is expected to be + /// This function should be called when `config.manual_propagation` is `true` in order to + /// propagate messages. Messages are stored in the ['Memcache'] and validation is expected to be /// fast enough that the messages should still exist in the cache. /// /// Calling this function will propagate a message stored in the cache, if it still exists. diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index cf447f6bf98..63785a4ced8 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -20,12 +20,14 @@ //! This section outlines the current implementation's potential discrepancies from that of other //! 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 is default configuration in rust-libp2p. Topics can be hashed (SHA256 hashed then base64 encoded) by setting the `hash_topics` configuration parameter to true. +//! - **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 +//! 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 +//! 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. //! @@ -42,25 +44,22 @@ //! - `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). +//! - `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. -//! - `propagate_messages` - 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, propagation_source)` once validated, to +//! - `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 @@ -69,8 +68,8 @@ //! //! ## 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 +//! 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`]. //! //! [`Gossipsub`]: struct.Gossipsub.html diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 5826ce7b31e..c1a88bd9cf4 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -4,14 +4,14 @@ use crate::protocol::GossipsubMessage; use crate::topic::TopicHash; use std::collections::HashMap; -/// CacheEntry stored in the history +/// CacheEntry stored in the history. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct CacheEntry { mid: String, topics: Vec, } -/// MessageCache struct holding history of messages +/// MessageCache struct holding history of messages. #[derive(Debug, Clone, PartialEq)] pub struct MessageCache { msgs: HashMap, @@ -19,7 +19,7 @@ pub struct MessageCache { gossip: usize, } -/// Implementation of the MessageCache +/// Implementation of the MessageCache. impl MessageCache { pub fn new(gossip: usize, history_capacity: usize) -> MessageCache { MessageCache { @@ -73,22 +73,12 @@ impl MessageCache { /// Shift the history array down one and delete messages associated with the /// last entry pub fn shift(&mut self) { - let last_index = self.history.len() - 1; - for entry in &self.history[last_index] { + for entry in self.history.pop().expect("history is always > 1") { self.msgs.remove(&entry.mid); } - // Pop the last value - self.history.pop(); - // Insert an empty vec in position 0 self.history.insert(0, Vec::new()); - - // TODO bench which one is quicker - // for i in (0..(self.history.len() - 1)).rev() { - // self.history[i+1] = self.history[i].clone(); - // } - // self.history[0] = Vec::new(); } } @@ -114,7 +104,7 @@ mod tests { } #[test] - /// Test that the message cache can be created + /// Test that the message cache can be created. fn test_new_cache() { let x: usize = 3; let mc = MessageCache::new(x, 5); @@ -123,7 +113,7 @@ mod tests { } #[test] - /// Test you can put one message and get one + /// Test you can put one message and get one. fn test_put_get_one() { let mut mc = MessageCache::new(10, 15); @@ -149,7 +139,7 @@ mod tests { } #[test] - /// Test attempting to 'get' with a wrong id + /// Test attempting to 'get' with a wrong id. fn test_get_wrong() { let mut mc = MessageCache::new(10, 15); @@ -167,7 +157,7 @@ mod tests { } #[test] - /// Test attempting to 'get' empty message cache + /// Test attempting to 'get' empty message cache. fn test_get_empty() { let mc = MessageCache::new(10, 15); @@ -178,7 +168,7 @@ mod tests { } #[test] - /// Test adding a message with no topics + /// Test adding a message with no topics. fn test_no_topic_put() { let mut mc = MessageCache::new(3, 5); @@ -196,7 +186,7 @@ mod tests { } #[test] - /// Test shift mechanism + /// Test shift mechanism. fn test_shift() { let mut mc = MessageCache::new(1, 5); @@ -220,7 +210,7 @@ mod tests { } #[test] - /// Test Shift with no additions + /// Test Shift with no additions. fn test_empty_shift() { let mut mc = MessageCache::new(1, 5); @@ -246,7 +236,7 @@ mod tests { } #[test] - /// Test shift to see if the last history messages are removed + /// Test shift to see if the last history messages are removed. fn test_remove_last_from_shift() { let mut mc = MessageCache::new(4, 5); diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index bf641335288..b578bc31c75 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -31,7 +31,6 @@ impl Default for ProtocolConfig { impl ProtocolConfig { /// Builds a new `ProtocolConfig`. - #[inline] /// Sets the maximum gossip transmission size. pub fn new( protocol_id: impl Into>, @@ -48,7 +47,6 @@ impl UpgradeInfo for ProtocolConfig { type Info = Cow<'static, [u8]>; type InfoIter = iter::Once; - #[inline] fn protocol_info(&self) -> Self::InfoIter { iter::once(self.protocol_id.clone()) } @@ -62,7 +60,6 @@ where type Error = io::Error; type Future = future::FutureResult; - #[inline] fn upgrade_inbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { let mut length_codec = codec::UviBytes::default(); length_codec.set_max_len(self.max_transmit_size); @@ -78,7 +75,6 @@ where type Error = io::Error; type Future = future::FutureResult; - #[inline] fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { let mut length_codec = codec::UviBytes::default(); length_codec.set_max_len(self.max_transmit_size); @@ -287,13 +283,14 @@ impl GossipsubMessage { // To be compatible with the go implementation pub fn id(&self) -> String { let mut source_string = self.source.to_base58(); - // the sequence number is a big endian uint64 (as per go implementation) + // the sequence number should be a big endian uint64 (as per go implementation) // avoid a potential panic by setting the seqno to 0 if it is not long enough. - // TODO: Check that this doesn't introduce a vulnerability or issue - let seqno = if self.sequence_number.len() >= 8 { - BigEndian::read_u64(&self.sequence_number) - } else { - 0 + let seqno = { + if (self.sequence_number.len() == 0) | (self.sequence_number.len() > 8) { + 0 + } else { + BigEndian::read_uint(&self.sequence_number, self.sequence_number.len()) + } }; source_string.push_str(&seqno.to_string()); source_string @@ -323,7 +320,7 @@ pub enum GossipsubSubscriptionAction { pub enum GossipsubControlAction { /// Node broadcasts known messages per topic - IHave control message. IHave { - /// The topic of the message. + /// The topic of the messages. topic_hash: TopicHash, /// A list of known message ids (peer_id + sequence _number) as a string. message_ids: Vec, From 76b6e1f53d61f01abd9f5c445a3a11bd9090b0c3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 9 Dec 2019 14:16:14 +1100 Subject: [PATCH 71/91] Resend message if outbound stream negotiated - Downgrades log warnings --- protocols/gossipsub/src/behaviour.rs | 2 +- protocols/gossipsub/src/handler.rs | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index c45d2618275..be7480b42c6 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -999,7 +999,7 @@ where let topics = match self.peer_topics.get(&id) { Some(topics) => (topics), None => { - error!("ERROR: Disconnected node, not in connected nodes"); + warn!("ERROR: Disconnected node, not in connected nodes"); return; } }; diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 674618bcb6f..ff4890b0388 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -5,7 +5,7 @@ use libp2p_core::upgrade::{InboundUpgrade, Negotiated, OutboundUpgrade}; use libp2p_swarm::protocols_handler::{ KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, }; -use log::{error, trace}; +use log::{trace, warn}; use smallvec::SmallVec; use std::borrow::Cow; use std::io; @@ -131,11 +131,12 @@ where // 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() { - error!("Established an outbound substream with one already available"); - return; + warn!("Established an outbound substream with one already available"); + // Add the message back to the send queue + self.send_queue.push(message); + } else { + self.outbound_substream = Some(OutboundSubstreamState::PendingSend(substream, message)); } - - self.outbound_substream = Some(OutboundSubstreamState::PendingSend(substream, message)); } #[inline] From 8716930567902bc9202b806cdbaaee53c335a0b3 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 9 Dec 2019 17:01:15 +1100 Subject: [PATCH 72/91] Implement further reviewer suggestions - Created associated functions to avoid unnecessary cloning - Messages are rejected if their sequence numbers are not u64 - `GossipsbuConfigBuilder` has the same defaults as `GossipsubConfig` - Miscellaneous typos --- examples/gossipsub-chat.rs | 16 +- protocols/gossipsub/Cargo.toml | 2 +- protocols/gossipsub/src/behaviour.rs | 237 +++++++++++---------- protocols/gossipsub/src/behaviour/tests.rs | 32 +-- protocols/gossipsub/src/config.rs | 127 ++++------- protocols/gossipsub/src/handler.rs | 25 ++- protocols/gossipsub/src/lib.rs | 20 ++ protocols/gossipsub/src/mcache.rs | 36 +++- protocols/gossipsub/src/protocol.rs | 41 ++-- protocols/gossipsub/src/topic.rs | 20 ++ 10 files changed, 303 insertions(+), 253 deletions(-) diff --git a/examples/gossipsub-chat.rs b/examples/gossipsub-chat.rs index b57835a2222..b982220fff2 100644 --- a/examples/gossipsub-chat.rs +++ b/examples/gossipsub-chat.rs @@ -46,11 +46,6 @@ //! //! The two nodes should then connect. -extern crate env_logger; -extern crate futures; -extern crate libp2p; -extern crate tokio; - use env_logger::{Builder, Env}; use futures::prelude::*; use libp2p::gossipsub::{GossipsubEvent, Topic}; @@ -91,8 +86,7 @@ fn main() { }; // 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(); - println!("Listening on {:?}", addr); + libp2p::Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); // Reach out to another node if specified if let Some(to_dial) = std::env::args().nth(1) { @@ -111,6 +105,7 @@ fn main() { let mut framed_stdin = FramedRead::new(stdin, LinesCodec::new()); // Kick it off + let mut listening = false; tokio::run(futures::future::poll_fn(move || -> Result<_, ()> { loop { match framed_stdin.poll().expect("Error while polling stdin") { @@ -134,6 +129,13 @@ fn main() { } } + if !listening { + for addr in libp2p::Swarm::listeners(&swarm) { + println!("Listening on {:?}", addr); + listening = true; + } + } + Ok(Async::NotReady) })); } diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 8eb44dc0cc8..2bda6be8e6d 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -15,7 +15,6 @@ fnv = "1.0.6" futures = "0.1.25" protobuf = "2.3.0" rand = "0.6.5" -smallvec = "0.6.8" tokio-codec = "0.1.1" tokio-io = "0.1.11" tokio-timer = "0.2.8" @@ -24,6 +23,7 @@ log = "0.4.6" sha2 = "0.8.0" base64 = "0.10.1" lru = "0.1.17" +smallvec = "1.0.0" [dev-dependencies] libp2p = { path = "../../" } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index be7480b42c6..f7dc7f81c2f 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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::config::GossipsubConfig; use crate::handler::GossipsubHandler; use crate::mcache::MessageCache; @@ -12,7 +32,6 @@ use log::{debug, error, info, trace, warn}; use lru::LruCache; use rand; use rand::{seq::SliceRandom, thread_rng}; -use smallvec::SmallVec; use std::collections::hash_map::HashMap; use std::collections::HashSet; use std::sync::Arc; @@ -32,7 +51,7 @@ pub struct Gossipsub { /// Events that need to be yielded to the outside when polling. events: VecDeque, GossipsubEvent>>, - /// Pools non-urgent control messages between heartbeats + /// Pools non-urgent control messages between heartbeats. control_pool: HashMap>, /// Peer id of the local node. Used for the source of the messages that we publish. @@ -42,7 +61,7 @@ pub struct Gossipsub { topic_peers: HashMap>, /// A map of all connected peers to their subscribed topics. - peer_topics: HashMap>, + peer_topics: HashMap>, /// Overlay network of connected peers - Maps topics to connected gossipsub peers. mesh: HashMap>, @@ -141,7 +160,7 @@ impl Gossipsub { let topic_hash = &self.topic_hash(topic); if self.mesh.get(topic_hash).is_none() { - debug!("Already unsubscribed to topic: {:?}", topic_hash); + debug!("Already unsubscribed from topic: {:?}", topic_hash); // we are not subscribed return false; } @@ -193,13 +212,9 @@ impl Gossipsub { let message = GossipsubMessage { source: self.local_peer_id.clone(), data: data.into(), - // If the sequence numbers are predictable, then an attacker could flood the network - // with packets with the predetermined sequence numbers and absorb our legitimate - // messages. We therefore use a random number. - // TODO: Check if the random sequence numbers causes issues with other clients. // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. - sequence_number: rand::random::<[u8; 8]>().to_vec(), + sequence_number: rand::random(), topics: topic.into_iter().map(|t| self.topic_hash(t)).collect(), }; @@ -223,7 +238,10 @@ impl Gossipsub { } else { // we have no fanout peers, select mesh_n of them and add them to the fanout let mesh_n = self.config.mesh_n; - let new_peers = self.get_random_peers(&topic_hash, mesh_n, { |_| true }); + let new_peers = + Self::get_random_peers(&self.topic_peers, &topic_hash, mesh_n, { + |_| true + }); // add the new peers to the fanout and recipient peers self.fanout.insert(topic_hash.clone(), new_peers.clone()); for peer in new_peers { @@ -312,10 +330,12 @@ impl Gossipsub { // check if we need to get more peers, which we randomly select if added_peers.len() < self.config.mesh_n { // get the peers - let new_peers = - self.get_random_peers(topic_hash, self.config.mesh_n - added_peers.len(), { - |_| true - }); + let new_peers = Self::get_random_peers( + &self.topic_peers, + topic_hash, + self.config.mesh_n - added_peers.len(), + { |_| true }, + ); added_peers.extend_from_slice(&new_peers); // add them to the mesh debug!( @@ -325,14 +345,15 @@ impl Gossipsub { let mesh_peers = self .mesh .entry(topic_hash.clone()) - .or_insert_with(|| vec![]); + .or_insert_with(|| Vec::new()); mesh_peers.extend_from_slice(&new_peers); } for peer_id in added_peers { // Send a GRAFT control message info!("JOIN: Sending Graft message to peer: {:?}", peer_id); - self.control_pool_add( + Self::control_pool_add( + &mut self.control_pool, peer_id.clone(), GossipsubControlAction::Graft { topic_hash: topic_hash.clone(), @@ -352,7 +373,8 @@ impl Gossipsub { for peer in peers { // Send a PRUNE control message info!("LEAVE: Sending PRUNE to peer: {:?}", peer); - self.control_pool_add( + Self::control_pool_add( + &mut self.control_pool, peer.clone(), GossipsubControlAction::Prune { topic_hash: topic_hash.clone(), @@ -368,17 +390,17 @@ impl Gossipsub { /// requests it with an IWANT control message. fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { debug!("Handling IHAVE for peer: {:?}", peer_id); - // use a hashmap to avoid duplicates efficiently + // use a hashset to avoid duplicates efficiently let mut iwant_ids = HashSet::new(); for (topic, ids) in ihave_msgs { // only process the message if we are subscribed if !self.mesh.contains_key(&topic) { - info!( + debug!( "IHAVE: Ignoring IHAVE - Not subscribed to topic: {:?}", topic ); - return; // continue + continue; } for id in ids { @@ -391,8 +413,9 @@ impl Gossipsub { if !iwant_ids.is_empty() { // Send the list of IWANT control messages - info!("IHAVE: Sending IWANT message"); - self.control_pool_add( + debug!("IHAVE: Sending IWANT message"); + Self::control_pool_add( + &mut self.control_pool, peer_id.clone(), GossipsubControlAction::IWant { message_ids: iwant_ids.iter().cloned().collect(), @@ -417,7 +440,7 @@ impl Gossipsub { } if !cached_messages.is_empty() { - info!("IWANT: Sending cached messages to peer: {:?}", peer_id); + debug!("IWANT: Sending cached messages to peer: {:?}", peer_id); // Send the messages to the peer let message_list = cached_messages.into_iter().map(|entry| entry.1).collect(); self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -432,8 +455,8 @@ impl Gossipsub { debug!("Completed IWANT handling for peer: {:?}", peer_id); } - /// Handles GRAFT control messages. If subscribed to the topic, adds the peer to mesh, if not, responds - /// with PRUNE messages. + /// Handles GRAFT control messages. If subscribed to the topic, adds the peer to mesh, if not, + /// responds with PRUNE messages. fn handle_graft(&mut self, peer_id: &PeerId, topics: Vec) { debug!("Handling GRAFT message for peer: {:?}", peer_id); @@ -505,9 +528,6 @@ impl Gossipsub { msg.id(), propagation_source ); - // if we have seen this message, ignore it - // there's a 3% chance this is a false positive - // TODO: Check this has no significant emergent behaviour if self.received.put(msg.id(), ()).is_some() { info!( "Message already received, ignoring. Message: {:?}", @@ -541,10 +561,9 @@ impl Gossipsub { subscriptions: &[GossipsubSubscription], propagation_source: &PeerId, ) { - trace!( + debug!( "Handling subscriptions: {:?}, from source: {:?}", - subscriptions, - propagation_source + subscriptions, propagation_source ); let subscribed_topics = match self.peer_topics.get_mut(&propagation_source) { Some(topics) => topics, @@ -618,7 +637,7 @@ impl Gossipsub { peers.retain(|peer| peer != propagation_source); } - // generate a subscription even to be polled + // generate an unsubscribe event to be polled self.events.push_back(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Unsubscribed { peer_id: propagation_source.clone(), @@ -644,7 +663,7 @@ impl Gossipsub { let mut to_prune = HashMap::new(); // maintain the mesh for each topic - for (topic_hash, peers) in self.mesh.clone().iter_mut() { + for (topic_hash, peers) in self.mesh.iter_mut() { // too little peers - add some if peers.len() < self.config.mesh_n_low { debug!( @@ -655,17 +674,17 @@ impl Gossipsub { ); // not enough peers - get mesh_n - current_length more let desired_peers = self.config.mesh_n - peers.len(); - let mut peer_list = self - .get_random_peers(topic_hash, desired_peers, { |peer| !peers.contains(peer) }); + let peer_list = + Self::get_random_peers(&self.topic_peers, topic_hash, desired_peers, { + |peer| !peers.contains(peer) + }); for peer in &peer_list { - // TODO: tagPeer let current_topic = to_graft.entry(peer.clone()).or_insert_with(|| vec![]); current_topic.push(topic_hash.clone()); } // update the mesh - peer_list.extend(peers.clone()); debug!("Updating mesh, new mesh: {:?}", peer_list); - self.mesh.insert(topic_hash.clone(), peer_list); + peers.extend(peer_list); } // too many peers - remove some @@ -687,14 +706,8 @@ impl Gossipsub { .expect("There should always be enough peers to remove"); let current_topic = to_prune.entry(peer).or_insert_with(|| vec![]); current_topic.push(topic_hash.clone()); - //TODO: untagPeer } - // update the mesh - self.mesh.insert(topic_hash.clone(), peers.clone()); } - - // emit gossip - self.emit_gossip(topic_hash.clone(), peers.clone()); } // remove expired fanout topics @@ -716,24 +729,27 @@ impl Gossipsub { // maintain fanout // check if our peers are still a part of the topic - for (topic_hash, peers) in self.fanout.clone().iter_mut() { - peers.retain(|peer| { + for (topic_hash, peers) in self.fanout.iter_mut() { + let mut to_remove_peers = Vec::new(); + for peer in peers.iter() { // is the peer still subscribed to the topic? - if let Some(topics) = self.peer_topics.get(peer) { - if !topics.contains(&topic_hash) { - debug!( - "HEARTBEAT: Peer removed from fanout for topic: {:?}", - topic_hash - ); - return false; + match self.peer_topics.get(peer) { + Some(topics) => { + if !topics.contains(&topic_hash) { + debug!( + "HEARTBEAT: Peer removed from fanout for topic: {:?}", + topic_hash + ); + to_remove_peers.push(peer.clone()); + } + } + None => { + // remove if the peer has disconnected + to_remove_peers.push(peer.clone()); } } - // remove if the peer has disconnected - else { - return false; - } - true - }); + } + peers.retain(|peer| to_remove_peers.contains(&peer)); // not enough peers if peers.len() < self.config.mesh_n { @@ -743,17 +759,16 @@ impl Gossipsub { self.config.mesh_n ); let needed_peers = self.config.mesh_n - peers.len(); - let mut new_peers = - self.get_random_peers(topic_hash, needed_peers, |peer| !peers.contains(peer)); - new_peers.extend(peers.clone()); - self.fanout.insert(topic_hash.clone(), new_peers); + let new_peers = + Self::get_random_peers(&self.topic_peers, topic_hash, needed_peers, |peer| { + !peers.contains(peer) + }); + peers.extend(new_peers); } - // update the entry - self.fanout.insert(topic_hash.clone(), peers.to_vec()); - - self.emit_gossip(topic_hash.clone(), peers.clone()); } + self.emit_gossip(); + // send graft/prunes if !to_graft.is_empty() | !to_prune.is_empty() { self.send_graft_prune(to_graft, to_prune); @@ -767,29 +782,36 @@ impl Gossipsub { debug!("Completed Heartbeat"); } - /// Emits gossip - Send IHAVE messages to a random set of gossip peers that are not in the mesh, but are subscribed to the `topic`. - fn emit_gossip(&mut self, topic_hash: TopicHash, peers: Vec) { + /// Emits gossip - Send IHAVE messages to a random set of gossip peers. This is applied to mesh + /// and fanout peers + fn emit_gossip(&mut self) { debug!("Started gossip"); - let message_ids = self.mcache.get_gossip_ids(&topic_hash); - if message_ids.is_empty() { - return; - } + for (topic_hash, peers) in self.mesh.iter().chain(self.fanout.iter()) { + let message_ids = self.mcache.get_gossip_ids(&topic_hash); + if message_ids.is_empty() { + return; + } - // get gossip_lazy random peers - let to_msg_peers = self.get_random_peers(&topic_hash, self.config.gossip_lazy, |peer| { - !peers.contains(peer) - }); - for peer in to_msg_peers { - // send an IHAVE message - self.control_pool_add( - peer.clone(), - GossipsubControlAction::IHave { - topic_hash: topic_hash.clone(), - message_ids: message_ids.clone(), - }, + // get gossip_lazy random peers + let to_msg_peers = Self::get_random_peers( + &self.topic_peers, + &topic_hash, + self.config.gossip_lazy, + |peer| !peers.contains(peer), ); + for peer in to_msg_peers { + // send an IHAVE message + Self::control_pool_add( + &mut self.control_pool, + peer.clone(), + GossipsubControlAction::IHave { + topic_hash: topic_hash.clone(), + message_ids: message_ids.clone(), + }, + ); + } + debug!("Completed gossip"); } - debug!("Completed gossip"); } /// Handles multiple GRAFT/PRUNE messages and coalesces them into chunked gossip control @@ -886,12 +908,12 @@ impl Gossipsub { /// Helper function to get a set of `n` random gossipsub peers for a `topic_hash` /// filtered by the function `f`. fn get_random_peers( - &self, + topic_peers: &HashMap>, topic_hash: &TopicHash, n: usize, mut f: impl FnMut(&PeerId) -> bool, ) -> Vec { - let mut gossip_peers = match self.topic_peers.get(topic_hash) { + let mut gossip_peers = match topic_peers.get(topic_hash) { // if they exist, filter the peers by `f` Some(peer_list) => peer_list.iter().cloned().filter(|p| f(p)).collect(), None => Vec::new(), @@ -913,12 +935,15 @@ impl Gossipsub { } // adds a control action to control_pool - fn control_pool_add(&mut self, peer: PeerId, control: GossipsubControlAction) { - (*self - .control_pool + fn control_pool_add( + control_pool: &mut HashMap>, + peer: PeerId, + control: GossipsubControlAction, + ) { + control_pool .entry(peer.clone()) - .or_insert_with(Vec::new)) - .push(control.clone()); + .or_insert_with(Vec::new) + .push(control); } /// Produces a `TopicHash` for a topic given the gossipsub configuration. @@ -930,7 +955,7 @@ impl Gossipsub { } } - // takes each control action mapping and turns it into a message + /// Takes each control action mapping and turns it into a message fn flush_control_pool(&mut self) { for (peer, controls) in self.control_pool.drain() { self.events.push_back(NetworkBehaviourAction::SendEvent { @@ -986,20 +1011,18 @@ where }); } - // TODO: Handle the peer addition - Specifically handle floodsub peers. // For the time being assume all gossipsub peers - self.peer_topics.insert(id.clone(), SmallVec::new()); + self.peer_topics.insert(id.clone(), Vec::new()); } fn inject_disconnected(&mut self, id: &PeerId, _: ConnectedPoint) { - // TODO: Refactor // remove from mesh, topic_peers, peer_topic and fanout debug!("Peer disconnected: {:?}", id); { let topics = match self.peer_topics.get(&id) { Some(topics) => (topics), None => { - warn!("ERROR: Disconnected node, not in connected nodes"); + warn!("Disconnected node, not in connected nodes"); return; } }; @@ -1011,7 +1034,6 @@ where // check if the peer is in the mesh and remove it if let Some(pos) = mesh_peers.iter().position(|p| p == id) { mesh_peers.remove(pos); - //TODO: untagPeer } } @@ -1019,28 +1041,22 @@ where if let Some(peer_list) = self.topic_peers.get_mut(&topic) { if let Some(pos) = peer_list.iter().position(|p| p == id) { peer_list.remove(pos); - //TODO: untagPeer } // debugging purposes else { - warn!( - "ERROR: Disconnected node: {:?} not in topic_peers peer list", - &id - ); + warn!("Disconnected node: {:?} not in topic_peers peer list", &id); } } else { warn!( - "ERROR: Disconnected node: {:?} with topic: {:?} not in topic_peers", + "Disconnected node: {:?} with topic: {:?} not in topic_peers", &id, &topic ); } // remove from fanout - if let Some(fanout_peers) = self.fanout.get_mut(&topic) { - if let Some(pos) = fanout_peers.iter().position(|p| p == id) { - fanout_peers.remove(pos); - } - } + self.fanout + .get_mut(&topic) + .map(|peers| peers.retain(|p| p != id)); } } @@ -1061,7 +1077,6 @@ where // Handle control messages // group some control messages, this minimises SendEvents (code is simplified to handle each event at a time however) - // TODO: Decide if the grouping is necessary let mut ihave_msgs = vec![]; let mut graft_msgs = vec![]; let mut prune_msgs = vec![]; diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index b8a0884e883..66190befca3 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -450,7 +450,7 @@ mod tests { for peer in peers { let known_topics = gs.peer_topics.get(&peer).unwrap(); assert!( - known_topics == &SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes.clone()), + known_topics == &topic_hashes, "The topics for each node should all topics" ); } @@ -497,12 +497,12 @@ mod tests { let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().clone(); assert!( - peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), + peer_topics == topic_hashes[..3].to_vec(), "First peer should be subscribed to three topics" ); let peer_topics = gs.peer_topics.get(&peers[1]).unwrap().clone(); assert!( - peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[..3].to_vec()), + peer_topics == topic_hashes[..3].to_vec(), "Second peer should be subscribed to three topics" ); @@ -531,7 +531,7 @@ mod tests { let peer_topics = gs.peer_topics.get(&peers[0]).unwrap().clone(); assert!( - peer_topics == SmallVec::<[TopicHash; 16]>::from_vec(topic_hashes[1..3].to_vec()), + peer_topics == topic_hashes[1..3].to_vec(), "Peer should be subscribed to two topics" ); @@ -559,20 +559,28 @@ mod tests { gs.topic_peers.insert(topic_hash.clone(), peers.clone()); - let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| true }); + let random_peers = + Gossipsub::::get_random_peers(&gs.topic_peers, &topic_hash, 5, { |_| true }); assert!(random_peers.len() == 5, "Expected 5 peers to be returned"); - let random_peers = gs.get_random_peers(&topic_hash, 30, { |_| true }); + let random_peers = + Gossipsub::::get_random_peers(&gs.topic_peers, &topic_hash, 30, { |_| true }); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic_hash, 20, { |_| true }); + let random_peers = + Gossipsub::::get_random_peers(&gs.topic_peers, &topic_hash, 20, { |_| true }); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!(random_peers == peers, "Expected no shuffling"); - let random_peers = gs.get_random_peers(&topic_hash, 0, { |_| true }); + let random_peers = + Gossipsub::::get_random_peers(&gs.topic_peers, &topic_hash, 0, { |_| true }); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); // test the filter - let random_peers = gs.get_random_peers(&topic_hash, 5, { |_| false }); + let random_peers = + Gossipsub::::get_random_peers(&gs.topic_peers, &topic_hash, 5, { |_| false }); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); - let random_peers = gs.get_random_peers(&topic_hash, 10, { |peer| peers.contains(peer) }); + let random_peers = + Gossipsub::::get_random_peers(&gs.topic_peers, &topic_hash, 10, { + |peer| peers.contains(peer) + }); assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); } @@ -584,7 +592,7 @@ mod tests { let message = GossipsubMessage { source: peers[11].clone(), data: vec![1, 2, 3, 4], - sequence_number: vec![0, 0, 0, 0, 0, 0, 0, 1], + sequence_number: 1u64, topics: Vec::new(), }; let msg_id = message.id(); @@ -622,7 +630,7 @@ mod tests { let message = GossipsubMessage { source: peers[11].clone(), data: vec![1, 2, 3, 4], - sequence_number: vec![0, 0, 0, 0, 0, 0, 0, shift], + sequence_number: shift, topics: Vec::new(), }; let msg_id = message.id(); diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index 10bf052ab1e..01e501f50aa 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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 std::borrow::Cow; use std::time::Duration; @@ -45,7 +65,7 @@ pub struct GossipsubConfig { /// allows a user to validate the messages before propagating them to their peers. If set to /// true, the user must manually call `propagate_message()` on the behaviour to forward message /// once validated. - pub manual_propagation: bool, + pub manual_propagation: bool, } impl Default for GossipsubConfig { @@ -69,62 +89,13 @@ impl Default for GossipsubConfig { } pub struct GossipsubConfigBuilder { - /// The protocol id to negotiate this protocol. - protocol_id: Cow<'static, [u8]>, - - /// Number of heartbeats to keep in the `memcache`. - history_length: usize, - - /// Number of past heartbeats to gossip about. - history_gossip: usize, - - /// Target number of peers for the mesh network (D in the spec). - mesh_n: usize, - - /// Minimum number of peers in mesh network before adding more (D_lo in the spec). - mesh_n_low: usize, - - /// Maximum number of peers in mesh network before removing some (D_high in the spec). - mesh_n_high: usize, - - /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). - gossip_lazy: usize, - - /// Initial delay in each heartbeat. - heartbeat_initial_delay: Duration, - - /// Time between each heartbeat. - heartbeat_interval: Duration, - - /// Time to live for fanout peers. - fanout_ttl: Duration, - - /// The maximum byte size for each message. - max_transmit_size: usize, - - /// Flag determining if gossipsub topics are hashed or sent as plain strings. - pub hash_topics: bool, - - /// Manually propagate messages to peers. - pub manual_propagation: bool, + config: GossipsubConfig, } impl Default for GossipsubConfigBuilder { fn default() -> GossipsubConfigBuilder { GossipsubConfigBuilder { - protocol_id: Cow::Borrowed(b"/meshsub/1.0.0"), - history_length: 5, - history_gossip: 3, - mesh_n: 6, - mesh_n_low: 4, - 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, - hash_topics: false, - manual_propagation: true, + config: GossipsubConfig::default(), } } } @@ -136,102 +107,88 @@ impl GossipsubConfigBuilder { } pub fn protocol_id(&mut self, protocol_id: impl Into>) -> &mut Self { - self.protocol_id = protocol_id.into(); + self.config.protocol_id = protocol_id.into(); self } pub fn history_length(&mut self, history_length: usize) -> &mut Self { assert!( - history_length >= self.history_gossip, + history_length >= self.config.history_gossip, "The history_length must be greater than or equal to the history_gossip length" ); - self.history_length = history_length; + self.config.history_length = history_length; self } pub fn history_gossip(&mut self, history_gossip: usize) -> &mut Self { assert!( - self.history_length >= history_gossip, + self.config.history_length >= history_gossip, "The history_length must be greater than or equal to the history_gossip length" ); - self.history_gossip = history_gossip; + self.config.history_gossip = history_gossip; self } pub fn mesh_n(&mut self, mesh_n: usize) -> &mut Self { assert!( - self.mesh_n_low <= mesh_n && mesh_n <= self.mesh_n_high, + 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.mesh_n = mesh_n; + self.config.mesh_n = mesh_n; self } pub fn mesh_n_low(&mut self, mesh_n_low: usize) -> &mut Self { assert!( - mesh_n_low <= self.mesh_n && self.mesh_n <= self.mesh_n_high, + 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.mesh_n_low = mesh_n_low; + self.config.mesh_n_low = mesh_n_low; self } pub fn mesh_n_high(&mut self, mesh_n_high: usize) -> &mut Self { assert!( - self.mesh_n_low <= self.mesh_n && self.mesh_n <= mesh_n_high, + 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.mesh_n_high = mesh_n_high; + self.config.mesh_n_high = mesh_n_high; self } pub fn gossip_lazy(&mut self, gossip_lazy: usize) -> &mut Self { - self.gossip_lazy = gossip_lazy; + self.config.gossip_lazy = gossip_lazy; self } pub fn heartbeat_initial_delay(&mut self, heartbeat_initial_delay: Duration) -> &mut Self { - self.heartbeat_initial_delay = heartbeat_initial_delay; + self.config.heartbeat_initial_delay = heartbeat_initial_delay; self } pub fn heartbeat_interval(&mut self, heartbeat_interval: Duration) -> &mut Self { - self.heartbeat_interval = heartbeat_interval; + self.config.heartbeat_interval = heartbeat_interval; self } pub fn fanout_ttl(&mut self, fanout_ttl: Duration) -> &mut Self { - self.fanout_ttl = fanout_ttl; + self.config.fanout_ttl = fanout_ttl; self } pub fn max_transmit_size(&mut self, max_transmit_size: usize) -> &mut Self { - self.max_transmit_size = max_transmit_size; + self.config.max_transmit_size = max_transmit_size; self } pub fn hash_topics(&mut self, hash_topics: bool) -> &mut Self { - self.hash_topics = hash_topics; + self.config.hash_topics = hash_topics; self } pub fn manual_propagation(&mut self, manual_propagation: bool) -> &mut Self { - self.manual_propagation = manual_propagation; + self.config.manual_propagation = manual_propagation; self } pub fn build(&self) -> GossipsubConfig { - GossipsubConfig { - protocol_id: self.protocol_id.clone(), - history_length: self.history_length, - history_gossip: self.history_gossip, - mesh_n: self.mesh_n, - mesh_n_low: self.mesh_n_low, - mesh_n_high: self.mesh_n_high, - gossip_lazy: self.gossip_lazy, - heartbeat_initial_delay: self.heartbeat_initial_delay, - heartbeat_interval: self.heartbeat_interval, - fanout_ttl: self.fanout_ttl, - max_transmit_size: self.max_transmit_size, - hash_topics: self.hash_topics, - manual_propagation: self.manual_propagation, - } + self.config.clone() } } diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index ff4890b0388..7498efaeeee 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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::behaviour::GossipsubRpc; use crate::protocol::{GossipsubCodec, ProtocolConfig}; use futures::prelude::*; @@ -109,7 +129,6 @@ where type OutboundProtocol = ProtocolConfig; type OutboundOpenInfo = GossipsubRpc; - #[inline] fn listen_protocol(&self) -> SubstreamProtocol { self.listen_protocol.clone() } @@ -139,12 +158,10 @@ where } } - #[inline] fn inject_event(&mut self, message: GossipsubRpc) { self.send_queue.push(message); } - #[inline] fn inject_dial_upgrade_error( &mut self, _: Self::OutboundOpenInfo, @@ -157,8 +174,6 @@ where // them. } - #[inline] - //TODO: Implement a manual shutdown. fn connection_keep_alive(&self) -> KeepAlive { self.keep_alive } diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 63785a4ced8..163f3aa192f 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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. + //! Gossipsub is a P2P pubsub (publish/subscription) routing layer designed to extend upon //! flooodsub and meshsub routing protocols. //! diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index c1a88bd9cf4..d645917279f 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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. + extern crate fnv; use crate::protocol::GossipsubMessage; @@ -88,11 +108,11 @@ mod tests { use crate::{Topic, TopicHash}; use libp2p_core::PeerId; - fn gen_testm(x: usize, topics: Vec) -> GossipsubMessage { + fn gen_testm(x: u64, topics: Vec) -> GossipsubMessage { let u8x: u8 = x as u8; let source = PeerId::random(); let data: Vec = vec![u8x]; - let sequence_number: Vec = vec![u8x]; + let sequence_number = x; let m = GossipsubMessage { source, @@ -120,7 +140,7 @@ mod tests { let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); - let m = gen_testm(10 as usize, vec![topic1_hash, topic2_hash]); + let m = gen_testm(10, vec![topic1_hash, topic2_hash]); mc.put(m.clone()); @@ -146,7 +166,7 @@ mod tests { let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); - let m = gen_testm(10 as usize, vec![topic1_hash, topic2_hash]); + let m = gen_testm(10, vec![topic1_hash, topic2_hash]); mc.put(m.clone()); @@ -173,7 +193,7 @@ mod tests { let mut mc = MessageCache::new(3, 5); // Build the message - let m = gen_testm(1 as usize, vec![]); + let m = gen_testm(1, vec![]); mc.put(m.clone()); let fetched = mc.get(&m.id()); @@ -195,7 +215,7 @@ mod tests { // Build the message for i in 0..10 { - let m = gen_testm(i as usize, vec![topic1_hash.clone(), topic2_hash.clone()]); + let m = gen_testm(i, vec![topic1_hash.clone(), topic2_hash.clone()]); mc.put(m.clone()); } @@ -218,7 +238,7 @@ mod tests { let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); // Build the message for i in 0..10 { - let m = gen_testm(i as usize, vec![topic1_hash.clone(), topic2_hash.clone()]); + let m = gen_testm(i, vec![topic1_hash.clone(), topic2_hash.clone()]); mc.put(m.clone()); } @@ -244,7 +264,7 @@ mod tests { let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); // Build the message for i in 0..10 { - let m = gen_testm(i as usize, vec![topic1_hash.clone(), topic2_hash.clone()]); + let m = gen_testm(i, vec![topic1_hash.clone(), topic2_hash.clone()]); mc.put(m.clone()); } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index b578bc31c75..047d1ece01d 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -100,7 +100,7 @@ impl Encoder for GossipsubCodec { let mut msg = rpc_proto::Message::new(); msg.set_from(message.source.into_bytes()); msg.set_data(message.data); - msg.set_seqno(message.sequence_number); + msg.set_seqno(message.sequence_number.to_be_bytes().to_vec()); msg.set_topicIDs( message .topics @@ -181,11 +181,19 @@ impl Decoder for GossipsubCodec { let mut messages = Vec::with_capacity(rpc.get_publish().len()); for mut publish in rpc.take_publish().into_iter() { + // ensure the sequence number is a u64 + let raw_seq = publish.take_seqno(); + if raw_seq.len() != 8 { + return Err(io::Error::new( + io::ErrorKind::InvalidData, + "sequence number has an incorrect size", + )); + } messages.push(GossipsubMessage { source: PeerId::from_bytes(publish.take_from()) .map_err(|_| io::Error::new(io::ErrorKind::InvalidData, "Invalid Peer Id"))?, data: publish.take_data(), - sequence_number: publish.take_seqno(), + sequence_number: BigEndian::read_u64(&raw_seq), topics: publish .take_topicIDs() .into_iter() @@ -200,23 +208,17 @@ impl Decoder for GossipsubCodec { let ihave_msgs: Vec = rpc_control .take_ihave() .into_iter() - .map(|mut ihave| { - GossipsubControlAction::IHave { - topic_hash: TopicHash::from_raw(ihave.take_topicID()), - // TODO: Potentially format the message ids better - message_ids: ihave.take_messageIDs().into_vec(), - } + .map(|mut ihave| GossipsubControlAction::IHave { + topic_hash: TopicHash::from_raw(ihave.take_topicID()), + message_ids: ihave.take_messageIDs().into_vec(), }) .collect(); let iwant_msgs: Vec = rpc_control .take_iwant() .into_iter() - .map(|mut iwant| { - GossipsubControlAction::IWant { - // TODO: Potentially format the message ids better - message_ids: iwant.take_messageIDs().into_vec(), - } + .map(|mut iwant| GossipsubControlAction::IWant { + message_ids: iwant.take_messageIDs().into_vec(), }) .collect(); @@ -270,7 +272,7 @@ pub struct GossipsubMessage { pub data: Vec, /// A random sequence number. - pub sequence_number: Vec, + pub sequence_number: u64, /// List of topics this message belongs to. /// @@ -283,16 +285,7 @@ impl GossipsubMessage { // To be compatible with the go implementation pub fn id(&self) -> String { let mut source_string = self.source.to_base58(); - // the sequence number should be a big endian uint64 (as per go implementation) - // avoid a potential panic by setting the seqno to 0 if it is not long enough. - let seqno = { - if (self.sequence_number.len() == 0) | (self.sequence_number.len() > 8) { - 0 - } else { - BigEndian::read_uint(&self.sequence_number, self.sequence_number.len()) - } - }; - source_string.push_str(&seqno.to_string()); + source_string.push_str(&self.sequence_number.to_string()); source_string } } diff --git a/protocols/gossipsub/src/topic.rs b/protocols/gossipsub/src/topic.rs index 5930a5b6778..7db5a9dd588 100644 --- a/protocols/gossipsub/src/topic.rs +++ b/protocols/gossipsub/src/topic.rs @@ -1,3 +1,23 @@ +// Copyright 2018 Parity Technologies (UK) 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::rpc_proto; use base64::encode; use protobuf::Message; From 82cf4dcfa1257e3775785e4561304da24a75cdc2 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 13 Dec 2019 15:04:15 +1100 Subject: [PATCH 73/91] Add MessageId type and remove unnecessary comments --- protocols/gossipsub/src/behaviour.rs | 19 +++++--------- protocols/gossipsub/src/behaviour/tests.rs | 17 +++++++----- protocols/gossipsub/src/mcache.rs | 16 ++++++------ protocols/gossipsub/src/protocol.rs | 30 ++++++++++++++++------ 4 files changed, 47 insertions(+), 35 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index f7dc7f81c2f..45b29ff5fda 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -23,6 +23,7 @@ use crate::handler::GossipsubHandler; use crate::mcache::MessageCache; use crate::protocol::{ GossipsubControlAction, GossipsubMessage, GossipsubSubscription, GossipsubSubscriptionAction, + MessageId, }; use crate::topic::{Topic, TopicHash}; use futures::prelude::*; @@ -77,7 +78,7 @@ pub struct Gossipsub { // We keep track of the messages we received (in the format `string(source ID, seq_no)`) so that // we don't dispatch the same message twice if we receive it twice on the network. - received: LruCache, + received: LruCache, /// Heartbeat interval stream. heartbeat: Interval, @@ -280,13 +281,13 @@ impl Gossipsub { /// fast enough that the messages should still exist in the cache. /// /// Calling this function will propagate a message stored in the cache, if it still exists. - pub fn propagate_message(&mut self, message_id: &str, propagation_source: &PeerId) { + pub fn propagate_message(&mut self, message_id: &MessageId, propagation_source: &PeerId) { let message = match self.mcache.get(message_id) { Some(message) => message.clone(), None => { warn!( "Message not in cache. Ignoring forwarding. Message Id: {}", - message_id + message_id.0 ); return; } @@ -359,7 +360,6 @@ impl Gossipsub { topic_hash: topic_hash.clone(), }, ); - //TODO: tagPeer } debug!("Completed JOIN for topic: {:?}", topic_hash); } @@ -380,7 +380,6 @@ impl Gossipsub { topic_hash: topic_hash.clone(), }, ); - //TODO: untag Peer } } debug!("Completed LEAVE for topic: {:?}", topic_hash); @@ -388,7 +387,7 @@ impl Gossipsub { /// Handles an IHAVE control message. Checks our cache of messages. If the message is unknown, /// requests it with an IWANT control message. - fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { + fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { debug!("Handling IHAVE for peer: {:?}", peer_id); // use a hashset to avoid duplicates efficiently let mut iwant_ids = HashSet::new(); @@ -427,7 +426,7 @@ impl Gossipsub { /// Handles an IWANT control message. Checks our cache of messages. If the message exists it is /// forwarded to the requesting peer. - fn handle_iwant(&mut self, peer_id: &PeerId, iwant_msgs: Vec) { + fn handle_iwant(&mut self, peer_id: &PeerId, iwant_msgs: Vec) { debug!("Handling IWANT for peer: {:?}", peer_id); // build a hashmap of available messages let mut cached_messages = HashMap::new(); @@ -472,7 +471,6 @@ impl Gossipsub { if !peers.contains(peer_id) { peers.push(peer_id.clone()); } - //TODO: tagPeer } else { to_prune_topics.insert(topic_hash.clone()); } @@ -514,7 +512,6 @@ impl Gossipsub { peer_id, topic_hash ); peers.retain(|p| p != peer_id); - //TODO: untagPeer } } debug!("Completed PRUNE handling for peer: {:?}", peer_id); @@ -653,12 +650,10 @@ impl Gossipsub { ); } - /// Heartbeat function which shifts the memcache and updates the mesh + /// Heartbeat function which shifts the memcache and updates the mesh. fn heartbeat(&mut self) { debug!("Starting heartbeat"); - //TODO: Clean up any state from last heartbeat. - let mut to_graft = HashMap::new(); let mut to_prune = HashMap::new(); diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 66190befca3..bb07b906a42 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -669,7 +669,7 @@ mod tests { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); let events_before = gs.events.len(); - gs.handle_iwant(&peers[7], vec![String::from("unknown id")]); + gs.handle_iwant(&peers[7], vec![MessageId(String::from("unknown id"))]); let events_after = gs.events.len(); assert_eq!( @@ -686,15 +686,18 @@ mod tests { gs.handle_ihave( &peers[7], - vec![(topic_hashes[0].clone(), vec![String::from("unknown id")])], + vec![( + topic_hashes[0].clone(), + vec![MessageId(String::from("unknown id"))], + )], ); // check that we sent an IWANT request for `unknown id` let iwant_exists = match gs.control_pool.get(&peers[7]) { Some(controls) => controls.iter().any(|c| match c { - GossipsubControlAction::IWant { message_ids } => { - message_ids.iter().any(|m| *m == String::from("unknown id")) - } + GossipsubControlAction::IWant { message_ids } => message_ids + .iter() + .any(|m| *m.0 == String::from("unknown id")), _ => false, }), _ => false, @@ -713,7 +716,7 @@ mod tests { let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, vec![String::from("topic1")], true); - let msg_id = String::from("known id"); + let msg_id = MessageId(String::from("known id")); gs.received.put(msg_id.clone(), ()); let events_before = gs.events.len(); @@ -737,7 +740,7 @@ mod tests { &peers[7], vec![( TopicHash::from_raw(String::from("unsubscribed topic")), - vec![String::from("irrelevant id")], + vec![MessageId(String::from("irrelevant id"))], )], ); let events_after = gs.events.len(); diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index d645917279f..9aa841d7420 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -20,21 +20,21 @@ extern crate fnv; -use crate::protocol::GossipsubMessage; +use crate::protocol::{GossipsubMessage, MessageId}; use crate::topic::TopicHash; use std::collections::HashMap; /// CacheEntry stored in the history. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct CacheEntry { - mid: String, + mid: MessageId, topics: Vec, } /// MessageCache struct holding history of messages. #[derive(Debug, Clone, PartialEq)] pub struct MessageCache { - msgs: HashMap, + msgs: HashMap, history: Vec>, gossip: usize, } @@ -63,17 +63,17 @@ impl MessageCache { } /// Get a message with `message_id` - pub fn get(&self, message_id: &str) -> Option<&GossipsubMessage> { + pub fn get(&self, message_id: &MessageId) -> Option<&GossipsubMessage> { self.msgs.get(message_id) } /// Get a list of GossipIds for a given topic - pub fn get_gossip_ids(&self, topic: &TopicHash) -> Vec { + pub fn get_gossip_ids(&self, topic: &TopicHash) -> Vec { self.history[..self.gossip] .iter() .fold(vec![], |mut current_entries, entries| { // search for entries with desired topic - let mut found_entries: Vec = entries + let mut found_entries: Vec = entries .iter() .filter_map(|entry| { if entry.topics.iter().any(|t| t == topic) { @@ -171,7 +171,7 @@ mod tests { mc.put(m.clone()); // Try to get an incorrect ID - let wrong_id = String::from("wrongid"); + let wrong_id = MessageId(String::from("wrongid")); let fetched = mc.get(&wrong_id); assert_eq!(fetched.is_none(), true); } @@ -182,7 +182,7 @@ mod tests { let mc = MessageCache::new(10, 15); // Try to get an incorrect ID - let wrong_string = String::from("imempty"); + let wrong_string = MessageId(String::from("imempty")); let fetched = mc.get(&wrong_string); assert_eq!(fetched.is_none(), true); } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 047d1ece01d..556d25e276e 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -132,14 +132,14 @@ impl Encoder for GossipsubCodec { let mut rpc_ihave = rpc_proto::ControlIHave::new(); rpc_ihave.set_topicID(topic_hash.into_string()); for msg_id in message_ids { - rpc_ihave.mut_messageIDs().push(msg_id); + rpc_ihave.mut_messageIDs().push(msg_id.0); } control_msg.mut_ihave().push(rpc_ihave); } GossipsubControlAction::IWant { message_ids } => { let mut rpc_iwant = rpc_proto::ControlIWant::new(); for msg_id in message_ids { - rpc_iwant.mut_messageIDs().push(msg_id); + rpc_iwant.mut_messageIDs().push(msg_id.0); } control_msg.mut_iwant().push(rpc_iwant); } @@ -210,7 +210,12 @@ impl Decoder for GossipsubCodec { .into_iter() .map(|mut ihave| GossipsubControlAction::IHave { topic_hash: TopicHash::from_raw(ihave.take_topicID()), - message_ids: ihave.take_messageIDs().into_vec(), + message_ids: ihave + .take_messageIDs() + .into_vec() + .into_iter() + .map(|x| MessageId(x)) + .collect::>(), }) .collect(); @@ -218,7 +223,12 @@ impl Decoder for GossipsubCodec { .take_iwant() .into_iter() .map(|mut iwant| GossipsubControlAction::IWant { - message_ids: iwant.take_messageIDs().into_vec(), + message_ids: iwant + .take_messageIDs() + .into_vec() + .into_iter() + .map(|x| MessageId(x)) + .collect::>(), }) .collect(); @@ -262,6 +272,10 @@ impl Decoder for GossipsubCodec { } } +/// A type for gossipsub message ids. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct MessageId(pub String); + /// A message received by the gossipsub system. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct GossipsubMessage { @@ -283,10 +297,10 @@ pub struct GossipsubMessage { impl GossipsubMessage { /// Converts message into a message_id. // To be compatible with the go implementation - pub fn id(&self) -> String { + pub fn id(&self) -> MessageId { let mut source_string = self.source.to_base58(); source_string.push_str(&self.sequence_number.to_string()); - source_string + MessageId(source_string) } } @@ -316,12 +330,12 @@ pub enum GossipsubControlAction { /// The topic of the messages. topic_hash: TopicHash, /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, + message_ids: Vec, }, /// The node requests specific message ids (peer_id + sequence _number) - IWant control message. IWant { /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, + message_ids: Vec, }, /// The node has been added to the mesh - Graft control message. Graft { From 9d69802c485537d679e6be26b616dc58cba576b0 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 13 Dec 2019 15:08:54 +1100 Subject: [PATCH 74/91] Add a return value to propagate_message function --- protocols/gossipsub/src/behaviour.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 45b29ff5fda..83231cbd783 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -281,7 +281,13 @@ impl Gossipsub { /// fast enough that the messages should still exist in the cache. /// /// Calling this function will propagate a message stored in the cache, if it still exists. - pub fn propagate_message(&mut self, message_id: &MessageId, propagation_source: &PeerId) { + /// If the message still exists in the cache, it will be forwarded and this function will return true, + /// otherwise it will return false. + pub fn propagate_message( + &mut self, + message_id: &MessageId, + propagation_source: &PeerId, + ) -> bool { let message = match self.mcache.get(message_id) { Some(message) => message.clone(), None => { @@ -289,10 +295,11 @@ impl Gossipsub { "Message not in cache. Ignoring forwarding. Message Id: {}", message_id.0 ); - return; + return false; } }; - self.forward_msg(message, propagation_source) + self.forward_msg(message, propagation_source); + true } /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. From 12226bcfa665b73233b3319452e06ca6fa5f6a5f Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 16 Dec 2019 17:14:57 +1100 Subject: [PATCH 75/91] Adds user-customised gossipsub message ids --- examples/gossipsub-chat.rs | 16 ++++++- protocols/gossipsub/src/behaviour.rs | 37 +++++++++------ protocols/gossipsub/src/behaviour/tests.rs | 17 ++++--- protocols/gossipsub/src/config.rs | 24 +++++++++- protocols/gossipsub/src/mcache.rs | 55 +++++++++++++++++----- protocols/gossipsub/src/protocol.rs | 10 ---- 6 files changed, 112 insertions(+), 47 deletions(-) diff --git a/examples/gossipsub-chat.rs b/examples/gossipsub-chat.rs index b982220fff2..de4d26833ef 100644 --- a/examples/gossipsub-chat.rs +++ b/examples/gossipsub-chat.rs @@ -48,16 +48,19 @@ use env_logger::{Builder, Env}; use futures::prelude::*; -use libp2p::gossipsub::{GossipsubEvent, Topic}; +use libp2p::gossipsub::protocol::MessageId; +use libp2p::gossipsub::{GossipsubEvent, GossipsubMessage, Topic}; use libp2p::{ gossipsub, identity, tokio_codec::{FramedRead, LinesCodec}, PeerId, }; +use std::collections::hash_map::DefaultHasher; +use std::hash::{Hash, Hasher}; use std::time::Duration; fn main() { - Builder::from_env(Env::default().default_filter_or("debug")).init(); + Builder::from_env(Env::default().default_filter_or("info")).init(); // Create a random PeerId let local_key = identity::Keypair::generate_ed25519(); @@ -75,9 +78,18 @@ fn main() { // 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(); + message.data.hash(&mut s); + MessageId(s.finish().to_string()) + }; + // set custom gossipsub let gossipsub_config = gossipsub::GossipsubConfigBuilder::new() .heartbeat_interval(Duration::from_secs(10)) + .message_id_fn(message_id_fn) // content-address messages. No two messages of the + //same content will be propagated. .build(); // build a gossipsub network behaviour let mut gossipsub = gossipsub::Gossipsub::new(local_peer_id.clone(), gossipsub_config); diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 83231cbd783..4e6a0ecd0d6 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -100,7 +100,11 @@ impl Gossipsub { mesh: HashMap::new(), fanout: HashMap::new(), fanout_last_pub: HashMap::new(), - mcache: MessageCache::new(gs_config.history_gossip, gs_config.history_length), + mcache: MessageCache::new( + gs_config.history_gossip, + gs_config.history_length, + gs_config.message_id_fn, + ), received: LruCache::new(256), // keep track of the last 256 messages heartbeat: Interval::new( Instant::now() + gs_config.heartbeat_initial_delay, @@ -219,7 +223,10 @@ impl Gossipsub { topics: topic.into_iter().map(|t| self.topic_hash(t)).collect(), }; - debug!("Publishing message: {:?}", message.id()); + debug!( + "Publishing message: {:?}", + (self.config.message_id_fn)(&message) + ); // forward the message to mesh and floodsub peers let local_peer_id = self.local_peer_id.clone(); @@ -257,12 +264,15 @@ impl Gossipsub { } // add published message to our received caches + let msg_id = (self.config.message_id_fn)(&message); self.mcache.put(message.clone()); - self.received.put(message.id(), ()); + self.received.put(msg_id.clone(), ()); + + info!("Published message: {:?}", msg_id); let event = Arc::new(GossipsubRpc { subscriptions: Vec::new(), - messages: vec![message.clone()], + messages: vec![message], control_msgs: Vec::new(), }); // Send to peers we know are subscribed to the topic. @@ -273,7 +283,6 @@ impl Gossipsub { event: event.clone(), }); } - info!("Published message: {:?}", message.id()); } /// This function should be called when `config.manual_propagation` is `true` in order to @@ -527,16 +536,13 @@ impl Gossipsub { /// Handles a newly received GossipsubMessage. /// Forwards the message to all peers in the mesh. fn handle_received_message(&mut self, msg: GossipsubMessage, propagation_source: &PeerId) { + let msg_id = (self.config.message_id_fn)(&msg); debug!( "Handling message: {:?} from peer: {:?}", - msg.id(), - propagation_source + msg_id, propagation_source ); - if self.received.put(msg.id(), ()).is_some() { - info!( - "Message already received, ignoring. Message: {:?}", - msg.id() - ); + if self.received.put(msg_id.clone(), ()).is_some() { + debug!("Message already received, ignoring. Message: {:?}", msg_id); return; } @@ -553,7 +559,7 @@ impl Gossipsub { // forward the message to mesh peers, if no validation is required if !self.config.manual_propagation { - let message_id = msg.id(); + let message_id = (self.config.message_id_fn)(&msg); self.forward_msg(msg, propagation_source); debug!("Completed message handling for message: {:?}", message_id); } @@ -873,7 +879,8 @@ impl Gossipsub { /// Helper function which forwards a message to mesh[topic] peers. fn forward_msg(&mut self, message: GossipsubMessage, source: &PeerId) { - debug!("Forwarding message: {:?}", message.id()); + let msg_id = (self.config.message_id_fn)(&message); + debug!("Forwarding message: {:?}", msg_id); let mut recipient_peers = HashSet::new(); // add mesh peers @@ -897,7 +904,7 @@ impl Gossipsub { }); for peer in recipient_peers.iter() { - debug!("Sending message: {:?} to peer {:?}", message.id(), peer); + debug!("Sending message: {:?} to peer {:?}", msg_id, peer); self.events.push_back(NetworkBehaviourAction::SendEvent { peer_id: peer.clone(), event: event.clone(), diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index bb07b906a42..92fbed96b53 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -321,7 +321,8 @@ mod tests { _ => collected_publish, }); - let msg_id = publishes.first().expect("Should contain > 0 entries").id(); + let msg_id = + (gs.config.message_id_fn)(&publishes.first().expect("Should contain > 0 entries")); assert!( publishes.len() == 20, @@ -386,7 +387,8 @@ mod tests { _ => collected_publish, }); - let msg_id = publishes.first().expect("Should contain > 0 entries").id(); + let msg_id = + (gs.config.message_id_fn)(&publishes.first().expect("Should contain > 0 entries")); assert_eq!( publishes.len(), @@ -589,13 +591,15 @@ mod tests { fn test_handle_iwant_msg_cached() { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + let id = gs.config.message_id_fn; + let message = GossipsubMessage { source: peers[11].clone(), data: vec![1, 2, 3, 4], sequence_number: 1u64, topics: Vec::new(), }; - let msg_id = message.id(); + let msg_id = id(&message); gs.mcache.put(message.clone()); gs.handle_iwant(&peers[7], vec![msg_id.clone()]); @@ -615,7 +619,7 @@ mod tests { }); assert!( - sent_messages.iter().any(|msg| msg.id() == msg_id), + sent_messages.iter().any(|msg| id(msg) == msg_id), "Expected the cached message to be sent to an IWANT peer" ); } @@ -625,6 +629,7 @@ mod tests { fn test_handle_iwant_msg_cached_shifted() { let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + let id = gs.config.message_id_fn; // perform 10 memshifts and check that it leaves the cache for shift in 1..10 { let message = GossipsubMessage { @@ -633,7 +638,7 @@ mod tests { sequence_number: shift, topics: Vec::new(), }; - let msg_id = message.id(); + let msg_id = id(&message); gs.mcache.put(message.clone()); for _ in 0..shift { gs.mcache.shift(); @@ -644,7 +649,7 @@ mod tests { // is the message is being sent? let message_exists = gs.events.iter().any(|e| match e { NetworkBehaviourAction::SendEvent { peer_id: _, event } => { - event.messages.iter().any(|msg| msg.id() == msg_id) + event.messages.iter().any(|msg| id(msg) == msg_id) } _ => false, }); diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index 01e501f50aa..99ae99d45f3 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -18,11 +18,12 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use crate::protocol::{GossipsubMessage, MessageId}; use std::borrow::Cow; use std::time::Duration; /// Configuration parameters that define the performance of the gossipsub network. -#[derive(Debug, Clone)] +#[derive(Clone)] pub struct GossipsubConfig { /// The protocol id to negotiate this protocol. pub protocol_id: Cow<'static, [u8]>, @@ -66,6 +67,16 @@ pub struct GossipsubConfig { /// true, the user must manually call `propagate_message()` on the behaviour to forward message /// once validated. pub manual_propagation: bool, + + /// 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 + /// parameter allows the user to address packets arbitrarily. One example is content based + /// 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. + pub message_id_fn: fn(&GossipsubMessage) -> MessageId, } impl Default for GossipsubConfig { @@ -84,6 +95,12 @@ impl Default for GossipsubConfig { max_transmit_size: 2048, hash_topics: false, // default compatibility with floodsub manual_propagation: false, + message_id_fn: |message| { + // default message id is: source + sequence number + let mut source_string = message.source.to_base58(); + source_string.push_str(&message.sequence_number.to_string()); + MessageId(source_string) + }, } } } @@ -188,6 +205,11 @@ impl GossipsubConfigBuilder { self } + pub fn message_id_fn(&mut self, id_fn: fn(&GossipsubMessage) -> MessageId) -> &mut Self { + self.config.message_id_fn = id_fn; + self + } + pub fn build(&self) -> GossipsubConfig { self.config.clone() } diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 9aa841d7420..b8d45755d78 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -32,26 +32,49 @@ pub struct CacheEntry { } /// MessageCache struct holding history of messages. -#[derive(Debug, Clone, PartialEq)] +#[derive(Clone)] pub struct MessageCache { msgs: HashMap, history: Vec>, gossip: usize, + msg_id: fn(&GossipsubMessage) -> MessageId, } /// Implementation of the MessageCache. impl MessageCache { - pub fn new(gossip: usize, history_capacity: usize) -> MessageCache { + pub fn new( + gossip: usize, + history_capacity: usize, + msg_id: fn(&GossipsubMessage) -> MessageId, + ) -> MessageCache { MessageCache { gossip, msgs: HashMap::default(), history: vec![Vec::new(); history_capacity], + msg_id, + } + } + + /// Creates a `MessageCache` with a default message id function. + #[allow(dead_code)] + pub fn new_default(gossip: usize, history_capacity: usize) -> MessageCache { + let default_id = |message: &GossipsubMessage| { + // default message id is: source + sequence number + let mut source_string = message.source.to_base58(); + source_string.push_str(&message.sequence_number.to_string()); + MessageId(source_string) + }; + MessageCache { + gossip, + msgs: HashMap::default(), + history: vec![Vec::new(); history_capacity], + msg_id: default_id, } } /// Put a message into the memory cache pub fn put(&mut self, msg: GossipsubMessage) { - let message_id = msg.id(); + let message_id = (self.msg_id)(&msg); let cache_entry = CacheEntry { mid: message_id.clone(), topics: msg.topics.clone(), @@ -126,8 +149,14 @@ mod tests { #[test] /// Test that the message cache can be created. fn test_new_cache() { + let default_id = |message: &GossipsubMessage| { + // default message id is: source + sequence number + let mut source_string = message.source.to_base58(); + source_string.push_str(&message.sequence_number.to_string()); + MessageId(source_string) + }; let x: usize = 3; - let mc = MessageCache::new(x, 5); + let mc = MessageCache::new(x, 5, default_id); assert_eq!(mc.gossip, x); } @@ -135,7 +164,7 @@ mod tests { #[test] /// Test you can put one message and get one. fn test_put_get_one() { - let mut mc = MessageCache::new(10, 15); + let mut mc = MessageCache::new_default(10, 15); let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); @@ -146,7 +175,7 @@ mod tests { assert!(mc.history[0].len() == 1); - let fetched = mc.get(&m.id()); + let fetched = mc.get(&(mc.msg_id)(&m)); assert_eq!(fetched.is_none(), false); assert_eq!(fetched.is_some(), true); @@ -161,7 +190,7 @@ mod tests { #[test] /// Test attempting to 'get' with a wrong id. fn test_get_wrong() { - let mut mc = MessageCache::new(10, 15); + let mut mc = MessageCache::new_default(10, 15); let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); @@ -179,7 +208,7 @@ mod tests { #[test] /// Test attempting to 'get' empty message cache. fn test_get_empty() { - let mc = MessageCache::new(10, 15); + let mc = MessageCache::new_default(10, 15); // Try to get an incorrect ID let wrong_string = MessageId(String::from("imempty")); @@ -190,13 +219,13 @@ mod tests { #[test] /// Test adding a message with no topics. fn test_no_topic_put() { - let mut mc = MessageCache::new(3, 5); + let mut mc = MessageCache::new_default(3, 5); // Build the message let m = gen_testm(1, vec![]); mc.put(m.clone()); - let fetched = mc.get(&m.id()); + let fetched = mc.get(&(mc.msg_id)(&m)); // Make sure it is the same fetched message match fetched { @@ -208,7 +237,7 @@ mod tests { #[test] /// Test shift mechanism. fn test_shift() { - let mut mc = MessageCache::new(1, 5); + let mut mc = MessageCache::new_default(1, 5); let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); @@ -232,7 +261,7 @@ mod tests { #[test] /// Test Shift with no additions. fn test_empty_shift() { - let mut mc = MessageCache::new(1, 5); + let mut mc = MessageCache::new_default(1, 5); let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); @@ -258,7 +287,7 @@ mod tests { #[test] /// Test shift to see if the last history messages are removed. fn test_remove_last_from_shift() { - let mut mc = MessageCache::new(4, 5); + let mut mc = MessageCache::new_default(4, 5); let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 556d25e276e..49249dfbb6f 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -294,16 +294,6 @@ pub struct GossipsubMessage { pub topics: Vec, } -impl GossipsubMessage { - /// Converts message into a message_id. - // To be compatible with the go implementation - pub fn id(&self) -> MessageId { - let mut source_string = self.source.to_base58(); - source_string.push_str(&self.sequence_number.to_string()); - MessageId(source_string) - } -} - /// A subscription received by the gossipsub system. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct GossipsubSubscription { From 3af999aeb10ed8c1afdd5b9a18e1f1e92c21e39e Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 16 Dec 2019 18:18:46 +1100 Subject: [PATCH 76/91] Adds the message id to GossipsubEvent --- examples/gossipsub-chat.rs | 5 +++-- protocols/gossipsub/src/behaviour.rs | 9 +++++---- protocols/gossipsub/src/lib.rs | 2 +- protocols/gossipsub/src/protocol.rs | 12 ++++++++++++ 4 files changed, 21 insertions(+), 7 deletions(-) diff --git a/examples/gossipsub-chat.rs b/examples/gossipsub-chat.rs index de4d26833ef..c6ef5f24bfd 100644 --- a/examples/gossipsub-chat.rs +++ b/examples/gossipsub-chat.rs @@ -130,9 +130,10 @@ fn main() { loop { match swarm.poll().expect("Error while polling swarm") { Async::Ready(Some(gossip_event)) => match gossip_event { - GossipsubEvent::Message(peer_id, message) => println!( - "Got message: {:?} from peer {:?}", + GossipsubEvent::Message(peer_id, id, message) => println!( + "Got message: {} with id: {} from peer: {:?}", String::from_utf8_lossy(&message.data), + id, peer_id ), _ => {} diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 4e6a0ecd0d6..c8942fa7877 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -553,7 +553,7 @@ impl Gossipsub { if self.mesh.keys().any(|t| msg.topics.iter().any(|u| t == u)) { debug!("Sending received message to user"); self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Message(propagation_source.clone(), msg.clone()), + GossipsubEvent::Message(propagation_source.clone(), msg_id, msg.clone()), )); } @@ -1178,9 +1178,10 @@ pub struct GossipsubRpc { /// Event that can happen on the gossipsub behaviour. #[derive(Debug)] pub enum GossipsubEvent { - /// A message has been received. This contains the PeerId that we received the message from - /// and the actual message. - Message(PeerId, GossipsubMessage), + /// A message has been received. This contains the PeerId that we received the message from, + /// the message id (used if the application layer needs to propagate the message) and the + /// message itself. + Message(PeerId, MessageId, GossipsubMessage), /// A remote subscribed to a topic. Subscribed { diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 163f3aa192f..5586bf563a8 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -146,5 +146,5 @@ mod topic; pub use self::behaviour::{Gossipsub, GossipsubEvent, GossipsubRpc}; pub use self::config::{GossipsubConfig, GossipsubConfigBuilder}; -pub use self::protocol::GossipsubMessage; +pub use self::protocol::{GossipsubMessage, MessageId}; pub use self::topic::{Topic, TopicHash}; diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 49249dfbb6f..00f94689fec 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -276,6 +276,18 @@ impl Decoder for GossipsubCodec { #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct MessageId(pub String); +impl std::fmt::Display for MessageId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +impl Into for MessageId { + fn into(self) -> String { + self.0.into() + } +} + /// A message received by the gossipsub system. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct GossipsubMessage { From 0bdf093e69b19d959b6690de225b525fa5920ae7 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Mon, 16 Dec 2019 23:16:09 +1100 Subject: [PATCH 77/91] Implement Debug for GossipsubConfig --- protocols/gossipsub/src/config.rs | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index 99ae99d45f3..989c3297278 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -214,3 +214,23 @@ impl GossipsubConfigBuilder { 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 _ = builder.field("protocol_id", &self.protocol_id); + 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("gossip_lazy", &self.gossip_lazy); + 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("hash_topics", &self.hash_topics); + let _ = builder.field("manual_propagation", &self.manual_propagation); + builder.finish() + } +} From 6ac92bcadc8636fb0148451d4c8053c4c022dcaa Mon Sep 17 00:00:00 2001 From: Max Inden Date: Mon, 23 Dec 2019 16:38:12 +0100 Subject: [PATCH 78/91] protocols/gossipsub: Add basic smoke test Implement a basic smoke test that: 1. Builds a fully connected graph of size N. 2. Subscribes each node to the same topic. 3. Publishes a single message. 4. Waits for all nodes to receive the above message. N and the structure of the graph are reproducibly randomized via Quickcheck. --- protocols/gossipsub/Cargo.toml | 6 +- protocols/gossipsub/tests/smoke.rs | 226 +++++++++++++++++++++++++++++ 2 files changed, 231 insertions(+), 1 deletion(-) create mode 100644 protocols/gossipsub/tests/smoke.rs diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 2bda6be8e6d..0a1a417f47f 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -26,7 +26,11 @@ lru = "0.1.17" smallvec = "1.0.0" [dev-dependencies] +env_logger = "0.6.0" libp2p = { path = "../../" } +libp2p-plaintext = { version = "0.13.0", path = "../plaintext" } +libp2p-yamux = { version = "0.13.0", path = "../../muxers/yamux" } +quickcheck = "0.9.0" +rand = "0.7.2" tokio = "0.1" tokio-stdin-stdout = "0.1" -env_logger = "0.6.0" diff --git a/protocols/gossipsub/tests/smoke.rs b/protocols/gossipsub/tests/smoke.rs new file mode 100644 index 00000000000..cc4f85ea476 --- /dev/null +++ b/protocols/gossipsub/tests/smoke.rs @@ -0,0 +1,226 @@ +// Copyright 2019 Parity Technologies (UK) 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. + +#![cfg(test)] + +use futures::prelude::*; +use libp2p_core::{ + identity, + multiaddr::Protocol, + muxing::StreamMuxerBox, + nodes::Substream, + transport::{boxed::Boxed, MemoryTransport}, + upgrade, Multiaddr, PeerId, Transport, +}; +use libp2p_plaintext::PlainText2Config; +use libp2p_swarm::Swarm; +use libp2p_yamux as yamux; +use log::debug; +use quickcheck::{QuickCheck, TestResult}; +use rand::{random, seq::SliceRandom, SeedableRng}; +use std::{io::Error, time::Duration}; +use tokio::{runtime::current_thread::Runtime, util::FutureExt}; + +use libp2p_gossipsub::{Gossipsub, GossipsubConfig, GossipsubEvent, Topic}; + +type TestSwarm = + Swarm, Gossipsub>>; + +struct Graph { + pub nodes: Vec<(Multiaddr, TestSwarm)>, +} + +impl Future for Graph { + type Item = (Multiaddr, GossipsubEvent); + type Error = (); + + fn poll(&mut self) -> Poll { + for (addr, node) in &mut self.nodes { + match node.poll().expect("no error while polling") { + Async::Ready(Some(event)) => return Ok(Async::Ready((addr.clone(), event))), + Async::Ready(None) => panic!("unexpected None when polling nodes"), + Async::NotReady => {} + } + } + + Ok(Async::NotReady) + } +} + +impl Graph { + fn new_connected(num_nodes: usize, seed: u64) -> Graph { + if num_nodes == 0 { + panic!("expecting at least one node"); + } + + let mut rng = rand::rngs::StdRng::seed_from_u64(seed); + + let mut not_connected_nodes = std::iter::once(()) + .cycle() + .take(num_nodes) + .map(|_| build_node()) + .collect::>(); + + let mut connected_nodes = vec![not_connected_nodes.pop().unwrap()]; + + while !not_connected_nodes.is_empty() { + connected_nodes.shuffle(&mut rng); + not_connected_nodes.shuffle(&mut rng); + + let mut next = not_connected_nodes.pop().unwrap(); + let connected_addr = &connected_nodes[0].0; + + // Memory transport can not handle addresses with `/p2p` suffix. + let mut connected_addr_no_p2p = connected_addr.clone(); + let p2p_suffix_connected = connected_addr_no_p2p.pop(); + + debug!( + "Connect: {} -> {}", + next.0.clone().pop().unwrap(), + p2p_suffix_connected.unwrap() + ); + + Swarm::dial_addr(&mut next.1, connected_addr_no_p2p).unwrap(); + + connected_nodes.push(next); + } + + Graph { + nodes: connected_nodes, + } + } + /// Polls the graph and passes each event into the provided FnMut until it returns `true`. + fn wait_for(self, rt: &mut Runtime, mut f: F) -> Self + where + F: FnMut(GossipsubEvent) -> bool, + { + // The future below should return self. Given that it is a FnMut and not a FnOnce, one needs + // to wrap `self` in an Option, leaving a `None` behind after the final `Async::Ready`. + let mut this = Some(self); + + let fut = futures::future::poll_fn(move || -> Result<_, ()> { + match &mut this { + Some(graph) => loop { + match graph.poll().expect("no error while polling") { + Async::Ready((_addr, ev)) => { + if f(ev) { + return Ok(Async::Ready(this.take().unwrap())); + } + } + Async::NotReady => return Ok(Async::NotReady), + } + }, + None => panic!("future called after final return"), + } + }) + .timeout(Duration::from_secs(10)) + .map_err(|e| panic!("{:?}", e)); + + rt.block_on(fut).unwrap() + } +} + +fn build_node() -> (Multiaddr, TestSwarm) { + let key = identity::Keypair::generate_ed25519(); + let public_key = key.public(); + + let transport = MemoryTransport::default() + .upgrade(upgrade::Version::V1) + .authenticate(PlainText2Config { + local_public_key: public_key.clone(), + }) + .multiplex(yamux::Config::default()) + .map(|(p, m), _| (p, StreamMuxerBox::new(m))) + .map_err(|e| panic!("Failed to create transport: {:?}", e)) + .boxed(); + + let peer_id = public_key.clone().into_peer_id(); + let behaviour = Gossipsub::new(peer_id.clone(), GossipsubConfig::default()); + let mut swarm = Swarm::new(transport, behaviour, peer_id); + + let port = 1 + random::(); + let mut addr: Multiaddr = Protocol::Memory(port).into(); + Swarm::listen_on(&mut swarm, addr.clone()).unwrap(); + + addr = addr.with(libp2p::core::multiaddr::Protocol::P2p( + public_key.into_peer_id().into(), + )); + + (addr, swarm) +} + +#[test] +fn multi_hop_propagation() { + let _ = env_logger::try_init(); + + fn prop(num_nodes: usize, seed: u64) -> TestResult { + if num_nodes < 2 || num_nodes > 100 { + return TestResult::discard(); + } + + debug!("number nodes: {:?}, seed: {:?}", num_nodes, seed); + + let mut rt = Runtime::new().unwrap(); + + let mut graph = Graph::new_connected(num_nodes, seed); + let number_nodes = graph.nodes.len(); + + // Subscribe each node to the same topic. + let topic = Topic::new("test-net".into()); + for (_addr, node) in &mut graph.nodes { + node.subscribe(topic.clone()); + } + + // Wait for all nodes to be subscribed. + let mut subscribed = 0; + graph = graph.wait_for(&mut rt, move |ev| { + if let GossipsubEvent::Subscribed { .. } = ev { + subscribed += 1; + if subscribed == (number_nodes - 1) * 2 { + return true; + } + } + + false + }); + + // Publish a single message. + graph.nodes[0].1.publish(&topic, vec![1, 2, 3]); + + // Wait for all nodes to receive the published message. + let mut received_msgs = 0; + graph.wait_for(&mut rt, move |ev| { + if let GossipsubEvent::Message(_peer_id, _message_id, msg) = ev { + received_msgs += 1; + if received_msgs == number_nodes - 1 { + return true; + } + } + + false + }); + + TestResult::passed() + } + + QuickCheck::new() + .max_tests(10) + .quickcheck(prop as fn(usize, u64) -> TestResult) +} From 1774a765da2d75839b9a3bb01e572d645e2fa5ed Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 3 Jan 2020 02:28:05 +0000 Subject: [PATCH 79/91] Corrections pointed out by @mxinden --- protocols/gossipsub/src/behaviour.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index c8942fa7877..eff15d6f1e0 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -126,8 +126,8 @@ impl Gossipsub { } // send subscription request to all peers in the topic - let mut fixed_event = None; // initialise the event once if needed if let Some(peer_list) = self.topic_peers.get(&topic_hash) { + let mut fixed_event = None; // initialise the event once if needed if fixed_event.is_none() { fixed_event = Some(Arc::new(GossipsubRpc { messages: Vec::new(), @@ -818,8 +818,8 @@ impl Gossipsub { }, ); } - debug!("Completed gossip"); } + debug!("Completed gossip"); } /// Handles multiple GRAFT/PRUNE messages and coalesces them into chunked gossip control From 776d13ef046358964c7d64cda3295a3a3cb24743 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 3 Jan 2020 03:31:10 +0000 Subject: [PATCH 80/91] Add option to remove source id publishing --- examples/gossipsub-chat.rs | 2 +- protocols/gossipsub/src/behaviour.rs | 8 ++++- protocols/gossipsub/src/config.rs | 49 ++++++++++++++++++---------- 3 files changed, 40 insertions(+), 19 deletions(-) diff --git a/examples/gossipsub-chat.rs b/examples/gossipsub-chat.rs index c6ef5f24bfd..6d6efc1a768 100644 --- a/examples/gossipsub-chat.rs +++ b/examples/gossipsub-chat.rs @@ -70,7 +70,7 @@ fn main() { // Set up an encrypted TCP Transport over the Mplex and Yamux protocols let transport = libp2p::build_development_transport(local_key); - // Create a Floodsub/Gossipsub topic + // Create a Gossipsub topic let topic = Topic::new("test-net".into()); // Create a Swarm to manage peers and events diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index eff15d6f1e0..23c7b89e5a4 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -90,6 +90,12 @@ pub struct Gossipsub { impl Gossipsub { /// Creates a `Gossipsub` struct given a set of parameters specified by `gs_config`. pub fn new(local_peer_id: PeerId, gs_config: GossipsubConfig) -> Self { + let local_peer_id = if gs_config.no_source_id { + PeerId::from_bytes(crate::config::IDENTITY_SOURCE.to_vec()).expect("Valid peer id") + } else { + local_peer_id + }; + Gossipsub { config: gs_config.clone(), events: VecDeque::new(), @@ -228,7 +234,7 @@ impl Gossipsub { (self.config.message_id_fn)(&message) ); - // forward the message to mesh and floodsub peers + // forward the message to mesh peers let local_peer_id = self.local_peer_id.clone(); self.forward_msg(message.clone(), &local_peer_id); diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index 989c3297278..fd56042ab4b 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -22,50 +22,58 @@ use crate::protocol::{GossipsubMessage, MessageId}; use std::borrow::Cow; use std::time::Duration; +/// If the `no_source_id` flag is set, the IDENTITY_SOURCE value is used as the source of the +/// packet. +pub const IDENTITY_SOURCE: [u8; 3] = [0, 1, 0]; + /// Configuration parameters that define the performance of the gossipsub network. #[derive(Clone)] pub struct GossipsubConfig { - /// The protocol id to negotiate this protocol. + /// The protocol id to negotiate this protocol (default is `/meshsub/1.0.0`). pub protocol_id: Cow<'static, [u8]>, // Overlay network parameters. - /// Number of heartbeats to keep in the `memcache`. + /// Number of heartbeats to keep in the `memcache` (default is 5). pub history_length: usize, - /// Number of past heartbeats to gossip about. + /// Number of past heartbeats to gossip about (default is 3). pub history_gossip: usize, - /// Target number of peers for the mesh network (D in the spec). + /// Target number of peers for the mesh network (D in the spec, default is 6). pub mesh_n: usize, - /// Minimum number of peers in mesh network before adding more (D_lo in the spec). + /// Minimum number of peers in mesh network before adding more (D_lo in the spec, default is 4). pub mesh_n_low: usize, - /// Maximum number of peers in mesh network before removing some (D_high in the spec). + /// Maximum number of peers in mesh network before removing some (D_high in the spec, default + /// is 12). pub mesh_n_high: usize, - /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec). + /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec, default is 6). pub gossip_lazy: usize, - /// Initial delay in each heartbeat. + /// Initial delay in each heartbeat (default is 5 seconds). pub heartbeat_initial_delay: Duration, - /// Time between each heartbeat. + /// Time between each heartbeat (default is 1 second). pub heartbeat_interval: Duration, - /// Time to live for fanout peers. + /// Time to live for fanout peers (default is 60 seconds). pub fanout_ttl: Duration, - /// The maximum byte size for each gossip. + /// The maximum byte size for each gossip (default is 2048 bytes). pub max_transmit_size: usize, - /// Flag determining if gossipsub topics are hashed or sent as plain strings. + /// Flag determining if gossipsub topics are hashed or sent as plain strings (default is false). pub hash_topics: bool, + /// When set, all published messages will have a 0 source `PeerId` (default is false). + pub no_source_id: bool, + /// 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 `propagate_message()` on the behaviour to forward message - /// once validated. + /// once validated (default is false). pub manual_propagation: bool, /// A user-defined function allowing the user to specify the message id of a gossipsub message. @@ -94,6 +102,7 @@ impl Default for GossipsubConfig { fanout_ttl: Duration::from_secs(60), max_transmit_size: 2048, hash_topics: false, // default compatibility with floodsub + no_source_id: false, manual_propagation: false, message_id_fn: |message| { // default message id is: source + sequence number @@ -195,13 +204,18 @@ impl GossipsubConfigBuilder { self } - pub fn hash_topics(&mut self, hash_topics: bool) -> &mut Self { - self.config.hash_topics = hash_topics; + pub fn hash_topics(&mut self) -> &mut Self { + self.config.hash_topics = true; + self + } + + pub fn no_source_id(&mut self) -> &mut Self { + self.config.no_source_id = true; self } - pub fn manual_propagation(&mut self, manual_propagation: bool) -> &mut Self { - self.config.manual_propagation = manual_propagation; + pub fn manual_propagation(&mut self) -> &mut Self { + self.config.manual_propagation = true; self } @@ -230,6 +244,7 @@ impl std::fmt::Debug for GossipsubConfig { let _ = builder.field("fanout_ttl", &self.fanout_ttl); let _ = builder.field("max_transmit_size", &self.max_transmit_size); let _ = builder.field("hash_topics", &self.hash_topics); + let _ = builder.field("no_source_id", &self.no_source_id); let _ = builder.field("manual_propagation", &self.manual_propagation); builder.finish() } From ff9565b9f9af3278385c56dad56a453f89f01d02 Mon Sep 17 00:00:00 2001 From: Max Inden Date: Mon, 6 Jan 2020 10:57:29 +0100 Subject: [PATCH 81/91] protocols/gossipsub/tests/smoke: Remove unused variable --- protocols/gossipsub/tests/smoke.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/gossipsub/tests/smoke.rs b/protocols/gossipsub/tests/smoke.rs index cc4f85ea476..65cc7d0c22b 100644 --- a/protocols/gossipsub/tests/smoke.rs +++ b/protocols/gossipsub/tests/smoke.rs @@ -207,7 +207,7 @@ fn multi_hop_propagation() { // Wait for all nodes to receive the published message. let mut received_msgs = 0; graph.wait_for(&mut rt, move |ev| { - if let GossipsubEvent::Message(_peer_id, _message_id, msg) = ev { + if let GossipsubEvent::Message(..) = ev { received_msgs += 1; if received_msgs == number_nodes - 1 { return true; From a2557687928621310afa5946d27a180aae621595 Mon Sep 17 00:00:00 2001 From: Age Manning Date: Wed, 15 Jan 2020 14:05:32 +1100 Subject: [PATCH 82/91] Merge latest master --- CHANGELOG.md | 12 + Cargo.toml | 64 +- core/Cargo.toml | 45 +- core/src/either.rs | 310 ++++---- core/src/identity/ed25519.rs | 7 +- core/src/lib.rs | 5 +- core/src/muxing.rs | 284 +++---- core/src/muxing/singleton.rs | 51 +- core/src/nodes/collection.rs | 74 +- core/src/nodes/collection/tests.rs | 373 --------- core/src/nodes/handled_node.rs | 41 +- core/src/nodes/handled_node/tests.rs | 170 ---- core/src/nodes/listeners.rs | 378 +++------ core/src/nodes/network.rs | 145 ++-- core/src/nodes/network/tests.rs | 413 ---------- core/src/nodes/node.rs | 111 +-- core/src/nodes/tasks/manager.rs | 264 ++---- core/src/nodes/tasks/mod.rs | 4 +- core/src/nodes/tasks/task.rs | 195 ++--- core/src/peer_id.rs | 22 +- core/src/tests/dummy_handler.rs | 125 --- core/src/tests/dummy_muxer.rs | 122 --- core/src/tests/dummy_transport.rs | 115 --- core/src/tests/mod.rs | 28 - core/src/transport/and_then.rs | 90 ++- core/src/transport/boxed.rs | 16 +- core/src/transport/dummy.rs | 47 +- core/src/transport/map.rs | 50 +- core/src/transport/map_err.rs | 67 +- core/src/transport/memory.rs | 159 ++-- core/src/transport/mod.rs | 10 +- core/src/transport/timeout.rs | 108 +-- core/src/transport/upgrade.rs | 223 +++--- core/src/upgrade/apply.rs | 134 ++-- core/src/upgrade/denied.rs | 14 +- core/src/upgrade/either.rs | 5 +- core/src/upgrade/map.rs | 73 +- core/src/upgrade/mod.rs | 13 +- core/src/upgrade/optional.rs | 5 +- core/src/upgrade/select.rs | 5 +- core/src/upgrade/transfer.rs | 490 +++--------- core/tests/network_dial_error.rs | 94 +-- core/tests/network_simult.rs | 253 +++--- core/tests/transport_upgrade.rs | 85 +- core/tests/util.rs | 14 +- examples/chat.rs | 96 ++- examples/distributed-key-value-store.rs | 81 +- examples/ipfs-kad.rs | 64 +- examples/mdns-passive-discovery.rs | 31 +- examples/ping.rs | 46 +- misc/core-derive/Cargo.toml | 6 +- misc/core-derive/src/lib.rs | 48 +- misc/core-derive/tests/test.rs | 16 +- misc/mdns/Cargo.toml | 23 +- misc/mdns/src/behaviour.rs | 126 ++- misc/mdns/src/service.rs | 611 +++++++------- misc/multiaddr/Cargo.toml | 10 +- misc/multiaddr/src/errors.rs | 4 +- misc/multiaddr/src/lib.rs | 76 +- misc/multiaddr/src/protocol.rs | 2 +- misc/multiaddr/src/util.rs | 21 - misc/multiaddr/tests/lib.rs | 12 + misc/multihash/Cargo.toml | 8 +- misc/multihash/src/lib.rs | 2 +- misc/multistream-select/Cargo.toml | 8 +- .../src/length_delimited.rs | 4 +- misc/multistream-select/src/negotiated.rs | 10 +- misc/multistream-select/src/protocol.rs | 6 +- misc/peer-id-generator/Cargo.toml | 2 +- misc/rw-stream-sink/Cargo.toml | 11 +- misc/rw-stream-sink/src/lib.rs | 243 +++--- muxers/mplex/Cargo.toml | 19 +- muxers/mplex/src/codec.rs | 2 +- muxers/mplex/src/lib.rs | 253 +++--- muxers/mplex/tests/async_write.rs | 71 +- muxers/mplex/tests/two_peers.rs | 149 ++-- muxers/yamux/Cargo.toml | 13 +- muxers/yamux/src/lib.rs | 287 +++++-- protocols/deflate/Cargo.toml | 18 +- protocols/deflate/src/lib.rs | 229 +++++- protocols/deflate/tests/test.rs | 121 ++- protocols/floodsub/Cargo.toml | 14 +- protocols/floodsub/src/layer.rs | 11 +- protocols/floodsub/src/protocol.rs | 27 +- protocols/gossipsub/Cargo.toml | 6 +- protocols/identify/Cargo.toml | 26 +- protocols/identify/src/handler.rs | 39 +- protocols/identify/src/identify.rs | 118 ++- protocols/identify/src/protocol.rs | 265 ++----- protocols/kad/Cargo.toml | 29 +- protocols/kad/src/behaviour.rs | 20 +- protocols/kad/src/behaviour/test.rs | 214 ++--- protocols/kad/src/handler.rs | 197 +++-- protocols/kad/src/jobs.rs | 110 +-- protocols/kad/src/protocol.rs | 72 +- protocols/kad/src/record.rs | 2 +- protocols/noise/Cargo.toml | 13 +- protocols/noise/src/io.rs | 342 ++++---- protocols/noise/src/io/handshake.rs | 607 +++++--------- protocols/noise/src/lib.rs | 130 ++- protocols/noise/src/protocol/x25519.rs | 11 +- protocols/noise/tests/smoke.rs | 95 ++- protocols/ping/Cargo.toml | 22 +- protocols/ping/src/handler.rs | 68 +- protocols/ping/src/lib.rs | 13 +- protocols/ping/src/protocol.rs | 170 +--- protocols/ping/tests/ping.rs | 77 +- protocols/plaintext/Cargo.toml | 20 +- protocols/plaintext/regen_structs_proto.sh | 6 +- protocols/plaintext/src/handshake.rs | 88 +- protocols/plaintext/src/lib.rs | 181 +++-- protocols/plaintext/src/pb.rs | 21 - protocols/plaintext/{ => src}/structs.proto | 0 .../src/{pb/structs.rs => structs_proto.rs} | 81 +- protocols/plaintext/tests/smoke.rs | 121 +++ protocols/secio/Cargo.toml | 36 +- protocols/secio/src/codec/decode.rs | 95 ++- protocols/secio/src/codec/encode.rs | 68 +- protocols/secio/src/codec/len_prefix.rs | 127 +++ protocols/secio/src/codec/mod.rs | 148 ++-- protocols/secio/src/exchange/impl_ring.rs | 18 +- .../secio/src/exchange/impl_webcrypto.rs | 193 +++-- protocols/secio/src/exchange/mod.rs | 6 +- protocols/secio/src/handshake.rs | 750 +++++++----------- protocols/secio/src/lib.rs | 140 ++-- src/bandwidth.rs | 123 ++- src/lib.rs | 89 ++- src/simple.rs | 44 +- swarm/Cargo.toml | 14 +- swarm/src/behaviour.rs | 29 +- swarm/src/lib.rs | 350 ++++---- swarm/src/protocols_handler/dummy.rs | 19 +- swarm/src/protocols_handler/map_in.rs | 17 +- swarm/src/protocols_handler/map_out.rs | 19 +- swarm/src/protocols_handler/mod.rs | 59 +- swarm/src/protocols_handler/node_handler.rs | 106 +-- swarm/src/protocols_handler/one_shot.rs | 45 +- swarm/src/protocols_handler/select.rs | 68 +- swarm/src/toggle.rs | 26 +- transports/dns/Cargo.toml | 10 +- transports/dns/src/lib.rs | 297 +++---- transports/tcp/Cargo.toml | 15 +- transports/tcp/src/lib.rs | 459 ++++------- transports/uds/Cargo.toml | 10 +- transports/uds/src/lib.rs | 176 ++-- transports/wasm-ext/Cargo.toml | 9 +- transports/wasm-ext/src/lib.rs | 103 ++- transports/websocket/Cargo.toml | 29 +- transports/websocket/src/framed.rs | 498 +++++++----- transports/websocket/src/lib.rs | 152 ++-- transports/websocket/src/tls.rs | 7 +- 151 files changed, 6860 insertions(+), 8752 deletions(-) delete mode 100644 core/src/nodes/collection/tests.rs delete mode 100644 core/src/nodes/handled_node/tests.rs delete mode 100644 core/src/tests/dummy_handler.rs delete mode 100644 core/src/tests/dummy_muxer.rs delete mode 100644 core/src/tests/dummy_transport.rs delete mode 100644 core/src/tests/mod.rs delete mode 100644 misc/multiaddr/src/util.rs delete mode 100644 protocols/plaintext/src/pb.rs rename protocols/plaintext/{ => src}/structs.proto (100%) rename protocols/plaintext/src/{pb/structs.rs => structs_proto.rs} (81%) create mode 100644 protocols/plaintext/tests/smoke.rs create mode 100644 protocols/secio/src/codec/len_prefix.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index 072fa0fd3a5..c42e8155dbd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,15 @@ +# Version 0.14.0-alpha.1 (2020-01-07) + +- Upgraded the crate to stable futures. +- Use varints instead of fixed sized (4 byte) integers to delimit plaintext 2.0 messages to align implementation with the specification. +- Refactored the `core::upgrade` module to provide async functions. +- Changed the `Stream` trait implementation of `Swarm` to no longer return a `Result`. +- Added the `Swarm::next` and `Swarm::next_event` functions and the `SwarmEvent` enum. +- Changed `ProtocolsHandler::poll` to no longer return an error. Instead, `ProtocolsHandlerEvent` has a new `Close` variant which corresponds to what an error represented before. +- Changed all the traits that have a `poll` function (i.e. `NetworkBehaviour`, `ProtocolsHandler`, `NodeHandler`) to have an additional `&mut Context` parameter, to reflect the changes in the `Future` trait. +- Revamped the API of `libp2p_websockets::framed`. +- Added protocol string to `Error::UnknownProtocolString`. + # Version 0.13.2 (2020-01-02) - Fixed the `libp2p-noise` handshake not flushing the underlying stream before waiting for a response. diff --git a/Cargo.toml b/Cargo.toml index 9bec3a4b0c4..7e66f3876ea 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p" edition = "2018" description = "Peer-to-peer networking library" -version = "0.13.2" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -14,44 +14,41 @@ default = ["secp256k1", "libp2p-websocket"] secp256k1 = ["libp2p-core/secp256k1", "libp2p-secio/secp256k1"] [dependencies] -bytes = "0.4" -futures = "0.1" -multiaddr = { package = "parity-multiaddr", version = "0.6.0", path = "misc/multiaddr" } -multihash = { package = "parity-multihash", version = "0.2.0", path = "misc/multihash" } +bytes = "0.5" +futures = "0.3.1" +multiaddr = { package = "parity-multiaddr", version = "0.7.0", path = "misc/multiaddr" } +multihash = { package = "parity-multihash", version = "0.2.1", path = "misc/multihash" } lazy_static = "1.2" -libp2p-mplex = { version = "0.13.0", path = "muxers/mplex" } -libp2p-identify = { version = "0.13.2", path = "protocols/identify" } -libp2p-kad = { version = "0.13.2", path = "protocols/kad" } -libp2p-floodsub = { version = "0.13.1", path = "protocols/floodsub" } -libp2p-gossipsub = { version = "0.1.0", path = "./protocols/gossipsub" } -libp2p-ping = { version = "0.13.1", path = "protocols/ping" } -libp2p-plaintext = { version = "0.13.1", path = "protocols/plaintext" } -libp2p-core = { version = "0.13.2", path = "core" } -libp2p-core-derive = { version = "0.13.0", path = "misc/core-derive" } -libp2p-secio = { version = "0.13.1", path = "protocols/secio", default-features = false } -libp2p-swarm = { version = "0.3.0", path = "swarm" } -libp2p-uds = { version = "0.13.0", path = "transports/uds" } -libp2p-wasm-ext = { version = "0.6.0", path = "transports/wasm-ext" } -libp2p-yamux = { version = "0.13.0", path = "muxers/yamux" } -parking_lot = "0.9.0" -smallvec = "0.6" -tokio-codec = "0.1" -tokio-executor = "0.1" -tokio-io = "0.1" -wasm-timer = "0.1" +libp2p-mplex = { version = "0.14.0-alpha.1", path = "muxers/mplex" } +libp2p-identify = { version = "0.14.0-alpha.1", path = "protocols/identify" } +libp2p-kad = { version = "0.14.0-alpha.1", path = "protocols/kad" } +libp2p-floodsub = { version = "0.14.0-alpha.1", path = "protocols/floodsub" } +libp2p-gossipsub = { version = "0.14.0-alpha.1", path = "./protocols/gossipsub" } +libp2p-ping = { version = "0.14.0-alpha.1", path = "protocols/ping" } +libp2p-plaintext = { version = "0.14.0-alpha.1", path = "protocols/plaintext" } +libp2p-core = { version = "0.14.0-alpha.1", path = "core" } +libp2p-core-derive = { version = "0.14.0-alpha.1", path = "misc/core-derive" } +libp2p-secio = { version = "0.14.0-alpha.1", path = "protocols/secio", default-features = false } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "swarm" } +libp2p-uds = { version = "0.14.0-alpha.1", path = "transports/uds" } +libp2p-wasm-ext = { version = "0.7.0-alpha.1", path = "transports/wasm-ext" } +libp2p-yamux = { version = "0.14.0-alpha.1", path = "muxers/yamux" } +parking_lot = "0.10.0" +pin-project = "0.4.6" +smallvec = "1.0" +wasm-timer = "0.2.4" [target.'cfg(not(any(target_os = "emscripten", target_os = "unknown")))'.dependencies] -libp2p-deflate = { version = "0.5.0", path = "protocols/deflate" } -libp2p-dns = { version = "0.13.0", path = "transports/dns" } -libp2p-mdns = { version = "0.13.1", path = "misc/mdns" } -libp2p-noise = { version = "0.11.1", path = "protocols/noise" } -libp2p-tcp = { version = "0.13.0", path = "transports/tcp" } -libp2p-websocket = { version = "0.13.0", path = "transports/websocket", optional = true } +libp2p-deflate = { version = "0.6.0-alpha.1", path = "protocols/deflate" } +libp2p-dns = { version = "0.14.0-alpha.1", path = "transports/dns" } +libp2p-mdns = { version = "0.14.0-alpha.1", path = "misc/mdns" } +libp2p-noise = { version = "0.12.0-alpha.1", path = "protocols/noise" } +libp2p-tcp = { version = "0.14.0-alpha.1", path = "transports/tcp" } +libp2p-websocket = { version = "0.14.0-alpha.1", path = "transports/websocket", optional = true } [dev-dependencies] +async-std = "1.0" env_logger = "0.7.1" -tokio = "0.1" -tokio-stdin-stdout = "0.1" [workspace] members = [ @@ -80,3 +77,4 @@ members = [ "transports/websocket", "transports/wasm-ext" ] + diff --git a/core/Cargo.toml b/core/Cargo.toml index 1b2446a0041..3bfb7522c34 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-core" edition = "2018" description = "Core traits and structs of libp2p" -version = "0.13.2" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -12,28 +12,25 @@ categories = ["network-programming", "asynchronous"] [dependencies] asn1_der = "0.6.1" bs58 = "0.3.0" -bytes = "0.4" ed25519-dalek = "1.0.0-pre.3" -failure = "0.1" fnv = "1.0" +futures = { version = "0.3.1", features = ["compat", "io-compat", "executor", "thread-pool"] } +futures-timer = "2" lazy_static = "1.2" +libsecp256k1 = { version = "0.3.1", optional = true } log = "0.4" -multiaddr = { package = "parity-multiaddr", version = "0.6.0", path = "../misc/multiaddr" } -multihash = { package = "parity-multihash", version = "0.2.0", path = "../misc/multihash" } -multistream-select = { version = "0.6.0", path = "../misc/multistream-select" } -futures = "0.1" -parking_lot = "0.9.0" +multiaddr = { package = "parity-multiaddr", version = "0.7.0", path = "../misc/multiaddr" } +multihash = { package = "parity-multihash", version = "0.2.1", path = "../misc/multihash" } +multistream-select = { version = "0.7.0", path = "../misc/multistream-select" } +parking_lot = "0.10.0" +pin-project = "0.4.6" protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 -quick-error = "1.2" rand = "0.7" -rw-stream-sink = { version = "0.1.1", path = "../misc/rw-stream-sink" } -libsecp256k1 = { version = "0.3.1", optional = true } +rw-stream-sink = { version = "0.2.0", path = "../misc/rw-stream-sink" } sha2 = "0.8.0" -smallvec = "0.6" -tokio-executor = "0.1.4" -tokio-io = "0.1" -wasm-timer = "0.1" -unsigned-varint = "0.2" +smallvec = "1.0" +thiserror = "1.0" +unsigned-varint = "0.3" void = "1" zeroize = "1" @@ -42,16 +39,14 @@ ring = { version = "0.16.9", features = ["alloc", "std"], default-features = fal untrusted = "0.7.0" [dev-dependencies] -libp2p-swarm = { version = "0.3.0", path = "../swarm" } -libp2p-tcp = { version = "0.13.0", path = "../transports/tcp" } -libp2p-mplex = { version = "0.13.0", path = "../muxers/mplex" } -libp2p-secio = { version = "0.13.0", path = "../protocols/secio" } -rand = "0.7.2" -quickcheck = "0.9.0" -tokio = "0.1" -wasm-timer = "0.1" assert_matches = "1.3" -tokio-mock-task = "0.1" +async-std = "1.0" +libp2p-mplex = { version = "0.14.0-alpha.1", path = "../muxers/mplex" } +libp2p-secio = { version = "0.14.0-alpha.1", path = "../protocols/secio" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../swarm" } +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../transports/tcp" } +quickcheck = "0.9.0" +wasm-timer = "0.2" [features] default = ["secp256k1"] diff --git a/core/src/either.rs b/core/src/either.rs index d17f8bb7aa3..a3c24adef5e 100644 --- a/core/src/either.rs +++ b/core/src/either.rs @@ -19,9 +19,9 @@ // DEALINGS IN THE SOFTWARE. use crate::{muxing::StreamMuxer, ProtocolName, transport::ListenerEvent}; -use futures::prelude::*; -use std::{fmt, io::{Error as IoError, Read, Write}}; -use tokio_io::{AsyncRead, AsyncWrite}; +use futures::{prelude::*, io::{IoSlice, IoSliceMut}}; +use pin_project::{pin_project, project}; +use std::{fmt, io::{Error as IoError}, pin::Pin, task::Context, task::Poll}; #[derive(Debug, Copy, Clone)] pub enum EitherError { @@ -57,10 +57,11 @@ where /// Implements `AsyncRead` and `AsyncWrite` and dispatches all method calls to /// either `First` or `Second`. +#[pin_project] #[derive(Debug, Copy, Clone)] pub enum EitherOutput { - First(A), - Second(B), + First(#[pin] A), + Second(#[pin] B), } impl AsyncRead for EitherOutput @@ -68,30 +69,23 @@ where A: AsyncRead, B: AsyncRead, { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - match self { - EitherOutput::First(a) => a.prepare_uninitialized_buffer(buf), - EitherOutput::Second(b) => b.prepare_uninitialized_buffer(buf), + #[project] + fn poll_read(self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => AsyncRead::poll_read(a, cx, buf), + EitherOutput::Second(b) => AsyncRead::poll_read(b, cx, buf), } } - fn read_buf(&mut self, buf: &mut Bu) -> Poll { - match self { - EitherOutput::First(a) => a.read_buf(buf), - EitherOutput::Second(b) => b.read_buf(buf), - } - } -} - -impl Read for EitherOutput -where - A: Read, - B: Read, -{ - fn read(&mut self, buf: &mut [u8]) -> Result { - match self { - EitherOutput::First(a) => a.read(buf), - EitherOutput::Second(b) => b.read(buf), + #[project] + fn poll_read_vectored(self: Pin<&mut Self>, cx: &mut Context, bufs: &mut [IoSliceMut]) + -> Poll> + { + #[project] + match self.project() { + EitherOutput::First(a) => AsyncRead::poll_read_vectored(a, cx, bufs), + EitherOutput::Second(b) => AsyncRead::poll_read_vectored(b, cx, bufs), } } } @@ -101,76 +95,104 @@ where A: AsyncWrite, B: AsyncWrite, { - fn shutdown(&mut self) -> Poll<(), IoError> { - match self { - EitherOutput::First(a) => a.shutdown(), - EitherOutput::Second(b) => b.shutdown(), + #[project] + fn poll_write(self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => AsyncWrite::poll_write(a, cx, buf), + EitherOutput::Second(b) => AsyncWrite::poll_write(b, cx, buf), } } -} -impl Write for EitherOutput -where - A: Write, - B: Write, -{ - fn write(&mut self, buf: &[u8]) -> Result { - match self { - EitherOutput::First(a) => a.write(buf), - EitherOutput::Second(b) => b.write(buf), + #[project] + fn poll_write_vectored(self: Pin<&mut Self>, cx: &mut Context, bufs: &[IoSlice]) + -> Poll> + { + #[project] + match self.project() { + EitherOutput::First(a) => AsyncWrite::poll_write_vectored(a, cx, bufs), + EitherOutput::Second(b) => AsyncWrite::poll_write_vectored(b, cx, bufs), } } - fn flush(&mut self) -> Result<(), IoError> { - match self { - EitherOutput::First(a) => a.flush(), - EitherOutput::Second(b) => b.flush(), + #[project] + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => AsyncWrite::poll_flush(a, cx), + EitherOutput::Second(b) => AsyncWrite::poll_flush(b, cx), + } + } + + #[project] + fn poll_close(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => AsyncWrite::poll_close(a, cx), + EitherOutput::Second(b) => AsyncWrite::poll_close(b, cx), } } } impl Stream for EitherOutput where - A: Stream, - B: Stream, + A: TryStream, + B: TryStream, { - type Item = I; - type Error = EitherError; - - fn poll(&mut self) -> Poll, Self::Error> { - match self { - EitherOutput::First(a) => a.poll().map_err(EitherError::A), - EitherOutput::Second(b) => b.poll().map_err(EitherError::B), + type Item = Result>; + + #[project] + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => TryStream::try_poll_next(a, cx) + .map(|v| v.map(|r| r.map_err(EitherError::A))), + EitherOutput::Second(b) => TryStream::try_poll_next(b, cx) + .map(|v| v.map(|r| r.map_err(EitherError::B))), } } } -impl Sink for EitherOutput +impl Sink for EitherOutput where - A: Sink, - B: Sink, + A: Sink, + B: Sink, { - type SinkItem = I; - type SinkError = EitherError; + type Error = EitherError; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - match self { - EitherOutput::First(a) => a.start_send(item).map_err(EitherError::A), - EitherOutput::Second(b) => b.start_send(item).map_err(EitherError::B), + #[project] + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => Sink::poll_ready(a, cx).map_err(EitherError::A), + EitherOutput::Second(b) => Sink::poll_ready(b, cx).map_err(EitherError::B), } } - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - match self { - EitherOutput::First(a) => a.poll_complete().map_err(EitherError::A), - EitherOutput::Second(b) => b.poll_complete().map_err(EitherError::B), + #[project] + fn start_send(self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> { + #[project] + match self.project() { + EitherOutput::First(a) => Sink::start_send(a, item).map_err(EitherError::A), + EitherOutput::Second(b) => Sink::start_send(b, item).map_err(EitherError::B), } } - fn close(&mut self) -> Poll<(), Self::SinkError> { - match self { - EitherOutput::First(a) => a.close().map_err(EitherError::A), - EitherOutput::Second(b) => b.close().map_err(EitherError::B), + #[project] + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => Sink::poll_flush(a, cx).map_err(EitherError::A), + EitherOutput::Second(b) => Sink::poll_flush(b, cx).map_err(EitherError::B), + } + } + + #[project] + fn poll_close(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherOutput::First(a) => Sink::poll_close(a, cx).map_err(EitherError::A), + EitherOutput::Second(b) => Sink::poll_close(b, cx).map_err(EitherError::B), } } } @@ -184,10 +206,10 @@ where type OutboundSubstream = EitherOutbound; type Error = IoError; - fn poll_inbound(&self) -> Poll { + fn poll_inbound(&self, cx: &mut Context) -> Poll> { match self { - EitherOutput::First(inner) => inner.poll_inbound().map(|p| p.map(EitherOutput::First)).map_err(|e| e.into()), - EitherOutput::Second(inner) => inner.poll_inbound().map(|p| p.map(EitherOutput::Second)).map_err(|e| e.into()), + EitherOutput::First(inner) => inner.poll_inbound(cx).map(|p| p.map(EitherOutput::First)).map_err(|e| e.into()), + EitherOutput::Second(inner) => inner.poll_inbound(cx).map(|p| p.map(EitherOutput::Second)).map_err(|e| e.into()), } } @@ -198,13 +220,13 @@ where } } - fn poll_outbound(&self, substream: &mut Self::OutboundSubstream) -> Poll { + fn poll_outbound(&self, cx: &mut Context, substream: &mut Self::OutboundSubstream) -> Poll> { match (self, substream) { (EitherOutput::First(ref inner), EitherOutbound::A(ref mut substream)) => { - inner.poll_outbound(substream).map(|p| p.map(EitherOutput::First)).map_err(|e| e.into()) + inner.poll_outbound(cx, substream).map(|p| p.map(EitherOutput::First)).map_err(|e| e.into()) }, (EitherOutput::Second(ref inner), EitherOutbound::B(ref mut substream)) => { - inner.poll_outbound(substream).map(|p| p.map(EitherOutput::Second)).map_err(|e| e.into()) + inner.poll_outbound(cx, substream).map(|p| p.map(EitherOutput::Second)).map_err(|e| e.into()) }, _ => panic!("Wrong API usage") } @@ -227,56 +249,49 @@ where } } - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - match self { - EitherOutput::First(ref inner) => inner.prepare_uninitialized_buffer(buf), - EitherOutput::Second(ref inner) => inner.prepare_uninitialized_buffer(buf), - } - } - - fn read_substream(&self, sub: &mut Self::Substream, buf: &mut [u8]) -> Poll { + fn read_substream(&self, cx: &mut Context, sub: &mut Self::Substream, buf: &mut [u8]) -> Poll> { match (self, sub) { (EitherOutput::First(ref inner), EitherOutput::First(ref mut sub)) => { - inner.read_substream(sub, buf).map_err(|e| e.into()) + inner.read_substream(cx, sub, buf).map_err(|e| e.into()) }, (EitherOutput::Second(ref inner), EitherOutput::Second(ref mut sub)) => { - inner.read_substream(sub, buf).map_err(|e| e.into()) + inner.read_substream(cx, sub, buf).map_err(|e| e.into()) }, _ => panic!("Wrong API usage") } } - fn write_substream(&self, sub: &mut Self::Substream, buf: &[u8]) -> Poll { + fn write_substream(&self, cx: &mut Context, sub: &mut Self::Substream, buf: &[u8]) -> Poll> { match (self, sub) { (EitherOutput::First(ref inner), EitherOutput::First(ref mut sub)) => { - inner.write_substream(sub, buf).map_err(|e| e.into()) + inner.write_substream(cx, sub, buf).map_err(|e| e.into()) }, (EitherOutput::Second(ref inner), EitherOutput::Second(ref mut sub)) => { - inner.write_substream(sub, buf).map_err(|e| e.into()) + inner.write_substream(cx, sub, buf).map_err(|e| e.into()) }, _ => panic!("Wrong API usage") } } - fn flush_substream(&self, sub: &mut Self::Substream) -> Poll<(), Self::Error> { + fn flush_substream(&self, cx: &mut Context, sub: &mut Self::Substream) -> Poll> { match (self, sub) { (EitherOutput::First(ref inner), EitherOutput::First(ref mut sub)) => { - inner.flush_substream(sub).map_err(|e| e.into()) + inner.flush_substream(cx, sub).map_err(|e| e.into()) }, (EitherOutput::Second(ref inner), EitherOutput::Second(ref mut sub)) => { - inner.flush_substream(sub).map_err(|e| e.into()) + inner.flush_substream(cx, sub).map_err(|e| e.into()) }, _ => panic!("Wrong API usage") } } - fn shutdown_substream(&self, sub: &mut Self::Substream) -> Poll<(), Self::Error> { + fn shutdown_substream(&self, cx: &mut Context, sub: &mut Self::Substream) -> Poll> { match (self, sub) { (EitherOutput::First(ref inner), EitherOutput::First(ref mut sub)) => { - inner.shutdown_substream(sub).map_err(|e| e.into()) + inner.shutdown_substream(cx, sub).map_err(|e| e.into()) }, (EitherOutput::Second(ref inner), EitherOutput::Second(ref mut sub)) => { - inner.shutdown_substream(sub).map_err(|e| e.into()) + inner.shutdown_substream(cx, sub).map_err(|e| e.into()) }, _ => panic!("Wrong API usage") } @@ -306,17 +321,17 @@ where } } - fn close(&self) -> Poll<(), Self::Error> { + fn close(&self, cx: &mut Context) -> Poll> { match self { - EitherOutput::First(inner) => inner.close().map_err(|e| e.into()), - EitherOutput::Second(inner) => inner.close().map_err(|e| e.into()), + EitherOutput::First(inner) => inner.close(cx).map_err(|e| e.into()), + EitherOutput::Second(inner) => inner.close(cx).map_err(|e| e.into()), } } - fn flush_all(&self) -> Poll<(), Self::Error> { + fn flush_all(&self, cx: &mut Context) -> Poll> { match self { - EitherOutput::First(inner) => inner.flush_all().map_err(|e| e.into()), - EitherOutput::Second(inner) => inner.flush_all().map_err(|e| e.into()), + EitherOutput::First(inner) => inner.flush_all(cx).map_err(|e| e.into()), + EitherOutput::Second(inner) => inner.flush_all(cx).map_err(|e| e.into()), } } } @@ -329,78 +344,89 @@ pub enum EitherOutbound { } /// Implements `Stream` and dispatches all method calls to either `First` or `Second`. +#[pin_project] #[derive(Debug, Copy, Clone)] #[must_use = "futures do nothing unless polled"] pub enum EitherListenStream { - First(A), - Second(B), + First(#[pin] A), + Second(#[pin] B), } impl Stream for EitherListenStream where - AStream: Stream>, - BStream: Stream>, + AStream: TryStream>, + BStream: TryStream>, { - type Item = ListenerEvent>; - type Error = EitherError; - - fn poll(&mut self) -> Poll, Self::Error> { - match self { - EitherListenStream::First(a) => a.poll() - .map(|i| (i.map(|v| (v.map(|e| e.map(EitherFuture::First)))))) - .map_err(EitherError::A), - EitherListenStream::Second(a) => a.poll() - .map(|i| (i.map(|v| (v.map(|e| e.map(EitherFuture::Second)))))) - .map_err(EitherError::B), + type Item = Result>, EitherError>; + + #[project] + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[project] + match self.project() { + EitherListenStream::First(a) => match TryStream::try_poll_next(a, cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(None) => Poll::Ready(None), + Poll::Ready(Some(Ok(le))) => Poll::Ready(Some(Ok(le.map(EitherFuture::First)))), + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(EitherError::A(err)))), + }, + EitherListenStream::Second(a) => match TryStream::try_poll_next(a, cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(None) => Poll::Ready(None), + Poll::Ready(Some(Ok(le))) => Poll::Ready(Some(Ok(le.map(EitherFuture::Second)))), + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(EitherError::B(err)))), + }, } } } /// Implements `Future` and dispatches all method calls to either `First` or `Second`. +#[pin_project] #[derive(Debug, Copy, Clone)] #[must_use = "futures do nothing unless polled"] pub enum EitherFuture { - First(A), - Second(B), + First(#[pin] A), + Second(#[pin] B), } impl Future for EitherFuture where - AFuture: Future, - BFuture: Future, + AFuture: TryFuture, + BFuture: TryFuture, { - type Item = EitherOutput; - type Error = EitherError; - - fn poll(&mut self) -> Poll { - match self { - EitherFuture::First(a) => a.poll().map(|v| v.map(EitherOutput::First)).map_err(EitherError::A), - EitherFuture::Second(a) => a.poll().map(|v| v.map(EitherOutput::Second)).map_err(EitherError::B), + type Output = Result, EitherError>; + + #[project] + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + #[project] + match self.project() { + EitherFuture::First(a) => TryFuture::try_poll(a, cx) + .map_ok(EitherOutput::First).map_err(EitherError::A), + EitherFuture::Second(a) => TryFuture::try_poll(a, cx) + .map_ok(EitherOutput::Second).map_err(EitherError::B), } } } +#[pin_project] #[derive(Debug, Copy, Clone)] #[must_use = "futures do nothing unless polled"] -pub enum EitherFuture2 { A(A), B(B) } +pub enum EitherFuture2 { A(#[pin] A), B(#[pin] B) } impl Future for EitherFuture2 where - AFut: Future, - BFut: Future + AFut: TryFuture, + BFut: TryFuture, { - type Item = EitherOutput; - type Error = EitherError; - - fn poll(&mut self) -> Poll { - match self { - EitherFuture2::A(a) => a.poll() - .map(|v| v.map(EitherOutput::First)) - .map_err(EitherError::A), - - EitherFuture2::B(b) => b.poll() - .map(|v| v.map(EitherOutput::Second)) - .map_err(EitherError::B) + type Output = Result, EitherError>; + + #[project] + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + #[project] + match self.project() { + EitherFuture2::A(a) => TryFuture::try_poll(a, cx) + .map_ok(EitherOutput::First).map_err(EitherError::A), + EitherFuture2::B(a) => TryFuture::try_poll(a, cx) + .map_ok(EitherOutput::Second).map_err(EitherError::B), } } } diff --git a/core/src/identity/ed25519.rs b/core/src/identity/ed25519.rs index 17b1dc27066..f68e5d03591 100644 --- a/core/src/identity/ed25519.rs +++ b/core/src/identity/ed25519.rs @@ -21,7 +21,6 @@ //! Ed25519 keys. use ed25519_dalek as ed25519; -use failure::Fail; use rand::RngCore; use super::error::DecodingError; use zeroize::Zeroize; @@ -48,7 +47,7 @@ impl Keypair { pub fn decode(kp: &mut [u8]) -> Result { ed25519::Keypair::from_bytes(kp) .map(|k| { kp.zeroize(); Keypair(k) }) - .map_err(|e| DecodingError::new("Ed25519 keypair").source(e.compat())) + .map_err(|e| DecodingError::new("Ed25519 keypair").source(e)) } /// Sign a message using the private key of this keypair. @@ -120,7 +119,7 @@ impl PublicKey { /// Decode a public key from a byte array as produced by `encode`. pub fn decode(k: &[u8]) -> Result { ed25519::PublicKey::from_bytes(k) - .map_err(|e| DecodingError::new("Ed25519 public key").source(e.compat())) + .map_err(|e| DecodingError::new("Ed25519 public key").source(e)) .map(PublicKey) } } @@ -164,7 +163,7 @@ impl SecretKey { pub fn from_bytes(mut sk_bytes: impl AsMut<[u8]>) -> Result { let sk_bytes = sk_bytes.as_mut(); let secret = ed25519::SecretKey::from_bytes(&*sk_bytes) - .map_err(|e| DecodingError::new("Ed25519 secret key").source(e.compat()))?; + .map_err(|e| DecodingError::new("Ed25519 secret key").source(e))?; sk_bytes.zeroize(); Ok(SecretKey(secret)) } diff --git a/core/src/lib.rs b/core/src/lib.rs index c3276415c67..f6af9c10dc8 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -37,15 +37,12 @@ /// Multi-address re-export. pub use multiaddr; -pub use multistream_select::Negotiated; +pub type Negotiated = futures::compat::Compat01As03>>; mod keys_proto; mod peer_id; mod translation; -#[cfg(test)] -mod tests; - pub mod either; pub mod identity; pub mod muxing; diff --git a/core/src/muxing.rs b/core/src/muxing.rs index 28245666d1c..c6a8aa6862c 100644 --- a/core/src/muxing.rs +++ b/core/src/muxing.rs @@ -52,13 +52,9 @@ //! implementation of `StreamMuxer` to control everything that happens on the wire. use fnv::FnvHashMap; -use futures::{future, prelude::*, try_ready}; +use futures::{future, prelude::*, task::Context, task::Poll}; use parking_lot::Mutex; -use std::io::{self, Read, Write}; -use std::ops::Deref; -use std::fmt; -use std::sync::atomic::{AtomicUsize, Ordering}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{io, ops::Deref, fmt, pin::Pin, sync::atomic::{AtomicUsize, Ordering}}; pub use self::singleton::SingletonMuxer; @@ -90,12 +86,12 @@ pub trait StreamMuxer { /// /// This function behaves the same as a `Stream`. /// - /// If `NotReady` is returned, then the current task will be notified once the muxer + /// If `Pending` is returned, then the current task will be notified once the muxer /// is ready to be polled, similar to the API of `Stream::poll()`. /// Only the latest task that was used to call this method may be notified. /// /// An error can be generated if the connection has been closed. - fn poll_inbound(&self) -> Poll; + fn poll_inbound(&self, cx: &mut Context) -> Poll>; /// Opens a new outgoing substream, and produces the equivalent to a future that will be /// resolved when it becomes available. @@ -106,22 +102,23 @@ pub trait StreamMuxer { /// Polls the outbound substream. /// - /// If `NotReady` is returned, then the current task will be notified once the substream + /// If `Pending` is returned, then the current task will be notified once the substream /// is ready to be polled, similar to the API of `Future::poll()`. /// However, for each individual outbound substream, only the latest task that was used to /// call this method may be notified. /// /// May panic or produce an undefined result if an earlier polling of the same substream /// returned `Ready` or `Err`. - fn poll_outbound(&self, s: &mut Self::OutboundSubstream) -> Poll; + fn poll_outbound(&self, cx: &mut Context, s: &mut Self::OutboundSubstream) + -> Poll>; /// Destroys an outbound substream future. Use this after the outbound substream has finished, /// or if you want to interrupt it. fn destroy_outbound(&self, s: Self::OutboundSubstream); - /// Reads data from a substream. The behaviour is the same as `tokio_io::AsyncRead::poll_read`. + /// Reads data from a substream. The behaviour is the same as `futures::AsyncRead::poll_read`. /// - /// If `NotReady` is returned, then the current task will be notified once the substream + /// If `Pending` is returned, then the current task will be notified once the substream /// is ready to be read. However, for each individual substream, only the latest task that /// was used to call this method may be notified. /// @@ -130,25 +127,12 @@ pub trait StreamMuxer { /// /// An error can be generated if the connection has been closed, or if a protocol misbehaviour /// happened. - fn read_substream(&self, s: &mut Self::Substream, buf: &mut [u8]) -> Poll; + fn read_substream(&self, cx: &mut Context, s: &mut Self::Substream, buf: &mut [u8]) + -> Poll>; - /// Mimics the `prepare_uninitialized_buffer` method of the `AsyncRead` trait. + /// Write data to a substream. The behaviour is the same as `futures::AsyncWrite::poll_write`. /// - /// This function isn't actually unsafe to call but unsafe to implement. The implementer must - /// ensure that either the whole buf has been zeroed or that `read_substream` overwrites the - /// buffer without reading it and returns correct value. - /// - /// If this function returns true, then the memory has been zeroed out. This allows - /// implementations of `AsyncRead` which are composed of multiple subimplementations to - /// efficiently implement `prepare_uninitialized_buffer`. - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - for b in buf.iter_mut() { *b = 0; } - true - } - - /// Write data to a substream. The behaviour is the same as `tokio_io::AsyncWrite::poll_write`. - /// - /// If `NotReady` is returned, then the current task will be notified once the substream + /// If `Pending` is returned, then the current task will be notified once the substream /// is ready to be read. For each individual substream, only the latest task that was used to /// call this method may be notified. /// @@ -157,24 +141,26 @@ pub trait StreamMuxer { /// /// It is incorrect to call this method on a substream if you called `shutdown_substream` on /// this substream earlier. - fn write_substream(&self, s: &mut Self::Substream, buf: &[u8]) -> Poll; + fn write_substream(&self, cx: &mut Context, s: &mut Self::Substream, buf: &[u8]) + -> Poll>; - /// Flushes a substream. The behaviour is the same as `tokio_io::AsyncWrite::poll_flush`. + /// Flushes a substream. The behaviour is the same as `futures::AsyncWrite::poll_flush`. /// /// After this method has been called, data written earlier on the substream is guaranteed to /// be received by the remote. /// - /// If `NotReady` is returned, then the current task will be notified once the substream + /// If `Pending` is returned, then the current task will be notified once the substream /// is ready to be read. For each individual substream, only the latest task that was used to /// call this method may be notified. /// /// > **Note**: This method may be implemented as a call to `flush_all`. - fn flush_substream(&self, s: &mut Self::Substream) -> Poll<(), Self::Error>; + fn flush_substream(&self, cx: &mut Context, s: &mut Self::Substream) + -> Poll>; /// Attempts to shut down the writing side of a substream. The behaviour is similar to - /// `tokio_io::AsyncWrite::shutdown`. + /// `AsyncWrite::poll_close`. /// - /// Contrary to `AsyncWrite::shutdown`, shutting down a substream does not imply + /// Contrary to `AsyncWrite::poll_close`, shutting down a substream does not imply /// `flush_substream`. If you want to make sure that the remote is immediately informed about /// the shutdown, use `flush_substream` or `flush_all`. /// @@ -182,7 +168,8 @@ pub trait StreamMuxer { /// /// An error can be generated if the connection has been closed, or if a protocol misbehaviour /// happened. - fn shutdown_substream(&self, s: &mut Self::Substream) -> Poll<(), Self::Error>; + fn shutdown_substream(&self, cx: &mut Context, s: &mut Self::Substream) + -> Poll>; /// Destroys a substream. fn destroy_substream(&self, s: Self::Substream); @@ -197,7 +184,7 @@ pub trait StreamMuxer { /// Closes this `StreamMuxer`. /// - /// After this has returned `Ok(Async::Ready(()))`, the muxer has become useless. All + /// After this has returned `Poll::Ready(Ok(()))`, the muxer has become useless. All /// subsequent reads must return either `EOF` or an error. All subsequent writes, shutdowns, /// or polls must generate an error or be ignored. /// @@ -207,14 +194,14 @@ pub trait StreamMuxer { /// > that the remote is properly informed of the shutdown. However, apart from /// > properly informing the remote, there is no difference between this and /// > immediately dropping the muxer. - fn close(&self) -> Poll<(), Self::Error>; + fn close(&self, cx: &mut Context) -> Poll>; /// Flush this `StreamMuxer`. /// /// This drains any write buffers of substreams and delivers any pending shutdown notifications /// due to `shutdown_substream` or `close`. One may thus shutdown groups of substreams /// followed by a final `flush_all` instead of having to do `flush_substream` for each. - fn flush_all(&self) -> Poll<(), Self::Error>; + fn flush_all(&self, cx: &mut Context) -> Poll>; } /// Polls for an inbound from the muxer but wraps the output in an object that @@ -222,14 +209,14 @@ pub trait StreamMuxer { #[inline] pub fn inbound_from_ref_and_wrap

( muxer: P, -) -> impl Future, Error = ::Error> +) -> impl Future, ::Error>> where P: Deref + Clone, P::Target: StreamMuxer, { let muxer2 = muxer.clone(); - future::poll_fn(move || muxer.poll_inbound()) - .map(|substream| substream_from_ref(muxer2, substream)) + future::poll_fn(move |cx| muxer.poll_inbound(cx)) + .map_ok(|substream| substream_from_ref(muxer2, substream)) } /// Same as `outbound_from_ref`, but wraps the output in an object that @@ -258,17 +245,16 @@ where P: Deref + Clone, P::Target: StreamMuxer, { - type Item = SubstreamRef

; - type Error = ::Error; + type Output = Result, ::Error>; - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(substream)) => { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + match Future::poll(Pin::new(&mut self.inner), cx) { + Poll::Ready(Ok(substream)) => { let out = substream_from_ref(self.inner.muxer.clone(), substream); - Ok(Async::Ready(out)) + Poll::Ready(Ok(out)) } - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => Err(err), + Poll::Pending => Poll::Pending, + Poll::Ready(Err(err)) => Poll::Ready(Err(err)), } } } @@ -297,18 +283,26 @@ where outbound: Option<::OutboundSubstream>, } +impl

Unpin for OutboundSubstreamRefFuture

+where + P: Deref, + P::Target: StreamMuxer, +{ +} + impl

Future for OutboundSubstreamRefFuture

where P: Deref, P::Target: StreamMuxer, { - type Item = ::Substream; - type Error = ::Error; + type Output = Result<::Substream, ::Error>; #[inline] - fn poll(&mut self) -> Poll { - self.muxer - .poll_outbound(self.outbound.as_mut().expect("outbound was empty")) + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + // We use a `this` because the compiler isn't smart enough to allow mutably borrowing + // multiple different fields from the `Pin` at the same time. + let this = &mut *self; + this.muxer.poll_outbound(cx, this.outbound.as_mut().expect("outbound was empty")) } } @@ -370,20 +364,11 @@ where } } - -impl

Read for SubstreamRef

+impl

Unpin for SubstreamRef

where P: Deref, P::Target: StreamMuxer, { - #[inline] - fn read(&mut self, buf: &mut [u8]) -> Result { - let s = self.substream.as_mut().expect("substream was empty"); - match self.muxer.read_substream(s, buf).map_err(|e| e.into())? { - Async::Ready(n) => Ok(n), - Async::NotReady => Err(io::ErrorKind::WouldBlock.into()) - } - } } impl

AsyncRead for SubstreamRef

@@ -391,75 +376,66 @@ where P: Deref, P::Target: StreamMuxer, { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.muxer.prepare_uninitialized_buffer(buf) - } + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + // We use a `this` because the compiler isn't smart enough to allow mutably borrowing + // multiple different fields from the `Pin` at the same time. + let this = &mut *self; - fn poll_read(&mut self, buf: &mut [u8]) -> Poll { - let s = self.substream.as_mut().expect("substream was empty"); - self.muxer.read_substream(s, buf).map_err(|e| e.into()) + let s = this.substream.as_mut().expect("substream was empty"); + this.muxer.read_substream(cx, s, buf).map_err(|e| e.into()) } } -impl

Write for SubstreamRef

+impl

AsyncWrite for SubstreamRef

where P: Deref, P::Target: StreamMuxer, { - #[inline] - fn write(&mut self, buf: &[u8]) -> Result { - let s = self.substream.as_mut().expect("substream was empty"); - match self.muxer.write_substream(s, buf).map_err(|e| e.into())? { - Async::Ready(n) => Ok(n), - Async::NotReady => Err(io::ErrorKind::WouldBlock.into()) - } - } + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll> { + // We use a `this` because the compiler isn't smart enough to allow mutably borrowing + // multiple different fields from the `Pin` at the same time. + let this = &mut *self; - #[inline] - fn flush(&mut self) -> Result<(), io::Error> { - let s = self.substream.as_mut().expect("substream was empty"); - match self.muxer.flush_substream(s).map_err(|e| e.into())? { - Async::Ready(()) => Ok(()), - Async::NotReady => Err(io::ErrorKind::WouldBlock.into()) - } + let s = this.substream.as_mut().expect("substream was empty"); + this.muxer.write_substream(cx, s, buf).map_err(|e| e.into()) } -} -impl

AsyncWrite for SubstreamRef

-where - P: Deref, - P::Target: StreamMuxer, -{ - #[inline] - fn poll_write(&mut self, buf: &[u8]) -> Poll { - let s = self.substream.as_mut().expect("substream was empty"); - self.muxer.write_substream(s, buf).map_err(|e| e.into()) - } + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + // We use a `this` because the compiler isn't smart enough to allow mutably borrowing + // multiple different fields from the `Pin` at the same time. + let this = &mut *self; - #[inline] - fn shutdown(&mut self) -> Poll<(), io::Error> { - let s = self.substream.as_mut().expect("substream was empty"); + let s = this.substream.as_mut().expect("substream was empty"); loop { - match self.shutdown_state { + match this.shutdown_state { ShutdownState::Shutdown => { - try_ready!(self.muxer.shutdown_substream(s).map_err(|e| e.into())); - self.shutdown_state = ShutdownState::Flush; + match this.muxer.shutdown_substream(cx, s) { + Poll::Ready(Ok(())) => this.shutdown_state = ShutdownState::Flush, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err.into())), + Poll::Pending => return Poll::Pending, + } } ShutdownState::Flush => { - try_ready!(self.muxer.flush_substream(s).map_err(|e| e.into())); - self.shutdown_state = ShutdownState::Done; + match this.muxer.flush_substream(cx, s) { + Poll::Ready(Ok(())) => this.shutdown_state = ShutdownState::Done, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err.into())), + Poll::Pending => return Poll::Pending, + } } ShutdownState::Done => { - return Ok(Async::Ready(())); + return Poll::Ready(Ok(())); } } } } - #[inline] - fn poll_flush(&mut self) -> Poll<(), io::Error> { - let s = self.substream.as_mut().expect("substream was empty"); - self.muxer.flush_substream(s).map_err(|e| e.into()) + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + // We use a `this` because the compiler isn't smart enough to allow mutably borrowing + // multiple different fields from the `Pin` at the same time. + let this = &mut *self; + + let s = this.substream.as_mut().expect("substream was empty"); + this.muxer.flush_substream(cx, s).map_err(|e| e.into()) } } @@ -507,8 +483,8 @@ impl StreamMuxer for StreamMuxerBox { type Error = io::Error; #[inline] - fn poll_inbound(&self) -> Poll { - self.inner.poll_inbound() + fn poll_inbound(&self, cx: &mut Context) -> Poll> { + self.inner.poll_inbound(cx) } #[inline] @@ -517,8 +493,8 @@ impl StreamMuxer for StreamMuxerBox { } #[inline] - fn poll_outbound(&self, s: &mut Self::OutboundSubstream) -> Poll { - self.inner.poll_outbound(s) + fn poll_outbound(&self, cx: &mut Context, s: &mut Self::OutboundSubstream) -> Poll> { + self.inner.poll_outbound(cx, s) } #[inline] @@ -526,28 +502,24 @@ impl StreamMuxer for StreamMuxerBox { self.inner.destroy_outbound(substream) } - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.inner.prepare_uninitialized_buffer(buf) - } - #[inline] - fn read_substream(&self, s: &mut Self::Substream, buf: &mut [u8]) -> Poll { - self.inner.read_substream(s, buf) + fn read_substream(&self, cx: &mut Context, s: &mut Self::Substream, buf: &mut [u8]) -> Poll> { + self.inner.read_substream(cx, s, buf) } #[inline] - fn write_substream(&self, s: &mut Self::Substream, buf: &[u8]) -> Poll { - self.inner.write_substream(s, buf) + fn write_substream(&self, cx: &mut Context, s: &mut Self::Substream, buf: &[u8]) -> Poll> { + self.inner.write_substream(cx, s, buf) } #[inline] - fn flush_substream(&self, s: &mut Self::Substream) -> Poll<(), Self::Error> { - self.inner.flush_substream(s) + fn flush_substream(&self, cx: &mut Context, s: &mut Self::Substream) -> Poll> { + self.inner.flush_substream(cx, s) } #[inline] - fn shutdown_substream(&self, s: &mut Self::Substream) -> Poll<(), Self::Error> { - self.inner.shutdown_substream(s) + fn shutdown_substream(&self, cx: &mut Context, s: &mut Self::Substream) -> Poll> { + self.inner.shutdown_substream(cx, s) } #[inline] @@ -556,8 +528,8 @@ impl StreamMuxer for StreamMuxerBox { } #[inline] - fn close(&self) -> Poll<(), Self::Error> { - self.inner.close() + fn close(&self, cx: &mut Context) -> Poll> { + self.inner.close(cx) } #[inline] @@ -566,8 +538,8 @@ impl StreamMuxer for StreamMuxerBox { } #[inline] - fn flush_all(&self) -> Poll<(), Self::Error> { - self.inner.flush_all() + fn flush_all(&self, cx: &mut Context) -> Poll> { + self.inner.flush_all(cx) } } @@ -588,11 +560,16 @@ where type Error = io::Error; #[inline] - fn poll_inbound(&self) -> Poll { - let substream = try_ready!(self.inner.poll_inbound().map_err(|e| e.into())); + fn poll_inbound(&self, cx: &mut Context) -> Poll> { + let substream = match self.inner.poll_inbound(cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(Ok(s)) => s, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err.into())), + }; + let id = self.next_substream.fetch_add(1, Ordering::Relaxed); self.substreams.lock().insert(id, substream); - Ok(Async::Ready(id)) + Poll::Ready(Ok(id)) } #[inline] @@ -606,13 +583,18 @@ where #[inline] fn poll_outbound( &self, + cx: &mut Context, substream: &mut Self::OutboundSubstream, - ) -> Poll { + ) -> Poll> { let mut list = self.outbound.lock(); - let substream = try_ready!(self.inner.poll_outbound(list.get_mut(substream).unwrap()).map_err(|e| e.into())); + let substream = match self.inner.poll_outbound(cx, list.get_mut(substream).unwrap()) { + Poll::Pending => return Poll::Pending, + Poll::Ready(Ok(s)) => s, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err.into())), + }; let id = self.next_substream.fetch_add(1, Ordering::Relaxed); self.substreams.lock().insert(id, substream); - Ok(Async::Ready(id)) + Poll::Ready(Ok(id)) } #[inline] @@ -621,32 +603,28 @@ where self.inner.destroy_outbound(list.remove(&substream).unwrap()) } - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.inner.prepare_uninitialized_buffer(buf) - } - #[inline] - fn read_substream(&self, s: &mut Self::Substream, buf: &mut [u8]) -> Poll { + fn read_substream(&self, cx: &mut Context, s: &mut Self::Substream, buf: &mut [u8]) -> Poll> { let mut list = self.substreams.lock(); - self.inner.read_substream(list.get_mut(s).unwrap(), buf).map_err(|e| e.into()) + self.inner.read_substream(cx, list.get_mut(s).unwrap(), buf).map_err(|e| e.into()) } #[inline] - fn write_substream(&self, s: &mut Self::Substream, buf: &[u8]) -> Poll { + fn write_substream(&self, cx: &mut Context, s: &mut Self::Substream, buf: &[u8]) -> Poll> { let mut list = self.substreams.lock(); - self.inner.write_substream(list.get_mut(s).unwrap(), buf).map_err(|e| e.into()) + self.inner.write_substream(cx, list.get_mut(s).unwrap(), buf).map_err(|e| e.into()) } #[inline] - fn flush_substream(&self, s: &mut Self::Substream) -> Poll<(), Self::Error> { + fn flush_substream(&self, cx: &mut Context, s: &mut Self::Substream) -> Poll> { let mut list = self.substreams.lock(); - self.inner.flush_substream(list.get_mut(s).unwrap()).map_err(|e| e.into()) + self.inner.flush_substream(cx, list.get_mut(s).unwrap()).map_err(|e| e.into()) } #[inline] - fn shutdown_substream(&self, s: &mut Self::Substream) -> Poll<(), Self::Error> { + fn shutdown_substream(&self, cx: &mut Context, s: &mut Self::Substream) -> Poll> { let mut list = self.substreams.lock(); - self.inner.shutdown_substream(list.get_mut(s).unwrap()).map_err(|e| e.into()) + self.inner.shutdown_substream(cx, list.get_mut(s).unwrap()).map_err(|e| e.into()) } #[inline] @@ -656,8 +634,8 @@ where } #[inline] - fn close(&self) -> Poll<(), Self::Error> { - self.inner.close().map_err(|e| e.into()) + fn close(&self, cx: &mut Context) -> Poll> { + self.inner.close(cx).map_err(|e| e.into()) } #[inline] @@ -666,7 +644,7 @@ where } #[inline] - fn flush_all(&self) -> Poll<(), Self::Error> { - self.inner.flush_all().map_err(|e| e.into()) + fn flush_all(&self, cx: &mut Context) -> Poll> { + self.inner.flush_all(cx).map_err(|e| e.into()) } } diff --git a/core/src/muxing/singleton.rs b/core/src/muxing/singleton.rs index 7bec14edf6d..c2b56d0c131 100644 --- a/core/src/muxing/singleton.rs +++ b/core/src/muxing/singleton.rs @@ -21,8 +21,7 @@ use crate::{Endpoint, muxing::StreamMuxer}; use futures::prelude::*; use parking_lot::Mutex; -use std::{io, sync::atomic::{AtomicBool, Ordering}}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{io, pin::Pin, sync::atomic::{AtomicBool, Ordering}, task::Context, task::Poll}; /// Implementation of `StreamMuxer` that allows only one substream on top of a connection, /// yielding the connection itself. @@ -62,22 +61,22 @@ pub struct OutboundSubstream {} impl StreamMuxer for SingletonMuxer where - TSocket: AsyncRead + AsyncWrite, + TSocket: AsyncRead + AsyncWrite + Unpin, { type Substream = Substream; type OutboundSubstream = OutboundSubstream; type Error = io::Error; - fn poll_inbound(&self) -> Poll { + fn poll_inbound(&self, _: &mut Context) -> Poll> { match self.endpoint { - Endpoint::Dialer => return Ok(Async::NotReady), + Endpoint::Dialer => return Poll::Pending, Endpoint::Listener => {} } if !self.substream_extracted.swap(true, Ordering::Relaxed) { - Ok(Async::Ready(Substream {})) + Poll::Ready(Ok(Substream {})) } else { - Ok(Async::NotReady) + Poll::Pending } } @@ -85,44 +84,40 @@ where OutboundSubstream {} } - fn poll_outbound(&self, _: &mut Self::OutboundSubstream) -> Poll { + fn poll_outbound(&self, _: &mut Context, _: &mut Self::OutboundSubstream) -> Poll> { match self.endpoint { - Endpoint::Listener => return Ok(Async::NotReady), + Endpoint::Listener => return Poll::Pending, Endpoint::Dialer => {} } if !self.substream_extracted.swap(true, Ordering::Relaxed) { - Ok(Async::Ready(Substream {})) + Poll::Ready(Ok(Substream {})) } else { - Ok(Async::NotReady) + Poll::Pending } } fn destroy_outbound(&self, _: Self::OutboundSubstream) { } - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.inner.lock().prepare_uninitialized_buffer(buf) - } - - fn read_substream(&self, _: &mut Self::Substream, buf: &mut [u8]) -> Poll { - let res = self.inner.lock().poll_read(buf); - if let Ok(Async::Ready(_)) = res { + fn read_substream(&self, cx: &mut Context, _: &mut Self::Substream, buf: &mut [u8]) -> Poll> { + let res = AsyncRead::poll_read(Pin::new(&mut *self.inner.lock()), cx, buf); + if let Poll::Ready(Ok(_)) = res { self.remote_acknowledged.store(true, Ordering::Release); } res } - fn write_substream(&self, _: &mut Self::Substream, buf: &[u8]) -> Poll { - self.inner.lock().poll_write(buf) + fn write_substream(&self, cx: &mut Context, _: &mut Self::Substream, buf: &[u8]) -> Poll> { + AsyncWrite::poll_write(Pin::new(&mut *self.inner.lock()), cx, buf) } - fn flush_substream(&self, _: &mut Self::Substream) -> Poll<(), io::Error> { - self.inner.lock().poll_flush() + fn flush_substream(&self, cx: &mut Context, _: &mut Self::Substream) -> Poll> { + AsyncWrite::poll_flush(Pin::new(&mut *self.inner.lock()), cx) } - fn shutdown_substream(&self, _: &mut Self::Substream) -> Poll<(), io::Error> { - self.inner.lock().shutdown() + fn shutdown_substream(&self, cx: &mut Context, _: &mut Self::Substream) -> Poll> { + AsyncWrite::poll_close(Pin::new(&mut *self.inner.lock()), cx) } fn destroy_substream(&self, _: Self::Substream) { @@ -132,12 +127,12 @@ where self.remote_acknowledged.load(Ordering::Acquire) } - fn close(&self) -> Poll<(), io::Error> { + fn close(&self, cx: &mut Context) -> Poll> { // The `StreamMuxer` trait requires that `close()` implies `flush_all()`. - self.flush_all() + self.flush_all(cx) } - fn flush_all(&self) -> Poll<(), io::Error> { - self.inner.lock().poll_flush() + fn flush_all(&self, cx: &mut Context) -> Poll> { + AsyncWrite::poll_flush(Pin::new(&mut *self.inner.lock()), cx) } } diff --git a/core/src/nodes/collection.rs b/core/src/nodes/collection.rs index af8601d27ec..f1f003d3ae4 100644 --- a/core/src/nodes/collection.rs +++ b/core/src/nodes/collection.rs @@ -29,11 +29,7 @@ use crate::{ }; use fnv::FnvHashMap; use futures::prelude::*; -use std::{error, fmt, hash::Hash, mem}; - -pub use crate::nodes::tasks::StartTakeOver; - -mod tests; +use std::{error, fmt, hash::Hash, mem, task::Context, task::Poll}; /// Implementation of `Stream` that handles a collection of nodes. pub struct CollectionStream { @@ -58,6 +54,9 @@ where } } +impl Unpin for + CollectionStream { } + /// State of a task. #[derive(Debug, Clone, PartialEq, Eq)] enum TaskState { @@ -323,7 +322,7 @@ where pub fn add_reach_attempt(&mut self, future: TFut, handler: THandler) -> ReachAttemptId where - TFut: Future + Send + 'static, + TFut: Future> + Send + 'static, THandler: IntoNodeHandler + Send + 'static, THandler::Handler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, ::OutboundOpenInfo: Send + 'static, @@ -358,17 +357,15 @@ where } /// Sends an event to all nodes. + /// + /// This function is "atomic", in the sense that if `Poll::Pending` is returned then no event + /// has been sent to any node yet. #[must_use] - pub fn start_broadcast(&mut self, event: &TInEvent) -> AsyncSink<()> + pub fn poll_broadcast(&mut self, event: &TInEvent, cx: &mut Context) -> Poll<()> where TInEvent: Clone { - self.inner.start_broadcast(event) - } - - #[must_use] - pub fn complete_broadcast(&mut self) -> Async<()> { - self.inner.complete_broadcast() + self.inner.poll_broadcast(event, cx) } /// Adds an existing connection to a node to the collection. @@ -447,13 +444,13 @@ where /// > **Note**: we use a regular `poll` method instead of implementing `Stream` in order to /// > remove the `Err` variant, but also because we want the `CollectionStream` to stay /// > borrowed if necessary. - pub fn poll(&mut self) -> Async> + pub fn poll(&mut self, cx: &mut Context) -> Poll> where TConnInfo: Clone, // TODO: Clone shouldn't be necessary { - let item = match self.inner.poll() { - Async::Ready(item) => item, - Async::NotReady => return Async::NotReady, + let item = match self.inner.poll(cx) { + Poll::Ready(item) => item, + Poll::Pending => return Poll::Pending, }; match item { @@ -463,7 +460,7 @@ where match (user_data, result, handler) { (TaskState::Pending, tasks::Error::Reach(err), Some(handler)) => { - Async::Ready(CollectionEvent::ReachError { + Poll::Ready(CollectionEvent::ReachError { id: ReachAttemptId(id), error: err, handler, @@ -482,7 +479,7 @@ where debug_assert!(_handler.is_none()); let _node_task_id = self.nodes.remove(conn_info.peer_id()); debug_assert_eq!(_node_task_id, Some(id)); - Async::Ready(CollectionEvent::NodeClosed { + Poll::Ready(CollectionEvent::NodeClosed { conn_info, error: err, user_data, @@ -497,8 +494,8 @@ where tasks::Event::NodeReached { task, conn_info } => { let id = task.id(); drop(task); - Async::Ready(CollectionEvent::NodeReached(CollectionReachEvent { - parent: self, + Poll::Ready(CollectionEvent::NodeReached(CollectionReachEvent { + parent: &mut *self, id, conn_info: Some(conn_info), })) @@ -512,7 +509,7 @@ where self.tasks is switched to the Connected state; QED"), }; drop(task); - Async::Ready(CollectionEvent::NodeEvent { + Poll::Ready(CollectionEvent::NodeEvent { // TODO: normally we'd build a `PeerMut` manually here, but the borrow checker // doesn't like it peer: self.peer_mut(&conn_info.peer_id()) @@ -616,14 +613,15 @@ where } } - /// Sends an event to the given node. - pub fn start_send_event(&mut self, event: TInEvent) -> StartSend { + /// Begin sending an event to the given node. Must be called only after a successful call to + /// `poll_ready_event`. + pub fn start_send_event(&mut self, event: TInEvent) { self.inner.start_send_event(event) } - /// Complete sending an event message initiated by `start_send_event`. - pub fn complete_send_event(&mut self) -> Poll<(), ()> { - self.inner.complete_send_event() + /// Make sure we are ready to accept an event to be sent with `start_send_event`. + pub fn poll_ready_event(&mut self, cx: &mut Context) -> Poll<()> { + self.inner.poll_ready_event(cx) } /// Closes the connections to this node. Returns the user data. @@ -648,23 +646,13 @@ where /// The reach attempt will only be effectively cancelled once the peer (the object you're /// manipulating) has received some network activity. However no event will be ever be /// generated from this reach attempt, and this takes effect immediately. - #[must_use] - pub fn start_take_over(&mut self, id: InterruptedReachAttempt) - -> StartTakeOver<(), InterruptedReachAttempt> - { - match self.inner.start_take_over(id.inner) { - StartTakeOver::Ready(_state) => { - debug_assert!(if let TaskState::Pending = _state { true } else { false }); - StartTakeOver::Ready(()) - } - StartTakeOver::NotReady(inner) => - StartTakeOver::NotReady(InterruptedReachAttempt { inner }), - StartTakeOver::Gone => StartTakeOver::Gone - } + pub fn start_take_over(&mut self, id: InterruptedReachAttempt) { + self.inner.start_take_over(id.inner) } - /// Complete a take over initiated by `start_take_over`. - pub fn complete_take_over(&mut self) -> Poll<(), ()> { - self.inner.complete_take_over() + /// Make sure we are ready to taking over with `start_take_over`. + #[must_use] + pub fn poll_ready_take_over(&mut self, cx: &mut Context) -> Poll<()> { + self.inner.poll_ready_take_over(cx) } } diff --git a/core/src/nodes/collection/tests.rs b/core/src/nodes/collection/tests.rs deleted file mode 100644 index 69f82c05428..00000000000 --- a/core/src/nodes/collection/tests.rs +++ /dev/null @@ -1,373 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -#![cfg(test)] - -use super::*; -use assert_matches::assert_matches; -use futures::future; -use crate::tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; -use crate::tests::dummy_handler::{Handler, InEvent, OutEvent, HandlerState}; -use tokio::runtime::current_thread::Runtime; -use tokio::runtime::Builder; -use crate::nodes::NodeHandlerEvent; -use std::{io, sync::Arc}; -use parking_lot::Mutex; - -type TestCollectionStream = CollectionStream; - -#[test] -fn has_connection_is_false_before_a_connection_has_been_made() { - let cs = TestCollectionStream::new(); - let peer_id = PeerId::random(); - assert!(!cs.has_connection(&peer_id)); -} - -#[test] -fn connections_is_empty_before_connecting() { - let cs = TestCollectionStream::new(); - assert!(cs.connections().next().is_none()); -} - -#[test] -fn retrieving_a_peer_is_none_if_peer_is_missing_or_not_connected() { - let mut cs = TestCollectionStream::new(); - let peer_id = PeerId::random(); - assert!(cs.peer_mut(&peer_id).is_none()); - - let handler = Handler::default(); - let fut = future::ok((peer_id.clone(), DummyMuxer::new())); - cs.add_reach_attempt(fut, handler); - assert!(cs.peer_mut(&peer_id).is_none()); // task is pending -} - -#[test] -fn collection_stream_reaches_the_nodes() { - let mut cs = TestCollectionStream::new(); - let peer_id = PeerId::random(); - - let mut muxer = DummyMuxer::new(); - muxer.set_inbound_connection_state(DummyConnectionState::Pending); - muxer.set_outbound_connection_state(DummyConnectionState::Opened); - - let fut = future::ok((peer_id, muxer)); - cs.add_reach_attempt(fut, Handler::default()); - let mut rt = Runtime::new().unwrap(); - let mut poll_count = 0; - let fut = future::poll_fn(move || -> Poll<(), ()> { - poll_count += 1; - let event = cs.poll(); - match poll_count { - 1 => assert_matches!(event, Async::NotReady), - 2 => { - assert_matches!(event, Async::Ready(CollectionEvent::NodeReached(_))); - return Ok(Async::Ready(())); // stop - } - _ => unreachable!() - } - Ok(Async::NotReady) - }); - rt.block_on(fut).unwrap(); -} - -#[test] -fn accepting_a_node_yields_new_entry() { - let mut cs = TestCollectionStream::new(); - let peer_id = PeerId::random(); - let fut = future::ok((peer_id.clone(), DummyMuxer::new())); - cs.add_reach_attempt(fut, Handler::default()); - - let mut rt = Runtime::new().unwrap(); - let mut poll_count = 0; - let fut = future::poll_fn(move || -> Poll<(), ()> { - poll_count += 1; - { - let event = cs.poll(); - match poll_count { - 1 => { - assert_matches!(event, Async::NotReady); - return Ok(Async::NotReady) - } - 2 => { - assert_matches!(event, Async::Ready(CollectionEvent::NodeReached(reach_ev)) => { - let (accept_ev, accepted_peer_id) = reach_ev.accept(()); - assert_eq!(accepted_peer_id, peer_id); - assert_matches!(accept_ev, CollectionNodeAccept::NewEntry); - }); - } - _ => unreachable!() - } - } - assert!(cs.peer_mut(&peer_id).is_some(), "peer is not in the list"); - assert!(cs.has_connection(&peer_id), "peer is not connected"); - assert_eq!(cs.connections().collect::>(), vec![&peer_id]); - Ok(Async::Ready(())) - }); - rt.block_on(fut).expect("running the future works"); -} - -#[test] -fn events_in_a_node_reaches_the_collection_stream() { - let cs = Arc::new(Mutex::new(TestCollectionStream::new())); - let task_peer_id = PeerId::random(); - - let mut handler = Handler::default(); - handler.state = Some(HandlerState::Ready(NodeHandlerEvent::Custom(OutEvent::Custom("init")))); - let handler_states = vec![ - HandlerState::Err, - HandlerState::Ready(NodeHandlerEvent::Custom(OutEvent::Custom("from handler 3") )), - HandlerState::Ready(NodeHandlerEvent::Custom(OutEvent::Custom("from handler 2") )), - HandlerState::Ready(NodeHandlerEvent::Custom(OutEvent::Custom("from handler 1") )), - ]; - handler.next_states = handler_states; - - let mut muxer = DummyMuxer::new(); - muxer.set_inbound_connection_state(DummyConnectionState::Pending); - muxer.set_outbound_connection_state(DummyConnectionState::Opened); - - let fut = future::ok((task_peer_id.clone(), muxer)); - cs.lock().add_reach_attempt(fut, handler); - - let mut rt = Builder::new().core_threads(1).build().unwrap(); - - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - assert_matches!(cs.poll(), Async::NotReady); - Ok(Async::Ready(())) - })).expect("tokio works"); - - let cs2 = cs.clone(); - rt.block_on(future::poll_fn(move || { - if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - if cs.complete_broadcast().is_not_ready() { - return Ok(Async::NotReady) - } - assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeReached(reach_ev)) => { - reach_ev.accept(()); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); - - let cs2 = cs.clone(); - rt.block_on(future::poll_fn(move || { - if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - if cs.complete_broadcast().is_not_ready() { - return Ok(Async::NotReady) - } - assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => { - assert_matches!(event, OutEvent::Custom("init")); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); - - - let cs2 = cs.clone(); - rt.block_on(future::poll_fn(move || { - if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - if cs.complete_broadcast().is_not_ready() { - return Ok(Async::NotReady) - } - assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => { - assert_matches!(event, OutEvent::Custom("from handler 1")); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); - - let cs2 = cs.clone(); - rt.block_on(future::poll_fn(move || { - if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - if cs.complete_broadcast().is_not_ready() { - return Ok(Async::NotReady) - } - assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => { - assert_matches!(event, OutEvent::Custom("from handler 2")); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); -} - -#[test] -fn task_closed_with_error_while_task_is_pending_yields_reach_error() { - let cs = Arc::new(Mutex::new(TestCollectionStream::new())); - let task_inner_fut = future::err(std::io::Error::new(std::io::ErrorKind::Other, "inner fut error")); - let reach_attempt_id = cs.lock().add_reach_attempt(task_inner_fut, Handler::default()); - - let mut rt = Builder::new().core_threads(1).build().unwrap(); - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - assert_matches!(cs.poll(), Async::NotReady); - Ok(Async::Ready(())) - })).expect("tokio works"); - - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - assert_matches!(cs.poll(), Async::Ready(collection_ev) => { - assert_matches!(collection_ev, CollectionEvent::ReachError {id, error, ..} => { - assert_eq!(id, reach_attempt_id); - assert_eq!(error.to_string(), "inner fut error"); - }); - - }); - Ok(Async::Ready(())) - })).expect("tokio works"); - -} - -#[test] -fn task_closed_with_error_when_task_is_connected_yields_node_error() { - let cs = Arc::new(Mutex::new(TestCollectionStream::new())); - let peer_id = PeerId::random(); - let muxer = DummyMuxer::new(); - let task_inner_fut = future::ok((peer_id.clone(), muxer)); - let mut handler = Handler::default(); - handler.next_states = vec![HandlerState::Err]; // triggered when sending a NextState event - - cs.lock().add_reach_attempt(task_inner_fut, handler); - let mut rt = Builder::new().core_threads(1).build().unwrap(); - - // Kick it off - let cs2 = cs.clone(); - rt.block_on(future::poll_fn(move || { - if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - assert_matches!(cs.poll(), Async::NotReady); - // send an event so the Handler errors in two polls - Ok(cs.complete_broadcast()) - })).expect("tokio works"); - - // Accept the new node - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - // NodeReached, accept the connection so the task transitions from Pending to Connected - assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeReached(reach_ev)) => { - reach_ev.accept(()); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); - - assert!(cs.lock().has_connection(&peer_id)); - - // Assert the node errored - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - assert_matches!(cs.poll(), Async::Ready(collection_ev) => { - assert_matches!(collection_ev, CollectionEvent::NodeClosed{..}); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); -} - -#[test] -fn interrupting_a_pending_connection_attempt_is_ok() { - let mut cs = TestCollectionStream::new(); - let fut = future::empty(); - let reach_id = cs.add_reach_attempt(fut, Handler::default()); - let interrupt = cs.interrupt(reach_id); - assert!(interrupt.is_ok()); -} - -#[test] -fn interrupting_a_connection_attempt_twice_is_err() { - let mut cs = TestCollectionStream::new(); - let fut = future::empty(); - let reach_id = cs.add_reach_attempt(fut, Handler::default()); - assert!(cs.interrupt(reach_id).is_ok()); - assert_matches!(cs.interrupt(reach_id), Err(InterruptError::ReachAttemptNotFound)) -} - -#[test] -fn interrupting_an_established_connection_is_err() { - let cs = Arc::new(Mutex::new(TestCollectionStream::new())); - let peer_id = PeerId::random(); - let muxer = DummyMuxer::new(); - let task_inner_fut = future::ok((peer_id.clone(), muxer)); - let handler = Handler::default(); - - let reach_id = cs.lock().add_reach_attempt(task_inner_fut, handler); - let mut rt = Builder::new().core_threads(1).build().unwrap(); - - // Kick it off - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - assert_matches!(cs.poll(), Async::NotReady); - // send an event so the Handler errors in two polls - Ok(Async::Ready(())) - })).expect("tokio works"); - - // Accept the new node - let cs_fut = cs.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut cs = cs_fut.lock(); - // NodeReached, accept the connection so the task transitions from Pending to Connected - assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeReached(reach_ev)) => { - reach_ev.accept(()); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); - - assert!(cs.lock().has_connection(&peer_id), "Connection was not established"); - - assert_matches!(cs.lock().interrupt(reach_id), Err(InterruptError::AlreadyReached)); -} diff --git a/core/src/nodes/handled_node.rs b/core/src/nodes/handled_node.rs index 150b5e45060..f8b08d11e21 100644 --- a/core/src/nodes/handled_node.rs +++ b/core/src/nodes/handled_node.rs @@ -20,10 +20,7 @@ use crate::{PeerId, muxing::StreamMuxer}; use crate::nodes::node::{NodeEvent, NodeStream, Substream, Close}; -use futures::prelude::*; -use std::{error, fmt, io}; - -mod tests; +use std::{error, fmt, io, pin::Pin, task::Context, task::Poll}; /// Handler for the substreams of a node. // TODO: right now it is possible for a node handler to be built, then shut down right after if we @@ -59,7 +56,8 @@ pub trait NodeHandler { /// Should behave like `Stream::poll()`. /// /// Returning an error will close the connection to the remote. - fn poll(&mut self) -> Poll, Self::Error>; + fn poll(&mut self, cx: &mut Context) + -> Poll, Self::Error>>; } /// Prototype for a `NodeHandler`. @@ -172,6 +170,13 @@ where } } +impl Unpin for HandledNode +where + TMuxer: StreamMuxer, + THandler: NodeHandler>, +{ +} + impl HandledNode where TMuxer: StreamMuxer, @@ -214,37 +219,41 @@ where } /// API similar to `Future::poll` that polls the node for events. - pub fn poll(&mut self) -> Poll> { + pub fn poll(mut self: Pin<&mut Self>, cx: &mut Context) + -> Poll>> + { loop { let mut node_not_ready = false; - match self.node.poll().map_err(HandledNodeError::Node)? { - Async::NotReady => node_not_ready = true, - Async::Ready(NodeEvent::InboundSubstream { substream }) => { + match self.node.poll(cx) { + Poll::Pending => node_not_ready = true, + Poll::Ready(Ok(NodeEvent::InboundSubstream { substream })) => { self.handler.inject_substream(substream, NodeHandlerEndpoint::Listener) } - Async::Ready(NodeEvent::OutboundSubstream { user_data, substream }) => { + Poll::Ready(Ok(NodeEvent::OutboundSubstream { user_data, substream })) => { let endpoint = NodeHandlerEndpoint::Dialer(user_data); self.handler.inject_substream(substream, endpoint) } + Poll::Ready(Err(err)) => return Poll::Ready(Err(HandledNodeError::Node(err))), } - match self.handler.poll().map_err(HandledNodeError::Handler)? { - Async::NotReady => { + match self.handler.poll(cx) { + Poll::Pending => { if node_not_ready { break } } - Async::Ready(NodeHandlerEvent::OutboundSubstreamRequest(user_data)) => { + Poll::Ready(Ok(NodeHandlerEvent::OutboundSubstreamRequest(user_data))) => { self.node.open_substream(user_data); } - Async::Ready(NodeHandlerEvent::Custom(event)) => { - return Ok(Async::Ready(event)); + Poll::Ready(Ok(NodeHandlerEvent::Custom(event))) => { + return Poll::Ready(Ok(event)); } + Poll::Ready(Err(err)) => return Poll::Ready(Err(HandledNodeError::Handler(err))), } } - Ok(Async::NotReady) + Poll::Pending } } diff --git a/core/src/nodes/handled_node/tests.rs b/core/src/nodes/handled_node/tests.rs deleted file mode 100644 index ee138c2e449..00000000000 --- a/core/src/nodes/handled_node/tests.rs +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -#![cfg(test)] - -use super::*; -use assert_matches::assert_matches; -use crate::tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; -use crate::tests::dummy_handler::{Handler, HandlerState, InEvent, OutEvent, TestHandledNode}; - -struct TestBuilder { - muxer: DummyMuxer, - handler: Handler, - want_open_substream: bool, - substream_user_data: usize, -} - -impl TestBuilder { - fn new() -> Self { - TestBuilder { - muxer: DummyMuxer::new(), - handler: Handler::default(), - want_open_substream: false, - substream_user_data: 0, - } - } - - fn with_muxer_inbound_state(&mut self, state: DummyConnectionState) -> &mut Self { - self.muxer.set_inbound_connection_state(state); - self - } - - fn with_muxer_outbound_state(&mut self, state: DummyConnectionState) -> &mut Self { - self.muxer.set_outbound_connection_state(state); - self - } - - fn with_handler_state(&mut self, state: HandlerState) -> &mut Self { - self.handler.state = Some(state); - self - } - - fn with_open_substream(&mut self, user_data: usize) -> &mut Self { - self.want_open_substream = true; - self.substream_user_data = user_data; - self - } - - fn handled_node(&mut self) -> TestHandledNode { - let mut h = HandledNode::new(self.muxer.clone(), self.handler.clone()); - if self.want_open_substream { - h.node.open_substream(self.substream_user_data); - } - h - } -} - -// Set the state of the `Handler` after `inject_outbound_closed` is called -fn set_next_handler_outbound_state( handled_node: &mut TestHandledNode, next_state: HandlerState) { - handled_node.handler.next_outbound_state = Some(next_state); -} - -#[test] -fn can_inject_event() { - let mut handled = TestBuilder::new() - .handled_node(); - - let event = InEvent::Custom("banana"); - handled.inject_event(event.clone()); - assert_eq!(handled.handler().events, vec![event]); -} - -#[test] -fn poll_with_unready_node_stream_and_handler_emits_custom_event() { - let expected_event = NodeHandlerEvent::Custom(OutEvent::Custom("pineapple")); - let mut handled = TestBuilder::new() - // make NodeStream return NotReady - .with_muxer_inbound_state(DummyConnectionState::Pending) - // make Handler return return Ready(Some(…)) - .with_handler_state(HandlerState::Ready(expected_event)) - .handled_node(); - - assert_matches!(handled.poll(), Ok(Async::Ready(event)) => { - assert_matches!(event, OutEvent::Custom("pineapple")) - }); -} - -#[test] -fn handler_emits_outbound_closed_when_opening_new_substream_on_closed_node() { - let open_event = NodeHandlerEvent::OutboundSubstreamRequest(456); - let mut handled = TestBuilder::new() - .with_muxer_inbound_state(DummyConnectionState::Pending) - .with_muxer_outbound_state(DummyConnectionState::Pending) - .with_handler_state(HandlerState::Ready(open_event)) - .handled_node(); - - set_next_handler_outbound_state( - &mut handled, - HandlerState::Ready(NodeHandlerEvent::Custom(OutEvent::Custom("pear"))) - ); - handled.poll().expect("poll works"); -} - -#[test] -fn poll_yields_inbound_closed_event() { - let mut h = TestBuilder::new() - .with_muxer_inbound_state(DummyConnectionState::Pending) - .with_handler_state(HandlerState::Err) // stop the loop - .handled_node(); - - assert_eq!(h.handler().events, vec![]); - let _ = h.poll(); -} - -#[test] -fn poll_yields_outbound_closed_event() { - let mut h = TestBuilder::new() - .with_muxer_inbound_state(DummyConnectionState::Pending) - .with_open_substream(32) - .with_muxer_outbound_state(DummyConnectionState::Pending) - .with_handler_state(HandlerState::Err) // stop the loop - .handled_node(); - - assert_eq!(h.handler().events, vec![]); - let _ = h.poll(); -} - -#[test] -fn poll_yields_outbound_substream() { - let mut h = TestBuilder::new() - .with_muxer_inbound_state(DummyConnectionState::Pending) - .with_muxer_outbound_state(DummyConnectionState::Opened) - .with_open_substream(1) - .with_handler_state(HandlerState::Err) // stop the loop - .handled_node(); - - assert_eq!(h.handler().events, vec![]); - let _ = h.poll(); - assert_eq!(h.handler().events, vec![InEvent::Substream(Some(1))]); -} - -#[test] -fn poll_yields_inbound_substream() { - let mut h = TestBuilder::new() - .with_muxer_inbound_state(DummyConnectionState::Opened) - .with_muxer_outbound_state(DummyConnectionState::Pending) - .with_handler_state(HandlerState::Err) // stop the loop - .handled_node(); - - assert_eq!(h.handler().events, vec![]); - let _ = h.poll(); - assert_eq!(h.handler().events, vec![InEvent::Substream(None)]); -} diff --git a/core/src/nodes/listeners.rs b/core/src/nodes/listeners.rs index f9c0f464d0e..22c9a9ee49b 100644 --- a/core/src/nodes/listeners.rs +++ b/core/src/nodes/listeners.rs @@ -21,11 +21,10 @@ //! Manage listening on multiple multiaddresses at once. use crate::{Multiaddr, Transport, transport::{TransportError, ListenerEvent}}; -use futures::prelude::*; +use futures::{prelude::*, task::Context, task::Poll}; use log::debug; use smallvec::SmallVec; -use std::{collections::VecDeque, fmt}; -use void::Void; +use std::{collections::VecDeque, fmt, pin::Pin}; /// Implementation of `futures::Stream` that allows listening on multiaddresses. /// @@ -52,32 +51,30 @@ use void::Void; /// listeners.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); /// /// // The `listeners` will now generate events when polled. -/// let future = listeners.for_each(move |event| { -/// match event { -/// ListenersEvent::NewAddress { listener_id, listen_addr } => { -/// println!("Listener {:?} is listening at address {}", listener_id, listen_addr); -/// }, -/// ListenersEvent::AddressExpired { listener_id, listen_addr } => { -/// println!("Listener {:?} is no longer listening at address {}", listener_id, listen_addr); -/// }, -/// ListenersEvent::Closed { listener_id, .. } => { -/// println!("Listener {:?} has been closed", listener_id); -/// }, -/// ListenersEvent::Error { listener_id, error } => { -/// println!("Listener {:?} has experienced an error: {}", listener_id, error); -/// }, -/// ListenersEvent::Incoming { listener_id, upgrade, local_addr, .. } => { -/// println!("Listener {:?} has a new connection on {}", listener_id, local_addr); -/// // We don't do anything with the newly-opened connection, but in a real-life -/// // program you probably want to use it! -/// drop(upgrade); -/// }, -/// }; -/// -/// Ok(()) -/// }); -/// -/// tokio::run(future.map_err(|_| ())); +/// futures::executor::block_on(async move { +/// while let Some(event) = listeners.next().await { +/// match event { +/// ListenersEvent::NewAddress { listener_id, listen_addr } => { +/// println!("Listener {:?} is listening at address {}", listener_id, listen_addr); +/// }, +/// ListenersEvent::AddressExpired { listener_id, listen_addr } => { +/// println!("Listener {:?} is no longer listening at address {}", listener_id, listen_addr); +/// }, +/// ListenersEvent::Closed { listener_id } => { +/// println!("Listener {:?} has been closed", listener_id); +/// }, +/// ListenersEvent::Error { listener_id, error } => { +/// println!("Listener {:?} has experienced an error: {}", listener_id, error); +/// }, +/// ListenersEvent::Incoming { listener_id, upgrade, local_addr, .. } => { +/// println!("Listener {:?} has a new connection on {}", listener_id, local_addr); +/// // We don't do anything with the newly-opened connection, but in a real-life +/// // program you probably want to use it! +/// drop(upgrade); +/// }, +/// } +/// } +/// }) /// # } /// ``` pub struct ListenersStream @@ -87,7 +84,9 @@ where /// Transport used to spawn listeners. transport: TTrans, /// All the active listeners. - listeners: VecDeque>, + /// The `Listener` struct contains a stream that we want to be pinned. Since the `VecDeque` + /// can be resized, the only way is to use a `Pin>`. + listeners: VecDeque>>>, /// The next listener ID to assign. next_id: ListenerId } @@ -100,6 +99,7 @@ where pub struct ListenerId(u64); /// A single active listener. +#[pin_project::pin_project] #[derive(Debug)] struct Listener where @@ -108,6 +108,7 @@ where /// The ID of this listener. id: ListenerId, /// The object that actually listens. + #[pin] listener: TTrans::Listener, /// Addresses it is listening on. addresses: SmallVec<[Multiaddr; 4]> @@ -147,8 +148,6 @@ where Closed { /// The ID of the listener that closed. listener_id: ListenerId, - /// The listener that closed. - listener: TTrans::Listener, }, /// A listener errored. /// @@ -158,7 +157,7 @@ where /// The ID of the listener that errored. listener_id: ListenerId, /// The error value. - error: ::Error + error: ::Error } } @@ -193,22 +192,25 @@ where TTrans: Clone, { let listener = self.transport.clone().listen_on(addr)?; - self.listeners.push_back(Listener { + self.listeners.push_back(Box::pin(Listener { id: self.next_id, listener, addresses: SmallVec::new() - }); + })); let id = self.next_id; self.next_id = ListenerId(self.next_id.0 + 1); Ok(id) } /// Remove the listener matching the given `ListenerId`. - pub fn remove_listener(&mut self, id: ListenerId) -> Option { + /// + /// Return `Ok(())` if a listener with this ID was in the list. + pub fn remove_listener(&mut self, id: ListenerId) -> Result<(), ()> { if let Some(i) = self.listeners.iter().position(|l| l.id == id) { - self.listeners.remove(i).map(|l| l.listener) + self.listeners.remove(i); + Ok(()) } else { - None + Err(()) } } @@ -222,59 +224,59 @@ where self.listeners.iter().flat_map(|l| l.addresses.iter()) } - /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll(&mut self) -> Async> { + /// Provides an API similar to `Stream`, except that it cannot end. + pub fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { // We remove each element from `listeners` one by one and add them back. let mut remaining = self.listeners.len(); while let Some(mut listener) = self.listeners.pop_back() { - match listener.listener.poll() { - Ok(Async::NotReady) => { + let mut listener_project = listener.as_mut().project(); + match TryStream::try_poll_next(listener_project.listener.as_mut(), cx) { + Poll::Pending => { self.listeners.push_front(listener); remaining -= 1; if remaining == 0 { break } } - Ok(Async::Ready(Some(ListenerEvent::Upgrade { upgrade, local_addr, remote_addr }))) => { - let id = listener.id; + Poll::Ready(Some(Ok(ListenerEvent::Upgrade { upgrade, local_addr, remote_addr }))) => { + let id = *listener_project.id; self.listeners.push_front(listener); - return Async::Ready(ListenersEvent::Incoming { + return Poll::Ready(ListenersEvent::Incoming { listener_id: id, upgrade, local_addr, send_back_addr: remote_addr }) } - Ok(Async::Ready(Some(ListenerEvent::NewAddress(a)))) => { - if listener.addresses.contains(&a) { + Poll::Ready(Some(Ok(ListenerEvent::NewAddress(a)))) => { + if listener_project.addresses.contains(&a) { debug!("Transport has reported address {} multiple times", a) } - if !listener.addresses.contains(&a) { - listener.addresses.push(a.clone()); + if !listener_project.addresses.contains(&a) { + listener_project.addresses.push(a.clone()); } - let id = listener.id; + let id = *listener_project.id; self.listeners.push_front(listener); - return Async::Ready(ListenersEvent::NewAddress { + return Poll::Ready(ListenersEvent::NewAddress { listener_id: id, listen_addr: a }) } - Ok(Async::Ready(Some(ListenerEvent::AddressExpired(a)))) => { - listener.addresses.retain(|x| x != &a); - let id = listener.id; + Poll::Ready(Some(Ok(ListenerEvent::AddressExpired(a)))) => { + listener_project.addresses.retain(|x| x != &a); + let id = *listener_project.id; self.listeners.push_front(listener); - return Async::Ready(ListenersEvent::AddressExpired { + return Poll::Ready(ListenersEvent::AddressExpired { listener_id: id, listen_addr: a }) } - Ok(Async::Ready(None)) => { - return Async::Ready(ListenersEvent::Closed { - listener_id: listener.id, - listener: listener.listener + Poll::Ready(None) => { + return Poll::Ready(ListenersEvent::Closed { + listener_id: *listener_project.id, }) } - Err(err) => { - return Async::Ready(ListenersEvent::Error { - listener_id: listener.id, + Poll::Ready(Some(Err(err))) => { + return Poll::Ready(ListenersEvent::Error { + listener_id: *listener_project.id, error: err }) } @@ -282,7 +284,7 @@ where } // We register the current task to be woken up if a new listener is added. - Async::NotReady + Poll::Pending } } @@ -291,13 +293,18 @@ where TTrans: Transport, { type Item = ListenersEvent; - type Error = Void; // TODO: use ! once stable - fn poll(&mut self) -> Poll, Self::Error> { - Ok(self.poll().map(Option::Some)) + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + ListenersStream::poll(self, cx).map(Option::Some) } } +impl Unpin for ListenersStream +where + TTrans: Transport, +{ +} + impl fmt::Debug for ListenersStream where TTrans: Transport + fmt::Debug, @@ -313,7 +320,7 @@ where impl fmt::Debug for ListenersEvent where TTrans: Transport, - ::Error: fmt::Debug, + ::Error: fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { match self { @@ -332,7 +339,7 @@ where .field("listener_id", listener_id) .field("local_addr", local_addr) .finish(), - ListenersEvent::Closed { listener_id, .. } => f + ListenersEvent::Closed { listener_id } => f .debug_struct("ListenersEvent::Closed") .field("listener_id", listener_id) .finish(), @@ -348,220 +355,37 @@ where #[cfg(test)] mod tests { use super::*; - use crate::transport::{self, ListenerEvent}; - use assert_matches::assert_matches; - use tokio::runtime::current_thread::Runtime; - use std::{io, iter::FromIterator}; - use futures::{future::{self}, stream}; - use crate::tests::dummy_transport::{DummyTransport, ListenerState}; - use crate::tests::dummy_muxer::DummyMuxer; - use crate::PeerId; - - fn set_listener_state(ls: &mut ListenersStream, idx: usize, state: ListenerState) { - ls.listeners[idx].listener = match state { - ListenerState::Error => - Box::new(stream::poll_fn(|| Err(io::Error::new(io::ErrorKind::Other, "oh noes")))), - ListenerState::Ok(state) => match state { - Async::NotReady => Box::new(stream::poll_fn(|| Ok(Async::NotReady))), - Async::Ready(Some(event)) => Box::new(stream::poll_fn(move || { - Ok(Async::Ready(Some(event.clone().map(future::ok)))) - })), - Async::Ready(None) => Box::new(stream::empty()) - } - ListenerState::Events(events) => - Box::new(stream::iter_ok(events.into_iter().map(|e| e.map(future::ok)))) - }; - } + use crate::transport; #[test] fn incoming_event() { - let mem_transport = transport::MemoryTransport::default(); - - let mut listeners = ListenersStream::new(mem_transport); - listeners.listen_on("/memory/0".parse().unwrap()).unwrap(); - - let address = { - let event = listeners.by_ref().wait().next().expect("some event").expect("no error"); - if let ListenersEvent::NewAddress { listen_addr, .. } = event { - listen_addr - } else { - panic!("Was expecting the listen address to be reported") - } - }; - - let dial = mem_transport.dial(address.clone()).unwrap(); - - let future = listeners - .into_future() - .map_err(|(err, _)| err) - .and_then(|(event, _)| { - match event { - Some(ListenersEvent::Incoming { local_addr, upgrade, send_back_addr, .. }) => { - assert_eq!(local_addr, address); - assert_eq!(send_back_addr, address); - upgrade.map(|_| ()).map_err(|_| panic!()) - }, - _ => panic!() + async_std::task::block_on(async move { + let mem_transport = transport::MemoryTransport::default(); + + let mut listeners = ListenersStream::new(mem_transport); + listeners.listen_on("/memory/0".parse().unwrap()).unwrap(); + + let address = { + let event = listeners.next().await.unwrap(); + if let ListenersEvent::NewAddress { listen_addr, .. } = event { + listen_addr + } else { + panic!("Was expecting the listen address to be reported") } - }) - .select(dial.map(|_| ()).map_err(|_| panic!())) - .map_err(|(err, _)| err); - - let mut runtime = Runtime::new().unwrap(); - let _ = runtime.block_on(future).unwrap(); - } - - #[test] - fn listener_stream_returns_transport() { - let t = DummyTransport::new(); - let t_clone = t.clone(); - let ls = ListenersStream::new(t); - assert_eq!(ls.transport(), &t_clone); - } - - #[test] - fn listener_stream_can_iterate_over_listeners() { - let mut t = DummyTransport::new(); - let addr1 = tcp4([127, 0, 0, 1], 1234); - let addr2 = tcp4([127, 0, 0, 1], 4321); - - t.set_initial_listener_state(ListenerState::Events(vec![ - ListenerEvent::NewAddress(addr1.clone()), - ListenerEvent::NewAddress(addr2.clone()) - ])); - - let mut ls = ListenersStream::new(t); - ls.listen_on(tcp4([0, 0, 0, 0], 0)).expect("listen_on"); - - assert_matches!(ls.by_ref().wait().next(), Some(Ok(ListenersEvent::NewAddress { listen_addr, .. })) => { - assert_eq!(addr1, listen_addr) - }); - assert_matches!(ls.by_ref().wait().next(), Some(Ok(ListenersEvent::NewAddress { listen_addr, .. })) => { - assert_eq!(addr2, listen_addr) - }) - } - - #[test] - fn listener_stream_poll_without_listeners_is_not_ready() { - let t = DummyTransport::new(); - let mut ls = ListenersStream::new(t); - assert_matches!(ls.poll(), Async::NotReady); - } - - #[test] - fn listener_stream_poll_with_listeners_that_arent_ready_is_not_ready() { - let t = DummyTransport::new(); - let addr = tcp4([127, 0, 0, 1], 1234); - let mut ls = ListenersStream::new(t); - ls.listen_on(addr).expect("listen_on failed"); - set_listener_state(&mut ls, 0, ListenerState::Ok(Async::NotReady)); - assert_matches!(ls.poll(), Async::NotReady); - assert_eq!(ls.listeners.len(), 1); // listener is still there - } - - #[test] - fn listener_stream_poll_with_ready_listeners_is_ready() { - let mut t = DummyTransport::new(); - let peer_id = PeerId::random(); - let muxer = DummyMuxer::new(); - let expected_output = (peer_id.clone(), muxer.clone()); - - t.set_initial_listener_state(ListenerState::Events(vec![ - ListenerEvent::NewAddress(tcp4([127, 0, 0, 1], 9090)), - ListenerEvent::Upgrade { - upgrade: (peer_id.clone(), muxer.clone()), - local_addr: tcp4([127, 0, 0, 1], 9090), - remote_addr: tcp4([127, 0, 0, 1], 32000) - }, - ListenerEvent::Upgrade { - upgrade: (peer_id.clone(), muxer.clone()), - local_addr: tcp4([127, 0, 0, 1], 9090), - remote_addr: tcp4([127, 0, 0, 1], 32000) - }, - ListenerEvent::Upgrade { - upgrade: (peer_id.clone(), muxer.clone()), - local_addr: tcp4([127, 0, 0, 1], 9090), - remote_addr: tcp4([127, 0, 0, 1], 32000) + }; + + let address2 = address.clone(); + async_std::task::spawn(async move { + mem_transport.dial(address2).unwrap().await.unwrap(); + }); + + match listeners.next().await.unwrap() { + ListenersEvent::Incoming { local_addr, send_back_addr, .. } => { + assert_eq!(local_addr, address); + assert_eq!(send_back_addr, address); + }, + _ => panic!() } - ])); - - let mut ls = ListenersStream::new(t); - ls.listen_on(tcp4([127, 0, 0, 1], 1234)).expect("listen_on"); - ls.listen_on(tcp4([127, 0, 0, 1], 4321)).expect("listen_on"); - assert_eq!(ls.listeners.len(), 2); - - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::NewAddress { .. }) - }); - - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::NewAddress { .. }) }); - - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Incoming { upgrade, .. } => { - assert_matches!(upgrade.wait(), Ok(output) => { - assert_eq!(output, expected_output) - }); - }) - }); - - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Incoming { upgrade, .. } => { - assert_matches!(upgrade.wait(), Ok(output) => { - assert_eq!(output, expected_output) - }); - }) - }); - - set_listener_state(&mut ls, 1, ListenerState::Ok(Async::NotReady)); - - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Incoming { upgrade, .. } => { - assert_matches!(upgrade.wait(), Ok(output) => { - assert_eq!(output, expected_output) - }); - }) - }); - } - - #[test] - fn listener_stream_poll_with_closed_listener_emits_closed_event() { - let t = DummyTransport::new(); - let addr = tcp4([127, 0, 0, 1], 1234); - let mut ls = ListenersStream::new(t); - ls.listen_on(addr).expect("listen_on failed"); - set_listener_state(&mut ls, 0, ListenerState::Ok(Async::Ready(None))); - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Closed{..}) - }); - assert_eq!(ls.listeners.len(), 0); // it's gone - } - - #[test] - fn listener_stream_poll_with_erroring_listener_emits_error_event() { - let mut t = DummyTransport::new(); - let peer_id = PeerId::random(); - let muxer = DummyMuxer::new(); - let event = ListenerEvent::Upgrade { - upgrade: (peer_id, muxer), - local_addr: tcp4([127, 0, 0, 1], 1234), - remote_addr: tcp4([127, 0, 0, 1], 32000) - }; - t.set_initial_listener_state(ListenerState::Ok(Async::Ready(Some(event)))); - let addr = tcp4([127, 0, 0, 1], 1234); - let mut ls = ListenersStream::new(t); - ls.listen_on(addr).expect("listen_on failed"); - set_listener_state(&mut ls, 0, ListenerState::Error); // simulate an error on the socket - assert_matches!(ls.by_ref().wait().next(), Some(Ok(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Error{..}) - }); - assert_eq!(ls.listeners.len(), 0); // it's gone - } - - fn tcp4(ip: [u8; 4], port: u16) -> Multiaddr { - let protos = std::iter::once(multiaddr::Protocol::Ip4(ip.into())) - .chain(std::iter::once(multiaddr::Protocol::Tcp(port))); - Multiaddr::from_iter(protos) } } diff --git a/core/src/nodes/network.rs b/core/src/nodes/network.rs index abe9e631681..0a92e9017b0 100644 --- a/core/src/nodes/network.rs +++ b/core/src/nodes/network.rs @@ -49,10 +49,10 @@ use std::{ fmt, hash::Hash, num::NonZeroUsize, + pin::Pin, + task::{Context, Poll}, }; -pub use crate::nodes::collection::StartTakeOver; - mod tests; /// Implementation of `Stream` that handles the nodes. @@ -70,7 +70,7 @@ where /// This needs to be a separate struct in order to handle multiple mutable borrows issues. reach_attempts: ReachAttempts, - /// Max numer of incoming connections. + /// Max number of incoming connections. incoming_limit: Option, /// Unfinished take over message to be delivered. @@ -81,7 +81,7 @@ where /// If the pair's second element is `AsyncSink::Ready`, the take over /// message has been sent and needs to be flushed using /// `PeerMut::complete_take_over`. - take_over_to_complete: Option<(TPeerId, AsyncSink>)> + take_over_to_complete: Option<(TPeerId, InterruptedReachAttempt)> } impl fmt::Debug for @@ -102,6 +102,13 @@ where } } +impl Unpin for + Network +where + TTrans: Transport +{ +} + impl ConnectionInfo for (TConnInfo, ConnectedPoint) where TConnInfo: ConnectionInfo @@ -164,8 +171,6 @@ where ListenerClosed { /// The listener ID that closed. listener_id: ListenerId, - /// The listener which closed. - listener: TTrans::Listener, }, /// One of the listeners errored. @@ -173,7 +178,7 @@ where /// The listener that errored. listener_id: ListenerId, /// The listener error. - error: ::Error + error: ::Error }, /// One of the listeners is now listening on an additional address. @@ -302,7 +307,7 @@ where .field("listen_addr", listen_addr) .finish() } - NetworkEvent::ListenerClosed { listener_id, .. } => { + NetworkEvent::ListenerClosed { listener_id } => { f.debug_struct("ListenerClosed") .field("listener_id", listener_id) .finish() @@ -609,9 +614,9 @@ where let connected_point = connected_point.clone(); move |(peer_id, muxer)| { if *peer_id.peer_id() == local_peer_id { - Err(InternalReachErr::FoundLocalPeerId) + future::ready(Err(InternalReachErr::FoundLocalPeerId)) } else { - Ok(((peer_id, connected_point), muxer)) + future::ready(Ok(((peer_id, connected_point), muxer))) } } }); @@ -728,7 +733,9 @@ where } /// Remove a previously added listener. - pub fn remove_listener(&mut self, id: ListenerId) -> Option { + /// + /// Returns `Ok(())` if a listener with this ID was in the list. + pub fn remove_listener(&mut self, id: ListenerId) -> Result<(), ()> { self.listeners.remove_listener(id) } @@ -797,9 +804,9 @@ where let connected_point = connected_point.clone(); move |(peer_id, muxer)| { if *peer_id.peer_id() == local_peer_id { - Err(InternalReachErr::FoundLocalPeerId) + future::ready(Err(InternalReachErr::FoundLocalPeerId)) } else { - Ok(((peer_id, connected_point), muxer)) + future::ready(Ok(((peer_id, connected_point), muxer))) } } }); @@ -838,21 +845,16 @@ where }) } - /// Start sending an event to all nodes. + /// Sends an event to all nodes. /// - /// Make sure to complete the broadcast with `complete_broadcast`. + /// This function is "atomic", in the sense that if `Poll::Pending` is returned then no event + /// has been sent to any node yet. #[must_use] - pub fn start_broadcast(&mut self, event: &TInEvent) -> AsyncSink<()> + pub fn poll_broadcast(&mut self, event: &TInEvent, cx: &mut Context) -> Poll<()> where TInEvent: Clone { - self.active_nodes.start_broadcast(event) - } - - /// Complete a broadcast initiated with `start_broadcast`. - #[must_use] - pub fn complete_broadcast(&mut self) -> Async<()> { - self.active_nodes.complete_broadcast() + self.active_nodes.poll_broadcast(event, cx) } /// Returns a list of all the peers we are currently connected to. @@ -950,9 +952,9 @@ where .map_err(|err| InternalReachErr::Transport(TransportError::Other(err))) .and_then(move |(actual_conn_info, muxer)| { if *actual_conn_info.peer_id() == expected_peer_id { - Ok(((actual_conn_info, connected_point), muxer)) + future::ready(Ok(((actual_conn_info, connected_point), muxer))) } else { - Err(InternalReachErr::PeerIdMismatch { obtained: actual_conn_info }) + future::ready(Err(InternalReachErr::PeerIdMismatch { obtained: actual_conn_info })) } }); self.active_nodes.add_reach_attempt(fut, handler) @@ -976,7 +978,7 @@ where } /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll(&mut self) -> Async> + pub fn poll<'a>(&'a mut self, cx: &mut Context) -> Poll> where TTrans: Transport, TTrans::Error: Send + 'static, @@ -998,9 +1000,9 @@ where Some(x) if self.incoming_negotiated().count() >= (x as usize) => (), _ => { - match self.listeners.poll() { - Async::NotReady => (), - Async::Ready(ListenersEvent::Incoming { listener_id, upgrade, local_addr, send_back_addr }) => { + match ListenersStream::poll(Pin::new(&mut self.listeners), cx) { + Poll::Pending => (), + Poll::Ready(ListenersEvent::Incoming { listener_id, upgrade, local_addr, send_back_addr }) => { let event = IncomingConnectionEvent { listener_id, upgrade, @@ -1010,19 +1012,19 @@ where active_nodes: &mut self.active_nodes, other_reach_attempts: &mut self.reach_attempts.other_reach_attempts, }; - return Async::Ready(NetworkEvent::IncomingConnection(event)); + return Poll::Ready(NetworkEvent::IncomingConnection(event)); } - Async::Ready(ListenersEvent::NewAddress { listener_id, listen_addr }) => { - return Async::Ready(NetworkEvent::NewListenerAddress { listener_id, listen_addr }) + Poll::Ready(ListenersEvent::NewAddress { listener_id, listen_addr }) => { + return Poll::Ready(NetworkEvent::NewListenerAddress { listener_id, listen_addr }) } - Async::Ready(ListenersEvent::AddressExpired { listener_id, listen_addr }) => { - return Async::Ready(NetworkEvent::ExpiredListenerAddress { listener_id, listen_addr }) + Poll::Ready(ListenersEvent::AddressExpired { listener_id, listen_addr }) => { + return Poll::Ready(NetworkEvent::ExpiredListenerAddress { listener_id, listen_addr }) } - Async::Ready(ListenersEvent::Closed { listener_id, listener }) => { - return Async::Ready(NetworkEvent::ListenerClosed { listener_id, listener }) + Poll::Ready(ListenersEvent::Closed { listener_id }) => { + return Poll::Ready(NetworkEvent::ListenerClosed { listener_id }) } - Async::Ready(ListenersEvent::Error { listener_id, error }) => { - return Async::Ready(NetworkEvent::ListenerError { listener_id, error }) + Poll::Ready(ListenersEvent::Error { listener_id, error }) => { + return Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) } } } @@ -1031,36 +1033,30 @@ where // Attempt to deliver any pending take over messages. if let Some((id, interrupted)) = self.take_over_to_complete.take() { if let Some(mut peer) = self.active_nodes.peer_mut(&id) { - if let AsyncSink::NotReady(i) = interrupted { - if let StartTakeOver::NotReady(i) = peer.start_take_over(i) { - self.take_over_to_complete = Some((id, AsyncSink::NotReady(i))) - } else if let Ok(Async::NotReady) = peer.complete_take_over() { - self.take_over_to_complete = Some((id, AsyncSink::Ready)) - } - } else if let Ok(Async::NotReady) = peer.complete_take_over() { - self.take_over_to_complete = Some((id, AsyncSink::Ready)) + if let Poll::Ready(()) = peer.poll_ready_take_over(cx) { + peer.start_take_over(interrupted); + } else { + self.take_over_to_complete = Some((id, interrupted)); + return Poll::Pending; } } } - if self.take_over_to_complete.is_some() { - return Async::NotReady - } // Poll the existing nodes. let (action, out_event); - match self.active_nodes.poll() { - Async::NotReady => return Async::NotReady, - Async::Ready(CollectionEvent::NodeReached(reach_event)) => { + match self.active_nodes.poll(cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(CollectionEvent::NodeReached(reach_event)) => { let (a, e) = handle_node_reached(&mut self.reach_attempts, reach_event); action = a; out_event = e; } - Async::Ready(CollectionEvent::ReachError { id, error, handler }) => { + Poll::Ready(CollectionEvent::ReachError { id, error, handler }) => { let (a, e) = handle_reach_error(&mut self.reach_attempts, id, error, handler); action = a; out_event = e; } - Async::Ready(CollectionEvent::NodeClosed { + Poll::Ready(CollectionEvent::NodeClosed { conn_info, error, .. @@ -1078,7 +1074,7 @@ where error, }; } - Async::Ready(CollectionEvent::NodeEvent { peer, event }) => { + Poll::Ready(CollectionEvent::NodeEvent { peer, event }) => { action = Default::default(); out_event = NetworkEvent::NodeEvent { conn_info: peer.info().0.clone(), event }; } @@ -1099,17 +1095,15 @@ where out_reach_attempts should always be in sync with the actual \ attempts; QED"); let mut peer = self.active_nodes.peer_mut(&peer_id).unwrap(); - if let StartTakeOver::NotReady(i) = peer.start_take_over(interrupted) { - self.take_over_to_complete = Some((peer_id, AsyncSink::NotReady(i))); - return Async::NotReady - } - if let Ok(Async::NotReady) = peer.complete_take_over() { - self.take_over_to_complete = Some((peer_id, AsyncSink::Ready)); - return Async::NotReady + if let Poll::Ready(()) = peer.poll_ready_take_over(cx) { + peer.start_take_over(interrupted); + } else { + self.take_over_to_complete = Some((peer_id, interrupted)); + return Poll::Pending } } - Async::Ready(out_event) + Poll::Ready(out_event) } } @@ -1644,18 +1638,33 @@ where closed messages; QED") } - /// Start sending an event to the node. - pub fn start_send_event(&mut self, event: TInEvent) -> StartSend { + /// Sends an event to the handler of the node. + pub fn send_event(&'a mut self, event: TInEvent) -> impl Future + 'a { + let mut event = Some(event); + futures::future::poll_fn(move |cx| { + match self.poll_ready_event(cx) { + Poll::Ready(()) => { + self.start_send_event(event.take().expect("Future called after finished")); + Poll::Ready(()) + }, + Poll::Pending => Poll::Pending, + } + }) + } + + /// Begin sending an event to the node. Must be called only after a successful call to + /// `poll_ready_event`. + pub fn start_send_event(&mut self, event: TInEvent) { self.active_nodes.peer_mut(&self.peer_id) .expect("A PeerConnected is always created with a PeerId in active_nodes; QED") .start_send_event(event) } - /// Complete sending an event message, initiated by `start_send_event`. - pub fn complete_send_event(&mut self) -> Poll<(), ()> { + /// Make sure we are ready to accept an event to be sent with `start_send_event`. + pub fn poll_ready_event(&mut self, cx: &mut Context) -> Poll<()> { self.active_nodes.peer_mut(&self.peer_id) .expect("A PeerConnected is always created with a PeerId in active_nodes; QED") - .complete_send_event() + .poll_ready_event(cx) } } diff --git a/core/src/nodes/network/tests.rs b/core/src/nodes/network/tests.rs index c64666aa798..c4f307bb6b1 100644 --- a/core/src/nodes/network/tests.rs +++ b/core/src/nodes/network/tests.rs @@ -21,363 +21,6 @@ #![cfg(test)] use super::*; -use crate::tests::dummy_transport::DummyTransport; -use crate::tests::dummy_handler::{Handler, HandlerState, InEvent, OutEvent}; -use crate::tests::dummy_transport::ListenerState; -use crate::tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; -use crate::nodes::NodeHandlerEvent; -use crate::transport::ListenerEvent; -use assert_matches::assert_matches; -use parking_lot::Mutex; -use std::sync::Arc; -use tokio::runtime::{Builder, Runtime}; - -#[test] -fn query_transport() { - let transport = DummyTransport::new(); - let transport2 = transport.clone(); - let network = Network::<_, _, _, Handler, _>::new(transport, PeerId::random()); - assert_eq!(network.transport(), &transport2); -} - -#[test] -fn local_node_peer() { - let peer_id = PeerId::random(); - let mut network = Network::<_, _, _, Handler, _>::new(DummyTransport::new(), peer_id.clone()); - assert_matches!(network.peer(peer_id), Peer::LocalNode); -} - -#[test] -fn successful_dial_reaches_a_node() { - let mut network = Network::<_, _, _, Handler, _>::new(DummyTransport::new(), PeerId::random()); - let addr = "/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr"); - let dial_res = network.dial(addr, Handler::default()); - assert!(dial_res.is_ok()); - - // Poll the network until we get a `NodeReached` then assert on the peer: - // it's there and it's connected. - let network = Arc::new(Mutex::new(network)); - - let mut rt = Runtime::new().unwrap(); - let mut peer_id : Option = None; - // Drive forward until we're Connected - while peer_id.is_none() { - let network_fut = network.clone(); - peer_id = rt.block_on(future::poll_fn(move || -> Poll, ()> { - let mut network = network_fut.lock(); - let poll_res = network.poll(); - match poll_res { - Async::Ready(NetworkEvent::Connected { conn_info, .. }) => Ok(Async::Ready(Some(conn_info))), - _ => Ok(Async::Ready(None)) - } - })).expect("tokio works"); - } - - let mut network = network.lock(); - let peer = network.peer(peer_id.unwrap()); - assert_matches!(peer, Peer::Connected(PeerConnected{..})); -} - -#[test] -fn num_incoming_negotiated() { - let mut transport = DummyTransport::new(); - let peer_id = PeerId::random(); - let muxer = DummyMuxer::new(); - - let events = vec![ - ListenerEvent::NewAddress("/ip4/127.0.0.1/tcp/1234".parse().unwrap()), - ListenerEvent::Upgrade { - upgrade: (peer_id.clone(), muxer.clone()), - local_addr: "/ip4/127.0.0.1/tcp/1234".parse().unwrap(), - remote_addr: "/ip4/127.0.0.1/tcp/32111".parse().unwrap() - } - ]; - transport.set_initial_listener_state(ListenerState::Events(events)); - - let mut network = Network::<_, _, _, Handler, _>::new(transport, PeerId::random()); - network.listen_on("/memory/0".parse().unwrap()).unwrap(); - - // no incoming yet - assert_eq!(network.incoming_negotiated().count(), 0); - - let mut rt = Runtime::new().unwrap(); - let network = Arc::new(Mutex::new(network)); - let network_fut = network.clone(); - let fut = future::poll_fn(move || -> Poll<_, ()> { - let mut network_fut = network_fut.lock(); - assert_matches!(network_fut.poll(), Async::Ready(NetworkEvent::NewListenerAddress {..})); - assert_matches!(network_fut.poll(), Async::Ready(NetworkEvent::IncomingConnection(incoming)) => { - incoming.accept(Handler::default()); - }); - Ok(Async::Ready(())) - }); - rt.block_on(fut).expect("tokio works"); - let network = network.lock(); - // Now there's an incoming connection - assert_eq!(network.incoming_negotiated().count(), 1); -} - -#[test] -fn broadcasted_events_reach_active_nodes() { - let mut network = Network::<_, _, _, Handler, _>::new(DummyTransport::new(), PeerId::random()); - let mut muxer = DummyMuxer::new(); - muxer.set_inbound_connection_state(DummyConnectionState::Pending); - muxer.set_outbound_connection_state(DummyConnectionState::Opened); - let addr = "/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr"); - let mut handler = Handler::default(); - handler.next_states = vec![HandlerState::Ready(NodeHandlerEvent::Custom(OutEvent::Custom("from handler 1") )),]; - let dial_result = network.dial(addr, handler); - assert!(dial_result.is_ok()); - - let network = Arc::new(Mutex::new(network)); - let mut rt = Runtime::new().unwrap(); - let network2 = network.clone(); - rt.block_on(future::poll_fn(move || { - if network2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - let mut peer_id : Option = None; - while peer_id.is_none() { - let network_fut = network.clone(); - peer_id = rt.block_on(future::poll_fn(move || -> Poll, ()> { - let mut network = network_fut.lock(); - if network.complete_broadcast().is_not_ready() { - return Ok(Async::NotReady) - } - let poll_res = network.poll(); - match poll_res { - Async::Ready(NetworkEvent::Connected { conn_info, .. }) => Ok(Async::Ready(Some(conn_info))), - _ => Ok(Async::Ready(None)) - } - })).expect("tokio works"); - } - - let mut keep_polling = true; - while keep_polling { - let network_fut = network.clone(); - keep_polling = rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut network = network_fut.lock(); - match network.poll() { - Async::Ready(event) => { - assert_matches!(event, NetworkEvent::NodeEvent { conn_info: _, event: inner_event } => { - // The event we sent reached the node and triggered sending the out event we told it to return - assert_matches!(inner_event, OutEvent::Custom("from handler 1")); - }); - Ok(Async::Ready(false)) - }, - _ => Ok(Async::Ready(true)) - } - })).expect("tokio works"); - } -} - -#[test] -fn querying_for_pending_peer() { - let mut network = Network::<_, _, _, Handler, _>::new(DummyTransport::new(), PeerId::random()); - let peer_id = PeerId::random(); - let peer = network.peer(peer_id.clone()); - assert_matches!(peer, Peer::NotConnected(PeerNotConnected{ .. })); - let addr = "/memory/0".parse().expect("bad multiaddr"); - let pending_peer = peer.into_not_connected().unwrap().connect(addr, Handler::default()); - assert_matches!(pending_peer, PeerPendingConnect { .. }); -} - -#[test] -fn querying_for_unknown_peer() { - let mut network = Network::<_, _, _, Handler, _>::new(DummyTransport::new(), PeerId::random()); - let peer_id = PeerId::random(); - let peer = network.peer(peer_id.clone()); - assert_matches!(peer, Peer::NotConnected( PeerNotConnected { nodes: _, peer_id: node_peer_id }) => { - assert_eq!(node_peer_id, peer_id); - }); -} - -#[test] -fn querying_for_connected_peer() { - let mut network = Network::<_, _, _, Handler, _>::new(DummyTransport::new(), PeerId::random()); - - // Dial a node - let addr = "/ip4/127.0.0.1/tcp/1234".parse().expect("bad multiaddr"); - network.dial(addr, Handler::default()).expect("dialing works"); - - let network = Arc::new(Mutex::new(network)); - let mut rt = Runtime::new().unwrap(); - // Drive it forward until we connect; extract the new PeerId. - let mut peer_id : Option = None; - while peer_id.is_none() { - let network_fut = network.clone(); - peer_id = rt.block_on(future::poll_fn(move || -> Poll, ()> { - let mut network = network_fut.lock(); - let poll_res = network.poll(); - match poll_res { - Async::Ready(NetworkEvent::Connected { conn_info, .. }) => Ok(Async::Ready(Some(conn_info))), - _ => Ok(Async::Ready(None)) - } - })).expect("tokio works"); - } - - // We're connected. - let mut network = network.lock(); - let peer = network.peer(peer_id.unwrap()); - assert_matches!(peer, Peer::Connected( PeerConnected { .. } )); -} - -#[test] -fn poll_with_closed_listener() { - let mut transport = DummyTransport::new(); - // Set up listener to be closed - transport.set_initial_listener_state(ListenerState::Ok(Async::Ready(None))); - - let mut network = Network::<_, _, _, Handler, _>::new(transport, PeerId::random()); - network.listen_on("/memory/0".parse().unwrap()).unwrap(); - - let mut rt = Runtime::new().unwrap(); - let network = Arc::new(Mutex::new(network)); - - let network_fut = network.clone(); - let fut = future::poll_fn(move || -> Poll<_, ()> { - let mut network = network_fut.lock(); - assert_matches!(network.poll(), Async::Ready(NetworkEvent::ListenerClosed { .. } )); - Ok(Async::Ready(())) - }); - rt.block_on(fut).expect("tokio works"); -} - -#[test] -fn unknown_peer_that_is_unreachable_yields_unknown_peer_dial_error() { - let mut transport = DummyTransport::new(); - transport.make_dial_fail(); - let mut network = Network::<_, _, _, Handler, _>::new(transport, PeerId::random()); - let addr = "/memory/0".parse::().expect("bad multiaddr"); - let handler = Handler::default(); - let dial_result = network.dial(addr, handler); - assert!(dial_result.is_ok()); - - let network = Arc::new(Mutex::new(network)); - let mut rt = Runtime::new().unwrap(); - // Drive it forward until we hear back from the node. - let mut keep_polling = true; - while keep_polling { - let network_fut = network.clone(); - keep_polling = rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut network = network_fut.lock(); - match network.poll() { - Async::NotReady => Ok(Async::Ready(true)), - Async::Ready(event) => { - assert_matches!(event, NetworkEvent::UnknownPeerDialError { .. } ); - Ok(Async::Ready(false)) - }, - } - })).expect("tokio works"); - } -} - -#[test] -fn known_peer_that_is_unreachable_yields_dial_error() { - let mut transport = DummyTransport::new(); - let peer_id = PeerId::random(); - transport.set_next_peer_id(&peer_id); - transport.make_dial_fail(); - let network = Arc::new(Mutex::new(Network::<_, _, _, Handler, _>::new(transport, PeerId::random()))); - - { - let network1 = network.clone(); - let mut network1 = network1.lock(); - let peer = network1.peer(peer_id.clone()); - assert_matches!(peer, Peer::NotConnected(PeerNotConnected{ .. })); - let addr = "/memory/0".parse::().expect("bad multiaddr"); - let pending_peer = peer.into_not_connected().unwrap().connect(addr, Handler::default()); - assert_matches!(pending_peer, PeerPendingConnect { .. }); - } - let mut rt = Runtime::new().unwrap(); - // Drive it forward until we hear back from the node. - let mut keep_polling = true; - while keep_polling { - let network_fut = network.clone(); - let peer_id = peer_id.clone(); - keep_polling = rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut network = network_fut.lock(); - match network.poll() { - Async::NotReady => Ok(Async::Ready(true)), - Async::Ready(event) => { - let failed_peer_id = assert_matches!( - event, - NetworkEvent::DialError { new_state: _, peer_id: failed_peer_id, .. } => failed_peer_id - ); - assert_eq!(peer_id, failed_peer_id); - Ok(Async::Ready(false)) - }, - } - })).expect("tokio works"); - } -} - -#[test] -fn yields_node_error_when_there_is_an_error_after_successful_connect() { - let mut transport = DummyTransport::new(); - let peer_id = PeerId::random(); - transport.set_next_peer_id(&peer_id); - let network = Arc::new(Mutex::new(Network::<_, _, _, Handler, _>::new(transport, PeerId::random()))); - - { - // Set up an outgoing connection with a PeerId we know - let network1 = network.clone(); - let mut network1 = network1.lock(); - let peer = network1.peer(peer_id.clone()); - let addr = "/unix/reachable".parse().expect("bad multiaddr"); - let mut handler = Handler::default(); - // Force an error - handler.next_states = vec![ HandlerState::Err ]; - peer.into_not_connected().unwrap().connect(addr, handler); - } - - // Ensure we run on a single thread - let mut rt = Builder::new().core_threads(1).build().unwrap(); - - // Drive it forward until we connect to the node. - let mut keep_polling = true; - while keep_polling { - let network_fut = network.clone(); - let network2 = network.clone(); - rt.block_on(future::poll_fn(move || { - if network2.lock().start_broadcast(&InEvent::NextState).is_not_ready() { - Ok::<_, ()>(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - })).unwrap(); - keep_polling = rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut network = network_fut.lock(); - // Push the Handler into an error state on the next poll - if network.complete_broadcast().is_not_ready() { - return Ok(Async::NotReady) - } - match network.poll() { - Async::NotReady => Ok(Async::Ready(true)), - Async::Ready(event) => { - assert_matches!(event, NetworkEvent::Connected { .. }); - // We're connected, we can move on - Ok(Async::Ready(false)) - }, - } - })).expect("tokio works"); - } - - // Poll again. It is going to be a NodeClosed because of how the - // handler's next state was set up. - let network_fut = network.clone(); - let expected_peer_id = peer_id.clone(); - rt.block_on(future::poll_fn(move || -> Poll<_, ()> { - let mut network = network_fut.lock(); - assert_matches!(network.poll(), Async::Ready(NetworkEvent::NodeClosed { conn_info, .. }) => { - assert_eq!(conn_info, expected_peer_id); - }); - Ok(Async::Ready(())) - })).expect("tokio works"); -} #[test] fn local_prio_equivalence_relation() { @@ -387,59 +30,3 @@ fn local_prio_equivalence_relation() { assert_ne!(has_dial_prio(&a, &b), has_dial_prio(&b, &a)); } } - -#[test] -fn limit_incoming_connections() { - let mut transport = DummyTransport::new(); - let peer_id = PeerId::random(); - let muxer = DummyMuxer::new(); - let limit = 1; - - let mut events = vec![ListenerEvent::NewAddress("/ip4/127.0.0.1/tcp/1234".parse().unwrap())]; - events.extend(std::iter::repeat( - ListenerEvent::Upgrade { - upgrade: (peer_id.clone(), muxer.clone()), - local_addr: "/ip4/127.0.0.1/tcp/1234".parse().unwrap(), - remote_addr: "/ip4/127.0.0.1/tcp/32111".parse().unwrap() - } - ).take(10)); - transport.set_initial_listener_state(ListenerState::Events(events)); - - let mut network = Network::<_, _, _, Handler, _>::new_with_incoming_limit(transport, PeerId::random(), Some(limit)); - assert_eq!(network.incoming_limit(), Some(limit)); - network.listen_on("/memory/0".parse().unwrap()).unwrap(); - assert_eq!(network.incoming_negotiated().count(), 0); - - let network = Arc::new(Mutex::new(network)); - let mut rt = Runtime::new().unwrap(); - for i in 1..10 { - let network_fut = network.clone(); - let fut = future::poll_fn(move || -> Poll<_, ()> { - let mut network_fut = network_fut.lock(); - if i <= limit { - assert_matches!(network_fut.poll(), Async::Ready(NetworkEvent::NewListenerAddress {..})); - assert_matches!(network_fut.poll(), - Async::Ready(NetworkEvent::IncomingConnection(incoming)) => { - incoming.accept(Handler::default()); - }); - } else { - match network_fut.poll() { - Async::NotReady => (), - Async::Ready(x) => { - match x { - NetworkEvent::NewListenerAddress {..} => {} - NetworkEvent::ExpiredListenerAddress {..} => {} - NetworkEvent::IncomingConnection(_) => {} - NetworkEvent::Connected {..} => {} - e => panic!("Not expected event: {:?}", e) - } - }, - } - } - Ok(Async::Ready(())) - }); - rt.block_on(fut).expect("tokio works"); - let network = network.lock(); - assert!(network.incoming_negotiated().count() <= (limit as usize)); - } -} diff --git a/core/src/nodes/node.rs b/core/src/nodes/node.rs index a1d0eac4e3f..99e5df61eb0 100644 --- a/core/src/nodes/node.rs +++ b/core/src/nodes/node.rs @@ -21,9 +21,7 @@ use futures::prelude::*; use crate::muxing; use smallvec::SmallVec; -use std::fmt; -use std::io::Error as IoError; -use std::sync::Arc; +use std::{fmt, io::Error as IoError, pin::Pin, sync::Arc, task::Context, task::Poll}; // Implementation notes // ================= @@ -135,7 +133,7 @@ where /// Destroys all outbound streams and returns the corresponding user data. pub fn cancel_outgoing(&mut self) -> Vec { let mut out = Vec::with_capacity(self.outbound_substreams.len()); - for (user_data, outbound) in self.outbound_substreams.drain() { + for (user_data, outbound) in self.outbound_substreams.drain(..) { out.push(user_data); self.muxer.destroy_outbound(outbound); } @@ -143,43 +141,44 @@ where } /// Provides an API similar to `Future`. - pub fn poll(&mut self) -> Poll, IoError> { + pub fn poll(&mut self, cx: &mut Context) -> Poll, IoError>> { // Polling inbound substream. - match self.muxer.poll_inbound().map_err(|e| e.into())? { - Async::Ready(substream) => { + match self.muxer.poll_inbound(cx) { + Poll::Ready(Ok(substream)) => { let substream = muxing::substream_from_ref(self.muxer.clone(), substream); - return Ok(Async::Ready(NodeEvent::InboundSubstream { + return Poll::Ready(Ok(NodeEvent::InboundSubstream { substream, })); } - Async::NotReady => {} + Poll::Ready(Err(err)) => return Poll::Ready(Err(err.into())), + Poll::Pending => {} } // Polling outbound substreams. // We remove each element from `outbound_substreams` one by one and add them back. for n in (0..self.outbound_substreams.len()).rev() { let (user_data, mut outbound) = self.outbound_substreams.swap_remove(n); - match self.muxer.poll_outbound(&mut outbound) { - Ok(Async::Ready(substream)) => { + match self.muxer.poll_outbound(cx, &mut outbound) { + Poll::Ready(Ok(substream)) => { let substream = muxing::substream_from_ref(self.muxer.clone(), substream); self.muxer.destroy_outbound(outbound); - return Ok(Async::Ready(NodeEvent::OutboundSubstream { + return Poll::Ready(Ok(NodeEvent::OutboundSubstream { user_data, substream, })); } - Ok(Async::NotReady) => { + Poll::Pending => { self.outbound_substreams.push((user_data, outbound)); } - Err(err) => { + Poll::Ready(Err(err)) => { self.muxer.destroy_outbound(outbound); - return Err(err.into()); + return Poll::Ready(Err(err.into())); } } } // Nothing happened. Register our task to be notified and return. - Ok(Async::NotReady) + Poll::Pending } } @@ -202,7 +201,7 @@ where // The substreams that were produced will continue to work, as the muxer is held in an Arc. // However we will no longer process any further inbound or outbound substream, and we // therefore close everything. - for (_, outbound) in self.outbound_substreams.drain() { + for (_, outbound) in self.outbound_substreams.drain(..) { self.muxer.destroy_outbound(outbound); } } @@ -212,11 +211,14 @@ impl Future for Close where TMuxer: muxing::StreamMuxer, { - type Item = (); - type Error = IoError; + type Output = Result<(), IoError>; - fn poll(&mut self) -> Poll { - self.muxer.close().map_err(|e| e.into()) + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + match self.muxer.close(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(Ok(())) => Poll::Ready(Ok(())), + Poll::Ready(Err(err)) => Poll::Ready(Err(err.into())), + } } } @@ -252,70 +254,3 @@ where } } } - -#[cfg(test)] -mod node_stream { - use super::{NodeEvent, NodeStream}; - use crate::tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; - use assert_matches::assert_matches; - use futures::prelude::*; - use tokio_mock_task::MockTask; - - fn build_node_stream() -> NodeStream> { - let muxer = DummyMuxer::new(); - NodeStream::<_, Vec>::new(muxer) - } - - #[test] - fn closing_a_node_stream_destroys_substreams_and_returns_submitted_user_data() { - let mut ns = build_node_stream(); - ns.open_substream(vec![2]); - ns.open_substream(vec![3]); - ns.open_substream(vec![5]); - let user_data_submitted = ns.close(); - assert_eq!(user_data_submitted.1, vec![ - vec![2], vec![3], vec![5] - ]); - } - - #[test] - fn poll_returns_not_ready_when_there_is_nothing_to_do() { - let mut task = MockTask::new(); - task.enter(|| { - // ensure the address never resolves - let mut muxer = DummyMuxer::new(); - // ensure muxer.poll_inbound() returns Async::NotReady - muxer.set_inbound_connection_state(DummyConnectionState::Pending); - // ensure muxer.poll_outbound() returns Async::NotReady - muxer.set_outbound_connection_state(DummyConnectionState::Pending); - let mut ns = NodeStream::<_, Vec>::new(muxer); - - assert_matches!(ns.poll(), Ok(Async::NotReady)); - }); - } - - #[test] - fn poll_keeps_outbound_substreams_when_the_outgoing_connection_is_not_ready() { - let mut muxer = DummyMuxer::new(); - // ensure muxer.poll_inbound() returns Async::NotReady - muxer.set_inbound_connection_state(DummyConnectionState::Pending); - // ensure muxer.poll_outbound() returns Async::NotReady - muxer.set_outbound_connection_state(DummyConnectionState::Pending); - let mut ns = NodeStream::<_, Vec>::new(muxer); - ns.open_substream(vec![1]); - ns.poll().unwrap(); // poll past inbound - ns.poll().unwrap(); // poll outbound - assert!(format!("{:?}", ns).contains("outbound_substreams: 1")); - } - - #[test] - fn poll_returns_incoming_substream() { - let mut muxer = DummyMuxer::new(); - // ensure muxer.poll_inbound() returns Async::Ready(subs) - muxer.set_inbound_connection_state(DummyConnectionState::Opened); - let mut ns = NodeStream::<_, Vec>::new(muxer); - assert_matches!(ns.poll(), Ok(Async::Ready(node_event)) => { - assert_matches!(node_event, NodeEvent::InboundSubstream{ substream: _ }); - }); - } -} diff --git a/core/src/nodes/tasks/manager.rs b/core/src/nodes/tasks/manager.rs index 33643aaac04..aa94e267005 100644 --- a/core/src/nodes/tasks/manager.rs +++ b/core/src/nodes/tasks/manager.rs @@ -27,9 +27,8 @@ use crate::{ } }; use fnv::FnvHashMap; -use futures::{prelude::*, future::Executor, sync::mpsc}; -use smallvec::SmallVec; -use std::{collections::hash_map::{Entry, OccupiedEntry}, error, fmt}; +use futures::{prelude::*, channel::mpsc, executor::ThreadPool, stream::FuturesUnordered}; +use std::{collections::hash_map::{Entry, OccupiedEntry}, error, fmt, pin::Pin, task::Context, task::Poll}; use super::{TaskId, task::{Task, FromTaskMessage, ToTaskMessage}, Error}; // Implementor notes @@ -64,12 +63,13 @@ pub struct Manager { /// Identifier for the next task to spawn. next_task_id: TaskId, - /// List of node tasks to spawn. - to_spawn: SmallVec<[Box + Send>; 8]>, + /// Threads pool where we spawn the nodes' tasks. If `None`, then we push tasks to the + /// `local_spawns` list instead. + threads_pool: Option, - /// If no tokio executor is available, we move tasks to this list, and futures are polled on - /// the current thread instead. - local_spawns: Vec + Send>>, + /// If no executor is available, we move tasks to this set, and futures are polled on the + /// current thread instead. + local_spawns: FuturesUnordered + Send>>>, /// Sender to emit events to the outside. Meant to be cloned and sent to tasks. events_tx: mpsc::Sender<(FromTaskMessage, TaskId)>, @@ -91,16 +91,13 @@ where /// Information about a running task. /// -/// Contains the sender to deliver event messages to the task, -/// the associated user data and a pending message if any, -/// meant to be delivered to the task via the sender. +/// Contains the sender to deliver event messages to the task, and +/// the associated user data. struct TaskInfo { /// channel endpoint to send messages to the task sender: mpsc::Sender>, /// task associated data user_data: T, - /// any pending event to deliver to the task - pending: Option>> } /// Event produced by the [`Manager`]. @@ -140,11 +137,15 @@ impl Manager { /// Creates a new task manager. pub fn new() -> Self { let (tx, rx) = mpsc::channel(1); + let threads_pool = ThreadPool::builder() + .name_prefix("libp2p-nodes-") + .create().ok(); + Self { tasks: FnvHashMap::default(), next_task_id: TaskId(0), - to_spawn: SmallVec::new(), - local_spawns: Vec::new(), + threads_pool, + local_spawns: FuturesUnordered::new(), events_tx: tx, events_rx: rx } @@ -156,7 +157,7 @@ impl Manager { /// processing the node's events. pub fn add_reach_attempt(&mut self, future: F, user_data: T, handler: H) -> TaskId where - F: Future + Send + 'static, + F: Future> + Send + 'static, H: IntoNodeHandler + Send + 'static, H::Handler: NodeHandler, InEvent = I, OutEvent = O, Error = HE> + Send + 'static, E: error::Error + Send + 'static, @@ -172,10 +173,14 @@ impl Manager { self.next_task_id.0 += 1; let (tx, rx) = mpsc::channel(4); - self.tasks.insert(task_id, TaskInfo { sender: tx, user_data, pending: None }); + self.tasks.insert(task_id, TaskInfo { sender: tx, user_data }); - let task = Box::new(Task::new(task_id, self.events_tx.clone(), rx, future, handler)); - self.to_spawn.push(task); + let task = Box::pin(Task::new(task_id, self.events_tx.clone(), rx, future, handler)); + if let Some(threads_pool) = &mut self.threads_pool { + threads_pool.spawn_ok(task); + } else { + self.local_spawns.push(task); + } task_id } @@ -202,71 +207,46 @@ impl Manager { self.next_task_id.0 += 1; let (tx, rx) = mpsc::channel(4); - self.tasks.insert(task_id, TaskInfo { sender: tx, user_data, pending: None }); + self.tasks.insert(task_id, TaskInfo { sender: tx, user_data }); - let task: Task, _, _, _, _, _, _> = + let task: Task>>, _, _, _, _, _, _> = Task::node(task_id, self.events_tx.clone(), rx, HandledNode::new(muxer, handler)); - self.to_spawn.push(Box::new(task)); + if let Some(threads_pool) = &mut self.threads_pool { + threads_pool.spawn_ok(Box::pin(task)); + } else { + self.local_spawns.push(Box::pin(task)); + } + task_id } - /// Start sending an event to all the tasks, including the pending ones. + /// Sends a message to all the tasks, including the pending ones. /// - /// After starting a broadcast make sure to finish it with `complete_broadcast`, - /// otherwise starting another broadcast or sending an event directly to a - /// task would overwrite the pending broadcast. + /// This function is "atomic", in the sense that if `Poll::Pending` is returned then no event + /// has been sent to any node yet. #[must_use] - pub fn start_broadcast(&mut self, event: &I) -> AsyncSink<()> + pub fn poll_broadcast(&mut self, event: &I, cx: &mut Context) -> Poll<()> where I: Clone { - if self.complete_broadcast().is_not_ready() { - return AsyncSink::NotReady(()) - } - for task in self.tasks.values_mut() { - let msg = ToTaskMessage::HandlerEvent(event.clone()); - task.pending = Some(AsyncSink::NotReady(msg)) + if let Poll::Pending = task.sender.poll_ready(cx) { + return Poll::Pending; + } } - AsyncSink::Ready - } - - /// Complete a started broadcast. - #[must_use] - pub fn complete_broadcast(&mut self) -> Async<()> { - let mut ready = true; - for task in self.tasks.values_mut() { - match task.pending.take() { - Some(AsyncSink::NotReady(msg)) => - match task.sender.start_send(msg) { - Ok(AsyncSink::NotReady(msg)) => { - task.pending = Some(AsyncSink::NotReady(msg)); - ready = false - } - Ok(AsyncSink::Ready) => - if let Ok(Async::NotReady) = task.sender.poll_complete() { - task.pending = Some(AsyncSink::Ready); - ready = false - } - Err(_) => {} - } - Some(AsyncSink::Ready) => - if let Ok(Async::NotReady) = task.sender.poll_complete() { - task.pending = Some(AsyncSink::Ready); - ready = false - } - None => {} + let msg = ToTaskMessage::HandlerEvent(event.clone()); + match task.sender.start_send(msg) { + Ok(()) => {}, + Err(ref err) if err.is_full() => + panic!("poll_ready returned Poll::Ready just above; qed"), + Err(_) => {}, } } - if ready { - Async::Ready(()) - } else { - Async::NotReady - } + Poll::Ready(()) } /// Grants access to an object that allows controlling a task of the collection. @@ -285,32 +265,13 @@ impl Manager { } /// Provides an API similar to `Stream`, except that it cannot produce an error. - pub fn poll(&mut self) -> Async> { - for to_spawn in self.to_spawn.drain() { - // We try to use the default executor, but fall back to polling the task manually if - // no executor is available. This makes it possible to use the core in environments - // outside of tokio. - let executor = tokio_executor::DefaultExecutor::current(); - if let Err(err) = executor.execute(to_spawn) { - self.local_spawns.push(err.into_future()) - } - } - - for n in (0 .. self.local_spawns.len()).rev() { - let mut task = self.local_spawns.swap_remove(n); - match task.poll() { - Ok(Async::Ready(())) => {} - Ok(Async::NotReady) => self.local_spawns.push(task), - // It would normally be desirable to either report or log when a background task - // errors. However the default tokio executor doesn't do anything in case of error, - // and therefore we mimic this behaviour by also not doing anything. - Err(()) => {} - } - } + pub fn poll(&mut self, cx: &mut Context) -> Poll> { + // Advance the content of `local_spawns`. + while let Poll::Ready(Some(_)) = Stream::poll_next(Pin::new(&mut self.local_spawns), cx) {} let (message, task_id) = loop { - match self.events_rx.poll() { - Ok(Async::Ready(Some((message, task_id)))) => { + match Stream::poll_next(Pin::new(&mut self.events_rx), cx) { + Poll::Ready(Some((message, task_id))) => { // If the task id is no longer in `self.tasks`, that means that the user called // `close()` on this task earlier. Therefore no new event should be generated // for this task. @@ -318,13 +279,12 @@ impl Manager { break (message, task_id) } } - Ok(Async::NotReady) => return Async::NotReady, - Ok(Async::Ready(None)) => unreachable!("sender and receiver have same lifetime"), - Err(()) => unreachable!("An `mpsc::Receiver` does not error.") + Poll::Pending => return Poll::Pending, + Poll::Ready(None) => unreachable!("sender and receiver have same lifetime"), } }; - Async::Ready(match message { + Poll::Ready(match message { FromTaskMessage::NodeEvent(event) => Event::NodeEvent { task: match self.tasks.entry(task_id) { @@ -360,24 +320,16 @@ pub struct TaskEntry<'a, E, T> { } impl<'a, E, T> TaskEntry<'a, E, T> { - /// Begin sending an event to the given node. - /// - /// Make sure to finish the send operation with `complete_send_event`. - pub fn start_send_event(&mut self, event: E) -> StartSend { + /// Begin sending an event to the given node. Must be called only after a successful call to + /// `poll_ready_event`. + pub fn start_send_event(&mut self, event: E) { let msg = ToTaskMessage::HandlerEvent(event); - if let AsyncSink::NotReady(msg) = self.start_send_event_msg(msg)? { - if let ToTaskMessage::HandlerEvent(event) = msg { - return Ok(AsyncSink::NotReady(event)) - } else { - unreachable!("we tried to send an handler event, so we get one back if not ready") - } - } - Ok(AsyncSink::Ready) + self.start_send_event_msg(msg); } - /// Finish a send operation started with `start_send_event`. - pub fn complete_send_event(&mut self) -> Poll<(), ()> { - self.complete_send_event_msg() + /// Make sure we are ready to accept an event to be sent with `start_send_event`. + pub fn poll_ready_event(&mut self, cx: &mut Context) -> Poll<()> { + self.poll_ready_event_msg(cx) } /// Returns the user data associated with the task. @@ -409,79 +361,38 @@ impl<'a, E, T> TaskEntry<'a, E, T> { /// As soon as our task (`self`) has some acknowledgment from the remote /// that its connection is alive, it will close the connection with `other`. /// - /// Make sure to complete this operation with `complete_take_over`. - #[must_use] - pub fn start_take_over(&mut self, t: ClosedTask) -> StartTakeOver> { - // It is possible that the sender is closed if the background task has already finished - // but the local state hasn't been updated yet because we haven't been polled in the - // meanwhile. - let id = t.id(); - match self.start_send_event_msg(ToTaskMessage::TakeOver(t.sender)) { - Ok(AsyncSink::Ready) => StartTakeOver::Ready(t.user_data), - Ok(AsyncSink::NotReady(ToTaskMessage::TakeOver(sender))) => - StartTakeOver::NotReady(ClosedTask::new(id, sender, t.user_data)), - Ok(AsyncSink::NotReady(_)) => - unreachable!("We tried to send a take over message, so we get one back."), - Err(()) => StartTakeOver::Gone - } + /// Must be called only after a successful call to `poll_ready_take_over`. + pub fn start_take_over(&mut self, t: ClosedTask) { + self.start_send_event_msg(ToTaskMessage::TakeOver(t.sender)); } - /// Finish take over started by `start_take_over`. - pub fn complete_take_over(&mut self) -> Poll<(), ()> { - self.complete_send_event_msg() + /// Make sure we are ready to taking over with `start_take_over`. + pub fn poll_ready_take_over(&mut self, cx: &mut Context) -> Poll<()> { + self.poll_ready_event_msg(cx) } - /// Begin to send a message to the task. + /// Sends a message to the task. Must be called only after a successful call to + /// `poll_ready_event`. /// - /// The API mimicks the one of [`futures::Sink`]. If this method returns - /// `Ok(AsyncSink::Ready)` drive the sending to completion with - /// `complete_send_event_msg`. If the receiving end does not longer exist, - /// i.e. the task has ended, we return this information as an error. - fn start_send_event_msg(&mut self, msg: ToTaskMessage) -> StartSend, ()> { - // We first drive any pending send to completion before starting another one. - if self.complete_send_event_msg()?.is_ready() { - self.inner.get_mut().pending = Some(AsyncSink::NotReady(msg)); - Ok(AsyncSink::Ready) - } else { - Ok(AsyncSink::NotReady(msg)) + /// The API mimicks the one of [`futures::Sink`]. + fn start_send_event_msg(&mut self, msg: ToTaskMessage) { + // It is possible that the sender is closed if the background task has already finished + // but the local state hasn't been updated yet because we haven't been polled in the + // meanwhile. + match self.inner.get_mut().sender.start_send(msg) { + Ok(()) => {}, + Err(ref err) if err.is_full() => {}, // TODO: somehow report to user? + Err(_) => {}, } } - /// Complete event message deliver started by `start_send_event_msg`. - fn complete_send_event_msg(&mut self) -> Poll<(), ()> { + /// Wait until we have space to send an event using `start_send_event_msg`. + fn poll_ready_event_msg(&mut self, cx: &mut Context) -> Poll<()> { // It is possible that the sender is closed if the background task has already finished // but the local state hasn't been updated yet because we haven't been polled in the // meanwhile. let task = self.inner.get_mut(); - let state = - if let Some(state) = task.pending.take() { - state - } else { - return Ok(Async::Ready(())) - }; - match state { - AsyncSink::NotReady(msg) => - match task.sender.start_send(msg).map_err(|_| ())? { - AsyncSink::Ready => - if task.sender.poll_complete().map_err(|_| ())?.is_not_ready() { - task.pending = Some(AsyncSink::Ready); - Ok(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - AsyncSink::NotReady(msg) => { - task.pending = Some(AsyncSink::NotReady(msg)); - Ok(Async::NotReady) - } - } - AsyncSink::Ready => - if task.sender.poll_complete().map_err(|_| ())?.is_not_ready() { - task.pending = Some(AsyncSink::Ready); - Ok(Async::NotReady) - } else { - Ok(Async::Ready(())) - } - } + task.sender.poll_ready(cx).map(|_| ()) } } @@ -494,18 +405,6 @@ impl fmt::Debug for TaskEntry<'_, E, T> { } } -/// Result of [`TaskEntry::start_take_over`]. -#[derive(Debug)] -pub enum StartTakeOver { - /// The take over message has been enqueued. - /// Complete the take over with [`TaskEntry::complete_take_over`]. - Ready(A), - /// Not ready to send the take over message to the task. - NotReady(B), - /// The task to send the take over message is no longer there. - Gone -} - /// Task after it has been closed. /// /// The connection to the remote is potentially still going on, but no new @@ -565,4 +464,3 @@ impl fmt::Debug for ClosedTask { .finish() } } - diff --git a/core/src/nodes/tasks/mod.rs b/core/src/nodes/tasks/mod.rs index baa1a081eac..5275121fd45 100644 --- a/core/src/nodes/tasks/mod.rs +++ b/core/src/nodes/tasks/mod.rs @@ -29,7 +29,7 @@ //! an existing connection to a node should be driven forward (cf. //! [`Manager::add_connection`]). Tasks can be referred to by [`TaskId`] //! and messages can be sent to individual tasks or all (cf. -//! [`Manager::start_broadcast`]). Messages produces by tasks can be +//! [`Manager::poll_broadcast`]). Messages produces by tasks can be //! retrieved by polling the manager (cf. [`Manager::poll`]). mod error; @@ -37,7 +37,7 @@ mod manager; mod task; pub use error::Error; -pub use manager::{ClosedTask, TaskEntry, Manager, Event, StartTakeOver}; +pub use manager::{ClosedTask, TaskEntry, Manager, Event}; /// Task identifier. #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)] diff --git a/core/src/nodes/tasks/task.rs b/core/src/nodes/tasks/task.rs index 05b801e16b4..24c4a280aeb 100644 --- a/core/src/nodes/tasks/task.rs +++ b/core/src/nodes/tasks/task.rs @@ -25,8 +25,9 @@ use crate::{ node::{Close, Substream} } }; -use futures::{prelude::*, stream, sync::mpsc}; +use futures::{prelude::*, channel::mpsc, stream}; use smallvec::SmallVec; +use std::{pin::Pin, task::Context, task::Poll}; use super::{TaskId, Error}; /// Message to transmit from the public API to a task. @@ -91,7 +92,7 @@ where id: i, sender: s, receiver: r.fuse(), - state: State::Future { future: f, handler: h, events_buffer: Vec::new() }, + state: State::Future { future: Box::pin(f), handler: h, events_buffer: Vec::new() }, taken_over: SmallVec::new() } } @@ -123,7 +124,8 @@ where /// Future to resolve to connect to the node. Future { /// The future that will attempt to reach the node. - future: F, + // TODO: don't pin this Future; this requires deeper changes though + future: Pin>, /// The handler that will be used to build the `HandledNode`. handler: H, /// While we are dialing the future, we need to buffer the events received on @@ -140,13 +142,6 @@ where event: FromTaskMessage::Error, C> }, - /// We started sending an event, now drive the sending to completion. - /// - /// The `bool` parameter determines if we transition to `State::Node` - /// afterwards or to `State::Closing` (assuming we have `Some` node, - /// otherwise the task will end). - PollComplete(Option>, bool), - /// Fully functional node. Node(HandledNode), @@ -158,94 +153,103 @@ where Undefined } +impl Unpin for Task +where + M: StreamMuxer, + H: IntoNodeHandler, + H::Handler: NodeHandler> +{ +} + impl Future for Task where M: StreamMuxer, - F: Future, + F: Future>, H: IntoNodeHandler, H::Handler: NodeHandler, InEvent = I, OutEvent = O> { - type Item = (); - type Error = (); + type Output = (); // NOTE: It is imperative to always consume all incoming event messages // first in order to not prevent the outside from making progress because // they are blocked on the channel capacity. - fn poll(&mut self) -> Poll<(), ()> { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<()> { + // We use a `this` because the compiler isn't smart enough to allow mutably borrowing + // multiple different fields from the `Pin` at the same time. + let this = &mut *self; + 'poll: loop { - match std::mem::replace(&mut self.state, State::Undefined) { + match std::mem::replace(&mut this.state, State::Undefined) { State::Future { mut future, handler, mut events_buffer } => { - // If self.receiver is closed, we stop the task. + // If this.receiver is closed, we stop the task. loop { - match self.receiver.poll() { - Ok(Async::NotReady) => break, - Ok(Async::Ready(None)) => return Ok(Async::Ready(())), - Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) => + match Stream::poll_next(Pin::new(&mut this.receiver), cx) { + Poll::Pending => break, + Poll::Ready(None) => return Poll::Ready(()), + Poll::Ready(Some(ToTaskMessage::HandlerEvent(event))) => events_buffer.push(event), - Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) => - self.taken_over.push(take_over), - Err(()) => unreachable!("An `mpsc::Receiver` does not error.") + Poll::Ready(Some(ToTaskMessage::TakeOver(take_over))) => + this.taken_over.push(take_over), } } // Check if dialing succeeded. - match future.poll() { - Ok(Async::Ready((conn_info, muxer))) => { + match Future::poll(Pin::new(&mut future), cx) { + Poll::Ready(Ok((conn_info, muxer))) => { let mut node = HandledNode::new(muxer, handler.into_handler(&conn_info)); for event in events_buffer { node.inject_event(event) } - self.state = State::SendEvent { + this.state = State::SendEvent { node: Some(node), event: FromTaskMessage::NodeReached(conn_info) } } - Ok(Async::NotReady) => { - self.state = State::Future { future, handler, events_buffer }; - return Ok(Async::NotReady) + Poll::Pending => { + this.state = State::Future { future, handler, events_buffer }; + return Poll::Pending } - Err(e) => { + Poll::Ready(Err(e)) => { let event = FromTaskMessage::TaskClosed(Error::Reach(e), Some(handler)); - self.state = State::SendEvent { node: None, event } + this.state = State::SendEvent { node: None, event } } } } State::Node(mut node) => { // Start by handling commands received from the outside of the task. loop { - match self.receiver.poll() { - Ok(Async::NotReady) => break, - Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) => + match Stream::poll_next(Pin::new(&mut this.receiver), cx) { + Poll::Pending => break, + Poll::Ready(Some(ToTaskMessage::HandlerEvent(event))) => node.inject_event(event), - Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) => - self.taken_over.push(take_over), - Ok(Async::Ready(None)) => { + Poll::Ready(Some(ToTaskMessage::TakeOver(take_over))) => + this.taken_over.push(take_over), + Poll::Ready(None) => { // Node closed by the external API; start closing. - self.state = State::Closing(node.close()); + this.state = State::Closing(node.close()); continue 'poll } - Err(()) => unreachable!("An `mpsc::Receiver` does not error.") } } // Process the node. loop { - if !self.taken_over.is_empty() && node.is_remote_acknowledged() { - self.taken_over.clear() + if !this.taken_over.is_empty() && node.is_remote_acknowledged() { + this.taken_over.clear() } - match node.poll() { - Ok(Async::NotReady) => { - self.state = State::Node(node); - return Ok(Async::NotReady) + match HandledNode::poll(Pin::new(&mut node), cx) { + Poll::Pending => { + this.state = State::Node(node); + return Poll::Pending } - Ok(Async::Ready(event)) => { - self.state = State::SendEvent { + Poll::Ready(Ok(event)) => { + this.state = State::SendEvent { node: Some(node), event: FromTaskMessage::NodeEvent(event) }; continue 'poll } - Err(err) => { + Poll::Ready(Err(err)) => { let event = FromTaskMessage::TaskClosed(Error::Node(err), None); - self.state = State::SendEvent { node: None, event }; + this.state = State::SendEvent { node: None, event }; continue 'poll } } @@ -254,23 +258,22 @@ where // Deliver an event to the outside. State::SendEvent { mut node, event } => { loop { - match self.receiver.poll() { - Ok(Async::NotReady) => break, - Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) => + match Stream::poll_next(Pin::new(&mut this.receiver), cx) { + Poll::Pending => break, + Poll::Ready(Some(ToTaskMessage::HandlerEvent(event))) => if let Some(ref mut n) = node { n.inject_event(event) } - Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) => - self.taken_over.push(take_over), - Ok(Async::Ready(None)) => + Poll::Ready(Some(ToTaskMessage::TakeOver(take_over))) => + this.taken_over.push(take_over), + Poll::Ready(None) => // Node closed by the external API; start closing. if let Some(n) = node { - self.state = State::Closing(n.close()); + this.state = State::Closing(n.close()); continue 'poll } else { - return Ok(Async::Ready(())) // end task + return Poll::Ready(()) // end task } - Err(()) => unreachable!("An `mpsc::Receiver` does not error.") } } // Check if this task is about to close. We pass the flag to @@ -281,80 +284,46 @@ where } else { false }; - match self.sender.start_send((event, self.id)) { - Ok(AsyncSink::NotReady((event, _))) => { + match this.sender.poll_ready(cx) { + Poll::Pending => { self.state = State::SendEvent { node, event }; - return Ok(Async::NotReady) - } - Ok(AsyncSink::Ready) => self.state = State::PollComplete(node, close), - Err(_) => { - if let Some(n) = node { - self.state = State::Closing(n.close()); - continue 'poll - } - // We can not communicate to the outside and there is no - // node to handle, so this is the end of this task. - return Ok(Async::Ready(())) - } - } - } - // We started delivering an event, now try to complete the sending. - State::PollComplete(mut node, close) => { - loop { - match self.receiver.poll() { - Ok(Async::NotReady) => break, - Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) => - if let Some(ref mut n) = node { - n.inject_event(event) - } - Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) => - self.taken_over.push(take_over), - Ok(Async::Ready(None)) => - // Node closed by the external API; start closing. - if let Some(n) = node { - self.state = State::Closing(n.close()); - continue 'poll - } else { - return Ok(Async::Ready(())) // end task - } - Err(()) => unreachable!("An `mpsc::Receiver` does not error.") - } - } - match self.sender.poll_complete() { - Ok(Async::NotReady) => { - self.state = State::PollComplete(node, close); - return Ok(Async::NotReady) + return Poll::Pending } - Ok(Async::Ready(())) => + Poll::Ready(Ok(())) => { + // We assume that if `poll_ready` has succeeded, then sending the event + // will succeed as well. If it turns out that it didn't, we will detect + // the closing at the next loop iteration. + let _ = this.sender.start_send((event, this.id)); if let Some(n) = node { if close { - self.state = State::Closing(n.close()) + this.state = State::Closing(n.close()) } else { - self.state = State::Node(n) + this.state = State::Node(n) } } else { // Since we have no node we terminate this task. assert!(close); - return Ok(Async::Ready(())) + return Poll::Ready(()) } - Err(_) => { + }, + Poll::Ready(Err(_)) => { if let Some(n) = node { - self.state = State::Closing(n.close()); + this.state = State::Closing(n.close()); continue 'poll } // We can not communicate to the outside and there is no // node to handle, so this is the end of this task. - return Ok(Async::Ready(())) + return Poll::Ready(()) } } } State::Closing(mut closing) => - match closing.poll() { - Ok(Async::Ready(())) | Err(_) => - return Ok(Async::Ready(())), // end task - Ok(Async::NotReady) => { - self.state = State::Closing(closing); - return Ok(Async::NotReady) + match Future::poll(Pin::new(&mut closing), cx) { + Poll::Ready(_) => + return Poll::Ready(()), // end task + Poll::Pending => { + this.state = State::Closing(closing); + return Poll::Pending } } // This happens if a previous poll has resolved the future. diff --git a/core/src/peer_id.rs b/core/src/peer_id.rs index 9ebb68299a7..cb1ef39a1ad 100644 --- a/core/src/peer_id.rs +++ b/core/src/peer_id.rs @@ -20,7 +20,7 @@ use crate::PublicKey; use bs58; -use quick_error::quick_error; +use thiserror::Error; use multihash; use std::{convert::TryFrom, fmt, str::FromStr}; @@ -28,7 +28,7 @@ use std::{convert::TryFrom, fmt, str::FromStr}; /// automatically used as the peer id using an identity multihash. // // Note: see `from_public_key` for how this value will be used in the future. -const MAX_INLINE_KEY_LENGTH: usize = 42; +const _MAX_INLINE_KEY_LENGTH: usize = 42; /// Identifier of a peer of the network. /// @@ -218,18 +218,12 @@ impl Into for PeerId { } } -quick_error! { - #[derive(Debug)] - pub enum ParseError { - B58(e: bs58::decode::Error) { - display("base-58 decode error: {}", e) - cause(e) - from() - } - MultiHash { - display("decoding multihash failed") - } - } +#[derive(Debug, Error)] +pub enum ParseError { + #[error("base-58 decode error: {0}")] + B58(#[from] bs58::decode::Error), + #[error("decoding multihash failed")] + MultiHash, } impl FromStr for PeerId { diff --git a/core/src/tests/dummy_handler.rs b/core/src/tests/dummy_handler.rs deleted file mode 100644 index 2f4ee3fab2b..00000000000 --- a/core/src/tests/dummy_handler.rs +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -//! Concrete `NodeHandler` implementation and assorted testing types - -use std::io::{self, Error as IoError}; - -use super::dummy_muxer::DummyMuxer; -use futures::prelude::*; -use crate::muxing::SubstreamRef; -use crate::nodes::handled_node::{HandledNode, NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent}; -use std::sync::Arc; - -#[derive(Debug, PartialEq, Clone)] -pub(crate) struct Handler { - /// Inspect events passed through the Handler - pub events: Vec, - /// Current state of the Handler - pub state: Option, - /// Next state for outbound streams of the Handler - pub next_outbound_state: Option, - /// Vec of states the Handler will assume - pub next_states: Vec, -} - -impl Default for Handler { - fn default() -> Self { - Handler { - events: Vec::new(), - state: None, - next_states: Vec::new(), - next_outbound_state: None, - } - } -} - -#[derive(Debug, PartialEq, Clone)] -pub(crate) enum HandlerState { - Ready(NodeHandlerEvent), - Err, -} - -#[derive(Debug, PartialEq, Clone)] -pub(crate) enum InEvent { - /// A custom inbound event - Custom(&'static str), - /// A substream request with a dummy payload - Substream(Option), - /// Request the handler to move to the next state - NextState, -} - -#[derive(Debug, PartialEq, Clone)] -pub(crate) enum OutEvent { - /// A message from the Handler upwards in the stack - Custom(&'static str), -} - -// Concrete `HandledNode` parametrised for the test helpers -pub(crate) type TestHandledNode = HandledNode; - -impl NodeHandler for Handler { - type InEvent = InEvent; - type OutEvent = OutEvent; - type Error = IoError; - type OutboundOpenInfo = usize; - type Substream = SubstreamRef>; - fn inject_substream( - &mut self, - _: Self::Substream, - endpoint: NodeHandlerEndpoint, - ) { - let user_data = match endpoint { - NodeHandlerEndpoint::Dialer(user_data) => Some(user_data), - NodeHandlerEndpoint::Listener => None, - }; - self.events.push(InEvent::Substream(user_data)); - } - fn inject_event(&mut self, inevent: Self::InEvent) { - self.events.push(inevent.clone()); - match inevent { - InEvent::Custom(s) => { - self.state = Some(HandlerState::Ready(NodeHandlerEvent::Custom( - OutEvent::Custom(s), - ))) - } - InEvent::Substream(Some(user_data)) => { - self.state = Some(HandlerState::Ready( - NodeHandlerEvent::OutboundSubstreamRequest(user_data), - )) - } - InEvent::NextState => { - let next_state = self.next_states.pop(); - self.state = next_state - } - _ => unreachable!(), - } - } - fn poll(&mut self) -> Poll, IoError> { - match self.state.take() { - Some(ref state) => match state { - HandlerState::Ready(event) => Ok(Async::Ready(event.clone())), - HandlerState::Err => Err(io::Error::new(io::ErrorKind::Other, "oh noes")), - }, - None => Ok(Async::NotReady), - } - } -} diff --git a/core/src/tests/dummy_muxer.rs b/core/src/tests/dummy_muxer.rs deleted file mode 100644 index eb4bbb1630b..00000000000 --- a/core/src/tests/dummy_muxer.rs +++ /dev/null @@ -1,122 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -//! `DummyMuxer` is a `StreamMuxer` to be used in tests. It implements a bare-bones -//! version of the trait along with a way to setup the muxer to behave in the -//! desired way when testing other components. - -use futures::prelude::*; -use crate::muxing::StreamMuxer; -use std::io::Error as IoError; - -/// Substream type -#[derive(Debug)] -pub struct DummySubstream {} - -/// OutboundSubstream type -#[derive(Debug)] -pub struct DummyOutboundSubstream {} - -/// Control the muxer state by setting the "connection" state as to set up a mock -/// muxer for higher level components. -#[derive(Debug, PartialEq, Clone)] -pub enum DummyConnectionState { - Pending, // use this to trigger the Async::NotReady code path - Opened, // use this to trigger the Async::Ready(_) code path -} -#[derive(Debug, PartialEq, Clone)] -struct DummyConnection { - state: DummyConnectionState, -} - -/// `DummyMuxer` implements `StreamMuxer` and methods to control its behaviour when used in tests -#[derive(Debug, PartialEq, Clone)] -pub struct DummyMuxer{ - in_connection: DummyConnection, - out_connection: DummyConnection, -} - -impl DummyMuxer { - /// Create a new `DummyMuxer` where the inbound substream is set to `Pending` - /// and the (single) outbound substream to `Pending`. - pub fn new() -> Self { - DummyMuxer { - in_connection: DummyConnection { - state: DummyConnectionState::Pending, - }, - out_connection: DummyConnection { - state: DummyConnectionState::Pending, - }, - } - } - /// Set the muxer state inbound "connection" state - pub fn set_inbound_connection_state(&mut self, state: DummyConnectionState) { - self.in_connection.state = state - } - /// Set the muxer state outbound "connection" state - pub fn set_outbound_connection_state(&mut self, state: DummyConnectionState) { - self.out_connection.state = state - } -} - -impl StreamMuxer for DummyMuxer { - type Substream = DummySubstream; - type OutboundSubstream = DummyOutboundSubstream; - type Error = IoError; - fn poll_inbound(&self) -> Poll { - match self.in_connection.state { - DummyConnectionState::Pending => Ok(Async::NotReady), - DummyConnectionState::Opened => Ok(Async::Ready(Self::Substream {})), - } - } - fn open_outbound(&self) -> Self::OutboundSubstream { - Self::OutboundSubstream {} - } - fn poll_outbound( - &self, - _substream: &mut Self::OutboundSubstream, - ) -> Poll { - match self.out_connection.state { - DummyConnectionState::Pending => Ok(Async::NotReady), - DummyConnectionState::Opened => Ok(Async::Ready(Self::Substream {})), - } - } - fn destroy_outbound(&self, _: Self::OutboundSubstream) {} - fn read_substream(&self, _: &mut Self::Substream, _buf: &mut [u8]) -> Poll { - unreachable!() - } - fn write_substream(&self, _: &mut Self::Substream, _buf: &[u8]) -> Poll { - unreachable!() - } - fn flush_substream(&self, _: &mut Self::Substream) -> Poll<(), IoError> { - unreachable!() - } - fn shutdown_substream(&self, _: &mut Self::Substream) -> Poll<(), IoError> { - unreachable!() - } - fn destroy_substream(&self, _: Self::Substream) {} - fn is_remote_acknowledged(&self) -> bool { true } - fn close(&self) -> Poll<(), IoError> { - Ok(Async::Ready(())) - } - fn flush_all(&self) -> Poll<(), IoError> { - Ok(Async::Ready(())) - } -} diff --git a/core/src/tests/dummy_transport.rs b/core/src/tests/dummy_transport.rs deleted file mode 100644 index 0622ec0e016..00000000000 --- a/core/src/tests/dummy_transport.rs +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -//! `DummyTransport` is a `Transport` used in tests. It implements a bare-bones -//! version of the trait along with a way to setup the transport listeners with -//! an initial state to facilitate testing. - -use futures::prelude::*; -use futures::{ - future::{self, FutureResult}, - stream, -}; -use std::io; -use crate::{Multiaddr, PeerId, Transport, transport::{ListenerEvent, TransportError}}; -use crate::tests::dummy_muxer::DummyMuxer; - -#[derive(Debug, PartialEq, Clone)] -pub(crate) enum ListenerState { - Ok(Async>>), - Error, - Events(Vec>) -} - -#[derive(Debug, PartialEq, Clone)] -pub(crate) struct DummyTransport { - /// The current state of Listeners. - listener_state: ListenerState, - /// The next peer returned from dial(). - next_peer_id: Option, - /// When true, all dial attempts return error. - dial_should_fail: bool, -} -impl DummyTransport { - pub(crate) fn new() -> Self { - DummyTransport { - listener_state: ListenerState::Ok(Async::NotReady), - next_peer_id: None, - dial_should_fail: false, - } - } - pub(crate) fn set_initial_listener_state(&mut self, state: ListenerState) { - self.listener_state = state; - } - - pub(crate) fn set_next_peer_id(&mut self, peer_id: &PeerId) { - self.next_peer_id = Some(peer_id.clone()); - } - - pub(crate) fn make_dial_fail(&mut self) { - self.dial_should_fail = true; - } -} -impl Transport for DummyTransport { - type Output = (PeerId, DummyMuxer); - type Error = io::Error; - type Listener = Box, Error=io::Error> + Send>; - type ListenerUpgrade = FutureResult; - type Dial = Box + Send>; - - fn listen_on(self, addr: Multiaddr) -> Result> - where - Self: Sized, - { - match self.listener_state { - ListenerState::Ok(state) => match state { - Async::NotReady => Ok(Box::new(stream::poll_fn(|| Ok(Async::NotReady)))), - Async::Ready(Some(event)) => Ok(Box::new(stream::poll_fn(move || { - Ok(Async::Ready(Some(event.clone().map(future::ok)))) - }))), - Async::Ready(None) => Ok(Box::new(stream::empty())) - }, - ListenerState::Error => Err(TransportError::MultiaddrNotSupported(addr)), - ListenerState::Events(events) => - Ok(Box::new(stream::iter_ok(events.into_iter().map(|e| e.map(future::ok))))) - } - } - - fn dial(self, _addr: Multiaddr) -> Result> - where - Self: Sized, - { - let peer_id = if let Some(peer_id) = self.next_peer_id { - peer_id - } else { - PeerId::random() - }; - - let fut = - if self.dial_should_fail { - let err_string = format!("unreachable host error, peer={:?}", peer_id); - future::err(io::Error::new(io::ErrorKind::Other, err_string)) - } else { - future::ok((peer_id, DummyMuxer::new())) - }; - - Ok(Box::new(fut)) - } -} diff --git a/core/src/tests/mod.rs b/core/src/tests/mod.rs deleted file mode 100644 index 5c86aec1c1d..00000000000 --- a/core/src/tests/mod.rs +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) 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. - -#[cfg(test)] -pub(crate) mod dummy_muxer; - -#[cfg(test)] -pub(crate) mod dummy_transport; - -#[cfg(test)] -pub(crate) mod dummy_handler; diff --git a/core/src/transport/and_then.rs b/core/src/transport/and_then.rs index d4233c443b2..7f8a52b0712 100644 --- a/core/src/transport/and_then.rs +++ b/core/src/transport/and_then.rs @@ -23,9 +23,9 @@ use crate::{ either::EitherError, transport::{Transport, TransportError, ListenerEvent} }; -use futures::{future::Either, prelude::*, try_ready}; +use futures::{future::Either, prelude::*}; use multiaddr::Multiaddr; -use std::error; +use std::{error, marker::PhantomPinned, pin::Pin, task::Context, task::Poll}; /// See the `Transport::and_then` method. #[derive(Debug, Clone)] @@ -41,14 +41,14 @@ impl Transport for AndThen where T: Transport, C: FnOnce(T::Output, ConnectedPoint) -> F + Clone, - F: IntoFuture, + F: TryFuture, F::Error: error::Error, { type Output = O; type Error = EitherError; type Listener = AndThenStream; - type ListenerUpgrade = AndThenFuture; - type Dial = AndThenFuture; + type ListenerUpgrade = AndThenFuture; + type Dial = AndThenFuture; fn listen_on(self, addr: Multiaddr) -> Result> { let listener = self.transport.listen_on(addr).map_err(|err| err.map(EitherError::A))?; @@ -63,8 +63,9 @@ where fn dial(self, addr: Multiaddr) -> Result> { let dialed_fut = self.transport.dial(addr.clone()).map_err(|err| err.map(EitherError::A))?; let future = AndThenFuture { - inner: Either::A(dialed_fut), - args: Some((self.fun, ConnectedPoint::Dialer { address: addr })) + inner: Either::Left(Box::pin(dialed_fut)), + args: Some((self.fun, ConnectedPoint::Dialer { address: addr })), + marker: PhantomPinned, }; Ok(future) } @@ -73,25 +74,30 @@ where /// Custom `Stream` to avoid boxing. /// /// Applies a function to every stream item. +#[pin_project::pin_project] #[derive(Debug, Clone)] pub struct AndThenStream { + #[pin] stream: TListener, fun: TMap } impl Stream for AndThenStream where - TListener: Stream, Error = TTransErr>, - TListUpgr: Future, + TListener: TryStream, Error = TTransErr>, + TListUpgr: TryFuture, TMap: FnOnce(TTransOut, ConnectedPoint) -> TMapOut + Clone, - TMapOut: IntoFuture + TMapOut: TryFuture { - type Item = ListenerEvent>; - type Error = EitherError; + type Item = Result< + ListenerEvent>, + EitherError + >; - fn poll(&mut self) -> Poll, Self::Error> { - match self.stream.poll().map_err(EitherError::A)? { - Async::Ready(Some(event)) => { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + match TryStream::try_poll_next(this.stream, cx) { + Poll::Ready(Some(Ok(event))) => { let event = match event { ListenerEvent::Upgrade { upgrade, local_addr, remote_addr } => { let point = ConnectedPoint::Listener { @@ -100,8 +106,9 @@ where }; ListenerEvent::Upgrade { upgrade: AndThenFuture { - inner: Either::A(upgrade), - args: Some((self.fun.clone(), point)) + inner: Either::Left(Box::pin(upgrade)), + args: Some((this.fun.clone(), point)), + marker: PhantomPinned, }, local_addr, remote_addr @@ -110,10 +117,11 @@ where ListenerEvent::NewAddress(a) => ListenerEvent::NewAddress(a), ListenerEvent::AddressExpired(a) => ListenerEvent::AddressExpired(a) }; - Ok(Async::Ready(Some(event))) + Poll::Ready(Some(Ok(event))) } - Async::Ready(None) => Ok(Async::Ready(None)), - Async::NotReady => Ok(Async::NotReady) + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(EitherError::A(err)))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending } } } @@ -123,32 +131,44 @@ where /// Applies a function to the result of the inner future. #[derive(Debug)] pub struct AndThenFuture { - inner: Either, - args: Option<(TMap, ConnectedPoint)> + inner: Either>, Pin>>, + args: Option<(TMap, ConnectedPoint)>, + marker: PhantomPinned, } -impl Future for AndThenFuture +impl Future for AndThenFuture where - TFut: Future, - TMap: FnOnce(TFut::Item, ConnectedPoint) -> TMapOut, - TMapOut: IntoFuture + TFut: TryFuture, + TMap: FnOnce(TFut::Ok, ConnectedPoint) -> TMapOut, + TMapOut: TryFuture, { - type Item = ::Item; - type Error = EitherError; + type Output = Result>; - fn poll(&mut self) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { loop { - let future = match self.inner { - Either::A(ref mut future) => { - let item = try_ready!(future.poll().map_err(EitherError::A)); + let future = match &mut self.inner { + Either::Left(future) => { + let item = match TryFuture::try_poll(future.as_mut(), cx) { + Poll::Ready(Ok(v)) => v, + Poll::Ready(Err(err)) => return Poll::Ready(Err(EitherError::A(err))), + Poll::Pending => return Poll::Pending, + }; let (f, a) = self.args.take().expect("AndThenFuture has already finished."); - f(item, a).into_future() + f(item, a) + } + Either::Right(future) => { + return match TryFuture::try_poll(future.as_mut(), cx) { + Poll::Ready(Ok(v)) => Poll::Ready(Ok(v)), + Poll::Ready(Err(err)) => return Poll::Ready(Err(EitherError::B(err))), + Poll::Pending => Poll::Pending, + } } - Either::B(ref mut future) => return future.poll().map_err(EitherError::B) }; - self.inner = Either::B(future); + self.inner = Either::Right(Box::pin(future)); } } } +impl Unpin for AndThenFuture { +} diff --git a/core/src/transport/boxed.rs b/core/src/transport/boxed.rs index 735894233c2..3d7b95b3733 100644 --- a/core/src/transport/boxed.rs +++ b/core/src/transport/boxed.rs @@ -21,7 +21,7 @@ use crate::transport::{ListenerEvent, Transport, TransportError}; use futures::prelude::*; use multiaddr::Multiaddr; -use std::{error, fmt, sync::Arc}; +use std::{error, fmt, pin::Pin, sync::Arc}; /// See the `Transport::boxed` method. #[inline] @@ -37,9 +37,9 @@ where } } -pub type Dial = Box + Send>; -pub type Listener = Box>, Error = E> + Send>; -pub type ListenerUpgrade = Box + Send>; +pub type Dial = Pin> + Send>>; +pub type Listener = Pin>, E>> + Send>>; +pub type ListenerUpgrade = Pin> + Send>>; trait Abstract { fn listen_on(&self, addr: Multiaddr) -> Result, TransportError>; @@ -56,15 +56,15 @@ where { fn listen_on(&self, addr: Multiaddr) -> Result, TransportError> { let listener = Transport::listen_on(self.clone(), addr)?; - let fut = listener.map(|event| event.map(|upgrade| { - Box::new(upgrade) as ListenerUpgrade + let fut = listener.map_ok(|event| event.map(|upgrade| { + Box::pin(upgrade) as ListenerUpgrade })); - Ok(Box::new(fut) as Box<_>) + Ok(Box::pin(fut)) } fn dial(&self, addr: Multiaddr) -> Result, TransportError> { let fut = Transport::dial(self.clone(), addr)?; - Ok(Box::new(fut) as Box<_>) + Ok(Box::pin(fut) as Dial<_, _>) } } diff --git a/core/src/transport/dummy.rs b/core/src/transport/dummy.rs index 4d47801639c..f3256b27802 100644 --- a/core/src/transport/dummy.rs +++ b/core/src/transport/dummy.rs @@ -20,7 +20,8 @@ use crate::transport::{Transport, TransportError, ListenerEvent}; use crate::Multiaddr; -use std::{fmt, io, marker::PhantomData}; +use futures::{prelude::*, task::Context, task::Poll}; +use std::{fmt, io, marker::PhantomData, pin::Pin}; /// Implementation of `Transport` that doesn't support any multiaddr. /// @@ -55,9 +56,9 @@ impl Clone for DummyTransport { impl Transport for DummyTransport { type Output = TOut; type Error = io::Error; - type Listener = futures::stream::Empty, io::Error>; - type ListenerUpgrade = futures::future::Empty; - type Dial = futures::future::Empty; + type Listener = futures::stream::Pending, io::Error>>; + type ListenerUpgrade = futures::future::Pending>; + type Dial = futures::future::Pending>; fn listen_on(self, addr: Multiaddr) -> Result> { Err(TransportError::MultiaddrNotSupported(addr)) @@ -68,7 +69,7 @@ impl Transport for DummyTransport { } } -/// Implementation of `Read` and `Write`. Not meant to be instanciated. +/// Implementation of `AsyncRead` and `AsyncWrite`. Not meant to be instanciated. pub struct DummyStream(()); impl fmt::Debug for DummyStream { @@ -77,30 +78,30 @@ impl fmt::Debug for DummyStream { } } -impl io::Read for DummyStream { - fn read(&mut self, _: &mut [u8]) -> io::Result { - Err(io::ErrorKind::Other.into()) +impl AsyncRead for DummyStream { + fn poll_read(self: Pin<&mut Self>, _: &mut Context, _: &mut [u8]) + -> Poll> + { + Poll::Ready(Err(io::ErrorKind::Other.into())) } } -impl io::Write for DummyStream { - fn write(&mut self, _: &[u8]) -> io::Result { - Err(io::ErrorKind::Other.into()) +impl AsyncWrite for DummyStream { + fn poll_write(self: Pin<&mut Self>, _: &mut Context, _: &[u8]) + -> Poll> + { + Poll::Ready(Err(io::ErrorKind::Other.into())) } - fn flush(&mut self) -> io::Result<()> { - Err(io::ErrorKind::Other.into()) + fn poll_flush(self: Pin<&mut Self>, _: &mut Context) + -> Poll> + { + Poll::Ready(Err(io::ErrorKind::Other.into())) } -} - -impl tokio_io::AsyncRead for DummyStream { - unsafe fn prepare_uninitialized_buffer(&self, _: &mut [u8]) -> bool { - false - } -} -impl tokio_io::AsyncWrite for DummyStream { - fn shutdown(&mut self) -> futures::Poll<(), io::Error> { - Err(io::ErrorKind::Other.into()) + fn poll_close(self: Pin<&mut Self>, _: &mut Context) + -> Poll> + { + Poll::Ready(Err(io::ErrorKind::Other.into())) } } diff --git a/core/src/transport/map.rs b/core/src/transport/map.rs index 53f49b75d03..33772cf2b0e 100644 --- a/core/src/transport/map.rs +++ b/core/src/transport/map.rs @@ -22,8 +22,9 @@ use crate::{ ConnectedPoint, transport::{Transport, TransportError, ListenerEvent} }; -use futures::{prelude::*, try_ready}; +use futures::prelude::*; use multiaddr::Multiaddr; +use std::{pin::Pin, task::Context, task::Poll}; /// See `Transport::map`. #[derive(Debug, Copy, Clone)] @@ -61,21 +62,22 @@ where /// Custom `Stream` implementation to avoid boxing. /// /// Maps a function over every stream item. +#[pin_project::pin_project] #[derive(Clone, Debug)] -pub struct MapStream { stream: T, fun: F } +pub struct MapStream { #[pin] stream: T, fun: F } impl Stream for MapStream where - T: Stream>, - X: Future, + T: TryStream>, + X: TryFuture, F: FnOnce(A, ConnectedPoint) -> B + Clone { - type Item = ListenerEvent>; - type Error = T::Error; + type Item = Result>, T::Error>; - fn poll(&mut self) -> Poll, Self::Error> { - match self.stream.poll()? { - Async::Ready(Some(event)) => { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + match TryStream::try_poll_next(this.stream, cx) { + Poll::Ready(Some(Ok(event))) => { let event = match event { ListenerEvent::Upgrade { upgrade, local_addr, remote_addr } => { let point = ConnectedPoint::Listener { @@ -85,7 +87,7 @@ where ListenerEvent::Upgrade { upgrade: MapFuture { inner: upgrade, - args: Some((self.fun.clone(), point)) + args: Some((this.fun.clone(), point)) }, local_addr, remote_addr @@ -94,10 +96,11 @@ where ListenerEvent::NewAddress(a) => ListenerEvent::NewAddress(a), ListenerEvent::AddressExpired(a) => ListenerEvent::AddressExpired(a) }; - Ok(Async::Ready(Some(event))) + Poll::Ready(Some(Ok(event))) } - Async::Ready(None) => Ok(Async::Ready(None)), - Async::NotReady => Ok(Async::NotReady) + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending } } } @@ -105,24 +108,29 @@ where /// Custom `Future` to avoid boxing. /// /// Applies a function to the inner future's result. +#[pin_project::pin_project] #[derive(Clone, Debug)] pub struct MapFuture { + #[pin] inner: T, args: Option<(F, ConnectedPoint)> } impl Future for MapFuture where - T: Future, + T: TryFuture, F: FnOnce(A, ConnectedPoint) -> B { - type Item = B; - type Error = T::Error; + type Output = Result; - fn poll(&mut self) -> Poll { - let item = try_ready!(self.inner.poll()); - let (f, a) = self.args.take().expect("MapFuture has already finished."); - Ok(Async::Ready(f(item, a))) + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + let item = match TryFuture::try_poll(this.inner, cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(Ok(v)) => v, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err)), + }; + let (f, a) = this.args.take().expect("MapFuture has already finished."); + Poll::Ready(Ok(f(item, a))) } } - diff --git a/core/src/transport/map_err.rs b/core/src/transport/map_err.rs index 0642c681880..ba36114682b 100644 --- a/core/src/transport/map_err.rs +++ b/core/src/transport/map_err.rs @@ -21,7 +21,7 @@ use crate::transport::{Transport, TransportError, ListenerEvent}; use futures::prelude::*; use multiaddr::Multiaddr; -use std::error; +use std::{error, pin::Pin, task::Context, task::Poll}; /// See `Transport::map_err`. #[derive(Debug, Copy, Clone)] @@ -67,7 +67,9 @@ where } /// Listening stream for `MapErr`. +#[pin_project::pin_project] pub struct MapErrListener { + #[pin] inner: T::Listener, map: F, } @@ -78,29 +80,32 @@ where F: FnOnce(T::Error) -> TErr + Clone, TErr: error::Error, { - type Item = ListenerEvent>; - type Error = TErr; + type Item = Result>, TErr>; - fn poll(&mut self) -> Poll, Self::Error> { - match self.inner.poll() { - Ok(Async::Ready(Some(event))) => { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + match TryStream::try_poll_next(this.inner, cx) { + Poll::Ready(Some(Ok(event))) => { + let map = &*this.map; let event = event.map(move |value| { MapErrListenerUpgrade { inner: value, - map: Some(self.map.clone()) + map: Some(map.clone()) } }); - Ok(Async::Ready(Some(event))) + Poll::Ready(Some(Ok(event))) } - Ok(Async::Ready(None)) => Ok(Async::Ready(None)), - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => Err((self.map.clone())(err)), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err((this.map.clone())(err)))), } } } /// Listening upgrade future for `MapErr`. +#[pin_project::pin_project] pub struct MapErrListenerUpgrade { + #[pin] inner: T::ListenerUpgrade, map: Option, } @@ -109,23 +114,25 @@ impl Future for MapErrListenerUpgrade where T: Transport, F: FnOnce(T::Error) -> TErr, { - type Item = T::Output; - type Error = TErr; + type Output = Result; - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(value)) => Ok(Async::Ready(value)), - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => { - let map = self.map.take().expect("poll() called again after error"); - Err(map(err)) + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + match Future::poll(this.inner, cx) { + Poll::Ready(Ok(value)) => Poll::Ready(Ok(value)), + Poll::Pending => Poll::Pending, + Poll::Ready(Err(err)) => { + let map = this.map.take().expect("poll() called again after error"); + Poll::Ready(Err(map(err))) } } } } /// Dialing future for `MapErr`. +#[pin_project::pin_project] pub struct MapErrDial { + #[pin] inner: T::Dial, map: Option, } @@ -135,18 +142,16 @@ where T: Transport, F: FnOnce(T::Error) -> TErr, { - type Item = T::Output; - type Error = TErr; + type Output = Result; - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(value)) => { - Ok(Async::Ready(value)) - }, - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => { - let map = self.map.take().expect("poll() called again after error"); - Err(map(err)) + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + match Future::poll(this.inner, cx) { + Poll::Ready(Ok(value)) => Poll::Ready(Ok(value)), + Poll::Pending => Poll::Pending, + Poll::Ready(Err(err)) => { + let map = this.map.take().expect("poll() called again after error"); + Poll::Ready(Err(map(err))) } } } diff --git a/core/src/transport/memory.rs b/core/src/transport/memory.rs index 1b399509dbe..1fbc448ebbe 100644 --- a/core/src/transport/memory.rs +++ b/core/src/transport/memory.rs @@ -19,17 +19,16 @@ // DEALINGS IN THE SOFTWARE. use crate::{Transport, transport::{TransportError, ListenerEvent}}; -use bytes::{Bytes, IntoBuf}; use fnv::FnvHashMap; -use futures::{future::{self, FutureResult}, prelude::*, sync::mpsc, try_ready}; +use futures::{future::{self, Ready}, prelude::*, channel::mpsc, task::Context, task::Poll}; use lazy_static::lazy_static; use multiaddr::{Protocol, Multiaddr}; use parking_lot::Mutex; use rw_stream_sink::RwStreamSink; -use std::{collections::hash_map::Entry, error, fmt, io, num::NonZeroU64}; +use std::{collections::hash_map::Entry, error, fmt, io, num::NonZeroU64, pin::Pin}; lazy_static! { - static ref HUB: Mutex>>> = + static ref HUB: Mutex>>>> = Mutex::new(FnvHashMap::default()); } @@ -39,40 +38,38 @@ pub struct MemoryTransport; /// Connection to a `MemoryTransport` currently being opened. pub struct DialFuture { - sender: mpsc::Sender>, - channel_to_send: Option>, - channel_to_return: Option>, + sender: mpsc::Sender>>, + channel_to_send: Option>>, + channel_to_return: Option>>, } impl Future for DialFuture { - type Item = Channel; - type Error = MemoryTransportError; + type Output = Result>, MemoryTransportError>; - fn poll(&mut self) -> Poll { - if let Some(c) = self.channel_to_send.take() { - match self.sender.start_send(c) { - Err(_) => return Err(MemoryTransportError::Unreachable), - Ok(AsyncSink::NotReady(t)) => { - self.channel_to_send = Some(t); - return Ok(Async::NotReady) - }, - _ => (), - } + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + match self.sender.poll_ready(cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(Ok(())) => {}, + Poll::Ready(Err(_)) => return Poll::Ready(Err(MemoryTransportError::Unreachable)), } - match self.sender.close() { - Err(_) => Err(MemoryTransportError::Unreachable), - Ok(Async::NotReady) => Ok(Async::NotReady), - Ok(Async::Ready(_)) => Ok(Async::Ready(self.channel_to_return.take() - .expect("Future should not be polled again once complete"))), + + let channel_to_send = self.channel_to_send.take() + .expect("Future should not be polled again once complete"); + match self.sender.start_send(channel_to_send) { + Err(_) => return Poll::Ready(Err(MemoryTransportError::Unreachable)), + Ok(()) => {} } + + Poll::Ready(Ok(self.channel_to_return.take() + .expect("Future should not be polled again once complete"))) } } impl Transport for MemoryTransport { - type Output = Channel; + type Output = Channel>; type Error = MemoryTransportError; type Listener = Listener; - type ListenerUpgrade = FutureResult; + type ListenerUpgrade = Ready>; type Dial = DialFuture; fn listen_on(self, addr: Multiaddr) -> Result> { @@ -170,32 +167,33 @@ pub struct Listener { /// The address we are listening on. addr: Multiaddr, /// Receives incoming connections. - receiver: mpsc::Receiver>, + receiver: mpsc::Receiver>>, /// Generate `ListenerEvent::NewAddress` to inform about our listen address. tell_listen_addr: bool } impl Stream for Listener { - type Item = ListenerEvent, MemoryTransportError>>; - type Error = MemoryTransportError; + type Item = Result>, MemoryTransportError>>>, MemoryTransportError>; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { if self.tell_listen_addr { self.tell_listen_addr = false; - return Ok(Async::Ready(Some(ListenerEvent::NewAddress(self.addr.clone())))) + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(self.addr.clone())))) } - let channel = try_ready!(Ok(self.receiver.poll() - .expect("Life listeners always have a sender."))); - let channel = match channel { - Some(c) => c, - None => return Ok(Async::Ready(None)) + + let channel = match Stream::poll_next(Pin::new(&mut self.receiver), cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(None) => panic!("Alive listeners always have a sender."), + Poll::Ready(Some(v)) => v, }; + let event = ListenerEvent::Upgrade { - upgrade: future::ok(channel), + upgrade: future::ready(Ok(channel)), local_addr: self.addr.clone(), remote_addr: Protocol::Memory(self.port.get()).into() }; - Ok(Async::Ready(Some(event))) + + Poll::Ready(Some(Ok(event))) } } @@ -231,43 +229,48 @@ pub type Channel = RwStreamSink>; /// A channel represents an established, in-memory, logical connection between two endpoints. /// /// Implements `Sink` and `Stream`. -pub struct Chan { +pub struct Chan> { incoming: mpsc::Receiver, outgoing: mpsc::Sender, } +impl Unpin for Chan { +} + impl Stream for Chan { - type Item = T; - type Error = io::Error; + type Item = Result; - #[inline] - fn poll(&mut self) -> Poll, Self::Error> { - self.incoming.poll().map_err(|()| io::ErrorKind::BrokenPipe.into()) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + match Stream::poll_next(Pin::new(&mut self.incoming), cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(None) => Poll::Ready(Some(Err(io::ErrorKind::BrokenPipe.into()))), + Poll::Ready(Some(v)) => Poll::Ready(Some(Ok(v))), + } } } -impl Sink for Chan { - type SinkItem = T; - type SinkError = io::Error; +impl Sink for Chan { + type Error = io::Error; + + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + self.outgoing.poll_ready(cx) + .map(|v| v.map_err(|_| io::ErrorKind::BrokenPipe.into())) + } - #[inline] - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { + fn start_send(mut self: Pin<&mut Self>, item: T) -> Result<(), Self::Error> { self.outgoing.start_send(item).map_err(|_| io::ErrorKind::BrokenPipe.into()) } - #[inline] - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.outgoing.poll_complete().map_err(|_| io::ErrorKind::BrokenPipe.into()) + fn poll_flush(self: Pin<&mut Self>, _: &mut Context) -> Poll> { + Poll::Ready(Ok(())) } - #[inline] - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.outgoing.close().map_err(|_| io::ErrorKind::BrokenPipe.into()) + fn poll_close(self: Pin<&mut Self>, _: &mut Context) -> Poll> { + Poll::Ready(Ok(())) } } -impl Into>> for Chan { - #[inline] +impl> Into>> for Chan { fn into(self) -> RwStreamSink> { RwStreamSink::new(self) } @@ -276,6 +279,8 @@ impl Into>> for Chan { #[cfg(test)] mod tests { use super::*; + use rand::Rng; + use std::io::Write; #[test] fn parse_memory_addr_works() { @@ -308,5 +313,43 @@ mod tests { assert!(transport.dial("/memory/810172461024613".parse().unwrap()).is_ok()); } - // TODO: test that is actually works + #[test] + fn communicating_between_dialer_and_listener() { + let msg = [1, 2, 3]; + + // Setup listener. + + let rand_port = rand::random::().saturating_add(1); + let t1_addr: Multiaddr = format!("/memory/{}", rand_port).parse().unwrap(); + let cloned_t1_addr = t1_addr.clone(); + + let t1 = MemoryTransport::default(); + + let listener = async move { + let listener = t1.listen_on(t1_addr.clone()).unwrap(); + + let upgrade = listener.filter_map(|ev| futures::future::ready( + ListenerEvent::into_upgrade(ev.unwrap()) + )).next().await.unwrap(); + + let mut socket = upgrade.0.await.unwrap(); + + let mut buf = [0; 3]; + socket.read_exact(&mut buf).await.unwrap(); + + assert_eq!(buf, msg); + }; + + // Setup dialer. + + let t2 = MemoryTransport::default(); + let dialer = async move { + let mut socket = t2.dial(cloned_t1_addr).unwrap().await.unwrap(); + socket.write_all(&msg).await.unwrap(); + }; + + // Wait for both to finish. + + futures::executor::block_on(futures::future::join(listener, dialer)); + } } diff --git a/core/src/transport/mod.rs b/core/src/transport/mod.rs index c864bb9c394..b3127cf2bed 100644 --- a/core/src/transport/mod.rs +++ b/core/src/transport/mod.rs @@ -91,7 +91,7 @@ pub trait Transport { /// transport stack. The item must be a [`ListenerUpgrade`](Transport::ListenerUpgrade) future /// that resolves to an [`Output`](Transport::Output) value once all protocol upgrades /// have been applied. - type Listener: Stream, Error = Self::Error>; + type Listener: TryStream, Error = Self::Error>; /// A pending [`Output`](Transport::Output) for an inbound connection, /// obtained from the [`Listener`](Transport::Listener) stream. @@ -102,11 +102,11 @@ pub trait Transport { /// connection, hence further connection setup proceeds asynchronously. /// Once a `ListenerUpgrade` future resolves it yields the [`Output`](Transport::Output) /// of the connection setup process. - type ListenerUpgrade: Future; + type ListenerUpgrade: Future>; /// A pending [`Output`](Transport::Output) for an outbound connection, /// obtained from [dialing](Transport::dial). - type Dial: Future; + type Dial: Future>; /// Listens on the given [`Multiaddr`], producing a stream of pending, inbound connections /// and addresses this transport is listening on (cf. [`ListenerEvent`]). @@ -175,8 +175,8 @@ pub trait Transport { where Self: Sized, C: FnOnce(Self::Output, ConnectedPoint) -> F + Clone, - F: IntoFuture, - ::Error: Error + 'static + F: TryFuture, + ::Error: Error + 'static { and_then::AndThen::new(self, f) } diff --git a/core/src/transport/timeout.rs b/core/src/transport/timeout.rs index 8a2bde99d5f..5effaeb99b1 100644 --- a/core/src/transport/timeout.rs +++ b/core/src/transport/timeout.rs @@ -25,11 +25,9 @@ // TODO: add example use crate::{Multiaddr, Transport, transport::{TransportError, ListenerEvent}}; -use futures::{try_ready, Async, Future, Poll, Stream}; -use log::debug; -use std::{error, fmt, time::Duration}; -use wasm_timer::Timeout; -use wasm_timer::timeout::Error as TimeoutError; +use futures::prelude::*; +use futures_timer::Delay; +use std::{error, fmt, io, pin::Pin, task::Context, task::Poll, time::Duration}; /// A `TransportTimeout` is a `Transport` that wraps another `Transport` and adds /// timeouts to all inbound and outbound connection attempts. @@ -80,8 +78,8 @@ where type Output = InnerTrans::Output; type Error = TransportTimeoutError; type Listener = TimeoutListener; - type ListenerUpgrade = TokioTimerMapErr>; - type Dial = TokioTimerMapErr>; + type ListenerUpgrade = Timeout; + type Dial = Timeout; fn listen_on(self, addr: Multiaddr) -> Result> { let listener = self.inner.listen_on(addr) @@ -98,36 +96,47 @@ where fn dial(self, addr: Multiaddr) -> Result> { let dial = self.inner.dial(addr) .map_err(|err| err.map(TransportTimeoutError::Other))?; - Ok(TokioTimerMapErr { - inner: Timeout::new(dial, self.outgoing_timeout), + Ok(Timeout { + inner: dial, + timer: Delay::new(self.outgoing_timeout), }) } } // TODO: can be removed and replaced with an `impl Stream` once impl Trait is fully stable // in Rust (https://github.com/rust-lang/rust/issues/34511) +#[pin_project::pin_project] pub struct TimeoutListener { + #[pin] inner: InnerStream, timeout: Duration, } impl Stream for TimeoutListener where - InnerStream: Stream> + InnerStream: TryStream>, { - type Item = ListenerEvent>>; - type Error = TransportTimeoutError; - - fn poll(&mut self) -> Poll, Self::Error> { - let poll_out = try_ready!(self.inner.poll().map_err(TransportTimeoutError::Other)); - if let Some(event) = poll_out { - let event = event.map(move |inner_fut| { - TokioTimerMapErr { inner: Timeout::new(inner_fut, self.timeout) } - }); - Ok(Async::Ready(Some(event))) - } else { - Ok(Async::Ready(None)) - } + type Item = Result>, TransportTimeoutError>; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + + let poll_out = match TryStream::try_poll_next(this.inner, cx) { + Poll::Ready(Some(Err(err))) => return Poll::Ready(Some(Err(TransportTimeoutError::Other(err)))), + Poll::Ready(Some(Ok(v))) => v, + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => return Poll::Pending, + }; + + let timeout = *this.timeout; + let event = poll_out.map(move |inner_fut| { + Timeout { + inner: inner_fut, + timer: Delay::new(timeout), + } + }); + + Poll::Ready(Some(Ok(event))) } } @@ -135,41 +144,48 @@ where /// `TransportTimeoutError`. // TODO: can be replaced with `impl Future` once `impl Trait` are fully stable in Rust // (https://github.com/rust-lang/rust/issues/34511) +#[pin_project::pin_project] #[must_use = "futures do nothing unless polled"] -pub struct TokioTimerMapErr { +pub struct Timeout { + #[pin] inner: InnerFut, + timer: Delay, } -impl Future for TokioTimerMapErr +impl Future for Timeout where - InnerFut: Future>, + InnerFut: TryFuture, { - type Item = InnerFut::Item; - type Error = TransportTimeoutError; - - fn poll(&mut self) -> Poll { - self.inner.poll().map_err(|err: TimeoutError| { - if err.is_inner() { - TransportTimeoutError::Other(err.into_inner().expect("ensured by is_inner()")) - } else if err.is_elapsed() { - debug!("timeout elapsed for connection"); - TransportTimeoutError::Timeout - } else { - assert!(err.is_timer()); - debug!("tokio timer error in timeout wrapper"); - TransportTimeoutError::TimerError - } - }) + type Output = Result>; + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + // It is debatable whether we should poll the inner future first or the timer first. + // For example, if you start dialing with a timeout of 10 seconds, then after 15 seconds + // the dialing succeeds on the wire, then after 20 seconds you poll, then depending on + // which gets polled first, the outcome will be success or failure. + + let mut this = self.project(); + + match TryFuture::try_poll(this.inner, cx) { + Poll::Pending => {}, + Poll::Ready(Ok(v)) => return Poll::Ready(Ok(v)), + Poll::Ready(Err(err)) => return Poll::Ready(Err(TransportTimeoutError::Other(err))), + } + + match Pin::new(&mut this.timer).poll(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(()) => Poll::Ready(Err(TransportTimeoutError::Timeout)) + } } } /// Error that can be produced by the `TransportTimeout` layer. -#[derive(Debug, Copy, Clone)] +#[derive(Debug)] pub enum TransportTimeoutError { /// The transport timed out. Timeout, /// An error happened in the timer. - TimerError, + TimerError(io::Error), /// Other kind of error. Other(TErr), } @@ -180,7 +196,7 @@ where TErr: fmt::Display, fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { TransportTimeoutError::Timeout => write!(f, "Timeout has been reached"), - TransportTimeoutError::TimerError => write!(f, "Error in the timer"), + TransportTimeoutError::TimerError(err) => write!(f, "Error in the timer: {}", err), TransportTimeoutError::Other(err) => write!(f, "{}", err), } } @@ -192,7 +208,7 @@ where TErr: error::Error + 'static, fn source(&self) -> Option<&(dyn error::Error + 'static)> { match self { TransportTimeoutError::Timeout => None, - TransportTimeoutError::TimerError => None, + TransportTimeoutError::TimerError(err) => Some(err), TransportTimeoutError::Other(err) => Some(err), } } diff --git a/core/src/transport/upgrade.rs b/core/src/transport/upgrade.rs index aad6fa5fe3e..64a182a6481 100644 --- a/core/src/transport/upgrade.rs +++ b/core/src/transport/upgrade.rs @@ -25,6 +25,7 @@ pub use crate::upgrade::Version; use crate::{ ConnectedPoint, ConnectionInfo, + Negotiated, transport::{ Transport, TransportError, @@ -43,10 +44,9 @@ use crate::{ InboundUpgradeApply } }; -use futures::{future, prelude::*, try_ready}; +use futures::{prelude::*, ready}; use multiaddr::Multiaddr; -use std::{error::Error, fmt}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{error::Error, fmt, pin::Pin, task::Context, task::Poll}; /// A `Builder` facilitates upgrading of a [`Transport`] for use with /// a [`Network`]. @@ -102,10 +102,10 @@ where > where T: Transport, I: ConnectionInfo, - C: AsyncRead + AsyncWrite, - D: AsyncRead + AsyncWrite, - U: InboundUpgrade, - U: OutboundUpgrade + Clone, + C: AsyncRead + AsyncWrite + Unpin, + D: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade, Output = (I, D), Error = E>, + U: OutboundUpgrade, Output = (I, D), Error = E> + Clone, E: Error + 'static, { let version = self.version; @@ -130,11 +130,11 @@ where pub fn apply(self, upgrade: U) -> Builder> where T: Transport, - C: AsyncRead + AsyncWrite, - D: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, + D: AsyncRead + AsyncWrite + Unpin, I: ConnectionInfo, - U: InboundUpgrade, - U: OutboundUpgrade + Clone, + U: InboundUpgrade, Output = D, Error = E>, + U: OutboundUpgrade, Output = D, Error = E> + Clone, E: Error + 'static, { Builder::new(Upgrade::new(self.inner, upgrade), self.version) @@ -155,11 +155,11 @@ where -> AndThen Multiplex + Clone> where T: Transport, - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, M: StreamMuxer, I: ConnectionInfo, - U: InboundUpgrade, - U: OutboundUpgrade + Clone, + U: InboundUpgrade, Output = M, Error = E>, + U: OutboundUpgrade, Output = M, Error = E> + Clone, E: Error + 'static, { let version = self.version; @@ -174,27 +174,29 @@ where /// in the context of negotiating a secure channel. /// /// Configured through [`Builder::authenticate`]. +#[pin_project::pin_project] pub struct Authenticate where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + OutboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> + OutboundUpgrade> { + #[pin] inner: EitherUpgrade } impl Future for Authenticate where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + OutboundUpgrade>::Output, - Error = >::Error + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> + OutboundUpgrade, + Output = >>::Output, + Error = >>::Error > { - type Item = as Future>::Item; - type Error = as Future>::Error; + type Output = as Future>::Output; - fn poll(&mut self) -> Poll { - self.inner.poll() + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + Future::poll(this.inner, cx) } } @@ -202,28 +204,33 @@ where /// top of an authenticated transport. /// /// Configured through [`Builder::multiplex`]. +#[pin_project::pin_project] pub struct Multiplex where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + OutboundUpgrade, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> + OutboundUpgrade>, { info: Option, + #[pin] upgrade: EitherUpgrade, } impl Future for Multiplex where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade, - U: OutboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade, Output = M, Error = E>, + U: OutboundUpgrade, Output = M, Error = E> { - type Item = (I, M); - type Error = UpgradeError; - - fn poll(&mut self) -> Poll { - let m = try_ready!(self.upgrade.poll()); - let i = self.info.take().expect("Multiplex future polled after completion."); - Ok(Async::Ready((i, m))) + type Output = Result<(I, M), UpgradeError>; + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + let m = match ready!(Future::poll(this.upgrade, cx)) { + Ok(m) => m, + Err(err) => return Poll::Ready(Err(err)), + }; + let i = this.info.take().expect("Multiplex future polled after completion."); + Poll::Ready(Ok((i, m))) } } @@ -246,9 +253,9 @@ impl Transport for Upgrade where T: Transport, T::Error: 'static, - C: AsyncRead + AsyncWrite, - U: InboundUpgrade, - U: OutboundUpgrade + Clone, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade, Output = D, Error = E>, + U: OutboundUpgrade, Output = D, Error = E> + Clone, E: Error + 'static { type Output = (I, D); @@ -261,8 +268,8 @@ where let future = self.inner.dial(addr.clone()) .map_err(|err| err.map(TransportUpgradeError::Transport))?; Ok(DialUpgradeFuture { - future, - upgrade: future::Either::A(Some(self.upgrade)) + future: Box::pin(future), + upgrade: future::Either::Left(Some(self.upgrade)) }) } @@ -270,7 +277,7 @@ where let stream = self.inner.listen_on(addr) .map_err(|err| err.map(TransportUpgradeError::Transport))?; Ok(ListenerStream { - stream, + stream: Box::pin(stream), upgrade: self.upgrade }) } @@ -314,108 +321,144 @@ where /// The [`Transport::Dial`] future of an [`Upgrade`]d transport. pub struct DialUpgradeFuture where - U: OutboundUpgrade, - C: AsyncRead + AsyncWrite, + U: OutboundUpgrade>, + C: AsyncRead + AsyncWrite + Unpin, { - future: F, + future: Pin>, upgrade: future::Either, (Option, OutboundUpgradeApply)> } impl Future for DialUpgradeFuture where - F: Future, - C: AsyncRead + AsyncWrite, - U: OutboundUpgrade, + F: TryFuture, + C: AsyncRead + AsyncWrite + Unpin, + U: OutboundUpgrade, Output = D>, U::Error: Error { - type Item = (I, D); - type Error = TransportUpgradeError; + type Output = Result<(I, D), TransportUpgradeError>; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + // We use a `this` variable because the compiler can't mutably borrow multiple times + // accross a `Deref`. + let this = &mut *self; - fn poll(&mut self) -> Poll { loop { - self.upgrade = match self.upgrade { - future::Either::A(ref mut up) => { - let (i, c) = try_ready!(self.future.poll().map_err(TransportUpgradeError::Transport)); - let u = up.take().expect("DialUpgradeFuture is constructed with Either::A(Some)."); - future::Either::B((Some(i), apply_outbound(c, u, upgrade::Version::V1))) + this.upgrade = match this.upgrade { + future::Either::Left(ref mut up) => { + let (i, c) = match ready!(TryFuture::try_poll(this.future.as_mut(), cx).map_err(TransportUpgradeError::Transport)) { + Ok(v) => v, + Err(err) => return Poll::Ready(Err(err)), + }; + let u = up.take().expect("DialUpgradeFuture is constructed with Either::Left(Some)."); + future::Either::Right((Some(i), apply_outbound(c, u, upgrade::Version::V1))) } - future::Either::B((ref mut i, ref mut up)) => { - let d = try_ready!(up.poll().map_err(TransportUpgradeError::Upgrade)); + future::Either::Right((ref mut i, ref mut up)) => { + let d = match ready!(Future::poll(Pin::new(up), cx).map_err(TransportUpgradeError::Upgrade)) { + Ok(d) => d, + Err(err) => return Poll::Ready(Err(err)), + }; let i = i.take().expect("DialUpgradeFuture polled after completion."); - return Ok(Async::Ready((i, d))) + return Poll::Ready(Ok((i, d))) } } } } } +impl Unpin for DialUpgradeFuture +where + U: OutboundUpgrade>, + C: AsyncRead + AsyncWrite + Unpin, +{ +} + /// The [`Transport::Listener`] stream of an [`Upgrade`]d transport. pub struct ListenerStream { - stream: S, + stream: Pin>, upgrade: U } impl Stream for ListenerStream where - S: Stream>, - F: Future, - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + Clone + S: TryStream>, + F: TryFuture, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade, Output = D> + Clone { - type Item = ListenerEvent>; - type Error = TransportUpgradeError; + type Item = Result>, TransportUpgradeError>; - fn poll(&mut self) -> Poll, Self::Error> { - match try_ready!(self.stream.poll().map_err(TransportUpgradeError::Transport)) { - Some(event) => { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + match ready!(TryStream::try_poll_next(self.stream.as_mut(), cx)) { + Some(Ok(event)) => { let event = event.map(move |future| { ListenerUpgradeFuture { - future, - upgrade: future::Either::A(Some(self.upgrade.clone())) + future: Box::pin(future), + upgrade: future::Either::Left(Some(self.upgrade.clone())) } }); - Ok(Async::Ready(Some(event))) + Poll::Ready(Some(Ok(event))) + } + Some(Err(err)) => { + Poll::Ready(Some(Err(TransportUpgradeError::Transport(err)))) } - None => Ok(Async::Ready(None)) + None => Poll::Ready(None) } } } +impl Unpin for ListenerStream { +} + /// The [`Transport::ListenerUpgrade`] future of an [`Upgrade`]d transport. pub struct ListenerUpgradeFuture where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> { - future: F, + future: Pin>, upgrade: future::Either, (Option, InboundUpgradeApply)> } impl Future for ListenerUpgradeFuture where - F: Future, - C: AsyncRead + AsyncWrite, - U: InboundUpgrade, + F: TryFuture, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade, Output = D>, U::Error: Error { - type Item = (I, D); - type Error = TransportUpgradeError; + type Output = Result<(I, D), TransportUpgradeError>; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + // We use a `this` variable because the compiler can't mutably borrow multiple times + // accross a `Deref`. + let this = &mut *self; - fn poll(&mut self) -> Poll { loop { - self.upgrade = match self.upgrade { - future::Either::A(ref mut up) => { - let (i, c) = try_ready!(self.future.poll().map_err(TransportUpgradeError::Transport)); - let u = up.take().expect("ListenerUpgradeFuture is constructed with Either::A(Some)."); - future::Either::B((Some(i), apply_inbound(c, u))) + this.upgrade = match this.upgrade { + future::Either::Left(ref mut up) => { + let (i, c) = match ready!(TryFuture::try_poll(this.future.as_mut(), cx).map_err(TransportUpgradeError::Transport)) { + Ok(v) => v, + Err(err) => return Poll::Ready(Err(err)) + }; + let u = up.take().expect("ListenerUpgradeFuture is constructed with Either::Left(Some)."); + future::Either::Right((Some(i), apply_inbound(c, u))) } - future::Either::B((ref mut i, ref mut up)) => { - let d = try_ready!(up.poll().map_err(TransportUpgradeError::Upgrade)); + future::Either::Right((ref mut i, ref mut up)) => { + let d = match ready!(TryFuture::try_poll(Pin::new(up), cx).map_err(TransportUpgradeError::Upgrade)) { + Ok(v) => v, + Err(err) => return Poll::Ready(Err(err)) + }; let i = i.take().expect("ListenerUpgradeFuture polled after completion."); - return Ok(Async::Ready((i, d))) + return Poll::Ready(Ok((i, d))) } } } } } +impl Unpin for ListenerUpgradeFuture +where + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> +{ +} diff --git a/core/src/upgrade/apply.rs b/core/src/upgrade/apply.rs index 13912c33c9b..219766831bd 100644 --- a/core/src/upgrade/apply.rs +++ b/core/src/upgrade/apply.rs @@ -18,14 +18,12 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::ConnectedPoint; -use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError}; -use crate::upgrade::ProtocolName; -use futures::{future::Either, prelude::*}; +use crate::{ConnectedPoint, Negotiated}; +use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError, ProtocolName}; +use futures::{future::Either, prelude::*, compat::Compat, compat::Compat01As03, compat::Future01CompatExt}; use log::debug; use multistream_select::{self, DialerSelectFuture, ListenerSelectFuture}; -use std::{iter, mem}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{iter, mem, pin::Pin, task::Context, task::Poll}; pub use multistream_select::Version; @@ -33,24 +31,24 @@ pub use multistream_select::Version; pub fn apply(conn: C, up: U, cp: ConnectedPoint, v: Version) -> Either, OutboundUpgradeApply> where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + OutboundUpgrade, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> + OutboundUpgrade>, { if cp.is_listener() { - Either::A(apply_inbound(conn, up)) + Either::Left(apply_inbound(conn, up)) } else { - Either::B(apply_outbound(conn, up, v)) + Either::Right(apply_outbound(conn, up, v)) } } /// Tries to perform an upgrade on an inbound connection or substream. pub fn apply_inbound(conn: C, up: U) -> InboundUpgradeApply where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade>, { let iter = up.protocol_info().into_iter().map(NameWrap as fn(_) -> NameWrap<_>); - let future = multistream_select::listener_select_proto(conn, iter); + let future = multistream_select::listener_select_proto(Compat::new(conn), iter).compat(); InboundUpgradeApply { inner: InboundUpgradeApplyState::Init { future, upgrade: up } } @@ -59,11 +57,11 @@ where /// Tries to perform an upgrade on an outbound connection or substream. pub fn apply_outbound(conn: C, up: U, v: Version) -> OutboundUpgradeApply where - C: AsyncRead + AsyncWrite, - U: OutboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: OutboundUpgrade> { let iter = up.protocol_info().into_iter().map(NameWrap as fn(_) -> NameWrap<_>); - let future = multistream_select::dialer_select_proto(conn, iter, v); + let future = multistream_select::dialer_select_proto(Compat::new(conn), iter, v).compat(); OutboundUpgradeApply { inner: OutboundUpgradeApplyState::Init { future, upgrade: up } } @@ -72,63 +70,69 @@ where /// Future returned by `apply_inbound`. Drives the upgrade process. pub struct InboundUpgradeApply where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade> { inner: InboundUpgradeApplyState } enum InboundUpgradeApplyState where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade>, { Init { - future: ListenerSelectFuture>, + future: Compat01As03, NameWrap>>, upgrade: U, }, Upgrade { - future: U::Future + future: Pin> }, Undefined } +impl Unpin for InboundUpgradeApply +where + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade>, +{ +} + impl Future for InboundUpgradeApply where - C: AsyncRead + AsyncWrite, - U: InboundUpgrade, + C: AsyncRead + AsyncWrite + Unpin, + U: InboundUpgrade>, { - type Item = U::Output; - type Error = UpgradeError; + type Output = Result>; - fn poll(&mut self) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { loop { match mem::replace(&mut self.inner, InboundUpgradeApplyState::Undefined) { InboundUpgradeApplyState::Init { mut future, upgrade } => { - let (info, io) = match future.poll()? { - Async::Ready(x) => x, - Async::NotReady => { + let (info, io) = match Future::poll(Pin::new(&mut future), cx)? { + Poll::Ready(x) => x, + Poll::Pending => { self.inner = InboundUpgradeApplyState::Init { future, upgrade }; - return Ok(Async::NotReady) + return Poll::Pending } }; self.inner = InboundUpgradeApplyState::Upgrade { - future: upgrade.upgrade_inbound(io, info.0) + future: Box::pin(upgrade.upgrade_inbound(Compat01As03::new(io), info.0)) }; } InboundUpgradeApplyState::Upgrade { mut future } => { - match future.poll() { - Ok(Async::NotReady) => { + match Future::poll(Pin::new(&mut future), cx) { + Poll::Pending => { self.inner = InboundUpgradeApplyState::Upgrade { future }; - return Ok(Async::NotReady) + return Poll::Pending } - Ok(Async::Ready(x)) => { + Poll::Ready(Ok(x)) => { debug!("Successfully applied negotiated protocol"); - return Ok(Async::Ready(x)) + return Poll::Ready(Ok(x)) } - Err(e) => { + Poll::Ready(Err(e)) => { debug!("Failed to apply negotiated protocol"); - return Err(UpgradeError::Apply(e)) + return Poll::Ready(Err(UpgradeError::Apply(e))) } } } @@ -142,63 +146,69 @@ where /// Future returned by `apply_outbound`. Drives the upgrade process. pub struct OutboundUpgradeApply where - C: AsyncRead + AsyncWrite, - U: OutboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: OutboundUpgrade> { inner: OutboundUpgradeApplyState } enum OutboundUpgradeApplyState where - C: AsyncRead + AsyncWrite, - U: OutboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: OutboundUpgrade> { Init { - future: DialerSelectFuture::IntoIter>>, + future: Compat01As03, NameWrapIter<::IntoIter>>>, upgrade: U }, Upgrade { - future: U::Future + future: Pin> }, Undefined } +impl Unpin for OutboundUpgradeApply +where + C: AsyncRead + AsyncWrite + Unpin, + U: OutboundUpgrade>, +{ +} + impl Future for OutboundUpgradeApply where - C: AsyncRead + AsyncWrite, - U: OutboundUpgrade + C: AsyncRead + AsyncWrite + Unpin, + U: OutboundUpgrade>, { - type Item = U::Output; - type Error = UpgradeError; + type Output = Result>; - fn poll(&mut self) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { loop { match mem::replace(&mut self.inner, OutboundUpgradeApplyState::Undefined) { OutboundUpgradeApplyState::Init { mut future, upgrade } => { - let (info, connection) = match future.poll()? { - Async::Ready(x) => x, - Async::NotReady => { + let (info, connection) = match Future::poll(Pin::new(&mut future), cx)? { + Poll::Ready(x) => x, + Poll::Pending => { self.inner = OutboundUpgradeApplyState::Init { future, upgrade }; - return Ok(Async::NotReady) + return Poll::Pending } }; self.inner = OutboundUpgradeApplyState::Upgrade { - future: upgrade.upgrade_outbound(connection, info.0) + future: Box::pin(upgrade.upgrade_outbound(Compat01As03::new(connection), info.0)) }; } OutboundUpgradeApplyState::Upgrade { mut future } => { - match future.poll() { - Ok(Async::NotReady) => { + match Future::poll(Pin::new(&mut future), cx) { + Poll::Pending => { self.inner = OutboundUpgradeApplyState::Upgrade { future }; - return Ok(Async::NotReady) + return Poll::Pending } - Ok(Async::Ready(x)) => { + Poll::Ready(Ok(x)) => { debug!("Successfully applied negotiated protocol"); - return Ok(Async::Ready(x)) + return Poll::Ready(Ok(x)) } - Err(e) => { + Poll::Ready(Err(e)) => { debug!("Failed to apply negotiated protocol"); - return Err(UpgradeError::Apply(e)) + return Poll::Ready(Err(UpgradeError::Apply(e))); } } } diff --git a/core/src/upgrade/denied.rs b/core/src/upgrade/denied.rs index 9dec47ee6ed..93438e0bea8 100644 --- a/core/src/upgrade/denied.rs +++ b/core/src/upgrade/denied.rs @@ -20,7 +20,6 @@ use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use futures::future; -use multistream_select::Negotiated; use std::iter; use void::Void; @@ -41,20 +40,19 @@ impl UpgradeInfo for DeniedUpgrade { impl InboundUpgrade for DeniedUpgrade { type Output = Void; type Error = Void; - type Future = future::Empty; + type Future = future::Pending>; - fn upgrade_inbound(self, _: Negotiated, _: Self::Info) -> Self::Future { - future::empty() + fn upgrade_inbound(self, _: C, _: Self::Info) -> Self::Future { + future::pending() } } impl OutboundUpgrade for DeniedUpgrade { type Output = Void; type Error = Void; - type Future = future::Empty; + type Future = future::Pending>; - fn upgrade_outbound(self, _: Negotiated, _: Self::Info) -> Self::Future { - future::empty() + fn upgrade_outbound(self, _: C, _: Self::Info) -> Self::Future { + future::pending() } } - diff --git a/core/src/upgrade/either.rs b/core/src/upgrade/either.rs index bf3d86b826d..28db987ccd7 100644 --- a/core/src/upgrade/either.rs +++ b/core/src/upgrade/either.rs @@ -22,7 +22,6 @@ use crate::{ either::{EitherOutput, EitherError, EitherFuture2, EitherName}, upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo} }; -use multistream_select::Negotiated; /// A type to represent two possible upgrade types (inbound or outbound). #[derive(Debug, Clone)] @@ -56,7 +55,7 @@ where type Error = EitherError; type Future = EitherFuture2; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { match (self, info) { (EitherUpgrade::A(a), EitherName::A(info)) => EitherFuture2::A(a.upgrade_inbound(sock, info)), (EitherUpgrade::B(b), EitherName::B(info)) => EitherFuture2::B(b.upgrade_inbound(sock, info)), @@ -74,7 +73,7 @@ where type Error = EitherError; type Future = EitherFuture2; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { match (self, info) { (EitherUpgrade::A(a), EitherName::A(info)) => EitherFuture2::A(a.upgrade_outbound(sock, info)), (EitherUpgrade::B(b), EitherName::B(info)) => EitherFuture2::B(b.upgrade_outbound(sock, info)), diff --git a/core/src/upgrade/map.rs b/core/src/upgrade/map.rs index ee17b845d35..d55971df82d 100644 --- a/core/src/upgrade/map.rs +++ b/core/src/upgrade/map.rs @@ -19,8 +19,8 @@ // DEALINGS IN THE SOFTWARE. use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; -use futures::{prelude::*, try_ready}; -use multistream_select::Negotiated; +use futures::prelude::*; +use std::{pin::Pin, task::Context, task::Poll}; /// Wraps around an upgrade and applies a closure to the output. #[derive(Debug, Clone)] @@ -53,7 +53,7 @@ where type Error = U::Error; type Future = MapFuture; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { MapFuture { inner: self.upgrade.upgrade_inbound(sock, info), map: Some(self.fun) @@ -63,13 +63,13 @@ where impl OutboundUpgrade for MapInboundUpgrade where - U: OutboundUpgrade + U: OutboundUpgrade, { type Output = U::Output; type Error = U::Error; type Future = U::Future; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { self.upgrade.upgrade_outbound(sock, info) } } @@ -98,13 +98,13 @@ where impl InboundUpgrade for MapOutboundUpgrade where - U: InboundUpgrade + U: InboundUpgrade, { type Output = U::Output; type Error = U::Error; type Future = U::Future; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { self.upgrade.upgrade_inbound(sock, info) } } @@ -118,7 +118,7 @@ where type Error = U::Error; type Future = MapFuture; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { MapFuture { inner: self.upgrade.upgrade_outbound(sock, info), map: Some(self.fun) @@ -157,7 +157,7 @@ where type Error = T; type Future = MapErrFuture; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { MapErrFuture { fut: self.upgrade.upgrade_inbound(sock, info), fun: Some(self.fun) @@ -167,13 +167,13 @@ where impl OutboundUpgrade for MapInboundUpgradeErr where - U: OutboundUpgrade + U: OutboundUpgrade, { type Output = U::Output; type Error = U::Error; type Future = U::Future; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { self.upgrade.upgrade_outbound(sock, info) } } @@ -209,7 +209,7 @@ where type Error = T; type Future = MapErrFuture; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { MapErrFuture { fut: self.upgrade.upgrade_outbound(sock, info), fun: Some(self.fun) @@ -225,51 +225,60 @@ where type Error = U::Error; type Future = U::Future; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { self.upgrade.upgrade_inbound(sock, info) } } +#[pin_project::pin_project] pub struct MapFuture { + #[pin] inner: TInnerFut, map: Option, } impl Future for MapFuture where - TInnerFut: Future, + TInnerFut: TryFuture, TMap: FnOnce(TIn) -> TOut, { - type Item = TOut; - type Error = TInnerFut::Error; - - fn poll(&mut self) -> Poll { - let item = try_ready!(self.inner.poll()); - let map = self.map.take().expect("Future has already finished"); - Ok(Async::Ready(map(item))) + type Output = Result; + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + let item = match TryFuture::try_poll(this.inner, cx) { + Poll::Ready(Ok(v)) => v, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err)), + Poll::Pending => return Poll::Pending, + }; + + let map = this.map.take().expect("Future has already finished"); + Poll::Ready(Ok(map(item))) } } +#[pin_project::pin_project] pub struct MapErrFuture { + #[pin] fut: T, fun: Option, } impl Future for MapErrFuture where - T: Future, + T: TryFuture, F: FnOnce(E) -> A, { - type Item = T::Item; - type Error = A; - - fn poll(&mut self) -> Poll { - match self.fut.poll() { - Ok(Async::NotReady) => Ok(Async::NotReady), - Ok(Async::Ready(x)) => Ok(Async::Ready(x)), - Err(e) => { - let f = self.fun.take().expect("Future has not resolved yet"); - Err(f(e)) + type Output = Result; + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + match TryFuture::try_poll(this.fut, cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(Ok(x)) => Poll::Ready(Ok(x)), + Poll::Ready(Err(e)) => { + let f = this.fun.take().expect("Future has not resolved yet"); + Poll::Ready(Err(f(e))) } } } diff --git a/core/src/upgrade/mod.rs b/core/src/upgrade/mod.rs index d9bb7b33154..cc870693592 100644 --- a/core/src/upgrade/mod.rs +++ b/core/src/upgrade/mod.rs @@ -68,7 +68,8 @@ mod transfer; use futures::future::Future; -pub use multistream_select::{Version, Negotiated, NegotiatedComplete, NegotiationError, ProtocolError}; +pub use crate::Negotiated; +pub use multistream_select::{Version, NegotiatedComplete, NegotiationError, ProtocolError}; pub use self::{ apply::{apply, apply_inbound, apply_outbound, InboundUpgradeApply, OutboundUpgradeApply}, denied::DeniedUpgrade, @@ -77,7 +78,7 @@ pub use self::{ map::{MapInboundUpgrade, MapOutboundUpgrade, MapInboundUpgradeErr, MapOutboundUpgradeErr}, optional::OptionalUpgrade, select::SelectUpgrade, - transfer::{write_one, WriteOne, read_one, ReadOne, read_one_then, ReadOneThen, ReadOneError, request_response, RequestResponse, read_respond, ReadRespond}, + transfer::{write_one, write_with_len_prefix, write_varint, read_one, ReadOneError, read_varint}, }; /// Types serving as protocol names. @@ -143,13 +144,13 @@ pub trait InboundUpgrade: UpgradeInfo { /// Possible error during the handshake. type Error; /// Future that performs the handshake with the remote. - type Future: Future; + type Future: Future>; /// After we have determined that the remote supports one of the protocols we support, this /// method is called to start the handshake. /// /// The `info` is the identifier of the protocol, as produced by `protocol_info`. - fn upgrade_inbound(self, socket: Negotiated, info: Self::Info) -> Self::Future; + fn upgrade_inbound(self, socket: C, info: Self::Info) -> Self::Future; } /// Extension trait for `InboundUpgrade`. Automatically implemented on all types that implement @@ -183,13 +184,13 @@ pub trait OutboundUpgrade: UpgradeInfo { /// Possible error during the handshake. type Error; /// Future that performs the handshake with the remote. - type Future: Future; + type Future: Future>; /// After we have determined that the remote supports one of the protocols we support, this /// method is called to start the handshake. /// /// The `info` is the identifier of the protocol, as produced by `protocol_info`. - fn upgrade_outbound(self, socket: Negotiated, info: Self::Info) -> Self::Future; + fn upgrade_outbound(self, socket: C, info: Self::Info) -> Self::Future; } /// Extention trait for `OutboundUpgrade`. Automatically implemented on all types that implement diff --git a/core/src/upgrade/optional.rs b/core/src/upgrade/optional.rs index b822d5b9309..02dc3c48f78 100644 --- a/core/src/upgrade/optional.rs +++ b/core/src/upgrade/optional.rs @@ -19,7 +19,6 @@ // DEALINGS IN THE SOFTWARE. use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; -use multistream_select::Negotiated; /// Upgrade that can be disabled at runtime. /// @@ -60,7 +59,7 @@ where type Error = T::Error; type Future = T::Future; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { if let Some(inner) = self.0 { inner.upgrade_inbound(sock, info) } else { @@ -77,7 +76,7 @@ where type Error = T::Error; type Future = T::Future; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { if let Some(inner) = self.0 { inner.upgrade_outbound(sock, info) } else { diff --git a/core/src/upgrade/select.rs b/core/src/upgrade/select.rs index 61c3ec5e501..8fa4c5b8a7a 100644 --- a/core/src/upgrade/select.rs +++ b/core/src/upgrade/select.rs @@ -22,7 +22,6 @@ use crate::{ either::{EitherOutput, EitherError, EitherFuture2, EitherName}, upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo} }; -use multistream_select::Negotiated; /// Upgrade that combines two upgrades into one. Supports all the protocols supported by either /// sub-upgrade. @@ -65,7 +64,7 @@ where type Error = EitherError; type Future = EitherFuture2; - fn upgrade_inbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_inbound(self, sock: C, info: Self::Info) -> Self::Future { match info { EitherName::A(info) => EitherFuture2::A(self.0.upgrade_inbound(sock, info)), EitherName::B(info) => EitherFuture2::B(self.1.upgrade_inbound(sock, info)) @@ -82,7 +81,7 @@ where type Error = EitherError; type Future = EitherFuture2; - fn upgrade_outbound(self, sock: Negotiated, info: Self::Info) -> Self::Future { + fn upgrade_outbound(self, sock: C, info: Self::Info) -> Self::Future { match info { EitherName::A(info) => EitherFuture2::A(self.0.upgrade_outbound(sock, info)), EitherName::B(info) => EitherFuture2::B(self.1.upgrade_outbound(sock, info)) diff --git a/core/src/upgrade/transfer.rs b/core/src/upgrade/transfer.rs index dd5aebcb0a0..28a9c29806e 100644 --- a/core/src/upgrade/transfer.rs +++ b/core/src/upgrade/transfer.rs @@ -20,104 +20,93 @@ //! Contains some helper futures for creating upgrades. -use futures::{prelude::*, try_ready}; -use std::{cmp, error, fmt, io::Cursor, mem}; -use tokio_io::{io, AsyncRead, AsyncWrite}; +use futures::prelude::*; +use std::{error, fmt, io}; + +// TODO: these methods could be on an Ext trait to AsyncWrite /// Send a message to the given socket, then shuts down the writing side. /// /// > **Note**: Prepends a variable-length prefix indicate the length of the message. This is /// > compatible with what `read_one` expects. -pub fn write_one(socket: TSocket, data: TData) -> WriteOne -where - TSocket: AsyncWrite, - TData: AsRef<[u8]>, +pub async fn write_one(socket: &mut (impl AsyncWrite + Unpin), data: impl AsRef<[u8]>) + -> Result<(), io::Error> { - let len_data = build_int_buffer(data.as_ref().len()); - WriteOne { - inner: WriteOneInner::WriteLen(io::write_all(socket, len_data), data), - } -} - -/// Builds a buffer that contains the given integer encoded as variable-length. -fn build_int_buffer(num: usize) -> io::Window<[u8; 10]> { - let mut len_data = unsigned_varint::encode::u64_buffer(); - let encoded_len = unsigned_varint::encode::u64(num as u64, &mut len_data).len(); - let mut len_data = io::Window::new(len_data); - len_data.set_end(encoded_len); - len_data + write_varint(socket, data.as_ref().len()).await?; + socket.write_all(data.as_ref()).await?; + socket.close().await?; + Ok(()) } -/// Future that makes `write_one` work. -#[derive(Debug)] -pub struct WriteOne> { - inner: WriteOneInner, -} - -#[derive(Debug)] -enum WriteOneInner { - /// We need to write the data length to the socket. - WriteLen(io::WriteAll>, TData), - /// We need to write the actual data to the socket. - Write(io::WriteAll), - /// We need to shut down the socket. - Shutdown(io::Shutdown), - /// A problem happened during the processing. - Poisoned, +/// Send a message to the given socket with a length prefix appended to it. Also flushes the socket. +/// +/// > **Note**: Prepends a variable-length prefix indicate the length of the message. This is +/// > compatible with what `read_one` expects. +pub async fn write_with_len_prefix(socket: &mut (impl AsyncWrite + Unpin), data: impl AsRef<[u8]>) + -> Result<(), io::Error> +{ + write_varint(socket, data.as_ref().len()).await?; + socket.write_all(data.as_ref()).await?; + socket.flush().await?; + Ok(()) } -impl Future for WriteOne -where - TSocket: AsyncWrite, - TData: AsRef<[u8]>, +/// Writes a variable-length integer to the `socket`. +/// +/// > **Note**: Does **NOT** flush the socket. +pub async fn write_varint(socket: &mut (impl AsyncWrite + Unpin), len: usize) + -> Result<(), io::Error> { - type Item = (); - type Error = std::io::Error; - - fn poll(&mut self) -> Poll { - Ok(self.inner.poll()?.map(|_socket| ())) - } + let mut len_data = unsigned_varint::encode::usize_buffer(); + let encoded_len = unsigned_varint::encode::usize(len, &mut len_data).len(); + socket.write_all(&len_data[..encoded_len]).await?; + Ok(()) } -impl Future for WriteOneInner -where - TSocket: AsyncWrite, - TData: AsRef<[u8]>, -{ - type Item = TSocket; - type Error = std::io::Error; - - fn poll(&mut self) -> Poll { - loop { - match mem::replace(self, WriteOneInner::Poisoned) { - WriteOneInner::WriteLen(mut inner, data) => match inner.poll()? { - Async::Ready((socket, _)) => { - *self = WriteOneInner::Write(io::write_all(socket, data)); - } - Async::NotReady => { - *self = WriteOneInner::WriteLen(inner, data); - } - }, - WriteOneInner::Write(mut inner) => match inner.poll()? { - Async::Ready((socket, _)) => { - *self = WriteOneInner::Shutdown(tokio_io::io::shutdown(socket)); - } - Async::NotReady => { - *self = WriteOneInner::Write(inner); - } - }, - WriteOneInner::Shutdown(ref mut inner) => { - let socket = try_ready!(inner.poll()); - return Ok(Async::Ready(socket)); +/// Reads a variable-length integer from the `socket`. +/// +/// As a special exception, if the `socket` is empty and EOFs right at the beginning, then we +/// return `Ok(0)`. +/// +/// > **Note**: This function reads bytes one by one from the `socket`. It is therefore encouraged +/// > to use some sort of buffering mechanism. +pub async fn read_varint(socket: &mut (impl AsyncRead + Unpin)) -> Result { + let mut buffer = unsigned_varint::encode::usize_buffer(); + let mut buffer_len = 0; + + loop { + match socket.read(&mut buffer[buffer_len..buffer_len+1]).await? { + 0 => { + // Reaching EOF before finishing to read the length is an error, unless the EOF is + // at the very beginning of the substream, in which case we assume that the data is + // empty. + if buffer_len == 0 { + return Ok(0); + } else { + return Err(io::ErrorKind::UnexpectedEof.into()); } - WriteOneInner::Poisoned => panic!(), } + n => debug_assert_eq!(n, 1), + } + + buffer_len += 1; + + match unsigned_varint::decode::usize(&buffer[..buffer_len]) { + Ok((len, _)) => return Ok(len), + Err(unsigned_varint::decode::Error::Overflow) => { + return Err(io::Error::new( + io::ErrorKind::InvalidData, + "overflow in variable-length integer" + )); + } + // TODO: why do we have a `__Nonexhaustive` variant in the error? I don't know how to process it + // Err(unsigned_varint::decode::Error::Insufficient) => {} + Err(_) => {} } } } -/// Reads a message from the given socket. Only one message is processed and the socket is dropped, -/// because we assume that the socket will not send anything more. +/// Reads a length-prefixed message from the given socket. /// /// The `max_size` parameter is the maximum size in bytes of the message that we accept. This is /// necessary in order to avoid DoS attacks where the remote sends us a message of several @@ -125,137 +114,20 @@ where /// /// > **Note**: Assumes that a variable-length prefix indicates the length of the message. This is /// > compatible with what `write_one` does. -pub fn read_one( - socket: TSocket, - max_size: usize, -) -> ReadOne +pub async fn read_one(socket: &mut (impl AsyncRead + Unpin), max_size: usize) + -> Result, ReadOneError> { - ReadOne { - inner: ReadOneInner::ReadLen { - socket, - len_buf: Cursor::new([0; 10]), - max_size, - }, - } -} - -/// Future that makes `read_one` work. -#[derive(Debug)] -pub struct ReadOne { - inner: ReadOneInner, -} - -#[derive(Debug)] -enum ReadOneInner { - // We need to read the data length from the socket. - ReadLen { - socket: TSocket, - /// A small buffer where we will right the variable-length integer representing the - /// length of the actual packet. - len_buf: Cursor<[u8; 10]>, - max_size: usize, - }, - // We need to read the actual data from the socket. - ReadRest(io::ReadExact>>), - /// A problem happened during the processing. - Poisoned, -} - -impl Future for ReadOne -where - TSocket: AsyncRead, -{ - type Item = Vec; - type Error = ReadOneError; - - fn poll(&mut self) -> Poll { - Ok(self.inner.poll()?.map(|(_, out)| out)) + let len = read_varint(socket).await?; + if len > max_size { + return Err(ReadOneError::TooLarge { + requested: len, + max: max_size, + }); } -} -impl Future for ReadOneInner -where - TSocket: AsyncRead, -{ - type Item = (TSocket, Vec); - type Error = ReadOneError; - - fn poll(&mut self) -> Poll { - loop { - match mem::replace(self, ReadOneInner::Poisoned) { - ReadOneInner::ReadLen { - mut socket, - mut len_buf, - max_size, - } => { - match socket.read_buf(&mut len_buf)? { - Async::Ready(num_read) => { - // Reaching EOF before finishing to read the length is an error, unless - // the EOF is at the very beginning of the substream, in which case we - // assume that the data is empty. - if num_read == 0 { - if len_buf.position() == 0 { - return Ok(Async::Ready((socket, Vec::new()))); - } else { - return Err(ReadOneError::Io( - std::io::ErrorKind::UnexpectedEof.into(), - )); - } - } - - let len_buf_with_data = - &len_buf.get_ref()[..len_buf.position() as usize]; - if let Ok((len, data_start)) = - unsigned_varint::decode::usize(len_buf_with_data) - { - if len >= max_size { - return Err(ReadOneError::TooLarge { - requested: len, - max: max_size, - }); - } - - // Create `data_buf` containing the start of the data that was - // already in `len_buf`. - let n = cmp::min(data_start.len(), len); - let mut data_buf = vec![0; len]; - data_buf[.. n].copy_from_slice(&data_start[.. n]); - let mut data_buf = io::Window::new(data_buf); - data_buf.set_start(data_start.len()); - *self = ReadOneInner::ReadRest(io::read_exact(socket, data_buf)); - } else { - *self = ReadOneInner::ReadLen { - socket, - len_buf, - max_size, - }; - } - } - Async::NotReady => { - *self = ReadOneInner::ReadLen { - socket, - len_buf, - max_size, - }; - return Ok(Async::NotReady); - } - } - } - ReadOneInner::ReadRest(mut inner) => { - match inner.poll()? { - Async::Ready((socket, data)) => { - return Ok(Async::Ready((socket, data.into_inner()))); - } - Async::NotReady => { - *self = ReadOneInner::ReadRest(inner); - return Ok(Async::NotReady); - } - } - } - ReadOneInner::Poisoned => panic!(), - } - } - } + let mut buf = vec![0; len]; + socket.read_exact(&mut buf).await?; + Ok(buf) } /// Error while reading one message. @@ -296,194 +168,9 @@ impl error::Error for ReadOneError { } } -/// Similar to `read_one`, but applies a transformation on the output buffer. -/// -/// > **Note**: The `param` parameter is an arbitrary value that will be passed back to `then`. -/// > This parameter is normally not necessary, as we could just pass a closure that has -/// > ownership of any data we want. In practice, though, this would make the -/// > `ReadRespond` type impossible to express as a concrete type. Once the `impl Trait` -/// > syntax is allowed within traits, we can remove this parameter. -pub fn read_one_then( - socket: TSocket, - max_size: usize, - param: TParam, - then: TThen, -) -> ReadOneThen -where - TSocket: AsyncRead, - TThen: FnOnce(Vec, TParam) -> Result, - TErr: From, -{ - ReadOneThen { - inner: read_one(socket, max_size), - then: Some((param, then)), - } -} - -/// Future that makes `read_one_then` work. -#[derive(Debug)] -pub struct ReadOneThen { - inner: ReadOne, - then: Option<(TParam, TThen)>, -} - -impl Future for ReadOneThen -where - TSocket: AsyncRead, - TThen: FnOnce(Vec, TParam) -> Result, - TErr: From, -{ - type Item = TOut; - type Error = TErr; - - fn poll(&mut self) -> Poll { - match self.inner.poll()? { - Async::Ready(buffer) => { - let (param, then) = self.then.take() - .expect("Future was polled after it was finished"); - Ok(Async::Ready(then(buffer, param)?)) - }, - Async::NotReady => Ok(Async::NotReady), - } - } -} - -/// Similar to `read_one`, but applies a transformation on the output buffer. -/// -/// > **Note**: The `param` parameter is an arbitrary value that will be passed back to `then`. -/// > This parameter is normally not necessary, as we could just pass a closure that has -/// > ownership of any data we want. In practice, though, this would make the -/// > `ReadRespond` type impossible to express as a concrete type. Once the `impl Trait` -/// > syntax is allowed within traits, we can remove this parameter. -pub fn read_respond( - socket: TSocket, - max_size: usize, - param: TParam, - then: TThen, -) -> ReadRespond -where - TSocket: AsyncRead, - TThen: FnOnce(TSocket, Vec, TParam) -> Result, - TErr: From, -{ - ReadRespond { - inner: read_one(socket, max_size).inner, - then: Some((then, param)), - } -} - -/// Future that makes `read_respond` work. -#[derive(Debug)] -pub struct ReadRespond { - inner: ReadOneInner, - then: Option<(TThen, TParam)>, -} - -impl Future for ReadRespond -where - TSocket: AsyncRead, - TThen: FnOnce(TSocket, Vec, TParam) -> Result, - TErr: From, -{ - type Item = TOut; - type Error = TErr; - - fn poll(&mut self) -> Poll { - match self.inner.poll()? { - Async::Ready((socket, buffer)) => { - let (then, param) = self.then.take().expect("Future was polled after it was finished"); - Ok(Async::Ready(then(socket, buffer, param)?)) - }, - Async::NotReady => Ok(Async::NotReady), - } - } -} - -/// Send a message to the given socket, then shuts down the writing side, then reads an answer. -/// -/// This combines `write_one` followed with `read_one_then`. -/// -/// > **Note**: The `param` parameter is an arbitrary value that will be passed back to `then`. -/// > This parameter is normally not necessary, as we could just pass a closure that has -/// > ownership of any data we want. In practice, though, this would make the -/// > `ReadRespond` type impossible to express as a concrete type. Once the `impl Trait` -/// > syntax is allowed within traits, we can remove this parameter. -pub fn request_response( - socket: TSocket, - data: TData, - max_size: usize, - param: TParam, - then: TThen, -) -> RequestResponse -where - TSocket: AsyncRead + AsyncWrite, - TData: AsRef<[u8]>, - TThen: FnOnce(Vec, TParam) -> Result, -{ - RequestResponse { - inner: RequestResponseInner::Write(write_one(socket, data).inner, max_size, param, then), - } -} - -/// Future that makes `request_response` work. -#[derive(Debug)] -pub struct RequestResponse> { - inner: RequestResponseInner, -} - -#[derive(Debug)] -enum RequestResponseInner { - // We need to write data to the socket. - Write(WriteOneInner, usize, TParam, TThen), - // We need to read the message. - Read(ReadOneThen), - // An error happened during the processing. - Poisoned, -} - -impl Future for RequestResponse -where - TSocket: AsyncRead + AsyncWrite, - TData: AsRef<[u8]>, - TThen: FnOnce(Vec, TParam) -> Result, - TErr: From, -{ - type Item = TOut; - type Error = TErr; - - fn poll(&mut self) -> Poll { - loop { - match mem::replace(&mut self.inner, RequestResponseInner::Poisoned) { - RequestResponseInner::Write(mut inner, max_size, param, then) => { - match inner.poll().map_err(ReadOneError::Io)? { - Async::Ready(socket) => { - self.inner = - RequestResponseInner::Read(read_one_then(socket, max_size, param, then)); - } - Async::NotReady => { - self.inner = RequestResponseInner::Write(inner, max_size, param, then); - return Ok(Async::NotReady); - } - } - } - RequestResponseInner::Read(mut inner) => match inner.poll()? { - Async::Ready(packet) => return Ok(Async::Ready(packet)), - Async::NotReady => { - self.inner = RequestResponseInner::Read(inner); - return Ok(Async::NotReady); - } - }, - RequestResponseInner::Poisoned => panic!(), - }; - } - } -} - #[cfg(test)] mod tests { use super::*; - use std::io::{self, Cursor}; - use tokio::runtime::current_thread::Runtime; #[test] fn write_one_works() { @@ -492,14 +179,17 @@ mod tests { .collect::>(); let mut out = vec![0; 10_000]; - let future = write_one(Cursor::new(&mut out[..]), data.clone()); - Runtime::new().unwrap().block_on(future).unwrap(); + futures::executor::block_on( + write_one(&mut futures::io::Cursor::new(&mut out[..]), data.clone()) + ).unwrap(); let (out_len, out_data) = unsigned_varint::decode::usize(&out).unwrap(); assert_eq!(out_len, data.len()); assert_eq!(&out_data[..out_len], &data[..]); } + // TODO: rewrite these tests +/* #[test] fn read_one_works() { let original_data = (0..rand::random::() % 10_000) @@ -517,7 +207,7 @@ mod tests { Ok(()) }); - Runtime::new().unwrap().block_on(future).unwrap(); + futures::executor::block_on(future).unwrap(); } #[test] @@ -527,7 +217,7 @@ mod tests { Ok(()) }); - Runtime::new().unwrap().block_on(future).unwrap(); + futures::executor::block_on(future).unwrap(); } #[test] @@ -542,7 +232,7 @@ mod tests { Ok(()) }); - match Runtime::new().unwrap().block_on(future) { + match futures::executor::block_on(future) { Err(ReadOneError::TooLarge { .. }) => (), _ => panic!(), } @@ -555,7 +245,7 @@ mod tests { Ok(()) }); - Runtime::new().unwrap().block_on(future).unwrap(); + futures::executor::block_on(future).unwrap(); } #[test] @@ -564,9 +254,9 @@ mod tests { unreachable!() }); - match Runtime::new().unwrap().block_on(future) { + match futures::executor::block_on(future) { Err(ReadOneError::Io(ref err)) if err.kind() == io::ErrorKind::UnexpectedEof => (), _ => panic!() } - } + }*/ } diff --git a/core/tests/network_dial_error.rs b/core/tests/network_dial_error.rs index f5306647508..d36690d66b9 100644 --- a/core/tests/network_dial_error.rs +++ b/core/tests/network_dial_error.rs @@ -20,7 +20,7 @@ mod util; -use futures::{future, prelude::*}; +use futures::prelude::*; use libp2p_core::identity; use libp2p_core::multiaddr::multiaddr; use libp2p_core::nodes::network::{Network, NetworkEvent, NetworkReachError, PeerState, UnknownPeerDialErr, IncomingError}; @@ -34,7 +34,7 @@ use libp2p_swarm::{ protocols_handler::NodeHandlerWrapperBuilder }; use rand::seq::SliceRandom; -use std::io; +use std::{io, task::Context, task::Poll}; // TODO: replace with DummyProtocolsHandler after https://github.com/servo/rust-smallvec/issues/139 ? struct TestHandler(std::marker::PhantomData); @@ -47,7 +47,7 @@ impl Default for TestHandler { impl ProtocolsHandler for TestHandler where - TSubstream: tokio_io::AsyncRead + tokio_io::AsyncWrite + TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static { type InEvent = (); // TODO: cannot be Void (https://github.com/servo/rust-smallvec/issues/139) type OutEvent = (); // TODO: cannot be Void (https://github.com/servo/rust-smallvec/issues/139) @@ -82,8 +82,8 @@ where fn connection_keep_alive(&self) -> KeepAlive { KeepAlive::No } - fn poll(&mut self) -> Poll, Self::Error> { - Ok(Async::NotReady) + fn poll(&mut self, _: &mut Context) -> Poll> { + Poll::Pending } } @@ -113,27 +113,28 @@ fn deny_incoming_connec() { swarm1.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()).unwrap(); - let address = - if let Async::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm1.poll() { - listen_addr + let address = async_std::task::block_on(future::poll_fn(|cx| { + if let Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm1.poll(cx) { + Poll::Ready(listen_addr) } else { panic!("Was expecting the listen address to be reported") - }; + } + })); swarm2 .peer(swarm1.local_peer_id().clone()) .into_not_connected().unwrap() .connect(address.clone(), TestHandler::default().into_node_handler_builder()); - let future = future::poll_fn(|| -> Poll<(), io::Error> { - match swarm1.poll() { - Async::Ready(NetworkEvent::IncomingConnection(inc)) => drop(inc), - Async::Ready(_) => unreachable!(), - Async::NotReady => (), + async_std::task::block_on(future::poll_fn(|cx| -> Poll> { + match swarm1.poll(cx) { + Poll::Ready(NetworkEvent::IncomingConnection(inc)) => drop(inc), + Poll::Ready(_) => unreachable!(), + Poll::Pending => (), } - match swarm2.poll() { - Async::Ready(NetworkEvent::DialError { + match swarm2.poll(cx) { + Poll::Ready(NetworkEvent::DialError { new_state: PeerState::NotConnected, peer_id, multiaddr, @@ -141,16 +142,14 @@ fn deny_incoming_connec() { }) => { assert_eq!(peer_id, *swarm1.local_peer_id()); assert_eq!(multiaddr, address); - return Ok(Async::Ready(())); + return Poll::Ready(Ok(())); }, - Async::Ready(_) => unreachable!(), - Async::NotReady => (), + Poll::Ready(_) => unreachable!(), + Poll::Pending => (), } - Ok(Async::NotReady) - }); - - tokio::runtime::current_thread::Runtime::new().unwrap().block_on(future).unwrap(); + Poll::Pending + })).unwrap(); } #[test] @@ -176,32 +175,31 @@ fn dial_self() { .and_then(|(peer, mplex), _| { // Gracefully close the connection to allow protocol // negotiation to complete. - util::CloseMuxer::new(mplex).map(move |mplex| (peer, mplex)) + util::CloseMuxer::new(mplex).map_ok(move |mplex| (peer, mplex)) }); Network::new(transport, local_public_key.into()) }; swarm.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()).unwrap(); - let (address, mut swarm) = - future::lazy(move || { - if let Async::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm.poll() { + let (address, mut swarm) = async_std::task::block_on( + future::lazy(move |cx| { + if let Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm.poll(cx) { Ok::<_, void::Void>((listen_addr, swarm)) } else { panic!("Was expecting the listen address to be reported") } - }) - .wait() + })) .unwrap(); swarm.dial(address.clone(), TestHandler::default().into_node_handler_builder()).unwrap(); let mut got_dial_err = false; let mut got_inc_err = false; - let future = future::poll_fn(|| -> Poll<(), io::Error> { + async_std::task::block_on(future::poll_fn(|cx| -> Poll> { loop { - match swarm.poll() { - Async::Ready(NetworkEvent::UnknownPeerDialError { + match swarm.poll(cx) { + Poll::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error: UnknownPeerDialErr::FoundLocalPeerId, handler: _ @@ -210,10 +208,10 @@ fn dial_self() { assert!(!got_dial_err); got_dial_err = true; if got_inc_err { - return Ok(Async::Ready(())); + return Poll::Ready(Ok(())); } }, - Async::Ready(NetworkEvent::IncomingConnectionError { + Poll::Ready(NetworkEvent::IncomingConnectionError { local_addr, send_back_addr: _, error: IncomingError::FoundLocalPeerId @@ -222,22 +220,20 @@ fn dial_self() { assert!(!got_inc_err); got_inc_err = true; if got_dial_err { - return Ok(Async::Ready(())); + return Poll::Ready(Ok(())); } }, - Async::Ready(NetworkEvent::IncomingConnection(inc)) => { + Poll::Ready(NetworkEvent::IncomingConnection(inc)) => { assert_eq!(*inc.local_addr(), address); inc.accept(TestHandler::default().into_node_handler_builder()); }, - Async::Ready(ev) => { + Poll::Ready(ev) => { panic!("Unexpected event: {:?}", ev) } - Async::NotReady => break Ok(Async::NotReady), + Poll::Pending => break Poll::Pending, } } - }); - - tokio::runtime::current_thread::Runtime::new().unwrap().block_on(future).unwrap(); + })).unwrap(); } #[test] @@ -288,10 +284,10 @@ fn multiple_addresses_err() { .connect_iter(addresses.clone(), TestHandler::default().into_node_handler_builder()) .unwrap(); - let future = future::poll_fn(|| -> Poll<(), io::Error> { + async_std::task::block_on(future::poll_fn(|cx| -> Poll> { loop { - match swarm.poll() { - Async::Ready(NetworkEvent::DialError { + match swarm.poll(cx) { + Poll::Ready(NetworkEvent::DialError { new_state, peer_id, multiaddr, @@ -302,7 +298,7 @@ fn multiple_addresses_err() { assert_eq!(multiaddr, expected); if addresses.is_empty() { assert_eq!(new_state, PeerState::NotConnected); - return Ok(Async::Ready(())); + return Poll::Ready(Ok(())); } else { match new_state { PeerState::Dialing { num_pending_addresses } => { @@ -312,11 +308,9 @@ fn multiple_addresses_err() { } } }, - Async::Ready(_) => unreachable!(), - Async::NotReady => break Ok(Async::NotReady), + Poll::Ready(_) => unreachable!(), + Poll::Pending => break Poll::Pending, } } - }); - - tokio::runtime::current_thread::Runtime::new().unwrap().block_on(future).unwrap(); + })).unwrap(); } diff --git a/core/tests/network_simult.rs b/core/tests/network_simult.rs index 2db3152e752..d01fea04764 100644 --- a/core/tests/network_simult.rs +++ b/core/tests/network_simult.rs @@ -18,9 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -mod util; - -use futures::{future, prelude::*}; +use futures::prelude::*; use libp2p_core::{identity, upgrade, Transport}; use libp2p_core::nodes::{Network, NetworkEvent, Peer}; use libp2p_core::nodes::network::IncomingError; @@ -31,10 +29,9 @@ use libp2p_swarm::{ ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, }; -use std::{io, time::Duration}; -use wasm_timer::{Delay, Instant}; +use std::{io, task::Context, task::Poll, time::Duration}; +use wasm_timer::Delay; -// TODO: replace with DummyProtocolsHandler after https://github.com/servo/rust-smallvec/issues/139 ? struct TestHandler(std::marker::PhantomData); impl Default for TestHandler { @@ -45,7 +42,7 @@ impl Default for TestHandler { impl ProtocolsHandler for TestHandler where - TSubstream: tokio_io::AsyncRead + tokio_io::AsyncWrite + TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static { type InEvent = (); // TODO: cannot be Void (https://github.com/servo/rust-smallvec/issues/139) type OutEvent = (); // TODO: cannot be Void (https://github.com/servo/rust-smallvec/issues/139) @@ -80,8 +77,8 @@ where fn connection_keep_alive(&self) -> KeepAlive { KeepAlive::Yes } - fn poll(&mut self) -> Poll, Self::Error> { - Ok(Async::NotReady) + fn poll(&mut self, _: &mut Context) -> Poll> { + Poll::Pending } } @@ -112,12 +109,7 @@ fn raw_swarm_simultaneous_connect() { let transport = libp2p_tcp::TcpConfig::new() .upgrade(upgrade::Version::V1Lazy) .authenticate(libp2p_secio::SecioConfig::new(local_key)) - .multiplex(libp2p_mplex::MplexConfig::new()) - .and_then(|(peer, mplex), _| { - // Gracefully close the connection to allow protocol - // negotiation to complete. - util::CloseMuxer::new(mplex).map(move |mplex| (peer, mplex)) - }); + .multiplex(libp2p_mplex::MplexConfig::new()); Network::new(transport, local_public_key.into_peer_id()) }; @@ -127,49 +119,50 @@ fn raw_swarm_simultaneous_connect() { let transport = libp2p_tcp::TcpConfig::new() .upgrade(upgrade::Version::V1Lazy) .authenticate(libp2p_secio::SecioConfig::new(local_key)) - .multiplex(libp2p_mplex::MplexConfig::new()) - .and_then(|(peer, mplex), _| { - // Gracefully close the connection to allow protocol - // negotiation to complete. - util::CloseMuxer::new(mplex).map(move |mplex| (peer, mplex)) - }); + .multiplex(libp2p_mplex::MplexConfig::new()); Network::new(transport, local_public_key.into_peer_id()) }; swarm1.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()).unwrap(); swarm2.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()).unwrap(); - let (swarm1_listen_addr, swarm2_listen_addr, mut swarm1, mut swarm2) = - future::lazy(move || { - let swarm1_listen_addr = - if let Async::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm1.poll() { - listen_addr - } else { - panic!("Was expecting the listen address to be reported") - }; - - let swarm2_listen_addr = - if let Async::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm2.poll() { - listen_addr - } else { - panic!("Was expecting the listen address to be reported") - }; - - Ok::<_, void::Void>((swarm1_listen_addr, swarm2_listen_addr, swarm1, swarm2)) - }) - .wait() - .unwrap(); - - let mut reactor = tokio::runtime::current_thread::Runtime::new().unwrap(); + let swarm1_listen_addr = future::poll_fn(|cx| { + if let Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm1.poll(cx) { + Poll::Ready(listen_addr) + } else { + panic!("Was expecting the listen address to be reported") + } + }) + .now_or_never() + .expect("listen address of swarm1"); + + let swarm2_listen_addr = future::poll_fn(|cx| { + if let Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm2.poll(cx) { + Poll::Ready(listen_addr) + } else { + panic!("Was expecting the listen address to be reported") + } + }) + .now_or_never() + .expect("listen address of swarm2"); + + #[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Ord)] + enum Step { + Start, + Dialing, + Connected, + Replaced, + Denied + } loop { - let mut swarm1_step = 0; - let mut swarm2_step = 0; + let mut swarm1_step = Step::Start; + let mut swarm2_step = Step::Start; - let mut swarm1_dial_start = Delay::new(Instant::now() + Duration::new(0, rand::random::() % 50_000_000)); - let mut swarm2_dial_start = Delay::new(Instant::now() + Duration::new(0, rand::random::() % 50_000_000)); + let mut swarm1_dial_start = Delay::new(Duration::new(0, rand::random::() % 50_000_000)); + let mut swarm2_dial_start = Delay::new(Duration::new(0, rand::random::() % 50_000_000)); - let future = future::poll_fn(|| -> Poll { + let future = future::poll_fn(|cx| { loop { let mut swarm1_not_ready = false; let mut swarm2_not_ready = false; @@ -177,123 +170,127 @@ fn raw_swarm_simultaneous_connect() { // We add a lot of randomness. In a real-life situation the swarm also has to // handle other nodes, which may delay the processing. - if swarm1_step == 0 { - match swarm1_dial_start.poll().unwrap() { - Async::Ready(_) => { - let handler = TestHandler::default().into_node_handler_builder(); - swarm1.peer(swarm2.local_peer_id().clone()) - .into_not_connected() - .unwrap() - .connect(swarm2_listen_addr.clone(), handler); - swarm1_step = 1; - }, - Async::NotReady => swarm1_not_ready = true, + if swarm1_step == Step::Start { + if swarm1_dial_start.poll_unpin(cx).is_ready() { + let handler = TestHandler::default().into_node_handler_builder(); + swarm1.peer(swarm2.local_peer_id().clone()) + .into_not_connected() + .unwrap() + .connect(swarm2_listen_addr.clone(), handler); + swarm1_step = Step::Dialing; + } else { + swarm1_not_ready = true } } - if swarm2_step == 0 { - match swarm2_dial_start.poll().unwrap() { - Async::Ready(_) => { - let handler = TestHandler::default().into_node_handler_builder(); - swarm2.peer(swarm1.local_peer_id().clone()) - .into_not_connected() - .unwrap() - .connect(swarm1_listen_addr.clone(), handler); - swarm2_step = 1; - }, - Async::NotReady => swarm2_not_ready = true, + if swarm2_step == Step::Start { + if swarm2_dial_start.poll_unpin(cx).is_ready() { + let handler = TestHandler::default().into_node_handler_builder(); + swarm2.peer(swarm1.local_peer_id().clone()) + .into_not_connected() + .unwrap() + .connect(swarm1_listen_addr.clone(), handler); + swarm2_step = Step::Dialing; + } else { + swarm2_not_ready = true } } if rand::random::() < 0.1 { - match swarm1.poll() { - Async::Ready(NetworkEvent::IncomingConnectionError { + match swarm1.poll(cx) { + Poll::Ready(NetworkEvent::IncomingConnectionError { error: IncomingError::DeniedLowerPriority, .. }) => { - assert_eq!(swarm1_step, 2); - swarm1_step = 3; - }, - Async::Ready(NetworkEvent::Connected { conn_info, .. }) => { + assert_eq!(swarm1_step, Step::Connected); + swarm1_step = Step::Denied + } + Poll::Ready(NetworkEvent::Connected { conn_info, .. }) => { assert_eq!(conn_info, *swarm2.local_peer_id()); - if swarm1_step == 0 { + if swarm1_step == Step::Start { // The connection was established before // swarm1 started dialing; discard the test run. - return Ok(Async::Ready(false)) + return Poll::Ready(false) } - assert_eq!(swarm1_step, 1); - swarm1_step = 2; - }, - Async::Ready(NetworkEvent::Replaced { new_info, .. }) => { + assert_eq!(swarm1_step, Step::Dialing); + swarm1_step = Step::Connected + } + Poll::Ready(NetworkEvent::Replaced { new_info, .. }) => { assert_eq!(new_info, *swarm2.local_peer_id()); - assert_eq!(swarm1_step, 2); - swarm1_step = 3; - }, - Async::Ready(NetworkEvent::IncomingConnection(inc)) => { - inc.accept(TestHandler::default().into_node_handler_builder()); - }, - Async::Ready(ev) => panic!("swarm1: unexpected event: {:?}", ev), - Async::NotReady => swarm1_not_ready = true, + assert_eq!(swarm1_step, Step::Connected); + swarm1_step = Step::Replaced + } + Poll::Ready(NetworkEvent::IncomingConnection(inc)) => { + inc.accept(TestHandler::default().into_node_handler_builder()) + } + Poll::Ready(ev) => panic!("swarm1: unexpected event: {:?}", ev), + Poll::Pending => swarm1_not_ready = true } } if rand::random::() < 0.1 { - match swarm2.poll() { - Async::Ready(NetworkEvent::IncomingConnectionError { + match swarm2.poll(cx) { + Poll::Ready(NetworkEvent::IncomingConnectionError { error: IncomingError::DeniedLowerPriority, .. }) => { - assert_eq!(swarm2_step, 2); - swarm2_step = 3; - }, - Async::Ready(NetworkEvent::Connected { conn_info, .. }) => { + assert_eq!(swarm2_step, Step::Connected); + swarm2_step = Step::Denied + } + Poll::Ready(NetworkEvent::Connected { conn_info, .. }) => { assert_eq!(conn_info, *swarm1.local_peer_id()); - if swarm2_step == 0 { + if swarm2_step == Step::Start { // The connection was established before // swarm2 started dialing; discard the test run. - return Ok(Async::Ready(false)) + return Poll::Ready(false) } - assert_eq!(swarm2_step, 1); - swarm2_step = 2; - }, - Async::Ready(NetworkEvent::Replaced { new_info, .. }) => { + assert_eq!(swarm2_step, Step::Dialing); + swarm2_step = Step::Connected + } + Poll::Ready(NetworkEvent::Replaced { new_info, .. }) => { assert_eq!(new_info, *swarm1.local_peer_id()); - assert_eq!(swarm2_step, 2); - swarm2_step = 3; - }, - Async::Ready(NetworkEvent::IncomingConnection(inc)) => { - inc.accept(TestHandler::default().into_node_handler_builder()); - }, - Async::Ready(ev) => panic!("swarm2: unexpected event: {:?}", ev), - Async::NotReady => swarm2_not_ready = true, + assert_eq!(swarm2_step, Step::Connected); + swarm2_step = Step::Replaced + } + Poll::Ready(NetworkEvent::IncomingConnection(inc)) => { + inc.accept(TestHandler::default().into_node_handler_builder()) + } + Poll::Ready(ev) => panic!("swarm2: unexpected event: {:?}", ev), + Poll::Pending => swarm2_not_ready = true } } - // TODO: make sure that >= 5 is correct - if swarm1_step + swarm2_step >= 5 { - return Ok(Async::Ready(true)); + match (swarm1_step, swarm2_step) { + | (Step::Connected, Step::Replaced) + | (Step::Connected, Step::Denied) + | (Step::Replaced, Step::Connected) + | (Step::Replaced, Step::Denied) + | (Step::Replaced, Step::Replaced) + | (Step::Denied, Step::Connected) + | (Step::Denied, Step::Replaced) => return Poll::Ready(true), + _else => () } if swarm1_not_ready && swarm2_not_ready { - return Ok(Async::NotReady); + return Poll::Pending } } }); - if reactor.block_on(future).unwrap() { + if async_std::task::block_on(future) { // The test exercised what we wanted to exercise: a simultaneous connect. break - } else { - // The test did not trigger a simultaneous connect; ensure the nodes - // are disconnected and re-run the test. - match swarm1.peer(swarm2.local_peer_id().clone()) { - Peer::Connected(p) => p.close(), - Peer::PendingConnect(p) => p.interrupt(), - x => panic!("Unexpected state for swarm1: {:?}", x) - } - match swarm2.peer(swarm1.local_peer_id().clone()) { - Peer::Connected(p) => p.close(), - Peer::PendingConnect(p) => p.interrupt(), - x => panic!("Unexpected state for swarm2: {:?}", x) - } + } + + // The test did not trigger a simultaneous connect; ensure the nodes + // are disconnected and re-run the test. + match swarm1.peer(swarm2.local_peer_id().clone()) { + Peer::Connected(p) => p.close(), + Peer::PendingConnect(p) => p.interrupt(), + x => panic!("Unexpected state for swarm1: {:?}", x) + } + match swarm2.peer(swarm1.local_peer_id().clone()) { + Peer::Connected(p) => p.close(), + Peer::PendingConnect(p) => p.interrupt(), + x => panic!("Unexpected state for swarm2: {:?}", x) } } } diff --git a/core/tests/transport_upgrade.rs b/core/tests/transport_upgrade.rs index 1620e8fc14c..b4c732b5de2 100644 --- a/core/tests/transport_upgrade.rs +++ b/core/tests/transport_upgrade.rs @@ -20,17 +20,15 @@ mod util; -use futures::future::Future; -use futures::stream::Stream; +use futures::prelude::*; use libp2p_core::identity; -use libp2p_core::transport::{Transport, MemoryTransport, ListenerEvent}; -use libp2p_core::upgrade::{self, UpgradeInfo, Negotiated, InboundUpgrade, OutboundUpgrade}; +use libp2p_core::transport::{Transport, MemoryTransport}; +use libp2p_core::upgrade::{self, UpgradeInfo, InboundUpgrade, OutboundUpgrade}; use libp2p_mplex::MplexConfig; use libp2p_secio::SecioConfig; -use multiaddr::Multiaddr; +use multiaddr::{Multiaddr, Protocol}; use rand::random; -use std::io; -use tokio_io::{io as nio, AsyncWrite, AsyncRead}; +use std::{io, pin::Pin}; #[derive(Clone)] struct HelloUpgrade {} @@ -46,30 +44,36 @@ impl UpgradeInfo for HelloUpgrade { impl InboundUpgrade for HelloUpgrade where - C: AsyncRead + AsyncWrite + Send + 'static + C: AsyncRead + AsyncWrite + Send + Unpin + 'static { - type Output = Negotiated; + type Output = C; type Error = io::Error; - type Future = Box + Send>; + type Future = Pin> + Send>>; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - Box::new(nio::read_exact(socket, [0u8; 5]).map(|(io, buf)| { + fn upgrade_inbound(self, mut socket: C, _: Self::Info) -> Self::Future { + Box::pin(async move { + let mut buf = [0u8; 5]; + socket.read_exact(&mut buf).await.unwrap(); assert_eq!(&buf[..], "hello".as_bytes()); - io - })) + Ok(socket) + }) } } impl OutboundUpgrade for HelloUpgrade where - C: AsyncWrite + AsyncRead + Send + 'static, + C: AsyncWrite + AsyncRead + Send + Unpin + 'static, { - type Output = Negotiated; + type Output = C; type Error = io::Error; - type Future = Box + Send>; + type Future = Pin> + Send>>; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - Box::new(nio::write_all(socket, "hello").map(|(io, _)| io)) + fn upgrade_outbound(self, mut socket: C, _: Self::Info) -> Self::Future { + Box::pin(async move { + socket.write_all(b"hello").await.unwrap(); + socket.flush().await.unwrap(); + Ok(socket) + }) } } @@ -87,7 +91,7 @@ fn upgrade_pipeline() { .and_then(|(peer, mplex), _| { // Gracefully close the connection to allow protocol // negotiation to complete. - util::CloseMuxer::new(mplex).map(move |mplex| (peer, mplex)) + util::CloseMuxer::new(mplex).map_ok(move |mplex| (peer, mplex)) }); let dialer_keys = identity::Keypair::generate_ed25519(); @@ -102,27 +106,32 @@ fn upgrade_pipeline() { .and_then(|(peer, mplex), _| { // Gracefully close the connection to allow protocol // negotiation to complete. - util::CloseMuxer::new(mplex).map(move |mplex| (peer, mplex)) + util::CloseMuxer::new(mplex).map_ok(move |mplex| (peer, mplex)) }); - let listen_addr: Multiaddr = format!("/memory/{}", random::()).parse().unwrap(); - let listener = listener_transport.listen_on(listen_addr.clone()).unwrap() - .filter_map(ListenerEvent::into_upgrade) - .for_each(move |(upgrade, _remote_addr)| { - let dialer = dialer_id.clone(); - upgrade.map(move |(peer, _mplex)| { - assert_eq!(peer, dialer) - }) - }) - .map_err(|e| panic!("Listener error: {}", e)); + let listen_addr1 = Multiaddr::from(Protocol::Memory(random::())); + let listen_addr2 = listen_addr1.clone(); - let dialer = dialer_transport.dial(listen_addr).unwrap() - .map(move |(peer, _mplex)| { - assert_eq!(peer, listener_id) - }); + let mut listener = listener_transport.listen_on(listen_addr1).unwrap(); + + let server = async move { + loop { + let (upgrade, _remote_addr) = + match listener.next().await.unwrap().unwrap().into_upgrade() { + Some(u) => u, + None => continue + }; + let (peer, _mplex) = upgrade.await.unwrap(); + assert_eq!(peer, dialer_id); + } + }; + + let client = async move { + let (peer, _mplex) = dialer_transport.dial(listen_addr2).unwrap().await.unwrap(); + assert_eq!(peer, listener_id); + }; - let mut rt = tokio::runtime::Runtime::new().unwrap(); - rt.spawn(listener); - rt.block_on(dialer).unwrap() + async_std::task::spawn(server); + async_std::task::block_on(client); } diff --git a/core/tests/util.rs b/core/tests/util.rs index b43442822cb..395e0d9cd2b 100644 --- a/core/tests/util.rs +++ b/core/tests/util.rs @@ -3,6 +3,7 @@ use futures::prelude::*; use libp2p_core::muxing::StreamMuxer; +use std::{pin::Pin, task::Context, task::Poll}; pub struct CloseMuxer { state: CloseMuxerState, @@ -26,18 +27,17 @@ where M: StreamMuxer, M::Error: From { - type Item = M; - type Error = M::Error; + type Output = Result; - fn poll(&mut self) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { loop { match std::mem::replace(&mut self.state, CloseMuxerState::Done) { CloseMuxerState::Close(muxer) => { - if muxer.close()?.is_not_ready() { + if !muxer.close(cx)?.is_ready() { self.state = CloseMuxerState::Close(muxer); - return Ok(Async::NotReady) + return Poll::Pending } - return Ok(Async::Ready(muxer)) + return Poll::Ready(Ok(muxer)) } CloseMuxerState::Done => panic!() } @@ -45,3 +45,5 @@ where } } +impl Unpin for CloseMuxer { +} diff --git a/examples/chat.rs b/examples/chat.rs index 183973ae1d4..607d2e0e64c 100644 --- a/examples/chat.rs +++ b/examples/chat.rs @@ -49,20 +49,21 @@ //! //! The two nodes then connect. -use futures::prelude::*; +use async_std::{io, task}; +use futures::{future, prelude::*}; use libp2p::{ + Multiaddr, PeerId, Swarm, NetworkBehaviour, identity, - tokio_codec::{FramedRead, LinesCodec}, - tokio_io::{AsyncRead, AsyncWrite}, floodsub::{self, Floodsub, FloodsubEvent}, mdns::{Mdns, MdnsEvent}, swarm::NetworkBehaviourEventProcess }; +use std::{error::Error, task::{Context, Poll}}; -fn main() { +fn main() -> Result<(), Box> { env_logger::init(); // Create a random PeerId @@ -71,52 +72,60 @@ fn main() { println!("Local peer id: {:?}", local_peer_id); // Set up a an encrypted DNS-enabled TCP Transport over the Mplex and Yamux protocols - let transport = libp2p::build_development_transport(local_key); + let transport = libp2p::build_development_transport(local_key)?; // Create a Floodsub topic let floodsub_topic = floodsub::TopicBuilder::new("chat").build(); // We create a custom network behaviour that combines floodsub and mDNS. // In the future, we want to improve libp2p to make this easier to do. + // Use the derive to generate delegating NetworkBehaviour impl and require the + // NetworkBehaviourEventProcess implementations below. #[derive(NetworkBehaviour)] struct MyBehaviour { floodsub: Floodsub, mdns: Mdns, + + // Struct fields which do not implement NetworkBehaviour need to be ignored + #[behaviour(ignore)] + #[allow(dead_code)] + ignored_member: bool, + } + + impl NetworkBehaviourEventProcess for MyBehaviour { + // Called when `floodsub` produces an event. + fn inject_event(&mut self, message: FloodsubEvent) { + if let FloodsubEvent::Message(message) = message { + println!("Received: '{:?}' from {:?}", String::from_utf8_lossy(&message.data), message.source); + } + } } impl NetworkBehaviourEventProcess for MyBehaviour { + // Called when `mdns` produces an event. fn inject_event(&mut self, event: MdnsEvent) { match event { - MdnsEvent::Discovered(list) => { + MdnsEvent::Discovered(list) => for (peer, _) in list { self.floodsub.add_node_to_partial_view(peer); } - }, - MdnsEvent::Expired(list) => { + MdnsEvent::Expired(list) => for (peer, _) in list { if !self.mdns.has_node(&peer) { self.floodsub.remove_node_from_partial_view(&peer); } } - } - } - } - } - - impl NetworkBehaviourEventProcess for MyBehaviour { - // Called when `floodsub` produces an event. - fn inject_event(&mut self, message: FloodsubEvent) { - if let FloodsubEvent::Message(message) = message { - println!("Received: '{:?}' from {:?}", String::from_utf8_lossy(&message.data), message.source); } } } // Create a Swarm to manage peers and events let mut swarm = { + let mdns = task::block_on(Mdns::new())?; let mut behaviour = MyBehaviour { floodsub: Floodsub::new(local_peer_id.clone()), - mdns: Mdns::new().expect("Failed to create mDNS service"), + mdns, + ignored_member: false, }; behaviour.floodsub.subscribe(floodsub_topic.clone()); @@ -125,45 +134,35 @@ fn main() { // Reach out to another node if specified if let Some(to_dial) = std::env::args().nth(1) { - let dialing = to_dial.clone(); - match to_dial.parse() { - Ok(to_dial) => { - match libp2p::Swarm::dial_addr(&mut swarm, to_dial) { - Ok(_) => println!("Dialed {:?}", dialing), - Err(e) => println!("Dial {:?} failed: {:?}", dialing, e) - } - }, - Err(err) => println!("Failed to parse address to dial: {:?}", err), - } + let addr: Multiaddr = to_dial.parse()?; + Swarm::dial_addr(&mut swarm, addr)?; + println!("Dialed {:?}", to_dial) } // Read full lines from stdin - let stdin = tokio_stdin_stdout::stdin(0); - let mut framed_stdin = FramedRead::new(stdin, LinesCodec::new()); + let mut stdin = io::BufReader::new(io::stdin()).lines(); // Listen on all interfaces and whatever port the OS assigns - Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); + Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse()?)?; // Kick it off let mut listening = false; - tokio::run(futures::future::poll_fn(move || -> Result<_, ()> { + task::block_on(future::poll_fn(move |cx: &mut Context| { loop { - match framed_stdin.poll().expect("Error while polling stdin") { - Async::Ready(Some(line)) => swarm.floodsub.publish(&floodsub_topic, line.as_bytes()), - Async::Ready(None) => panic!("Stdin closed"), - Async::NotReady => break, - }; + match stdin.try_poll_next_unpin(cx)? { + Poll::Ready(Some(line)) => swarm.floodsub.publish(&floodsub_topic, line.as_bytes()), + Poll::Ready(None) => panic!("Stdin closed"), + Poll::Pending => break + } } - loop { - match swarm.poll().expect("Error while polling swarm") { - Async::Ready(Some(_)) => { - - }, - Async::Ready(None) | Async::NotReady => { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => println!("{:?}", event), + Poll::Ready(None) => return Poll::Ready(Ok(())), + Poll::Pending => { if !listening { - if let Some(a) = Swarm::listeners(&swarm).next() { - println!("Listening on {:?}", a); + for addr in Swarm::listeners(&swarm) { + println!("Listening on {:?}", addr); listening = true; } } @@ -171,7 +170,6 @@ fn main() { } } } - - Ok(Async::NotReady) - })); + Poll::Pending + })) } diff --git a/examples/distributed-key-value-store.rs b/examples/distributed-key-value-store.rs index d8f649d812a..84c16c1598e 100644 --- a/examples/distributed-key-value-store.rs +++ b/examples/distributed-key-value-store.rs @@ -29,19 +29,22 @@ //! //! 4. Close with Ctrl-c. +use async_std::{io, task}; use futures::prelude::*; use libp2p::kad::record::store::MemoryStore; use libp2p::kad::{record::Key, Kademlia, KademliaEvent, PutRecordOk, Quorum, Record}; use libp2p::{ - build_development_transport, identity, + NetworkBehaviour, + PeerId, + Swarm, + build_development_transport, + identity, mdns::{Mdns, MdnsEvent}, - swarm::NetworkBehaviourEventProcess, - tokio_codec::{FramedRead, LinesCodec}, - tokio_io::{AsyncRead, AsyncWrite}, - NetworkBehaviour, PeerId, Swarm, + swarm::NetworkBehaviourEventProcess }; +use std::{error::Error, task::{Context, Poll}}; -fn main() { +fn main() -> Result<(), Box> { env_logger::init(); // Create a random key for ourselves. @@ -49,17 +52,18 @@ fn main() { let local_peer_id = PeerId::from(local_key.public()); // Set up a an encrypted DNS-enabled TCP Transport over the Mplex protocol. - let transport = build_development_transport(local_key); + let transport = build_development_transport(local_key)?; // We create a custom network behaviour that combines Kademlia and mDNS. #[derive(NetworkBehaviour)] struct MyBehaviour { kademlia: Kademlia, - mdns: Mdns, + mdns: Mdns } - impl NetworkBehaviourEventProcess - for MyBehaviour + impl NetworkBehaviourEventProcess for MyBehaviour + where + T: AsyncRead + AsyncWrite { // Called when `mdns` produces an event. fn inject_event(&mut self, event: MdnsEvent) { @@ -71,8 +75,9 @@ fn main() { } } - impl NetworkBehaviourEventProcess - for MyBehaviour + impl NetworkBehaviourEventProcess for MyBehaviour + where + T: AsyncRead + AsyncWrite { // Called when `kademlia` produces an event. fn inject_event(&mut self, message: KademliaEvent) { @@ -108,58 +113,50 @@ fn main() { // Create a Kademlia behaviour. let store = MemoryStore::new(local_peer_id.clone()); let kademlia = Kademlia::new(local_peer_id.clone(), store); - - let behaviour = MyBehaviour { - kademlia, - mdns: Mdns::new().expect("Failed to create mDNS service"), - }; - + let mdns = task::block_on(Mdns::new())?; + let behaviour = MyBehaviour { kademlia, mdns }; Swarm::new(transport, behaviour, local_peer_id) }; - // Read full lines from stdin. - let stdin = tokio_stdin_stdout::stdin(0); - let mut framed_stdin = FramedRead::new(stdin, LinesCodec::new()); + // Read full lines from stdin + let mut stdin = io::BufReader::new(io::stdin()).lines(); // Listen on all interfaces and whatever port the OS assigns. - Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); + Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse()?)?; // Kick it off. let mut listening = false; - tokio::run(futures::future::poll_fn(move || { + task::block_on(future::poll_fn(move |cx: &mut Context| { loop { - match framed_stdin.poll().expect("Error while polling stdin") { - Async::Ready(Some(line)) => { - handle_input_line(&mut swarm.kademlia, line); - } - Async::Ready(None) => panic!("Stdin closed"), - Async::NotReady => break, - }; + match stdin.try_poll_next_unpin(cx)? { + Poll::Ready(Some(line)) => handle_input_line(&mut swarm.kademlia, line), + Poll::Ready(None) => panic!("Stdin closed"), + Poll::Pending => break + } } - loop { - match swarm.poll().expect("Error while polling swarm") { - Async::Ready(Some(_)) => {} - Async::Ready(None) | Async::NotReady => { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => println!("{:?}", event), + Poll::Ready(None) => return Poll::Ready(Ok(())), + Poll::Pending => { if !listening { if let Some(a) = Swarm::listeners(&swarm).next() { println!("Listening on {:?}", a); listening = true; } } - break; + break } } } - - Ok(Async::NotReady) - })); + Poll::Pending + })) } -fn handle_input_line( - kademlia: &mut Kademlia, - line: String, -) { +fn handle_input_line(kademlia: &mut Kademlia, line: String) +where + T: AsyncRead + AsyncWrite +{ let mut args = line.split(" "); match args.next() { diff --git a/examples/ipfs-kad.rs b/examples/ipfs-kad.rs index 7ee1f88eeea..bb1738e5815 100644 --- a/examples/ipfs-kad.rs +++ b/examples/ipfs-kad.rs @@ -23,7 +23,7 @@ //! You can pass as parameter a base58 peer ID to search for. If you don't pass any parameter, a //! peer ID will be generated randomly. -use futures::prelude::*; +use async_std::task; use libp2p::{ Swarm, PeerId, @@ -32,10 +32,9 @@ use libp2p::{ }; use libp2p::kad::{Kademlia, KademliaConfig, KademliaEvent, GetClosestPeersError}; use libp2p::kad::record::store::MemoryStore; -use std::env; -use std::time::Duration; +use std::{env, error::Error, time::Duration}; -fn main() { +fn main() -> Result<(), Box> { env_logger::init(); // Create a random key for ourselves. @@ -43,7 +42,7 @@ fn main() { let local_peer_id = PeerId::from(local_key.public()); // Set up a an encrypted DNS-enabled TCP Transport over the Mplex protocol - let transport = build_development_transport(local_key); + let transport = build_development_transport(local_key)?; // Create a swarm to manage peers and events. let mut swarm = { @@ -60,7 +59,7 @@ fn main() { behaviour.add_address(&"QmcZf59bWwK5XFi76CZX8cbJ4BhTzzA3gU1ZjYZcYW3dwt".parse().unwrap(), "/dnsaddr/bootstrap.libp2p.io".parse().unwrap());*/ // The only address that currently works. - behaviour.add_address(&"QmaCpDMGvV2BGHeYERUEnRQAwe3N8SzbUtfsmvsqQLuvuJ".parse().unwrap(), "/ip4/104.131.131.82/tcp/4001".parse().unwrap()); + behaviour.add_address(&"QmaCpDMGvV2BGHeYERUEnRQAwe3N8SzbUtfsmvsqQLuvuJ".parse()?, "/ip4/104.131.131.82/tcp/4001".parse()?); // The following addresses always fail signature verification, possibly due to // RSA keys with < 2048 bits. @@ -80,7 +79,7 @@ fn main() { // Order Kademlia to search for a peer. let to_search: PeerId = if let Some(peer_id) = env::args().nth(1) { - peer_id.parse().expect("Failed to parse peer ID to find") + peer_id.parse()? } else { identity::Keypair::generate_ed25519().public().into() }; @@ -89,38 +88,33 @@ fn main() { swarm.get_closest_peers(to_search); // Kick it off! - tokio::run(futures::future::poll_fn(move || { + task::block_on(async move { loop { - match swarm.poll().expect("Error while polling swarm") { - Async::Ready(Some(KademliaEvent::GetClosestPeersResult(res))) => { - match res { - Ok(ok) => { - if !ok.peers.is_empty() { - println!("Query finished with closest peers: {:#?}", ok.peers); - return Ok(Async::Ready(())); - } else { - // The example is considered failed as there - // should always be at least 1 reachable peer. - panic!("Query finished with no closest peers."); - } + let event = swarm.next().await; + if let KademliaEvent::GetClosestPeersResult(result) = event { + match result { + Ok(ok) => + if !ok.peers.is_empty() { + println!("Query finished with closest peers: {:#?}", ok.peers) + } else { + // The example is considered failed as there + // should always be at least 1 reachable peer. + println!("Query finished with no closest peers.") } - Err(GetClosestPeersError::Timeout { peers, .. }) => { - if !peers.is_empty() { - println!("Query timed out with closest peers: {:#?}", peers); - return Ok(Async::Ready(())); - } else { - // The example is considered failed as there - // should always be at least 1 reachable peer. - panic!("Query timed out with no closest peers."); - } + Err(GetClosestPeersError::Timeout { peers, .. }) => + if !peers.is_empty() { + println!("Query timed out with closest peers: {:#?}", peers) + } else { + // The example is considered failed as there + // should always be at least 1 reachable peer. + println!("Query timed out with no closest peers."); } - } - }, - Async::Ready(Some(_)) => {}, - Async::Ready(None) | Async::NotReady => break, + }; + + break; } } - Ok(Async::NotReady) - })); + Ok(()) + }) } diff --git a/examples/mdns-passive-discovery.rs b/examples/mdns-passive-discovery.rs index 32c760e9acd..a8f4323a4b1 100644 --- a/examples/mdns-passive-discovery.rs +++ b/examples/mdns-passive-discovery.rs @@ -18,26 +18,17 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use futures::prelude::*; +use async_std::task; use libp2p::mdns::service::{MdnsPacket, MdnsService}; -use std::io; +use std::error::Error; -fn main() { - // This example provides passive discovery of the libp2p nodes on the network that send - // mDNS queries and answers. - - // We start by creating the service. - let mut service = MdnsService::new().expect("Error while creating mDNS service"); - - // Create a never-ending `Future` that polls the service for events. - let future = futures::future::poll_fn(move || -> Poll<(), io::Error> { +fn main() -> Result<(), Box> { + // This example provides passive discovery of the libp2p nodes on the + // network that send mDNS queries and answers. + task::block_on(async move { + let mut service = MdnsService::new().await?; loop { - // Grab the next available packet from the service. - let packet = match service.poll() { - Async::Ready(packet) => packet, - Async::NotReady => return Ok(Async::NotReady), - }; - + let (srv, packet) = service.next().await; match packet { MdnsPacket::Query(query) => { // We detected a libp2p mDNS query on the network. In a real application, you @@ -63,9 +54,7 @@ fn main() { println!("Detected service query from {:?}", query.remote_addr()); } } + service = srv } - }); - - // Blocks the thread until the future runs to completion (which will never happen). - tokio::run(future.map_err(|err| panic!("{:?}", err))); + }) } diff --git a/examples/ping.rs b/examples/ping.rs index a8a6981b6a7..aa9e1f8d99a 100644 --- a/examples/ping.rs +++ b/examples/ping.rs @@ -38,11 +38,12 @@ //! The two nodes establish a connection, negotiate the ping protocol //! and begin pinging each other. -use futures::{prelude::*, future}; -use libp2p::{ identity, PeerId, ping::{Ping, PingConfig}, Swarm }; -use std::env; +use async_std::task; +use futures::{future, prelude::*}; +use libp2p::{identity, PeerId, ping::{Ping, PingConfig}, Swarm}; +use std::{error::Error, task::{Context, Poll}}; -fn main() { +fn main() -> Result<(), Box> { env_logger::init(); // Create a random PeerId. @@ -51,7 +52,7 @@ fn main() { println!("Local peer id: {:?}", peer_id); // Create a transport. - let transport = libp2p::build_development_transport(id_keys); + let transport = libp2p::build_development_transport(id_keys)?; // Create a ping network behaviour. // @@ -66,38 +67,33 @@ fn main() { // Dial the peer identified by the multi-address given as the second // command-line argument, if any. - if let Some(addr) = env::args().nth(1) { - let remote_addr = addr.clone(); - match addr.parse() { - Ok(remote) => { - match Swarm::dial_addr(&mut swarm, remote) { - Ok(()) => println!("Dialed {:?}", remote_addr), - Err(e) => println!("Dialing {:?} failed with: {:?}", remote_addr, e) - } - }, - Err(err) => println!("Failed to parse address to dial: {:?}", err), - } + if let Some(addr) = std::env::args().nth(1) { + let remote = addr.parse()?; + Swarm::dial_addr(&mut swarm, remote)?; + println!("Dialed {}", addr) } // Tell the swarm to listen on all interfaces and a random, OS-assigned port. - Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); + Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse()?)?; - // Use tokio to drive the `Swarm`. let mut listening = false; - tokio::run(future::poll_fn(move || -> Result<_, ()> { + task::block_on(future::poll_fn(move |cx: &mut Context| { loop { - match swarm.poll().expect("Error while polling swarm") { - Async::Ready(Some(e)) => println!("{:?}", e), - Async::Ready(None) | Async::NotReady => { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => println!("{:?}", event), + Poll::Ready(None) => return Poll::Ready(()), + Poll::Pending => { if !listening { - if let Some(a) = Swarm::listeners(&swarm).next() { - println!("Listening on {:?}", a); + for addr in Swarm::listeners(&swarm) { + println!("Listening on {}", addr); listening = true; } } - return Ok(Async::NotReady) + return Poll::Pending } } } })); + + Ok(()) } diff --git a/misc/core-derive/Cargo.toml b/misc/core-derive/Cargo.toml index 6b447a6ca3a..57ac1783245 100644 --- a/misc/core-derive/Cargo.toml +++ b/misc/core-derive/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-core-derive" edition = "2018" description = "Procedural macros of libp2p-core" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -13,8 +13,8 @@ categories = ["network-programming", "asynchronous"] proc-macro = true [dependencies] -syn = { version = "1.0", default-features = false, features = ["clone-impls", "derive", "parsing", "printing", "proc-macro"] } +syn = { version = "1.0.8", default-features = false, features = ["clone-impls", "derive", "parsing", "printing", "proc-macro"] } quote = "1.0" [dev-dependencies] -libp2p = { version = "0.13.0", path = "../.." } +libp2p = { version = "0.14.0-alpha.1", path = "../.." } diff --git a/misc/core-derive/src/lib.rs b/misc/core-derive/src/lib.rs index 383a6b9bb1f..8e9fd667017 100644 --- a/misc/core-derive/src/lib.rs +++ b/misc/core-derive/src/lib.rs @@ -26,7 +26,8 @@ use quote::quote; use proc_macro::TokenStream; use syn::{parse_macro_input, DeriveInput, Data, DataStruct, Ident}; -/// The interface that satisfies Rust. +/// Generates a delegating `NetworkBehaviour` implementation for the struct this is used for. See +/// the trait documentation for better description. #[proc_macro_derive(NetworkBehaviour, attributes(behaviour))] pub fn hello_macro_derive(input: TokenStream) -> TokenStream { let ast = parse_macro_input!(input as DeriveInput); @@ -90,14 +91,15 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { quote!{Self: #net_behv_event_proc<<#ty as #trait_to_impl>::OutEvent>}, quote!{<<#ty as #trait_to_impl>::ProtocolsHandler as #into_protocols_handler>::Handler: #protocols_handler}, // Note: this bound is required because of https://github.com/rust-lang/rust/issues/55697 - quote!{<<<#ty as #trait_to_impl>::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::InboundProtocol: ::libp2p::core::InboundUpgrade<#substream_generic>}, - quote!{<<<#ty as #trait_to_impl>::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::OutboundProtocol: ::libp2p::core::OutboundUpgrade<#substream_generic>}, + quote!{<<<#ty as #trait_to_impl>::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::InboundProtocol: ::libp2p::core::InboundUpgrade<::libp2p::core::Negotiated<#substream_generic>>}, + quote!{<<<#ty as #trait_to_impl>::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::OutboundProtocol: ::libp2p::core::OutboundUpgrade<::libp2p::core::Negotiated<#substream_generic>>}, ] }) .collect::>(); - additional.push(quote!{#substream_generic: ::libp2p::tokio_io::AsyncRead}); - additional.push(quote!{#substream_generic: ::libp2p::tokio_io::AsyncWrite}); + additional.push(quote!{#substream_generic: ::libp2p::futures::io::AsyncRead}); + additional.push(quote!{#substream_generic: ::libp2p::futures::io::AsyncWrite}); + additional.push(quote!{#substream_generic: Unpin}); if let Some(where_clause) = where_clause { if where_clause.predicates.trailing_punct() { @@ -381,14 +383,14 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // If we find a `#[behaviour(poll_method = "poll")]` attribute on the struct, we call // `self.poll()` at the end of the polling. let poll_method = { - let mut poll_method = quote!{Async::NotReady}; + let mut poll_method = quote!{std::task::Poll::Pending}; for meta_items in ast.attrs.iter().filter_map(get_meta_items) { for meta_item in meta_items { match meta_item { syn::NestedMeta::Meta(syn::Meta::NameValue(ref m)) if m.path.is_ident("poll_method") => { if let syn::Lit::Str(ref s) = m.lit { let ident: Ident = syn::parse_str(&s.value()).unwrap(); - poll_method = quote!{#name::#ident(self)}; + poll_method = quote!{#name::#ident(self, cx)}; } } _ => () @@ -418,26 +420,26 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { Some(quote!{ loop { - match #field_name.poll(poll_params) { - Async::Ready(#network_behaviour_action::GenerateEvent(event)) => { + match #field_name.poll(cx, poll_params) { + std::task::Poll::Ready(#network_behaviour_action::GenerateEvent(event)) => { #net_behv_event_proc::inject_event(self, event) } - Async::Ready(#network_behaviour_action::DialAddress { address }) => { - return Async::Ready(#network_behaviour_action::DialAddress { address }); + std::task::Poll::Ready(#network_behaviour_action::DialAddress { address }) => { + return std::task::Poll::Ready(#network_behaviour_action::DialAddress { address }); } - Async::Ready(#network_behaviour_action::DialPeer { peer_id }) => { - return Async::Ready(#network_behaviour_action::DialPeer { peer_id }); + std::task::Poll::Ready(#network_behaviour_action::DialPeer { peer_id }) => { + return std::task::Poll::Ready(#network_behaviour_action::DialPeer { peer_id }); } - Async::Ready(#network_behaviour_action::SendEvent { peer_id, event }) => { - return Async::Ready(#network_behaviour_action::SendEvent { + std::task::Poll::Ready(#network_behaviour_action::SendEvent { peer_id, event }) => { + return std::task::Poll::Ready(#network_behaviour_action::SendEvent { peer_id, event: #wrapped_event, }); } - Async::Ready(#network_behaviour_action::ReportObservedAddr { address }) => { - return Async::Ready(#network_behaviour_action::ReportObservedAddr { address }); + std::task::Poll::Ready(#network_behaviour_action::ReportObservedAddr { address }) => { + return std::task::Poll::Ready(#network_behaviour_action::ReportObservedAddr { address }); } - Async::NotReady => break, + std::task::Poll::Pending => break, } } }) @@ -512,10 +514,10 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { } } - fn poll(&mut self, poll_params: &mut impl #poll_parameters) -> ::libp2p::futures::Async<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> { + fn poll(&mut self, cx: &mut std::task::Context, poll_params: &mut impl #poll_parameters) -> std::task::Poll<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> { use libp2p::futures::prelude::*; #(#poll_stmts)* - let f: ::libp2p::futures::Async<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> = #poll_method; + let f: std::task::Poll<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> = #poll_method; f } } @@ -525,10 +527,12 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { } fn get_meta_items(attr: &syn::Attribute) -> Option> { - if attr.path.is_ident("behaviour") { + if attr.path.segments.len() == 1 && attr.path.segments[0].ident == "behaviour" { match attr.parse_meta() { Ok(syn::Meta::List(ref meta)) => Some(meta.nested.iter().cloned().collect()), - _ => { + Ok(_) => None, + Err(e) => { + eprintln!("error parsing attribute metadata: {}", e); None } } diff --git a/misc/core-derive/tests/test.rs b/misc/core-derive/tests/test.rs index 7213a1cf22e..8fae16ca70b 100644 --- a/misc/core-derive/tests/test.rs +++ b/misc/core-derive/tests/test.rs @@ -46,7 +46,7 @@ fn one_field() { } #[allow(dead_code)] - fn foo() { + fn foo() { require_net_behaviour::>(); } } @@ -71,7 +71,7 @@ fn two_fields() { } #[allow(dead_code)] - fn foo() { + fn foo() { require_net_behaviour::>(); } } @@ -104,7 +104,7 @@ fn three_fields() { } #[allow(dead_code)] - fn foo() { + fn foo() { require_net_behaviour::>(); } } @@ -130,11 +130,11 @@ fn custom_polling() { } impl Foo { - fn foo(&mut self) -> libp2p::futures::Async> { libp2p::futures::Async::NotReady } + fn foo(&mut self, _: &mut std::task::Context) -> std::task::Poll> { std::task::Poll::Pending } } #[allow(dead_code)] - fn foo() { + fn foo() { require_net_behaviour::>(); } } @@ -160,7 +160,7 @@ fn custom_event_no_polling() { } #[allow(dead_code)] - fn foo() { + fn foo() { require_net_behaviour::>(); } } @@ -186,11 +186,11 @@ fn custom_event_and_polling() { } impl Foo { - fn foo(&mut self) -> libp2p::futures::Async> { libp2p::futures::Async::NotReady } + fn foo(&mut self, _: &mut std::task::Context) -> std::task::Poll> { std::task::Poll::Pending } } #[allow(dead_code)] - fn foo() { + fn foo() { require_net_behaviour::>(); } } diff --git a/misc/mdns/Cargo.toml b/misc/mdns/Cargo.toml index d1f0dce7b84..2e6a32c77bb 100644 --- a/misc/mdns/Cargo.toml +++ b/misc/mdns/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "libp2p-mdns" edition = "2018" -version = "0.13.1" +version = "0.14.0-alpha.1" description = "Implementation of the libp2p mDNS discovery method" authors = ["Parity Technologies "] license = "MIT" @@ -10,21 +10,20 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] +async-std = "1.0" data-encoding = "2.0" dns-parser = "0.8" -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -libp2p-swarm = { version = "0.3.0", path = "../../swarm" } +either = "1.5.3" +futures = "0.3.1" +lazy_static = "1.2" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } log = "0.4" -multiaddr = { package = "parity-multiaddr", version = "0.6.0", path = "../multiaddr" } net2 = "0.2" -rand = "0.6" -smallvec = "0.6" -tokio-io = "0.1" -tokio-reactor = "0.1" -wasm-timer = "0.1" -tokio-udp = "0.1" +rand = "0.7" +smallvec = "1.0" void = "1.0" +wasm-timer = "0.2.4" [dev-dependencies] -tokio = "0.1" +get_if_addrs = "0.5.3" diff --git a/misc/mdns/src/behaviour.rs b/misc/mdns/src/behaviour.rs index 7d9332114a7..61da92b90e3 100644 --- a/misc/mdns/src/behaviour.rs +++ b/misc/mdns/src/behaviour.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::service::{MdnsService, MdnsPacket}; +use crate::service::{MdnsService, MdnsPacket, build_query_response, build_service_discovery_response}; use futures::prelude::*; use libp2p_core::{address_translation, ConnectedPoint, Multiaddr, PeerId, multiaddr::Protocol}; use libp2p_swarm::{ @@ -30,15 +30,16 @@ use libp2p_swarm::{ }; use log::warn; use smallvec::SmallVec; -use std::{cmp, fmt, io, iter, marker::PhantomData, time::Duration}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{cmp, fmt, io, iter, marker::PhantomData, mem, pin::Pin, time::Duration, task::Context, task::Poll}; use wasm_timer::{Delay, Instant}; +const MDNS_RESPONSE_TTL: std::time::Duration = Duration::from_secs(5 * 60); + /// A `NetworkBehaviour` for mDNS. Automatically discovers peers on the local network and adds /// them to the topology. pub struct Mdns { /// The inner service. - service: MdnsService, + service: MaybeBusyMdnsService, /// List of nodes that we have discovered, the address, and when their TTL expires. /// @@ -46,7 +47,7 @@ pub struct Mdns { /// can appear multiple times. discovered_nodes: SmallVec<[(PeerId, Multiaddr, Instant); 8]>, - /// Future that fires when the TTL at least one node in `discovered_nodes` expires. + /// Future that fires when the TTL of at least one node in `discovered_nodes` expires. /// /// `None` if `discovered_nodes` is empty. closest_expiration: Option, @@ -55,11 +56,41 @@ pub struct Mdns { marker: PhantomData, } +/// `MdnsService::next` takes ownership of `self`, returning a future that resolves with both itself +/// and a `MdnsPacket` (similar to the old Tokio socket send style). The two states are thus `Free` +/// with an `MdnsService` or `Busy` with a future returning the original `MdnsService` and an +/// `MdnsPacket`. +enum MaybeBusyMdnsService { + Free(MdnsService), + Busy(Pin + Send>>), + Poisoned, +} + +impl fmt::Debug for MaybeBusyMdnsService { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + MaybeBusyMdnsService::Free(service) => { + fmt.debug_struct("MaybeBusyMdnsService::Free") + .field("service", service) + .finish() + }, + MaybeBusyMdnsService::Busy(_) => { + fmt.debug_struct("MaybeBusyMdnsService::Busy") + .finish() + } + MaybeBusyMdnsService::Poisoned => { + fmt.debug_struct("MaybeBusyMdnsService::Poisoned") + .finish() + } + } + } +} + impl Mdns { /// Builds a new `Mdns` behaviour. - pub fn new() -> io::Result> { + pub async fn new() -> io::Result> { Ok(Mdns { - service: MdnsService::new()?, + service: MaybeBusyMdnsService::Free(MdnsService::new().await?), discovered_nodes: SmallVec::new(), closest_expiration: None, marker: PhantomData, @@ -81,7 +112,7 @@ pub enum MdnsEvent { /// The given combinations of `PeerId` and `Multiaddr` have expired. /// /// Each discovered record has a time-to-live. When this TTL expires and the address hasn't - /// been refreshed, we remove it from the list emit it as an `Expired` event. + /// been refreshed, we remove it from the list and emit it as an `Expired` event. Expired(ExpiredAddrsIter), } @@ -145,7 +176,7 @@ impl fmt::Debug for ExpiredAddrsIter { impl NetworkBehaviour for Mdns where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { type ProtocolsHandler = DummyProtocolsHandler; type OutEvent = MdnsEvent; @@ -177,8 +208,9 @@ where fn poll( &mut self, + cx: &mut Context, params: &mut impl PollParameters, - ) -> Async< + ) -> Poll< NetworkBehaviourAction< ::InEvent, Self::OutEvent, @@ -186,8 +218,8 @@ where > { // Remove expired peers. if let Some(ref mut closest_expiration) = self.closest_expiration { - match closest_expiration.poll() { - Ok(Async::Ready(())) => { + match Future::poll(Pin::new(closest_expiration), cx) { + Poll::Ready(Ok(())) => { let now = Instant::now(); let mut expired = SmallVec::<[(PeerId, Multiaddr); 4]>::new(); while let Some(pos) = self.discovered_nodes.iter().position(|(_, _, exp)| *exp < now) { @@ -200,28 +232,50 @@ where inner: expired.into_iter(), }); - return Async::Ready(NetworkBehaviourAction::GenerateEvent(event)); + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)); } }, - Ok(Async::NotReady) => (), - Err(err) => warn!("tokio timer has errored: {:?}", err), + Poll::Pending => (), + Poll::Ready(Err(err)) => warn!("timer has errored: {:?}", err), } } // Polling the mDNS service, and obtain the list of nodes discovered this round. let discovered = loop { - let event = match self.service.poll() { - Async::Ready(ev) => ev, - Async::NotReady => return Async::NotReady, + let service = mem::replace(&mut self.service, MaybeBusyMdnsService::Poisoned); + + let packet = match service { + MaybeBusyMdnsService::Free(service) => { + self.service = MaybeBusyMdnsService::Busy(Box::pin(service.next())); + continue; + }, + MaybeBusyMdnsService::Busy(mut fut) => { + match fut.as_mut().poll(cx) { + Poll::Ready((service, packet)) => { + self.service = MaybeBusyMdnsService::Free(service); + packet + }, + Poll::Pending => { + self.service = MaybeBusyMdnsService::Busy(fut); + return Poll::Pending; + } + } + }, + MaybeBusyMdnsService::Poisoned => panic!("Mdns poisoned"), }; - match event { + match packet { MdnsPacket::Query(query) => { - let _ = query.respond( - params.local_peer_id().clone(), - params.listened_addresses(), - Duration::from_secs(5 * 60) - ); + // MaybeBusyMdnsService should always be Free. + if let MaybeBusyMdnsService::Free(ref mut service) = self.service { + let resp = build_query_response( + query.query_id(), + params.local_peer_id().clone(), + params.listened_addresses().into_iter(), + MDNS_RESPONSE_TTL, + ); + service.enqueue_response(resp.unwrap()); + } else { debug_assert!(false); } }, MdnsPacket::Response(response) => { // We replace the IP address with the address we observe the @@ -240,12 +294,12 @@ where let new_expiration = Instant::now() + peer.ttl(); - let mut addrs = Vec::new(); + let mut addrs: Vec = Vec::new(); for addr in peer.addresses() { if let Some(new_addr) = address_translation(&addr, &observed) { - addrs.push(new_addr) + addrs.push(new_addr.clone()) } - addrs.push(addr) + addrs.push(addr.clone()) } for addr in addrs { @@ -264,18 +318,27 @@ where break discovered; }, MdnsPacket::ServiceDiscovery(disc) => { - disc.respond(Duration::from_secs(5 * 60)); + // MaybeBusyMdnsService should always be Free. + if let MaybeBusyMdnsService::Free(ref mut service) = self.service { + let resp = build_service_discovery_response( + disc.query_id(), + MDNS_RESPONSE_TTL, + ); + service.enqueue_response(resp); + } else { debug_assert!(false); } }, } }; - // As the final step, we need to refresh `closest_expiration`. + // Getting this far implies that we discovered new nodes. As the final step, we need to + // refresh `closest_expiration`. self.closest_expiration = self.discovered_nodes.iter() .fold(None, |exp, &(_, _, elem_exp)| { Some(exp.map(|exp| cmp::min(exp, elem_exp)).unwrap_or(elem_exp)) }) - .map(Delay::new); - Async::Ready(NetworkBehaviourAction::GenerateEvent(MdnsEvent::Discovered(DiscoveredAddrsIter { + .map(Delay::new_at); + + Poll::Ready(NetworkBehaviourAction::GenerateEvent(MdnsEvent::Discovered(DiscoveredAddrsIter { inner: discovered.into_iter(), }))) } @@ -288,4 +351,3 @@ impl fmt::Debug for Mdns { .finish() } } - diff --git a/misc/mdns/src/service.rs b/misc/mdns/src/service.rs index c2557a4da37..8620c93e5fd 100644 --- a/misc/mdns/src/service.rs +++ b/misc/mdns/src/service.rs @@ -19,16 +19,23 @@ // DEALINGS IN THE SOFTWARE. use crate::{SERVICE_NAME, META_QUERY_SERVICE, dns}; +use async_std::net::UdpSocket; use dns_parser::{Packet, RData}; -use futures::{prelude::*, task}; -use libp2p_core::{Multiaddr, PeerId}; -use multiaddr::Protocol; -use std::{fmt, io, net::Ipv4Addr, net::SocketAddr, str, time::Duration}; -use tokio_reactor::Handle; -use wasm_timer::{Instant, Interval}; -use tokio_udp::UdpSocket; - -pub use dns::MdnsResponseError; +use either::Either::{Left, Right}; +use futures::{future, prelude::*}; +use libp2p_core::{multiaddr::{Multiaddr, Protocol}, PeerId}; +use std::{fmt, io, net::Ipv4Addr, net::SocketAddr, str, time::{Duration, Instant}}; +use wasm_timer::Interval; +use lazy_static::lazy_static; + +pub use dns::{MdnsResponseError, build_query_response, build_service_discovery_response}; + +lazy_static! { + static ref IPV4_MDNS_MULTICAST_ADDRESS: SocketAddr = SocketAddr::from(( + Ipv4Addr::new(224, 0, 0, 251), + 5353, + )); +} /// A running service that discovers libp2p peers and responds to other libp2p peers' queries on /// the local network. @@ -53,43 +60,47 @@ pub use dns::MdnsResponseError; /// /// ```rust /// # use futures::prelude::*; -/// # use libp2p_core::{identity, PeerId}; -/// # use libp2p_mdns::service::{MdnsService, MdnsPacket}; -/// # use std::{io, time::Duration}; +/// # use futures::executor::block_on; +/// # use libp2p_core::{identity, Multiaddr, PeerId}; +/// # use libp2p_mdns::service::{MdnsService, MdnsPacket, build_query_response, build_service_discovery_response}; +/// # use std::{io, time::Duration, task::Poll}; /// # fn main() { /// # let my_peer_id = PeerId::from(identity::Keypair::generate_ed25519().public()); -/// # let my_listened_addrs = Vec::new(); -/// let mut service = MdnsService::new().expect("Error while creating mDNS service"); -/// let _future_to_poll = futures::stream::poll_fn(move || -> Poll, io::Error> { -/// loop { -/// let packet = match service.poll() { -/// Async::Ready(packet) => packet, -/// Async::NotReady => return Ok(Async::NotReady), -/// }; +/// # let my_listened_addrs: Vec = vec![]; +/// # block_on(async { +/// let mut service = MdnsService::new().await.expect("Error while creating mDNS service"); +/// let _future_to_poll = async { +/// let (mut service, packet) = service.next().await; /// -/// match packet { -/// MdnsPacket::Query(query) => { -/// println!("Query from {:?}", query.remote_addr()); -/// query.respond( -/// my_peer_id.clone(), -/// my_listened_addrs.clone(), -/// Duration::from_secs(120), -/// ); -/// } -/// MdnsPacket::Response(response) => { -/// for peer in response.discovered_peers() { -/// println!("Discovered peer {:?}", peer.id()); -/// for addr in peer.addresses() { -/// println!("Address = {:?}", addr); -/// } +/// match packet { +/// MdnsPacket::Query(query) => { +/// println!("Query from {:?}", query.remote_addr()); +/// let resp = build_query_response( +/// query.query_id(), +/// my_peer_id.clone(), +/// vec![].into_iter(), +/// Duration::from_secs(120), +/// ).unwrap(); +/// service.enqueue_response(resp); +/// } +/// MdnsPacket::Response(response) => { +/// for peer in response.discovered_peers() { +/// println!("Discovered peer {:?}", peer.id()); +/// for addr in peer.addresses() { +/// println!("Address = {:?}", addr); /// } /// } -/// MdnsPacket::ServiceDiscovery(query) => { -/// query.respond(std::time::Duration::from_secs(120)); -/// } +/// } +/// MdnsPacket::ServiceDiscovery(disc) => { +/// let resp = build_service_discovery_response( +/// disc.query_id(), +/// Duration::from_secs(120), +/// ); +/// service.enqueue_response(resp); /// } /// } -/// }).for_each(|_| Ok(())); +/// }; +/// # }) /// # } pub struct MdnsService { /// Main socket for listening. @@ -113,18 +124,18 @@ pub struct MdnsService { impl MdnsService { /// Starts a new mDNS service. #[inline] - pub fn new() -> io::Result { - Self::new_inner(false) + pub async fn new() -> io::Result { + Self::new_inner(false).await } /// Same as `new`, but we don't send automatically send queries on the network. #[inline] - pub fn silent() -> io::Result { - Self::new_inner(true) + pub async fn silent() -> io::Result { + Self::new_inner(true).await } /// Starts a new mDNS service. - fn new_inner(silent: bool) -> io::Result { + async fn new_inner(silent: bool) -> io::Result { let socket = { #[cfg(unix)] fn platform_specific(s: &net2::UdpBuilder) -> io::Result<()> { @@ -139,16 +150,16 @@ impl MdnsService { builder.bind(("0.0.0.0", 5353))? }; - let socket = UdpSocket::from_std(socket, &Handle::default())?; + let socket = UdpSocket::from(socket); socket.set_multicast_loop_v4(true)?; socket.set_multicast_ttl_v4(255)?; // TODO: correct interfaces? - socket.join_multicast_v4(&From::from([224, 0, 0, 251]), &Ipv4Addr::UNSPECIFIED)?; + socket.join_multicast_v4(From::from([224, 0, 0, 251]), Ipv4Addr::UNSPECIFIED)?; Ok(MdnsService { socket, - query_socket: UdpSocket::bind(&From::from(([0, 0, 0, 0], 0)))?, - query_interval: Interval::new(Instant::now(), Duration::from_secs(20)), + query_socket: UdpSocket::bind((Ipv4Addr::from([0u8, 0, 0, 0]), 0u16)).await?, + query_interval: Interval::new_at(Instant::now(), Duration::from_secs(20)), silent, recv_buffer: [0; 2048], send_buffers: Vec::new(), @@ -156,132 +167,102 @@ impl MdnsService { }) } - /// Polls the service for packets. - pub fn poll(&mut self) -> Async> { - // Send a query every time `query_interval` fires. - // Note that we don't use a loop here—it is pretty unlikely that we need it, and there is - // no point in sending multiple requests in a row. - match self.query_interval.poll() { - Ok(Async::Ready(_)) => { - if !self.silent { - let query = dns::build_query(); - self.query_send_buffers.push(query.to_vec()); - } - } - Ok(Async::NotReady) => (), - _ => unreachable!("A wasm_timer::Interval never errors"), // TODO: is that true? - }; + pub fn enqueue_response(&mut self, rsp: Vec) { + self.send_buffers.push(rsp); + } - // Flush the send buffer of the main socket. - while !self.send_buffers.is_empty() { - let to_send = self.send_buffers.remove(0); - match self - .socket - .poll_send_to(&to_send, &From::from(([224, 0, 0, 251], 5353))) - { - Ok(Async::Ready(bytes_written)) => { - debug_assert_eq!(bytes_written, to_send.len()); - } - Ok(Async::NotReady) => { - self.send_buffers.insert(0, to_send); - break; - } - Err(_) => { - // Errors are non-fatal because they can happen for example if we lose - // connection to the network. - self.send_buffers.clear(); - break; + /// Returns a future resolving to itself and the next received `MdnsPacket`. + // + // **Note**: Why does `next` take ownership of itself? + // + // `MdnsService::next` needs to be called from within `NetworkBehaviour` + // implementations. Given that traits cannot have async methods the + // respective `NetworkBehaviour` implementation needs to somehow keep the + // Future returned by `MdnsService::next` across classic `poll` + // invocations. The instance method `next` can either take a reference or + // ownership of itself: + // + // 1. Taking a reference - If `MdnsService::poll` takes a reference to + // `&self` the respective `NetworkBehaviour` implementation would need to + // keep both the Future as well as its `MdnsService` instance across poll + // invocations. Given that in this case the Future would have a reference + // to `MdnsService`, the `NetworkBehaviour` implementation struct would + // need to be self-referential which is not possible without unsafe code in + // Rust. + // + // 2. Taking ownership - Instead `MdnsService::next` takes ownership of + // self and returns it alongside an `MdnsPacket` once the actual future + // resolves, not forcing self-referential structures on the caller. + pub async fn next(mut self) -> (Self, MdnsPacket) { + loop { + // Flush the send buffer of the main socket. + while !self.send_buffers.is_empty() { + let to_send = self.send_buffers.remove(0); + + match self.socket.send_to(&to_send, *IPV4_MDNS_MULTICAST_ADDRESS).await { + Ok(bytes_written) => { + debug_assert_eq!(bytes_written, to_send.len()); + } + Err(_) => { + // Errors are non-fatal because they can happen for example if we lose + // connection to the network. + self.send_buffers.clear(); + break; + } } } - } - // Flush the query send buffer. - // This has to be after the push to `query_send_buffers`. - while !self.query_send_buffers.is_empty() { - let to_send = self.query_send_buffers.remove(0); - match self - .query_socket - .poll_send_to(&to_send, &From::from(([224, 0, 0, 251], 5353))) - { - Ok(Async::Ready(bytes_written)) => { - debug_assert_eq!(bytes_written, to_send.len()); - } - Ok(Async::NotReady) => { - self.query_send_buffers.insert(0, to_send); - break; - } - Err(_) => { - // Errors are non-fatal because they can happen for example if we lose - // connection to the network. - self.query_send_buffers.clear(); - break; - } - } - } + // Flush the query send buffer. + while !self.query_send_buffers.is_empty() { + let to_send = self.query_send_buffers.remove(0); - // Check for any incoming packet. - match self.socket.poll_recv_from(&mut self.recv_buffer) { - Ok(Async::Ready((len, from))) => { - match Packet::parse(&self.recv_buffer[..len]) { - Ok(packet) => { - if packet.header.query { - if packet - .questions - .iter() - .any(|q| q.qname.to_string().as_bytes() == SERVICE_NAME) - { - return Async::Ready(MdnsPacket::Query(MdnsQuery { - from, - query_id: packet.header.id, - send_buffers: &mut self.send_buffers, - })); - } else if packet - .questions - .iter() - .any(|q| q.qname.to_string().as_bytes() == META_QUERY_SERVICE) - { - // TODO: what if multiple questions, one with SERVICE_NAME and one with META_QUERY_SERVICE? - return Async::Ready(MdnsPacket::ServiceDiscovery( - MdnsServiceDiscovery { - from, - query_id: packet.header.id, - send_buffers: &mut self.send_buffers, - }, - )); - } else { - // Note that ideally we would use a loop instead. However as of the - // writing of this code non-lexical lifetimes haven't been merged - // yet, and I can't manage to write this code without having borrow - // issues. - task::current().notify(); - return Async::NotReady; - } - } else { - return Async::Ready(MdnsPacket::Response(MdnsResponse { - packet, - from, - })); - } + match self.query_socket.send_to(&to_send, *IPV4_MDNS_MULTICAST_ADDRESS).await { + Ok(bytes_written) => { + debug_assert_eq!(bytes_written, to_send.len()); } Err(_) => { - // Ignore errors while parsing the packet. We need to poll again for the - // next packet. - // Note that ideally we would use a loop instead. However as of the writing - // of this code non-lexical lifetimes haven't been merged yet, and I can't - // manage to write this code without having borrow issues. - task::current().notify(); - return Async::NotReady; + // Errors are non-fatal because they can happen for example if we lose + // connection to the network. + self.query_send_buffers.clear(); + break; } } } - Ok(Async::NotReady) => (), - Err(_) => { - // Error are non-fatal and can happen if we get disconnected from example. - // The query interval will wake up the task at some point so that we can try again. - } - }; - Async::NotReady + // Either (left) listen for incoming packets or (right) send query packets whenever the + // query interval fires. + let selected_output = match futures::future::select( + Box::pin(self.socket.recv_from(&mut self.recv_buffer)), + Box::pin(self.query_interval.next()), + ).await { + future::Either::Left((recved, _)) => Left(recved), + future::Either::Right(_) => Right(()), + }; + + match selected_output { + Left(left) => match left { + Ok((len, from)) => { + match MdnsPacket::new_from_bytes(&self.recv_buffer[..len], from) { + Some(packet) => return (self, packet), + None => {}, + } + }, + Err(_) => { + // Errors are non-fatal and can happen if we get disconnected from the network. + // The query interval will wake up the task at some point so that we can try again. + }, + }, + Right(_) => { + // Ensure underlying task is woken up on the next interval tick. + while let Some(_) = self.query_interval.next().now_or_never() {}; + + if !self.silent { + let query = dns::build_query(); + self.query_send_buffers.push(query.to_vec()); + } + } + }; + } } } @@ -295,58 +276,82 @@ impl fmt::Debug for MdnsService { /// A valid mDNS packet received by the service. #[derive(Debug)] -pub enum MdnsPacket<'a> { +pub enum MdnsPacket { /// A query made by a remote. - Query(MdnsQuery<'a>), + Query(MdnsQuery), /// A response sent by a remote in response to one of our queries. - Response(MdnsResponse<'a>), + Response(MdnsResponse), /// A request for service discovery. - ServiceDiscovery(MdnsServiceDiscovery<'a>), + ServiceDiscovery(MdnsServiceDiscovery), +} + +impl MdnsPacket { + fn new_from_bytes(buf: &[u8], from: SocketAddr) -> Option { + match Packet::parse(buf) { + Ok(packet) => { + if packet.header.query { + if packet + .questions + .iter() + .any(|q| q.qname.to_string().as_bytes() == SERVICE_NAME) + { + let query = MdnsPacket::Query(MdnsQuery { + from, + query_id: packet.header.id, + }); + return Some(query); + } else if packet + .questions + .iter() + .any(|q| q.qname.to_string().as_bytes() == META_QUERY_SERVICE) + { + // TODO: what if multiple questions, one with SERVICE_NAME and one with META_QUERY_SERVICE? + let discovery = MdnsPacket::ServiceDiscovery( + MdnsServiceDiscovery { + from, + query_id: packet.header.id, + }, + ); + return Some(discovery); + } else { + return None; + } + } else { + let resp = MdnsPacket::Response(MdnsResponse::new ( + packet, + from, + )); + return Some(resp); + } + } + Err(_) => { + return None; + } + } + } } /// A received mDNS query. -pub struct MdnsQuery<'a> { +pub struct MdnsQuery { /// Sender of the address. from: SocketAddr, /// Id of the received DNS query. We need to pass this ID back in the results. query_id: u16, - /// Queue of pending buffers. - send_buffers: &'a mut Vec>, } -impl<'a> MdnsQuery<'a> { - /// Respond to the query. - /// - /// Pass the ID of the local peer, and the list of addresses we're listening on. - /// - /// If there are more than 2^16-1 addresses, ignores the others. - /// - /// > **Note**: Keep in mind that we will also receive this response in an `MdnsResponse`. - #[inline] - pub fn respond( - self, - peer_id: PeerId, - addresses: TAddresses, - ttl: Duration, - ) -> Result<(), MdnsResponseError> - where - TAddresses: IntoIterator, - TAddresses::IntoIter: ExactSizeIterator, - { - let response = - dns::build_query_response(self.query_id, peer_id, addresses.into_iter(), ttl)?; - self.send_buffers.push(response); - Ok(()) - } - +impl MdnsQuery { /// Source address of the packet. - #[inline] pub fn remote_addr(&self) -> &SocketAddr { &self.from } + + /// Query id of the packet. + pub fn query_id(&self) -> u16 { + self.query_id + } } -impl<'a> fmt::Debug for MdnsQuery<'a> { +impl fmt::Debug for MdnsQuery { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("MdnsQuery") .field("from", self.remote_addr()) @@ -356,31 +361,26 @@ impl<'a> fmt::Debug for MdnsQuery<'a> { } /// A received mDNS service discovery query. -pub struct MdnsServiceDiscovery<'a> { +pub struct MdnsServiceDiscovery { /// Sender of the address. from: SocketAddr, /// Id of the received DNS query. We need to pass this ID back in the results. query_id: u16, - /// Queue of pending buffers. - send_buffers: &'a mut Vec>, } -impl<'a> MdnsServiceDiscovery<'a> { - /// Respond to the query. - #[inline] - pub fn respond(self, ttl: Duration) { - let response = dns::build_service_discovery_response(self.query_id, ttl); - self.send_buffers.push(response); - } - +impl MdnsServiceDiscovery { /// Source address of the packet. - #[inline] pub fn remote_addr(&self) -> &SocketAddr { &self.from } + + /// Query id of the packet. + pub fn query_id(&self) -> u16 { + self.query_id + } } -impl<'a> fmt::Debug for MdnsServiceDiscovery<'a> { +impl fmt::Debug for MdnsServiceDiscovery { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("MdnsServiceDiscovery") .field("from", self.remote_addr()) @@ -390,18 +390,15 @@ impl<'a> fmt::Debug for MdnsServiceDiscovery<'a> { } /// A received mDNS response. -pub struct MdnsResponse<'a> { - packet: Packet<'a>, +pub struct MdnsResponse { + peers: Vec, from: SocketAddr, } -impl<'a> MdnsResponse<'a> { - /// Returns the list of peers that have been reported in this packet. - /// - /// > **Note**: Keep in mind that this will also contain the responses we sent ourselves. - pub fn discovered_peers<'b>(&'b self) -> impl Iterator> { - let packet = &self.packet; - self.packet.answers.iter().filter_map(move |record| { +impl MdnsResponse { + /// Creates a new `MdnsResponse` based on the provided `Packet`. + fn new(packet: Packet, from: SocketAddr) -> MdnsResponse { + let peers = packet.answers.iter().filter_map(|record| { if record.name.to_string().as_bytes() != SERVICE_NAME { return None; } @@ -427,13 +424,25 @@ impl<'a> MdnsResponse<'a> { Err(_) => return None, }; - Some(MdnsPeer { - packet, + Some(MdnsPeer::new ( + &packet, record_value, peer_id, - ttl: record.ttl, - }) - }) + record.ttl, + )) + }).collect(); + + MdnsResponse { + peers, + from, + } + } + + /// Returns the list of peers that have been reported in this packet. + /// + /// > **Note**: Keep in mind that this will also contain the responses we sent ourselves. + pub fn discovered_peers(&self) -> impl Iterator { + self.peers.iter() } /// Source address of the packet. @@ -443,7 +452,7 @@ impl<'a> MdnsResponse<'a> { } } -impl<'a> fmt::Debug for MdnsResponse<'a> { +impl fmt::Debug for MdnsResponse { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("MdnsResponse") .field("from", self.remote_addr()) @@ -452,41 +461,22 @@ impl<'a> fmt::Debug for MdnsResponse<'a> { } /// A peer discovered by the service. -pub struct MdnsPeer<'a> { - /// The original packet which will be used to determine the addresses. - packet: &'a Packet<'a>, - /// Cached value of `concat(base32(peer_id), service name)`. - record_value: String, +pub struct MdnsPeer { + addrs: Vec, /// Id of the peer. peer_id: PeerId, /// TTL of the record in seconds. ttl: u32, } -impl<'a> MdnsPeer<'a> { - /// Returns the id of the peer. - #[inline] - pub fn id(&self) -> &PeerId { - &self.peer_id - } - - /// Returns the requested time-to-live for the record. - #[inline] - pub fn ttl(&self) -> Duration { - Duration::from_secs(u64::from(self.ttl)) - } - - /// Returns the list of addresses the peer says it is listening on. - /// - /// Filters out invalid addresses. - pub fn addresses<'b>(&'b self) -> impl Iterator + 'b { - let my_peer_id = &self.peer_id; - let record_value = &self.record_value; - self.packet +impl MdnsPeer { + /// Creates a new `MdnsPeer` based on the provided `Packet`. + pub fn new(packet: &Packet, record_value: String, my_peer_id: PeerId, ttl: u32) -> MdnsPeer { + let addrs = packet .additional .iter() - .filter_map(move |add_record| { - if &add_record.name.to_string() != record_value { + .filter_map(|add_record| { + if add_record.name.to_string() != record_value { return None; } @@ -497,7 +487,7 @@ impl<'a> MdnsPeer<'a> { } }) .flat_map(|txt| txt.iter()) - .filter_map(move |txt| { + .filter_map(|txt| { // TODO: wrong, txt can be multiple character strings let addr = match dns::decode_character_string(txt) { Ok(a) => a, @@ -515,15 +505,40 @@ impl<'a> MdnsPeer<'a> { Err(_) => return None, }; match addr.pop() { - Some(Protocol::P2p(ref peer_id)) if peer_id == my_peer_id => (), + Some(Protocol::P2p(ref peer_id)) if peer_id == &my_peer_id => (), _ => return None, }; Some(addr) - }) + }).collect(); + + MdnsPeer { + addrs, + peer_id: my_peer_id.clone(), + ttl, + } + } + + /// Returns the id of the peer. + #[inline] + pub fn id(&self) -> &PeerId { + &self.peer_id + } + + /// Returns the requested time-to-live for the record. + #[inline] + pub fn ttl(&self) -> Duration { + Duration::from_secs(u64::from(self.ttl)) + } + + /// Returns the list of addresses the peer says it is listening on. + /// + /// Filters out invalid addresses. + pub fn addresses(&self) -> &Vec { + &self.addrs } } -impl<'a> fmt::Debug for MdnsPeer<'a> { +impl fmt::Debug for MdnsPeer { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("MdnsPeer") .field("peer_id", &self.peer_id) @@ -533,42 +548,86 @@ impl<'a> fmt::Debug for MdnsPeer<'a> { #[cfg(test)] mod tests { - use libp2p_core::PeerId; - use std::{io, time::Duration}; - use tokio::{self, prelude::*}; + use futures::executor::block_on; + use libp2p_core::{PeerId, multiaddr::multihash::*}; + use std::{io::{Error, ErrorKind}, time::Duration}; + use wasm_timer::ext::TryFutureExt; use crate::service::{MdnsPacket, MdnsService}; - use multiaddr::multihash::*; fn discover(peer_id: PeerId) { - let mut service = MdnsService::new().unwrap(); - let stream = stream::poll_fn(move || -> Poll, io::Error> { + block_on(async { + let mut service = MdnsService::new().await.unwrap(); loop { - let packet = match service.poll() { - Async::Ready(packet) => packet, - Async::NotReady => return Ok(Async::NotReady), - }; + let next = service.next().await; + service = next.0; - match packet { + match next.1 { MdnsPacket::Query(query) => { - query.respond(peer_id.clone(), None, Duration::from_secs(120)).unwrap(); + let resp = crate::dns::build_query_response( + query.query_id(), + peer_id.clone(), + vec![].into_iter(), + Duration::from_secs(120), + ).unwrap(); + service.enqueue_response(resp); } MdnsPacket::Response(response) => { for peer in response.discovered_peers() { if peer.id() == &peer_id { - return Ok(Async::Ready(None)); + return; } } } - MdnsPacket::ServiceDiscovery(_) => {} + MdnsPacket::ServiceDiscovery(_) => panic!("did not expect a service discovery packet") + } + } + }) + } + + // As of today the underlying UDP socket is not stubbed out. Thus tests run in parallel to this + // unit tests inter fear with it. Test needs to be run in sequence to ensure test properties. + #[test] + fn respect_query_interval() { + let own_ips: Vec = get_if_addrs::get_if_addrs().unwrap() + .into_iter() + .map(|i| i.addr.ip()) + .collect(); + + let fut = async { + let mut service = MdnsService::new().await.unwrap(); + let mut sent_queries = vec![]; + + loop { + let next = service.next().await; + service = next.0; + + match next.1 { + MdnsPacket::Query(query) => { + // Ignore queries from other nodes. + let source_ip = query.remote_addr().ip(); + if !own_ips.contains(&source_ip) { + continue; + } + + sent_queries.push(query); + + if sent_queries.len() > 1 { + return Ok(()) + } + } + // Ignore response packets. We don't stub out the UDP socket, thus this is + // either random noise from the network, or noise from other unit tests running + // in parallel. + MdnsPacket::Response(_) => {}, + MdnsPacket::ServiceDiscovery(_) => { + return Err(Error::new(ErrorKind::Other, "did not expect a service discovery packet")); + }, } } - }); + }; - tokio::run( - stream - .map_err(|err| panic!("{:?}", err)) - .for_each(|_| Ok(())), - ); + // TODO: This might be too long for a unit test. + block_on(fut.timeout(Duration::from_secs(41))).unwrap(); } #[test] diff --git a/misc/multiaddr/Cargo.toml b/misc/multiaddr/Cargo.toml index c7b6b1bc671..8b75d1da3e7 100644 --- a/misc/multiaddr/Cargo.toml +++ b/misc/multiaddr/Cargo.toml @@ -6,24 +6,22 @@ description = "Implementation of the multiaddr format" homepage = "https://github.com/libp2p/rust-libp2p" keywords = ["multiaddr", "ipfs"] license = "MIT" -version = "0.6.0" +version = "0.7.0" [dependencies] arrayref = "0.3" bs58 = "0.3.0" byteorder = "1.3.1" -bytes = "0.4.12" data-encoding = "2.1" -multihash = { package = "parity-multihash", version = "0.2.0", path = "../multihash" } +multihash = { package = "parity-multihash", version = "0.2.1", path = "../multihash" } percent-encoding = "2.1.0" serde = "1.0.70" -unsigned-varint = "0.2" +static_assertions = "1.1" +unsigned-varint = "0.3" url = { version = "2.1.0", default-features = false } [dev-dependencies] bincode = "1" -bs58 = "0.3.0" -data-encoding = "2" quickcheck = "0.9.0" rand = "0.7.2" serde_json = "1.0" diff --git a/misc/multiaddr/src/errors.rs b/misc/multiaddr/src/errors.rs index 2636358319e..964cdffff50 100644 --- a/misc/multiaddr/src/errors.rs +++ b/misc/multiaddr/src/errors.rs @@ -14,7 +14,7 @@ pub enum Error { InvalidUvar(decode::Error), ParsingError(Box), UnknownProtocolId(u32), - UnknownProtocolString, + UnknownProtocolString(String), #[doc(hidden)] __Nonexhaustive } @@ -28,7 +28,7 @@ impl fmt::Display for Error { Error::InvalidUvar(e) => write!(f, "failed to decode unsigned varint: {}", e), Error::ParsingError(e) => write!(f, "failed to parse: {}", e), Error::UnknownProtocolId(id) => write!(f, "unknown protocol id: {}", id), - Error::UnknownProtocolString => f.write_str("unknown protocol string"), + Error::UnknownProtocolString(string) => write!(f, "unknown protocol string: {}", string), Error::__Nonexhaustive => f.write_str("__Nonexhaustive") } } diff --git a/misc/multiaddr/src/lib.rs b/misc/multiaddr/src/lib.rs index 5d3f0ae67ce..8e50b4dcfb2 100644 --- a/misc/multiaddr/src/lib.rs +++ b/misc/multiaddr/src/lib.rs @@ -5,9 +5,7 @@ pub use multihash; mod protocol; mod errors; mod from_url; -mod util; -use bytes::{Bytes, BytesMut}; use serde::{ Deserialize, Deserializer, @@ -18,28 +16,36 @@ use serde::{ use std::{ convert::TryFrom, fmt, + io, iter::FromIterator, net::{IpAddr, Ipv4Addr, Ipv6Addr}, result::Result as StdResult, - str::FromStr + str::FromStr, + sync::Arc }; pub use self::errors::{Result, Error}; pub use self::from_url::{FromUrlErr, from_url, from_url_lossy}; pub use self::protocol::Protocol; +static_assertions::const_assert! { + // This check is most certainly overkill right now, but done here + // anyway to ensure the `as u64` casts in this crate are safe. + std::mem::size_of::() <= std::mem::size_of::() +} + /// Representation of a Multiaddr. #[derive(PartialEq, Eq, Clone, Hash)] -pub struct Multiaddr { bytes: Bytes } +pub struct Multiaddr { bytes: Arc> } impl Multiaddr { /// Create a new, empty multiaddress. pub fn empty() -> Self { - Self { bytes: Bytes::new() } + Self { bytes: Arc::new(Vec::new()) } } /// Create a new, empty multiaddress with the given capacity. pub fn with_capacity(n: usize) -> Self { - Self { bytes: Bytes::with_capacity(n) } + Self { bytes: Arc::new(Vec::with_capacity(n)) } } /// Return the length in bytes of this multiaddress. @@ -65,9 +71,9 @@ impl Multiaddr { /// ``` /// pub fn push(&mut self, p: Protocol<'_>) { - let mut w = Vec::new(); - p.write_bytes(&mut w).expect("Writing to a `Vec` never fails."); - self.bytes.extend_from_slice(&w); + let mut w = io::Cursor::<&mut Vec>::new(Arc::make_mut(&mut self.bytes)); + w.set_position(w.get_ref().len() as u64); + p.write_bytes(&mut w).expect("Writing to a `io::Cursor<&mut Vec>` never fails.") } /// Pops the last `Protocol` of this multiaddr, or `None` if the multiaddr is empty. @@ -93,25 +99,16 @@ impl Multiaddr { slice = s }; let remaining_len = self.bytes.len() - slice.len(); - self.bytes.truncate(remaining_len); + Arc::make_mut(&mut self.bytes).truncate(remaining_len); Some(protocol) } - /// Like [`push`] but more efficient if this `Multiaddr` has no living clones. - pub fn with(self, p: Protocol<'_>) -> Self { - match self.bytes.try_mut() { - Ok(bytes) => { - let mut w = util::BytesWriter(bytes); - p.write_bytes(&mut w).expect("Writing to a `BytesWriter` never fails."); - Multiaddr { bytes: w.0.freeze() } - } - Err(mut bytes) => { - let mut w = Vec::new(); - p.write_bytes(&mut w).expect("Writing to a `Vec` never fails."); - bytes.extend_from_slice(&w); - Multiaddr { bytes } - } - } + /// Like [`push`] but consumes `self`. + pub fn with(mut self, p: Protocol<'_>) -> Self { + let mut w = io::Cursor::<&mut Vec>::new(Arc::make_mut(&mut self.bytes)); + w.set_position(w.get_ref().len() as u64); + p.write_bytes(&mut w).expect("Writing to a `io::Cursor<&mut Vec>` never fails."); + self } /// Returns the components of this multiaddress. @@ -217,7 +214,7 @@ impl<'a> FromIterator> for Multiaddr { for cmp in iter { cmp.write_bytes(&mut writer).expect("Writing to a `Vec` never fails."); } - Multiaddr { bytes: writer.into() } + Multiaddr { bytes: Arc::new(writer) } } } @@ -238,7 +235,7 @@ impl FromStr for Multiaddr { p.write_bytes(&mut writer).expect("Writing to a `Vec` never fails."); } - Ok(Multiaddr { bytes: writer.into() }) + Ok(Multiaddr { bytes: Arc::new(writer) }) } } @@ -265,7 +262,7 @@ impl<'a> From> for Multiaddr { fn from(p: Protocol<'a>) -> Multiaddr { let mut w = Vec::new(); p.write_bytes(&mut w).expect("Writing to a `Vec` never fails."); - Multiaddr { bytes: w.into() } + Multiaddr { bytes: Arc::new(w) } } } @@ -290,33 +287,17 @@ impl From for Multiaddr { } } -impl TryFrom for Multiaddr { +impl TryFrom> for Multiaddr { type Error = Error; - fn try_from(v: Bytes) -> Result { + fn try_from(v: Vec) -> Result { // Check if the argument is a valid `Multiaddr` by reading its protocols. let mut slice = &v[..]; while !slice.is_empty() { let (_, s) = Protocol::from_bytes(slice)?; slice = s } - Ok(Multiaddr { bytes: v.into() }) - } -} - -impl TryFrom for Multiaddr { - type Error = Error; - - fn try_from(v: BytesMut) -> Result { - Multiaddr::try_from(v.freeze()) - } -} - -impl TryFrom> for Multiaddr { - type Error = Error; - - fn try_from(v: Vec) -> Result { - Multiaddr::try_from(Bytes::from(v)) + Ok(Multiaddr { bytes: Arc::new(v) }) } } @@ -429,4 +410,3 @@ macro_rules! multiaddr { } } } - diff --git a/misc/multiaddr/src/protocol.rs b/misc/multiaddr/src/protocol.rs index 2b687b126e9..e424a5c746c 100644 --- a/misc/multiaddr/src/protocol.rs +++ b/misc/multiaddr/src/protocol.rs @@ -171,7 +171,7 @@ impl<'a> Protocol<'a> { let s = iter.next().ok_or(Error::InvalidProtocolString)?; Ok(Protocol::Memory(s.parse()?)) } - _ => Err(Error::UnknownProtocolString) + unknown => Err(Error::UnknownProtocolString(unknown.to_string())) } } diff --git a/misc/multiaddr/src/util.rs b/misc/multiaddr/src/util.rs deleted file mode 100644 index a0b28d35773..00000000000 --- a/misc/multiaddr/src/util.rs +++ /dev/null @@ -1,21 +0,0 @@ -use bytes::BytesMut; - -/// An [`io::Write`] impl for [`BytesMut`]. -/// -/// In contrast to [`bytes::buf::Writer`] this [`io::Write] implementation -/// transparently reserves enough space for [`io::Write::write_all`] to -/// succeed, i.e. it does not require upfront reservation of space. -pub(crate) struct BytesWriter(pub(crate) BytesMut); - -impl std::io::Write for BytesWriter { - fn write(&mut self, src: &[u8]) -> std::io::Result { - self.0.extend_from_slice(src); - Ok(src.len()) - } - - fn flush(&mut self) -> std::io::Result<()> { - Ok(()) - } -} - - diff --git a/misc/multiaddr/tests/lib.rs b/misc/multiaddr/tests/lib.rs index 0516a682629..d31e45cd953 100644 --- a/misc/multiaddr/tests/lib.rs +++ b/misc/multiaddr/tests/lib.rs @@ -328,3 +328,15 @@ fn replace_ip4_with_ip6() { assert_eq!(result.unwrap(), "/ip6/2001:db8::1/tcp/10000".parse::().unwrap()) } +#[test] +fn unknown_protocol_string() { + match "/unknown/1.2.3.4".parse::() { + Ok(_) => assert!(false, "The UnknownProtocolString error should be caused"), + Err(e) => match e { + crate::Error::UnknownProtocolString(protocol) => { + assert_eq!(protocol, "unknown") + }, + _ => assert!(false, "The UnknownProtocolString error should be caused") + } + } +} diff --git a/misc/multihash/Cargo.toml b/misc/multihash/Cargo.toml index 82a231fb6d5..dfbb03d049c 100644 --- a/misc/multihash/Cargo.toml +++ b/misc/multihash/Cargo.toml @@ -4,16 +4,16 @@ edition = "2018" description = "Implementation of the multihash format" repository = "https://github.com/libp2p/rust-libp2p" keywords = ["multihash", "ipfs"] -version = "0.2.0" +version = "0.2.1" authors = ["dignifiedquire ", "Parity Technologies "] license = "MIT" documentation = "https://docs.rs/parity-multihash/" [dependencies] blake2 = { version = "0.8", default-features = false } -bytes = "0.4.12" -rand = { version = "0.6", default-features = false, features = ["std"] } +bytes = "0.5" +rand = { version = "0.7", default-features = false, features = ["std"] } sha-1 = { version = "0.8", default-features = false } sha2 = { version = "0.8", default-features = false } sha3 = { version = "0.8", default-features = false } -unsigned-varint = "0.2" +unsigned-varint = "0.3" diff --git a/misc/multihash/src/lib.rs b/misc/multihash/src/lib.rs index 25a1d82434d..ec7eaeab1df 100644 --- a/misc/multihash/src/lib.rs +++ b/misc/multihash/src/lib.rs @@ -247,7 +247,7 @@ impl<'a> MultihashRef<'a> { /// This operation allocates. pub fn into_owned(self) -> Multihash { Multihash { - bytes: Bytes::from(self.bytes) + bytes: Bytes::copy_from_slice(self.bytes) } } diff --git a/misc/multistream-select/Cargo.toml b/misc/multistream-select/Cargo.toml index f9b04c77856..fee5cffb605 100644 --- a/misc/multistream-select/Cargo.toml +++ b/misc/multistream-select/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "multistream-select" description = "Multistream-select negotiation protocol for libp2p" -version = "0.6.1" +version = "0.7.0" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,12 +10,12 @@ categories = ["network-programming", "asynchronous"] edition = "2018" [dependencies] -bytes = "0.4" +bytes = "0.5" futures = "0.1" log = "0.4" -smallvec = "0.6" +smallvec = "1.0" tokio-io = "0.1" -unsigned-varint = "0.2.2" +unsigned-varint = "0.3" [dev-dependencies] tokio = "0.1" diff --git a/misc/multistream-select/src/length_delimited.rs b/misc/multistream-select/src/length_delimited.rs index 91e3fe88870..bc363c7e167 100644 --- a/misc/multistream-select/src/length_delimited.rs +++ b/misc/multistream-select/src/length_delimited.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use bytes::{Bytes, BytesMut, BufMut}; +use bytes::{Bytes, BytesMut, Buf, BufMut}; use futures::{try_ready, Async, Poll, Sink, StartSend, Stream, AsyncSink}; use std::{io, u16}; use tokio_io::{AsyncRead, AsyncWrite}; @@ -136,7 +136,7 @@ impl LengthDelimited { "Failed to write buffered frame.")) } - self.write_buffer.split_to(n); + self.write_buffer.advance(n); } Ok(Async::Ready(())) diff --git a/misc/multistream-select/src/negotiated.rs b/misc/multistream-select/src/negotiated.rs index 5e2c7ac991a..7611aee5f60 100644 --- a/misc/multistream-select/src/negotiated.rs +++ b/misc/multistream-select/src/negotiated.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use bytes::BytesMut; +use bytes::{BytesMut, Buf}; use crate::protocol::{Protocol, MessageReader, Message, Version, ProtocolError}; use futures::{prelude::*, Async, try_ready}; use log::debug; @@ -93,7 +93,7 @@ impl Negotiated { } if let State::Completed { remaining, .. } = &mut self.state { - let _ = remaining.take(); // Drop remaining data flushed above. + let _ = remaining.split_to(remaining.len()); // Drop remaining data flushed above. return Ok(Async::Ready(())) } @@ -232,7 +232,7 @@ where if n == 0 { return Err(io::ErrorKind::WriteZero.into()) } - remaining.split_to(n); + remaining.advance(n); } io.write(buf) }, @@ -251,7 +251,7 @@ where io::ErrorKind::WriteZero, "Failed to write remaining buffer.")) } - remaining.split_to(n); + remaining.advance(n); } io.flush() }, @@ -363,7 +363,7 @@ mod tests { let cap = rem.len() + free as usize; let step = u8::min(free, step) as usize + 1; let buf = Capped { buf: Vec::with_capacity(cap), step }; - let rem = BytesMut::from(rem); + let rem = BytesMut::from(&rem[..]); let mut io = Negotiated::completed(buf, rem.clone()); let mut written = 0; loop { diff --git a/misc/multistream-select/src/protocol.rs b/misc/multistream-select/src/protocol.rs index a21b80030f8..d895a2272ed 100644 --- a/misc/multistream-select/src/protocol.rs +++ b/misc/multistream-select/src/protocol.rs @@ -143,7 +143,7 @@ impl TryFrom<&[u8]> for Protocol { type Error = ProtocolError; fn try_from(value: &[u8]) -> Result { - Self::try_from(Bytes::from(value)) + Self::try_from(Bytes::copy_from_slice(value)) } } @@ -208,7 +208,7 @@ impl Message { out_msg.push(b'\n') } dest.reserve(out_msg.len()); - dest.put(out_msg); + dest.put(out_msg.as_ref()); Ok(()) } Message::NotAvailable => { @@ -254,7 +254,7 @@ impl Message { if len == 0 || len > rem.len() || rem[len - 1] != b'\n' { return Err(ProtocolError::InvalidMessage) } - let p = Protocol::try_from(Bytes::from(&rem[.. len - 1]))?; + let p = Protocol::try_from(Bytes::copy_from_slice(&rem[.. len - 1]))?; protocols.push(p); remaining = &rem[len ..] } diff --git a/misc/peer-id-generator/Cargo.toml b/misc/peer-id-generator/Cargo.toml index c6f2cad4496..c68532b7c27 100644 --- a/misc/peer-id-generator/Cargo.toml +++ b/misc/peer-id-generator/Cargo.toml @@ -11,5 +11,5 @@ categories = ["network-programming", "asynchronous"] publish = false [dependencies] -libp2p-core = { version = "0.13.0", path = "../../core" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } num_cpus = "1.8" diff --git a/misc/rw-stream-sink/Cargo.toml b/misc/rw-stream-sink/Cargo.toml index a10be35a088..76464be571e 100644 --- a/misc/rw-stream-sink/Cargo.toml +++ b/misc/rw-stream-sink/Cargo.toml @@ -2,7 +2,7 @@ name = "rw-stream-sink" edition = "2018" description = "Adaptator between Stream/Sink and AsyncRead/AsyncWrite" -version = "0.1.2" +version = "0.2.0" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,6 +10,9 @@ keywords = ["networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4" -futures = "0.1" -tokio-io = "0.1" +futures = "0.3.1" +pin-project = "0.4.6" +static_assertions = "1" + +[dev-dependencies] +async-std = "1.0" diff --git a/misc/rw-stream-sink/src/lib.rs b/misc/rw-stream-sink/src/lib.rs index d73cb5d6592..69b30205155 100644 --- a/misc/rw-stream-sink/src/lib.rs +++ b/misc/rw-stream-sink/src/lib.rs @@ -18,202 +18,185 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -//! This crate provides the `RwStreamSink` type. It wraps around a `Stream + Sink` that produces -//! and accepts byte arrays, and implements `AsyncRead` and `AsyncWrite`. +//! This crate provides the [`RwStreamSink`] type. It wraps around a [`Stream`] +//! and [`Sink`] that produces and accepts byte arrays, and implements +//! [`AsyncRead`] and [`AsyncWrite`]. //! -//! Each call to `write()` will send one packet on the sink. Calls to `read()` will read from -//! incoming packets. -//! -//! > **Note**: Although this crate is hosted in the libp2p repo, it is purely a utility crate and -//! > not at all specific to libp2p. - -use bytes::{Buf, IntoBuf}; -use futures::{Async, AsyncSink, Poll, Sink, Stream}; -use std::cmp; -use std::io::Error as IoError; -use std::io::ErrorKind as IoErrorKind; -use std::io::{Read, Write}; -use tokio_io::{AsyncRead, AsyncWrite}; - -/// Wraps around a `Stream + Sink` whose items are buffers. Implements `AsyncRead` and `AsyncWrite`. -pub struct RwStreamSink -where - S: Stream, - S::Item: IntoBuf, -{ +//! Each call to [`AsyncWrite::poll_write`] will send one packet to the sink. +//! Calls to [`AsyncRead::read`] will read from the stream's incoming packets. + +use futures::{prelude::*, ready}; +use std::{io::{self, Read}, pin::Pin, task::{Context, Poll}}; + +static_assertions::const_assert!(std::mem::size_of::() <= std::mem::size_of::()); + +/// Wraps a [`Stream`] and [`Sink`] whose items are buffers. +/// Implements [`AsyncRead`] and [`AsyncWrite`]. +#[pin_project::pin_project] +pub struct RwStreamSink { + #[pin] inner: S, - current_item: Option<::Buf>, + current_item: Option::Ok>> } -impl RwStreamSink -where - S: Stream, - S::Item: IntoBuf, -{ +impl RwStreamSink { /// Wraps around `inner`. - pub fn new(inner: S) -> RwStreamSink { + pub fn new(inner: S) -> Self { RwStreamSink { inner, current_item: None } } } -impl Read for RwStreamSink +impl AsyncRead for RwStreamSink where - S: Stream, - S::Item: IntoBuf, + S: TryStream + Unpin, + ::Ok: AsRef<[u8]> { - fn read(&mut self, buf: &mut [u8]) -> Result { + fn poll_read(self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + let mut this = self.project(); + // Grab the item to copy from. let item_to_copy = loop { - if let Some(ref mut i) = self.current_item { - if i.has_remaining() { - break i; + if let Some(ref mut i) = this.current_item { + if i.position() < i.get_ref().as_ref().len() as u64 { + break i } } - - self.current_item = Some(match self.inner.poll()? { - Async::Ready(Some(i)) => i.into_buf(), - Async::Ready(None) => return Ok(0), // EOF - Async::NotReady => return Err(IoErrorKind::WouldBlock.into()), + *this.current_item = Some(match ready!(this.inner.as_mut().try_poll_next(cx)) { + Some(Ok(i)) => std::io::Cursor::new(i), + Some(Err(e)) => return Poll::Ready(Err(e)), + None => return Poll::Ready(Ok(0)) // EOF }); }; // Copy it! - debug_assert!(item_to_copy.has_remaining()); - let to_copy = cmp::min(buf.len(), item_to_copy.remaining()); - item_to_copy.take(to_copy).copy_to_slice(&mut buf[..to_copy]); - Ok(to_copy) + Poll::Ready(Ok(item_to_copy.read(buf)?)) } } -impl AsyncRead for RwStreamSink -where - S: Stream, - S::Item: IntoBuf, -{ - unsafe fn prepare_uninitialized_buffer(&self, _: &mut [u8]) -> bool { - false - } -} - -impl Write for RwStreamSink +impl AsyncWrite for RwStreamSink where - S: Stream + Sink, - S::SinkItem: for<'r> From<&'r [u8]>, - S::Item: IntoBuf, + S: TryStream + Sink<::Ok, Error = io::Error> + Unpin, + ::Ok: for<'r> From<&'r [u8]> { - fn write(&mut self, buf: &[u8]) -> Result { - let len = buf.len(); - match self.inner.start_send(buf.into())? { - AsyncSink::Ready => Ok(len), - AsyncSink::NotReady(_) => Err(IoError::new(IoErrorKind::WouldBlock, "not ready")), + fn poll_write(self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll> { + let mut this = self.project(); + ready!(this.inner.as_mut().poll_ready(cx)?); + let n = buf.len(); + if let Err(e) = this.inner.start_send(buf.into()) { + return Poll::Ready(Err(e)) } + Poll::Ready(Ok(n)) } - fn flush(&mut self) -> Result<(), IoError> { - match self.inner.poll_complete()? { - Async::Ready(()) => Ok(()), - Async::NotReady => Err(IoError::new(IoErrorKind::WouldBlock, "not ready")) - } + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + this.inner.poll_flush(cx) } -} -impl AsyncWrite for RwStreamSink -where - S: Stream + Sink, - S::SinkItem: for<'r> From<&'r [u8]>, - S::Item: IntoBuf, -{ - fn shutdown(&mut self) -> Poll<(), IoError> { - self.inner.close() + fn poll_close(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + this.inner.poll_close(cx) } } #[cfg(test)] mod tests { - use bytes::Bytes; - use crate::RwStreamSink; - use futures::{prelude::*, stream, sync::mpsc::channel}; - use std::io::Read; + use async_std::task; + use futures::{channel::mpsc, prelude::*, stream}; + use std::{pin::Pin, task::{Context, Poll}}; + use super::RwStreamSink; // This struct merges a stream and a sink and is quite useful for tests. struct Wrapper(St, Si); + impl Stream for Wrapper where - St: Stream, + St: Stream + Unpin, + Si: Unpin { type Item = St::Item; - type Error = St::Error; - fn poll(&mut self) -> Poll, Self::Error> { - self.0.poll() + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + self.0.poll_next_unpin(cx) } } - impl Sink for Wrapper + + impl Sink for Wrapper where - Si: Sink, + St: Unpin, + Si: Sink + Unpin, { - type SinkItem = Si::SinkItem; - type SinkError = Si::SinkError; - fn start_send( - &mut self, - item: Self::SinkItem, - ) -> StartSend { - self.1.start_send(item) + type Error = Si::Error; + + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.1).poll_ready(cx) } - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.1.poll_complete() + + fn start_send(mut self: Pin<&mut Self>, item: T) -> Result<(), Self::Error> { + Pin::new(&mut self.1).start_send(item) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.1.close() + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.1).poll_flush(cx) + } + + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.1).poll_close(cx) } } #[test] fn basic_reading() { - let (tx1, _) = channel::>(10); - let (tx2, rx2) = channel(10); + let (tx1, _) = mpsc::channel::>(10); + let (mut tx2, rx2) = mpsc::channel(10); - let mut wrapper = RwStreamSink::new(Wrapper(rx2.map_err(|_| panic!()), tx1)); + let mut wrapper = RwStreamSink::new(Wrapper(rx2.map(Ok), tx1)); - tx2.send(Bytes::from("hel")) - .and_then(|tx| tx.send(Bytes::from("lo wor"))) - .and_then(|tx| tx.send(Bytes::from("ld"))) - .wait() - .unwrap(); + task::block_on(async move { + tx2.send(Vec::from("hel")).await.unwrap(); + tx2.send(Vec::from("lo wor")).await.unwrap(); + tx2.send(Vec::from("ld")).await.unwrap(); + tx2.close().await.unwrap(); - let mut data = Vec::new(); - wrapper.read_to_end(&mut data).unwrap(); - assert_eq!(data, b"hello world"); + let mut data = Vec::new(); + wrapper.read_to_end(&mut data).await.unwrap(); + assert_eq!(data, b"hello world"); + }) } #[test] fn skip_empty_stream_items() { let data: Vec<&[u8]> = vec![b"", b"foo", b"", b"bar", b"", b"baz", b""]; - let mut rws = RwStreamSink::new(stream::iter_ok::<_, std::io::Error>(data)); + let mut rws = RwStreamSink::new(stream::iter(data).map(Ok)); let mut buf = [0; 9]; - assert_eq!(3, rws.read(&mut buf).unwrap()); - assert_eq!(3, rws.read(&mut buf[3..]).unwrap()); - assert_eq!(3, rws.read(&mut buf[6..]).unwrap()); - assert_eq!(0, rws.read(&mut buf).unwrap()); - assert_eq!(b"foobarbaz", &buf[..]); + task::block_on(async move { + assert_eq!(3, rws.read(&mut buf).await.unwrap()); + assert_eq!(3, rws.read(&mut buf[3..]).await.unwrap()); + assert_eq!(3, rws.read(&mut buf[6..]).await.unwrap()); + assert_eq!(0, rws.read(&mut buf).await.unwrap()); + assert_eq!(b"foobarbaz", &buf[..]) + }) } #[test] fn partial_read() { let data: Vec<&[u8]> = vec![b"hell", b"o world"]; - let mut rws = RwStreamSink::new(stream::iter_ok::<_, std::io::Error>(data)); + let mut rws = RwStreamSink::new(stream::iter(data).map(Ok)); let mut buf = [0; 3]; - assert_eq!(3, rws.read(&mut buf).unwrap()); - assert_eq!(b"hel", &buf[..3]); - assert_eq!(0, rws.read(&mut buf[..0]).unwrap()); - assert_eq!(1, rws.read(&mut buf).unwrap()); - assert_eq!(b"l", &buf[..1]); - assert_eq!(3, rws.read(&mut buf).unwrap()); - assert_eq!(b"o w", &buf[..3]); - assert_eq!(0, rws.read(&mut buf[..0]).unwrap()); - assert_eq!(3, rws.read(&mut buf).unwrap()); - assert_eq!(b"orl", &buf[..3]); - assert_eq!(1, rws.read(&mut buf).unwrap()); - assert_eq!(b"d", &buf[..1]); - assert_eq!(0, rws.read(&mut buf).unwrap()); + task::block_on(async move { + assert_eq!(3, rws.read(&mut buf).await.unwrap()); + assert_eq!(b"hel", &buf[..3]); + assert_eq!(0, rws.read(&mut buf[..0]).await.unwrap()); + assert_eq!(1, rws.read(&mut buf).await.unwrap()); + assert_eq!(b"l", &buf[..1]); + assert_eq!(3, rws.read(&mut buf).await.unwrap()); + assert_eq!(b"o w", &buf[..3]); + assert_eq!(0, rws.read(&mut buf[..0]).await.unwrap()); + assert_eq!(3, rws.read(&mut buf).await.unwrap()); + assert_eq!(b"orl", &buf[..3]); + assert_eq!(1, rws.read(&mut buf).await.unwrap()); + assert_eq!(b"d", &buf[..1]); + assert_eq!(0, rws.read(&mut buf).await.unwrap()); + }) } } diff --git a/muxers/mplex/Cargo.toml b/muxers/mplex/Cargo.toml index b3c3649ea0c..09ef47679ca 100644 --- a/muxers/mplex/Cargo.toml +++ b/muxers/mplex/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-mplex" edition = "2018" description = "Mplex multiplexing protocol for libp2p" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,16 +10,15 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4.5" +bytes = "0.5" fnv = "1.0" -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } +futures = "0.3.1" +futures_codec = "0.3.4" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } log = "0.4" -parking_lot = "0.9" -tokio-codec = "0.1" -tokio-io = "0.1" -unsigned-varint = { version = "0.2.1", features = ["codec"] } +parking_lot = "0.10" +unsigned-varint = { version = "0.3", features = ["futures-codec"] } [dev-dependencies] -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } -tokio = "0.1" +async-std = "1.0" +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../../transports/tcp" } diff --git a/muxers/mplex/src/codec.rs b/muxers/mplex/src/codec.rs index 012862ba482..e04aa4c20b0 100644 --- a/muxers/mplex/src/codec.rs +++ b/muxers/mplex/src/codec.rs @@ -19,10 +19,10 @@ // DEALINGS IN THE SOFTWARE. use libp2p_core::Endpoint; +use futures_codec::{Decoder, Encoder}; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::mem; use bytes::{BufMut, Bytes, BytesMut}; -use tokio_io::codec::{Decoder, Encoder}; use unsigned_varint::{codec, encode}; // Maximum size for a packet: 1MB as per the spec. diff --git a/muxers/mplex/src/lib.rs b/muxers/mplex/src/lib.rs index 8806b031551..64c532f853f 100644 --- a/muxers/mplex/src/lib.rs +++ b/muxers/mplex/src/lib.rs @@ -20,21 +20,22 @@ mod codec; -use std::{cmp, iter, mem}; +use std::{cmp, iter, mem, pin::Pin, task::Context, task::Poll}; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use std::sync::{atomic::AtomicUsize, atomic::Ordering, Arc}; +use std::sync::Arc; +use std::task::Waker; use bytes::Bytes; use libp2p_core::{ Endpoint, StreamMuxer, - upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}, + upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}, }; use log::{debug, trace}; use parking_lot::Mutex; -use fnv::{FnvHashMap, FnvHashSet}; -use futures::{prelude::*, executor, future, stream::Fuse, task, task_local, try_ready}; -use tokio_codec::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; +use fnv::FnvHashSet; +use futures::{prelude::*, future, ready, stream::Fuse}; +use futures::task::{ArcWake, waker_ref}; +use futures_codec::Framed; /// Configuration for the multiplexer. #[derive(Debug, Clone)] @@ -96,22 +97,22 @@ impl MplexConfig { #[inline] fn upgrade(self, i: C) -> Multiplex where - C: AsyncRead + AsyncWrite + C: AsyncRead + AsyncWrite + Unpin { let max_buffer_len = self.max_buffer_len; Multiplex { inner: Mutex::new(MultiplexInner { error: Ok(()), - inner: executor::spawn(Framed::new(i, codec::Codec::new()).fuse()), + inner: Framed::new(i, codec::Codec::new()).fuse(), config: self, buffer: Vec::with_capacity(cmp::min(max_buffer_len, 512)), opened_substreams: Default::default(), next_outbound_stream_id: 0, notifier_read: Arc::new(Notifier { - to_notify: Mutex::new(Default::default()), + to_wake: Mutex::new(Default::default()), }), notifier_write: Arc::new(Notifier { - to_notify: Mutex::new(Default::default()), + to_wake: Mutex::new(Default::default()), }), is_shutdown: false, is_acknowledged: false, @@ -156,27 +157,27 @@ impl UpgradeInfo for MplexConfig { impl InboundUpgrade for MplexConfig where - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, { - type Output = Multiplex>; + type Output = Multiplex; type Error = IoError; - type Future = future::FutureResult; + type Future = future::Ready>; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - future::ok(self.upgrade(socket)) + fn upgrade_inbound(self, socket: C, _: Self::Info) -> Self::Future { + future::ready(Ok(self.upgrade(socket))) } } impl OutboundUpgrade for MplexConfig where - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, { - type Output = Multiplex>; + type Output = Multiplex; type Error = IoError; - type Future = future::FutureResult; + type Future = future::Ready>; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - future::ok(self.upgrade(socket)) + fn upgrade_outbound(self, socket: C, _: Self::Info) -> Self::Future { + future::ready(Ok(self.upgrade(socket))) } } @@ -190,7 +191,7 @@ struct MultiplexInner { // Error that happened earlier. Should poison any attempt to use this `MultiplexError`. error: Result<(), IoError>, // Underlying stream. - inner: executor::Spawn>>, + inner: Fuse>, /// The original configuration. config: MplexConfig, // Buffer of elements pulled from the stream but not processed yet. @@ -202,9 +203,9 @@ struct MultiplexInner { opened_substreams: FnvHashSet<(u32, Endpoint)>, // Id of the next outgoing substream. next_outbound_stream_id: u32, - /// List of tasks to notify when a read event happens on the underlying stream. + /// List of wakers to wake when a read event happens on the underlying stream. notifier_read: Arc, - /// List of tasks to notify when a write event happens on the underlying stream. + /// List of wakers to wake when a write event happens on the underlying stream. notifier_write: Arc, /// If true, the connection has been shut down. We need to be careful not to accidentally /// call `Sink::poll_complete` or `Sink::start_send` after `Sink::close`. @@ -214,23 +215,26 @@ struct MultiplexInner { } struct Notifier { - /// List of tasks to notify. - to_notify: Mutex>, + /// List of wakers to wake. + to_wake: Mutex>, } -impl executor::Notify for Notifier { - fn notify(&self, _: usize) { - let tasks = mem::replace(&mut *self.to_notify.lock(), Default::default()); - for (_, task) in tasks { - task.notify(); +impl Notifier { + fn insert(&self, waker: &Waker) { + let mut to_wake = self.to_wake.lock(); + if to_wake.iter().all(|w| !w.will_wake(waker)) { + to_wake.push(waker.clone()); } } } -// TODO: replace with another system -static NEXT_TASK_ID: AtomicUsize = AtomicUsize::new(0); -task_local!{ - static TASK_ID: usize = NEXT_TASK_ID.fetch_add(1, Ordering::Relaxed) +impl ArcWake for Notifier { + fn wake_by_ref(arc_self: &Arc) { + let wakers = mem::replace(&mut *arc_self.to_wake.lock(), Default::default()); + for waker in wakers { + waker.wake(); + } + } } // Note [StreamId]: mplex no longer partitions stream IDs into odd (for initiators) and @@ -245,25 +249,27 @@ task_local!{ /// Processes elements in `inner` until one matching `filter` is found. /// -/// If `NotReady` is returned, the current task is scheduled for later, just like with any `Poll`. -/// `Ready(Some())` is almost always returned. An error is returned if the stream is EOF. -fn next_match(inner: &mut MultiplexInner, mut filter: F) -> Poll -where C: AsyncRead + AsyncWrite, +/// If `Pending` is returned, the waker is kept and notified later, just like with any `Poll`. +/// `Ready(Ok())` is almost always returned. An error is returned if the stream is EOF. +fn next_match(inner: &mut MultiplexInner, cx: &mut Context, mut filter: F) -> Poll> +where C: AsyncRead + AsyncWrite + Unpin, F: FnMut(&codec::Elem) -> Option, { // If an error happened earlier, immediately return it. if let Err(ref err) = inner.error { - return Err(IoError::new(err.kind(), err.to_string())); + return Poll::Ready(Err(IoError::new(err.kind(), err.to_string()))); } if let Some((offset, out)) = inner.buffer.iter().enumerate().filter_map(|(n, v)| filter(v).map(|v| (n, v))).next() { + // Found a matching entry in the existing buffer! + // The buffer was full and no longer is, so let's notify everything. if inner.buffer.len() == inner.config.max_buffer_len { - executor::Notify::notify(&*inner.notifier_read, 0); + ArcWake::wake_by_ref(&inner.notifier_read); } inner.buffer.remove(offset); - return Ok(Async::Ready(out)); + return Poll::Ready(Ok(out)); } loop { @@ -274,24 +280,24 @@ where C: AsyncRead + AsyncWrite, match inner.config.max_buffer_behaviour { MaxBufferBehaviour::CloseAll => { inner.error = Err(IoError::new(IoErrorKind::Other, "reached maximum buffer length")); - return Err(IoError::new(IoErrorKind::Other, "reached maximum buffer length")); + return Poll::Ready(Err(IoError::new(IoErrorKind::Other, "reached maximum buffer length"))); }, MaxBufferBehaviour::Block => { - inner.notifier_read.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - return Ok(Async::NotReady); + inner.notifier_read.insert(cx.waker()); + return Poll::Pending }, } } - inner.notifier_read.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - let elem = match inner.inner.poll_stream_notify(&inner.notifier_read, 0) { - Ok(Async::Ready(Some(item))) => item, - Ok(Async::Ready(None)) => return Err(IoErrorKind::BrokenPipe.into()), - Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(err) => { + inner.notifier_read.insert(cx.waker()); + let elem = match Stream::poll_next(Pin::new(&mut inner.inner), &mut Context::from_waker(&waker_ref(&inner.notifier_read))) { + Poll::Ready(Some(Ok(item))) => item, + Poll::Ready(None) => return Poll::Ready(Err(IoErrorKind::BrokenPipe.into())), + Poll::Pending => return Poll::Pending, + Poll::Ready(Some(Err(err))) => { let err2 = IoError::new(err.kind(), err.to_string()); inner.error = Err(err); - return Err(err2); + return Poll::Ready(Err(err2)); }, }; @@ -312,7 +318,7 @@ where C: AsyncRead + AsyncWrite, } if let Some(out) = filter(&elem) { - return Ok(Async::Ready(out)); + return Poll::Ready(Ok(out)); } else { let endpoint = elem.endpoint().unwrap_or(Endpoint::Dialer); if inner.opened_substreams.contains(&(elem.substream_id(), !endpoint)) || elem.is_open_msg() { @@ -325,45 +331,57 @@ where C: AsyncRead + AsyncWrite, } // Small convenience function that tries to write `elem` to the stream. -fn poll_send(inner: &mut MultiplexInner, elem: codec::Elem) -> Poll<(), IoError> -where C: AsyncRead + AsyncWrite +fn poll_send(inner: &mut MultiplexInner, cx: &mut Context, elem: codec::Elem) -> Poll> +where C: AsyncRead + AsyncWrite + Unpin { if inner.is_shutdown { - return Err(IoError::new(IoErrorKind::Other, "connection is shut down")) + return Poll::Ready(Err(IoError::new(IoErrorKind::Other, "connection is shut down"))) } - inner.notifier_write.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - match inner.inner.start_send_notify(elem, &inner.notifier_write, 0) { - Ok(AsyncSink::Ready) => Ok(Async::Ready(())), - Ok(AsyncSink::NotReady(_)) => Ok(Async::NotReady), - Err(err) => Err(err) + + inner.notifier_write.insert(cx.waker()); + + match Sink::poll_ready(Pin::new(&mut inner.inner), &mut Context::from_waker(&waker_ref(&inner.notifier_write))) { + Poll::Ready(Ok(())) => { + match Sink::start_send(Pin::new(&mut inner.inner), elem) { + Ok(()) => Poll::Ready(Ok(())), + Err(err) => Poll::Ready(Err(err)) + } + }, + Poll::Pending => Poll::Pending, + Poll::Ready(Err(err)) => Poll::Ready(Err(err)) } } impl StreamMuxer for Multiplex -where C: AsyncRead + AsyncWrite +where C: AsyncRead + AsyncWrite + Unpin { type Substream = Substream; type OutboundSubstream = OutboundSubstream; type Error = IoError; - fn poll_inbound(&self) -> Poll { + fn poll_inbound(&self, cx: &mut Context) -> Poll> { let mut inner = self.inner.lock(); if inner.opened_substreams.len() >= inner.config.max_substreams { debug!("Refused substream; reached maximum number of substreams {}", inner.config.max_substreams); - return Err(IoError::new(IoErrorKind::ConnectionRefused, - "exceeded maximum number of open substreams")); + return Poll::Ready(Err(IoError::new(IoErrorKind::ConnectionRefused, + "exceeded maximum number of open substreams"))); } - let num = try_ready!(next_match(&mut inner, |elem| { + let num = ready!(next_match(&mut inner, cx, |elem| { match elem { codec::Elem::Open { substream_id } => Some(*substream_id), _ => None, } })); + let num = match num { + Ok(n) => n, + Err(err) => return Poll::Ready(Err(err)), + }; + debug!("Successfully opened inbound substream {}", num); - Ok(Async::Ready(Substream { + Poll::Ready(Ok(Substream { current_data: Bytes::new(), num, endpoint: Endpoint::Listener, @@ -391,21 +409,21 @@ where C: AsyncRead + AsyncWrite } } - fn poll_outbound(&self, substream: &mut Self::OutboundSubstream) -> Poll { + fn poll_outbound(&self, cx: &mut Context, substream: &mut Self::OutboundSubstream) -> Poll> { loop { let mut inner = self.inner.lock(); let polling = match substream.state { OutboundSubstreamState::SendElem(ref elem) => { - poll_send(&mut inner, elem.clone()) + poll_send(&mut inner, cx, elem.clone()) }, OutboundSubstreamState::Flush => { if inner.is_shutdown { - return Err(IoError::new(IoErrorKind::Other, "connection is shut down")) + return Poll::Ready(Err(IoError::new(IoErrorKind::Other, "connection is shut down"))) } let inner = &mut *inner; // Avoids borrow errors - inner.notifier_write.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - inner.inner.poll_flush_notify(&inner.notifier_write, 0) + inner.notifier_write.insert(cx.waker()); + Sink::poll_flush(Pin::new(&mut inner.inner), &mut Context::from_waker(&waker_ref(&inner.notifier_write))) }, OutboundSubstreamState::Done => { panic!("Polling outbound substream after it's been succesfully open"); @@ -413,16 +431,14 @@ where C: AsyncRead + AsyncWrite }; match polling { - Ok(Async::Ready(())) => (), - Ok(Async::NotReady) => { - return Ok(Async::NotReady) - }, - Err(err) => { + Poll::Ready(Ok(())) => (), + Poll::Pending => return Poll::Pending, + Poll::Ready(Err(err)) => { debug!("Failed to open outbound substream {}", substream.num); inner.buffer.retain(|elem| { elem.substream_id() != substream.num || elem.endpoint() == Some(Endpoint::Dialer) }); - return Err(err) + return Poll::Ready(Err(err)); }, }; @@ -436,7 +452,7 @@ where C: AsyncRead + AsyncWrite OutboundSubstreamState::Flush => { debug!("Successfully opened outbound substream {}", substream.num); substream.state = OutboundSubstreamState::Done; - return Ok(Async::Ready(Substream { + return Poll::Ready(Ok(Substream { num: substream.num, current_data: Bytes::new(), endpoint: Endpoint::Dialer, @@ -454,27 +470,23 @@ where C: AsyncRead + AsyncWrite // Nothing to do. } - unsafe fn prepare_uninitialized_buffer(&self, _: &mut [u8]) -> bool { - false - } - - fn read_substream(&self, substream: &mut Self::Substream, buf: &mut [u8]) -> Poll { + fn read_substream(&self, cx: &mut Context, substream: &mut Self::Substream, buf: &mut [u8]) -> Poll> { loop { // First, transfer from `current_data`. if !substream.current_data.is_empty() { let len = cmp::min(substream.current_data.len(), buf.len()); buf[..len].copy_from_slice(&substream.current_data.split_to(len)); - return Ok(Async::Ready(len)); + return Poll::Ready(Ok(len)); } // If the remote writing side is closed, return EOF. if !substream.remote_open { - return Ok(Async::Ready(0)); + return Poll::Ready(Ok(0)); } // Try to find a packet of data in the buffer. let mut inner = self.inner.lock(); - let next_data_poll = next_match(&mut inner, |elem| { + let next_data_poll = next_match(&mut inner, cx, |elem| { match elem { codec::Elem::Data { substream_id, endpoint, data, .. } if *substream_id == substream.num && *endpoint != substream.endpoint => // see note [StreamId] @@ -492,28 +504,29 @@ where C: AsyncRead + AsyncWrite // We're in a loop, so all we need to do is set `substream.current_data` to the data we // just read and wait for the next iteration. - match next_data_poll? { - Async::Ready(Some(data)) => substream.current_data = data, - Async::Ready(None) => { + match next_data_poll { + Poll::Ready(Ok(Some(data))) => substream.current_data = data, + Poll::Ready(Err(err)) => return Poll::Ready(Err(err)), + Poll::Ready(Ok(None)) => { substream.remote_open = false; - return Ok(Async::Ready(0)); + return Poll::Ready(Ok(0)); }, - Async::NotReady => { + Poll::Pending => { // There was no data packet in the buffer about this substream; maybe it's // because it has been closed. if inner.opened_substreams.contains(&(substream.num, substream.endpoint)) { - return Ok(Async::NotReady) + return Poll::Pending } else { - return Ok(Async::Ready(0)) + return Poll::Ready(Ok(0)) } }, } } } - fn write_substream(&self, substream: &mut Self::Substream, buf: &[u8]) -> Poll { + fn write_substream(&self, cx: &mut Context, substream: &mut Self::Substream, buf: &[u8]) -> Poll> { if !substream.local_open { - return Err(IoErrorKind::BrokenPipe.into()); + return Poll::Ready(Err(IoErrorKind::BrokenPipe.into())); } let mut inner = self.inner.lock(); @@ -522,30 +535,31 @@ where C: AsyncRead + AsyncWrite let elem = codec::Elem::Data { substream_id: substream.num, - data: From::from(&buf[..to_write]), + data: Bytes::copy_from_slice(&buf[..to_write]), endpoint: substream.endpoint, }; - match poll_send(&mut inner, elem)? { - Async::Ready(()) => Ok(Async::Ready(to_write)), - Async::NotReady => Ok(Async::NotReady) + match poll_send(&mut inner, cx, elem) { + Poll::Ready(Ok(())) => Poll::Ready(Ok(to_write)), + Poll::Ready(Err(err)) => Poll::Ready(Err(err)), + Poll::Pending => Poll::Pending, } } - fn flush_substream(&self, _substream: &mut Self::Substream) -> Poll<(), IoError> { + fn flush_substream(&self, cx: &mut Context, _substream: &mut Self::Substream) -> Poll> { let mut inner = self.inner.lock(); if inner.is_shutdown { - return Err(IoError::new(IoErrorKind::Other, "connection is shut down")) + return Poll::Ready(Err(IoError::new(IoErrorKind::Other, "connection is shut down"))) } let inner = &mut *inner; // Avoids borrow errors - inner.notifier_write.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - inner.inner.poll_flush_notify(&inner.notifier_write, 0) + inner.notifier_write.insert(cx.waker()); + Sink::poll_flush(Pin::new(&mut inner.inner), &mut Context::from_waker(&waker_ref(&inner.notifier_write))) } - fn shutdown_substream(&self, sub: &mut Self::Substream) -> Poll<(), IoError> { + fn shutdown_substream(&self, cx: &mut Context, sub: &mut Self::Substream) -> Poll> { if !sub.local_open { - return Ok(Async::Ready(())); + return Poll::Ready(Ok(())); } let elem = codec::Elem::Close { @@ -554,8 +568,8 @@ where C: AsyncRead + AsyncWrite }; let mut inner = self.inner.lock(); - let result = poll_send(&mut inner, elem); - if let Ok(Async::Ready(())) = result { + let result = poll_send(&mut inner, cx, elem); + if let Poll::Ready(Ok(())) = result { sub.local_open = false; } result @@ -572,22 +586,27 @@ where C: AsyncRead + AsyncWrite } #[inline] - fn close(&self) -> Poll<(), IoError> { + fn close(&self, cx: &mut Context) -> Poll> { let inner = &mut *self.inner.lock(); - inner.notifier_write.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - try_ready!(inner.inner.close_notify(&inner.notifier_write, 0)); - inner.is_shutdown = true; - Ok(Async::Ready(())) + inner.notifier_write.insert(cx.waker()); + match Sink::poll_close(Pin::new(&mut inner.inner), &mut Context::from_waker(&waker_ref(&inner.notifier_write))) { + Poll::Ready(Ok(())) => { + inner.is_shutdown = true; + Poll::Ready(Ok(())) + } + Poll::Ready(Err(err)) => Poll::Ready(Err(err)), + Poll::Pending => Poll::Pending, + } } #[inline] - fn flush_all(&self) -> Poll<(), IoError> { + fn flush_all(&self, cx: &mut Context) -> Poll> { let inner = &mut *self.inner.lock(); if inner.is_shutdown { - return Ok(Async::Ready(())) + return Poll::Ready(Ok(())) } - inner.notifier_write.to_notify.lock().insert(TASK_ID.with(|&t| t), task::current()); - inner.inner.poll_flush_notify(&inner.notifier_write, 0) + inner.notifier_write.insert(cx.waker()); + Sink::poll_flush(Pin::new(&mut inner.inner), &mut Context::from_waker(&waker_ref(&inner.notifier_write))) } } diff --git a/muxers/mplex/tests/async_write.rs b/muxers/mplex/tests/async_write.rs index 4fe3c319cb0..e0b708e340f 100644 --- a/muxers/mplex/tests/async_write.rs +++ b/muxers/mplex/tests/async_write.rs @@ -18,20 +18,18 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p_core::{muxing, upgrade, Transport, transport::ListenerEvent}; +use libp2p_core::{muxing, upgrade, Transport}; use libp2p_tcp::TcpConfig; -use futures::prelude::*; -use std::sync::{Arc, mpsc}; -use std::thread; -use tokio::runtime::current_thread::Runtime; +use futures::{prelude::*, channel::oneshot}; +use std::sync::Arc; #[test] fn async_write() { - // Tests that `AsyncWrite::shutdown` implies flush. + // Tests that `AsyncWrite::close` implies flush. - let (tx, rx) = mpsc::channel(); + let (tx, rx) = oneshot::channel(); - let bg_thread = thread::spawn(move || { + let bg_thread = async_std::task::spawn(async move { let mplex = libp2p_mplex::MplexConfig::new(); let transport = TcpConfig::new().and_then(move |c, e| @@ -41,8 +39,7 @@ fn async_write() { .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) .unwrap(); - let addr = listener.by_ref().wait() - .next() + let addr = listener.next().await .expect("some event") .expect("no error") .into_new_address() @@ -50,41 +47,31 @@ fn async_write() { tx.send(addr).unwrap(); - let future = listener - .filter_map(ListenerEvent::into_upgrade) - .into_future() - .map_err(|(err, _)| panic!("{:?}", err)) - .and_then(|(client, _)| client.unwrap().0) - .map_err(|err| panic!("{:?}", err)) - .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) - .and_then(|client| { - tokio::io::read_to_end(client, vec![]) - }) - .and_then(|(_, msg)| { - assert_eq!(msg, b"hello world"); - Ok(()) - }); + let client = listener + .next().await + .unwrap() + .unwrap() + .into_upgrade().unwrap().0.await.unwrap(); + + let mut outbound = muxing::outbound_from_ref_and_wrap(Arc::new(client)).await.unwrap(); - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); + let mut buf = Vec::new(); + outbound.read_to_end(&mut buf).await.unwrap(); + assert_eq!(buf, b"hello world"); }); - let mplex = libp2p_mplex::MplexConfig::new(); - let transport = TcpConfig::new().and_then(move |c, e| - upgrade::apply(c, mplex, e, upgrade::Version::V1)); + async_std::task::block_on(async { + let mplex = libp2p_mplex::MplexConfig::new(); + let transport = TcpConfig::new().and_then(move |c, e| + upgrade::apply(c, mplex, e, upgrade::Version::V1)); + + let client = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); + let mut inbound = muxing::inbound_from_ref_and_wrap(Arc::new(client)).await.unwrap(); + inbound.write_all(b"hello world").await.unwrap(); - let future = transport - .dial(rx.recv().unwrap()) - .unwrap() - .map_err(|err| panic!("{:?}", err)) - .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) - .and_then(|server| tokio::io::write_all(server, b"hello world")) - .and_then(|(server, _)| { - tokio::io::shutdown(server) - }) - .map(|_| ()); + // The test consists in making sure that this flushes the substream. + inbound.close().await.unwrap(); - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); - bg_thread.join().unwrap(); + bg_thread.await; + }); } diff --git a/muxers/mplex/tests/two_peers.rs b/muxers/mplex/tests/two_peers.rs index e3e7d5d7fbc..51293a37cfc 100644 --- a/muxers/mplex/tests/two_peers.rs +++ b/muxers/mplex/tests/two_peers.rs @@ -18,23 +18,18 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p_core::{muxing, upgrade, Transport, transport::ListenerEvent}; +use libp2p_core::{muxing, upgrade, Transport}; use libp2p_tcp::TcpConfig; -use futures::prelude::*; -use std::sync::{Arc, mpsc}; -use std::thread; -use tokio::{ - codec::length_delimited::Builder, - runtime::current_thread::Runtime -}; +use futures::{channel::oneshot, prelude::*}; +use std::sync::Arc; #[test] fn client_to_server_outbound() { // Simulate a client sending a message to a server through a multiplex upgrade. - let (tx, rx) = mpsc::channel(); + let (tx, rx) = oneshot::channel(); - let bg_thread = thread::spawn(move || { + let bg_thread = async_std::task::spawn(async move { let mplex = libp2p_mplex::MplexConfig::new(); let transport = TcpConfig::new().and_then(move |c, e| @@ -44,8 +39,7 @@ fn client_to_server_outbound() { .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) .unwrap(); - let addr = listener.by_ref().wait() - .next() + let addr = listener.next().await .expect("some event") .expect("no error") .into_new_address() @@ -53,56 +47,42 @@ fn client_to_server_outbound() { tx.send(addr).unwrap(); - let future = listener - .filter_map(ListenerEvent::into_upgrade) - .into_future() - .map_err(|(err, _)| panic!("{:?}", err)) - .and_then(|(client, _)| client.unwrap().0) - .map_err(|err| panic!("{:?}", err)) - .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) - .map(|client| Builder::new().new_read(client)) - .and_then(|client| { - client - .into_future() - .map_err(|(err, _)| err) - .map(|(msg, _)| msg) - }) - .and_then(|msg| { - let msg = msg.unwrap(); - assert_eq!(msg, "hello world"); - Ok(()) - }); - - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); + let client = listener + .next().await + .unwrap() + .unwrap() + .into_upgrade().unwrap().0.await.unwrap(); + + let mut outbound = muxing::outbound_from_ref_and_wrap(Arc::new(client)).await.unwrap(); + + let mut buf = Vec::new(); + outbound.read_to_end(&mut buf).await.unwrap(); + assert_eq!(buf, b"hello world"); }); - let mplex = libp2p_mplex::MplexConfig::new(); - let transport = TcpConfig::new().and_then(move |c, e| - upgrade::apply(c, mplex, e, upgrade::Version::V1)); - - let future = transport - .dial(rx.recv().unwrap()) - .unwrap() - .map_err(|err| panic!("{:?}", err)) - .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) - .map(|server| Builder::new().new_write(server)) - .and_then(|server| server.send("hello world".into())) - .map(|_| ()); - - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); - bg_thread.join().unwrap(); + async_std::task::block_on(async { + let mplex = libp2p_mplex::MplexConfig::new(); + let transport = TcpConfig::new().and_then(move |c, e| + upgrade::apply(c, mplex, e, upgrade::Version::V1)); + + let client = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); + let mut inbound = muxing::inbound_from_ref_and_wrap(Arc::new(client)).await.unwrap(); + inbound.write_all(b"hello world").await.unwrap(); + inbound.close().await.unwrap(); + + bg_thread.await; + }); } #[test] fn client_to_server_inbound() { // Simulate a client sending a message to a server through a multiplex upgrade. - let (tx, rx) = mpsc::channel(); + let (tx, rx) = oneshot::channel(); - let bg_thread = thread::spawn(move || { + let bg_thread = async_std::task::spawn(async move { let mplex = libp2p_mplex::MplexConfig::new(); + let transport = TcpConfig::new().and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)); @@ -110,54 +90,37 @@ fn client_to_server_inbound() { .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) .unwrap(); - let addr = listener.by_ref().wait() - .next() + let addr = listener.next().await .expect("some event") .expect("no error") .into_new_address() .expect("listen address"); - tx.send(addr).unwrap(); - let future = listener - .filter_map(ListenerEvent::into_upgrade) - .into_future() - .map_err(|(err, _)| panic!("{:?}", err)) - .and_then(|(client, _)| client.unwrap().0) - .map_err(|err| panic!("{:?}", err)) - .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) - .map(|client| Builder::new().new_read(client)) - .and_then(|client| { - client - .into_future() - .map_err(|(err, _)| err) - .map(|(msg, _)| msg) - }) - .and_then(|msg| { - let msg = msg.unwrap(); - assert_eq!(msg, "hello world"); - Ok(()) - }); - - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); + let client = listener + .next().await + .unwrap() + .unwrap() + .into_upgrade().unwrap().0.await.unwrap(); + + let mut inbound = muxing::inbound_from_ref_and_wrap(Arc::new(client)).await.unwrap(); + + let mut buf = Vec::new(); + inbound.read_to_end(&mut buf).await.unwrap(); + assert_eq!(buf, b"hello world"); }); - let mplex = libp2p_mplex::MplexConfig::new(); - let transport = TcpConfig::new().and_then(move |c, e| - upgrade::apply(c, mplex, e, upgrade::Version::V1)); - - let future = transport - .dial(rx.recv().unwrap()) - .unwrap() - .map_err(|err| panic!("{:?}", err)) - .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) - .map(|server| Builder::new().new_write(server)) - .and_then(|server| server.send("hello world".into())) - .map(|_| ()); - - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); - bg_thread.join().unwrap(); + async_std::task::block_on(async { + let mplex = libp2p_mplex::MplexConfig::new(); + let transport = TcpConfig::new().and_then(move |c, e| + upgrade::apply(c, mplex, e, upgrade::Version::V1)); + + let client = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); + let mut outbound = muxing::outbound_from_ref_and_wrap(Arc::new(client)).await.unwrap(); + outbound.write_all(b"hello world").await.unwrap(); + outbound.close().await.unwrap(); + + bg_thread.await; + }); } diff --git a/muxers/yamux/Cargo.toml b/muxers/yamux/Cargo.toml index 6410e21f55a..a536710c015 100644 --- a/muxers/yamux/Cargo.toml +++ b/muxers/yamux/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-yamux" edition = "2018" description = "Yamux multiplexing protocol for libp2p" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,8 +10,9 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -log = "0.4" -tokio-io = "0.1" -yamux = "0.2.1" +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +log = "0.4.8" +parking_lot = "0.10" +thiserror = "1.0" +yamux = "0.4" diff --git a/muxers/yamux/src/lib.rs b/muxers/yamux/src/lib.rs index dd062a6daef..bc8db663854 100644 --- a/muxers/yamux/src/lib.rs +++ b/muxers/yamux/src/lib.rs @@ -21,112 +21,160 @@ //! Implements the Yamux multiplexing protocol for libp2p, see also the //! [specification](https://github.com/hashicorp/yamux/blob/master/spec.md). -use futures::{future::{self, FutureResult}, prelude::*}; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}; -use log::debug; -use std::{io, iter, sync::atomic}; -use std::io::{Error as IoError}; -use tokio_io::{AsyncRead, AsyncWrite}; +use futures::{future, prelude::*, ready, stream::{BoxStream, LocalBoxStream}}; +use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; +use parking_lot::Mutex; +use std::{fmt, io, iter, pin::Pin, task::Context}; +use thiserror::Error; -// TODO: add documentation and field names -pub struct Yamux(yamux::Connection, atomic::AtomicBool); +/// A Yamux connection. +pub struct Yamux(Mutex>); -impl Yamux +impl fmt::Debug for Yamux { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.write_str("Yamux") + } +} + +struct Inner { + /// The `futures::stream::Stream` of incoming substreams. + incoming: S, + /// Handle to control the connection. + control: yamux::Control, + /// True, once we have received an inbound substream. + acknowledged: bool +} + +/// A token to poll for an outbound substream. +#[derive(Debug)] +pub struct OpenSubstreamToken(()); + +impl Yamux> where - C: AsyncRead + AsyncWrite + 'static + C: AsyncRead + AsyncWrite + Send + Unpin + 'static { - pub fn new(c: C, mut cfg: yamux::Config, mode: yamux::Mode) -> Self { + /// Create a new Yamux connection. + pub fn new(io: C, mut cfg: yamux::Config, mode: yamux::Mode) -> Self { cfg.set_read_after_close(false); - Yamux(yamux::Connection::new(c, cfg, mode), atomic::AtomicBool::new(false)) + let conn = yamux::Connection::new(io, cfg, mode); + let ctrl = conn.control(); + let inner = Inner { + incoming: Incoming { + stream: yamux::into_stream(conn).err_into().boxed(), + _marker: std::marker::PhantomData + }, + control: ctrl, + acknowledged: false + }; + Yamux(Mutex::new(inner)) } } -impl libp2p_core::StreamMuxer for Yamux +impl Yamux> where - C: AsyncRead + AsyncWrite + 'static + C: AsyncRead + AsyncWrite + Unpin + 'static { - type Substream = yamux::StreamHandle; - type OutboundSubstream = FutureResult, io::Error>; - type Error = IoError; - - fn poll_inbound(&self) -> Poll { - match self.0.poll() { - Err(e) => { - debug!("connection error: {}", e); - Err(io::Error::new(io::ErrorKind::Other, e)) - } - Ok(Async::NotReady) => Ok(Async::NotReady), - Ok(Async::Ready(None)) => Err(io::ErrorKind::BrokenPipe.into()), - Ok(Async::Ready(Some(stream))) => { - self.1.store(true, atomic::Ordering::Release); - Ok(Async::Ready(stream)) + /// Create a new Yamux connection (which is ![`Send`]). + pub fn local(io: C, mut cfg: yamux::Config, mode: yamux::Mode) -> Self { + cfg.set_read_after_close(false); + let conn = yamux::Connection::new(io, cfg, mode); + let ctrl = conn.control(); + let inner = Inner { + incoming: LocalIncoming { + stream: yamux::into_stream(conn).err_into().boxed_local(), + _marker: std::marker::PhantomData + }, + control: ctrl, + acknowledged: false + }; + Yamux(Mutex::new(inner)) + } +} + +type Poll = std::task::Poll>; + +impl libp2p_core::StreamMuxer for Yamux +where + S: Stream> + Unpin +{ + type Substream = yamux::Stream; + type OutboundSubstream = OpenSubstreamToken; + type Error = YamuxError; + + fn poll_inbound(&self, c: &mut Context) -> Poll { + let mut inner = self.0.lock(); + match ready!(inner.incoming.poll_next_unpin(c)) { + Some(Ok(s)) => { + inner.acknowledged = true; + Poll::Ready(Ok(s)) } + Some(Err(e)) => Poll::Ready(Err(e)), + None => Poll::Ready(Err(yamux::ConnectionError::Closed.into())) } } fn open_outbound(&self) -> Self::OutboundSubstream { - let stream = self.0.open_stream().map_err(|e| io::Error::new(io::ErrorKind::Other, e)); - future::result(stream) + OpenSubstreamToken(()) } - fn poll_outbound(&self, substream: &mut Self::OutboundSubstream) -> Poll { - match substream.poll()? { - Async::Ready(Some(s)) => Ok(Async::Ready(s)), - Async::Ready(None) => Err(io::ErrorKind::BrokenPipe.into()), - Async::NotReady => Ok(Async::NotReady), - } + fn poll_outbound(&self, c: &mut Context, _: &mut OpenSubstreamToken) -> Poll { + let mut inner = self.0.lock(); + Pin::new(&mut inner.control).poll_open_stream(c).map_err(YamuxError) } fn destroy_outbound(&self, _: Self::OutboundSubstream) { + self.0.lock().control.abort_open_stream() } - unsafe fn prepare_uninitialized_buffer(&self, _: &mut [u8]) -> bool { - false + fn read_substream(&self, c: &mut Context, s: &mut Self::Substream, b: &mut [u8]) -> Poll { + Pin::new(s).poll_read(c, b).map_err(|e| YamuxError(e.into())) } - fn read_substream(&self, sub: &mut Self::Substream, buf: &mut [u8]) -> Poll { - let result = sub.poll_read(buf); - if let Ok(Async::Ready(_)) = result { - self.1.store(true, atomic::Ordering::Release); - } - result + fn write_substream(&self, c: &mut Context, s: &mut Self::Substream, b: &[u8]) -> Poll { + Pin::new(s).poll_write(c, b).map_err(|e| YamuxError(e.into())) } - fn write_substream(&self, sub: &mut Self::Substream, buf: &[u8]) -> Poll { - sub.poll_write(buf) + fn flush_substream(&self, c: &mut Context, s: &mut Self::Substream) -> Poll<()> { + Pin::new(s).poll_flush(c).map_err(|e| YamuxError(e.into())) } - fn flush_substream(&self, sub: &mut Self::Substream) -> Poll<(), IoError> { - sub.poll_flush() + fn shutdown_substream(&self, c: &mut Context, s: &mut Self::Substream) -> Poll<()> { + Pin::new(s).poll_close(c).map_err(|e| YamuxError(e.into())) } - fn shutdown_substream(&self, sub: &mut Self::Substream) -> Poll<(), IoError> { - sub.shutdown() - } - - fn destroy_substream(&self, _: Self::Substream) { - } + fn destroy_substream(&self, _: Self::Substream) { } fn is_remote_acknowledged(&self) -> bool { - self.1.load(atomic::Ordering::Acquire) + self.0.lock().acknowledged } - fn close(&self) -> Poll<(), IoError> { - self.0.close().map_err(|e| io::Error::new(io::ErrorKind::Other, e)) + fn close(&self, c: &mut Context) -> Poll<()> { + let mut inner = self.0.lock(); + Pin::new(&mut inner.control).poll_close(c).map_err(YamuxError) } - fn flush_all(&self) -> Poll<(), IoError> { - self.0.flush().map_err(|e| io::Error::new(io::ErrorKind::Other, e)) + fn flush_all(&self, _: &mut Context) -> Poll<()> { + Poll::Ready(Ok(())) } } +/// The yamux configuration. #[derive(Clone)] pub struct Config(yamux::Config); +/// The yamux configuration for upgrading I/O resources which are ![`Send`]. +#[derive(Clone)] +pub struct LocalConfig(Config); + impl Config { pub fn new(cfg: yamux::Config) -> Self { Config(cfg) } + + /// Turn this into a `LocalConfig` for use with upgrades of !Send resources. + pub fn local(self) -> LocalConfig { + LocalConfig(self) + } } impl Default for Config { @@ -144,29 +192,128 @@ impl UpgradeInfo for Config { } } +impl UpgradeInfo for LocalConfig { + type Info = &'static [u8]; + type InfoIter = iter::Once; + + fn protocol_info(&self) -> Self::InfoIter { + iter::once(b"/yamux/1.0.0") + } +} + impl InboundUpgrade for Config where - C: AsyncRead + AsyncWrite + 'static, + C: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + type Output = Yamux>; + type Error = io::Error; + type Future = future::Ready>; + + fn upgrade_inbound(self, io: C, _: Self::Info) -> Self::Future { + future::ready(Ok(Yamux::new(io, self.0, yamux::Mode::Server))) + } +} + +impl InboundUpgrade for LocalConfig +where + C: AsyncRead + AsyncWrite + Unpin + 'static { - type Output = Yamux>; + type Output = Yamux>; type Error = io::Error; - type Future = FutureResult>, io::Error>; + type Future = future::Ready>; - fn upgrade_inbound(self, i: Negotiated, _: Self::Info) -> Self::Future { - future::ok(Yamux::new(i, self.0, yamux::Mode::Server)) + fn upgrade_inbound(self, io: C, _: Self::Info) -> Self::Future { + future::ready(Ok(Yamux::local(io, (self.0).0, yamux::Mode::Server))) } } impl OutboundUpgrade for Config where - C: AsyncRead + AsyncWrite + 'static, + C: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + type Output = Yamux>; + type Error = io::Error; + type Future = future::Ready>; + + fn upgrade_outbound(self, io: C, _: Self::Info) -> Self::Future { + future::ready(Ok(Yamux::new(io, self.0, yamux::Mode::Client))) + } +} + +impl OutboundUpgrade for LocalConfig +where + C: AsyncRead + AsyncWrite + Unpin + 'static { - type Output = Yamux>; + type Output = Yamux>; type Error = io::Error; - type Future = FutureResult>, io::Error>; + type Future = future::Ready>; + + fn upgrade_outbound(self, io: C, _: Self::Info) -> Self::Future { + future::ready(Ok(Yamux::local(io, (self.0).0, yamux::Mode::Client))) + } +} + +/// The Yamux [`StreamMuxer`] error type. +#[derive(Debug, Error)] +#[error("yamux error: {0}")] +pub struct YamuxError(#[from] pub yamux::ConnectionError); + +impl Into for YamuxError { + fn into(self: YamuxError) -> io::Error { + io::Error::new(io::ErrorKind::Other, self.to_string()) + } +} + +/// The [`futures::stream::Stream`] of incoming substreams. +pub struct Incoming { + stream: BoxStream<'static, Result>, + _marker: std::marker::PhantomData +} + +impl fmt::Debug for Incoming { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.write_str("Incoming") + } +} + +/// The [`futures::stream::Stream`] of incoming substreams (`!Send`). +pub struct LocalIncoming { + stream: LocalBoxStream<'static, Result>, + _marker: std::marker::PhantomData +} + +impl fmt::Debug for LocalIncoming { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.write_str("LocalIncoming") + } +} + +impl Stream for Incoming { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> std::task::Poll> { + self.stream.as_mut().poll_next_unpin(cx) + } + + fn size_hint(&self) -> (usize, Option) { + self.stream.size_hint() + } +} + +impl Unpin for Incoming { +} + +impl Stream for LocalIncoming { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> std::task::Poll> { + self.stream.as_mut().poll_next_unpin(cx) + } - fn upgrade_outbound(self, i: Negotiated, _: Self::Info) -> Self::Future { - future::ok(Yamux::new(i, self.0, yamux::Mode::Client)) + fn size_hint(&self) -> (usize, Option) { + self.stream.size_hint() } } +impl Unpin for LocalIncoming { +} \ No newline at end of file diff --git a/protocols/deflate/Cargo.toml b/protocols/deflate/Cargo.toml index a4f7afd7e85..257c4263012 100644 --- a/protocols/deflate/Cargo.toml +++ b/protocols/deflate/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-deflate" edition = "2018" description = "Deflate encryption protocol for libp2p" -version = "0.5.0" +version = "0.6.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,14 +10,12 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -tokio-io = "0.1.12" -flate2 = { version = "1.0", features = ["tokio"] } +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +flate2 = "1.0" [dev-dependencies] -env_logger = "0.7.1" -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } -quickcheck = "0.9.0" -tokio = "0.1" -log = "0.4" +async-std = "1.0" +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../../transports/tcp" } +rand = "0.7" +quickcheck = "0.9" diff --git a/protocols/deflate/src/lib.rs b/protocols/deflate/src/lib.rs index 7dbf03ebf6c..32a82f247d2 100644 --- a/protocols/deflate/src/lib.rs +++ b/protocols/deflate/src/lib.rs @@ -18,21 +18,22 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use flate2::read::DeflateDecoder; -use flate2::write::DeflateEncoder; -use flate2::Compression; -use std::io; - -use futures::future::{self, FutureResult}; -use libp2p_core::{upgrade::Negotiated, InboundUpgrade, OutboundUpgrade, UpgradeInfo}; -use std::iter; -use tokio_io::{AsyncRead, AsyncWrite}; +use futures::{prelude::*, ready}; +use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; +use std::{io, iter, pin::Pin, task::Context, task::Poll}; #[derive(Debug, Copy, Clone)] -pub struct DeflateConfig; +pub struct DeflateConfig { + compression: flate2::Compression, +} -/// Output of the deflate protocol. -pub type DeflateOutput = DeflateDecoder>; +impl Default for DeflateConfig { + fn default() -> Self { + DeflateConfig { + compression: flate2::Compression::fast(), + } + } +} impl UpgradeInfo for DeflateConfig { type Info = &'static [u8]; @@ -47,15 +48,12 @@ impl InboundUpgrade for DeflateConfig where C: AsyncRead + AsyncWrite, { - type Output = DeflateOutput>; + type Output = DeflateOutput; type Error = io::Error; - type Future = FutureResult; + type Future = future::Ready>; - fn upgrade_inbound(self, r: Negotiated, _: Self::Info) -> Self::Future { - future::ok(DeflateDecoder::new(DeflateEncoder::new( - r, - Compression::default(), - ))) + fn upgrade_inbound(self, r: C, _: Self::Info) -> Self::Future { + future::ok(DeflateOutput::new(r, self.compression)) } } @@ -63,14 +61,193 @@ impl OutboundUpgrade for DeflateConfig where C: AsyncRead + AsyncWrite, { - type Output = DeflateOutput>; + type Output = DeflateOutput; type Error = io::Error; - type Future = FutureResult; + type Future = future::Ready>; + + fn upgrade_outbound(self, w: C, _: Self::Info) -> Self::Future { + future::ok(DeflateOutput::new(w, self.compression)) + } +} + +/// Decodes and encodes traffic using DEFLATE. +#[derive(Debug)] +pub struct DeflateOutput { + /// Inner stream where we read compressed data from and write compressed data to. + inner: S, + /// Internal object used to hold the state of the compression. + compress: flate2::Compress, + /// Internal object used to hold the state of the decompression. + decompress: flate2::Decompress, + /// Temporary buffer between `compress` and `inner`. Stores compressed bytes that need to be + /// sent out once `inner` is ready to accept more. + write_out: Vec, + /// Temporary buffer between `decompress` and `inner`. Stores compressed bytes that need to be + /// given to `decompress`. + read_interm: Vec, + /// When we read from `inner` and `Ok(0)` is returned, we set this to `true` so that we don't + /// read from it again. + inner_read_eof: bool, +} + +impl DeflateOutput { + fn new(inner: S, compression: flate2::Compression) -> Self { + DeflateOutput { + inner, + compress: flate2::Compress::new(compression, false), + decompress: flate2::Decompress::new(false), + write_out: Vec::with_capacity(256), + read_interm: Vec::with_capacity(256), + inner_read_eof: false, + } + } + + /// Tries to write the content of `self.write_out` to `self.inner`. + /// Returns `Ready(Ok(()))` if `self.write_out` is empty. + fn flush_write_out(&mut self, cx: &mut Context) -> Poll> + where S: AsyncWrite + Unpin + { + loop { + if self.write_out.is_empty() { + return Poll::Ready(Ok(())) + } + + match AsyncWrite::poll_write(Pin::new(&mut self.inner), cx, &self.write_out) { + Poll::Ready(Ok(0)) => return Poll::Ready(Err(io::ErrorKind::WriteZero.into())), + Poll::Ready(Ok(n)) => self.write_out = self.write_out.split_off(n), + Poll::Ready(Err(err)) => return Poll::Ready(Err(err)), + Poll::Pending => return Poll::Pending, + }; + } + } +} + +impl AsyncRead for DeflateOutput + where S: AsyncRead + Unpin +{ + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + // We use a `this` variable because the compiler doesn't allow multiple mutable borrows + // across a `Deref`. + let this = &mut *self; + + loop { + // Read from `self.inner` into `self.read_interm` if necessary. + if this.read_interm.is_empty() && !this.inner_read_eof { + unsafe { + this.read_interm.reserve(256); + this.read_interm.set_len(this.read_interm.capacity()); + } + + match AsyncRead::poll_read(Pin::new(&mut this.inner), cx, &mut this.read_interm) { + Poll::Ready(Ok(0)) => { + this.inner_read_eof = true; + this.read_interm.clear(); + } + Poll::Ready(Ok(n)) => { + this.read_interm.truncate(n) + }, + Poll::Ready(Err(err)) => { + this.read_interm.clear(); + return Poll::Ready(Err(err)) + }, + Poll::Pending => { + this.read_interm.clear(); + return Poll::Pending + }, + } + } + debug_assert!(!this.read_interm.is_empty() || this.inner_read_eof); + + let before_out = this.decompress.total_out(); + let before_in = this.decompress.total_in(); + let ret = this.decompress.decompress(&this.read_interm, buf, if this.inner_read_eof { flate2::FlushDecompress::Finish } else { flate2::FlushDecompress::None })?; + + // Remove from `self.read_interm` the bytes consumed by the decompressor. + let consumed = (this.decompress.total_in() - before_in) as usize; + this.read_interm = this.read_interm.split_off(consumed); + + let read = (this.decompress.total_out() - before_out) as usize; + if read != 0 || ret == flate2::Status::StreamEnd { + return Poll::Ready(Ok(read)) + } + } + } +} + +impl AsyncWrite for DeflateOutput + where S: AsyncWrite + Unpin +{ + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) + -> Poll> + { + // We use a `this` variable because the compiler doesn't allow multiple mutable borrows + // across a `Deref`. + let this = &mut *self; + + // We don't want to accumulate too much data in `self.write_out`, so we only proceed if it + // is empty. + ready!(this.flush_write_out(cx))?; + + // We special-case this, otherwise an empty buffer would make the loop below infinite. + if buf.is_empty() { + return Poll::Ready(Ok(0)); + } + + // Unfortunately, the compressor might be in a "flushing mode", not accepting any input + // data. We don't want to return `Ok(0)` in that situation, as that would be wrong. + // Instead, we invoke the compressor in a loop until it accepts some of our data. + loop { + let before_in = this.compress.total_in(); + this.write_out.reserve(256); // compress_vec uses the Vec's capacity + let ret = this.compress.compress_vec(buf, &mut this.write_out, flate2::FlushCompress::None)?; + let written = (this.compress.total_in() - before_in) as usize; + + if written != 0 || ret == flate2::Status::StreamEnd { + return Poll::Ready(Ok(written)); + } + } + } + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + // We use a `this` variable because the compiler doesn't allow multiple mutable borrows + // across a `Deref`. + let this = &mut *self; + + ready!(this.flush_write_out(cx))?; + this.compress.compress_vec(&[], &mut this.write_out, flate2::FlushCompress::Sync)?; + + loop { + ready!(this.flush_write_out(cx))?; + + debug_assert!(this.write_out.is_empty()); + // We ask the compressor to flush everything into `self.write_out`. + this.write_out.reserve(256); // compress_vec uses the Vec's capacity + this.compress.compress_vec(&[], &mut this.write_out, flate2::FlushCompress::None)?; + if this.write_out.is_empty() { + break; + } + } + + AsyncWrite::poll_flush(Pin::new(&mut this.inner), cx) + } + + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + // We use a `this` variable because the compiler doesn't allow multiple mutable borrows + // across a `Deref`. + let this = &mut *self; + + loop { + ready!(this.flush_write_out(cx))?; + + // We ask the compressor to flush everything into `self.write_out`. + debug_assert!(this.write_out.is_empty()); + this.write_out.reserve(256); // compress_vec uses the Vec's capacity + this.compress.compress_vec(&[], &mut this.write_out, flate2::FlushCompress::Finish)?; + if this.write_out.is_empty() { + break; + } + } - fn upgrade_outbound(self, w: Negotiated, _: Self::Info) -> Self::Future { - future::ok(DeflateDecoder::new(DeflateEncoder::new( - w, - Compression::default(), - ))) + AsyncWrite::poll_close(Pin::new(&mut this.inner), cx) } } diff --git a/protocols/deflate/tests/test.rs b/protocols/deflate/tests/test.rs index dd714836b94..896fb491349 100644 --- a/protocols/deflate/tests/test.rs +++ b/protocols/deflate/tests/test.rs @@ -18,82 +18,77 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use futures::prelude::*; -use libp2p_core::transport::{ListenerEvent, Transport}; -use libp2p_core::upgrade::{self, Negotiated}; -use libp2p_deflate::{DeflateConfig, DeflateOutput}; -use libp2p_tcp::{TcpConfig, TcpTransStream}; -use log::info; -use quickcheck::QuickCheck; -use tokio::{self, io}; +use futures::{future, prelude::*}; +use libp2p_core::{transport::Transport, upgrade}; +use libp2p_deflate::DeflateConfig; +use libp2p_tcp::TcpConfig; +use quickcheck::{QuickCheck, RngCore, TestResult}; #[test] fn deflate() { - let _ = env_logger::try_init(); - - fn prop(message: Vec) -> bool { - let client = TcpConfig::new().and_then(|c, e| - upgrade::apply(c, DeflateConfig {}, e, upgrade::Version::V1)); - let server = client.clone(); - run(server, client, message); - true + fn prop(message: Vec) -> TestResult { + if message.is_empty() { + return TestResult::discard() + } + async_std::task::block_on(run(message)); + TestResult::passed() } + QuickCheck::new().quickcheck(prop as fn(Vec) -> TestResult) +} - QuickCheck::new() - .max_tests(30) - .quickcheck(prop as fn(Vec) -> bool) +#[test] +fn lot_of_data() { + let mut v = vec![0; 2 * 1024 * 1024]; + rand::thread_rng().fill_bytes(&mut v); + async_std::task::block_on(run(v)) } -type Output = DeflateOutput>; +async fn run(message1: Vec) { + let transport = TcpConfig::new() + .and_then(|conn, endpoint| { + upgrade::apply(conn, DeflateConfig::default(), endpoint, upgrade::Version::V1) + }); -fn run(server_transport: T, client_transport: T, message1: Vec) -where - T: Transport, - T::Dial: Send + 'static, - T::Listener: Send + 'static, - T::ListenerUpgrade: Send + 'static, -{ - let message2 = message1.clone(); + let mut listener = transport.clone() + .listen_on("/ip4/0.0.0.0/tcp/0".parse().expect("multiaddr")) + .expect("listener"); - let mut server = server_transport - .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) - .unwrap(); - let server_address = server - .by_ref() - .wait() - .next() + let listen_addr = listener.by_ref().next().await .expect("some event") .expect("no error") .into_new_address() - .expect("listen address"); - let server = server - .take(1) - .filter_map(ListenerEvent::into_upgrade) - .and_then(|(client, _)| client) - .map_err(|e| panic!("server error: {}", e)) - .and_then(|client| { - info!("server: reading message"); - io::read_to_end(client, Vec::new()) - }) - .for_each(move |(_, msg)| { - info!("server: read message: {:?}", msg); - assert_eq!(msg, message1); - Ok(()) - }); + .expect("new address"); + + let message2 = message1.clone(); + + let listener_task = async_std::task::spawn(async move { + let mut conn = listener + .filter(|e| future::ready(e.as_ref().map(|e| e.is_upgrade()).unwrap_or(false))) + .next() + .await + .expect("some event") + .expect("no error") + .into_upgrade() + .expect("upgrade") + .0 + .await + .expect("connection"); + + let mut buf = vec![0; message2.len()]; + conn.read_exact(&mut buf).await.expect("read_exact"); + assert_eq!(&buf[..], &message2[..]); + + conn.write_all(&message2).await.expect("write_all"); + conn.close().await.expect("close") + }); - let client = client_transport - .dial(server_address.clone()) - .unwrap() - .map_err(|e| panic!("client error: {}", e)) - .and_then(move |server| { - io::write_all(server, message2).and_then(|(client, _)| io::shutdown(client)) - }) - .map(|_| ()); + let mut conn = transport.dial(listen_addr).expect("dialer").await.expect("connection"); + conn.write_all(&message1).await.expect("write_all"); + conn.close().await.expect("close"); - let future = client - .join(server) - .map_err(|e| panic!("{:?}", e)) - .map(|_| ()); + let mut buf = Vec::new(); + conn.read_to_end(&mut buf).await.expect("read_to_end"); + assert_eq!(&buf[..], &message1[..]); - tokio::run(future) + listener_task.await } diff --git a/protocols/floodsub/Cargo.toml b/protocols/floodsub/Cargo.toml index 8e847f6619f..15b66877deb 100644 --- a/protocols/floodsub/Cargo.toml +++ b/protocols/floodsub/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-floodsub" edition = "2018" description = "Floodsub protocol for libp2p" -version = "0.13.1" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -11,13 +11,11 @@ categories = ["network-programming", "asynchronous"] [dependencies] bs58 = "0.3.0" -bytes = "0.4" cuckoofilter = "0.3.2" fnv = "1.0" -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -libp2p-swarm = { version = "0.3.0", path = "../../swarm" } +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 -rand = "0.6" -smallvec = "0.6.5" -tokio-io = "0.1" +rand = "0.7" +smallvec = "1.0" diff --git a/protocols/floodsub/src/layer.rs b/protocols/floodsub/src/layer.rs index ba46dfdf94f..929ce6806ec 100644 --- a/protocols/floodsub/src/layer.rs +++ b/protocols/floodsub/src/layer.rs @@ -35,7 +35,7 @@ use rand; use smallvec::SmallVec; use std::{collections::VecDeque, iter, marker::PhantomData}; use std::collections::hash_map::{DefaultHasher, HashMap}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::task::{Context, Poll}; /// Network behaviour that automatically identifies nodes periodically, and returns information /// about them. @@ -230,7 +230,7 @@ impl Floodsub { impl NetworkBehaviour for Floodsub where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Send + Unpin + 'static, { type ProtocolsHandler = OneShotHandler; type OutEvent = FloodsubEvent; @@ -359,18 +359,19 @@ where fn poll( &mut self, + _: &mut Context, _: &mut impl PollParameters, - ) -> Async< + ) -> Poll< NetworkBehaviourAction< ::InEvent, Self::OutEvent, >, > { if let Some(event) = self.events.pop_front() { - return Async::Ready(event); + return Poll::Ready(event); } - Async::NotReady + Poll::Pending } } diff --git a/protocols/floodsub/src/protocol.rs b/protocols/floodsub/src/protocol.rs index e6951321dc5..6a9ac91b144 100644 --- a/protocols/floodsub/src/protocol.rs +++ b/protocols/floodsub/src/protocol.rs @@ -20,10 +20,10 @@ use crate::rpc_proto; use crate::topic::TopicHash; +use futures::prelude::*; use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, PeerId, upgrade}; use protobuf::{ProtobufError, Message as ProtobufMessage}; -use std::{error, fmt, io, iter}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{error, fmt, io, iter, pin::Pin}; /// Implementation of `ConnectionUpgrade` for the floodsub protocol. #[derive(Debug, Clone, Default)] @@ -49,15 +49,15 @@ impl UpgradeInfo for FloodsubConfig { impl InboundUpgrade for FloodsubConfig where - TSocket: AsyncRead + AsyncWrite, + TSocket: AsyncRead + AsyncWrite + Send + Unpin + 'static, { type Output = FloodsubRpc; type Error = FloodsubDecodeError; - type Future = upgrade::ReadOneThen, (), fn(Vec, ()) -> Result>; + type Future = Pin> + Send>>; - #[inline] - fn upgrade_inbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { - upgrade::read_one_then(socket, 2048, (), |packet, ()| { + fn upgrade_inbound(self, mut socket: TSocket, _: Self::Info) -> Self::Future { + Box::pin(async move { + let packet = upgrade::read_one(&mut socket, 2048).await?; let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; let mut messages = Vec::with_capacity(rpc.get_publish().len()); @@ -164,16 +164,19 @@ impl UpgradeInfo for FloodsubRpc { impl OutboundUpgrade for FloodsubRpc where - TSocket: AsyncWrite + AsyncRead, + TSocket: AsyncWrite + AsyncRead + Send + Unpin + 'static, { type Output = (); type Error = io::Error; - type Future = upgrade::WriteOne>; + type Future = Pin> + Send>>; #[inline] - fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { - let bytes = self.into_bytes(); - upgrade::write_one(socket, bytes) + fn upgrade_outbound(self, mut socket: TSocket, _: Self::Info) -> Self::Future { + Box::pin(async move { + let bytes = self.into_bytes(); + upgrade::write_one(&mut socket, bytes).await?; + Ok(()) + }) } } diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 0a1a417f47f..b483254106c 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "libp2p-gossipsub" edition = "2018" -version = "0.1.0" +version = "0.14.0-alpha.1" authors = ["Age Manning "] license = "MIT" @@ -28,8 +28,8 @@ smallvec = "1.0.0" [dev-dependencies] env_logger = "0.6.0" libp2p = { path = "../../" } -libp2p-plaintext = { version = "0.13.0", path = "../plaintext" } -libp2p-yamux = { version = "0.13.0", path = "../../muxers/yamux" } +libp2p-plaintext = { path = "../plaintext" } +libp2p-yamux = { path = "../../muxers/yamux" } quickcheck = "0.9.0" rand = "0.7.2" tokio = "0.1" diff --git a/protocols/identify/Cargo.toml b/protocols/identify/Cargo.toml index 2c433592805..2594c8a6c92 100644 --- a/protocols/identify/Cargo.toml +++ b/protocols/identify/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-identify" edition = "2018" description = "Nodes identifcation protocol for libp2p" -version = "0.13.2" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,22 +10,16 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4" -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -libp2p-swarm = { version = "0.3.0", path = "../../swarm" } +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } log = "0.4.1" -multiaddr = { package = "parity-multiaddr", version = "0.6.0", path = "../../misc/multiaddr" } protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 -smallvec = "0.6" -tokio-codec = "0.1" -tokio-io = "0.1.0" -wasm-timer = "0.1" -unsigned-varint = { version = "0.2.1", features = ["codec"] } +smallvec = "1.0" +wasm-timer = "0.2" [dev-dependencies] -libp2p-mplex = { version = "0.13.0", path = "../../muxers/mplex" } -libp2p-secio = { version = "0.13.0", path = "../../protocols/secio" } -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } -rand = "0.6" -tokio = "0.1" +async-std = "1.0" +libp2p-mplex = { version = "0.14.0-alpha.1", path = "../../muxers/mplex" } +libp2p-secio = { version = "0.14.0-alpha.1", path = "../../protocols/secio" } +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../../transports/tcp" } diff --git a/protocols/identify/src/handler.rs b/protocols/identify/src/handler.rs index 54664ae70c0..5afff5c4361 100644 --- a/protocols/identify/src/handler.rs +++ b/protocols/identify/src/handler.rs @@ -23,6 +23,7 @@ use futures::prelude::*; use libp2p_core::upgrade::{ InboundUpgrade, OutboundUpgrade, + ReadOneError, Negotiated }; use libp2p_swarm::{ @@ -33,9 +34,8 @@ use libp2p_swarm::{ ProtocolsHandlerUpgrErr }; use smallvec::SmallVec; -use std::{io, marker::PhantomData, time::Duration}; -use tokio_io::{AsyncRead, AsyncWrite}; -use wasm_timer::{Delay, Instant}; +use std::{marker::PhantomData, pin::Pin, task::Context, task::Poll, time::Duration}; +use wasm_timer::Delay; /// Delay between the moment we connect and the first time we identify. const DELAY_TO_FIRST_ID: Duration = Duration::from_millis(500); @@ -74,7 +74,7 @@ pub enum IdentifyHandlerEvent { /// We received a request for identification. Identify(ReplySubstream>), /// Failed to identify the remote. - IdentificationError(ProtocolsHandlerUpgrErr), + IdentificationError(ProtocolsHandlerUpgrErr), } impl IdentifyHandler { @@ -83,7 +83,7 @@ impl IdentifyHandler { IdentifyHandler { config: IdentifyProtocolConfig, events: SmallVec::new(), - next_id: Delay::new(Instant::now() + DELAY_TO_FIRST_ID), + next_id: Delay::new(DELAY_TO_FIRST_ID), keep_alive: KeepAlive::Yes, marker: PhantomData, } @@ -92,11 +92,11 @@ impl IdentifyHandler { impl ProtocolsHandler for IdentifyHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static, { type InEvent = (); type OutEvent = IdentifyHandlerEvent; - type Error = wasm_timer::Error; + type Error = ReadOneError; type Substream = TSubstream; type InboundProtocol = IdentifyProtocolConfig; type OutboundProtocol = IdentifyProtocolConfig; @@ -108,14 +108,14 @@ where fn inject_fully_negotiated_inbound( &mut self, - protocol: >::Output + protocol: >>::Output ) { self.events.push(IdentifyHandlerEvent::Identify(protocol)) } fn inject_fully_negotiated_outbound( &mut self, - protocol: >::Output, + protocol: >>::Output, _info: Self::OutboundOpenInfo, ) { self.events.push(IdentifyHandlerEvent::Identified(protocol)); @@ -133,38 +133,39 @@ where ) { self.events.push(IdentifyHandlerEvent::IdentificationError(err)); self.keep_alive = KeepAlive::No; - self.next_id.reset(Instant::now() + TRY_AGAIN_ON_ERR); + self.next_id.reset(TRY_AGAIN_ON_ERR); } fn connection_keep_alive(&self) -> KeepAlive { self.keep_alive } - fn poll(&mut self) -> Poll< + fn poll(&mut self, cx: &mut Context) -> Poll< ProtocolsHandlerEvent< Self::OutboundProtocol, Self::OutboundOpenInfo, IdentifyHandlerEvent, + Self::Error, >, - Self::Error, > { if !self.events.is_empty() { - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom( + return Poll::Ready(ProtocolsHandlerEvent::Custom( self.events.remove(0), - ))); + )); } // Poll the future that fires when we need to identify the node again. - match self.next_id.poll()? { - Async::NotReady => Ok(Async::NotReady), - Async::Ready(()) => { - self.next_id.reset(Instant::now() + DELAY_TO_NEXT_ID); + match Future::poll(Pin::new(&mut self.next_id), cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(Ok(())) => { + self.next_id.reset(DELAY_TO_NEXT_ID); let ev = ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(self.config.clone()), info: (), }; - Ok(Async::Ready(ev)) + Poll::Ready(ev) } + Poll::Ready(Err(err)) => Poll::Ready(ProtocolsHandlerEvent::Close(err.into())) } } } diff --git a/protocols/identify/src/identify.rs b/protocols/identify/src/identify.rs index 90666250561..361600cc5dc 100644 --- a/protocols/identify/src/identify.rs +++ b/protocols/identify/src/identify.rs @@ -19,14 +19,14 @@ // DEALINGS IN THE SOFTWARE. use crate::handler::{IdentifyHandler, IdentifyHandlerEvent}; -use crate::protocol::{IdentifyInfo, ReplySubstream, ReplyFuture}; +use crate::protocol::{IdentifyInfo, ReplySubstream}; use futures::prelude::*; use libp2p_core::{ ConnectedPoint, Multiaddr, PeerId, PublicKey, - upgrade::{Negotiated, UpgradeError} + upgrade::{Negotiated, ReadOneError, UpgradeError} }; use libp2p_swarm::{ NetworkBehaviour, @@ -35,8 +35,7 @@ use libp2p_swarm::{ ProtocolsHandler, ProtocolsHandlerUpgrErr }; -use std::{collections::HashMap, collections::VecDeque, io}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{collections::HashMap, collections::VecDeque, io, pin::Pin, task::Context, task::Poll}; /// Network behaviour that automatically identifies nodes periodically, returns information /// about them, and answers identify queries from other nodes. @@ -66,7 +65,7 @@ enum Reply { /// The reply is being sent. Sending { peer: PeerId, - io: ReplyFuture> + io: Pin> + Send>>, } } @@ -86,7 +85,7 @@ impl Identify { impl NetworkBehaviour for Identify where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static, { type ProtocolsHandler = IdentifyHandler; type OutEvent = IdentifyEvent; @@ -153,15 +152,16 @@ where fn poll( &mut self, + cx: &mut Context, params: &mut impl PollParameters, - ) -> Async< + ) -> Poll< NetworkBehaviourAction< ::InEvent, Self::OutEvent, >, > { if let Some(event) = self.events.pop_front() { - return Async::Ready(event); + return Poll::Ready(event); } if let Some(r) = self.pending_replies.pop_front() { @@ -188,17 +188,17 @@ where listen_addrs: listen_addrs.clone(), protocols: protocols.clone(), }; - let io = io.send(info, &observed); + let io = Box::pin(io.send(info, &observed)); reply = Some(Reply::Sending { peer, io }); } Some(Reply::Sending { peer, mut io }) => { sending += 1; - match io.poll() { - Ok(Async::Ready(())) => { + match Future::poll(Pin::new(&mut io), cx) { + Poll::Ready(Ok(())) => { let event = IdentifyEvent::Sent { peer_id: peer }; - return Async::Ready(NetworkBehaviourAction::GenerateEvent(event)); + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)); }, - Ok(Async::NotReady) => { + Poll::Pending => { self.pending_replies.push_back(Reply::Sending { peer, io }); if sending == to_send { // All remaining futures are NotReady @@ -207,12 +207,12 @@ where reply = self.pending_replies.pop_front(); } } - Err(err) => { + Poll::Ready(Err(err)) => { let event = IdentifyEvent::Error { peer_id: peer, - error: ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(err)) + error: ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(err.into())) }; - return Async::Ready(NetworkBehaviourAction::GenerateEvent(event)); + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)); }, } } @@ -221,7 +221,7 @@ where } } - Async::NotReady + Poll::Pending } } @@ -247,29 +247,26 @@ pub enum IdentifyEvent { /// The peer with whom the error originated. peer_id: PeerId, /// The error that occurred. - error: ProtocolsHandlerUpgrErr, + error: ProtocolsHandlerUpgrErr, }, } #[cfg(test)] mod tests { use crate::{Identify, IdentifyEvent}; - use futures::{future, prelude::*}; + use futures::{prelude::*, pin_mut}; use libp2p_core::{ identity, PeerId, muxing::StreamMuxer, - Multiaddr, Transport, upgrade }; use libp2p_tcp::TcpConfig; use libp2p_secio::SecioConfig; - use libp2p_swarm::Swarm; + use libp2p_swarm::{Swarm, SwarmEvent}; use libp2p_mplex::MplexConfig; - use rand::{Rng, thread_rng}; use std::{fmt, io}; - use tokio::runtime::current_thread; fn transport() -> (identity::PublicKey, impl Transport< Output = (PeerId, impl StreamMuxer>), @@ -304,52 +301,51 @@ mod tests { (swarm, pubkey) }; - let addr: Multiaddr = { - let port = thread_rng().gen_range(49152, std::u16::MAX); - format!("/ip4/127.0.0.1/tcp/{}", port).parse().unwrap() - }; + Swarm::listen_on(&mut swarm1, "/ip4/127.0.0.1/tcp/0".parse().unwrap()).unwrap(); - Swarm::listen_on(&mut swarm1, addr.clone()).unwrap(); - Swarm::dial_addr(&mut swarm2, addr.clone()).unwrap(); + let listen_addr = async_std::task::block_on(async { + loop { + let swarm1_fut = swarm1.next_event(); + pin_mut!(swarm1_fut); + match swarm1_fut.await { + SwarmEvent::NewListenAddr(addr) => return addr, + _ => {} + } + } + }); + Swarm::dial_addr(&mut swarm2, listen_addr).unwrap(); // nb. Either swarm may receive the `Identified` event first, upon which // it will permit the connection to be closed, as defined by // `IdentifyHandler::connection_keep_alive`. Hence the test succeeds if // either `Identified` event arrives correctly. - current_thread::Runtime::new().unwrap().block_on( - future::poll_fn(move || -> Result<_, io::Error> { - loop { - match swarm1.poll().unwrap() { - Async::Ready(Some(IdentifyEvent::Received { info, .. })) => { - assert_eq!(info.public_key, pubkey2); - assert_eq!(info.protocol_version, "c"); - assert_eq!(info.agent_version, "d"); - assert!(!info.protocols.is_empty()); - assert!(info.listen_addrs.is_empty()); - return Ok(Async::Ready(())) - }, - Async::Ready(Some(IdentifyEvent::Sent { .. })) => (), - Async::Ready(e) => panic!("{:?}", e), - Async::NotReady => {} - } + async_std::task::block_on(async move { + loop { + let swarm1_fut = swarm1.next(); + pin_mut!(swarm1_fut); + let swarm2_fut = swarm2.next(); + pin_mut!(swarm2_fut); - match swarm2.poll().unwrap() { - Async::Ready(Some(IdentifyEvent::Received { info, .. })) => { - assert_eq!(info.public_key, pubkey1); - assert_eq!(info.protocol_version, "a"); - assert_eq!(info.agent_version, "b"); - assert!(!info.protocols.is_empty()); - assert_eq!(info.listen_addrs.len(), 1); - return Ok(Async::Ready(())) - }, - Async::Ready(Some(IdentifyEvent::Sent { .. })) => (), - Async::Ready(e) => panic!("{:?}", e), - Async::NotReady => break + match future::select(swarm1_fut, swarm2_fut).await.factor_second().0 { + future::Either::Left(IdentifyEvent::Received { info, .. }) => { + assert_eq!(info.public_key, pubkey2); + assert_eq!(info.protocol_version, "c"); + assert_eq!(info.agent_version, "d"); + assert!(!info.protocols.is_empty()); + assert!(info.listen_addrs.is_empty()); + return; + } + future::Either::Right(IdentifyEvent::Received { info, .. }) => { + assert_eq!(info.public_key, pubkey1); + assert_eq!(info.protocol_version, "a"); + assert_eq!(info.agent_version, "b"); + assert!(!info.protocols.is_empty()); + assert_eq!(info.listen_addrs.len(), 1); + return; } + _ => {} } - - Ok(Async::NotReady) - })) - .unwrap(); + } + }) } } diff --git a/protocols/identify/src/protocol.rs b/protocols/identify/src/protocol.rs index adee47a5f04..4edb4a827c0 100644 --- a/protocols/identify/src/protocol.rs +++ b/protocols/identify/src/protocol.rs @@ -18,25 +18,19 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use bytes::BytesMut; use crate::structs_proto; -use futures::{future::{self, FutureResult}, Async, AsyncSink, Future, Poll, Sink, Stream}; -use futures::try_ready; +use futures::prelude::*; use libp2p_core::{ Multiaddr, PublicKey, - upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated} + upgrade::{self, InboundUpgrade, OutboundUpgrade, UpgradeInfo} }; use log::{debug, trace}; use protobuf::Message as ProtobufMessage; use protobuf::parse_from_bytes as protobuf_parse_from_bytes; use protobuf::RepeatedField; use std::convert::TryFrom; -use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use std::{fmt, iter}; -use tokio_codec::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; -use unsigned_varint::codec; +use std::{fmt, io, iter, pin::Pin}; /// Configuration for an upgrade to the `Identify` protocol. #[derive(Debug, Clone)] @@ -54,7 +48,7 @@ pub struct RemoteInfo { /// The substream on which a reply is expected to be sent. pub struct ReplySubstream { - inner: Framed>>, + inner: T, } impl fmt::Debug for ReplySubstream { @@ -65,13 +59,15 @@ impl fmt::Debug for ReplySubstream { impl ReplySubstream where - T: AsyncWrite + T: AsyncWrite + Unpin { /// Sends back the requested information on the substream. /// /// Consumes the substream, returning a `ReplyFuture` that resolves /// when the reply has been sent on the underlying connection. - pub fn send(self, info: IdentifyInfo, observed_addr: &Multiaddr) -> ReplyFuture { + pub fn send(mut self, info: IdentifyInfo, observed_addr: &Multiaddr) + -> impl Future> + { debug!("Sending identify info to client"); trace!("Sending: {:?}", info); @@ -90,50 +86,15 @@ where message.set_observedAddr(observed_addr.to_vec()); message.set_protocols(RepeatedField::from_vec(info.protocols)); - let bytes = message - .write_to_bytes() - .expect("writing protobuf failed; should never happen"); - - ReplyFuture { - inner: self.inner, - item: Some(bytes), + async move { + let bytes = message + .write_to_bytes() + .expect("writing protobuf failed; should never happen"); + upgrade::write_one(&mut self.inner, &bytes).await } } } -/// Future returned by `IdentifySender::send()`. Must be processed to the end in order to send -/// the information to the remote. -// Note: we don't use a `futures::sink::Sink` because it requires `T` to implement `Sink`, which -// means that we would require `T: AsyncWrite` in this struct definition. This requirement -// would then propagate everywhere. -#[must_use = "futures do nothing unless polled"] -pub struct ReplyFuture { - /// The Sink where to send the data. - inner: Framed>>, - /// Bytes to send, or `None` if we've already sent them. - item: Option>, -} - -impl Future for ReplyFuture -where T: AsyncWrite -{ - type Item = (); - type Error = IoError; - - fn poll(&mut self) -> Poll { - if let Some(item) = self.item.take() { - if let AsyncSink::NotReady(item) = self.inner.start_send(item)? { - self.item = Some(item); - return Ok(Async::NotReady); - } - } - - // A call to `close()` implies flushing. - try_ready!(self.inner.close()); - Ok(Async::Ready(())) - } -} - /// Information of a peer sent in `Identify` protocol responses. #[derive(Debug, Clone)] pub struct IdentifyInfo { @@ -162,93 +123,60 @@ impl UpgradeInfo for IdentifyProtocolConfig { impl InboundUpgrade for IdentifyProtocolConfig where - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, { - type Output = ReplySubstream>; - type Error = IoError; - type Future = FutureResult; + type Output = ReplySubstream; + type Error = io::Error; + type Future = future::Ready>; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: C, _: Self::Info) -> Self::Future { trace!("Upgrading inbound connection"); - let inner = Framed::new(socket, codec::UviBytes::default()); - future::ok(ReplySubstream { inner }) + future::ok(ReplySubstream { inner: socket }) } } impl OutboundUpgrade for IdentifyProtocolConfig where - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin + Send + 'static, { type Output = RemoteInfo; - type Error = IoError; - type Future = IdentifyOutboundFuture>; - - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - IdentifyOutboundFuture { - inner: Framed::new(socket, codec::UviBytes::::default()), - shutdown: false, - } - } -} - -/// Future returned by `OutboundUpgrade::upgrade_outbound`. -pub struct IdentifyOutboundFuture { - inner: Framed>, - /// If true, we have finished shutting down the writing part of `inner`. - shutdown: bool, -} - -impl Future for IdentifyOutboundFuture -where T: AsyncRead + AsyncWrite, -{ - type Item = RemoteInfo; - type Error = IoError; - - fn poll(&mut self) -> Poll { - if !self.shutdown { - try_ready!(self.inner.close()); - self.shutdown = true; - } - - let msg = match try_ready!(self.inner.poll()) { - Some(i) => i, - None => { - debug!("Identify protocol stream closed before receiving info"); - return Err(IoErrorKind::InvalidData.into()); - } - }; - - debug!("Received identify message"); - - let (info, observed_addr) = match parse_proto_msg(msg) { - Ok(v) => v, - Err(err) => { - debug!("Failed to parse protobuf message; error = {:?}", err); - return Err(err) - } - }; + type Error = upgrade::ReadOneError; + type Future = Pin> + Send>>; + + fn upgrade_outbound(self, mut socket: C, _: Self::Info) -> Self::Future { + Box::pin(async move { + socket.close().await?; + let msg = upgrade::read_one(&mut socket, 4096).await?; + let (info, observed_addr) = match parse_proto_msg(msg) { + Ok(v) => v, + Err(err) => { + debug!("Failed to parse protobuf message; error = {:?}", err); + return Err(err.into()) + } + }; - trace!("Remote observes us as {:?}", observed_addr); - trace!("Information received: {:?}", info); + trace!("Remote observes us as {:?}", observed_addr); + trace!("Information received: {:?}", info); - Ok(Async::Ready(RemoteInfo { - info, - observed_addr: observed_addr.clone(), - _priv: () - })) + Ok(RemoteInfo { + info, + observed_addr: observed_addr.clone(), + _priv: () + }) + }) } } // Turns a protobuf message into an `IdentifyInfo` and an observed address. If something bad -// happens, turn it into an `IoError`. -fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError> { - match protobuf_parse_from_bytes::(&msg) { +// happens, turn it into an `io::Error`. +fn parse_proto_msg(msg: impl AsRef<[u8]>) -> Result<(IdentifyInfo, Multiaddr), io::Error> { + match protobuf_parse_from_bytes::(msg.as_ref()) { Ok(mut msg) => { // Turn a `Vec` into a `Multiaddr`. If something bad happens, turn it into - // an `IoError`. - fn bytes_to_multiaddr(bytes: Vec) -> Result { + // an `io::Error`. + fn bytes_to_multiaddr(bytes: Vec) -> Result { Multiaddr::try_from(bytes) - .map_err(|err| IoError::new(IoErrorKind::InvalidData, err)) + .map_err(|err| io::Error::new(io::ErrorKind::InvalidData, err)) } let listen_addrs = { @@ -260,7 +188,7 @@ fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError> }; let public_key = PublicKey::from_protobuf_encoding(msg.get_publicKey()) - .map_err(|e| IoError::new(IoErrorKind::InvalidData, e))?; + .map_err(|e| io::Error::new(io::ErrorKind::InvalidData, e))?; let observed_addr = bytes_to_multiaddr(msg.take_observedAddr())?; let info = IdentifyInfo { @@ -274,23 +202,20 @@ fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError> Ok((info, observed_addr)) } - Err(err) => Err(IoError::new(IoErrorKind::InvalidData, err)), + Err(err) => Err(io::Error::new(io::ErrorKind::InvalidData, err)), } } #[cfg(test)] mod tests { use crate::protocol::{IdentifyInfo, RemoteInfo, IdentifyProtocolConfig}; - use tokio::runtime::current_thread::Runtime; use libp2p_tcp::TcpConfig; - use futures::{Future, Stream}; + use futures::{prelude::*, channel::oneshot}; use libp2p_core::{ identity, Transport, - transport::ListenerEvent, upgrade::{self, apply_outbound, apply_inbound} }; - use std::{io, sync::mpsc, thread}; #[test] fn correct_transfer() { @@ -299,75 +224,55 @@ mod tests { let send_pubkey = identity::Keypair::generate_ed25519().public(); let recv_pubkey = send_pubkey.clone(); - let (tx, rx) = mpsc::channel(); + let (tx, rx) = oneshot::channel(); - let bg_thread = thread::spawn(move || { + let bg_task = async_std::task::spawn(async move { let transport = TcpConfig::new(); let mut listener = transport .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) .unwrap(); - let addr = listener.by_ref().wait() - .next() + let addr = listener.next().await .expect("some event") .expect("no error") .into_new_address() .expect("listen address"); - - tx.send(addr).unwrap(); - let future = listener - .filter_map(ListenerEvent::into_upgrade) - .into_future() - .map_err(|(err, _)| err) - .and_then(|(client, _)| client.unwrap().0) - .and_then(|socket| { - apply_inbound(socket, IdentifyProtocolConfig) - .map_err(|e| io::Error::new(io::ErrorKind::Other, e)) - }) - .and_then(|sender| { - sender.send( - IdentifyInfo { - public_key: send_pubkey, - protocol_version: "proto_version".to_owned(), - agent_version: "agent_version".to_owned(), - listen_addrs: vec![ - "/ip4/80.81.82.83/tcp/500".parse().unwrap(), - "/ip6/::1/udp/1000".parse().unwrap(), - ], - protocols: vec!["proto1".to_string(), "proto2".to_string()], - }, - &"/ip4/100.101.102.103/tcp/5000".parse().unwrap(), - ) - }); - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); + let socket = listener.next().await.unwrap().unwrap().into_upgrade().unwrap().0.await.unwrap(); + let sender = apply_inbound(socket, IdentifyProtocolConfig).await.unwrap(); + sender.send( + IdentifyInfo { + public_key: send_pubkey, + protocol_version: "proto_version".to_owned(), + agent_version: "agent_version".to_owned(), + listen_addrs: vec![ + "/ip4/80.81.82.83/tcp/500".parse().unwrap(), + "/ip6/::1/udp/1000".parse().unwrap(), + ], + protocols: vec!["proto1".to_string(), "proto2".to_string()], + }, + &"/ip4/100.101.102.103/tcp/5000".parse().unwrap(), + ).await.unwrap(); }); - let transport = TcpConfig::new(); + async_std::task::block_on(async move { + let transport = TcpConfig::new(); - let future = transport.dial(rx.recv().unwrap()) - .unwrap() - .and_then(|socket| { - apply_outbound(socket, IdentifyProtocolConfig, upgrade::Version::V1) - .map_err(|e| io::Error::new(io::ErrorKind::Other, e)) - }) - .and_then(|RemoteInfo { info, observed_addr, .. }| { - assert_eq!(observed_addr, "/ip4/100.101.102.103/tcp/5000".parse().unwrap()); - assert_eq!(info.public_key, recv_pubkey); - assert_eq!(info.protocol_version, "proto_version"); - assert_eq!(info.agent_version, "agent_version"); - assert_eq!(info.listen_addrs, - &["/ip4/80.81.82.83/tcp/500".parse().unwrap(), - "/ip6/::1/udp/1000".parse().unwrap()]); - assert_eq!(info.protocols, &["proto1".to_string(), "proto2".to_string()]); - Ok(()) - }); - - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); - bg_thread.join().unwrap(); + let socket = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); + let RemoteInfo { info, observed_addr, .. } = + apply_outbound(socket, IdentifyProtocolConfig, upgrade::Version::V1).await.unwrap(); + assert_eq!(observed_addr, "/ip4/100.101.102.103/tcp/5000".parse().unwrap()); + assert_eq!(info.public_key, recv_pubkey); + assert_eq!(info.protocol_version, "proto_version"); + assert_eq!(info.agent_version, "agent_version"); + assert_eq!(info.listen_addrs, + &["/ip4/80.81.82.83/tcp/500".parse().unwrap(), + "/ip6/::1/udp/1000".parse().unwrap()]); + assert_eq!(info.protocols, &["proto1".to_string(), "proto2".to_string()]); + + bg_task.await; + }); } } diff --git a/protocols/kad/Cargo.toml b/protocols/kad/Cargo.toml index 26ea0d6cf4c..27c99a6fefb 100644 --- a/protocols/kad/Cargo.toml +++ b/protocols/kad/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-kad" edition = "2018" description = "Kademlia protocol for libp2p" -version = "0.13.2" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -11,30 +11,25 @@ categories = ["network-programming", "asynchronous"] [dependencies] arrayvec = "0.5.1" -bytes = "0.4" +bytes = "0.5" either = "1.5" fnv = "1.0" -futures = "0.1" +futures_codec = "0.3.4" +futures = "0.3.1" log = "0.4" -libp2p-core = { version = "0.13.0", path = "../../core" } -libp2p-swarm = { version = "0.3.0", path = "../../swarm" } -multiaddr = { package = "parity-multiaddr", version = "0.6.0", path = "../../misc/multiaddr" } -multihash = { package = "parity-multihash", version = "0.2.0", path = "../../misc/multihash" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } +multihash = { package = "parity-multihash", version = "0.2.1", path = "../../misc/multihash" } protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 rand = "0.7.2" sha2 = "0.8.0" -smallvec = "0.6" -tokio-codec = "0.1" -tokio-io = "0.1" -wasm-timer = "0.1" +smallvec = "1.0" +wasm-timer = "0.2" uint = "0.8" -unsigned-varint = { version = "0.2.1", features = ["codec"] } +unsigned-varint = { version = "0.3", features = ["futures-codec"] } void = "1.0" [dev-dependencies] -libp2p-secio = { version = "0.13.0", path = "../secio" } -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } -libp2p-yamux = { version = "0.13.0", path = "../../muxers/yamux" } +libp2p-secio = { version = "0.14.0-alpha.1", path = "../secio" } +libp2p-yamux = { version = "0.14.0-alpha.1", path = "../../muxers/yamux" } quickcheck = "0.9.0" -rand = "0.7.2" -tokio = "0.1" diff --git a/protocols/kad/src/behaviour.rs b/protocols/kad/src/behaviour.rs index bf52621d654..588bdd8aca2 100644 --- a/protocols/kad/src/behaviour.rs +++ b/protocols/kad/src/behaviour.rs @@ -39,7 +39,7 @@ use smallvec::SmallVec; use std::{borrow::Cow, error, iter, marker::PhantomData, time::Duration}; use std::collections::VecDeque; use std::num::NonZeroUsize; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::task::{Context, Poll}; use wasm_timer::Instant; /// Network behaviour that handles Kademlia. @@ -1010,7 +1010,7 @@ where impl NetworkBehaviour for Kademlia where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, for<'a> TStore: RecordStore<'a>, { type ProtocolsHandler = KademliaHandler; @@ -1304,7 +1304,7 @@ where }; } - fn poll(&mut self, parameters: &mut impl PollParameters) -> Async< + fn poll(&mut self, cx: &mut Context, parameters: &mut impl PollParameters) -> Poll< NetworkBehaviourAction< ::InEvent, Self::OutEvent, @@ -1319,7 +1319,7 @@ where if let Some(mut job) = self.add_provider_job.take() { let num = usize::min(JOBS_MAX_NEW_QUERIES, jobs_query_capacity); for _ in 0 .. num { - if let Async::Ready(r) = job.poll(&mut self.store, now) { + if let Poll::Ready(r) = job.poll(cx, &mut self.store, now) { self.start_add_provider(r.key, AddProviderContext::Republish) } else { break @@ -1333,7 +1333,7 @@ where if let Some(mut job) = self.put_record_job.take() { let num = usize::min(JOBS_MAX_NEW_QUERIES, jobs_query_capacity); for _ in 0 .. num { - if let Async::Ready(r) = job.poll(&mut self.store, now) { + if let Poll::Ready(r) = job.poll(cx, &mut self.store, now) { let context = if r.publisher.as_ref() == Some(self.kbuckets.local_key().preimage()) { PutRecordContext::Republish } else { @@ -1350,7 +1350,7 @@ where loop { // Drain queued events first. if let Some(event) = self.queued_events.pop_front() { - return Async::Ready(event); + return Poll::Ready(event); } // Drain applied pending entries from the routing table. @@ -1361,7 +1361,7 @@ where addresses: value, old_peer: entry.evicted.map(|n| n.key.into_preimage()) }; - return Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) } // Look for a finished query. @@ -1369,12 +1369,12 @@ where match self.queries.poll(now) { QueryPoolState::Finished(q) => { if let Some(event) = self.query_finished(q, parameters) { - return Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) } } QueryPoolState::Timeout(q) => { if let Some(event) = self.query_timeout(q) { - return Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) } } QueryPoolState::Waiting(Some((query, peer_id))) => { @@ -1406,7 +1406,7 @@ where // If no new events have been queued either, signal `NotReady` to // be polled again later. if self.queued_events.is_empty() { - return Async::NotReady + return Poll::Pending } } } diff --git a/protocols/kad/src/behaviour/test.rs b/protocols/kad/src/behaviour/test.rs index 7786762dda2..efed1ee24e1 100644 --- a/protocols/kad/src/behaviour/test.rs +++ b/protocols/kad/src/behaviour/test.rs @@ -25,7 +25,11 @@ use super::*; use crate::K_VALUE; use crate::kbucket::Distance; use crate::record::store::MemoryStore; -use futures::future; +use futures::{ + prelude::*, + executor::block_on, + future::poll_fn, +}; use libp2p_core::{ PeerId, Transport, @@ -42,7 +46,6 @@ use libp2p_yamux as yamux; use quickcheck::*; use rand::{Rng, random, thread_rng}; use std::{collections::{HashSet, HashMap}, io, num::NonZeroUsize, u64}; -use tokio::runtime::current_thread; use multihash::{Multihash, Hash::SHA2256}; type TestSwarm = Swarm< @@ -120,27 +123,30 @@ fn bootstrap() { let expected_known = swarm_ids.iter().skip(1).cloned().collect::>(); // Run test - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for (i, swarm) in swarms.iter_mut().enumerate() { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::BootstrapResult(Ok(ok)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::BootstrapResult(Ok(ok)))) => { assert_eq!(i, 0); assert_eq!(ok.peer, swarm_ids[0]); let known = swarm.kbuckets.iter() .map(|e| e.node.key.preimage().clone()) .collect::>(); assert_eq!(expected_known, known); - return Ok(Async::Ready(())); + return Poll::Ready(()) } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } let mut rng = thread_rng(); @@ -175,27 +181,30 @@ fn query_iter() { expected_distances.sort(); // Run test - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for (i, swarm) in swarms.iter_mut().enumerate() { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::GetClosestPeersResult(Ok(ok)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::GetClosestPeersResult(Ok(ok)))) => { assert_eq!(&ok.key[..], search_target.as_bytes()); assert_eq!(swarm_ids[i], expected_swarm_id); assert_eq!(swarm.queries.size(), 0); assert!(expected_peer_ids.iter().all(|p| ok.peers.contains(p))); let key = kbucket::Key::new(ok.key); assert_eq!(expected_distances, distances(&key, ok.peers)); - return Ok(Async::Ready(())); + return Poll::Ready(()); } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } let mut rng = thread_rng(); @@ -220,24 +229,27 @@ fn unresponsive_not_returned_direct() { let search_target = PeerId::random(); swarms[0].get_closest_peers(search_target.clone()); - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::GetClosestPeersResult(Ok(ok)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::GetClosestPeersResult(Ok(ok)))) => { assert_eq!(&ok.key[..], search_target.as_bytes()); assert_eq!(ok.peers.len(), 0); - return Ok(Async::Ready(())); + return Poll::Ready(()); } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } #[test] @@ -261,25 +273,28 @@ fn unresponsive_not_returned_indirect() { let search_target = PeerId::random(); swarms[1].get_closest_peers(search_target.clone()); - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::GetClosestPeersResult(Ok(ok)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::GetClosestPeersResult(Ok(ok)))) => { assert_eq!(&ok.key[..], search_target.as_bytes()); assert_eq!(ok.peers.len(), 1); assert_eq!(ok.peers[0], first_peer_id); - return Ok(Async::Ready(())); + return Poll::Ready(()); } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } #[test] @@ -294,30 +309,33 @@ fn get_record_not_found() { let target_key = record::Key::from(Multihash::random(SHA2256)); swarms[0].get_record(&target_key, Quorum::One); - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::GetRecordResult(Err(e)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::GetRecordResult(Err(e)))) => { if let GetRecordError::NotFound { key, closest_peers, } = e { assert_eq!(key, target_key); assert_eq!(closest_peers.len(), 2); assert!(closest_peers.contains(&swarm_ids[1])); assert!(closest_peers.contains(&swarm_ids[2])); - return Ok(Async::Ready(())); + return Poll::Ready(()); } else { panic!("Unexpected error result: {:?}", e); } } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } #[test] @@ -351,14 +369,14 @@ fn put_record() { // The accumulated results for one round of publishing. let mut results = Vec::new(); - current_thread::run( - future::poll_fn(move || loop { - // Poll all swarms until they are "NotReady". + block_on( + poll_fn(move |ctx| loop { + // Poll all swarms until they are "Pending". for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::PutRecordResult(res))) | - Async::Ready(Some(KademliaEvent::RepublishRecordResult(res))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::PutRecordResult(res))) | + Poll::Ready(Some(KademliaEvent::RepublishRecordResult(res))) => { match res { Err(e) => panic!(e), Ok(ok) => { @@ -368,16 +386,18 @@ fn put_record() { } } } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - // All swarms are NotReady and not enough results have been collected + // All swarms are Pending and not enough results have been collected // so far, thus wait to be polled again for further progress. if results.len() != records.len() { - return Ok(Async::NotReady) + return Poll::Pending } // Consume the results, checking that each record was replicated @@ -422,7 +442,7 @@ fn put_record() { } assert_eq!(swarms[0].store.records().count(), 0); // All records have been republished, thus the test is complete. - return Ok(Async::Ready(())); + return Poll::Ready(()); } // Tell the replication job to republish asap. @@ -449,24 +469,27 @@ fn get_value() { swarms[1].store.put(record.clone()).unwrap(); swarms[0].get_record(&record.key, Quorum::One); - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::GetRecordResult(Ok(ok)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::GetRecordResult(Ok(ok)))) => { assert_eq!(ok.records.len(), 1); assert_eq!(ok.records.first(), Some(&record)); - return Ok(Async::Ready(())); + return Poll::Ready(()); } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } #[test] @@ -485,23 +508,26 @@ fn get_value_many() { let quorum = Quorum::N(NonZeroUsize::new(num_results).unwrap()); swarms[0].get_record(&record.key, quorum); - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::GetRecordResult(Ok(ok)))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::GetRecordResult(Ok(ok)))) => { assert_eq!(ok.records.len(), num_results); assert_eq!(ok.records.first(), Some(&record)); - return Ok(Async::Ready(())); + return Poll::Ready(()); } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } - Ok(Async::NotReady) - })) + Poll::Pending + }) + ) } #[test] @@ -529,14 +555,14 @@ fn add_provider() { swarms[0].start_providing(k.clone()); } - current_thread::run( - future::poll_fn(move || loop { - // Poll all swarms until they are "NotReady". + block_on( + poll_fn(move |ctx| loop { + // Poll all swarms until they are "Pending". for swarm in &mut swarms { loop { - match swarm.poll().unwrap() { - Async::Ready(Some(KademliaEvent::StartProvidingResult(res))) | - Async::Ready(Some(KademliaEvent::RepublishProviderResult(res))) => { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::StartProvidingResult(res))) | + Poll::Ready(Some(KademliaEvent::RepublishProviderResult(res))) => { match res { Err(e) => panic!(e), Ok(ok) => { @@ -545,8 +571,10 @@ fn add_provider() { } } } - Async::Ready(_) => (), - Async::NotReady => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Pending => break, } } } @@ -559,7 +587,7 @@ fn add_provider() { if !published { // Still waiting for all requests to be sent for one round // of publishing. - return Ok(Async::NotReady) + return Poll::Pending } // A round of publishing is complete. Consume the results, checking that @@ -578,7 +606,7 @@ fn add_provider() { if actual.len() != replication_factor.get() { // Still waiting for some nodes to process the request. results.push(key); - return Ok(Async::NotReady) + return Poll::Pending } let mut expected = swarm_ids.clone().split_off(1); @@ -608,7 +636,7 @@ fn add_provider() { } assert_eq!(swarms[0].store.provided().count(), 0); // All records have been republished, thus the test is complete. - return Ok(Async::Ready(())); + return Poll::Ready(()); } // Initiate the second round of publishing by telling the @@ -636,11 +664,11 @@ fn exceed_jobs_max_queries() { assert_eq!(swarms[0].queries.size(), num); - current_thread::run( - future::poll_fn(move || { + block_on( + poll_fn(move |ctx| { for _ in 0 .. num { // There are no other nodes, so the queries finish instantly. - if let Ok(Async::Ready(Some(e))) = swarms[0].poll() { + if let Poll::Ready(Some(e)) = swarms[0].poll_next_unpin(ctx) { if let KademliaEvent::BootstrapResult(r) = e { assert!(r.is_ok(), "Unexpected error") } else { @@ -650,7 +678,7 @@ fn exceed_jobs_max_queries() { panic!("Expected event") } } - Ok(Async::Ready(())) - })) + Poll::Ready(()) + }) + ) } - diff --git a/protocols/kad/src/handler.rs b/protocols/kad/src/handler.rs index 5a559433b4d..59bf26c76d0 100644 --- a/protocols/kad/src/handler.rs +++ b/protocols/kad/src/handler.rs @@ -36,8 +36,7 @@ use libp2p_core::{ upgrade::{self, InboundUpgrade, OutboundUpgrade, Negotiated} }; use log::trace; -use std::{borrow::Cow, error, fmt, io, time::Duration}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{borrow::Cow, error, fmt, io, pin::Pin, task::Context, task::Poll, time::Duration}; use wasm_timer::Instant; /// Protocol handler that handles Kademlia communications with the remote. @@ -48,7 +47,7 @@ use wasm_timer::Instant; /// It also handles requests made by the remote. pub struct KademliaHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { /// Configuration for the Kademlia protocol. config: KademliaProtocolConfig, @@ -60,7 +59,7 @@ where next_connec_unique_id: UniqueConnecId, /// List of active substreams with the state they are in. - substreams: Vec, TUserData>>, + substreams: Vec>, /// Until when to keep the connection alive. keep_alive: KeepAlive, @@ -69,63 +68,63 @@ where /// State of an active substream, opened either by us or by the remote. enum SubstreamState where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { /// We haven't started opening the outgoing substream yet. /// Contains the request we want to send, and the user data if we expect an answer. OutPendingOpen(KadRequestMsg, Option), /// Waiting to send a message to the remote. OutPendingSend( - KadOutStreamSink, + KadOutStreamSink>, KadRequestMsg, Option, ), /// Waiting to flush the substream so that the data arrives to the remote. - OutPendingFlush(KadOutStreamSink, Option), + OutPendingFlush(KadOutStreamSink>, Option), /// Waiting for an answer back from the remote. // TODO: add timeout - OutWaitingAnswer(KadOutStreamSink, TUserData), + OutWaitingAnswer(KadOutStreamSink>, TUserData), /// An error happened on the substream and we should report the error to the user. OutReportError(KademliaHandlerQueryErr, TUserData), /// The substream is being closed. - OutClosing(KadOutStreamSink), + OutClosing(KadOutStreamSink>), /// Waiting for a request from the remote. - InWaitingMessage(UniqueConnecId, KadInStreamSink), + InWaitingMessage(UniqueConnecId, KadInStreamSink>), /// Waiting for the user to send a `KademliaHandlerIn` event containing the response. - InWaitingUser(UniqueConnecId, KadInStreamSink), + InWaitingUser(UniqueConnecId, KadInStreamSink>), /// Waiting to send an answer back to the remote. - InPendingSend(UniqueConnecId, KadInStreamSink, KadResponseMsg), + InPendingSend(UniqueConnecId, KadInStreamSink>, KadResponseMsg), /// Waiting to flush an answer back to the remote. - InPendingFlush(UniqueConnecId, KadInStreamSink), + InPendingFlush(UniqueConnecId, KadInStreamSink>), /// The substream is being closed. - InClosing(KadInStreamSink), + InClosing(KadInStreamSink>), } impl SubstreamState where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { - /// Consumes this state and tries to close the substream. + /// Tries to close the substream. /// /// If the substream is not ready to be closed, returns it back. - fn try_close(self) -> AsyncSink { + fn try_close(&mut self, cx: &mut Context) -> Poll<()> { match self { SubstreamState::OutPendingOpen(_, _) - | SubstreamState::OutReportError(_, _) => AsyncSink::Ready, - SubstreamState::OutPendingSend(mut stream, _, _) - | SubstreamState::OutPendingFlush(mut stream, _) - | SubstreamState::OutWaitingAnswer(mut stream, _) - | SubstreamState::OutClosing(mut stream) => match stream.close() { - Ok(Async::Ready(())) | Err(_) => AsyncSink::Ready, - Ok(Async::NotReady) => AsyncSink::NotReady(SubstreamState::OutClosing(stream)), + | SubstreamState::OutReportError(_, _) => Poll::Ready(()), + SubstreamState::OutPendingSend(ref mut stream, _, _) + | SubstreamState::OutPendingFlush(ref mut stream, _) + | SubstreamState::OutWaitingAnswer(ref mut stream, _) + | SubstreamState::OutClosing(ref mut stream) => match Sink::poll_close(Pin::new(stream), cx) { + Poll::Ready(_) => Poll::Ready(()), + Poll::Pending => Poll::Pending, }, - SubstreamState::InWaitingMessage(_, mut stream) - | SubstreamState::InWaitingUser(_, mut stream) - | SubstreamState::InPendingSend(_, mut stream, _) - | SubstreamState::InPendingFlush(_, mut stream) - | SubstreamState::InClosing(mut stream) => match stream.close() { - Ok(Async::Ready(())) | Err(_) => AsyncSink::Ready, - Ok(Async::NotReady) => AsyncSink::NotReady(SubstreamState::InClosing(stream)), + SubstreamState::InWaitingMessage(_, ref mut stream) + | SubstreamState::InWaitingUser(_, ref mut stream) + | SubstreamState::InPendingSend(_, ref mut stream, _) + | SubstreamState::InPendingFlush(_, ref mut stream) + | SubstreamState::InClosing(ref mut stream) => match Sink::poll_close(Pin::new(stream), cx) { + Poll::Ready(_) => Poll::Ready(()), + Poll::Pending => Poll::Pending, }, } } @@ -382,7 +381,7 @@ struct UniqueConnecId(u64); impl KademliaHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { /// Create a `KademliaHandler` that only allows sending messages to the remote but denying /// incoming connections. @@ -418,7 +417,7 @@ where impl Default for KademliaHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { #[inline] fn default() -> Self { @@ -428,7 +427,7 @@ where impl ProtocolsHandler for KademliaHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, TUserData: Clone, { type InEvent = KademliaHandlerIn; @@ -451,7 +450,7 @@ where fn inject_fully_negotiated_outbound( &mut self, - protocol: >::Output, + protocol: >>::Output, (msg, user_data): Self::OutboundOpenInfo, ) { self.substreams @@ -460,7 +459,7 @@ where fn inject_fully_negotiated_inbound( &mut self, - protocol: >::Output, + protocol: >>::Output, ) { // If `self.allow_listening` is false, then we produced a `DeniedUpgrade` and `protocol` // is a `Void`. @@ -485,7 +484,10 @@ where _ => false, }); if let Some(pos) = pos { - let _ = self.substreams.remove(pos).try_close(); + // TODO: we don't properly close down the substream + let waker = futures::task::noop_waker(); + let mut cx = Context::from_waker(&waker); + let _ = self.substreams.remove(pos).try_close(&mut cx); } } KademliaHandlerIn::FindNodeReq { key, user_data } => { @@ -639,22 +641,22 @@ where fn poll( &mut self, + cx: &mut Context, ) -> Poll< - ProtocolsHandlerEvent, - io::Error, + ProtocolsHandlerEvent, > { // We remove each element from `substreams` one by one and add them back. for n in (0..self.substreams.len()).rev() { let mut substream = self.substreams.swap_remove(n); loop { - match advance_substream(substream, self.config.clone()) { + match advance_substream(substream, self.config.clone(), cx) { (Some(new_state), Some(event), _) => { self.substreams.push(new_state); - return Ok(Async::Ready(event)); + return Poll::Ready(event); } (None, Some(event), _) => { - return Ok(Async::Ready(event)); + return Poll::Ready(event); } (Some(new_state), None, false) => { self.substreams.push(new_state); @@ -677,7 +679,7 @@ where self.keep_alive = KeepAlive::Yes; } - Ok(Async::NotReady) + Poll::Pending } } @@ -688,6 +690,7 @@ where fn advance_substream( state: SubstreamState, upgrade: KademliaProtocolConfig, + cx: &mut Context, ) -> ( Option>, Option< @@ -695,12 +698,13 @@ fn advance_substream( KademliaProtocolConfig, (KadRequestMsg, Option), KademliaHandlerEvent, + io::Error, >, >, bool, ) where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { match state { SubstreamState::OutPendingOpen(msg, user_data) => { @@ -711,18 +715,34 @@ where (None, Some(ev), false) } SubstreamState::OutPendingSend(mut substream, msg, user_data) => { - match substream.start_send(msg) { - Ok(AsyncSink::Ready) => ( - Some(SubstreamState::OutPendingFlush(substream, user_data)), - None, - true, - ), - Ok(AsyncSink::NotReady(msg)) => ( + match Sink::poll_ready(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { + match Sink::start_send(Pin::new(&mut substream), msg) { + Ok(()) => ( + Some(SubstreamState::OutPendingFlush(substream, user_data)), + None, + true, + ), + Err(error) => { + let event = if let Some(user_data) = user_data { + Some(ProtocolsHandlerEvent::Custom(KademliaHandlerEvent::QueryError { + error: KademliaHandlerQueryErr::Io(error), + user_data + })) + } else { + None + }; + + (None, event, false) + } + } + }, + Poll::Pending => ( Some(SubstreamState::OutPendingSend(substream, msg, user_data)), None, false, ), - Err(error) => { + Poll::Ready(Err(error)) => { let event = if let Some(user_data) = user_data { Some(ProtocolsHandlerEvent::Custom(KademliaHandlerEvent::QueryError { error: KademliaHandlerQueryErr::Io(error), @@ -737,8 +757,8 @@ where } } SubstreamState::OutPendingFlush(mut substream, user_data) => { - match substream.poll_complete() { - Ok(Async::Ready(())) => { + match Sink::poll_flush(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { if let Some(user_data) = user_data { ( Some(SubstreamState::OutWaitingAnswer(substream, user_data)), @@ -749,12 +769,12 @@ where (Some(SubstreamState::OutClosing(substream)), None, true) } } - Ok(Async::NotReady) => ( + Poll::Pending => ( Some(SubstreamState::OutPendingFlush(substream, user_data)), None, false, ), - Err(error) => { + Poll::Ready(Err(error)) => { let event = if let Some(user_data) = user_data { Some(ProtocolsHandlerEvent::Custom(KademliaHandlerEvent::QueryError { error: KademliaHandlerQueryErr::Io(error), @@ -768,8 +788,8 @@ where } } } - SubstreamState::OutWaitingAnswer(mut substream, user_data) => match substream.poll() { - Ok(Async::Ready(Some(msg))) => { + SubstreamState::OutWaitingAnswer(mut substream, user_data) => match Stream::poll_next(Pin::new(&mut substream), cx) { + Poll::Ready(Some(Ok(msg))) => { let new_state = SubstreamState::OutClosing(substream); let event = process_kad_response(msg, user_data); ( @@ -778,19 +798,19 @@ where true, ) } - Ok(Async::NotReady) => ( + Poll::Pending => ( Some(SubstreamState::OutWaitingAnswer(substream, user_data)), None, false, ), - Err(error) => { + Poll::Ready(Some(Err(error))) => { let event = KademliaHandlerEvent::QueryError { error: KademliaHandlerQueryErr::Io(error), user_data, }; (None, Some(ProtocolsHandlerEvent::Custom(event)), false) } - Ok(Async::Ready(None)) => { + Poll::Ready(None) => { let event = KademliaHandlerEvent::QueryError { error: KademliaHandlerQueryErr::Io(io::ErrorKind::UnexpectedEof.into()), user_data, @@ -802,13 +822,13 @@ where let event = KademliaHandlerEvent::QueryError { error, user_data }; (None, Some(ProtocolsHandlerEvent::Custom(event)), false) } - SubstreamState::OutClosing(mut stream) => match stream.close() { - Ok(Async::Ready(())) => (None, None, false), - Ok(Async::NotReady) => (Some(SubstreamState::OutClosing(stream)), None, false), - Err(_) => (None, None, false), + SubstreamState::OutClosing(mut stream) => match Sink::poll_close(Pin::new(&mut stream), cx) { + Poll::Ready(Ok(())) => (None, None, false), + Poll::Pending => (Some(SubstreamState::OutClosing(stream)), None, false), + Poll::Ready(Err(_)) => (None, None, false), }, - SubstreamState::InWaitingMessage(id, mut substream) => match substream.poll() { - Ok(Async::Ready(Some(msg))) => { + SubstreamState::InWaitingMessage(id, mut substream) => match Stream::poll_next(Pin::new(&mut substream), cx) { + Poll::Ready(Some(Ok(msg))) => { if let Ok(ev) = process_kad_request(msg, id) { ( Some(SubstreamState::InWaitingUser(id, substream)), @@ -819,16 +839,16 @@ where (Some(SubstreamState::InClosing(substream)), None, true) } } - Ok(Async::NotReady) => ( + Poll::Pending => ( Some(SubstreamState::InWaitingMessage(id, substream)), None, false, ), - Ok(Async::Ready(None)) => { + Poll::Ready(None) => { trace!("Inbound substream: EOF"); (None, None, false) } - Err(e) => { + Poll::Ready(Some(Err(e))) => { trace!("Inbound substream error: {:?}", e); (None, None, false) }, @@ -838,36 +858,39 @@ where None, false, ), - SubstreamState::InPendingSend(id, mut substream, msg) => match substream.start_send(msg) { - Ok(AsyncSink::Ready) => ( - Some(SubstreamState::InPendingFlush(id, substream)), - None, - true, - ), - Ok(AsyncSink::NotReady(msg)) => ( + SubstreamState::InPendingSend(id, mut substream, msg) => match Sink::poll_ready(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => match Sink::start_send(Pin::new(&mut substream), msg) { + Ok(()) => ( + Some(SubstreamState::InPendingFlush(id, substream)), + None, + true, + ), + Err(_) => (None, None, false), + }, + Poll::Pending => ( Some(SubstreamState::InPendingSend(id, substream, msg)), None, false, ), - Err(_) => (None, None, false), - }, - SubstreamState::InPendingFlush(id, mut substream) => match substream.poll_complete() { - Ok(Async::Ready(())) => ( + Poll::Ready(Err(_)) => (None, None, false), + } + SubstreamState::InPendingFlush(id, mut substream) => match Sink::poll_flush(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => ( Some(SubstreamState::InWaitingMessage(id, substream)), None, true, ), - Ok(Async::NotReady) => ( + Poll::Pending => ( Some(SubstreamState::InPendingFlush(id, substream)), None, false, ), - Err(_) => (None, None, false), + Poll::Ready(Err(_)) => (None, None, false), }, - SubstreamState::InClosing(mut stream) => match stream.close() { - Ok(Async::Ready(())) => (None, None, false), - Ok(Async::NotReady) => (Some(SubstreamState::InClosing(stream)), None, false), - Err(_) => (None, None, false), + SubstreamState::InClosing(mut stream) => match Sink::poll_close(Pin::new(&mut stream), cx) { + Poll::Ready(Ok(())) => (None, None, false), + Poll::Pending => (Some(SubstreamState::InClosing(stream)), None, false), + Poll::Ready(Err(_)) => (None, None, false), }, } } diff --git a/protocols/kad/src/jobs.rs b/protocols/kad/src/jobs.rs index e7909c90bc0..9f5f8c6758c 100644 --- a/protocols/kad/src/jobs.rs +++ b/protocols/kad/src/jobs.rs @@ -65,6 +65,8 @@ use crate::record::{self, Record, ProviderRecord, store::RecordStore}; use libp2p_core::PeerId; use futures::prelude::*; use std::collections::HashSet; +use std::pin::Pin; +use std::task::{Context, Poll}; use std::time::Duration; use std::vec; use wasm_timer::{Instant, Delay}; @@ -96,16 +98,18 @@ impl PeriodicJob { /// Cuts short the remaining delay, if the job is currently waiting /// for the delay to expire. fn asap(&mut self) { - if let PeriodicJobState::Waiting(delay) = &mut self.state { - delay.reset(Instant::now() - Duration::from_secs(1)) + if let PeriodicJobState::Waiting(delay, deadline) = &mut self.state { + let new_deadline = Instant::now() - Duration::from_secs(1); + *deadline = new_deadline; + delay.reset_at(new_deadline); } } /// Returns `true` if the job is currently not running but ready /// to be run, `false` otherwise. - fn is_ready(&mut self, now: Instant) -> bool { - if let PeriodicJobState::Waiting(delay) = &mut self.state { - if now >= delay.deadline() || delay.poll().map(|a| a.is_ready()).unwrap_or(false) { + fn is_ready(&mut self, cx: &mut Context, now: Instant) -> bool { + if let PeriodicJobState::Waiting(delay, deadline) = &mut self.state { + if now >= *deadline || !Future::poll(Pin::new(delay), cx).is_pending() { return true } } @@ -117,7 +121,7 @@ impl PeriodicJob { #[derive(Debug)] enum PeriodicJobState { Running(T), - Waiting(Delay) + Waiting(Delay, Instant) } ////////////////////////////////////////////////////////////////////////////// @@ -143,7 +147,8 @@ impl PutRecordJob { record_ttl: Option, ) -> Self { let now = Instant::now(); - let delay = Delay::new(now + replicate_interval); + let deadline = now + replicate_interval; + let delay = Delay::new_at(deadline); let next_publish = publish_interval.map(|i| now + i); Self { local_id, @@ -153,7 +158,7 @@ impl PutRecordJob { skipped: HashSet::new(), inner: PeriodicJob { interval: replicate_interval, - state: PeriodicJobState::Waiting(delay) + state: PeriodicJobState::Waiting(delay, deadline) } } } @@ -185,11 +190,11 @@ impl PutRecordJob { /// Must be called in the context of a task. When `NotReady` is returned, /// the current task is registered to be notified when the job is ready /// to be run. - pub fn poll(&mut self, store: &mut T, now: Instant) -> Async + pub fn poll(&mut self, cx: &mut Context, store: &mut T, now: Instant) -> Poll where for<'a> T: RecordStore<'a> { - if self.inner.is_ready(now) { + if self.inner.is_ready(cx, now) { let publish = self.next_publish.map_or(false, |t_pub| now >= t_pub); let records = store.records() .filter_map(|r| { @@ -224,7 +229,7 @@ impl PutRecordJob { if r.is_expired(now) { store.remove(&r.key) } else { - return Async::Ready(r) + return Poll::Ready(r) } } else { break @@ -232,12 +237,13 @@ impl PutRecordJob { } // Wait for the next run. - let delay = Delay::new(now + self.inner.interval); - self.inner.state = PeriodicJobState::Waiting(delay); - assert!(!self.inner.is_ready(now)); + let deadline = now + self.inner.interval; + let delay = Delay::new_at(deadline); + self.inner.state = PeriodicJobState::Waiting(delay, deadline); + assert!(!self.inner.is_ready(cx, now)); } - Async::NotReady + Poll::Pending } } @@ -256,7 +262,10 @@ impl AddProviderJob { Self { inner: PeriodicJob { interval, - state: PeriodicJobState::Waiting(Delay::new(now + interval)) + state: { + let deadline = now + interval; + PeriodicJobState::Waiting(Delay::new_at(deadline), deadline) + } } } } @@ -279,11 +288,11 @@ impl AddProviderJob { /// Must be called in the context of a task. When `NotReady` is returned, /// the current task is registered to be notified when the job is ready /// to be run. - pub fn poll(&mut self, store: &mut T, now: Instant) -> Async + pub fn poll(&mut self, cx: &mut Context, store: &mut T, now: Instant) -> Poll where for<'a> T: RecordStore<'a> { - if self.inner.is_ready(now) { + if self.inner.is_ready(cx, now) { let records = store.provided() .map(|r| r.into_owned()) .collect::>() @@ -297,25 +306,27 @@ impl AddProviderJob { if r.is_expired(now) { store.remove_provider(&r.key, &r.provider) } else { - return Async::Ready(r) + return Poll::Ready(r) } } else { break } } - let delay = Delay::new(now + self.inner.interval); - self.inner.state = PeriodicJobState::Waiting(delay); - assert!(!self.inner.is_ready(now)); + let deadline = now + self.inner.interval; + let delay = Delay::new_at(deadline); + self.inner.state = PeriodicJobState::Waiting(delay, deadline); + assert!(!self.inner.is_ready(cx, now)); } - Async::NotReady + Poll::Pending } } #[cfg(test)] mod tests { use crate::record::store::MemoryStore; + use futures::{executor::block_on, future::poll_fn}; use quickcheck::*; use rand::Rng; use super::*; @@ -352,20 +363,20 @@ mod tests { for r in records { let _ = store.put(r); } - // Polling with an instant beyond the deadline for the next run - // is guaranteed to run the job, without the job needing to poll the `Delay` - // and thus without needing to run `poll` in the context of a task - // for testing purposes. - let now = Instant::now() + job.inner.interval; - // All (non-expired) records in the store must be yielded by the job. - for r in store.records().map(|r| r.into_owned()).collect::>() { - if !r.is_expired(now) { - assert_eq!(job.poll(&mut store, now), Async::Ready(r)); - assert!(job.is_running()); + + block_on(poll_fn(|ctx| { + let now = Instant::now() + job.inner.interval; + // All (non-expired) records in the store must be yielded by the job. + for r in store.records().map(|r| r.into_owned()).collect::>() { + if !r.is_expired(now) { + assert_eq!(job.poll(ctx, &mut store, now), Poll::Ready(r)); + assert!(job.is_running()); + } } - } - assert_eq!(job.poll(&mut store, now), Async::NotReady); - assert!(!job.is_running()); + assert_eq!(job.poll(ctx, &mut store, now), Poll::Pending); + assert!(!job.is_running()); + Poll::Ready(()) + })); } quickcheck(prop as fn(_)) @@ -382,23 +393,22 @@ mod tests { r.provider = id.clone(); let _ = store.add_provider(r); } - // Polling with an instant beyond the deadline for the next run - // is guaranteed to run the job, without the job needing to poll the `Delay` - // and thus without needing to run `poll` in the context of a task - // for testing purposes. - let now = Instant::now() + job.inner.interval; - // All (non-expired) records in the store must be yielded by the job. - for r in store.provided().map(|r| r.into_owned()).collect::>() { - if !r.is_expired(now) { - assert_eq!(job.poll(&mut store, now), Async::Ready(r)); - assert!(job.is_running()); + + block_on(poll_fn(|ctx| { + let now = Instant::now() + job.inner.interval; + // All (non-expired) records in the store must be yielded by the job. + for r in store.provided().map(|r| r.into_owned()).collect::>() { + if !r.is_expired(now) { + assert_eq!(job.poll(ctx, &mut store, now), Poll::Ready(r)); + assert!(job.is_running()); + } } - } - assert_eq!(job.poll(&mut store, now), Async::NotReady); - assert!(!job.is_running()); + assert_eq!(job.poll(ctx, &mut store, now), Poll::Pending); + assert!(!job.is_running()); + Poll::Ready(()) + })); } quickcheck(prop as fn(_)) } } - diff --git a/protocols/kad/src/protocol.rs b/protocols/kad/src/protocol.rs index 5a5110531f0..b1e79224f81 100644 --- a/protocols/kad/src/protocol.rs +++ b/protocols/kad/src/protocol.rs @@ -34,14 +34,13 @@ use bytes::BytesMut; use codec::UviBytes; use crate::dht_proto as proto; use crate::record::{self, Record}; -use futures::{future::{self, FutureResult}, sink, stream, Sink, Stream}; +use futures::prelude::*; +use futures_codec::Framed; use libp2p_core::{Multiaddr, PeerId}; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}; +use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use protobuf::{self, Message}; use std::{borrow::Cow, convert::TryFrom, time::Duration}; use std::{io, iter}; -use tokio_codec::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; use unsigned_varint::codec; use wasm_timer::Instant; @@ -59,7 +58,6 @@ pub enum KadConnectionType { } impl From for KadConnectionType { - #[inline] fn from(raw: proto::Message_ConnectionType) -> KadConnectionType { use proto::Message_ConnectionType::{ CAN_CONNECT, CANNOT_CONNECT, CONNECTED, NOT_CONNECTED @@ -74,7 +72,6 @@ impl From for KadConnectionType { } impl Into for KadConnectionType { - #[inline] fn into(self) -> proto::Message_ConnectionType { use proto::Message_ConnectionType::{ CAN_CONNECT, CANNOT_CONNECT, CONNECTED, NOT_CONNECTED @@ -176,27 +173,31 @@ impl UpgradeInfo for KademliaProtocolConfig { impl InboundUpgrade for KademliaProtocolConfig where - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, { - type Output = KadInStreamSink>; - type Future = FutureResult; + type Output = KadInStreamSink; + type Future = future::Ready>; type Error = io::Error; - #[inline] - fn upgrade_inbound(self, incoming: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, incoming: C, _: Self::Info) -> Self::Future { let mut codec = UviBytes::default(); codec.set_max_len(4096); future::ok( Framed::new(incoming, codec) - .from_err() - .with::<_, fn(_) -> _, _>(|response| { + .err_into() + .with::<_, _, fn(_) -> _, _>(|response| { let proto_struct = resp_msg_to_proto(response); - proto_struct.write_to_bytes().map_err(invalid_data) + future::ready(proto_struct.write_to_bytes() + .map(io::Cursor::new) + .map_err(invalid_data)) }) - .and_then:: _, _>(|bytes| { - let request = protobuf::parse_from_bytes(&bytes)?; - proto_to_req_msg(request) + .and_then::<_, fn(_) -> _>(|bytes| { + let request = match protobuf::parse_from_bytes(&bytes) { + Ok(r) => r, + Err(err) => return future::ready(Err(err.into())) + }; + future::ready(proto_to_req_msg(request)) }), ) } @@ -204,27 +205,31 @@ where impl OutboundUpgrade for KademliaProtocolConfig where - C: AsyncRead + AsyncWrite, + C: AsyncRead + AsyncWrite + Unpin, { - type Output = KadOutStreamSink>; - type Future = FutureResult; + type Output = KadOutStreamSink; + type Future = future::Ready>; type Error = io::Error; - #[inline] - fn upgrade_outbound(self, incoming: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, incoming: C, _: Self::Info) -> Self::Future { let mut codec = UviBytes::default(); codec.set_max_len(4096); future::ok( Framed::new(incoming, codec) - .from_err() - .with::<_, fn(_) -> _, _>(|request| { + .err_into() + .with::<_, _, fn(_) -> _, _>(|request| { let proto_struct = req_msg_to_proto(request); - proto_struct.write_to_bytes().map_err(invalid_data) + future::ready(proto_struct.write_to_bytes() + .map(io::Cursor::new) + .map_err(invalid_data)) }) - .and_then:: _, _>(|bytes| { - let response = protobuf::parse_from_bytes(&bytes)?; - proto_to_resp_msg(response) + .and_then::<_, fn(_) -> _>(|bytes| { + let response = match protobuf::parse_from_bytes(&bytes) { + Ok(r) => r, + Err(err) => return future::ready(Err(err.into())) + }; + future::ready(proto_to_resp_msg(response)) }), ) } @@ -238,13 +243,14 @@ pub type KadOutStreamSink = KadStreamSink; pub type KadStreamSink = stream::AndThen< sink::With< - stream::FromErr>>, io::Error>, + stream::ErrInto>>>, io::Error>, + io::Cursor>, A, - fn(A) -> Result, io::Error>, - Result, io::Error>, + future::Ready>, io::Error>>, + fn(A) -> future::Ready>, io::Error>>, >, - fn(BytesMut) -> Result, - Result, + future::Ready>, + fn(BytesMut) -> future::Ready>, >; /// Request that we can send to a peer or that we received from a peer. diff --git a/protocols/kad/src/record.rs b/protocols/kad/src/record.rs index c33b3106080..dcd724b5e2d 100644 --- a/protocols/kad/src/record.rs +++ b/protocols/kad/src/record.rs @@ -35,7 +35,7 @@ pub struct Key(Bytes); impl Key { /// Creates a new key from the bytes of the input. pub fn new>(key: &K) -> Self { - Key(Bytes::from(key.as_ref())) + Key(Bytes::copy_from_slice(key.as_ref())) } /// Copies the bytes of the key into a new vector. diff --git a/protocols/noise/Cargo.toml b/protocols/noise/Cargo.toml index 2956eda231d..68fdaed1bca 100644 --- a/protocols/noise/Cargo.toml +++ b/protocols/noise/Cargo.toml @@ -1,30 +1,27 @@ [package] name = "libp2p-noise" description = "Cryptographic handshake protocol using the noise framework." -version = "0.11.1" +version = "0.12.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" edition = "2018" [dependencies] -bytes = "0.4" curve25519-dalek = "1" -futures = "0.1" +futures = "0.3.1" lazy_static = "1.2" -libp2p-core = { version = "0.13.0", path = "../../core" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } log = "0.4" protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 -rand = "^0.7.2" +rand = "0.7.2" ring = { version = "0.16.9", features = ["alloc"], default-features = false } snow = { version = "0.6.1", features = ["ring-resolver"], default-features = false } -tokio-io = "0.1" x25519-dalek = "0.5" zeroize = "1" [dev-dependencies] env_logger = "0.7.1" -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../../transports/tcp" } quickcheck = "0.9.0" -tokio = "0.1" sodiumoxide = "^0.2.5" diff --git a/protocols/noise/src/io.rs b/protocols/noise/src/io.rs index ad7f541fe4a..4370e626fc9 100644 --- a/protocols/noise/src/io.rs +++ b/protocols/noise/src/io.rs @@ -22,12 +22,11 @@ pub mod handshake; -use futures::{Async, Poll}; +use futures::ready; +use futures::prelude::*; use log::{debug, trace}; use snow; -use snow::error::{StateProblem, Error as SnowError}; -use std::{fmt, io}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{fmt, io, pin::Pin, ops::DerefMut, task::{Context, Poll}}; const MAX_NOISE_PKG_LEN: usize = 65535; const MAX_WRITE_BUF_LEN: usize = 16384; @@ -63,14 +62,14 @@ pub(crate) enum SnowState { } impl SnowState { - pub fn read_message(&mut self, message: &[u8], payload: &mut [u8]) -> Result { + pub fn read_message(&mut self, message: &[u8], payload: &mut [u8]) -> Result { match self { SnowState::Handshake(session) => session.read_message(message, payload), SnowState::Transport(session) => session.read_message(message, payload), } } - pub fn write_message(&mut self, message: &[u8], payload: &mut [u8]) -> Result { + pub fn write_message(&mut self, message: &[u8], payload: &mut [u8]) -> Result { match self { SnowState::Handshake(session) => session.write_message(message, payload), SnowState::Transport(session) => session.write_message(message, payload), @@ -84,10 +83,10 @@ impl SnowState { } } - pub fn into_transport_mode(self) -> Result { + pub fn into_transport_mode(self) -> Result { match self { SnowState::Handshake(session) => session.into_transport_mode(), - SnowState::Transport(_) => Err(SnowError::State(StateProblem::HandshakeAlreadyFinished)), + SnowState::Transport(_) => Err(snow::Error::State(snow::error::StateProblem::HandshakeAlreadyFinished)), } } } @@ -115,7 +114,7 @@ impl fmt::Debug for NoiseOutput { impl NoiseOutput { fn new(io: T, session: SnowState) -> Self { NoiseOutput { - io, + io, session, buffer: Buffer { inner: Box::new([0; TOTAL_BUFFER_LEN]) }, read_state: ReadState::Init, @@ -159,57 +158,75 @@ enum WriteState { EncErr } -impl io::Read for NoiseOutput { - fn read(&mut self, buf: &mut [u8]) -> io::Result { - let buffer = self.buffer.borrow_mut(); +impl AsyncRead for NoiseOutput { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + let mut this = self.deref_mut(); + + let buffer = this.buffer.borrow_mut(); + loop { - trace!("read state: {:?}", self.read_state); - match self.read_state { + trace!("read state: {:?}", this.read_state); + match this.read_state { ReadState::Init => { - self.read_state = ReadState::ReadLen { buf: [0, 0], off: 0 }; + this.read_state = ReadState::ReadLen { buf: [0, 0], off: 0 }; } ReadState::ReadLen { mut buf, mut off } => { - let n = match read_frame_len(&mut self.io, &mut buf, &mut off) { - Ok(Some(n)) => n, - Ok(None) => { + let n = match read_frame_len(&mut this.io, cx, &mut buf, &mut off) { + Poll::Ready(Ok(Some(n))) => n, + Poll::Ready(Ok(None)) => { trace!("read: eof"); - self.read_state = ReadState::Eof(Ok(())); - return Ok(0) + this.read_state = ReadState::Eof(Ok(())); + return Poll::Ready(Ok(0)) } - Err(e) => { - if e.kind() == io::ErrorKind::WouldBlock { - // Preserve read state - self.read_state = ReadState::ReadLen { buf, off }; - } - return Err(e) + Poll::Ready(Err(e)) => { + return Poll::Ready(Err(e)) + } + Poll::Pending => { + this.read_state = ReadState::ReadLen { buf, off }; + + return Poll::Pending; } }; trace!("read: next frame len = {}", n); if n == 0 { trace!("read: empty frame"); - self.read_state = ReadState::Init; + this.read_state = ReadState::Init; continue } - self.read_state = ReadState::ReadData { len: usize::from(n), off: 0 } + this.read_state = ReadState::ReadData { len: usize::from(n), off: 0 } } ReadState::ReadData { len, ref mut off } => { - let n = self.io.read(&mut buffer.read[*off .. len])?; + let n = match ready!( + Pin::new(&mut this.io).poll_read(cx, &mut buffer.read[*off ..len]) + ) { + Ok(n) => n, + Err(e) => return Poll::Ready(Err(e)), + }; + trace!("read: read {}/{} bytes", *off + n, len); if n == 0 { trace!("read: eof"); - self.read_state = ReadState::Eof(Err(())); - return Err(io::ErrorKind::UnexpectedEof.into()) + this.read_state = ReadState::Eof(Err(())); + return Poll::Ready(Err(io::ErrorKind::UnexpectedEof.into())) } + *off += n; if len == *off { trace!("read: decrypting {} bytes", len); - if let Ok(n) = self.session.read_message(&buffer.read[.. len], buffer.read_crypto) { + if let Ok(n) = this.session.read_message( + &buffer.read[.. len], + buffer.read_crypto + ){ trace!("read: payload len = {} bytes", n); - self.read_state = ReadState::CopyData { len: n, off: 0 } + this.read_state = ReadState::CopyData { len: n, off: 0 } } else { debug!("decryption error"); - self.read_state = ReadState::DecErr; - return Err(io::ErrorKind::InvalidData.into()) + this.read_state = ReadState::DecErr; + return Poll::Ready(Err(io::ErrorKind::InvalidData.into())) } } } @@ -219,32 +236,39 @@ impl io::Read for NoiseOutput { trace!("read: copied {}/{} bytes", *off + n, len); *off += n; if len == *off { - self.read_state = ReadState::ReadLen { buf: [0, 0], off: 0 }; + this.read_state = ReadState::ReadLen { buf: [0, 0], off: 0 }; } - return Ok(n) + return Poll::Ready(Ok(n)) } ReadState::Eof(Ok(())) => { trace!("read: eof"); - return Ok(0) + return Poll::Ready(Ok(0)) } ReadState::Eof(Err(())) => { trace!("read: eof (unexpected)"); - return Err(io::ErrorKind::UnexpectedEof.into()) + return Poll::Ready(Err(io::ErrorKind::UnexpectedEof.into())) } - ReadState::DecErr => return Err(io::ErrorKind::InvalidData.into()) + ReadState::DecErr => return Poll::Ready(Err(io::ErrorKind::InvalidData.into())) } } } } -impl io::Write for NoiseOutput { - fn write(&mut self, buf: &[u8]) -> io::Result { - let buffer = self.buffer.borrow_mut(); +impl AsyncWrite for NoiseOutput { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll>{ + let mut this = self.deref_mut(); + + let buffer = this.buffer.borrow_mut(); + loop { - trace!("write state: {:?}", self.write_state); - match self.write_state { + trace!("write state: {:?}", this.write_state); + match this.write_state { WriteState::Init => { - self.write_state = WriteState::BufferData { off: 0 } + this.write_state = WriteState::BufferData { off: 0 } } WriteState::BufferData { ref mut off } => { let n = std::cmp::min(MAX_WRITE_BUF_LEN - *off, buf.len()); @@ -253,138 +277,155 @@ impl io::Write for NoiseOutput { *off += n; if *off == MAX_WRITE_BUF_LEN { trace!("write: encrypting {} bytes", *off); - if let Ok(n) = self.session.write_message(buffer.write, buffer.write_crypto) { - trace!("write: cipher text len = {} bytes", n); - self.write_state = WriteState::WriteLen { - len: n, - buf: u16::to_be_bytes(n as u16), - off: 0 + match this.session.write_message(buffer.write, buffer.write_crypto) { + Ok(n) => { + trace!("write: cipher text len = {} bytes", n); + this.write_state = WriteState::WriteLen { + len: n, + buf: u16::to_be_bytes(n as u16), + off: 0 + } + } + Err(e) => { + debug!("encryption error: {:?}", e); + this.write_state = WriteState::EncErr; + return Poll::Ready(Err(io::ErrorKind::InvalidData.into())) } - } else { - debug!("encryption error"); - self.write_state = WriteState::EncErr; - return Err(io::ErrorKind::InvalidData.into()) } } - return Ok(n) + return Poll::Ready(Ok(n)) } WriteState::WriteLen { len, mut buf, mut off } => { trace!("write: writing len ({}, {:?}, {}/2)", len, buf, off); - match write_frame_len(&mut self.io, &mut buf, &mut off) { - Err(e) => { - if e.kind() == io::ErrorKind::WouldBlock { - self.write_state = WriteState::WriteLen{ len, buf, off }; - } - return Err(e) - } - Ok(false) => { + match write_frame_len(&mut this.io, cx, &mut buf, &mut off) { + Poll::Ready(Ok(true)) => (), + Poll::Ready(Ok(false)) => { trace!("write: eof"); - self.write_state = WriteState::Eof; - return Err(io::ErrorKind::WriteZero.into()) + this.write_state = WriteState::Eof; + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) + } + Poll::Ready(Err(e)) => { + return Poll::Ready(Err(e)) + } + Poll::Pending => { + this.write_state = WriteState::WriteLen{ len, buf, off }; + + return Poll::Pending } - Ok(true) => () } - self.write_state = WriteState::WriteData { len, off: 0 } + this.write_state = WriteState::WriteData { len, off: 0 } } WriteState::WriteData { len, ref mut off } => { - let n = self.io.write(&buffer.write_crypto[*off .. len])?; + let n = match ready!( + Pin::new(&mut this.io).poll_write(cx, &buffer.write_crypto[*off .. len]) + ) { + Ok(n) => n, + Err(e) => return Poll::Ready(Err(e)), + }; trace!("write: wrote {}/{} bytes", *off + n, len); if n == 0 { trace!("write: eof"); - self.write_state = WriteState::Eof; - return Err(io::ErrorKind::WriteZero.into()) + this.write_state = WriteState::Eof; + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } *off += n; if len == *off { trace!("write: finished writing {} bytes", len); - self.write_state = WriteState::Init + this.write_state = WriteState::Init } } WriteState::Eof => { trace!("write: eof"); - return Err(io::ErrorKind::WriteZero.into()) + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } - WriteState::EncErr => return Err(io::ErrorKind::InvalidData.into()) + WriteState::EncErr => return Poll::Ready(Err(io::ErrorKind::InvalidData.into())) } } } - fn flush(&mut self) -> io::Result<()> { - let buffer = self.buffer.borrow_mut(); + fn poll_flush( + mut self: Pin<&mut Self>, + cx: &mut Context<'_> + ) -> Poll> { + let mut this = self.deref_mut(); + + let buffer = this.buffer.borrow_mut(); + loop { - match self.write_state { - WriteState::Init => return self.io.flush(), + match this.write_state { + WriteState::Init => return Pin::new(&mut this.io).poll_flush(cx), WriteState::BufferData { off } => { trace!("flush: encrypting {} bytes", off); - if let Ok(n) = self.session.write_message(&buffer.write[.. off], buffer.write_crypto) { - trace!("flush: cipher text len = {} bytes", n); - self.write_state = WriteState::WriteLen { - len: n, - buf: u16::to_be_bytes(n as u16), - off: 0 + match this.session.write_message(&buffer.write[.. off], buffer.write_crypto) { + Ok(n) => { + trace!("flush: cipher text len = {} bytes", n); + this.write_state = WriteState::WriteLen { + len: n, + buf: u16::to_be_bytes(n as u16), + off: 0 + } + } + Err(e) => { + debug!("encryption error: {:?}", e); + this.write_state = WriteState::EncErr; + return Poll::Ready(Err(io::ErrorKind::InvalidData.into())) } - } else { - debug!("encryption error"); - self.write_state = WriteState::EncErr; - return Err(io::ErrorKind::InvalidData.into()) } } WriteState::WriteLen { len, mut buf, mut off } => { trace!("flush: writing len ({}, {:?}, {}/2)", len, buf, off); - match write_frame_len(&mut self.io, &mut buf, &mut off) { - Ok(true) => (), - Ok(false) => { + match write_frame_len(&mut this.io, cx, &mut buf, &mut off) { + Poll::Ready(Ok(true)) => (), + Poll::Ready(Ok(false)) => { trace!("write: eof"); - self.write_state = WriteState::Eof; - return Err(io::ErrorKind::WriteZero.into()) + this.write_state = WriteState::Eof; + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } - Err(e) => { - if e.kind() == io::ErrorKind::WouldBlock { - // Preserve write state - self.write_state = WriteState::WriteLen { len, buf, off }; - } - return Err(e) + Poll::Ready(Err(e)) => { + return Poll::Ready(Err(e)) + } + Poll::Pending => { + this.write_state = WriteState::WriteLen { len, buf, off }; + + return Poll::Pending } } - self.write_state = WriteState::WriteData { len, off: 0 } + this.write_state = WriteState::WriteData { len, off: 0 } } WriteState::WriteData { len, ref mut off } => { - let n = self.io.write(&buffer.write_crypto[*off .. len])?; + let n = match ready!( + Pin::new(&mut this.io).poll_write(cx, &buffer.write_crypto[*off .. len]) + ) { + Ok(n) => n, + Err(e) => return Poll::Ready(Err(e)), + }; trace!("flush: wrote {}/{} bytes", *off + n, len); if n == 0 { trace!("flush: eof"); - self.write_state = WriteState::Eof; - return Err(io::ErrorKind::WriteZero.into()) + this.write_state = WriteState::Eof; + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } *off += n; if len == *off { trace!("flush: finished writing {} bytes", len); - self.write_state = WriteState::Init; + this.write_state = WriteState::Init; } } WriteState::Eof => { trace!("flush: eof"); - return Err(io::ErrorKind::WriteZero.into()) + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())) } - WriteState::EncErr => return Err(io::ErrorKind::InvalidData.into()) + WriteState::EncErr => return Poll::Ready(Err(io::ErrorKind::InvalidData.into())) } } } -} -impl AsyncRead for NoiseOutput { - unsafe fn prepare_uninitialized_buffer(&self, _: &mut [u8]) -> bool { - false - } -} - -impl AsyncWrite for NoiseOutput { - fn shutdown(&mut self) -> Poll<(), io::Error> { - match io::Write::flush(self) { - Ok(_) => self.io.shutdown(), - Err(ref e) if e.kind() == io::ErrorKind::WouldBlock => Ok(Async::NotReady), - Err(e) => Err(e), - } + fn poll_close( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>{ + ready!(self.as_mut().poll_flush(cx))?; + Pin::new(&mut self.io).poll_close(cx) } } @@ -397,17 +438,26 @@ impl AsyncWrite for NoiseOutput { /// for the next invocation. /// /// Returns `None` if EOF has been encountered. -fn read_frame_len(io: &mut R, buf: &mut [u8; 2], off: &mut usize) - -> io::Result> -{ +fn read_frame_len( + mut io: &mut R, + cx: &mut Context<'_>, + buf: &mut [u8; 2], + off: &mut usize, +) -> Poll, std::io::Error>> { loop { - let n = io.read(&mut buf[*off ..])?; - if n == 0 { - return Ok(None) - } - *off += n; - if *off == 2 { - return Ok(Some(u16::from_be_bytes(*buf))) + match ready!(Pin::new(&mut io).poll_read(cx, &mut buf[*off ..])) { + Ok(n) => { + if n == 0 { + return Poll::Ready(Ok(None)); + } + *off += n; + if *off == 2 { + return Poll::Ready(Ok(Some(u16::from_be_bytes(*buf)))); + } + }, + Err(e) => { + return Poll::Ready(Err(e)); + }, } } } @@ -421,18 +471,26 @@ fn read_frame_len(io: &mut R, buf: &mut [u8; 2], off: &mut usize) /// be preserved for the next invocation. /// /// Returns `false` if EOF has been encountered. -fn write_frame_len(io: &mut W, buf: &[u8; 2], off: &mut usize) - -> io::Result -{ +fn write_frame_len( + mut io: &mut W, + cx: &mut Context<'_>, + buf: &[u8; 2], + off: &mut usize, +) -> Poll> { loop { - let n = io.write(&buf[*off ..])?; - if n == 0 { - return Ok(false) - } - *off += n; - if *off == 2 { - return Ok(true) + match ready!(Pin::new(&mut io).poll_write(cx, &buf[*off ..])) { + Ok(n) => { + if n == 0 { + return Poll::Ready(Ok(false)) + } + *off += n; + if *off == 2 { + return Poll::Ready(Ok(true)) + } + } + Err(e) => { + return Poll::Ready(Err(e)); + } } } } - diff --git a/protocols/noise/src/io/handshake.rs b/protocols/noise/src/io/handshake.rs index bdb40981f37..504b0118b64 100644 --- a/protocols/noise/src/io/handshake.rs +++ b/protocols/noise/src/io/handshake.rs @@ -26,30 +26,13 @@ use crate::error::NoiseError; use crate::protocol::{Protocol, PublicKey, KeypairIdentity}; use crate::io::SnowState; use libp2p_core::identity; -use futures::{future, Async, Future, future::FutureResult, Poll}; -use std::{mem, io}; -use tokio_io::{io as nio, AsyncWrite, AsyncRead}; +use futures::prelude::*; +use futures::task; +use futures::io::AsyncReadExt; use protobuf::Message; - +use std::{pin::Pin, task::Context}; use super::NoiseOutput; -/// A future performing a Noise handshake pattern. -pub struct Handshake( - Box as Future>::Item, - Error = as Future>::Error - > + Send> -); - -impl Future for Handshake { - type Error = NoiseError; - type Item = (RemoteIdentity, NoiseOutput); - - fn poll(&mut self) -> Poll { - self.0.poll() - } -} - /// The identity of the remote established during a handshake. pub enum RemoteIdentity { /// The remote provided no identifying information. @@ -105,133 +88,162 @@ pub enum IdentityExchange { None { remote: identity::PublicKey } } -impl Handshake +/// A future performing a Noise handshake pattern. +pub struct Handshake( + Pin, NoiseOutput), NoiseError>, + > + Send>> +); + +impl Future for Handshake { + type Output = Result<(RemoteIdentity, NoiseOutput), NoiseError>; + + fn poll(mut self: Pin<&mut Self>, ctx: &mut Context<'_>) -> task::Poll { + Pin::new(&mut self.0).poll(ctx) + } +} + +/// Creates an authenticated Noise handshake for the initiator of a +/// single roundtrip (2 message) handshake pattern. +/// +/// Subject to the chosen [`IdentityExchange`], this message sequence +/// identifies the local node to the remote with the first message payload +/// (i.e. unencrypted) and expects the remote to identify itself in the +/// second message payload. +/// +/// This message sequence is suitable for authenticated 2-message Noise handshake +/// patterns where the static keys of the initiator and responder are either +/// known (i.e. appear in the pre-message pattern) or are sent with +/// the first and second message, respectively (e.g. `IK` or `IX`). +/// +/// ```raw +/// initiator -{id}-> responder +/// initiator <-{id}- responder +/// ``` +pub fn rt1_initiator( + io: T, + session: Result, + identity: KeypairIdentity, + identity_x: IdentityExchange +) -> Handshake where - T: AsyncRead + AsyncWrite + Send + 'static, - C: Protocol + AsRef<[u8]> + Send + 'static, + T: AsyncWrite + AsyncRead + Send + Unpin + 'static, + C: Protocol + AsRef<[u8]> { - /// Creates an authenticated Noise handshake for the initiator of a - /// single roundtrip (2 message) handshake pattern. - /// - /// Subject to the chosen [`IdentityExchange`], this message sequence - /// identifies the local node to the remote with the first message payload - /// (i.e. unencrypted) and expects the remote to identify itself in the - /// second message payload. - /// - /// This message sequence is suitable for authenticated 2-message Noise handshake - /// patterns where the static keys of the initiator and responder are either - /// known (i.e. appear in the pre-message pattern) or are sent with - /// the first and second message, respectively (e.g. `IK` or `IX`). - /// - /// ```raw - /// initiator -{id}-> responder - /// initiator <-{id}- responder - /// ``` - pub fn rt1_initiator( - io: T, - session: Result, - identity: KeypairIdentity, - identity_x: IdentityExchange - ) -> Handshake { - Handshake(Box::new( - State::new(io, session, identity, identity_x) - .and_then(State::send_identity) - .and_then(State::recv_identity) - .and_then(State::finish))) - } + Handshake(Box::pin(async move { + let mut state = State::new(io, session, identity, identity_x)?; + send_identity(&mut state).await?; + recv_identity(&mut state).await?; + state.finish() + })) +} - /// Creates an authenticated Noise handshake for the responder of a - /// single roundtrip (2 message) handshake pattern. - /// - /// Subject to the chosen [`IdentityExchange`], this message sequence expects the - /// remote to identify itself in the first message payload (i.e. unencrypted) - /// and identifies the local node to the remote in the second message payload. - /// - /// This message sequence is suitable for authenticated 2-message Noise handshake - /// patterns where the static keys of the initiator and responder are either - /// known (i.e. appear in the pre-message pattern) or are sent with the first - /// and second message, respectively (e.g. `IK` or `IX`). - /// - /// ```raw - /// initiator -{id}-> responder - /// initiator <-{id}- responder - /// ``` - pub fn rt1_responder( - io: T, - session: Result, - identity: KeypairIdentity, - identity_x: IdentityExchange, - ) -> Handshake { - Handshake(Box::new( - State::new(io, session, identity, identity_x) - .and_then(State::recv_identity) - .and_then(State::send_identity) - .and_then(State::finish))) - } +/// Creates an authenticated Noise handshake for the responder of a +/// single roundtrip (2 message) handshake pattern. +/// +/// Subject to the chosen [`IdentityExchange`], this message sequence expects the +/// remote to identify itself in the first message payload (i.e. unencrypted) +/// and identifies the local node to the remote in the second message payload. +/// +/// This message sequence is suitable for authenticated 2-message Noise handshake +/// patterns where the static keys of the initiator and responder are either +/// known (i.e. appear in the pre-message pattern) or are sent with the first +/// and second message, respectively (e.g. `IK` or `IX`). +/// +/// ```raw +/// initiator -{id}-> responder +/// initiator <-{id}- responder +/// ``` +pub fn rt1_responder( + io: T, + session: Result, + identity: KeypairIdentity, + identity_x: IdentityExchange, +) -> Handshake +where + T: AsyncWrite + AsyncRead + Send + Unpin + 'static, + C: Protocol + AsRef<[u8]> +{ + Handshake(Box::pin(async move { + let mut state = State::new(io, session, identity, identity_x)?; + recv_identity(&mut state).await?; + send_identity(&mut state).await?; + state.finish() + })) +} - /// Creates an authenticated Noise handshake for the initiator of a - /// 1.5-roundtrip (3 message) handshake pattern. - /// - /// Subject to the chosen [`IdentityExchange`], this message sequence expects - /// the remote to identify itself in the second message payload and - /// identifies the local node to the remote in the third message payload. - /// The first (unencrypted) message payload is always empty. - /// - /// This message sequence is suitable for authenticated 3-message Noise handshake - /// patterns where the static keys of the responder and initiator are either known - /// (i.e. appear in the pre-message pattern) or are sent with the second and third - /// message, respectively (e.g. `XX`). - /// - /// ```raw - /// initiator --{}--> responder - /// initiator <-{id}- responder - /// initiator -{id}-> responder - /// ``` - pub fn rt15_initiator( - io: T, - session: Result, - identity: KeypairIdentity, - identity_x: IdentityExchange - ) -> Handshake { - Handshake(Box::new( - State::new(io, session, identity, identity_x) - .and_then(State::send_empty) - .and_then(State::recv_identity) - .and_then(State::send_identity) - .and_then(State::finish))) - } +/// Creates an authenticated Noise handshake for the initiator of a +/// 1.5-roundtrip (3 message) handshake pattern. +/// +/// Subject to the chosen [`IdentityExchange`], this message sequence expects +/// the remote to identify itself in the second message payload and +/// identifies the local node to the remote in the third message payload. +/// The first (unencrypted) message payload is always empty. +/// +/// This message sequence is suitable for authenticated 3-message Noise handshake +/// patterns where the static keys of the responder and initiator are either known +/// (i.e. appear in the pre-message pattern) or are sent with the second and third +/// message, respectively (e.g. `XX`). +/// +/// ```raw +/// initiator --{}--> responder +/// initiator <-{id}- responder +/// initiator -{id}-> responder +/// ``` +pub fn rt15_initiator( + io: T, + session: Result, + identity: KeypairIdentity, + identity_x: IdentityExchange +) -> Handshake +where + T: AsyncWrite + AsyncRead + Unpin + Send + 'static, + C: Protocol + AsRef<[u8]> +{ + Handshake(Box::pin(async move { + let mut state = State::new(io, session, identity, identity_x)?; + send_empty(&mut state).await?; + recv_identity(&mut state).await?; + send_identity(&mut state).await?; + state.finish() + })) +} - /// Creates an authenticated Noise handshake for the responder of a - /// 1.5-roundtrip (3 message) handshake pattern. - /// - /// Subject to the chosen [`IdentityExchange`], this message sequence - /// identifies the local node in the second message payload and expects - /// the remote to identify itself in the third message payload. The first - /// (unencrypted) message payload is always empty. - /// - /// This message sequence is suitable for authenticated 3-message Noise handshake - /// patterns where the static keys of the responder and initiator are either known - /// (i.e. appear in the pre-message pattern) or are sent with the second and third - /// message, respectively (e.g. `XX`). - /// - /// ```raw - /// initiator --{}--> responder - /// initiator <-{id}- responder - /// initiator -{id}-> responder - /// ``` - pub fn rt15_responder( - io: T, - session: Result, - identity: KeypairIdentity, - identity_x: IdentityExchange - ) -> Handshake { - Handshake(Box::new( - State::new(io, session, identity, identity_x) - .and_then(State::recv_empty) - .and_then(State::send_identity) - .and_then(State::recv_identity) - .and_then(State::finish))) - } +/// Creates an authenticated Noise handshake for the responder of a +/// 1.5-roundtrip (3 message) handshake pattern. +/// +/// Subject to the chosen [`IdentityExchange`], this message sequence +/// identifies the local node in the second message payload and expects +/// the remote to identify itself in the third message payload. The first +/// (unencrypted) message payload is always empty. +/// +/// This message sequence is suitable for authenticated 3-message Noise handshake +/// patterns where the static keys of the responder and initiator are either known +/// (i.e. appear in the pre-message pattern) or are sent with the second and third +/// message, respectively (e.g. `XX`). +/// +/// ```raw +/// initiator --{}--> responder +/// initiator <-{id}- responder +/// initiator -{id}-> responder +/// ``` +pub fn rt15_responder( + io: T, + session: Result, + identity: KeypairIdentity, + identity_x: IdentityExchange +) -> Handshake +where + T: AsyncWrite + AsyncRead + Unpin + Send + 'static, + C: Protocol + AsRef<[u8]> +{ + Handshake(Box::pin(async move { + let mut state = State::new(io, session, identity, identity_x)?; + recv_empty(&mut state).await?; + send_identity(&mut state).await?; + recv_identity(&mut state).await?; + state.finish() + })) } ////////////////////////////////////////////////////////////////////////////// @@ -252,36 +264,6 @@ struct State { send_identity: bool, } -impl io::Read for State { - fn read(&mut self, buf: &mut [u8]) -> io::Result { - self.io.read(buf) - } -} - -impl io::Write for State { - fn write(&mut self, buf: &[u8]) -> io::Result { - self.io.write(buf) - } - fn flush(&mut self) -> io::Result<()> { - self.io.flush() - } -} - -impl AsyncRead for State { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.io.prepare_uninitialized_buffer(buf) - } - fn read_buf(&mut self, buf: &mut B) -> Poll { - self.io.read_buf(buf) - } -} - -impl AsyncWrite for State { - fn shutdown(&mut self) -> Poll<(), io::Error> { - self.io.shutdown() - } -} - impl State { /// Initializes the state for a new Noise handshake, using the given local /// identity keypair and local DH static public key. The handshake messages @@ -293,14 +275,14 @@ impl State { session: Result, identity: KeypairIdentity, identity_x: IdentityExchange - ) -> FutureResult { + ) -> Result { let (id_remote_pubkey, send_identity) = match identity_x { IdentityExchange::Mutual => (None, true), IdentityExchange::Send { remote } => (Some(remote), true), IdentityExchange::Receive => (None, false), IdentityExchange::None { remote } => (Some(remote), false) }; - future::result(session.map(|s| + session.map(|s| State { identity, io: NoiseOutput::new(io, SnowState::Handshake(s)), @@ -308,7 +290,7 @@ impl State { id_remote_pubkey, send_identity } - )) + ) } } @@ -316,19 +298,19 @@ impl State { /// Finish a handshake, yielding the established remote identity and the /// [`NoiseOutput`] for communicating on the encrypted channel. - fn finish(self) -> FutureResult<(RemoteIdentity, NoiseOutput), NoiseError> + fn finish(self) -> Result<(RemoteIdentity, NoiseOutput), NoiseError> where C: Protocol + AsRef<[u8]> { let dh_remote_pubkey = match self.io.session.get_remote_static() { None => None, Some(k) => match C::public_from_bytes(k) { - Err(e) => return future::err(e), + Err(e) => return Err(e), Ok(dh_pk) => Some(dh_pk) } }; match self.io.session.into_transport_mode() { - Err(e) => future::err(e.into()), + Err(e) => Err(e.into()), Ok(s) => { let remote = match (self.id_remote_pubkey, dh_remote_pubkey) { (_, None) => RemoteIdentity::Unknown, @@ -337,258 +319,85 @@ impl State if C::verify(&id_pk, &dh_pk, &self.dh_remote_pubkey_sig) { RemoteIdentity::IdentityKey(id_pk) } else { - return future::err(NoiseError::InvalidKey) + return Err(NoiseError::InvalidKey) } } }; - future::ok((remote, NoiseOutput { session: SnowState::Transport(s), .. self.io })) + Ok((remote, NoiseOutput { session: SnowState::Transport(s), .. self.io })) } } } } -impl State { - /// Creates a future that sends a Noise handshake message with an empty payload. - fn send_empty(self) -> SendEmpty { - SendEmpty { state: SendState::Write(self) } - } - - /// Creates a future that expects to receive a Noise handshake message with an empty payload. - fn recv_empty(self) -> RecvEmpty { - RecvEmpty { state: RecvState::Read(self) } - } - - /// Creates a future that sends a Noise handshake message with a payload identifying - /// the local node to the remote. - fn send_identity(self) -> SendIdentity { - SendIdentity { state: SendIdentityState::Init(self) } - } - - /// Creates a future that expects to receive a Noise handshake message with a - /// payload identifying the remote. - fn recv_identity(self) -> RecvIdentity { - RecvIdentity { state: RecvIdentityState::Init(self) } - } -} - ////////////////////////////////////////////////////////////////////////////// // Handshake Message Futures -// RecvEmpty ----------------------------------------------------------------- - /// A future for receiving a Noise handshake message with an empty payload. -/// -/// Obtained from [`Handshake::recv_empty`]. -struct RecvEmpty { - state: RecvState -} - -enum RecvState { - Read(State), - Done -} - -impl Future for RecvEmpty +async fn recv_empty(state: &mut State) -> Result<(), NoiseError> where - T: AsyncRead + T: AsyncRead + Unpin { - type Error = NoiseError; - type Item = State; - - fn poll(&mut self) -> Poll { - match mem::replace(&mut self.state, RecvState::Done) { - RecvState::Read(mut st) => { - if !st.io.poll_read(&mut [])?.is_ready() { - self.state = RecvState::Read(st); - return Ok(Async::NotReady) - } - Ok(Async::Ready(st)) - }, - RecvState::Done => panic!("RecvEmpty polled after completion") - } - } + state.io.read(&mut []).await?; + Ok(()) } -// SendEmpty ----------------------------------------------------------------- - /// A future for sending a Noise handshake message with an empty payload. -/// -/// Obtained from [`Handshake::send_empty`]. -struct SendEmpty { - state: SendState -} - -enum SendState { - Write(State), - Flush(State), - Done -} - -impl Future for SendEmpty +async fn send_empty(state: &mut State) -> Result<(), NoiseError> where - T: AsyncWrite + T: AsyncWrite + Unpin { - type Error = NoiseError; - type Item = State; - - fn poll(&mut self) -> Poll { - loop { - match mem::replace(&mut self.state, SendState::Done) { - SendState::Write(mut st) => { - if !st.io.poll_write(&mut [])?.is_ready() { - self.state = SendState::Write(st); - return Ok(Async::NotReady) - } - self.state = SendState::Flush(st); - }, - SendState::Flush(mut st) => { - if !st.io.poll_flush()?.is_ready() { - self.state = SendState::Flush(st); - return Ok(Async::NotReady) - } - return Ok(Async::Ready(st)) - } - SendState::Done => panic!("SendEmpty polled after completion") - } - } - } + state.io.write(&[]).await?; + state.io.flush().await?; + Ok(()) } -// RecvIdentity -------------------------------------------------------------- - /// A future for receiving a Noise handshake message with a payload /// identifying the remote. -/// -/// Obtained from [`Handshake::recv_identity`]. -struct RecvIdentity { - state: RecvIdentityState -} - -enum RecvIdentityState { - Init(State), - ReadPayloadLen(nio::ReadExact, [u8; 2]>), - ReadPayload(nio::ReadExact, Vec>), - Done -} - -impl Future for RecvIdentity +async fn recv_identity(state: &mut State) -> Result<(), NoiseError> where - T: AsyncRead, + T: AsyncRead + Unpin, { - type Error = NoiseError; - type Item = State; - - fn poll(&mut self) -> Poll { - loop { - match mem::replace(&mut self.state, RecvIdentityState::Done) { - RecvIdentityState::Init(st) => { - self.state = RecvIdentityState::ReadPayloadLen(nio::read_exact(st, [0, 0])); - }, - RecvIdentityState::ReadPayloadLen(mut read_len) => { - if let Async::Ready((st, bytes)) = read_len.poll()? { - let len = u16::from_be_bytes(bytes) as usize; - let buf = vec![0; len]; - self.state = RecvIdentityState::ReadPayload(nio::read_exact(st, buf)); - } else { - self.state = RecvIdentityState::ReadPayloadLen(read_len); - return Ok(Async::NotReady); - } - }, - RecvIdentityState::ReadPayload(mut read_payload) => { - if let Async::Ready((mut st, bytes)) = read_payload.poll()? { - let pb: payload_proto::Identity = protobuf::parse_from_bytes(&bytes)?; - if !pb.pubkey.is_empty() { - let pk = identity::PublicKey::from_protobuf_encoding(pb.get_pubkey()) - .map_err(|_| NoiseError::InvalidKey)?; - if let Some(ref k) = st.id_remote_pubkey { - if k != &pk { - return Err(NoiseError::InvalidKey) - } - } - st.id_remote_pubkey = Some(pk); - } - if !pb.signature.is_empty() { - st.dh_remote_pubkey_sig = Some(pb.signature) - } - return Ok(Async::Ready(st)) - } else { - self.state = RecvIdentityState::ReadPayload(read_payload); - return Ok(Async::NotReady) - } - }, - RecvIdentityState::Done => panic!("RecvIdentity polled after completion") + let mut len_buf = [0,0]; + state.io.read_exact(&mut len_buf).await?; + let len = u16::from_be_bytes(len_buf) as usize; + + let mut payload_buf = vec![0; len]; + state.io.read_exact(&mut payload_buf).await?; + let pb: payload_proto::Identity = protobuf::parse_from_bytes(&payload_buf)?; + + if !pb.pubkey.is_empty() { + let pk = identity::PublicKey::from_protobuf_encoding(pb.get_pubkey()) + .map_err(|_| NoiseError::InvalidKey)?; + if let Some(ref k) = state.id_remote_pubkey { + if k != &pk { + return Err(NoiseError::InvalidKey) } } + state.id_remote_pubkey = Some(pk); + } + if !pb.signature.is_empty() { + state.dh_remote_pubkey_sig = Some(pb.signature); } -} - -// SendIdentity -------------------------------------------------------------- - -/// A future for sending a Noise handshake message with a payload -/// identifying the local node to the remote. -/// -/// Obtained from [`Handshake::send_identity`]. -struct SendIdentity { - state: SendIdentityState -} -enum SendIdentityState { - Init(State), - WritePayloadLen(nio::WriteAll, [u8; 2]>, Vec), - WritePayload(nio::WriteAll, Vec>), - Flush(State), - Done + Ok(()) } -impl Future for SendIdentity +/// Send a Noise handshake message with a payload identifying the local node to the remote. +async fn send_identity(state: &mut State) -> Result<(), NoiseError> where - T: AsyncWrite, + T: AsyncWrite + Unpin, { - type Error = NoiseError; - type Item = State; - - fn poll(&mut self) -> Poll { - loop { - match mem::replace(&mut self.state, SendIdentityState::Done) { - SendIdentityState::Init(st) => { - let mut pb = payload_proto::Identity::new(); - if st.send_identity { - pb.set_pubkey(st.identity.public.clone().into_protobuf_encoding()); - } - if let Some(ref sig) = st.identity.signature { - pb.set_signature(sig.clone()); - } - let pb_bytes = pb.write_to_bytes()?; - let len = (pb_bytes.len() as u16).to_be_bytes(); - let write_len = nio::write_all(st, len); - self.state = SendIdentityState::WritePayloadLen(write_len, pb_bytes); - }, - SendIdentityState::WritePayloadLen(mut write_len, payload) => { - if let Async::Ready((st, _)) = write_len.poll()? { - self.state = SendIdentityState::WritePayload(nio::write_all(st, payload)); - } else { - self.state = SendIdentityState::WritePayloadLen(write_len, payload); - return Ok(Async::NotReady) - } - }, - SendIdentityState::WritePayload(mut write_payload) => { - if let Async::Ready((st, _)) = write_payload.poll()? { - self.state = SendIdentityState::Flush(st); - } else { - self.state = SendIdentityState::WritePayload(write_payload); - return Ok(Async::NotReady) - } - }, - SendIdentityState::Flush(mut st) => { - if !st.poll_flush()?.is_ready() { - self.state = SendIdentityState::Flush(st); - return Ok(Async::NotReady) - } - return Ok(Async::Ready(st)) - }, - SendIdentityState::Done => panic!("SendIdentity polled after completion") - } - } + let mut pb = payload_proto::Identity::new(); + if state.send_identity { + pb.set_pubkey(state.identity.public.clone().into_protobuf_encoding()); } + if let Some(ref sig) = state.identity.signature { + pb.set_signature(sig.clone()); + } + let pb_bytes = pb.write_to_bytes()?; + let len = (pb_bytes.len() as u16).to_be_bytes(); + state.io.write_all(&len).await?; + state.io.write_all(&pb_bytes).await?; + state.io.flush().await?; + Ok(()) } - diff --git a/protocols/noise/src/lib.rs b/protocols/noise/src/lib.rs index 9805aefb373..e5a54d2049a 100644 --- a/protocols/noise/src/lib.rs +++ b/protocols/noise/src/lib.rs @@ -25,11 +25,11 @@ //! //! This crate provides `libp2p_core::InboundUpgrade` and `libp2p_core::OutboundUpgrade` //! implementations for various noise handshake patterns (currently `IK`, `IX`, and `XX`) -//! over a particular choice of DH key agreement (currently only X25519). +//! over a particular choice of Diffie–Hellman key agreement (currently only X25519). //! //! All upgrades produce as output a pair, consisting of the remote's static public key //! and a `NoiseOutput` which represents the established cryptographic session with the -//! remote, implementing `tokio_io::AsyncRead` and `tokio_io::AsyncWrite`. +//! remote, implementing `futures::io::AsyncRead` and `futures::io::AsyncWrite`. //! //! # Usage //! @@ -57,13 +57,14 @@ mod protocol; pub use error::NoiseError; pub use io::NoiseOutput; +pub use io::handshake; pub use io::handshake::{Handshake, RemoteIdentity, IdentityExchange}; pub use protocol::{Keypair, AuthenticKeypair, KeypairIdentity, PublicKey, SecretKey}; pub use protocol::{Protocol, ProtocolParams, x25519::X25519, IX, IK, XX}; -use futures::{future::{self, FutureResult}, Future}; -use libp2p_core::{identity, PeerId, UpgradeInfo, InboundUpgrade, OutboundUpgrade, Negotiated}; -use tokio_io::{AsyncRead, AsyncWrite}; +use futures::prelude::*; +use libp2p_core::{identity, PeerId, UpgradeInfo, InboundUpgrade, OutboundUpgrade}; +use std::pin::Pin; use zeroize::Zeroize; /// The protocol upgrade configuration. @@ -158,19 +159,19 @@ where impl InboundUpgrade for NoiseConfig where NoiseConfig: UpgradeInfo, - T: AsyncRead + AsyncWrite + Send + 'static, + T: AsyncRead + AsyncWrite + Unpin + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (RemoteIdentity, NoiseOutput>); + type Output = (RemoteIdentity, NoiseOutput); type Error = NoiseError; - type Future = Handshake, C>; + type Future = Handshake; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future { let session = self.params.into_builder() .local_private_key(self.dh_keys.secret().as_ref()) .build_responder() .map_err(NoiseError::from); - Handshake::rt1_responder(socket, session, + handshake::rt1_responder(socket, session, self.dh_keys.into_identity(), IdentityExchange::Mutual) } @@ -179,21 +180,21 @@ where impl OutboundUpgrade for NoiseConfig where NoiseConfig: UpgradeInfo, - T: AsyncRead + AsyncWrite + Send + 'static, + T: AsyncRead + AsyncWrite + Unpin + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (RemoteIdentity, NoiseOutput>); + type Output = (RemoteIdentity, NoiseOutput); type Error = NoiseError; - type Future = Handshake, C>; + type Future = Handshake; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future { let session = self.params.into_builder() .local_private_key(self.dh_keys.secret().as_ref()) .build_initiator() .map_err(NoiseError::from); - Handshake::rt1_initiator(socket, session, - self.dh_keys.into_identity(), - IdentityExchange::Mutual) + handshake::rt1_initiator(socket, session, + self.dh_keys.into_identity(), + IdentityExchange::Mutual) } } @@ -202,19 +203,19 @@ where impl InboundUpgrade for NoiseConfig where NoiseConfig: UpgradeInfo, - T: AsyncRead + AsyncWrite + Send + 'static, + T: AsyncRead + AsyncWrite + Unpin + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (RemoteIdentity, NoiseOutput>); + type Output = (RemoteIdentity, NoiseOutput); type Error = NoiseError; - type Future = Handshake, C>; + type Future = Handshake; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future { let session = self.params.into_builder() .local_private_key(self.dh_keys.secret().as_ref()) .build_responder() .map_err(NoiseError::from); - Handshake::rt15_responder(socket, session, + handshake::rt15_responder(socket, session, self.dh_keys.into_identity(), IdentityExchange::Mutual) } @@ -223,19 +224,19 @@ where impl OutboundUpgrade for NoiseConfig where NoiseConfig: UpgradeInfo, - T: AsyncRead + AsyncWrite + Send + 'static, + T: AsyncRead + AsyncWrite + Unpin + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (RemoteIdentity, NoiseOutput>); + type Output = (RemoteIdentity, NoiseOutput); type Error = NoiseError; - type Future = Handshake, C>; + type Future = Handshake; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future { let session = self.params.into_builder() .local_private_key(self.dh_keys.secret().as_ref()) .build_initiator() .map_err(NoiseError::from); - Handshake::rt15_initiator(socket, session, + handshake::rt15_initiator(socket, session, self.dh_keys.into_identity(), IdentityExchange::Mutual) } @@ -243,22 +244,22 @@ where // Handshake pattern IK ///////////////////////////////////////////////////// -impl InboundUpgrade for NoiseConfig +impl InboundUpgrade for NoiseConfig where - NoiseConfig: UpgradeInfo, - T: AsyncRead + AsyncWrite + Send + 'static, + NoiseConfig: UpgradeInfo, + T: AsyncRead + AsyncWrite + Unpin + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (RemoteIdentity, NoiseOutput>); + type Output = (RemoteIdentity, NoiseOutput); type Error = NoiseError; - type Future = Handshake, C>; + type Future = Handshake; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future { let session = self.params.into_builder() .local_private_key(self.dh_keys.secret().as_ref()) .build_responder() .map_err(NoiseError::from); - Handshake::rt1_responder(socket, session, + handshake::rt1_responder(socket, session, self.dh_keys.into_identity(), IdentityExchange::Receive) } @@ -267,20 +268,20 @@ where impl OutboundUpgrade for NoiseConfig, identity::PublicKey)> where NoiseConfig, identity::PublicKey)>: UpgradeInfo, - T: AsyncRead + AsyncWrite + Send + 'static, + T: AsyncRead + AsyncWrite + Unpin + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (RemoteIdentity, NoiseOutput>); + type Output = (RemoteIdentity, NoiseOutput); type Error = NoiseError; - type Future = Handshake, C>; + type Future = Handshake; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future { let session = self.params.into_builder() .local_private_key(self.dh_keys.secret().as_ref()) .remote_public_key(self.remote.0.as_ref()) .build_initiator() .map_err(NoiseError::from); - Handshake::rt1_initiator(socket, session, + handshake::rt1_initiator(socket, session, self.dh_keys.into_identity(), IdentityExchange::Send { remote: self.remote.1 }) } @@ -318,25 +319,22 @@ where impl InboundUpgrade for NoiseAuthenticated where NoiseConfig: UpgradeInfo + InboundUpgrade, NoiseOutput>), + Output = (RemoteIdentity, NoiseOutput), Error = NoiseError - >, + > + 'static, + as InboundUpgrade>::Future: Send, T: AsyncRead + AsyncWrite + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (PeerId, NoiseOutput>); + type Output = (PeerId, NoiseOutput); type Error = NoiseError; - type Future = future::AndThen< - as InboundUpgrade>::Future, - FutureResult, - fn((RemoteIdentity, NoiseOutput>)) -> FutureResult - >; - - fn upgrade_inbound(self, socket: Negotiated, info: Self::Info) -> Self::Future { - self.config.upgrade_inbound(socket, info) - .and_then(|(remote, io)| future::result(match remote { - RemoteIdentity::IdentityKey(pk) => Ok((pk.into_peer_id(), io)), - _ => Err(NoiseError::AuthenticationFailed) + type Future = Pin> + Send>>; + + fn upgrade_inbound(self, socket: T, info: Self::Info) -> Self::Future { + Box::pin(self.config.upgrade_inbound(socket, info) + .and_then(|(remote, io)| match remote { + RemoteIdentity::IdentityKey(pk) => future::ok((pk.into_peer_id(), io)), + _ => future::err(NoiseError::AuthenticationFailed) })) } } @@ -344,26 +342,22 @@ where impl OutboundUpgrade for NoiseAuthenticated where NoiseConfig: UpgradeInfo + OutboundUpgrade, NoiseOutput>), + Output = (RemoteIdentity, NoiseOutput), Error = NoiseError - >, + > + 'static, + as OutboundUpgrade>::Future: Send, T: AsyncRead + AsyncWrite + Send + 'static, C: Protocol + AsRef<[u8]> + Zeroize + Send + 'static, { - type Output = (PeerId, NoiseOutput>); + type Output = (PeerId, NoiseOutput); type Error = NoiseError; - type Future = future::AndThen< - as OutboundUpgrade>::Future, - FutureResult, - fn((RemoteIdentity, NoiseOutput>)) -> FutureResult - >; - - fn upgrade_outbound(self, socket: Negotiated, info: Self::Info) -> Self::Future { - self.config.upgrade_outbound(socket, info) - .and_then(|(remote, io)| future::result(match remote { - RemoteIdentity::IdentityKey(pk) => Ok((pk.into_peer_id(), io)), - _ => Err(NoiseError::AuthenticationFailed) + type Future = Pin> + Send>>; + + fn upgrade_outbound(self, socket: T, info: Self::Info) -> Self::Future { + Box::pin(self.config.upgrade_outbound(socket, info) + .and_then(|(remote, io)| match remote { + RemoteIdentity::IdentityKey(pk) => future::ok((pk.into_peer_id(), io)), + _ => future::err(NoiseError::AuthenticationFailed) })) } } - diff --git a/protocols/noise/src/protocol/x25519.rs b/protocols/noise/src/protocol/x25519.rs index 99fc06e5b2d..de414a36209 100644 --- a/protocols/noise/src/protocol/x25519.rs +++ b/protocols/noise/src/protocol/x25519.rs @@ -83,16 +83,7 @@ impl UpgradeInfo for NoiseConfig { } } -impl UpgradeInfo for NoiseConfig { - type Info = &'static [u8]; - type InfoIter = std::iter::Once; - - fn protocol_info(&self) -> Self::InfoIter { - std::iter::once(b"/noise/ik/25519/chachapoly/sha256/0.1.0") - } -} - -impl UpgradeInfo for NoiseConfig, identity::PublicKey)> { +impl UpgradeInfo for NoiseConfig { type Info = &'static [u8]; type InfoIter = std::iter::Once; diff --git a/protocols/noise/tests/smoke.rs b/protocols/noise/tests/smoke.rs index 2dafaaab5f1..3168e6043bb 100644 --- a/protocols/noise/tests/smoke.rs +++ b/protocols/noise/tests/smoke.rs @@ -26,7 +26,6 @@ use libp2p_noise::{Keypair, X25519, NoiseConfig, RemoteIdentity, NoiseError, Noi use libp2p_tcp::{TcpConfig, TcpTransStream}; use log::info; use quickcheck::QuickCheck; -use tokio::{self, io}; #[allow(dead_code)] fn core_upgrade_compat() { @@ -113,9 +112,9 @@ fn ik_xx() { let server_transport = TcpConfig::new() .and_then(move |output, endpoint| { if endpoint.is_listener() { - Either::A(apply_inbound(output, NoiseConfig::ik_listener(server_dh))) + Either::Left(apply_inbound(output, NoiseConfig::ik_listener(server_dh))) } else { - Either::B(apply_outbound(output, NoiseConfig::xx(server_dh), + Either::Right(apply_outbound(output, NoiseConfig::xx(server_dh), upgrade::Version::V1)) } }) @@ -126,11 +125,11 @@ fn ik_xx() { let client_transport = TcpConfig::new() .and_then(move |output, endpoint| { if endpoint.is_dialer() { - Either::A(apply_outbound(output, + Either::Left(apply_outbound(output, NoiseConfig::ik_dialer(client_dh, server_id_public, server_dh_public), upgrade::Version::V1)) } else { - Either::B(apply_inbound(output, NoiseConfig::xx(client_dh))) + Either::Right(apply_inbound(output, NoiseConfig::xx(client_dh))) } }) .and_then(move |out, _| expect_identity(out, &server_id_public2)); @@ -147,55 +146,63 @@ fn run(server_transport: T, client_transport: U, message1: Vec) where T: Transport, T::Dial: Send + 'static, - T::Listener: Send + 'static, + T::Listener: Send + Unpin + futures::stream::TryStream + 'static, T::ListenerUpgrade: Send + 'static, U: Transport, U::Dial: Send + 'static, U::Listener: Send + 'static, U::ListenerUpgrade: Send + 'static, { - let message2 = message1.clone(); - - let mut server = server_transport - .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) - .unwrap(); - - let server_address = server.by_ref().wait() - .next() - .expect("some event") - .expect("no error") - .into_new_address() - .expect("listen address"); - - let server = server.take(1) - .filter_map(ListenerEvent::into_upgrade) - .and_then(|client| client.0) - .map_err(|e| panic!("server error: {}", e)) - .and_then(|(_, client)| { + futures::executor::block_on(async { + let mut message2 = message1.clone(); + + let mut server: T::Listener = server_transport + .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) + .unwrap(); + + let server_address = server.try_next() + .await + .expect("some event") + .expect("no error") + .into_new_address() + .expect("listen address"); + + let client_fut = async { + let mut client_session = client_transport.dial(server_address.clone()) + .unwrap() + .await + .map(|(_, session)| session) + .expect("no error"); + + client_session.write_all(&mut message2).await.expect("no error"); + client_session.flush().await.expect("no error"); + }; + + let server_fut = async { + let mut server_session = server.try_next() + .await + .expect("some event") + .map(ListenerEvent::into_upgrade) + .expect("no error") + .map(|client| client.0) + .expect("listener upgrade") + .await + .map(|(_, session)| session) + .expect("no error"); + + let mut server_buffer = vec![]; info!("server: reading message"); - io::read_to_end(client, Vec::new()) - }) - .for_each(move |msg| { - assert_eq!(msg.1, message1); - Ok(()) - }); - - let client = client_transport.dial(server_address.clone()).unwrap() - .map_err(|e| panic!("client error: {}", e)) - .and_then(move |(_, server)| { - io::write_all(server, message2).and_then(|(client, _)| io::flush(client)) - }) - .map(|_| ()); - - let future = client.join(server) - .map_err(|e| panic!("{:?}", e)) - .map(|_| ()); - - tokio::run(future) + server_session.read_to_end(&mut server_buffer).await.expect("no error"); + + assert_eq!(server_buffer, message1); + }; + + futures::future::join(server_fut, client_fut).await; + }) } fn expect_identity(output: Output, pk: &identity::PublicKey) - -> impl Future + -> impl Future> { match output.0 { RemoteIdentity::IdentityKey(ref k) if k == pk => future::ok(output), diff --git a/protocols/ping/Cargo.toml b/protocols/ping/Cargo.toml index d9ab42fe1ad..61eae9df01c 100644 --- a/protocols/ping/Cargo.toml +++ b/protocols/ping/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-ping" edition = "2018" description = "Ping protocol for libp2p" -version = "0.13.1" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,21 +10,17 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4" -libp2p-core = { version = "0.13.0", path = "../../core" } -libp2p-swarm = { version = "0.3.0", path = "../../swarm" } +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } log = "0.4.1" -multiaddr = { package = "parity-multiaddr", version = "0.6.0", path = "../../misc/multiaddr" } -futures = "0.1" rand = "0.7.2" -tokio-io = "0.1" -wasm-timer = "0.1" void = "1.0" +wasm-timer = "0.2" [dev-dependencies] -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } -libp2p-secio = { version = "0.13.0", path = "../../protocols/secio" } -libp2p-yamux = { version = "0.13.0", path = "../../muxers/yamux" } +async-std = "1.0" +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../../transports/tcp" } +libp2p-secio = { version = "0.14.0-alpha.1", path = "../../protocols/secio" } +libp2p-yamux = { version = "0.14.0-alpha.1", path = "../../muxers/yamux" } quickcheck = "0.9.0" -tokio = "0.1" -tokio-tcp = "0.1" diff --git a/protocols/ping/src/handler.rs b/protocols/ping/src/handler.rs index 0c3116bf620..5ade98a8706 100644 --- a/protocols/ping/src/handler.rs +++ b/protocols/ping/src/handler.rs @@ -27,10 +27,9 @@ use libp2p_swarm::{ ProtocolsHandlerUpgrErr, ProtocolsHandlerEvent }; -use std::{error::Error, io, fmt, num::NonZeroU32, time::Duration}; +use std::{error::Error, io, fmt, num::NonZeroU32, pin::Pin, task::Context, task::Poll, time::Duration}; use std::collections::VecDeque; -use tokio_io::{AsyncRead, AsyncWrite}; -use wasm_timer::{Delay, Instant}; +use wasm_timer::Delay; use void::Void; /// The configuration for outbound pings. @@ -176,7 +175,7 @@ impl PingHandler { pub fn new(config: PingConfig) -> Self { PingHandler { config, - next_ping: Delay::new(Instant::now()), + next_ping: Delay::new(Duration::new(0, 0)), pending_results: VecDeque::with_capacity(2), failures: 0, _marker: std::marker::PhantomData @@ -186,7 +185,7 @@ impl PingHandler { impl ProtocolsHandler for PingHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Send + Unpin + 'static, { type InEvent = Void; type OutEvent = PingResult; @@ -228,36 +227,36 @@ where } } - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, cx: &mut Context) -> Poll> { if let Some(result) = self.pending_results.pop_back() { if let Ok(PingSuccess::Ping { .. }) = result { - let next_ping = Instant::now() + self.config.interval; self.failures = 0; - self.next_ping.reset(next_ping); + self.next_ping.reset(self.config.interval); } if let Err(e) = result { self.failures += 1; if self.failures >= self.config.max_failures.get() { - return Err(e) + return Poll::Ready(ProtocolsHandlerEvent::Close(e)) } else { - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(Err(e)))) + return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(e))) } } - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(result))) + return Poll::Ready(ProtocolsHandlerEvent::Custom(result)) } - match self.next_ping.poll() { - Ok(Async::Ready(())) => { - self.next_ping.reset(Instant::now() + self.config.timeout); + match Future::poll(Pin::new(&mut self.next_ping), cx) { + Poll::Ready(Ok(())) => { + self.next_ping.reset(self.config.timeout); let protocol = SubstreamProtocol::new(protocol::Ping) .with_timeout(self.config.timeout); - Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info: (), - })) + }) }, - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(e) => Err(PingFailure::Other { error: Box::new(e) }) + Poll::Pending => Poll::Pending, + Poll::Ready(Err(e)) => + Poll::Ready(ProtocolsHandlerEvent::Close(PingFailure::Other { error: Box::new(e) })) } } } @@ -266,11 +265,10 @@ where mod tests { use super::*; + use async_std::net::TcpStream; use futures::future; use quickcheck::*; use rand::Rng; - use tokio_tcp::TcpStream; - use tokio::runtime::current_thread::Runtime; impl Arbitrary for PingConfig { fn arbitrary(g: &mut G) -> PingConfig { @@ -281,11 +279,10 @@ mod tests { } } - fn tick(h: &mut PingHandler) -> Result< - ProtocolsHandlerEvent, - PingFailure - > { - Runtime::new().unwrap().block_on(future::poll_fn(|| h.poll() )) + fn tick(h: &mut PingHandler) + -> ProtocolsHandlerEvent + { + async_std::task::block_on(future::poll_fn(|cx| h.poll(cx) )) } #[test] @@ -293,34 +290,25 @@ mod tests { fn prop(cfg: PingConfig, ping_rtt: Duration) -> bool { let mut h = PingHandler::::new(cfg); - // The first ping is scheduled "immediately". - let start = h.next_ping.deadline(); - assert!(start <= Instant::now()); - // Send ping match tick(&mut h) { - Ok(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info: _ }) => { + ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info: _ } => { // The handler must use the configured timeout. assert_eq!(protocol.timeout(), &h.config.timeout); - // The next ping must be scheduled no earlier than the ping timeout. - assert!(h.next_ping.deadline() >= start + h.config.timeout); } e => panic!("Unexpected event: {:?}", e) } - let now = Instant::now(); - // Receive pong h.inject_fully_negotiated_outbound(ping_rtt, ()); match tick(&mut h) { - Ok(ProtocolsHandlerEvent::Custom(Ok(PingSuccess::Ping { rtt }))) => { + ProtocolsHandlerEvent::Custom(Ok(PingSuccess::Ping { rtt })) => { // The handler must report the given RTT. assert_eq!(rtt, ping_rtt); - // The next ping must be scheduled no earlier than the ping interval. - assert!(now + h.config.interval <= h.next_ping.deadline()); } e => panic!("Unexpected event: {:?}", e) } + true } @@ -334,20 +322,20 @@ mod tests { for _ in 0 .. h.config.max_failures.get() - 1 { h.inject_dial_upgrade_error((), ProtocolsHandlerUpgrErr::Timeout); match tick(&mut h) { - Ok(ProtocolsHandlerEvent::Custom(Err(PingFailure::Timeout))) => {} + ProtocolsHandlerEvent::Custom(Err(PingFailure::Timeout)) => {} e => panic!("Unexpected event: {:?}", e) } } h.inject_dial_upgrade_error((), ProtocolsHandlerUpgrErr::Timeout); match tick(&mut h) { - Err(PingFailure::Timeout) => { + ProtocolsHandlerEvent::Close(PingFailure::Timeout) => { assert_eq!(h.failures, h.config.max_failures.get()); } e => panic!("Unexpected event: {:?}", e) } h.inject_fully_negotiated_outbound(Duration::from_secs(1), ()); match tick(&mut h) { - Ok(ProtocolsHandlerEvent::Custom(Ok(PingSuccess::Ping { .. }))) => { + ProtocolsHandlerEvent::Custom(Ok(PingSuccess::Ping { .. })) => { // A success resets the counter for consecutive failures. assert_eq!(h.failures, 0); } diff --git a/protocols/ping/src/lib.rs b/protocols/ping/src/lib.rs index 1353ffa12a9..dbdad493c41 100644 --- a/protocols/ping/src/lib.rs +++ b/protocols/ping/src/lib.rs @@ -50,9 +50,7 @@ use handler::PingHandler; use futures::prelude::*; use libp2p_core::{ConnectedPoint, Multiaddr, PeerId}; use libp2p_swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters}; -use std::collections::VecDeque; -use std::marker::PhantomData; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{collections::VecDeque, marker::PhantomData, task::Context, task::Poll}; use void::Void; /// `Ping` is a [`NetworkBehaviour`] that responds to inbound pings and @@ -95,7 +93,7 @@ impl Default for Ping { impl NetworkBehaviour for Ping where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Send + Unpin + 'static, { type ProtocolsHandler = PingHandler; type OutEvent = PingEvent; @@ -116,12 +114,13 @@ where self.events.push_front(PingEvent { peer, result }) } - fn poll(&mut self, _: &mut impl PollParameters) -> Async> + fn poll(&mut self, _: &mut Context, _: &mut impl PollParameters) + -> Poll> { if let Some(e) = self.events.pop_back() { - Async::Ready(NetworkBehaviourAction::GenerateEvent(e)) + Poll::Ready(NetworkBehaviourAction::GenerateEvent(e)) } else { - Async::NotReady + Poll::Pending } } } diff --git a/protocols/ping/src/protocol.rs b/protocols/ping/src/protocol.rs index ffb77a28213..f7e9b90bc63 100644 --- a/protocols/ping/src/protocol.rs +++ b/protocols/ping/src/protocol.rs @@ -18,12 +18,11 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use futures::{prelude::*, future, try_ready}; -use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, upgrade::Negotiated}; +use futures::{future::BoxFuture, prelude::*}; +use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use log::debug; use rand::{distributions, prelude::*}; use std::{io, iter, time::Duration}; -use tokio_io::{io as nio, AsyncRead, AsyncWrite}; use wasm_timer::Instant; /// Represents a prototype for an upgrade to handle the ping protocol. @@ -54,126 +53,49 @@ impl UpgradeInfo for Ping { } } -type RecvPing = nio::ReadExact, [u8; 32]>; -type SendPong = nio::WriteAll, [u8; 32]>; -type Flush = nio::Flush>; -type Shutdown = nio::Shutdown>; - impl InboundUpgrade for Ping where - TSocket: AsyncRead + AsyncWrite, + TSocket: AsyncRead + AsyncWrite + Send + Unpin + 'static, { type Output = (); type Error = io::Error; - type Future = future::Map< - future::AndThen< - future::AndThen< - future::AndThen< - RecvPing, - SendPong, fn((Negotiated, [u8; 32])) -> SendPong>, - Flush, fn((Negotiated, [u8; 32])) -> Flush>, - Shutdown, fn(Negotiated) -> Shutdown>, - fn(Negotiated) -> ()>; - - #[inline] - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - nio::read_exact(socket, [0; 32]) - .and_then:: _, _>(|(sock, buf)| nio::write_all(sock, buf)) - .and_then:: _, _>(|(sock, _)| nio::flush(sock)) - .and_then:: _, _>(|sock| nio::shutdown(sock)) - .map(|_| ()) + type Future = BoxFuture<'static, Result<(), io::Error>>; + + fn upgrade_inbound(self, mut socket: TSocket, _: Self::Info) -> Self::Future { + async move { + let mut payload = [0u8; 32]; + socket.read_exact(&mut payload).await?; + socket.write_all(&payload).await?; + socket.close().await?; + Ok(()) + }.boxed() } } impl OutboundUpgrade for Ping where - TSocket: AsyncRead + AsyncWrite, + TSocket: AsyncRead + AsyncWrite + Send + Unpin + 'static, { type Output = Duration; type Error = io::Error; - type Future = PingDialer>; + type Future = BoxFuture<'static, Result>; - #[inline] - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, mut socket: TSocket, _: Self::Info) -> Self::Future { let payload: [u8; 32] = thread_rng().sample(distributions::Standard); debug!("Preparing ping payload {:?}", payload); - - PingDialer { - state: PingDialerState::Write { - inner: nio::write_all(socket, payload), - }, - } - } -} - -/// A `PingDialer` is a future that sends a ping and expects to receive a pong. -pub struct PingDialer { - state: PingDialerState -} - -enum PingDialerState { - Write { - inner: nio::WriteAll, - }, - Flush { - inner: nio::Flush, - payload: [u8; 32], - }, - Read { - inner: nio::ReadExact, - payload: [u8; 32], - started: Instant, - }, - Shutdown { - inner: nio::Shutdown, - rtt: Duration, - }, -} - -impl Future for PingDialer -where - TSocket: AsyncRead + AsyncWrite, -{ - type Item = Duration; - type Error = io::Error; - - fn poll(&mut self) -> Poll { - loop { - self.state = match self.state { - PingDialerState::Write { ref mut inner } => { - let (socket, payload) = try_ready!(inner.poll()); - PingDialerState::Flush { - inner: nio::flush(socket), - payload, - } - }, - PingDialerState::Flush { ref mut inner, payload } => { - let socket = try_ready!(inner.poll()); - let started = Instant::now(); - PingDialerState::Read { - inner: nio::read_exact(socket, [0; 32]), - payload, - started, - } - }, - PingDialerState::Read { ref mut inner, payload, started } => { - let (socket, payload_received) = try_ready!(inner.poll()); - let rtt = started.elapsed(); - if payload_received != payload { - return Err(io::Error::new( - io::ErrorKind::InvalidData, "Ping payload mismatch")); - } - PingDialerState::Shutdown { - inner: nio::shutdown(socket), - rtt, - } - }, - PingDialerState::Shutdown { ref mut inner, rtt } => { - try_ready!(inner.poll()); - return Ok(Async::Ready(rtt)); - }, + async move { + socket.write_all(&payload).await?; + socket.close().await?; + let started = Instant::now(); + + let mut recv_payload = [0u8; 32]; + socket.read_exact(&mut recv_payload).await?; + if recv_payload == payload { + Ok(started.elapsed()) + } else { + Err(io::Error::new(io::ErrorKind::InvalidData, "Ping payload mismatch")) } - } + }.boxed() } } @@ -199,31 +121,23 @@ mod tests { let mut listener = MemoryTransport.listen_on(mem_addr).unwrap(); let listener_addr = - if let Ok(Async::Ready(Some(ListenerEvent::NewAddress(a)))) = listener.poll() { + if let Some(Some(Ok(ListenerEvent::NewAddress(a)))) = listener.next().now_or_never() { a } else { panic!("MemoryTransport not listening on an address!"); }; - - let server = listener - .into_future() - .map_err(|(e, _)| e) - .and_then(|(listener_event, _)| { - let (listener_upgrade, _) = listener_event.unwrap().into_upgrade().unwrap(); - let conn = listener_upgrade.wait().unwrap(); - upgrade::apply_inbound(conn, Ping::default()) - .map_err(|e| panic!(e)) - }); - - let client = MemoryTransport.dial(listener_addr).unwrap() - .and_then(|c| { - upgrade::apply_outbound(c, Ping::default(), upgrade::Version::V1) - .map_err(|e| panic!(e)) - }); - - let mut runtime = tokio::runtime::Runtime::new().unwrap(); - runtime.spawn(server.map_err(|e| panic!(e))); - let rtt = runtime.block_on(client).expect("RTT"); - assert!(rtt > Duration::from_secs(0)); + + async_std::task::spawn(async move { + let listener_event = listener.next().await.unwrap(); + let (listener_upgrade, _) = listener_event.unwrap().into_upgrade().unwrap(); + let conn = listener_upgrade.await.unwrap(); + upgrade::apply_inbound(conn, Ping::default()).await.unwrap(); + }); + + async_std::task::block_on(async move { + let c = MemoryTransport.dial(listener_addr).unwrap().await.unwrap(); + let rtt = upgrade::apply_outbound(c, Ping::default(), upgrade::Version::V1).await.unwrap(); + assert!(rtt > Duration::from_secs(0)); + }); } } diff --git a/protocols/ping/tests/ping.rs b/protocols/ping/tests/ping.rs index 7c05ff772ee..6fb51cd80e4 100644 --- a/protocols/ping/tests/ping.rs +++ b/protocols/ping/tests/ping.rs @@ -23,20 +23,18 @@ use libp2p_core::{ Multiaddr, PeerId, - Negotiated, identity, + muxing::StreamMuxerBox, transport::{Transport, boxed::Boxed}, either::EitherError, upgrade::{self, UpgradeError} }; use libp2p_ping::*; -use libp2p_yamux::{self as yamux, Yamux}; -use libp2p_secio::{SecioConfig, SecioOutput, SecioError}; +use libp2p_secio::{SecioConfig, SecioError}; use libp2p_swarm::Swarm; -use libp2p_tcp::{TcpConfig, TcpTransStream}; -use futures::{future, prelude::*}; -use std::{io, time::Duration, sync::mpsc::sync_channel}; -use tokio::runtime::Runtime; +use libp2p_tcp::TcpConfig; +use futures::{prelude::*, channel::mpsc}; +use std::{io, time::Duration}; #[test] fn ping() { @@ -48,56 +46,45 @@ fn ping() { let (peer2_id, trans) = mk_transport(); let mut swarm2 = Swarm::new(trans, Ping::new(cfg), peer2_id.clone()); - let (tx, rx) = sync_channel::(1); + let (mut tx, mut rx) = mpsc::channel::(1); let pid1 = peer1_id.clone(); let addr = "/ip4/127.0.0.1/tcp/0".parse().unwrap(); - let mut listening = false; Swarm::listen_on(&mut swarm1, addr).unwrap(); - let peer1 = future::poll_fn(move || -> Result<_, ()> { + + let peer1 = async move { + while let Some(_) = swarm1.next().now_or_never() {} + + for l in Swarm::listeners(&swarm1) { + tx.send(l.clone()).await.unwrap(); + } + loop { - match swarm1.poll().expect("Error while polling swarm") { - Async::Ready(Some(PingEvent { peer, result })) => match result { - Ok(PingSuccess::Ping { rtt }) => - return Ok(Async::Ready((pid1.clone(), peer, rtt))), - _ => {} + match swarm1.next().await { + PingEvent { peer, result: Ok(PingSuccess::Ping { rtt }) } => { + return (pid1.clone(), peer, rtt) }, - _ => { - if !listening { - for l in Swarm::listeners(&swarm1) { - tx.send(l.clone()).unwrap(); - listening = true; - } - } - return Ok(Async::NotReady) - } + _ => {} } } - }); + }; let pid2 = peer2_id.clone(); - let mut dialing = false; - let peer2 = future::poll_fn(move || -> Result<_, ()> { + let peer2 = async move { + Swarm::dial_addr(&mut swarm2, rx.next().await.unwrap()).unwrap(); + loop { - match swarm2.poll().expect("Error while polling swarm") { - Async::Ready(Some(PingEvent { peer, result })) => match result { - Ok(PingSuccess::Ping { rtt }) => - return Ok(Async::Ready((pid2.clone(), peer, rtt))), - _ => {} + match swarm2.next().await { + PingEvent { peer, result: Ok(PingSuccess::Ping { rtt }) } => { + return (pid2.clone(), peer, rtt) }, - _ => { - if !dialing { - Swarm::dial_addr(&mut swarm2, rx.recv().unwrap()).unwrap(); - dialing = true; - } - return Ok(Async::NotReady) - } + _ => {} } } - }); + }; - let result = peer1.select(peer2).map_err(|e| panic!(e)); - let ((p1, p2, rtt), _) = Runtime::new().unwrap().block_on(result).unwrap(); + let result = future::select(Box::pin(peer1), Box::pin(peer2)); + let ((p1, p2, rtt), _) = async_std::task::block_on(result).factor_first(); assert!(p1 == peer1_id && p2 == peer2_id || p1 == peer2_id && p2 == peer1_id); assert!(rtt < Duration::from_millis(50)); } @@ -105,7 +92,7 @@ fn ping() { fn mk_transport() -> ( PeerId, Boxed< - (PeerId, Yamux>>>), + (PeerId, StreamMuxerBox), EitherError>, UpgradeError> > ) { @@ -115,8 +102,8 @@ fn mk_transport() -> ( .nodelay(true) .upgrade(upgrade::Version::V1) .authenticate(SecioConfig::new(id_keys)) - .multiplex(yamux::Config::default()) + .multiplex(libp2p_yamux::Config::default()) + .map(|(peer, muxer), _| (peer, StreamMuxerBox::new(muxer))) .boxed(); (peer_id, transport) } - diff --git a/protocols/plaintext/Cargo.toml b/protocols/plaintext/Cargo.toml index 9f5cf38c730..38d60407c68 100644 --- a/protocols/plaintext/Cargo.toml +++ b/protocols/plaintext/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-plaintext" edition = "2018" description = "Plaintext encryption dummy protocol for libp2p" -version = "0.13.1" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,11 +10,17 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -futures = "0.1.29" -libp2p-core = { version = "0.13.0", path = "../../core" } -bytes = "0.4.12" +bytes = "0.5" +futures = "0.3.1" +futures_codec = "0.3.4" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } log = "0.4.8" -void = "1.0.2" -tokio-io = "0.1.12" protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 -rw-stream-sink = { version = "0.1.1", path = "../../misc/rw-stream-sink" } +rw-stream-sink = { version = "0.2.0", path = "../../misc/rw-stream-sink" } +unsigned-varint = { version = "0.3", features = ["futures-codec"] } +void = "1.0.2" + +[dev-dependencies] +env_logger = "0.7.1" +quickcheck = "0.9.0" +rand = "0.7" diff --git a/protocols/plaintext/regen_structs_proto.sh b/protocols/plaintext/regen_structs_proto.sh index 9ac7ab8fd8d..42ecc3ecc43 100755 --- a/protocols/plaintext/regen_structs_proto.sh +++ b/protocols/plaintext/regen_structs_proto.sh @@ -1,8 +1,4 @@ #!/bin/sh -docker run --rm -v "`pwd`/../../":/usr/code:z -w /usr/code rust /bin/bash -c " \ - apt-get update; \ - apt-get install -y protobuf-compiler; \ - cargo install --version 2.3.0 protobuf-codegen; \ - protoc --rust_out=./protocols/plaintext/src/pb ./protocols/plaintext/structs.proto;" +../../scripts/protobuf/gen.sh ./src/structs.proto diff --git a/protocols/plaintext/src/handshake.rs b/protocols/plaintext/src/handshake.rs index 8b073937c62..c0b02acc5ee 100644 --- a/protocols/plaintext/src/handshake.rs +++ b/protocols/plaintext/src/handshake.rs @@ -18,21 +18,18 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use crate::PlainText2Config; +use crate::error::PlainTextError; +use crate::structs_proto::Exchange; + use bytes::BytesMut; -use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use futures::Future; -use futures::future; -use futures::sink::Sink; -use futures::stream::Stream; +use futures::prelude::*; +use futures_codec::Framed; use libp2p_core::{PublicKey, PeerId}; use log::{debug, trace}; -use crate::pb::structs::Exchange; -use tokio_io::{AsyncRead, AsyncWrite}; -use tokio_io::codec::length_delimited; -use tokio_io::codec::length_delimited::Framed; use protobuf::Message; -use crate::error::PlainTextError; -use crate::PlainText2Config; +use std::io::{Error as IoError, ErrorKind as IoErrorKind}; +use unsigned_varint::codec::UviBytes; struct HandshakeContext { config: PlainText2Config, @@ -68,7 +65,9 @@ impl HandshakeContext { }) } - fn with_remote(self, exchange_bytes: BytesMut) -> Result, PlainTextError> { + fn with_remote(self, exchange_bytes: BytesMut) + -> Result, PlainTextError> + { let mut prop = match protobuf::parse_from_bytes::(&exchange_bytes) { Ok(prop) => prop, Err(e) => { @@ -95,7 +94,7 @@ impl HandshakeContext { // Check the validity of the remote's `Exchange`. if peer_id != public_key.clone().into_peer_id() { - debug!("The remote's `PeerId` of the exchange isn't consist with the remote public key"); + debug!("the remote's `PeerId` isn't consistent with the remote's public key"); return Err(PlainTextError::InvalidPeerId) } @@ -109,45 +108,30 @@ impl HandshakeContext { } } -pub fn handshake(socket: S, config: PlainText2Config) - -> impl Future, Remote), Error = PlainTextError> +pub async fn handshake(socket: S, config: PlainText2Config) + -> Result<(Framed>, Remote), PlainTextError> where - S: AsyncRead + AsyncWrite + Send, + S: AsyncRead + AsyncWrite + Send + Unpin, { - let socket = length_delimited::Builder::new() - .big_endian() - .length_field_length(4) - .new_framed(socket); - - future::ok::<_, PlainTextError>(()) - .and_then(|_| { - trace!("starting handshake"); - Ok(HandshakeContext::new(config)?) - }) - // Send our local `Exchange`. - .and_then(|context| { - trace!("sending exchange to remote"); - socket.send(BytesMut::from(context.state.exchange_bytes.clone())) - .from_err() - .map(|s| (s, context)) - }) - // Receive the remote's `Exchange`. - .and_then(move |(socket, context)| { - trace!("receiving the remote's exchange"); - socket.into_future() - .map_err(|(e, _)| e.into()) - .and_then(move |(prop_raw, socket)| { - let context = match prop_raw { - Some(p) => context.with_remote(p)?, - None => { - debug!("unexpected eof while waiting for remote's exchange"); - let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof"); - return Err(err.into()); - } - }; - - trace!("received exchange from remote; pubkey = {:?}", context.state.public_key); - Ok((socket, context.state)) - }) - }) + // The handshake messages all start with a variable-length integer indicating the size. + let mut socket = Framed::new(socket, UviBytes::default()); + + trace!("starting handshake"); + let context = HandshakeContext::new(config)?; + + trace!("sending exchange to remote"); + socket.send(BytesMut::from(&context.state.exchange_bytes[..])).await?; + + trace!("receiving the remote's exchange"); + let context = match socket.next().await { + Some(p) => context.with_remote(p?)?, + None => { + debug!("unexpected eof while waiting for remote's exchange"); + let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof"); + return Err(err.into()); + } + }; + + trace!("received exchange from remote; pubkey = {:?}", context.state.public_key); + Ok((socket, context.state)) } diff --git a/protocols/plaintext/src/lib.rs b/protocols/plaintext/src/lib.rs index d1a8943555b..23d447d4ef0 100644 --- a/protocols/plaintext/src/lib.rs +++ b/protocols/plaintext/src/lib.rs @@ -18,26 +18,31 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use crate::error::PlainTextError; +use crate::handshake::Remote; + use bytes::BytesMut; -use futures::{Future, StartSend, Poll, future}; -use futures::sink::Sink; -use futures::stream::MapErr as StreamMapErr; -use futures::stream::Stream; -use libp2p_core::{identity, InboundUpgrade, OutboundUpgrade, UpgradeInfo, upgrade::Negotiated, PeerId, PublicKey}; +use futures::future::{self, Ready}; +use futures::prelude::*; +use futures::{future::BoxFuture, Sink, Stream}; +use futures_codec::Framed; +use libp2p_core::{ + identity, + InboundUpgrade, + OutboundUpgrade, + UpgradeInfo, + PeerId, + PublicKey, +}; use log::debug; use rw_stream_sink::RwStreamSink; -use std::io; -use std::iter; -use tokio_io::{AsyncRead, AsyncWrite}; -use tokio_io::codec::length_delimited::Framed; -use crate::error::PlainTextError; +use std::{io, iter, pin::Pin, task::{Context, Poll}}; +use unsigned_varint::codec::UviBytes; use void::Void; -use futures::future::FutureResult; -use crate::handshake::Remote; mod error; mod handshake; -mod pb; +mod structs_proto; /// `PlainText1Config` is an insecure connection handshake for testing purposes only. /// @@ -78,22 +83,22 @@ impl UpgradeInfo for PlainText1Config { } impl InboundUpgrade for PlainText1Config { - type Output = Negotiated; + type Output = C; type Error = Void; - type Future = FutureResult, Self::Error>; + type Future = Ready>; - fn upgrade_inbound(self, i: Negotiated, _: Self::Info) -> Self::Future { - future::ok(i) + fn upgrade_inbound(self, i: C, _: Self::Info) -> Self::Future { + future::ready(Ok(i)) } } impl OutboundUpgrade for PlainText1Config { - type Output = Negotiated; + type Output = C; type Error = Void; - type Future = FutureResult, Self::Error>; + type Future = Ready>; - fn upgrade_outbound(self, i: Negotiated, _: Self::Info) -> Self::Future { - future::ok(i) + fn upgrade_outbound(self, i: C, _: Self::Info) -> Self::Future { + future::ready(Ok(i)) } } @@ -115,144 +120,138 @@ impl UpgradeInfo for PlainText2Config { impl InboundUpgrade for PlainText2Config where - C: AsyncRead + AsyncWrite + Send + 'static + C: AsyncRead + AsyncWrite + Send + Unpin + 'static { - type Output = (PeerId, PlainTextOutput>); + type Output = (PeerId, PlainTextOutput); type Error = PlainTextError; - type Future = Box + Send>; + type Future = BoxFuture<'static, Result>; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - Box::new(self.handshake(socket)) + fn upgrade_inbound(self, socket: C, _: Self::Info) -> Self::Future { + Box::pin(self.handshake(socket)) } } impl OutboundUpgrade for PlainText2Config where - C: AsyncRead + AsyncWrite + Send + 'static + C: AsyncRead + AsyncWrite + Send + Unpin + 'static { - type Output = (PeerId, PlainTextOutput>); + type Output = (PeerId, PlainTextOutput); type Error = PlainTextError; - type Future = Box + Send>; + type Future = BoxFuture<'static, Result>; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - Box::new(self.handshake(socket)) + fn upgrade_outbound(self, socket: C, _: Self::Info) -> Self::Future { + Box::pin(self.handshake(socket)) } } impl PlainText2Config { - fn handshake(self, socket: T) -> impl Future), Error = PlainTextError> + async fn handshake(self, socket: T) -> Result<(PeerId, PlainTextOutput), PlainTextError> where - T: AsyncRead + AsyncWrite + Send + 'static + T: AsyncRead + AsyncWrite + Send + Unpin + 'static { debug!("Starting plaintext upgrade"); - PlainTextMiddleware::handshake(socket, self) - .map(|(stream_sink, remote)| { - let mapped = stream_sink.map_err(map_err as fn(_) -> _); - ( - remote.peer_id, - PlainTextOutput { - stream: RwStreamSink::new(mapped), - remote_key: remote.public_key, - } - ) - }) + let (stream_sink, remote) = PlainTextMiddleware::handshake(socket, self).await?; + let mapped = stream_sink.map_err(map_err as fn(_) -> _); + Ok(( + remote.peer_id, + PlainTextOutput { + stream: RwStreamSink::new(mapped), + remote_key: remote.public_key, + } + )) } } -#[inline] fn map_err(err: io::Error) -> io::Error { debug!("error during plaintext handshake {:?}", err); io::Error::new(io::ErrorKind::InvalidData, err) } pub struct PlainTextMiddleware { - inner: Framed, + inner: Framed>, } impl PlainTextMiddleware where - S: AsyncRead + AsyncWrite + Send, + S: AsyncRead + AsyncWrite + Send + Unpin, { - fn handshake(socket: S, config: PlainText2Config) - -> impl Future, Remote), Error = PlainTextError> + async fn handshake(socket: S, config: PlainText2Config) + -> Result<(PlainTextMiddleware, Remote), PlainTextError> { - handshake::handshake(socket, config).map(|(inner, remote)| { - (PlainTextMiddleware { inner }, remote) - }) + let (inner, remote) = handshake::handshake(socket, config).await?; + Ok((PlainTextMiddleware { inner }, remote)) } } -impl Sink for PlainTextMiddleware +impl Sink for PlainTextMiddleware where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin, { - type SinkItem = BytesMut; - type SinkError = io::Error; + type Error = io::Error; + + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Sink::poll_ready(Pin::new(&mut self.inner), cx) + } - #[inline] - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - self.inner.start_send(item) + fn start_send(mut self: Pin<&mut Self>, item: BytesMut) -> Result<(), Self::Error> { + Sink::start_send(Pin::new(&mut self.inner), item) } - #[inline] - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.inner.poll_complete() + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Sink::poll_flush(Pin::new(&mut self.inner), cx) } - #[inline] - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.inner.close() + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Sink::poll_close(Pin::new(&mut self.inner), cx) } } impl Stream for PlainTextMiddleware where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin, { - type Item = BytesMut; - type Error = io::Error; + type Item = Result; - #[inline] - fn poll(&mut self) -> Poll, Self::Error> { - self.inner.poll() + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Stream::poll_next(Pin::new(&mut self.inner), cx) } } /// Output of the plaintext protocol. pub struct PlainTextOutput where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin, { /// The plaintext stream. - pub stream: RwStreamSink, fn(io::Error) -> io::Error>>, + pub stream: RwStreamSink, fn(io::Error) -> io::Error>>, /// The public key of the remote. pub remote_key: PublicKey, } -impl std::io::Read for PlainTextOutput { - fn read(&mut self, buf: &mut [u8]) -> std::io::Result { - self.stream.read(buf) - } -} - -impl AsyncRead for PlainTextOutput { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.stream.prepare_uninitialized_buffer(buf) +impl AsyncRead for PlainTextOutput { + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) + -> Poll> + { + AsyncRead::poll_read(Pin::new(&mut self.stream), cx, buf) } } -impl std::io::Write for PlainTextOutput { - fn write(&mut self, buf: &[u8]) -> std::io::Result { - self.stream.write(buf) +impl AsyncWrite for PlainTextOutput { + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) + -> Poll> + { + AsyncWrite::poll_write(Pin::new(&mut self.stream), cx, buf) } - fn flush(&mut self) -> std::io::Result<()> { - self.stream.flush() + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) + -> Poll> + { + AsyncWrite::poll_flush(Pin::new(&mut self.stream), cx) } -} -impl AsyncWrite for PlainTextOutput { - fn shutdown(&mut self) -> Poll<(), io::Error> { - self.stream.shutdown() + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) + -> Poll> + { + AsyncWrite::poll_close(Pin::new(&mut self.stream), cx) } } diff --git a/protocols/plaintext/src/pb.rs b/protocols/plaintext/src/pb.rs deleted file mode 100644 index 64e83e5da73..00000000000 --- a/protocols/plaintext/src/pb.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2019 Parity Technologies (UK) 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. - -pub mod structs; diff --git a/protocols/plaintext/structs.proto b/protocols/plaintext/src/structs.proto similarity index 100% rename from protocols/plaintext/structs.proto rename to protocols/plaintext/src/structs.proto diff --git a/protocols/plaintext/src/pb/structs.rs b/protocols/plaintext/src/structs_proto.rs similarity index 81% rename from protocols/plaintext/src/pb/structs.rs rename to protocols/plaintext/src/structs_proto.rs index 85da2b49a2e..4063daea33e 100644 --- a/protocols/plaintext/src/pb/structs.rs +++ b/protocols/plaintext/src/structs_proto.rs @@ -1,9 +1,9 @@ -// This file is generated by rust-protobuf 2.3.0. Do not edit +// This file is generated by rust-protobuf 2.8.1. Do not edit // @generated // https://github.com/Manishearth/rust-clippy/issues/702 #![allow(unknown_lints)] -#![allow(clippy)] +#![allow(clippy::all)] #![cfg_attr(rustfmt, rustfmt_skip)] @@ -17,10 +17,15 @@ #![allow(unsafe_code)] #![allow(unused_imports)] #![allow(unused_results)] +//! Generated file from `src/structs.proto` use protobuf::Message as Message_imported_for_functions; use protobuf::ProtobufEnum as ProtobufEnum_imported_for_functions; +/// Generated files are compatible only with the same version +/// of protobuf runtime. +const _PROTOBUF_VERSION_CHECK: () = ::protobuf::VERSION_2_8_1; + #[derive(PartialEq,Clone,Default)] pub struct Exchange { // message fields @@ -31,6 +36,12 @@ pub struct Exchange { pub cached_size: ::protobuf::CachedSize, } +impl<'a> ::std::default::Default for &'a Exchange { + fn default() -> &'a Exchange { + ::default_instance() + } +} + impl Exchange { pub fn new() -> Exchange { ::std::default::Default::default() @@ -38,6 +49,13 @@ impl Exchange { // optional bytes id = 1; + + pub fn get_id(&self) -> &[u8] { + match self.id.as_ref() { + Some(v) => &v, + None => &[], + } + } pub fn clear_id(&mut self) { self.id.clear(); } @@ -65,15 +83,15 @@ impl Exchange { self.id.take().unwrap_or_else(|| ::std::vec::Vec::new()) } - pub fn get_id(&self) -> &[u8] { - match self.id.as_ref() { + // optional bytes pubkey = 2; + + + pub fn get_pubkey(&self) -> &[u8] { + match self.pubkey.as_ref() { Some(v) => &v, None => &[], } } - - // optional bytes pubkey = 2; - pub fn clear_pubkey(&mut self) { self.pubkey.clear(); } @@ -100,13 +118,6 @@ impl Exchange { pub fn take_pubkey(&mut self) -> ::std::vec::Vec { self.pubkey.take().unwrap_or_else(|| ::std::vec::Vec::new()) } - - pub fn get_pubkey(&self) -> &[u8] { - match self.pubkey.as_ref() { - Some(v) => &v, - None => &[], - } - } } impl ::protobuf::Message for Exchange { @@ -114,7 +125,7 @@ impl ::protobuf::Message for Exchange { true } - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> { + fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { while !is.eof()? { let (field_number, wire_type) = is.read_tag_unpack()?; match field_number { @@ -147,7 +158,7 @@ impl ::protobuf::Message for Exchange { my_size } - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> { + fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { if let Some(ref v) = self.id.as_ref() { os.write_bytes(1, &v)?; } @@ -170,13 +181,13 @@ impl ::protobuf::Message for Exchange { &mut self.unknown_fields } - fn as_any(&self) -> &::std::any::Any { - self as &::std::any::Any + fn as_any(&self) -> &dyn (::std::any::Any) { + self as &dyn (::std::any::Any) } - fn as_any_mut(&mut self) -> &mut ::std::any::Any { - self as &mut ::std::any::Any + fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { + self as &mut dyn (::std::any::Any) } - fn into_any(self: Box) -> ::std::boxed::Box<::std::any::Any> { + fn into_any(self: Box) -> ::std::boxed::Box { self } @@ -228,14 +239,14 @@ impl ::protobuf::Message for Exchange { impl ::protobuf::Clear for Exchange { fn clear(&mut self) { - self.clear_id(); - self.clear_pubkey(); + self.id.clear(); + self.pubkey.clear(); self.unknown_fields.clear(); } } impl ::std::fmt::Debug for Exchange { - fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result { + fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { ::protobuf::text_format::fmt(self, f) } } @@ -247,17 +258,17 @@ impl ::protobuf::reflect::ProtobufValue for Exchange { } static file_descriptor_proto_data: &'static [u8] = b"\ - \n!protocols/plaintext/structs.proto\"2\n\x08Exchange\x12\x0e\n\x02id\ - \x18\x01\x20\x01(\x0cR\x02id\x12\x16\n\x06pubkey\x18\x02\x20\x01(\x0cR\ - \x06pubkeyJ\xb4\x01\n\x06\x12\x04\0\0\x05\x01\n\x08\n\x01\x0c\x12\x03\0\ - \0\x12\n\n\n\x02\x04\0\x12\x04\x02\0\x05\x01\n\n\n\x03\x04\0\x01\x12\x03\ - \x02\x08\x10\n\x0b\n\x04\x04\0\x02\0\x12\x03\x03\x02\x18\n\x0c\n\x05\x04\ - \0\x02\0\x04\x12\x03\x03\x02\n\n\x0c\n\x05\x04\0\x02\0\x05\x12\x03\x03\ - \x0b\x10\n\x0c\n\x05\x04\0\x02\0\x01\x12\x03\x03\x11\x13\n\x0c\n\x05\x04\ - \0\x02\0\x03\x12\x03\x03\x16\x17\n\x0b\n\x04\x04\0\x02\x01\x12\x03\x04\ - \x02\x1c\n\x0c\n\x05\x04\0\x02\x01\x04\x12\x03\x04\x02\n\n\x0c\n\x05\x04\ - \0\x02\x01\x05\x12\x03\x04\x0b\x10\n\x0c\n\x05\x04\0\x02\x01\x01\x12\x03\ - \x04\x11\x17\n\x0c\n\x05\x04\0\x02\x01\x03\x12\x03\x04\x1a\x1b\ + \n\x11src/structs.proto\"2\n\x08Exchange\x12\x0e\n\x02id\x18\x01\x20\x01\ + (\x0cR\x02id\x12\x16\n\x06pubkey\x18\x02\x20\x01(\x0cR\x06pubkeyJ\xb4\ + \x01\n\x06\x12\x04\0\0\x05\x01\n\x08\n\x01\x0c\x12\x03\0\0\x12\n\n\n\x02\ + \x04\0\x12\x04\x02\0\x05\x01\n\n\n\x03\x04\0\x01\x12\x03\x02\x08\x10\n\ + \x0b\n\x04\x04\0\x02\0\x12\x03\x03\x02\x18\n\x0c\n\x05\x04\0\x02\0\x04\ + \x12\x03\x03\x02\n\n\x0c\n\x05\x04\0\x02\0\x05\x12\x03\x03\x0b\x10\n\x0c\ + \n\x05\x04\0\x02\0\x01\x12\x03\x03\x11\x13\n\x0c\n\x05\x04\0\x02\0\x03\ + \x12\x03\x03\x16\x17\n\x0b\n\x04\x04\0\x02\x01\x12\x03\x04\x02\x1c\n\x0c\ + \n\x05\x04\0\x02\x01\x04\x12\x03\x04\x02\n\n\x0c\n\x05\x04\0\x02\x01\x05\ + \x12\x03\x04\x0b\x10\n\x0c\n\x05\x04\0\x02\x01\x01\x12\x03\x04\x11\x17\n\ + \x0c\n\x05\x04\0\x02\x01\x03\x12\x03\x04\x1a\x1b\ "; static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy { diff --git a/protocols/plaintext/tests/smoke.rs b/protocols/plaintext/tests/smoke.rs new file mode 100644 index 00000000000..aedbda21239 --- /dev/null +++ b/protocols/plaintext/tests/smoke.rs @@ -0,0 +1,121 @@ +// Copyright 2019 Parity Technologies (UK) 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 futures::io::{AsyncWriteExt, AsyncReadExt}; +use futures::stream::TryStreamExt; +use libp2p_core::{ + identity, + multiaddr::Multiaddr, + transport::{Transport, ListenerEvent}, + upgrade, +}; +use libp2p_plaintext::PlainText2Config; +use log::debug; +use quickcheck::QuickCheck; + +#[test] +fn variable_msg_length() { + let _ = env_logger::try_init(); + + fn prop(msg: Vec) { + let mut msg_to_send = msg.clone(); + let msg_to_receive = msg; + + let server_id = identity::Keypair::generate_ed25519(); + let server_id_public = server_id.public(); + + let client_id = identity::Keypair::generate_ed25519(); + let client_id_public = client_id.public(); + + futures::executor::block_on(async { + let server_transport = libp2p_core::transport::MemoryTransport{}.and_then( + move |output, endpoint| { + upgrade::apply( + output, + PlainText2Config{local_public_key: server_id_public}, + endpoint, + libp2p_core::upgrade::Version::V1, + ) + } + ); + + let client_transport = libp2p_core::transport::MemoryTransport{}.and_then( + move |output, endpoint| { + upgrade::apply( + output, + PlainText2Config{local_public_key: client_id_public}, + endpoint, + libp2p_core::upgrade::Version::V1, + ) + } + ); + + + let server_address: Multiaddr = format!( + "/memory/{}", + std::cmp::Ord::max(1, rand::random::()) + ).parse().unwrap(); + + let mut server = server_transport.listen_on(server_address.clone()).unwrap(); + + // Ignore server listen address event. + let _ = server.try_next() + .await + .expect("some event") + .expect("no error") + .into_new_address() + .expect("listen address"); + + let client_fut = async { + debug!("dialing {:?}", server_address); + let (received_server_id, mut client_channel) = client_transport.dial(server_address).unwrap().await.unwrap(); + assert_eq!(received_server_id, server_id.public().into_peer_id()); + + debug!("Client: writing message."); + client_channel.write_all(&mut msg_to_send).await.expect("no error"); + debug!("Client: flushing channel."); + client_channel.flush().await.expect("no error"); + }; + + let server_fut = async { + let mut server_channel = server.try_next() + .await + .expect("some event") + .map(ListenerEvent::into_upgrade) + .expect("no error") + .map(|client| client.0) + .expect("listener upgrade xyz") + .await + .map(|(_, session)| session) + .expect("no error"); + + let mut server_buffer = vec![0; msg_to_receive.len()]; + debug!("Server: reading message."); + server_channel.read_exact(&mut server_buffer).await.expect("reading client message"); + + assert_eq!(server_buffer, msg_to_receive); + }; + + futures::future::join(server_fut, client_fut).await; + }) + } + + QuickCheck::new().max_tests(30).quickcheck(prop as fn(Vec)) +} diff --git a/protocols/secio/Cargo.toml b/protocols/secio/Cargo.toml index e62a86eba2f..ada91d28ce1 100644 --- a/protocols/secio/Cargo.toml +++ b/protocols/secio/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-secio" edition = "2018" description = "Secio encryption protocol for libp2p" -version = "0.13.1" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,32 +10,31 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4" -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -log = "0.4.6" -protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 -rand = "0.6.5" aes-ctr = "0.3" aesni = { version = "0.6", features = ["nocheck"], optional = true } -twofish = "0.2.0" ctr = "0.3" +futures = "0.3.1" +hmac = "0.7.0" lazy_static = "1.2.0" -rw-stream-sink = { version = "0.1.1", path = "../../misc/rw-stream-sink" } -tokio-io = "0.1.0" -tokio-codec = "0.1.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +log = "0.4.6" +pin-project = "0.4.6" +protobuf = "=2.8.1" # note: see https://github.com/libp2p/rust-libp2p/issues/1363 +quicksink = "0.1" +rand = "0.7" +rw-stream-sink = { version = "0.2.0", path = "../../misc/rw-stream-sink" } sha2 = "0.8.0" -hmac = "0.7.0" +static_assertions = "1" +twofish = "0.2.0" [target.'cfg(not(target_arch = "wasm32"))'.dependencies] ring = { version = "0.16.9", features = ["alloc"], default-features = false } -untrusted = "0.7.0" [target.'cfg(target_arch = "wasm32")'.dependencies] js-sys = "0.3.10" parity-send-wrapper = "0.1" wasm-bindgen = "0.2.33" -wasm-bindgen-futures = "0.3.10" +wasm-bindgen-futures = "0.4.5" web-sys = { version = "0.3.10", features = ["Crypto", "CryptoKey", "SubtleCrypto", "Window"] } [features] @@ -44,11 +43,10 @@ secp256k1 = [] aes-all = ["aesni"] [dev-dependencies] -criterion = "0.3.0" -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } -libp2p-mplex = { version = "0.13.0", path = "../../muxers/mplex" } -tokio = "0.1" -tokio-tcp = "0.1" +async-std = "1.0" +criterion = "0.3" +libp2p-mplex = { version = "0.14.0-alpha.1", path = "../../muxers/mplex" } +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../../transports/tcp" } [[bench]] name = "bench" diff --git a/protocols/secio/src/codec/decode.rs b/protocols/secio/src/codec/decode.rs index 4b0c73b3d7c..04bbad56b63 100644 --- a/protocols/secio/src/codec/decode.rs +++ b/protocols/secio/src/codec/decode.rs @@ -20,19 +20,14 @@ //! Individual messages decoding. -use bytes::BytesMut; use super::{Hmac, StreamCipher}; use crate::error::SecioError; -use futures::sink::Sink; -use futures::stream::Stream; -use futures::Async; -use futures::Poll; -use futures::StartSend; +use futures::prelude::*; use log::debug; -use std::cmp::min; +use std::{cmp::min, pin::Pin, task::Context, task::Poll}; -/// Wraps around a `Stream`. The buffers produced by the underlying stream +/// Wraps around a `Stream>`. The buffers produced by the underlying stream /// are decoded using the cipher and hmac. /// /// This struct implements `Stream`, whose stream item are frames of data without the length @@ -40,9 +35,11 @@ use std::cmp::min; /// frames isn't handled by this module. /// /// Also implements `Sink` for convenience. +#[pin_project::pin_project] pub struct DecoderMiddleware { cipher_state: StreamCipher, hmac: Hmac, + #[pin] raw_stream: S, nonce: Vec } @@ -52,7 +49,6 @@ impl DecoderMiddleware { /// /// The `nonce` parameter denotes a sequence of bytes which are expected to be found at the /// beginning of the stream and are checked for equality. - #[inline] pub fn new(raw_stream: S, cipher: StreamCipher, hmac: Hmac, nonce: Vec) -> DecoderMiddleware { DecoderMiddleware { cipher_state: cipher, @@ -65,73 +61,76 @@ impl DecoderMiddleware { impl Stream for DecoderMiddleware where - S: Stream, + S: TryStream>, S::Error: Into, { - type Item = Vec; - type Error = SecioError; - - #[inline] - fn poll(&mut self) -> Poll, Self::Error> { - let frame = match self.raw_stream.poll() { - Ok(Async::Ready(Some(t))) => t, - Ok(Async::Ready(None)) => return Ok(Async::Ready(None)), - Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(err) => return Err(err.into()), + type Item = Result, SecioError>; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + + let frame = match TryStream::try_poll_next(this.raw_stream, cx) { + Poll::Ready(Some(Ok(t))) => t, + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => return Poll::Pending, + Poll::Ready(Some(Err(err))) => return Poll::Ready(Some(Err(err.into()))), }; - if frame.len() < self.hmac.num_bytes() { + if frame.len() < this.hmac.num_bytes() { debug!("frame too short when decoding secio frame"); - return Err(SecioError::FrameTooShort); + return Poll::Ready(Some(Err(SecioError::FrameTooShort))); } - let content_length = frame.len() - self.hmac.num_bytes(); + let content_length = frame.len() - this.hmac.num_bytes(); { let (crypted_data, expected_hash) = frame.split_at(content_length); - debug_assert_eq!(expected_hash.len(), self.hmac.num_bytes()); + debug_assert_eq!(expected_hash.len(), this.hmac.num_bytes()); - if self.hmac.verify(crypted_data, expected_hash).is_err() { + if this.hmac.verify(crypted_data, expected_hash).is_err() { debug!("hmac mismatch when decoding secio frame"); - return Err(SecioError::HmacNotMatching); + return Poll::Ready(Some(Err(SecioError::HmacNotMatching))); } } - let mut data_buf = frame.to_vec(); + let mut data_buf = frame; data_buf.truncate(content_length); - self.cipher_state - .decrypt(&mut data_buf); + this.cipher_state.decrypt(&mut data_buf); - if !self.nonce.is_empty() { - let n = min(data_buf.len(), self.nonce.len()); - if data_buf[.. n] != self.nonce[.. n] { - return Err(SecioError::NonceVerificationFailed) + if !this.nonce.is_empty() { + let n = min(data_buf.len(), this.nonce.len()); + if data_buf[.. n] != this.nonce[.. n] { + return Poll::Ready(Some(Err(SecioError::NonceVerificationFailed))) } - self.nonce.drain(.. n); + this.nonce.drain(.. n); data_buf.drain(.. n); } - Ok(Async::Ready(Some(data_buf))) + Poll::Ready(Some(Ok(data_buf))) } } -impl Sink for DecoderMiddleware +impl Sink for DecoderMiddleware where - S: Sink, + S: Sink, { - type SinkItem = S::SinkItem; - type SinkError = S::SinkError; + type Error = S::Error; + + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Sink::poll_ready(this.raw_stream, cx) + } - #[inline] - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - self.raw_stream.start_send(item) + fn start_send(self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> { + let this = self.project(); + Sink::start_send(this.raw_stream, item) } - #[inline] - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.raw_stream.poll_complete() + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Sink::poll_flush(this.raw_stream, cx) } - #[inline] - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.raw_stream.close() + fn poll_close(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Sink::poll_close(this.raw_stream, cx) } } diff --git a/protocols/secio/src/codec/encode.rs b/protocols/secio/src/codec/encode.rs index 36c3bcade02..88611bc1492 100644 --- a/protocols/secio/src/codec/encode.rs +++ b/protocols/secio/src/codec/encode.rs @@ -20,9 +20,9 @@ //! Individual messages encoding. -use bytes::BytesMut; use super::{Hmac, StreamCipher}; use futures::prelude::*; +use std::{pin::Pin, task::Context, task::Poll}; /// Wraps around a `Sink`. Encodes the buffers passed to it and passes it to the underlying sink. /// @@ -31,11 +31,12 @@ use futures::prelude::*; /// prefix is not covered by this module. /// /// Also implements `Stream` for convenience. +#[pin_project::pin_project] pub struct EncoderMiddleware { cipher_state: StreamCipher, hmac: Hmac, + #[pin] raw_sink: S, - pending: Option // buffer encrypted data which can not be sent right away } impl EncoderMiddleware { @@ -44,56 +45,38 @@ impl EncoderMiddleware { cipher_state: cipher, hmac, raw_sink: raw, - pending: None } } } -impl Sink for EncoderMiddleware +impl Sink> for EncoderMiddleware where - S: Sink, + S: Sink>, { - type SinkItem = BytesMut; - type SinkError = S::SinkError; + type Error = S::Error; - fn start_send(&mut self, mut data_buf: Self::SinkItem) -> StartSend { - if let Some(data) = self.pending.take() { - if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data)? { - self.pending = Some(data); - return Ok(AsyncSink::NotReady(data_buf)) - } - } - debug_assert!(self.pending.is_none()); + fn poll_ready(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Sink::poll_ready(this.raw_sink, cx) + } + + fn start_send(self: Pin<&mut Self>, mut data_buf: Vec) -> Result<(), Self::Error> { + let this = self.project(); // TODO if SinkError gets refactor to SecioError, then use try_apply_keystream - self.cipher_state.encrypt(&mut data_buf[..]); - let signature = self.hmac.sign(&data_buf[..]); + this.cipher_state.encrypt(&mut data_buf[..]); + let signature = this.hmac.sign(&data_buf[..]); data_buf.extend_from_slice(signature.as_ref()); - if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data_buf)? { - self.pending = Some(data) - } - Ok(AsyncSink::Ready) + Sink::start_send(this.raw_sink, data_buf) } - #[inline] - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - if let Some(data) = self.pending.take() { - if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data)? { - self.pending = Some(data); - return Ok(Async::NotReady) - } - } - self.raw_sink.poll_complete() + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Sink::poll_flush(this.raw_sink, cx) } - #[inline] - fn close(&mut self) -> Poll<(), Self::SinkError> { - if let Some(data) = self.pending.take() { - if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data)? { - self.pending = Some(data); - return Ok(Async::NotReady) - } - } - self.raw_sink.close() + fn poll_close(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Sink::poll_close(this.raw_sink, cx) } } @@ -102,10 +85,9 @@ where S: Stream, { type Item = S::Item; - type Error = S::Error; - #[inline] - fn poll(&mut self) -> Poll, Self::Error> { - self.raw_sink.poll() + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + Stream::poll_next(this.raw_sink, cx) } } diff --git a/protocols/secio/src/codec/len_prefix.rs b/protocols/secio/src/codec/len_prefix.rs new file mode 100644 index 00000000000..8b70083b9ce --- /dev/null +++ b/protocols/secio/src/codec/len_prefix.rs @@ -0,0 +1,127 @@ +// Copyright 2019 Parity Technologies (UK) 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 futures::{prelude::*, stream::BoxStream}; +use quicksink::Action; +use std::{fmt, io, pin::Pin, task::{Context, Poll}}; + +/// `Stream` & `Sink` that reads and writes a length prefix in front of the actual data. +pub struct LenPrefixCodec { + stream: BoxStream<'static, io::Result>>, + sink: Pin, Error = io::Error> + Send>>, + _mark: std::marker::PhantomData +} + +impl fmt::Debug for LenPrefixCodec { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.write_str("LenPrefixCodec") + } +} + +static_assertions::const_assert! { + std::mem::size_of::() <= std::mem::size_of::() +} + +impl LenPrefixCodec +where + T: AsyncRead + AsyncWrite + Unpin + Send + 'static +{ + pub fn new(socket: T, max_len: usize) -> Self { + let (r, w) = socket.split(); + + let stream = futures::stream::unfold(r, move |mut r| async move { + let mut len = [0; 4]; + if let Err(e) = r.read_exact(&mut len).await { + if e.kind() == io::ErrorKind::UnexpectedEof { + return None + } + return Some((Err(e), r)) + } + let n = u32::from_be_bytes(len) as usize; + if n > max_len { + let msg = format!("data length {} exceeds allowed maximum {}", n, max_len); + return Some((Err(io::Error::new(io::ErrorKind::PermissionDenied, msg)), r)) + } + let mut v = vec![0; n]; + if let Err(e) = r.read_exact(&mut v).await { + return Some((Err(e), r)) + } + Some((Ok(v), r)) + }); + + let sink = quicksink::make_sink(w, move |mut w, action: Action>| async move { + match action { + Action::Send(data) => { + if data.len() > max_len { + log::error!("data length {} exceeds allowed maximum {}", data.len(), max_len) + } + w.write_all(&(data.len() as u32).to_be_bytes()).await?; + w.write_all(&data).await? + } + Action::Flush => w.flush().await?, + Action::Close => w.close().await? + } + Ok(w) + }); + + LenPrefixCodec { + stream: stream.boxed(), + sink: Box::pin(sink), + _mark: std::marker::PhantomData + } + } +} + +impl Stream for LenPrefixCodec +where + T: AsyncRead + AsyncWrite + Send + 'static +{ + type Item = io::Result>; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + self.stream.poll_next_unpin(cx) + } +} + +impl Sink> for LenPrefixCodec +where + T: AsyncRead + AsyncWrite + Send + 'static +{ + type Error = io::Error; + + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.sink).poll_ready(cx) + } + + fn start_send(mut self: Pin<&mut Self>, item: Vec) -> Result<(), Self::Error> { + Pin::new(&mut self.sink).start_send(item) + } + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.sink).poll_flush(cx) + } + + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.sink).poll_close(cx) + } +} + +impl Unpin for LenPrefixCodec { +} diff --git a/protocols/secio/src/codec/mod.rs b/protocols/secio/src/codec/mod.rs index 51a711cc5a7..8a4fabe5c00 100644 --- a/protocols/secio/src/codec/mod.rs +++ b/protocols/secio/src/codec/mod.rs @@ -21,21 +21,22 @@ //! Individual messages encoding and decoding. Use this after the algorithms have been //! successfully negotiated. -use self::decode::DecoderMiddleware; -use self::encode::EncoderMiddleware; +mod decode; +mod encode; +mod len_prefix; use aes_ctr::stream_cipher; use crate::algo_support::Digest; +use decode::DecoderMiddleware; +use encode::EncoderMiddleware; +use futures::prelude::*; use hmac::{self, Mac}; use sha2::{Sha256, Sha512}; -use tokio_io::codec::length_delimited; -use tokio_io::{AsyncRead, AsyncWrite}; -mod decode; -mod encode; +pub use len_prefix::LenPrefixCodec; /// Type returned by `full_codec`. -pub type FullCodec = DecoderMiddleware>>; +pub type FullCodec = DecoderMiddleware>>; pub type StreamCipher = Box; @@ -103,12 +104,12 @@ impl Hmac { } /// Takes control of `socket`. Returns an object that implements `future::Sink` and -/// `future::Stream`. The `Stream` and `Sink` produce and accept `BytesMut` objects. +/// `future::Stream`. The `Stream` and `Sink` produce and accept `Vec` objects. /// /// The conversion between the stream/sink items and the socket is done with the given cipher and /// hash algorithm (which are generally decided during the handshake). pub fn full_codec( - socket: length_delimited::Framed, + socket: LenPrefixCodec, cipher_encoding: StreamCipher, encoding_hmac: Hmac, cipher_decoder: StreamCipher, @@ -116,64 +117,50 @@ pub fn full_codec( remote_nonce: Vec ) -> FullCodec where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin + Send + 'static { let encoder = EncoderMiddleware::new(socket, cipher_encoding, encoding_hmac); DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac, remote_nonce) } + #[cfg(test)] mod tests { - use tokio::runtime::current_thread::Runtime; - use tokio_tcp::{TcpListener, TcpStream}; - use crate::stream_cipher::{ctr, Cipher}; - use super::full_codec; - use super::DecoderMiddleware; - use super::EncoderMiddleware; - use super::Hmac; + use super::{full_codec, DecoderMiddleware, EncoderMiddleware, Hmac, LenPrefixCodec}; use crate::algo_support::Digest; + use crate::stream_cipher::{ctr, Cipher}; use crate::error::SecioError; - use bytes::BytesMut; - use futures::sync::mpsc::channel; - use futures::{Future, Sink, Stream, stream}; - use rand; - use std::io::Error as IoError; - use tokio_io::codec::length_delimited::Framed; + use async_std::net::{TcpListener, TcpStream}; + use futures::{prelude::*, channel::mpsc, channel::oneshot}; - const NULL_IV : [u8; 16] = [0;16]; + const NULL_IV : [u8; 16] = [0; 16]; #[test] fn raw_encode_then_decode() { - let (data_tx, data_rx) = channel::(256); - let data_tx = data_tx.sink_map_err::<_, IoError>(|_| panic!()); - let data_rx = data_rx.map_err::(|_| panic!()); + let (data_tx, data_rx) = mpsc::channel::>(256); let cipher_key: [u8; 32] = rand::random(); let hmac_key: [u8; 32] = rand::random(); - - let encoder = EncoderMiddleware::new( + let mut encoder = EncoderMiddleware::new( data_tx, ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]), Hmac::from_key(Digest::Sha256, &hmac_key), ); - let decoder = DecoderMiddleware::new( - data_rx, + + let mut decoder = DecoderMiddleware::new( + data_rx.map(|v| Ok::<_, SecioError>(v)), ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]), Hmac::from_key(Digest::Sha256, &hmac_key), Vec::new() ); let data = b"hello world"; - - let data_sent = encoder.send(BytesMut::from(data.to_vec())).from_err(); - let data_received = decoder.into_future().map(|(n, _)| n).map_err(|(e, _)| e); - let mut rt = Runtime::new().unwrap(); - - let (_, decoded) = rt.block_on(data_sent.join(data_received)) - .map_err(|_| ()) - .unwrap(); - assert_eq!(&decoded.unwrap()[..], &data[..]); + async_std::task::block_on(async move { + encoder.send(data.to_vec()).await.unwrap(); + let rx = decoder.next().await.unwrap().unwrap(); + assert_eq!(rx, data); + }); } fn full_codec_encode_then_decode(cipher: Cipher) { @@ -185,53 +172,44 @@ mod tests { let data = b"hello world"; let data_clone = data.clone(); let nonce = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]; - - let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap()).unwrap(); - let listener_addr = listener.local_addr().unwrap(); + let (l_a_tx, l_a_rx) = oneshot::channel(); let nonce2 = nonce.clone(); - let server = listener.incoming() - .into_future() - .map_err(|(e, _)| e) - .map(move |(connec, _)| { - full_codec( - Framed::new(connec.unwrap()), - ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]), - Hmac::from_key(Digest::Sha256, &hmac_key), - ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]), - Hmac::from_key(Digest::Sha256, &hmac_key), - nonce2 - ) - }, - ); - - let client = TcpStream::connect(&listener_addr) - .map_err(|e| e.into()) - .map(move |stream| { - full_codec( - Framed::new(stream), - ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]), - Hmac::from_key(Digest::Sha256, &hmac_key_clone), - ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]), - Hmac::from_key(Digest::Sha256, &hmac_key_clone), - Vec::new() - ) - }); - - let fin = server - .join(client) - .from_err::() - .and_then(|(server, client)| { - client - .send_all(stream::iter_ok::<_, IoError>(vec![nonce.into(), data_clone[..].into()])) - .map(move |_| server) - .from_err() - }) - .and_then(|server| server.concat2().from_err()); - - let mut rt = Runtime::new().unwrap(); - let received = rt.block_on(fin).unwrap(); - assert_eq!(received, data); + let server = async { + let listener = TcpListener::bind(&"127.0.0.1:0").await.unwrap(); + let listener_addr = listener.local_addr().unwrap(); + l_a_tx.send(listener_addr).unwrap(); + + let (connec, _) = listener.accept().await.unwrap(); + let codec = full_codec( + LenPrefixCodec::new(connec, 1024), + ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]), + Hmac::from_key(Digest::Sha256, &hmac_key), + ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]), + Hmac::from_key(Digest::Sha256, &hmac_key), + nonce2.clone() + ); + + let outcome = codec.map(|v| v.unwrap()).concat().await; + assert_eq!(outcome, data_clone); + }; + + let client = async { + let listener_addr = l_a_rx.await.unwrap(); + let stream = TcpStream::connect(&listener_addr).await.unwrap(); + let mut codec = full_codec( + LenPrefixCodec::new(stream, 1024), + ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]), + Hmac::from_key(Digest::Sha256, &hmac_key_clone), + ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]), + Hmac::from_key(Digest::Sha256, &hmac_key_clone), + Vec::new() + ); + codec.send(nonce.into()).await.unwrap(); + codec.send(data.to_vec().into()).await.unwrap(); + }; + + async_std::task::block_on(future::join(client, server)); } #[test] diff --git a/protocols/secio/src/exchange/impl_ring.rs b/protocols/secio/src/exchange/impl_ring.rs index 04acf866ca1..b7f42be78b7 100644 --- a/protocols/secio/src/exchange/impl_ring.rs +++ b/protocols/secio/src/exchange/impl_ring.rs @@ -42,7 +42,7 @@ pub type AgreementPrivateKey = ring_agreement::EphemeralPrivateKey; /// Generates a new key pair as part of the exchange. /// /// Returns the opaque private key and the corresponding public key. -pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future), Error = SecioError> { +pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future), SecioError>> { let rng = ring_rand::SystemRandom::new(); match ring_agreement::EphemeralPrivateKey::generate(algorithm.into(), &rng) { @@ -50,22 +50,22 @@ pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future { debug!("failed to generate ECDH key"); - future::err(SecioError::EphemeralKeyGenerationFailed) + future::ready(Err(SecioError::EphemeralKeyGenerationFailed)) }, } } /// Finish the agreement. On success, returns the shared key that both remote agreed upon. pub fn agree(algorithm: KeyAgreement, my_private_key: AgreementPrivateKey, other_public_key: &[u8], _out_size: usize) - -> impl Future, Error = SecioError> + -> impl Future, SecioError>> { - ring_agreement::agree_ephemeral(my_private_key, - &ring_agreement::UnparsedPublicKey::new(algorithm.into(), other_public_key), - SecioError::SecretGenerationFailed, - |key_material| Ok(key_material.to_vec())) - .into_future() + let ret = ring_agreement::agree_ephemeral(my_private_key, + &ring_agreement::UnparsedPublicKey::new(algorithm.into(), other_public_key), + SecioError::SecretGenerationFailed, + |key_material| Ok(key_material.to_vec())); + future::ready(ret) } diff --git a/protocols/secio/src/exchange/impl_webcrypto.rs b/protocols/secio/src/exchange/impl_webcrypto.rs index 2a8831034da..a7a363cab6a 100644 --- a/protocols/secio/src/exchange/impl_webcrypto.rs +++ b/protocols/secio/src/exchange/impl_webcrypto.rs @@ -23,7 +23,7 @@ use crate::{KeyAgreement, SecioError}; use futures::prelude::*; use parity_send_wrapper::SendWrapper; -use std::io; +use std::{io, pin::Pin, task::Context, task::Poll}; use wasm_bindgen::prelude::*; /// Opaque private key type. Contains the private key and the `SubtleCrypto` object. @@ -35,12 +35,11 @@ pub type AgreementPrivateKey = SendSyncHack<(JsValue, web_sys::SubtleCrypto)>; pub struct SendSyncHack(SendWrapper); impl Future for SendSyncHack -where T: Future { - type Item = T::Item; - type Error = T::Error; +where T: Future + Unpin { + type Output = T::Output; - fn poll(&mut self) -> Poll { - self.0.poll() + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + self.0.poll_unpin(cx) } } @@ -48,128 +47,114 @@ where T: Future { /// /// Returns the opaque private key and the corresponding public key. pub fn generate_agreement(algorithm: KeyAgreement) - -> impl Future), Error = SecioError> + -> impl Future), SecioError>> { - // First step is to create the `SubtleCrypto` object. - let crypto = build_crypto_future(); - - // We then generate the ephemeral key. - let key_promise = crypto.and_then(move |crypto| { - let crypto = crypto.clone(); - let obj = build_curve_obj(algorithm); - - let usages = js_sys::Array::new(); - usages.push(&JsValue::from_str("deriveKey")); - usages.push(&JsValue::from_str("deriveBits")); - - crypto.generate_key_with_object(&obj, true, usages.as_ref()) - .map(wasm_bindgen_futures::JsFuture::from) - .into_future() - .flatten() - .map(|key_pair| (key_pair, crypto)) - }); - - // WebCrypto has generated a key-pair. Let's split this key pair into a private key and a - // public key. - let split_key = key_promise.and_then(move |(key_pair, crypto)| { - let private = js_sys::Reflect::get(&key_pair, &JsValue::from_str("privateKey")); - let public = js_sys::Reflect::get(&key_pair, &JsValue::from_str("publicKey")); - match (private, public) { - (Ok(pr), Ok(pu)) => Ok((pr, pu, crypto)), - (Err(err), _) => Err(err), - (_, Err(err)) => Err(err), - } - }); - - // Then we turn the public key into an `ArrayBuffer`. - let export_key = split_key.and_then(move |(private, public, crypto)| { - crypto.export_key("raw", &public.into()) - .map(wasm_bindgen_futures::JsFuture::from) - .into_future() - .flatten() - .map(|public| ((private, crypto), public)) - }); - - // And finally we convert this `ArrayBuffer` into a `Vec`. - let future = export_key - .map(|((private, crypto), public)| { - let public = js_sys::Uint8Array::new(&public); - let mut public_buf = vec![0; public.length() as usize]; - public.copy_to(&mut public_buf); - (SendSyncHack(SendWrapper::new((private, crypto))), public_buf) - }); + let future = async move { + // First step is to create the `SubtleCrypto` object. + let crypto = build_crypto_future().await?; + + // We then generate the ephemeral key. + let key_pair = { + let obj = build_curve_obj(algorithm); + + let usages = js_sys::Array::new(); + usages.push(&JsValue::from_str("deriveKey")); + usages.push(&JsValue::from_str("deriveBits")); + + let promise = crypto.generate_key_with_object(&obj, true, usages.as_ref())?; + wasm_bindgen_futures::JsFuture::from(promise).await? + }; + + // WebCrypto has generated a key-pair. Let's split this key pair into a private key and a + // public key. + let (private, public) = { + let private = js_sys::Reflect::get(&key_pair, &JsValue::from_str("privateKey")); + let public = js_sys::Reflect::get(&key_pair, &JsValue::from_str("publicKey")); + match (private, public) { + (Ok(pr), Ok(pu)) => (pr, pu), + (Err(err), _) => return Err(err), + (_, Err(err)) => return Err(err), + } + }; + + // Then we turn the public key into an `ArrayBuffer`. + let public = { + let promise = crypto.export_key("raw", &public.into())?; + wasm_bindgen_futures::JsFuture::from(promise).await? + }; - SendSyncHack(SendWrapper::new(future.map_err(|err| { - SecioError::IoError(io::Error::new(io::ErrorKind::Other, format!("{:?}", err))) - }))) + // And finally we convert this `ArrayBuffer` into a `Vec`. + let public = js_sys::Uint8Array::new(&public); + let mut public_buf = vec![0; public.length() as usize]; + public.copy_to(&mut public_buf); + Ok((SendSyncHack(SendWrapper::new((private, crypto))), public_buf)) + }; + + let future = future + .map_err(|err| { + SecioError::IoError(io::Error::new(io::ErrorKind::Other, format!("{:?}", err))) + }); + SendSyncHack(SendWrapper::new(Box::pin(future))) } /// Finish the agreement. On success, returns the shared key that both remote agreed upon. pub fn agree(algorithm: KeyAgreement, key: AgreementPrivateKey, other_public_key: &[u8], out_size: usize) - -> impl Future, Error = SecioError> + -> impl Future, SecioError>> { - let (private_key, crypto) = key.0.take(); - - // We start by importing the remote's public key into the WebCrypto world. - let import_promise = { - let other_public_key = { - // This unsafe is here because the lifetime of `other_public_key` must not outlive the - // `tmp_view`. This is guaranteed by the fact that we clone this array right below. - // See also https://github.com/rustwasm/wasm-bindgen/issues/1303 - let tmp_view = unsafe { js_sys::Uint8Array::view(other_public_key) }; - js_sys::Uint8Array::new(tmp_view.as_ref()) - }; - - // Note: contrary to what one might think, we shouldn't add the "deriveBits" usage. - crypto - .import_key_with_object( - "raw", &js_sys::Object::from(other_public_key.buffer()), - &build_curve_obj(algorithm), false, &js_sys::Array::new() - ) - .into_future() - .map(wasm_bindgen_futures::JsFuture::from) - .flatten() + let other_public_key = { + // This unsafe is here because the lifetime of `other_public_key` must not outlive the + // `tmp_view`. This is guaranteed by the fact that we clone this array right below. + // See also https://github.com/rustwasm/wasm-bindgen/issues/1303 + let tmp_view = unsafe { js_sys::Uint8Array::view(other_public_key) }; + js_sys::Uint8Array::new(tmp_view.as_ref()) }; - // We then derive the final private key. - let derive = import_promise.and_then({ - let crypto = crypto.clone(); - move |public_key| { + let future = async move { + let (private_key, crypto) = key.0.take(); + + // We start by importing the remote's public key into the WebCrypto world. + let public_key = { + // Note: contrary to what one might think, we shouldn't add the "deriveBits" usage. + let promise = crypto + .import_key_with_object( + "raw", &js_sys::Object::from(other_public_key.buffer()), + &build_curve_obj(algorithm), false, &js_sys::Array::new() + )?; + wasm_bindgen_futures::JsFuture::from(promise).await? + }; + + // We then derive the final private key. + let bytes = { let derive_params = build_curve_obj(algorithm); let _ = js_sys::Reflect::set(derive_params.as_ref(), &JsValue::from_str("public"), &public_key); - crypto + let promise = crypto .derive_bits_with_object( &derive_params, &web_sys::CryptoKey::from(private_key), 8 * out_size as u32 - ) - .into_future() - .map(wasm_bindgen_futures::JsFuture::from) - .flatten() - } - }); - - let future = derive - .map(|bytes| { - let bytes = js_sys::Uint8Array::new(&bytes); - let mut buf = vec![0; bytes.length() as usize]; - bytes.copy_to(&mut buf); - buf - }) - .map_err(|err| { + )?; + wasm_bindgen_futures::JsFuture::from(promise).await? + }; + + let bytes = js_sys::Uint8Array::new(&bytes); + let mut buf = vec![0; bytes.length() as usize]; + bytes.copy_to(&mut buf); + Ok(buf) + }; + + let future = future + .map_err(|err: JsValue| { SecioError::IoError(io::Error::new(io::ErrorKind::Other, format!("{:?}", err))) }); - - SendSyncHack(SendWrapper::new(future)) + SendSyncHack(SendWrapper::new(Box::pin(future))) } /// Builds a future that returns the `SubtleCrypto` object. -fn build_crypto_future() -> impl Future { +async fn build_crypto_future() -> Result { web_sys::window() .ok_or_else(|| JsValue::from_str("Window object not available")) .and_then(|window| window.crypto()) .map(|crypto| crypto.subtle()) - .into_future() } /// Builds a `EcKeyGenParams` object. diff --git a/protocols/secio/src/exchange/mod.rs b/protocols/secio/src/exchange/mod.rs index bb59b4e6525..5fdecbb8c36 100644 --- a/protocols/secio/src/exchange/mod.rs +++ b/protocols/secio/src/exchange/mod.rs @@ -44,14 +44,14 @@ pub struct AgreementPrivateKey(platform::AgreementPrivateKey); /// /// Returns the opaque private key and the corresponding public key. #[inline] -pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future), Error = SecioError> { - platform::generate_agreement(algorithm).map(|(pr, pu)| (AgreementPrivateKey(pr), pu)) +pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future), SecioError>> { + platform::generate_agreement(algorithm).map_ok(|(pr, pu)| (AgreementPrivateKey(pr), pu)) } /// Finish the agreement. On success, returns the shared key that both remote agreed upon. #[inline] pub fn agree(algorithm: KeyAgreement, my_private_key: AgreementPrivateKey, other_public_key: &[u8], out_size: usize) - -> impl Future, Error = SecioError> + -> impl Future, SecioError>> { platform::agree(algorithm, my_private_key.0, other_public_key, out_size) } diff --git a/protocols/secio/src/handshake.rs b/protocols/secio/src/handshake.rs index 1a8be4ebdec..edf7216c417 100644 --- a/protocols/secio/src/handshake.rs +++ b/protocols/secio/src/handshake.rs @@ -18,465 +18,303 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use crate::SecioConfig; use crate::algo_support; -use bytes::BytesMut; -use crate::codec::{full_codec, FullCodec, Hmac}; -use crate::stream_cipher::{Cipher, ctr}; +use crate::codec::{full_codec, FullCodec, Hmac, LenPrefixCodec}; use crate::error::SecioError; use crate::exchange; -use futures::future; -use futures::sink::Sink; -use futures::stream::Stream; -use futures::Future; +use crate::stream_cipher::ctr; +use crate::structs_proto::{Exchange, Propose}; +use futures::prelude::*; use libp2p_core::PublicKey; use log::{debug, trace}; -use protobuf::parse_from_bytes as protobuf_parse_from_bytes; use protobuf::Message as ProtobufMessage; +use protobuf::parse_from_bytes as protobuf_parse_from_bytes; use rand::{self, RngCore}; use sha2::{Digest as ShaDigestTrait, Sha256}; use std::cmp::{self, Ordering}; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use crate::structs_proto::{Exchange, Propose}; -use tokio_io::{AsyncRead, AsyncWrite, codec::length_delimited}; -use crate::{KeyAgreement, SecioConfig}; - -// This struct contains the whole context of a handshake, and is filled progressively -// throughout the various parts of the handshake. -struct HandshakeContext { - config: SecioConfig, - state: T -} - -// HandshakeContext<()> --with_local-> HandshakeContext -struct Local { - // Locally-generated random number. The array size can be changed without any repercussion. - nonce: [u8; 16], - // Our encoded local public key - public_key_encoded: Vec, - // Our local proposition's raw bytes: - proposition_bytes: Vec -} - -// HandshakeContext --with_remote-> HandshakeContext -struct Remote { - local: Local, - // The remote's proposition's raw bytes: - proposition_bytes: BytesMut, - // The remote's public key: - public_key: PublicKey, - // The remote's `nonce`. - // If the NONCE size is actually part of the protocol, we can change this to a fixed-size - // array instead of a `Vec`. - nonce: Vec, - // Set to `ordering( - // hash(concat(remote-pubkey, local-none)), - // hash(concat(local-pubkey, remote-none)) - // )`. - // `Ordering::Equal` is an invalid value (as it would mean we're talking to ourselves). - // - // Since everything is symmetrical, this value is used to determine what should be ours - // and what should be the remote's. - hashes_ordering: Ordering, - // Crypto algorithms chosen for the communication: - chosen_exchange: KeyAgreement, - chosen_cipher: Cipher, - chosen_hash: algo_support::Digest, -} - -// HandshakeContext --with_ephemeral-> HandshakeContext -struct Ephemeral { - remote: Remote, - // Ephemeral keypair generated for the handshake: - local_tmp_priv_key: exchange::AgreementPrivateKey, - local_tmp_pub_key: Vec -} -// HandshakeContext --take_private_key-> HandshakeContext -struct PubEphemeral { - remote: Remote, - local_tmp_pub_key: Vec -} -impl HandshakeContext<()> { - fn new(config: SecioConfig) -> Self { - HandshakeContext { - config, - state: () - } - } +/// Performs a handshake on the given socket. +/// +/// This function expects that the remote is identified with `remote_public_key`, and the remote +/// will expect that we are identified with `local_key`. Any mismatch somewhere will produce a +/// `SecioError`. +/// +/// On success, returns an object that implements the `Sink` and `Stream` trait whose items are +/// buffers of data, plus the public key of the remote, plus the ephemeral public key used during +/// negotiation. +pub async fn handshake(socket: S, config: SecioConfig) + -> Result<(FullCodec, PublicKey, Vec), SecioError> +where + S: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + let mut socket = LenPrefixCodec::new(socket, config.max_frame_len); - // Setup local proposition. - fn with_local(self) -> Result, SecioError> { - let mut nonce = [0; 16]; + let local_nonce = { + let mut local_nonce = [0; 16]; rand::thread_rng() - .try_fill_bytes(&mut nonce) + .try_fill_bytes(&mut local_nonce) .map_err(|_| SecioError::NonceGenerationFailed)?; + local_nonce + }; + + let local_public_key_encoded = config.key.public().into_protobuf_encoding(); + + // Send our proposition with our nonce, public key and supported protocols. + let mut local_proposition = Propose::new(); + local_proposition.set_rand(local_nonce.to_vec()); + local_proposition.set_pubkey(local_public_key_encoded.clone()); + + if let Some(ref p) = config.agreements_prop { + trace!("agreements proposition: {}", p); + local_proposition.set_exchanges(p.clone()) + } else { + trace!("agreements proposition: {}", algo_support::DEFAULT_AGREEMENTS_PROPOSITION); + local_proposition.set_exchanges(algo_support::DEFAULT_AGREEMENTS_PROPOSITION.into()) + } - let public_key_encoded = self.config.key.public().into_protobuf_encoding(); + if let Some(ref p) = config.ciphers_prop { + trace!("ciphers proposition: {}", p); + local_proposition.set_ciphers(p.clone()) + } else { + trace!("ciphers proposition: {}", algo_support::DEFAULT_CIPHERS_PROPOSITION); + local_proposition.set_ciphers(algo_support::DEFAULT_CIPHERS_PROPOSITION.into()) + } - // Send our proposition with our nonce, public key and supported protocols. - let mut proposition = Propose::new(); - proposition.set_rand(nonce.to_vec()); - proposition.set_pubkey(public_key_encoded.clone()); + if let Some(ref p) = config.digests_prop { + trace!("digests proposition: {}", p); + local_proposition.set_hashes(p.clone()) + } else { + trace!("digests proposition: {}", algo_support::DEFAULT_DIGESTS_PROPOSITION); + local_proposition.set_hashes(algo_support::DEFAULT_DIGESTS_PROPOSITION.into()) + } - if let Some(ref p) = self.config.agreements_prop { - trace!("agreements proposition: {}", p); - proposition.set_exchanges(p.clone()) - } else { - trace!("agreements proposition: {}", algo_support::DEFAULT_AGREEMENTS_PROPOSITION); - proposition.set_exchanges(algo_support::DEFAULT_AGREEMENTS_PROPOSITION.into()) + let local_proposition_bytes = local_proposition.write_to_bytes()?; + trace!("starting handshake; local nonce = {:?}", local_nonce); + + trace!("sending proposition to remote"); + socket.send(local_proposition_bytes.clone()).await?; + + // Receive the remote's proposition. + let remote_proposition_bytes = match socket.next().await { + Some(b) => b?, + None => { + let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof"); + debug!("unexpected eof while waiting for remote's proposition"); + return Err(err.into()) + }, + }; + + let mut remote_proposition = match protobuf_parse_from_bytes::(&remote_proposition_bytes) { + Ok(prop) => prop, + Err(_) => { + debug!("failed to parse remote's proposition protobuf message"); + return Err(SecioError::HandshakeParsingFailure); } + }; + + let remote_public_key_encoded = remote_proposition.take_pubkey(); + let remote_nonce = remote_proposition.take_rand(); + + let remote_public_key = match PublicKey::from_protobuf_encoding(&remote_public_key_encoded) { + Ok(p) => p, + Err(_) => { + debug!("failed to parse remote's proposition's pubkey protobuf"); + return Err(SecioError::HandshakeParsingFailure); + }, + }; + trace!("received proposition from remote; pubkey = {:?}; nonce = {:?}", + remote_public_key, remote_nonce); + + // In order to determine which protocols to use, we compute two hashes and choose + // based on which hash is larger. + let hashes_ordering = { + let oh1 = { + let mut ctx = Sha256::new(); + ctx.input(&remote_public_key_encoded); + ctx.input(&local_nonce); + ctx.result() + }; - if let Some(ref p) = self.config.ciphers_prop { - trace!("ciphers proposition: {}", p); - proposition.set_ciphers(p.clone()) - } else { - trace!("ciphers proposition: {}", algo_support::DEFAULT_CIPHERS_PROPOSITION); - proposition.set_ciphers(algo_support::DEFAULT_CIPHERS_PROPOSITION.into()) - } + let oh2 = { + let mut ctx = Sha256::new(); + ctx.input(&local_public_key_encoded); + ctx.input(&remote_nonce); + ctx.result() + }; - if let Some(ref p) = self.config.digests_prop { - trace!("digests proposition: {}", p); - proposition.set_hashes(p.clone()) - } else { - trace!("digests proposition: {}", algo_support::DEFAULT_DIGESTS_PROPOSITION); - proposition.set_hashes(algo_support::DEFAULT_DIGESTS_PROPOSITION.into()) + oh1.as_ref().cmp(&oh2.as_ref()) + }; + + let chosen_exchange = { + let ours = config.agreements_prop.as_ref() + .map(|s| s.as_ref()) + .unwrap_or(algo_support::DEFAULT_AGREEMENTS_PROPOSITION); + let theirs = &remote_proposition.get_exchanges(); + match algo_support::select_agreement(hashes_ordering, ours, theirs) { + Ok(a) => a, + Err(err) => { + debug!("failed to select an exchange protocol"); + return Err(err); + } } - - let proposition_bytes = proposition.write_to_bytes()?; - - Ok(HandshakeContext { - config: self.config, - state: Local { - nonce, - public_key_encoded, - proposition_bytes + }; + + let chosen_cipher = { + let ours = config.ciphers_prop.as_ref() + .map(|s| s.as_ref()) + .unwrap_or(algo_support::DEFAULT_CIPHERS_PROPOSITION); + let theirs = &remote_proposition.get_ciphers(); + match algo_support::select_cipher(hashes_ordering, ours, theirs) { + Ok(a) => { + debug!("selected cipher: {:?}", a); + a } - }) - } -} - -impl HandshakeContext { - // Process remote proposition. - fn with_remote(self, b: BytesMut) -> Result, SecioError> { - let mut prop = match protobuf_parse_from_bytes::(&b) { - Ok(prop) => prop, - Err(_) => { - debug!("failed to parse remote's proposition protobuf message"); - return Err(SecioError::HandshakeParsingFailure); + Err(err) => { + debug!("failed to select a cipher protocol"); + return Err(err); } - }; - - let public_key_encoded = prop.take_pubkey(); - let nonce = prop.take_rand(); - - let pubkey = match PublicKey::from_protobuf_encoding(&public_key_encoded) { - Ok(p) => p, - Err(_) => { - debug!("failed to parse remote's proposition's pubkey protobuf"); - return Err(SecioError::HandshakeParsingFailure); - }, - }; - - // In order to determine which protocols to use, we compute two hashes and choose - // based on which hash is larger. - let hashes_ordering = { - let oh1 = { - let mut ctx = Sha256::new(); - ctx.input(&public_key_encoded); - ctx.input(&self.state.nonce); - ctx.result() - }; - - let oh2 = { - let mut ctx = Sha256::new(); - ctx.input(&self.state.public_key_encoded); - ctx.input(&nonce); - ctx.result() - }; - - oh1.as_ref().cmp(&oh2.as_ref()) - }; - - let chosen_exchange = { - let ours = self.config.agreements_prop.as_ref() - .map(|s| s.as_ref()) - .unwrap_or(algo_support::DEFAULT_AGREEMENTS_PROPOSITION); - let theirs = &prop.get_exchanges(); - match algo_support::select_agreement(hashes_ordering, ours, theirs) { - Ok(a) => a, - Err(err) => { - debug!("failed to select an exchange protocol"); - return Err(err); - } + } + }; + + let chosen_hash = { + let ours = config.digests_prop.as_ref() + .map(|s| s.as_ref()) + .unwrap_or(algo_support::DEFAULT_DIGESTS_PROPOSITION); + let theirs = &remote_proposition.get_hashes(); + match algo_support::select_digest(hashes_ordering, ours, theirs) { + Ok(a) => { + debug!("selected hash: {:?}", a); + a } - }; - - let chosen_cipher = { - let ours = self.config.ciphers_prop.as_ref() - .map(|s| s.as_ref()) - .unwrap_or(algo_support::DEFAULT_CIPHERS_PROPOSITION); - let theirs = &prop.get_ciphers(); - match algo_support::select_cipher(hashes_ordering, ours, theirs) { - Ok(a) => { - debug!("selected cipher: {:?}", a); - a - } - Err(err) => { - debug!("failed to select a cipher protocol"); - return Err(err); - } + Err(err) => { + debug!("failed to select a hash protocol"); + return Err(err); } + } + }; + + // Generate an ephemeral key for the negotiation. + let (tmp_priv_key, tmp_pub_key) = exchange::generate_agreement(chosen_exchange).await?; + + // Send the ephemeral pub key to the remote in an `Exchange` struct. The `Exchange` also + // contains a signature of the two propositions encoded with our static public key. + let local_exchange = { + let mut data_to_sign = local_proposition_bytes.clone(); + data_to_sign.extend_from_slice(&remote_proposition_bytes); + data_to_sign.extend_from_slice(&tmp_pub_key); + + let mut exchange = Exchange::new(); + exchange.set_epubkey(tmp_pub_key.clone()); + match config.key.sign(&data_to_sign) { + Ok(sig) => exchange.set_signature(sig), + Err(_) => return Err(SecioError::SigningFailure) + } + exchange + }; + let local_exch = local_exchange.write_to_bytes()?; + + // Send our local `Exchange`. + trace!("sending exchange to remote"); + socket.send(local_exch).await?; + + // Receive the remote's `Exchange`. + let remote_exch = { + let raw = match socket.next().await { + Some(r) => r?, + None => { + let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof"); + debug!("unexpected eof while waiting for remote's exchange"); + return Err(err.into()) + }, }; - let chosen_hash = { - let ours = self.config.digests_prop.as_ref() - .map(|s| s.as_ref()) - .unwrap_or(algo_support::DEFAULT_DIGESTS_PROPOSITION); - let theirs = &prop.get_hashes(); - match algo_support::select_digest(hashes_ordering, ours, theirs) { - Ok(a) => { - debug!("selected hash: {:?}", a); - a - } - Err(err) => { - debug!("failed to select a hash protocol"); - return Err(err); - } + match protobuf_parse_from_bytes::(&raw) { + Ok(e) => { + trace!("received and decoded the remote's exchange"); + e + }, + Err(err) => { + debug!("failed to parse remote's exchange protobuf; {:?}", err); + return Err(SecioError::HandshakeParsingFailure); } - }; + } + }; + + // Check the validity of the remote's `Exchange`. This verifies that the remote was really + // the sender of its proposition, and that it is the owner of both its global and ephemeral + // keys. + { + let mut data_to_verify = remote_proposition_bytes.clone(); + data_to_verify.extend_from_slice(&local_proposition_bytes); + data_to_verify.extend_from_slice(remote_exch.get_epubkey()); + + if !remote_public_key.verify(&data_to_verify, remote_exch.get_signature()) { + return Err(SecioError::SignatureVerificationFailed) + } - Ok(HandshakeContext { - config: self.config, - state: Remote { - local: self.state, - proposition_bytes: b, - public_key: pubkey, - nonce, - hashes_ordering, - chosen_exchange, - chosen_cipher, - chosen_hash - } - }) + trace!("successfully verified the remote's signature"); } -} -impl HandshakeContext { - fn with_ephemeral(self, sk: exchange::AgreementPrivateKey, pk: Vec) -> HandshakeContext { - HandshakeContext { - config: self.config, - state: Ephemeral { - remote: self.state, - local_tmp_priv_key: sk, - local_tmp_pub_key: pk + // Generate a key from the local ephemeral private key and the remote ephemeral public key, + // derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder. + let key_material = exchange::agree(chosen_exchange, tmp_priv_key, remote_exch.get_epubkey(), chosen_hash.num_bytes()).await?; + + // Generate a key from the local ephemeral private key and the remote ephemeral public key, + // derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder. + let mut codec = { + let cipher_key_size = chosen_cipher.key_size(); + let iv_size = chosen_cipher.iv_size(); + + let key = Hmac::from_key(chosen_hash, &key_material); + let mut longer_key = vec![0u8; 2 * (iv_size + cipher_key_size + 20)]; + stretch_key(key, &mut longer_key); + + let (local_infos, remote_infos) = { + let (first_half, second_half) = longer_key.split_at(longer_key.len() / 2); + match hashes_ordering { + Ordering::Equal => { + let msg = "equal digest of public key and nonce for local and remote"; + return Err(SecioError::InvalidProposition(msg)) + } + Ordering::Less => (second_half, first_half), + Ordering::Greater => (first_half, second_half), } - } - } -} + }; -impl HandshakeContext { - fn take_private_key(self) -> (HandshakeContext, exchange::AgreementPrivateKey) { - let context = HandshakeContext { - config: self.config, - state: PubEphemeral { - remote: self.state.remote, - local_tmp_pub_key: self.state.local_tmp_pub_key - } + let (encoding_cipher, encoding_hmac) = { + let (iv, rest) = local_infos.split_at(iv_size); + let (cipher_key, mac_key) = rest.split_at(cipher_key_size); + let hmac = Hmac::from_key(chosen_hash, mac_key); + let cipher = ctr(chosen_cipher, cipher_key, iv); + (cipher, hmac) }; - (context, self.state.local_tmp_priv_key) - } -} -/// Performs a handshake on the given socket. -/// -/// This function expects that the remote is identified with `remote_public_key`, and the remote -/// will expect that we are identified with `local_key`.Any mismatch somewhere will produce a -/// `SecioError`. -/// -/// On success, returns an object that implements the `Sink` and `Stream` trait whose items are -/// buffers of data, plus the public key of the remote, plus the ephemeral public key used during -/// negotiation. -pub fn handshake<'a, S: 'a>(socket: S, config: SecioConfig) - -> impl Future, PublicKey, Vec), Error = SecioError> -where - S: AsyncRead + AsyncWrite + Send, -{ - // The handshake messages all start with a 4-bytes message length prefix. - let socket = length_delimited::Builder::new() - .big_endian() - .length_field_length(4) - .new_framed(socket); - - future::ok::<_, SecioError>(HandshakeContext::new(config)) - .and_then(|context| { - // Generate our nonce. - let context = context.with_local()?; - trace!("starting handshake; local nonce = {:?}", context.state.nonce); - Ok(context) - }) - .and_then(|context| { - trace!("sending proposition to remote"); - socket.send(BytesMut::from(context.state.proposition_bytes.clone())) - .from_err() - .map(|s| (s, context)) - }) - // Receive the remote's proposition. - .and_then(move |(socket, context)| { - socket.into_future() - .map_err(|(e, _)| e.into()) - .and_then(move |(prop_raw, socket)| { - let context = match prop_raw { - Some(p) => context.with_remote(p)?, - None => { - let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof"); - debug!("unexpected eof while waiting for remote's proposition"); - return Err(err.into()) - }, - }; - trace!("received proposition from remote; pubkey = {:?}; nonce = {:?}", - context.state.public_key, context.state.nonce); - Ok((socket, context)) - }) - }) - // Generate an ephemeral key for the negotiation. - .and_then(|(socket, context)| { - exchange::generate_agreement(context.state.chosen_exchange) - .map(move |(tmp_priv_key, tmp_pub_key)| (socket, context, tmp_priv_key, tmp_pub_key)) - }) - // Send the ephemeral pub key to the remote in an `Exchange` struct. The `Exchange` also - // contains a signature of the two propositions encoded with our static public key. - .and_then(|(socket, context, tmp_priv, tmp_pub_key)| { - let context = context.with_ephemeral(tmp_priv, tmp_pub_key.clone()); - let exchange = { - let mut data_to_sign = context.state.remote.local.proposition_bytes.clone(); - data_to_sign.extend_from_slice(&context.state.remote.proposition_bytes); - data_to_sign.extend_from_slice(&tmp_pub_key); - - let mut exchange = Exchange::new(); - exchange.set_epubkey(tmp_pub_key); - match context.config.key.sign(&data_to_sign) { - Ok(sig) => exchange.set_signature(sig), - Err(_) => return Err(SecioError::SigningFailure) - } - exchange - }; - let local_exch = exchange.write_to_bytes()?; - Ok((BytesMut::from(local_exch), socket, context)) - }) - // Send our local `Exchange`. - .and_then(|(local_exch, socket, context)| { - trace!("sending exchange to remote"); - socket.send(local_exch) - .from_err() - .map(|s| (s, context)) - }) - // Receive the remote's `Exchange`. - .and_then(move |(socket, context)| { - socket.into_future() - .map_err(|(e, _)| e.into()) - .and_then(move |(raw, socket)| { - let raw = match raw { - Some(r) => r, - None => { - let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof"); - debug!("unexpected eof while waiting for remote's exchange"); - return Err(err.into()) - }, - }; - - let remote_exch = match protobuf_parse_from_bytes::(&raw) { - Ok(e) => e, - Err(err) => { - debug!("failed to parse remote's exchange protobuf; {:?}", err); - return Err(SecioError::HandshakeParsingFailure); - } - }; - - trace!("received and decoded the remote's exchange"); - Ok((remote_exch, socket, context)) - }) - }) - // Check the validity of the remote's `Exchange`. This verifies that the remote was really - // the sender of its proposition, and that it is the owner of both its global and ephemeral - // keys. - .and_then(|(remote_exch, socket, context)| { - let mut data_to_verify = context.state.remote.proposition_bytes.clone(); - data_to_verify.extend_from_slice(&context.state.remote.local.proposition_bytes); - data_to_verify.extend_from_slice(remote_exch.get_epubkey()); - - if !context.state.remote.public_key.verify(&data_to_verify, remote_exch.get_signature()) { - return Err(SecioError::SignatureVerificationFailed) - } + let (decoding_cipher, decoding_hmac) = { + let (iv, rest) = remote_infos.split_at(iv_size); + let (cipher_key, mac_key) = rest.split_at(cipher_key_size); + let hmac = Hmac::from_key(chosen_hash, mac_key); + let cipher = ctr(chosen_cipher, cipher_key, iv); + (cipher, hmac) + }; - trace!("successfully verified the remote's signature"); - Ok((remote_exch, socket, context)) - }) - // Generate a key from the local ephemeral private key and the remote ephemeral public key, - // derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder. - .and_then(|(remote_exch, socket, context)| { - let (context, local_priv_key) = context.take_private_key(); - let key_size = context.state.remote.chosen_hash.num_bytes(); - exchange::agree(context.state.remote.chosen_exchange, local_priv_key, remote_exch.get_epubkey(), key_size) - .map(move |key_material| (socket, context, key_material)) - }) - // Generate a key from the local ephemeral private key and the remote ephemeral public key, - // derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder. - .and_then(|(socket, context, key_material)| { - let chosen_cipher = context.state.remote.chosen_cipher; - let cipher_key_size = chosen_cipher.key_size(); - let iv_size = chosen_cipher.iv_size(); - - let key = Hmac::from_key(context.state.remote.chosen_hash, &key_material); - let mut longer_key = vec![0u8; 2 * (iv_size + cipher_key_size + 20)]; - stretch_key(key, &mut longer_key); - - let (local_infos, remote_infos) = { - let (first_half, second_half) = longer_key.split_at(longer_key.len() / 2); - match context.state.remote.hashes_ordering { - Ordering::Equal => { - let msg = "equal digest of public key and nonce for local and remote"; - return Err(SecioError::InvalidProposition(msg)) - } - Ordering::Less => (second_half, first_half), - Ordering::Greater => (first_half, second_half), - } - }; - - let (encoding_cipher, encoding_hmac) = { - let (iv, rest) = local_infos.split_at(iv_size); - let (cipher_key, mac_key) = rest.split_at(cipher_key_size); - let hmac = Hmac::from_key(context.state.remote.chosen_hash, mac_key); - let cipher = ctr(chosen_cipher, cipher_key, iv); - (cipher, hmac) - }; - - let (decoding_cipher, decoding_hmac) = { - let (iv, rest) = remote_infos.split_at(iv_size); - let (cipher_key, mac_key) = rest.split_at(cipher_key_size); - let hmac = Hmac::from_key(context.state.remote.chosen_hash, mac_key); - let cipher = ctr(chosen_cipher, cipher_key, iv); - (cipher, hmac) - }; - - let codec = full_codec( - socket, - encoding_cipher, - encoding_hmac, - decoding_cipher, - decoding_hmac, - context.state.remote.local.nonce.to_vec() - ); - Ok((codec, context)) - }) - // We send back their nonce to check if the connection works. - .and_then(|(codec, context)| { - let remote_nonce = context.state.remote.nonce.clone(); - trace!("checking encryption by sending back remote's nonce"); - codec.send(BytesMut::from(remote_nonce)) - .map(|s| (s, context.state.remote.public_key, context.state.local_tmp_pub_key)) - .from_err() - }) + full_codec( + socket, + encoding_cipher, + encoding_hmac, + decoding_cipher, + decoding_hmac, + local_nonce.to_vec() + ) + }; + + // We send back their nonce to check if the connection works. + trace!("checking encryption by sending back remote's nonce"); + codec.send(remote_nonce).await?; + + Ok((codec, remote_public_key, tmp_pub_key)) } /// Custom algorithm translated from reference implementations. Needs to be the same algorithm @@ -521,16 +359,10 @@ where D: ::hmac::digest::Input + ::hmac::digest::BlockInput + #[cfg(test)] mod tests { - use bytes::BytesMut; + use super::{handshake, stretch_key}; + use crate::{algo_support::Digest, codec::Hmac, SecioConfig}; use libp2p_core::identity; - use tokio::runtime::current_thread::Runtime; - use tokio_tcp::{TcpListener, TcpStream}; - use crate::{SecioConfig, SecioError}; - use super::handshake; - use super::stretch_key; - use crate::algo_support::Digest; - use crate::codec::Hmac; - use futures::prelude::*; + use futures::{prelude::*, channel::oneshot}; #[test] #[cfg(not(any(target_os = "emscripten", target_os = "unknown")))] @@ -572,38 +404,30 @@ mod tests { } fn handshake_with_self_succeeds(key1: SecioConfig, key2: SecioConfig) { - let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap()).unwrap(); - let listener_addr = listener.local_addr().unwrap(); - - let server = listener - .incoming() - .into_future() - .map_err(|(e, _)| e.into()) - .and_then(move |(connec, _)| handshake(connec.unwrap(), key1)) - .and_then(|(connec, _, _)| { - let (sink, stream) = connec.split(); - stream - .filter(|v| !v.is_empty()) - .forward(sink.with(|v| Ok::<_, SecioError>(BytesMut::from(v)))) - }); - - let client = TcpStream::connect(&listener_addr) - .map_err(|e| e.into()) - .and_then(move |stream| handshake(stream, key2)) - .and_then(|(connec, _, _)| { - connec.send("hello".into()) - .from_err() - .and_then(|connec| { - connec.filter(|v| !v.is_empty()) - .into_future() - .map(|(v, _)| v) - .map_err(|(e, _)| e) - }) - .map(|v| assert_eq!(b"hello", &v.unwrap()[..])) - }); - - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(server.join(client)).unwrap(); + let (l_a_tx, l_a_rx) = oneshot::channel(); + + async_std::task::spawn(async move { + let listener = async_std::net::TcpListener::bind(&"127.0.0.1:0").await.unwrap(); + l_a_tx.send(listener.local_addr().unwrap()).unwrap(); + let connec = listener.accept().await.unwrap().0; + let mut codec = handshake(connec, key1).await.unwrap().0; + while let Some(packet) = codec.next().await { + let packet = packet.unwrap(); + if !packet.is_empty() { + codec.send(packet.into()).await.unwrap(); + } + } + }); + + async_std::task::block_on(async move { + let listen_addr = l_a_rx.await.unwrap(); + let connec = async_std::net::TcpStream::connect(&listen_addr).await.unwrap(); + let mut codec = handshake(connec, key2).await.unwrap().0; + codec.send(b"hello".to_vec().into()).await.unwrap(); + let mut packets_stream = codec.filter(|p| future::ready(!p.as_ref().unwrap().is_empty())); + let packet = packets_stream.next().await.unwrap(); + assert_eq!(packet.unwrap(), b"hello"); + }); } #[test] diff --git a/protocols/secio/src/lib.rs b/protocols/secio/src/lib.rs index cba09b475b6..d6d640e9f14 100644 --- a/protocols/secio/src/lib.rs +++ b/protocols/secio/src/lib.rs @@ -29,7 +29,7 @@ //! //! ```no_run //! # fn main() { -//! use futures::Future; +//! use futures::prelude::*; //! use libp2p_secio::{SecioConfig, SecioOutput}; //! use libp2p_core::{PeerId, Multiaddr, identity, upgrade}; //! use libp2p_core::transport::Transport; @@ -57,20 +57,12 @@ pub use self::error::SecioError; -use bytes::BytesMut; use futures::stream::MapErr as StreamMapErr; -use futures::{Future, Poll, Sink, StartSend, Stream}; -use libp2p_core::{ - PeerId, - PublicKey, - identity, - upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade, Negotiated} -}; +use futures::prelude::*; +use libp2p_core::{PeerId, PublicKey, identity, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade}}; use log::debug; use rw_stream_sink::RwStreamSink; -use std::io; -use std::iter; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{io, iter, pin::Pin, task::Context, task::Poll}; mod algo_support; mod codec; @@ -93,7 +85,8 @@ pub struct SecioConfig { pub(crate) key: identity::Keypair, pub(crate) agreements_prop: Option, pub(crate) ciphers_prop: Option, - pub(crate) digests_prop: Option + pub(crate) digests_prop: Option, + pub(crate) max_frame_len: usize } impl SecioConfig { @@ -103,7 +96,8 @@ impl SecioConfig { key: kp, agreements_prop: None, ciphers_prop: None, - digests_prop: None + digests_prop: None, + max_frame_len: 8 * 1024 * 1024 } } @@ -134,13 +128,19 @@ impl SecioConfig { self } - fn handshake(self, socket: T) -> impl Future), Error=SecioError> + /// Override the default max. frame length of 8MiB. + pub fn max_frame_len(mut self, n: usize) -> Self { + self.max_frame_len = n; + self + } + + fn handshake(self, socket: T) -> impl Future), SecioError>> where - T: AsyncRead + AsyncWrite + Send + 'static + T: AsyncRead + AsyncWrite + Unpin + Send + 'static { debug!("Starting secio upgrade"); SecioMiddleware::handshake(socket, self) - .map(|(stream_sink, pubkey, ephemeral)| { + .map_ok(|(stream_sink, pubkey, ephemeral)| { let mapped = stream_sink.map_err(map_err as fn(_) -> _); let peer = pubkey.clone().into_peer_id(); let io = SecioOutput { @@ -156,7 +156,7 @@ impl SecioConfig { /// Output of the secio protocol. pub struct SecioOutput where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin + Send + 'static { /// The encrypted stream. pub stream: RwStreamSink, fn(SecioError) -> io::Error>>, @@ -177,55 +177,61 @@ impl UpgradeInfo for SecioConfig { impl InboundUpgrade for SecioConfig where - T: AsyncRead + AsyncWrite + Send + 'static + T: AsyncRead + AsyncWrite + Unpin + Send + 'static { - type Output = (PeerId, SecioOutput>); + type Output = (PeerId, SecioOutput); type Error = SecioError; - type Future = Box + Send>; + type Future = Pin> + Send>>; - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - Box::new(self.handshake(socket)) + fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future { + Box::pin(self.handshake(socket)) } } impl OutboundUpgrade for SecioConfig where - T: AsyncRead + AsyncWrite + Send + 'static + T: AsyncRead + AsyncWrite + Unpin + Send + 'static { - type Output = (PeerId, SecioOutput>); + type Output = (PeerId, SecioOutput); type Error = SecioError; - type Future = Box + Send>; + type Future = Pin> + Send>>; - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { - Box::new(self.handshake(socket)) + fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future { + Box::pin(self.handshake(socket)) } } -impl io::Read for SecioOutput { - fn read(&mut self, buf: &mut [u8]) -> io::Result { - self.stream.read(buf) - } -} - -impl AsyncRead for SecioOutput { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.stream.prepare_uninitialized_buffer(buf) +impl AsyncRead for SecioOutput +where + S: AsyncRead + AsyncWrite + Unpin + Send + 'static +{ + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) + -> Poll> + { + AsyncRead::poll_read(Pin::new(&mut self.stream), cx, buf) } } -impl io::Write for SecioOutput { - fn write(&mut self, buf: &[u8]) -> io::Result { - self.stream.write(buf) +impl AsyncWrite for SecioOutput +where + S: AsyncRead + AsyncWrite + Unpin + Send + 'static +{ + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) + -> Poll> + { + AsyncWrite::poll_write(Pin::new(&mut self.stream), cx, buf) } - fn flush(&mut self) -> io::Result<()> { - self.stream.flush() + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) + -> Poll> + { + AsyncWrite::poll_flush(Pin::new(&mut self.stream), cx) } -} -impl AsyncWrite for SecioOutput { - fn shutdown(&mut self) -> Poll<(), io::Error> { - self.stream.shutdown() + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) + -> Poll> + { + AsyncWrite::poll_close(Pin::new(&mut self.stream), cx) } } @@ -244,54 +250,52 @@ pub struct SecioMiddleware { impl SecioMiddleware where - S: AsyncRead + AsyncWrite + Send, + S: AsyncRead + AsyncWrite + Send + Unpin + 'static, { /// Attempts to perform a handshake on the given socket. /// /// On success, produces a `SecioMiddleware` that can then be used to encode/decode /// communications, plus the public key of the remote, plus the ephemeral public key. pub fn handshake(socket: S, config: SecioConfig) - -> impl Future, PublicKey, Vec), Error = SecioError> + -> impl Future, PublicKey, Vec), SecioError>> { - handshake::handshake(socket, config).map(|(inner, pubkey, ephemeral)| { + handshake::handshake(socket, config).map_ok(|(inner, pubkey, ephemeral)| { let inner = SecioMiddleware { inner }; (inner, pubkey, ephemeral) }) } } -impl Sink for SecioMiddleware +impl Sink> for SecioMiddleware where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin + Send + 'static { - type SinkItem = BytesMut; - type SinkError = io::Error; + type Error = io::Error; + + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Sink::poll_ready(Pin::new(&mut self.inner), cx) + } - #[inline] - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - self.inner.start_send(item) + fn start_send(mut self: Pin<&mut Self>, item: Vec) -> Result<(), Self::Error> { + Sink::start_send(Pin::new(&mut self.inner), item) } - #[inline] - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.inner.poll_complete() + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Sink::poll_flush(Pin::new(&mut self.inner), cx) } - #[inline] - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.inner.close() + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Sink::poll_close(Pin::new(&mut self.inner), cx) } } impl Stream for SecioMiddleware where - S: AsyncRead + AsyncWrite, + S: AsyncRead + AsyncWrite + Unpin + Send + 'static { - type Item = Vec; - type Error = SecioError; + type Item = Result, SecioError>; - #[inline] - fn poll(&mut self) -> Poll, Self::Error> { - self.inner.poll() + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Stream::poll_next(Pin::new(&mut self.inner), cx) } } diff --git a/src/bandwidth.rs b/src/bandwidth.rs index 4395d7e7fa8..c5497dde7c7 100644 --- a/src/bandwidth.rs +++ b/src/bandwidth.rs @@ -19,11 +19,11 @@ // DEALINGS IN THE SOFTWARE. use crate::{Multiaddr, core::{Transport, transport::{ListenerEvent, TransportError}}}; -use futures::{prelude::*, try_ready}; +use futures::{prelude::*, io::{IoSlice, IoSliceMut}, ready}; use lazy_static::lazy_static; use parking_lot::Mutex; use smallvec::{smallvec, SmallVec}; -use std::{cmp, io, io::Read, io::Write, sync::Arc, time::Duration}; +use std::{cmp, io, pin::Pin, sync::Arc, task::{Context, Poll}, time::Duration}; use wasm_timer::Instant; /// Wraps around a `Transport` and logs the bandwidth that goes through all the opened connections. @@ -35,7 +35,6 @@ pub struct BandwidthLogging { impl BandwidthLogging { /// Creates a new `BandwidthLogging` around the transport. - #[inline] pub fn new(inner: TInner, period: Duration) -> (Self, Arc) { let mut period_seconds = cmp::min(period.as_secs(), 86400) as u32; if period.subsec_nanos() > 0 { @@ -83,51 +82,55 @@ where /// Wraps around a `Stream` that produces connections. Wraps each connection around a bandwidth /// counter. +#[pin_project::pin_project] pub struct BandwidthListener { + #[pin] inner: TInner, sinks: Arc, } impl Stream for BandwidthListener where - TInner: Stream>, + TInner: TryStream> { - type Item = ListenerEvent>; - type Error = TInner::Error; + type Item = Result>, TInner::Error>; - fn poll(&mut self) -> Poll, Self::Error> { - let event = match try_ready!(self.inner.poll()) { - Some(v) => v, - None => return Ok(Async::Ready(None)) - }; + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); - let event = event.map(|inner| { - BandwidthFuture { inner, sinks: self.sinks.clone() } + let event = + if let Some(event) = ready!(this.inner.try_poll_next(cx)?) { + event + } else { + return Poll::Ready(None) + }; + + let event = event.map({ + let sinks = this.sinks.clone(); + |inner| BandwidthFuture { inner, sinks } }); - Ok(Async::Ready(Some(event))) + Poll::Ready(Some(Ok(event))) } } /// Wraps around a `Future` that produces a connection. Wraps the connection around a bandwidth /// counter. +#[pin_project::pin_project] pub struct BandwidthFuture { + #[pin] inner: TInner, sinks: Arc, } -impl Future for BandwidthFuture - where TInner: Future, -{ - type Item = BandwidthConnecLogging; - type Error = TInner::Error; +impl Future for BandwidthFuture { + type Output = Result, TInner::Error>; - fn poll(&mut self) -> Poll { - let inner = try_ready!(self.inner.poll()); - Ok(Async::Ready(BandwidthConnecLogging { - inner, - sinks: self.sinks.clone(), - })) + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = self.project(); + let inner = ready!(this.inner.try_poll(cx)?); + let logged = BandwidthConnecLogging { inner, sinks: this.sinks.clone() }; + Poll::Ready(Ok(logged)) } } @@ -139,74 +142,67 @@ pub struct BandwidthSinks { impl BandwidthSinks { /// Returns the average number of bytes that have been downloaded in the period. - #[inline] pub fn average_download_per_sec(&self) -> u64 { self.download.lock().get() } /// Returns the average number of bytes that have been uploaded in the period. - #[inline] pub fn average_upload_per_sec(&self) -> u64 { self.upload.lock().get() } } /// Wraps around an `AsyncRead + AsyncWrite` and logs the bandwidth that goes through it. +#[pin_project::pin_project] pub struct BandwidthConnecLogging { + #[pin] inner: TInner, sinks: Arc, } -impl Read for BandwidthConnecLogging - where TInner: Read -{ - #[inline] - fn read(&mut self, buf: &mut [u8]) -> io::Result { - let num_bytes = self.inner.read(buf)?; - self.sinks.download.lock().inject(num_bytes); - Ok(num_bytes) +impl AsyncRead for BandwidthConnecLogging { + fn poll_read(self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + let this = self.project(); + let num_bytes = ready!(this.inner.poll_read(cx, buf))?; + this.sinks.download.lock().inject(num_bytes); + Poll::Ready(Ok(num_bytes)) } -} -impl tokio_io::AsyncRead for BandwidthConnecLogging - where TInner: tokio_io::AsyncRead -{ - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.inner.prepare_uninitialized_buffer(buf) + fn poll_read_vectored(self: Pin<&mut Self>, cx: &mut Context, bufs: &mut [IoSliceMut]) -> Poll> { + let this = self.project(); + let num_bytes = ready!(this.inner.poll_read_vectored(cx, bufs))?; + this.sinks.download.lock().inject(num_bytes); + Poll::Ready(Ok(num_bytes)) } +} - fn read_buf(&mut self, buf: &mut B) -> Poll { - self.inner.read_buf(buf) +impl AsyncWrite for BandwidthConnecLogging { + fn poll_write(self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll> { + let this = self.project(); + let num_bytes = ready!(this.inner.poll_write(cx, buf))?; + this.sinks.upload.lock().inject(num_bytes); + Poll::Ready(Ok(num_bytes)) } -} -impl Write for BandwidthConnecLogging - where TInner: Write -{ - #[inline] - fn write(&mut self, buf: &[u8]) -> io::Result { - let num_bytes = self.inner.write(buf)?; - self.sinks.upload.lock().inject(num_bytes); - Ok(num_bytes) + fn poll_write_vectored(self: Pin<&mut Self>, cx: &mut Context, bufs: &[IoSlice]) -> Poll> { + let this = self.project(); + let num_bytes = ready!(this.inner.poll_write_vectored(cx, bufs))?; + this.sinks.upload.lock().inject(num_bytes); + Poll::Ready(Ok(num_bytes)) } - #[inline] - fn flush(&mut self) -> io::Result<()> { - self.inner.flush() + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + this.inner.poll_flush(cx) } -} -impl tokio_io::AsyncWrite for BandwidthConnecLogging - where TInner: tokio_io::AsyncWrite -{ - #[inline] - fn shutdown(&mut self) -> Poll<(), io::Error> { - self.inner.shutdown() + fn poll_close(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let this = self.project(); + this.inner.poll_close(cx) } } /// Returns the number of seconds that have elapsed between an arbitrary EPOCH and now. -#[inline] fn current_second() -> u32 { lazy_static! { static ref EPOCH: Instant = Instant::now(); @@ -267,7 +263,6 @@ impl BandwidthSink { self.bytes.remove(0); self.bytes.push(0); } - self.latest_update = current_second; } } diff --git a/src/lib.rs b/src/lib.rs index 799b8897ba5..df01b2c6936 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -158,8 +158,6 @@ pub use futures; pub use multiaddr; #[doc(inline)] pub use multihash; -pub use tokio_codec; -pub use tokio_io; #[doc(inline)] pub use libp2p_core as core; @@ -234,21 +232,23 @@ use std::{error, io, time::Duration}; /// > reserves the right to support additional protocols or remove deprecated protocols. pub fn build_development_transport( keypair: identity::Keypair, -) -> impl Transport< - Output = ( - PeerId, - impl core::muxing::StreamMuxer< - OutboundSubstream = impl Send, - Substream = impl Send, - Error = impl Into, - > + Send - + Sync, - ), - Error = impl error::Error + Send, - Listener = impl Send, - Dial = impl Send, - ListenerUpgrade = impl Send, -> + Clone { +) -> io::Result< + impl Transport< + Output = ( + PeerId, + impl core::muxing::StreamMuxer< + OutboundSubstream = impl Send, + Substream = impl Send, + Error = impl Into, + > + Send + + Sync, + ), + Error = impl error::Error + Send, + Listener = impl Send, + Dial = impl Send, + ListenerUpgrade = impl Send, + > + Clone, +> { build_tcp_ws_secio_mplex_yamux(keypair) } @@ -260,27 +260,32 @@ pub fn build_development_transport( /// > **Note**: If you ever need to express the type of this `Transport`. pub fn build_tcp_ws_secio_mplex_yamux( keypair: identity::Keypair, -) -> impl Transport< - Output = ( - PeerId, - impl core::muxing::StreamMuxer< - OutboundSubstream = impl Send, - Substream = impl Send, - Error = impl Into, - > + Send - + Sync, - ), - Error = impl error::Error + Send, - Listener = impl Send, - Dial = impl Send, - ListenerUpgrade = impl Send, -> + Clone { - CommonTransport::new() +) -> io::Result< + impl Transport< + Output = ( + PeerId, + impl core::muxing::StreamMuxer< + OutboundSubstream = impl Send, + Substream = impl Send, + Error = impl Into, + > + Send + + Sync, + ), + Error = impl error::Error + Send, + Listener = impl Send, + Dial = impl Send, + ListenerUpgrade = impl Send, + > + Clone, +> { + Ok(CommonTransport::new()? .upgrade(core::upgrade::Version::V1) .authenticate(secio::SecioConfig::new(keypair)) - .multiplex(core::upgrade::SelectUpgrade::new(yamux::Config::default(), mplex::MplexConfig::new())) + .multiplex(core::upgrade::SelectUpgrade::new( + yamux::Config::default(), + mplex::MplexConfig::new(), + )) .map(|(peer, muxer), _| (peer, core::muxing::StreamMuxerBox::new(muxer))) - .timeout(Duration::from_secs(20)) + .timeout(Duration::from_secs(20))) } /// Implementation of `Transport` that supports the most common protocols. @@ -317,27 +322,27 @@ struct CommonTransportInner { impl CommonTransport { /// Initializes the `CommonTransport`. #[cfg(not(any(target_os = "emscripten", target_os = "unknown")))] - pub fn new() -> CommonTransport { + pub fn new() -> io::Result { let tcp = tcp::TcpConfig::new().nodelay(true); - let transport = dns::DnsConfig::new(tcp); + let transport = dns::DnsConfig::new(tcp)?; #[cfg(feature = "libp2p-websocket")] let transport = { let trans_clone = transport.clone(); transport.or_transport(websocket::WsConfig::new(trans_clone)) }; - CommonTransport { + Ok(CommonTransport { inner: CommonTransportInner { inner: transport }, - } + }) } /// Initializes the `CommonTransport`. #[cfg(any(target_os = "emscripten", target_os = "unknown"))] - pub fn new() -> CommonTransport { + pub fn new() -> io::Result { let inner = core::transport::dummy::DummyTransport::new(); - CommonTransport { + Ok(CommonTransport { inner: CommonTransportInner { inner }, - } + }) } } diff --git a/src/simple.rs b/src/simple.rs index 2395fb37439..fb4d3b735d2 100644 --- a/src/simple.rs +++ b/src/simple.rs @@ -18,11 +18,10 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}; +use crate::core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use bytes::Bytes; -use futures::{future::FromErr, prelude::*}; -use std::{iter, io::Error as IoError, sync::Arc}; -use tokio_io::{AsyncRead, AsyncWrite}; +use futures::prelude::*; +use std::{iter, sync::Arc}; /// Implementation of `ConnectionUpgrade`. Convenient to use with small protocols. #[derive(Debug)] @@ -35,7 +34,6 @@ pub struct SimpleProtocol { impl SimpleProtocol { /// Builds a `SimpleProtocol`. - #[inline] pub fn new(info: N, upgrade: F) -> SimpleProtocol where N: Into, @@ -48,7 +46,6 @@ impl SimpleProtocol { } impl Clone for SimpleProtocol { - #[inline] fn clone(&self) -> Self { SimpleProtocol { info: self.info.clone(), @@ -61,42 +58,39 @@ impl UpgradeInfo for SimpleProtocol { type Info = Bytes; type InfoIter = iter::Once; - #[inline] fn protocol_info(&self) -> Self::InfoIter { iter::once(self.info.clone()) } } -impl InboundUpgrade for SimpleProtocol +impl InboundUpgrade for SimpleProtocol where C: AsyncRead + AsyncWrite, - F: Fn(Negotiated) -> O, - O: IntoFuture + F: Fn(C) -> O, + O: Future>, { - type Output = O::Item; - type Error = IoError; - type Future = FromErr; + type Output = A; + type Error = E; + type Future = O; - #[inline] - fn upgrade_inbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: C, _: Self::Info) -> Self::Future { let upgrade = &self.upgrade; - upgrade(socket).into_future().from_err() + upgrade(socket) } } -impl OutboundUpgrade for SimpleProtocol +impl OutboundUpgrade for SimpleProtocol where C: AsyncRead + AsyncWrite, - F: Fn(Negotiated) -> O, - O: IntoFuture + F: Fn(C) -> O, + O: Future>, { - type Output = O::Item; - type Error = IoError; - type Future = FromErr; + type Output = A; + type Error = E; + type Future = O; - #[inline] - fn upgrade_outbound(self, socket: Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: C, _: Self::Info) -> Self::Future { let upgrade = &self.upgrade; - upgrade(socket).into_future().from_err() + upgrade(socket) } } diff --git a/swarm/Cargo.toml b/swarm/Cargo.toml index 57624acc4da..438f1b0d9bc 100644 --- a/swarm/Cargo.toml +++ b/swarm/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-swarm" edition = "2018" description = "The libp2p swarm" -version = "0.3.0" +version = "0.4.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,15 +10,13 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../core" } -smallvec = "0.6" -tokio-io = "0.1" -wasm-timer = "0.1" +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../core" } +smallvec = "1.0" +wasm-timer = "0.2" void = "1" [dev-dependencies] -libp2p-mplex = { version = "0.13.0", path = "../muxers/mplex" } +libp2p-mplex = { version = "0.14.0-alpha.1", path = "../muxers/mplex" } quickcheck = "0.9.0" rand = "0.7.2" - diff --git a/swarm/src/behaviour.rs b/swarm/src/behaviour.rs index e3d72490399..eaec51378b7 100644 --- a/swarm/src/behaviour.rs +++ b/swarm/src/behaviour.rs @@ -20,13 +20,29 @@ use crate::protocols_handler::{IntoProtocolsHandler, ProtocolsHandler}; use libp2p_core::{ConnectedPoint, Multiaddr, PeerId, nodes::ListenerId}; -use futures::prelude::*; -use std::error; +use std::{error, task::Context, task::Poll}; /// A behaviour for the network. Allows customizing the swarm. /// /// This trait has been designed to be composable. Multiple implementations can be combined into /// one that handles all the behaviours at once. +/// +/// # Deriving `NetworkBehaviour` +/// +/// Crate users can implement this trait by using the the `#[derive(NetworkBehaviour)]` +/// proc macro re-exported by the `libp2p` crate. The macro generates a delegating `trait` +/// implementation for the `struct`, which delegates method calls to all trait members. Any events +/// generated by struct members are delegated to [`NetworkBehaviourEventProcess`] implementations +/// which are expected to be provided by the user. +/// +/// Optionally one can implement a custom `poll` function, which needs to be tagged with the +/// `#[behaviour(poll_method = "poll")]` attribute, and would be called last with no parameters. +/// +/// By default the derive sets the `NetworkBehaviour::OutEvent` as `()` but this can be overriden +/// with `#[behaviour(out_event = "AnotherType")]`. +/// +/// `#[behaviour(ignore)]` can be added on a struct field to disable generation of delegation to +/// the fields which do not implement `NetworkBehaviour`. pub trait NetworkBehaviour { /// Handler for all the protocols the network behaviour supports. type ProtocolsHandler: IntoProtocolsHandler; @@ -133,8 +149,8 @@ pub trait NetworkBehaviour { /// /// This API mimics the API of the `Stream` trait. The method may register the current task in /// order to wake it up at a later point in time. - fn poll(&mut self, params: &mut impl PollParameters) - -> Async::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>; + fn poll(&mut self, cx: &mut Context, params: &mut impl PollParameters) + -> Poll::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>; } /// Parameters passed to `poll()`, that the `NetworkBehaviour` has access to. @@ -164,9 +180,8 @@ pub trait PollParameters { fn local_peer_id(&self) -> &PeerId; } -/// Used when deriving `NetworkBehaviour`. When deriving `NetworkBehaviour`, must be implemented -/// for all the possible event types generated by the various fields. -// TODO: document how the custom behaviour works and link this here +/// When deriving [`NetworkBehaviour`] this trait must be implemented for all the possible event types +/// generated by the inner behaviours. pub trait NetworkBehaviourEventProcess { /// Called when one of the fields of the type you're deriving `NetworkBehaviour` on generates /// an event. diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index fd49bdb7149..6dbf7b18bb5 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -80,7 +80,7 @@ pub use protocols_handler::{ use protocols_handler::{NodeHandlerWrapperBuilder, NodeHandlerWrapper, NodeHandlerWrapperError}; use futures::prelude::*; use libp2p_core::{ - Transport, Multiaddr, PeerId, InboundUpgrade, OutboundUpgrade, UpgradeInfo, ProtocolName, + Transport, Multiaddr, Negotiated, PeerId, InboundUpgrade, OutboundUpgrade, UpgradeInfo, ProtocolName, muxing::StreamMuxer, nodes::{ ListenerId, @@ -93,7 +93,7 @@ use libp2p_core::{ }; use registry::{Addresses, AddressIntoIter}; use smallvec::SmallVec; -use std::{error, fmt, io, ops::{Deref, DerefMut}}; +use std::{error, fmt, ops::{Deref, DerefMut}, pin::Pin, task::{Context, Poll}}; use std::collections::HashSet; /// Contains the state of the network, plus the way it should behave. @@ -107,6 +107,33 @@ pub type Swarm = ExpandedSwarm< TConnInfo, >; +/// Event generated by the `Swarm`. +#[derive(Debug)] +pub enum SwarmEvent { + /// Event generated by the `NetworkBehaviour`. + Behaviour(TBvEv), + /// We are now connected to the given peer. + Connected(PeerId), + /// We are now disconnected from the given peer. + Disconnected(PeerId), + /// One of our listeners has reported a new local listening address. + NewListenAddr(Multiaddr), + /// One of our listeners has reported the expiration of a listening address. + ExpiredListenAddr(Multiaddr), + /// Tried to dial an address but it ended up being unreachaable. + UnreachableAddr { + /// `PeerId` that we were trying to reach. `None` if we don't know in advance which peer + /// we were trying to reach. + peer_id: Option, + /// Address that we failed to reach. + address: Multiaddr, + /// Error that has been encountered. + error: Box, + }, + /// Startng to try to reach the given peer. + StartConnect(PeerId), +} + /// Contains the state of the network, plus the way it should behave. pub struct ExpandedSwarm where @@ -140,14 +167,7 @@ where banned_peers: HashSet, /// Pending event message to be delivered. - /// - /// If the pair's second element is `AsyncSink::NotReady`, the event - /// message has yet to be sent using `PeerMut::start_send_event`. - /// - /// If the pair's second element is `AsyncSink::Ready`, the event - /// message has been sent and needs to be flushed using - /// `PeerMut::complete_send_event`. - send_event_to_complete: Option<(PeerId, AsyncSink)> + send_event_to_complete: Option<(PeerId, TInEvent)> } impl Deref for @@ -172,6 +192,13 @@ where } } +impl Unpin for + ExpandedSwarm +where + TTransport: Transport, +{ +} + impl ExpandedSwarm where TBehaviour: NetworkBehaviour, @@ -190,18 +217,18 @@ where TBehaviour: NetworkBehaviour, <::Handler as ProtocolsHandler>::OutEvent: Send + 'static, <::Handler as ProtocolsHandler>::Error: Send + 'static, <::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary - <::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade> + Send + 'static, + <::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade>> + Send + 'static, <<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static, <<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, - <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>::Error: Send + 'static, - <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>::Future: Send + 'static, - <::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade> + Send + 'static, + <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>>::Error: Send + 'static, + <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>>::Future: Send + 'static, + <::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade>> + Send + 'static, <<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static, <<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, - <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>::Future: Send + 'static, - <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>::Error: Send + 'static, + <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>>::Future: Send + 'static, + <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>>::Error: Send + 'static, ::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary TConnInfo: ConnectionInfo + fmt::Debug + Clone + Send + 'static, { @@ -224,7 +251,9 @@ where TBehaviour: NetworkBehaviour, } /// Remove some listener. - pub fn remove_listener(me: &mut Self, id: ListenerId) -> Option { + /// + /// Returns `Ok(())` if there was a listener with this ID. + pub fn remove_listener(me: &mut Self, id: ListenerId) -> Result<(), ()> { me.network.remove_listener(id) } @@ -305,158 +334,161 @@ where TBehaviour: NetworkBehaviour, pub fn unban_peer_id(me: &mut Self, peer_id: PeerId) { me.banned_peers.remove(&peer_id); } -} -impl Stream for - ExpandedSwarm -where TBehaviour: NetworkBehaviour, - TMuxer: StreamMuxer + Send + Sync + 'static, - ::OutboundSubstream: Send + 'static, - ::Substream: Send + 'static, - TTransport: Transport + Clone, - TTransport::Error: Send + 'static, - TTransport::Listener: Send + 'static, - TTransport::ListenerUpgrade: Send + 'static, - TTransport::Dial: Send + 'static, - THandlerErr: error::Error, - THandler: IntoProtocolsHandler + Send + 'static, - ::Handler: ProtocolsHandler, Error = THandlerErr> + Send + 'static, - <::Handler as ProtocolsHandler>::InEvent: Send + 'static, - <::Handler as ProtocolsHandler>::OutEvent: Send + 'static, - <::Handler as ProtocolsHandler>::Error: Send + 'static, - <::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary - <::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade> + Send + 'static, - <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>::Future: Send + 'static, - <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>::Error: Send + 'static, - <<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static, - <<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, - <<<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, - <::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade> + Send + 'static, - <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>::Future: Send + 'static, - <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>::Error: Send + 'static, - <<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static, - <<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, - <<<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, - ::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary - TConnInfo: ConnectionInfo + fmt::Debug + Clone + Send + 'static, -{ - type Item = TBehaviour::OutEvent; - type Error = io::Error; + /// Returns the next event that happens in the `Swarm`. + /// + /// Includes events from the `NetworkBehaviour` but also events about the connections status. + pub async fn next_event(&mut self) -> SwarmEvent { + future::poll_fn(move |cx| ExpandedSwarm::poll_next_event(Pin::new(self), cx)).await + } + + /// Returns the next event produced by the [`NetworkBehaviour`]. + pub async fn next(&mut self) -> TBehaviour::OutEvent { + future::poll_fn(move |cx| { + loop { + let event = futures::ready!(ExpandedSwarm::poll_next_event(Pin::new(self), cx)); + if let SwarmEvent::Behaviour(event) = event { + return Poll::Ready(event); + } + } + }).await + } + + /// Internal function used by everything event-related. + /// + /// Polls the `Swarm` for the next event. + fn poll_next_event(mut self: Pin<&mut Self>, cx: &mut Context) + -> Poll> + { + // We use a `this` variable because the compiler can't mutably borrow multiple times + // across a `Deref`. + let this = &mut *self; - fn poll(&mut self) -> Poll, io::Error> { loop { let mut network_not_ready = false; - match self.network.poll() { - Async::NotReady => network_not_ready = true, - Async::Ready(NetworkEvent::NodeEvent { conn_info, event }) => { - self.behaviour.inject_node_event(conn_info.peer_id().clone(), event); + match this.network.poll(cx) { + Poll::Pending => network_not_ready = true, + Poll::Ready(NetworkEvent::NodeEvent { conn_info, event }) => { + this.behaviour.inject_node_event(conn_info.peer_id().clone(), event); }, - Async::Ready(NetworkEvent::Connected { conn_info, endpoint }) => { - if self.banned_peers.contains(conn_info.peer_id()) { - self.network.peer(conn_info.peer_id().clone()) + Poll::Ready(NetworkEvent::Connected { conn_info, endpoint }) => { + if this.banned_peers.contains(conn_info.peer_id()) { + this.network.peer(conn_info.peer_id().clone()) .into_connected() .expect("the Network just notified us that we were connected; QED") .close(); } else { - self.behaviour.inject_connected(conn_info.peer_id().clone(), endpoint); + this.behaviour.inject_connected(conn_info.peer_id().clone(), endpoint); + return Poll::Ready(SwarmEvent::Connected(conn_info.peer_id().clone())); } }, - Async::Ready(NetworkEvent::NodeClosed { conn_info, endpoint, .. }) => { - self.behaviour.inject_disconnected(conn_info.peer_id(), endpoint); + Poll::Ready(NetworkEvent::NodeClosed { conn_info, endpoint, .. }) => { + this.behaviour.inject_disconnected(conn_info.peer_id(), endpoint); + return Poll::Ready(SwarmEvent::Disconnected(conn_info.peer_id().clone())); }, - Async::Ready(NetworkEvent::Replaced { new_info, closed_endpoint, endpoint, .. }) => { - self.behaviour.inject_replaced(new_info.peer_id().clone(), closed_endpoint, endpoint); + Poll::Ready(NetworkEvent::Replaced { new_info, closed_endpoint, endpoint, .. }) => { + this.behaviour.inject_replaced(new_info.peer_id().clone(), closed_endpoint, endpoint); }, - Async::Ready(NetworkEvent::IncomingConnection(incoming)) => { - let handler = self.behaviour.new_handler(); + Poll::Ready(NetworkEvent::IncomingConnection(incoming)) => { + let handler = this.behaviour.new_handler(); incoming.accept(handler.into_node_handler_builder()); }, - Async::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => { - if !self.listened_addrs.contains(&listen_addr) { - self.listened_addrs.push(listen_addr.clone()) + Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => { + if !this.listened_addrs.contains(&listen_addr) { + this.listened_addrs.push(listen_addr.clone()) } - self.behaviour.inject_new_listen_addr(&listen_addr); + this.behaviour.inject_new_listen_addr(&listen_addr); + return Poll::Ready(SwarmEvent::NewListenAddr(listen_addr)); } - Async::Ready(NetworkEvent::ExpiredListenerAddress { listen_addr, .. }) => { - self.listened_addrs.retain(|a| a != &listen_addr); - self.behaviour.inject_expired_listen_addr(&listen_addr); + Poll::Ready(NetworkEvent::ExpiredListenerAddress { listen_addr, .. }) => { + this.listened_addrs.retain(|a| a != &listen_addr); + this.behaviour.inject_expired_listen_addr(&listen_addr); + return Poll::Ready(SwarmEvent::ExpiredListenAddr(listen_addr)); } - Async::Ready(NetworkEvent::ListenerClosed { listener_id, .. }) => - self.behaviour.inject_listener_closed(listener_id), - Async::Ready(NetworkEvent::ListenerError { listener_id, error }) => - self.behaviour.inject_listener_error(listener_id, &error), - Async::Ready(NetworkEvent::IncomingConnectionError { .. }) => {}, - Async::Ready(NetworkEvent::DialError { peer_id, multiaddr, error, new_state }) => { - self.behaviour.inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error); + Poll::Ready(NetworkEvent::ListenerClosed { listener_id, .. }) => + this.behaviour.inject_listener_closed(listener_id), + Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) => + this.behaviour.inject_listener_error(listener_id, &error), + Poll::Ready(NetworkEvent::IncomingConnectionError { .. }) => {}, + Poll::Ready(NetworkEvent::DialError { peer_id, multiaddr, error, new_state }) => { + this.behaviour.inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error); if let network::PeerState::NotConnected = new_state { - self.behaviour.inject_dial_failure(&peer_id); + this.behaviour.inject_dial_failure(&peer_id); } + return Poll::Ready(SwarmEvent::UnreachableAddr { + peer_id: Some(peer_id.clone()), + address: multiaddr, + error: Box::new(error), + }); }, - Async::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error, .. }) => { - self.behaviour.inject_addr_reach_failure(None, &multiaddr, &error); + Poll::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error, .. }) => { + this.behaviour.inject_addr_reach_failure(None, &multiaddr, &error); + return Poll::Ready(SwarmEvent::UnreachableAddr { + peer_id: None, + address: multiaddr, + error: Box::new(error), + }); }, } // Try to deliver pending event. - if let Some((id, pending)) = self.send_event_to_complete.take() { - if let Some(mut peer) = self.network.peer(id.clone()).into_connected() { - if let AsyncSink::NotReady(e) = pending { - if let Ok(a@AsyncSink::NotReady(_)) = peer.start_send_event(e) { - self.send_event_to_complete = Some((id, a)) - } else if let Ok(Async::NotReady) = peer.complete_send_event() { - self.send_event_to_complete = Some((id, AsyncSink::Ready)) - } - } else if let Ok(Async::NotReady) = peer.complete_send_event() { - self.send_event_to_complete = Some((id, AsyncSink::Ready)) + if let Some((id, pending)) = this.send_event_to_complete.take() { + if let Some(mut peer) = this.network.peer(id.clone()).into_connected() { + match peer.poll_ready_event(cx) { + Poll::Ready(()) => peer.start_send_event(pending), + Poll::Pending => { + this.send_event_to_complete = Some((id, pending)); + return Poll::Pending + }, } } } - if self.send_event_to_complete.is_some() { - return Ok(Async::NotReady) - } let behaviour_poll = { let mut parameters = SwarmPollParameters { - local_peer_id: &mut self.network.local_peer_id(), - supported_protocols: &self.supported_protocols, - listened_addrs: &self.listened_addrs, - external_addrs: &self.external_addrs + local_peer_id: &mut this.network.local_peer_id(), + supported_protocols: &this.supported_protocols, + listened_addrs: &this.listened_addrs, + external_addrs: &this.external_addrs }; - self.behaviour.poll(&mut parameters) + this.behaviour.poll(cx, &mut parameters) }; match behaviour_poll { - Async::NotReady if network_not_ready => return Ok(Async::NotReady), - Async::NotReady => (), - Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) => { - return Ok(Async::Ready(Some(event))) + Poll::Pending if network_not_ready => return Poll::Pending, + Poll::Pending => (), + Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => { + return Poll::Ready(SwarmEvent::Behaviour(event)) }, - Async::Ready(NetworkBehaviourAction::DialAddress { address }) => { - let _ = ExpandedSwarm::dial_addr(self, address); + Poll::Ready(NetworkBehaviourAction::DialAddress { address }) => { + let _ = ExpandedSwarm::dial_addr(&mut *this, address); }, - Async::Ready(NetworkBehaviourAction::DialPeer { peer_id }) => { - if self.banned_peers.contains(&peer_id) { - self.behaviour.inject_dial_failure(&peer_id); + Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id }) => { + if this.banned_peers.contains(&peer_id) { + this.behaviour.inject_dial_failure(&peer_id); } else { - ExpandedSwarm::dial(self, peer_id); + ExpandedSwarm::dial(&mut *this, peer_id.clone()); + return Poll::Ready(SwarmEvent::StartConnect(peer_id)) } }, - Async::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }) => { - if let Some(mut peer) = self.network.peer(peer_id.clone()).into_connected() { - if let Ok(a@AsyncSink::NotReady(_)) = peer.start_send_event(event) { - self.send_event_to_complete = Some((peer_id, a)) - } else if let Ok(Async::NotReady) = peer.complete_send_event() { - self.send_event_to_complete = Some((peer_id, AsyncSink::Ready)) + Poll::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }) => { + if let Some(mut peer) = this.network.peer(peer_id.clone()).into_connected() { + if let Poll::Ready(()) = peer.poll_ready_event(cx) { + peer.start_send_event(event); + } else { + debug_assert!(this.send_event_to_complete.is_none()); + this.send_event_to_complete = Some((peer_id, event)); + return Poll::Pending; } } }, - Async::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => { - for addr in self.network.address_translation(&address) { - if self.external_addrs.iter().all(|a| *a != addr) { - self.behaviour.inject_new_external_addr(&addr); + Poll::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => { + for addr in this.network.address_translation(&address) { + if this.external_addrs.iter().all(|a| *a != addr) { + this.behaviour.inject_new_external_addr(&addr); } - self.external_addrs.add(addr) + this.external_addrs.add(addr); } }, } @@ -464,6 +496,51 @@ where TBehaviour: NetworkBehaviour, } } +impl Stream for + ExpandedSwarm +where TBehaviour: NetworkBehaviour, + TMuxer: StreamMuxer + Send + Sync + 'static, + ::OutboundSubstream: Send + 'static, + ::Substream: Send + 'static, + TTransport: Transport + Clone, + TTransport::Error: Send + 'static, + TTransport::Listener: Send + 'static, + TTransport::ListenerUpgrade: Send + 'static, + TTransport::Dial: Send + 'static, + THandlerErr: error::Error, + THandler: IntoProtocolsHandler + Send + 'static, + ::Handler: ProtocolsHandler, Error = THandlerErr> + Send + 'static, + <::Handler as ProtocolsHandler>::InEvent: Send + 'static, + <::Handler as ProtocolsHandler>::OutEvent: Send + 'static, + <::Handler as ProtocolsHandler>::Error: Send + 'static, + <::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary + <::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade>> + Send + 'static, + <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>>::Future: Send + 'static, + <<::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>>::Error: Send + 'static, + <<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static, + <<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, + <<<::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, + <::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade>> + Send + 'static, + <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>>::Future: Send + 'static, + <<::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>>::Error: Send + 'static, + <<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static, + <<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, + <<<::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, + ::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary + TConnInfo: ConnectionInfo + fmt::Debug + Clone + Send + 'static, +{ + type Item = TBehaviour::OutEvent; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + loop { + let event = futures::ready!(ExpandedSwarm::poll_next_event(self.as_mut(), cx)); + if let SwarmEvent::Behaviour(event) = event { + return Poll::Ready(Some(event)); + } + } + } +} + /// Parameters passed to `poll()`, that the `NetworkBehaviour` has access to. // TODO: #[derive(Debug)] pub struct SwarmPollParameters<'a> { @@ -518,18 +595,18 @@ where TBehaviour: NetworkBehaviour, <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent: Send + 'static, <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error: Send + 'static, <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary - <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade> + Send + 'static, + <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade>> + Send + 'static, <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static, <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, - <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>::Error: Send + 'static, - <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>::Future: Send + 'static, - <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade> + Send + 'static, + <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>>::Error: Send + 'static, + <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade>>>::Future: Send + 'static, + <<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade>> + Send + 'static, <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static, <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, - <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>::Future: Send + 'static, - <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>::Error: Send + 'static, + <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>>::Future: Send + 'static, + <<<::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade>>>::Error: Send + 'static, ::ProtocolsHandler as IntoProtocolsHandler>::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary TConnInfo: ConnectionInfo + fmt::Debug + Clone + Send + 'static, { @@ -584,8 +661,7 @@ mod tests { }; use libp2p_mplex::Multiplex; use futures::prelude::*; - use std::marker::PhantomData; - use tokio_io::{AsyncRead, AsyncWrite}; + use std::{marker::PhantomData, task::Context, task::Poll}; use void::Void; #[derive(Clone)] @@ -593,11 +669,9 @@ mod tests { marker: PhantomData, } - trait TSubstream: AsyncRead + AsyncWrite {} - impl NetworkBehaviour for DummyBehaviour - where TSubstream: AsyncRead + AsyncWrite + where TSubstream: AsyncRead + AsyncWrite + Unpin { type ProtocolsHandler = DummyProtocolsHandler; type OutEvent = Void; @@ -617,11 +691,11 @@ mod tests { fn inject_node_event(&mut self, _: PeerId, _: ::OutEvent) {} - fn poll(&mut self, _: &mut impl PollParameters) -> - Async + Poll::InEvent, Self::OutEvent>> { - Async::NotReady + Poll::Pending } } diff --git a/swarm/src/protocols_handler/dummy.rs b/swarm/src/protocols_handler/dummy.rs index a9719b8540a..a1de1841d35 100644 --- a/swarm/src/protocols_handler/dummy.rs +++ b/swarm/src/protocols_handler/dummy.rs @@ -26,9 +26,8 @@ use crate::protocols_handler::{ ProtocolsHandlerUpgrErr }; use futures::prelude::*; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, DeniedUpgrade}; -use std::marker::PhantomData; -use tokio_io::{AsyncRead, AsyncWrite}; +use libp2p_core::{Negotiated, upgrade::{InboundUpgrade, OutboundUpgrade, DeniedUpgrade}}; +use std::{marker::PhantomData, task::Context, task::Poll}; use void::Void; /// Implementation of `ProtocolsHandler` that doesn't handle anything. @@ -47,7 +46,7 @@ impl Default for DummyProtocolsHandler { impl ProtocolsHandler for DummyProtocolsHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin, { type InEvent = Void; type OutEvent = Void; @@ -65,14 +64,14 @@ where #[inline] fn inject_fully_negotiated_inbound( &mut self, - _: >::Output + _: >>::Output ) { } #[inline] fn inject_fully_negotiated_outbound( &mut self, - _: >::Output, + _: >>::Output, _: Self::OutboundOpenInfo ) { } @@ -81,7 +80,7 @@ where fn inject_event(&mut self, _: Self::InEvent) {} #[inline] - fn inject_dial_upgrade_error(&mut self, _: Self::OutboundOpenInfo, _: ProtocolsHandlerUpgrErr<>::Error>) {} + fn inject_dial_upgrade_error(&mut self, _: Self::OutboundOpenInfo, _: ProtocolsHandlerUpgrErr<>>::Error>) {} #[inline] fn connection_keep_alive(&self) -> KeepAlive { KeepAlive::No } @@ -89,10 +88,10 @@ where #[inline] fn poll( &mut self, + _: &mut Context, ) -> Poll< - ProtocolsHandlerEvent, - Void, + ProtocolsHandlerEvent, > { - Ok(Async::NotReady) + Poll::Pending } } diff --git a/swarm/src/protocols_handler/map_in.rs b/swarm/src/protocols_handler/map_in.rs index e478e58fc09..c80e264f5b5 100644 --- a/swarm/src/protocols_handler/map_in.rs +++ b/swarm/src/protocols_handler/map_in.rs @@ -25,9 +25,8 @@ use crate::protocols_handler::{ ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr }; -use futures::prelude::*; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade}; -use std::marker::PhantomData; +use libp2p_core::{Negotiated, upgrade::{InboundUpgrade, OutboundUpgrade}}; +use std::{marker::PhantomData, task::Context, task::Poll}; /// Wrapper around a protocol handler that turns the input event into something else. pub struct MapInEvent { @@ -69,7 +68,7 @@ where #[inline] fn inject_fully_negotiated_inbound( &mut self, - protocol: >::Output + protocol: >>::Output ) { self.inner.inject_fully_negotiated_inbound(protocol) } @@ -77,7 +76,7 @@ where #[inline] fn inject_fully_negotiated_outbound( &mut self, - protocol: >::Output, + protocol: >>::Output, info: Self::OutboundOpenInfo ) { self.inner.inject_fully_negotiated_outbound(protocol, info) @@ -91,7 +90,7 @@ where } #[inline] - fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>::Error>) { + fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>>::Error>) { self.inner.inject_dial_upgrade_error(info, error) } @@ -103,10 +102,10 @@ where #[inline] fn poll( &mut self, + cx: &mut Context, ) -> Poll< - ProtocolsHandlerEvent, - Self::Error, + ProtocolsHandlerEvent, > { - self.inner.poll() + self.inner.poll(cx) } } diff --git a/swarm/src/protocols_handler/map_out.rs b/swarm/src/protocols_handler/map_out.rs index 5815d9494b2..fd521d0d017 100644 --- a/swarm/src/protocols_handler/map_out.rs +++ b/swarm/src/protocols_handler/map_out.rs @@ -25,8 +25,8 @@ use crate::protocols_handler::{ ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr }; -use futures::prelude::*; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade}; +use libp2p_core::{Negotiated, upgrade::{InboundUpgrade, OutboundUpgrade}}; +use std::task::{Context, Poll}; /// Wrapper around a protocol handler that turns the output event into something else. pub struct MapOutEvent { @@ -66,7 +66,7 @@ where #[inline] fn inject_fully_negotiated_inbound( &mut self, - protocol: >::Output + protocol: >>::Output ) { self.inner.inject_fully_negotiated_inbound(protocol) } @@ -74,7 +74,7 @@ where #[inline] fn inject_fully_negotiated_outbound( &mut self, - protocol: >::Output, + protocol: >>::Output, info: Self::OutboundOpenInfo ) { self.inner.inject_fully_negotiated_outbound(protocol, info) @@ -86,7 +86,7 @@ where } #[inline] - fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>::Error>) { + fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>>::Error>) { self.inner.inject_dial_upgrade_error(info, error) } @@ -98,17 +98,18 @@ where #[inline] fn poll( &mut self, + cx: &mut Context, ) -> Poll< - ProtocolsHandlerEvent, - Self::Error, + ProtocolsHandlerEvent, > { - Ok(self.inner.poll()?.map(|ev| { + self.inner.poll(cx).map(|ev| { match ev { ProtocolsHandlerEvent::Custom(ev) => ProtocolsHandlerEvent::Custom((self.map)(ev)), + ProtocolsHandlerEvent::Close(err) => ProtocolsHandlerEvent::Close(err), ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => { ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } } } - })) + }) } } diff --git a/swarm/src/protocols_handler/mod.rs b/swarm/src/protocols_handler/mod.rs index 3ad4d3037a0..ef339fa4bbf 100644 --- a/swarm/src/protocols_handler/mod.rs +++ b/swarm/src/protocols_handler/mod.rs @@ -47,11 +47,11 @@ mod select; use futures::prelude::*; use libp2p_core::{ ConnectedPoint, + Negotiated, PeerId, upgrade::{self, InboundUpgrade, OutboundUpgrade, UpgradeError}, }; -use std::{cmp::Ordering, error, fmt, time::Duration}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{cmp::Ordering, error, fmt, task::Context, task::Poll, time::Duration}; use wasm_timer::Instant; pub use dummy::DummyProtocolsHandler; @@ -101,11 +101,11 @@ pub trait ProtocolsHandler { /// The type of errors returned by [`ProtocolsHandler::poll`]. type Error: error::Error; /// The type of substreams on which the protocol(s) are negotiated. - type Substream: AsyncRead + AsyncWrite; + type Substream: AsyncRead + AsyncWrite + Unpin; /// The inbound upgrade for the protocol(s) used by the handler. - type InboundProtocol: InboundUpgrade; + type InboundProtocol: InboundUpgrade>; /// The outbound upgrade for the protocol(s) used by the handler. - type OutboundProtocol: OutboundUpgrade; + type OutboundProtocol: OutboundUpgrade>; /// The type of additional information passed to an `OutboundSubstreamRequest`. type OutboundOpenInfo; @@ -121,7 +121,7 @@ pub trait ProtocolsHandler { /// Injects the output of a successful upgrade on a new inbound substream. fn inject_fully_negotiated_inbound( &mut self, - protocol: >::Output + protocol: >>::Output ); /// Injects the output of a successful upgrade on a new outbound substream. @@ -130,7 +130,7 @@ pub trait ProtocolsHandler { /// [`ProtocolsHandlerEvent::OutboundSubstreamRequest`]. fn inject_fully_negotiated_outbound( &mut self, - protocol: >::Output, + protocol: >>::Output, info: Self::OutboundOpenInfo ); @@ -142,7 +142,7 @@ pub trait ProtocolsHandler { &mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr< - >::Error + >>::Error > ); @@ -169,11 +169,8 @@ pub trait ProtocolsHandler { fn connection_keep_alive(&self) -> KeepAlive; /// Should behave like `Stream::poll()`. - /// - /// Returning an error will close the connection to the remote. - fn poll(&mut self) -> Poll< - ProtocolsHandlerEvent, - Self::Error + fn poll(&mut self, cx: &mut Context) -> Poll< + ProtocolsHandlerEvent >; /// Adds a closure that turns the input event into something else. @@ -310,7 +307,7 @@ impl From for SubstreamProtocol { /// Event produced by a handler. #[derive(Debug, Copy, Clone, PartialEq, Eq)] -pub enum ProtocolsHandlerEvent { +pub enum ProtocolsHandlerEvent { /// Request a new outbound substream to be opened with the remote. OutboundSubstreamRequest { /// The protocol(s) to apply on the substream. @@ -319,13 +316,16 @@ pub enum ProtocolsHandlerEvent { info: TOutboundOpenInfo, }, + /// Close the connection for the given reason. + Close(TErr), + /// Other event. Custom(TCustom), } /// Event produced by a handler. -impl - ProtocolsHandlerEvent +impl + ProtocolsHandlerEvent { /// If this is an `OutboundSubstreamRequest`, maps the `info` member from a /// `TOutboundOpenInfo` to something else. @@ -333,7 +333,7 @@ impl pub fn map_outbound_open_info( self, map: F, - ) -> ProtocolsHandlerEvent + ) -> ProtocolsHandlerEvent where F: FnOnce(TOutboundOpenInfo) -> I, { @@ -345,6 +345,7 @@ impl } } ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(val), + ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(val), } } @@ -354,7 +355,7 @@ impl pub fn map_protocol( self, map: F, - ) -> ProtocolsHandlerEvent + ) -> ProtocolsHandlerEvent where F: FnOnce(TConnectionUpgrade) -> I, { @@ -366,6 +367,7 @@ impl } } ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(val), + ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(val), } } @@ -374,7 +376,7 @@ impl pub fn map_custom( self, map: F, - ) -> ProtocolsHandlerEvent + ) -> ProtocolsHandlerEvent where F: FnOnce(TCustom) -> I, { @@ -383,6 +385,25 @@ impl ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } } ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(map(val)), + ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(val), + } + } + + /// If this is a `Close` event, maps the content to something else. + #[inline] + pub fn map_close( + self, + map: F, + ) -> ProtocolsHandlerEvent + where + F: FnOnce(TErr) -> I, + { + match self { + ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => { + ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } + } + ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(val), + ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(map(val)), } } } diff --git a/swarm/src/protocols_handler/node_handler.rs b/swarm/src/protocols_handler/node_handler.rs index 15c9bcc0e87..686b14bc2ea 100644 --- a/swarm/src/protocols_handler/node_handler.rs +++ b/swarm/src/protocols_handler/node_handler.rs @@ -33,8 +33,8 @@ use libp2p_core::{ nodes::handled_node::{IntoNodeHandler, NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent}, upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply} }; -use std::{error, fmt, time::Duration}; -use wasm_timer::{Delay, Timeout}; +use std::{error, fmt, pin::Pin, task::Context, task::Poll, time::Duration}; +use wasm_timer::{Delay, Instant}; /// Prototype for a `NodeHandlerWrapper`. pub struct NodeHandlerWrapperBuilder { @@ -102,12 +102,13 @@ where handler: TProtoHandler, /// Futures that upgrade incoming substreams. negotiating_in: - Vec>>, + Vec<(InboundUpgradeApply, Delay)>, /// Futures that upgrade outgoing substreams. The first element of the tuple is the userdata /// to pass back once successfully opened. negotiating_out: Vec<( TProtoHandler::OutboundOpenInfo, - Timeout>, + OutboundUpgradeApply, + Delay, )>, /// For each outbound substream request, how to upgrade it. The first element of the tuple /// is the unique identifier (see `unique_dial_upgrade_id`). @@ -133,7 +134,7 @@ enum Shutdown { /// A shut down is planned as soon as possible. Asap, /// A shut down is planned for when a `Delay` has elapsed. - Later(Delay) + Later(Delay, Instant) } /// Error generated by the `NodeHandlerWrapper`. @@ -198,8 +199,8 @@ where let protocol = self.handler.listen_protocol(); let timeout = protocol.timeout().clone(); let upgrade = upgrade::apply_inbound(substream, protocol.into_upgrade().1); - let with_timeout = Timeout::new(upgrade, timeout); - self.negotiating_in.push(with_timeout); + let timeout = Delay::new(timeout); + self.negotiating_in.push((upgrade, timeout)); } NodeHandlerEndpoint::Dialer((upgrade_id, user_data, timeout)) => { let pos = match self @@ -216,8 +217,8 @@ where let (_, (version, upgrade)) = self.queued_dial_upgrades.remove(pos); let upgrade = upgrade::apply_outbound(substream, upgrade, version); - let with_timeout = Timeout::new(upgrade, timeout); - self.negotiating_out.push((user_data, with_timeout)); + let timeout = Delay::new(timeout); + self.negotiating_out.push((user_data, upgrade, timeout)); } } } @@ -227,44 +228,50 @@ where self.handler.inject_event(event); } - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, cx: &mut Context) -> Poll, Self::Error>> { // Continue negotiation of newly-opened substreams on the listening side. // We remove each element from `negotiating_in` one by one and add them back if not ready. for n in (0..self.negotiating_in.len()).rev() { - let mut in_progress = self.negotiating_in.swap_remove(n); - match in_progress.poll() { - Ok(Async::Ready(upgrade)) => + let (mut in_progress, mut timeout) = self.negotiating_in.swap_remove(n); + match Future::poll(Pin::new(&mut timeout), cx) { + Poll::Ready(_) => continue, + Poll::Pending => {}, + } + match Future::poll(Pin::new(&mut in_progress), cx) { + Poll::Ready(Ok(upgrade)) => self.handler.inject_fully_negotiated_inbound(upgrade), - Ok(Async::NotReady) => self.negotiating_in.push(in_progress), + Poll::Pending => self.negotiating_in.push((in_progress, timeout)), // TODO: return a diagnostic event? - Err(_err) => {} + Poll::Ready(Err(_err)) => {} } } // Continue negotiation of newly-opened substreams. // We remove each element from `negotiating_out` one by one and add them back if not ready. for n in (0..self.negotiating_out.len()).rev() { - let (upgr_info, mut in_progress) = self.negotiating_out.swap_remove(n); - match in_progress.poll() { - Ok(Async::Ready(upgrade)) => { + let (upgr_info, mut in_progress, mut timeout) = self.negotiating_out.swap_remove(n); + match Future::poll(Pin::new(&mut timeout), cx) { + Poll::Ready(Ok(_)) => { + let err = ProtocolsHandlerUpgrErr::Timeout; + self.handler.inject_dial_upgrade_error(upgr_info, err); + continue; + }, + Poll::Ready(Err(_)) => { + let err = ProtocolsHandlerUpgrErr::Timer; + self.handler.inject_dial_upgrade_error(upgr_info, err); + continue; + }, + Poll::Pending => {}, + } + match Future::poll(Pin::new(&mut in_progress), cx) { + Poll::Ready(Ok(upgrade)) => { self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info); } - Ok(Async::NotReady) => { - self.negotiating_out.push((upgr_info, in_progress)); + Poll::Pending => { + self.negotiating_out.push((upgr_info, in_progress, timeout)); } - Err(err) => { - let err = if err.is_elapsed() { - ProtocolsHandlerUpgrErr::Timeout - } else if err.is_timer() { - ProtocolsHandlerUpgrErr::Timer - } else { - debug_assert!(err.is_inner()); - let err = err.into_inner().expect("Timeout error is one of {elapsed, \ - timer, inner}; is_elapsed and is_timer are both false; error is \ - inner; QED"); - ProtocolsHandlerUpgrErr::Upgrade(err) - }; - + Poll::Ready(Err(err)) => { + let err = ProtocolsHandlerUpgrErr::Upgrade(err); self.handler.inject_dial_upgrade_error(upgr_info, err); } } @@ -272,25 +279,26 @@ where // Poll the handler at the end so that we see the consequences of the method // calls on `self.handler`. - let poll_result = self.handler.poll()?; + let poll_result = self.handler.poll(cx); // Ask the handler whether it wants the connection (and the handler itself) // to be kept alive, which determines the planned shutdown, if any. match (&mut self.shutdown, self.handler.connection_keep_alive()) { - (Shutdown::Later(d), KeepAlive::Until(t)) => - if d.deadline() != t { - d.reset(t) + (Shutdown::Later(timer, deadline), KeepAlive::Until(t)) => + if *deadline != t { + *deadline = t; + timer.reset_at(t) }, - (_, KeepAlive::Until(t)) => self.shutdown = Shutdown::Later(Delay::new(t)), + (_, KeepAlive::Until(t)) => self.shutdown = Shutdown::Later(Delay::new_at(t), t), (_, KeepAlive::No) => self.shutdown = Shutdown::Asap, (_, KeepAlive::Yes) => self.shutdown = Shutdown::None }; match poll_result { - Async::Ready(ProtocolsHandlerEvent::Custom(event)) => { - return Ok(Async::Ready(NodeHandlerEvent::Custom(event))); + Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => { + return Poll::Ready(Ok(NodeHandlerEvent::Custom(event))); } - Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info, }) => { @@ -298,11 +306,12 @@ where let timeout = protocol.timeout().clone(); self.unique_dial_upgrade_id += 1; self.queued_dial_upgrades.push((id, protocol.into_upgrade())); - return Ok(Async::Ready( + return Poll::Ready(Ok( NodeHandlerEvent::OutboundSubstreamRequest((id, info, timeout)), )); } - Async::NotReady => (), + Poll::Ready(ProtocolsHandlerEvent::Close(err)) => return Poll::Ready(Err(err.into())), + Poll::Pending => (), }; // Check if the connection (and handler) should be shut down. @@ -310,15 +319,14 @@ where if self.negotiating_in.is_empty() && self.negotiating_out.is_empty() { match self.shutdown { Shutdown::None => {}, - Shutdown::Asap => return Err(NodeHandlerWrapperError::UselessTimeout), - Shutdown::Later(ref mut delay) => match delay.poll() { - Ok(Async::Ready(_)) | Err(_) => - return Err(NodeHandlerWrapperError::UselessTimeout), - Ok(Async::NotReady) => {} + Shutdown::Asap => return Poll::Ready(Err(NodeHandlerWrapperError::UselessTimeout)), + Shutdown::Later(ref mut delay, _) => match Future::poll(Pin::new(delay), cx) { + Poll::Ready(_) => return Poll::Ready(Err(NodeHandlerWrapperError::UselessTimeout)), + Poll::Pending => {} } } } - Ok(Async::NotReady) + Poll::Pending } } diff --git a/swarm/src/protocols_handler/one_shot.rs b/swarm/src/protocols_handler/one_shot.rs index c685dfb9e96..5375a734626 100644 --- a/swarm/src/protocols_handler/one_shot.rs +++ b/swarm/src/protocols_handler/one_shot.rs @@ -26,10 +26,9 @@ use crate::protocols_handler::{ SubstreamProtocol }; use futures::prelude::*; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade}; +use libp2p_core::{Negotiated, upgrade::{InboundUpgrade, OutboundUpgrade}}; use smallvec::SmallVec; -use std::{error, marker::PhantomData, time::Duration}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{error, marker::PhantomData, task::Context, task::Poll, time::Duration}; use wasm_timer::Instant; /// Implementation of `ProtocolsHandler` that opens a new substream for each individual message. @@ -38,13 +37,13 @@ use wasm_timer::Instant; // TODO: Debug pub struct OneShotHandler where - TOutProto: OutboundUpgrade, + TOutProto: OutboundUpgrade>, { /// The upgrade for inbound substreams. listen_protocol: SubstreamProtocol, /// If `Some`, something bad happened and we should shut down the handler with an error. pending_error: - Option>::Error>>, + Option>>::Error>>, /// Queue of events to produce in `poll()`. events_out: SmallVec<[TOutEvent; 4]>, /// Queue of outbound substreams to open. @@ -64,7 +63,7 @@ where impl OneShotHandler where - TOutProto: OutboundUpgrade, + TOutProto: OutboundUpgrade>, { /// Creates a `OneShotHandler`. #[inline] @@ -120,8 +119,8 @@ where impl Default for OneShotHandler where - TOutProto: OutboundUpgrade, - TInProto: InboundUpgrade + Default, + TOutProto: OutboundUpgrade>, + TInProto: InboundUpgrade> + Default, { #[inline] fn default() -> Self { @@ -132,9 +131,9 @@ where impl ProtocolsHandler for OneShotHandler where - TSubstream: AsyncRead + AsyncWrite, - TInProto: InboundUpgrade, - TOutProto: OutboundUpgrade, + TSubstream: AsyncRead + AsyncWrite + Unpin, + TInProto: InboundUpgrade>, + TOutProto: OutboundUpgrade>, TInProto::Output: Into, TOutProto::Output: Into, TOutProto::Error: error::Error + 'static, @@ -143,7 +142,7 @@ where type InEvent = TOutProto; type OutEvent = TOutEvent; type Error = ProtocolsHandlerUpgrErr< - >::Error, + >>::Error, >; type Substream = TSubstream; type InboundProtocol = TInProto; @@ -158,7 +157,7 @@ where #[inline] fn inject_fully_negotiated_inbound( &mut self, - out: >::Output, + out: >>::Output, ) { // If we're shutting down the connection for inactivity, reset the timeout. if !self.keep_alive.is_yes() { @@ -171,7 +170,7 @@ where #[inline] fn inject_fully_negotiated_outbound( &mut self, - out: >::Output, + out: >>::Output, _: Self::OutboundOpenInfo, ) { self.dial_negotiated -= 1; @@ -193,7 +192,7 @@ where &mut self, _: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr< - >::Error, + >>::Error, >, ) { if self.pending_error.is_none() { @@ -208,18 +207,18 @@ where fn poll( &mut self, + _: &mut Context, ) -> Poll< - ProtocolsHandlerEvent, - Self::Error, + ProtocolsHandlerEvent, > { if let Some(err) = self.pending_error.take() { - return Err(err); + return Poll::Ready(ProtocolsHandlerEvent::Close(err)); } if !self.events_out.is_empty() { - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom( + return Poll::Ready(ProtocolsHandlerEvent::Custom( self.events_out.remove(0), - ))); + )); } else { self.events_out.shrink_to_fit(); } @@ -227,17 +226,17 @@ where if !self.dial_queue.is_empty() { if self.dial_negotiated < self.max_dial_negotiated { self.dial_negotiated += 1; - return Ok(Async::Ready( + return Poll::Ready( ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(self.dial_queue.remove(0)), info: (), }, - )); + ); } } else { self.dial_queue.shrink_to_fit(); } - Ok(Async::NotReady) + Poll::Pending } } diff --git a/swarm/src/protocols_handler/select.rs b/swarm/src/protocols_handler/select.rs index 7e930596eb2..b9ddc9e1153 100644 --- a/swarm/src/protocols_handler/select.rs +++ b/swarm/src/protocols_handler/select.rs @@ -29,12 +29,12 @@ use crate::protocols_handler::{ use futures::prelude::*; use libp2p_core::{ ConnectedPoint, + Negotiated, PeerId, either::{EitherError, EitherOutput}, upgrade::{InboundUpgrade, OutboundUpgrade, EitherUpgrade, SelectUpgrade, UpgradeError} }; -use std::cmp; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{cmp, task::Context, task::Poll}; /// Implementation of `IntoProtocolsHandler` that combines two protocols into one. #[derive(Debug, Clone)] @@ -62,11 +62,11 @@ where TProto2: IntoProtocolsHandler, TProto1::Handler: ProtocolsHandler, TProto2::Handler: ProtocolsHandler, - TSubstream: AsyncRead + AsyncWrite, - ::InboundProtocol: InboundUpgrade, - ::InboundProtocol: InboundUpgrade, - ::OutboundProtocol: OutboundUpgrade, - ::OutboundProtocol: OutboundUpgrade + TSubstream: AsyncRead + AsyncWrite + Unpin, + ::InboundProtocol: InboundUpgrade>, + ::InboundProtocol: InboundUpgrade>, + ::OutboundProtocol: OutboundUpgrade>, + ::OutboundProtocol: OutboundUpgrade> { type Handler = ProtocolsHandlerSelect; @@ -107,11 +107,11 @@ impl where TProto1: ProtocolsHandler, TProto2: ProtocolsHandler, - TSubstream: AsyncRead + AsyncWrite, - TProto1::InboundProtocol: InboundUpgrade, - TProto2::InboundProtocol: InboundUpgrade, - TProto1::OutboundProtocol: OutboundUpgrade, - TProto2::OutboundProtocol: OutboundUpgrade + TSubstream: AsyncRead + AsyncWrite + Unpin, + TProto1::InboundProtocol: InboundUpgrade>, + TProto2::InboundProtocol: InboundUpgrade>, + TProto1::OutboundProtocol: OutboundUpgrade>, + TProto2::OutboundProtocol: OutboundUpgrade> { type InEvent = EitherOutput; type OutEvent = EitherOutput; @@ -130,7 +130,7 @@ where SubstreamProtocol::new(choice).with_timeout(timeout) } - fn inject_fully_negotiated_outbound(&mut self, protocol: >::Output, endpoint: Self::OutboundOpenInfo) { + fn inject_fully_negotiated_outbound(&mut self, protocol: >>::Output, endpoint: Self::OutboundOpenInfo) { match (protocol, endpoint) { (EitherOutput::First(protocol), EitherOutput::First(info)) => self.proto1.inject_fully_negotiated_outbound(protocol, info), @@ -143,7 +143,7 @@ where } } - fn inject_fully_negotiated_inbound(&mut self, protocol: >::Output) { + fn inject_fully_negotiated_inbound(&mut self, protocol: >>::Output) { match protocol { EitherOutput::First(protocol) => self.proto1.inject_fully_negotiated_inbound(protocol), @@ -161,7 +161,7 @@ where } #[inline] - fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>::Error>) { + fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<>>::Error>) { match (info, error) { (EitherOutput::First(info), ProtocolsHandlerUpgrErr::Timer) => { self.proto1.inject_dial_upgrade_error(info, ProtocolsHandlerUpgrErr::Timer) @@ -201,40 +201,46 @@ where cmp::max(self.proto1.connection_keep_alive(), self.proto2.connection_keep_alive()) } - fn poll(&mut self) -> Poll, Self::Error> { + fn poll(&mut self, cx: &mut Context) -> Poll> { - match self.proto1.poll().map_err(EitherError::A)? { - Async::Ready(ProtocolsHandlerEvent::Custom(event)) => { - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::First(event)))); + match self.proto1.poll(cx) { + Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => { + return Poll::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::First(event))); }, - Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + Poll::Ready(ProtocolsHandlerEvent::Close(event)) => { + return Poll::Ready(ProtocolsHandlerEvent::Close(EitherError::A(event))); + }, + Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info, }) => { - return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol: protocol.map_upgrade(EitherUpgrade::A), info: EitherOutput::First(info), - })); + }); }, - Async::NotReady => () + Poll::Pending => () }; - match self.proto2.poll().map_err(EitherError::B)? { - Async::Ready(ProtocolsHandlerEvent::Custom(event)) => { - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::Second(event)))); + match self.proto2.poll(cx) { + Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => { + return Poll::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::Second(event))); + }, + Poll::Ready(ProtocolsHandlerEvent::Close(event)) => { + return Poll::Ready(ProtocolsHandlerEvent::Close(EitherError::B(event))); }, - Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info, }) => { - return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol: protocol.map_upgrade(EitherUpgrade::B), info: EitherOutput::Second(info), - })); + }); }, - Async::NotReady => () + Poll::Pending => () }; - Ok(Async::NotReady) + Poll::Pending } } diff --git a/swarm/src/toggle.rs b/swarm/src/toggle.rs index 002ab626c4b..e62a20e0c8a 100644 --- a/swarm/src/toggle.rs +++ b/swarm/src/toggle.rs @@ -31,11 +31,11 @@ use libp2p_core::{ ConnectedPoint, PeerId, Multiaddr, + Negotiated, either::EitherOutput, upgrade::{InboundUpgrade, OutboundUpgrade, DeniedUpgrade, EitherUpgrade} }; -use futures::prelude::*; -use std::error; +use std::{error, task::Context, task::Poll}; /// Implementation of `NetworkBehaviour` that can be either in the disabled or enabled state. /// @@ -132,13 +132,13 @@ where } } - fn poll(&mut self, params: &mut impl PollParameters) - -> Async::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>> + fn poll(&mut self, cx: &mut Context, params: &mut impl PollParameters) + -> Poll::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>> { if let Some(inner) = self.inner.as_mut() { - inner.poll(params) + inner.poll(cx, params) } else { - Async::NotReady + Poll::Pending } } } @@ -207,7 +207,7 @@ where fn inject_fully_negotiated_inbound( &mut self, - out: >::Output + out: >>::Output ) { let out = match out { EitherOutput::First(out) => out, @@ -220,7 +220,7 @@ where fn inject_fully_negotiated_outbound( &mut self, - out: >::Output, + out: >>::Output, info: Self::OutboundOpenInfo ) { self.inner.as_mut().expect("Can't receive an outbound substream if disabled; QED") @@ -232,7 +232,7 @@ where .inject_event(event) } - fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, err: ProtocolsHandlerUpgrErr<>::Error>) { + fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, err: ProtocolsHandlerUpgrErr<>>::Error>) { self.inner.as_mut().expect("Can't receive an outbound substream if disabled; QED") .inject_dial_upgrade_error(info, err) } @@ -244,14 +244,14 @@ where fn poll( &mut self, + cx: &mut Context, ) -> Poll< - ProtocolsHandlerEvent, - Self::Error, + ProtocolsHandlerEvent > { if let Some(inner) = self.inner.as_mut() { - inner.poll() + inner.poll(cx) } else { - Ok(Async::NotReady) + Poll::Pending } } } diff --git a/transports/dns/Cargo.toml b/transports/dns/Cargo.toml index ba01f323d52..44c6089e34b 100644 --- a/transports/dns/Cargo.toml +++ b/transports/dns/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-dns" edition = "2018" description = "DNS transport implementation for libp2p" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,10 +10,6 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -libp2p-core = { version = "0.13.0", path = "../../core" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } log = "0.4.1" -futures = "0.1" -tokio-dns-unofficial = "0.4" - -[dev-dependencies] -libp2p-tcp = { version = "0.13.0", path = "../../transports/tcp" } +futures = "0.3.1" diff --git a/transports/dns/src/lib.rs b/transports/dns/src/lib.rs index 7f0dddfdd7d..6e03b99ab73 100644 --- a/transports/dns/src/lib.rs +++ b/transports/dns/src/lib.rs @@ -33,15 +33,14 @@ //! replaced with respectively an `/ip4/` or an `/ip6/` component. //! -use futures::{future::{self, Either, FutureResult, JoinAll}, prelude::*, stream, try_ready}; +use futures::{prelude::*, channel::oneshot, future::BoxFuture}; use libp2p_core::{ Transport, multiaddr::{Protocol, Multiaddr}, transport::{TransportError, ListenerEvent} }; -use log::{debug, trace, log_enabled, Level}; -use std::{error, fmt, io, marker::PhantomData, net::IpAddr}; -use tokio_dns::{CpuPoolResolver, Resolver}; +use log::{error, debug, trace}; +use std::{error, fmt, io, net::ToSocketAddrs}; /// Represents the configuration for a DNS transport capability of libp2p. /// @@ -52,24 +51,31 @@ use tokio_dns::{CpuPoolResolver, Resolver}; /// Listening is unaffected. #[derive(Clone)] pub struct DnsConfig { + /// Underlying transport to use once the DNS addresses have been resolved. inner: T, - resolver: CpuPoolResolver, + /// Pool of threads to use when resolving DNS addresses. + thread_pool: futures::executor::ThreadPool, } impl DnsConfig { /// Creates a new configuration object for DNS. - pub fn new(inner: T) -> DnsConfig { + pub fn new(inner: T) -> Result, io::Error> { DnsConfig::with_resolve_threads(inner, 1) } /// Same as `new`, but allows specifying a number of threads for the resolving. - pub fn with_resolve_threads(inner: T, num_threads: usize) -> DnsConfig { - trace!("Created a CpuPoolResolver"); + pub fn with_resolve_threads(inner: T, num_threads: usize) -> Result, io::Error> { + let thread_pool = futures::executor::ThreadPool::builder() + .pool_size(num_threads) + .name_prefix("libp2p-dns-") + .create()?; - DnsConfig { + trace!("Created a DNS thread pool"); + + Ok(DnsConfig { inner, - resolver: CpuPoolResolver::new(num_threads), - } + thread_pool, + }) } } @@ -84,34 +90,35 @@ where impl Transport for DnsConfig where - T: Transport, - T::Error: 'static, + T: Transport + Send + 'static, + T::Error: Send, + T::Dial: Send { type Output = T::Output; type Error = DnsErr; type Listener = stream::MapErr< - stream::Map) -> ListenerEvent>, fn(T::Error) -> Self::Error>; type ListenerUpgrade = future::MapErr Self::Error>; - type Dial = Either Self::Error>, - DialFuture>, T::Error>, - FutureResult, Self::Error>>>> - >> + type Dial = future::Either< + future::MapErr Self::Error>, + BoxFuture<'static, Result> >; fn listen_on(self, addr: Multiaddr) -> Result> { let listener = self.inner.listen_on(addr).map_err(|err| err.map(DnsErr::Underlying))?; let listener = listener - .map::<_, fn(_) -> _>(|event| event.map(|upgr| { - upgr.map_err:: _, _>(DnsErr::Underlying) + .map_ok::<_, fn(_) -> _>(|event| event.map(|upgr| { + upgr.map_err::<_, fn(_) -> _>(DnsErr::Underlying) })) .map_err::<_, fn(_) -> _>(DnsErr::Underlying); Ok(listener) } fn dial(self, addr: Multiaddr) -> Result> { + // As an optimization, we immediately pass through if no component of the address contain + // a DNS protocol. let contains_dns = addr.iter().any(|cmp| match cmp { Protocol::Dns4(_) => true, Protocol::Dns6(_) => true, @@ -120,44 +127,70 @@ where if !contains_dns { trace!("Pass-through address without DNS: {}", addr); - let inner_dial = self.inner.dial(addr).map_err(|err| err.map(DnsErr::Underlying))?; - return Ok(Either::A(inner_dial.map_err(DnsErr::Underlying))); + let inner_dial = self.inner.dial(addr) + .map_err(|err| err.map(DnsErr::Underlying))?; + return Ok(inner_dial.map_err::<_, fn(_) -> _>(DnsErr::Underlying).left_future()); } - let resolver = self.resolver; - trace!("Dialing address with DNS: {}", addr); - let resolve_iters = addr.iter() - .map(move |cmp| match cmp { - Protocol::Dns4(ref name) => - Either::A(ResolveFuture { - name: if log_enabled!(Level::Trace) { - Some(name.clone().into_owned()) - } else { - None - }, - inner: resolver.resolve(name), - ty: ResolveTy::Dns4, - error_ty: PhantomData, - }), - Protocol::Dns6(ref name) => - Either::A(ResolveFuture { - name: if log_enabled!(Level::Trace) { - Some(name.clone().into_owned()) - } else { - None - }, - inner: resolver.resolve(name), - ty: ResolveTy::Dns6, - error_ty: PhantomData, - }), - cmp => Either::B(future::ok(cmp.acquire())) + let resolve_futs = addr.iter() + .map(|cmp| match cmp { + Protocol::Dns4(ref name) | Protocol::Dns6(ref name) => { + let name = name.to_string(); + let to_resolve = format!("{}:0", name); + let (tx, rx) = oneshot::channel(); + self.thread_pool.spawn_ok(async { + let to_resolve = to_resolve; + let _ = tx.send(match to_resolve[..].to_socket_addrs() { + Ok(list) => Ok(list.map(|s| s.ip()).collect::>()), + Err(e) => Err(e), + }); + }); + + let is_dns4 = if let Protocol::Dns4(_) = cmp { true } else { false }; + + async move { + let list = rx.await + .map_err(|_| { + error!("DNS resolver crashed"); + DnsErr::ResolveFail(name.clone()) + })? + .map_err(|err| DnsErr::ResolveError { + domain_name: name.clone(), + error: err, + })?; + + list.into_iter() + .filter_map(|addr| { + if (is_dns4 && addr.is_ipv4()) || (!is_dns4 && addr.is_ipv6()) { + Some(Protocol::from(addr)) + } else { + None + } + }) + .next() + .ok_or_else(|| DnsErr::ResolveFail(name)) + }.left_future() + }, + cmp => future::ready(Ok(cmp.acquire())).right_future() }) - .collect::>() - .into_iter(); + .collect::>(); + + let future = resolve_futs.collect::>() + .then(move |outcome| async move { + let outcome = outcome.into_iter().collect::, _>>()?; + let outcome = outcome.into_iter().collect::(); + debug!("DNS resolution outcome: {} => {}", addr, outcome); + + match self.inner.dial(outcome) { + Ok(d) => d.await.map_err(DnsErr::Underlying), + Err(TransportError::MultiaddrNotSupported(_addr)) => + Err(DnsErr::MultiaddrNotSupported), + Err(TransportError::Other(err)) => Err(DnsErr::Underlying(err)) + } + }); - let new_addr = JoinFuture { addr, future: future::join_all(resolve_iters) }; - Ok(Either::B(DialFuture { trans: Some(self.inner), future: Either::A(new_addr) })) + Ok(future.boxed().right_future()) } } @@ -205,116 +238,16 @@ where TErr: error::Error + 'static } } -// How to resolve; to an IPv4 address or an IPv6 address? -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -enum ResolveTy { - Dns4, - Dns6, -} - -/// Future, performing DNS resolution. -#[derive(Debug)] -pub struct ResolveFuture { - name: Option, - inner: T, - ty: ResolveTy, - error_ty: PhantomData, -} - -impl Future for ResolveFuture -where - T: Future, Error = io::Error> -{ - type Item = Protocol<'static>; - type Error = DnsErr; - - fn poll(&mut self) -> Poll { - let ty = self.ty; - let addrs = try_ready!(self.inner.poll().map_err(|error| { - let domain_name = self.name.take().unwrap_or_default(); - DnsErr::ResolveError { domain_name, error } - })); - - trace!("DNS component resolution: {:?} => {:?}", self.name, addrs); - let mut addrs = addrs - .into_iter() - .filter_map(move |addr| match (addr, ty) { - (IpAddr::V4(addr), ResolveTy::Dns4) => Some(Protocol::Ip4(addr)), - (IpAddr::V6(addr), ResolveTy::Dns6) => Some(Protocol::Ip6(addr)), - _ => None, - }); - match addrs.next() { - Some(a) => Ok(Async::Ready(a)), - None => Err(DnsErr::ResolveFail(self.name.take().unwrap_or_default())) - } - } -} - -/// Build final multi-address from resolving futures. -#[derive(Debug)] -pub struct JoinFuture { - addr: Multiaddr, - future: T -} - -impl Future for JoinFuture -where - T: Future>> -{ - type Item = Multiaddr; - type Error = T::Error; - - fn poll(&mut self) -> Poll { - let outcome = try_ready!(self.future.poll()); - let outcome: Multiaddr = outcome.into_iter().collect(); - debug!("DNS resolution outcome: {} => {}", self.addr, outcome); - Ok(Async::Ready(outcome)) - } -} - -/// Future, dialing the resolved multi-address. -#[derive(Debug)] -pub struct DialFuture { - trans: Option, - future: Either, -} - -impl Future for DialFuture -where - T: Transport, - F: Future>, - TErr: error::Error, -{ - type Item = T::Output; - type Error = DnsErr; - - fn poll(&mut self) -> Poll { - loop { - let next = match self.future { - Either::A(ref mut f) => { - let addr = try_ready!(f.poll()); - match self.trans.take().unwrap().dial(addr) { - Ok(dial) => Either::B(dial), - Err(_) => return Err(DnsErr::MultiaddrNotSupported) - } - } - Either::B(ref mut f) => return f.poll().map_err(DnsErr::Underlying) - }; - self.future = next - } - } -} - #[cfg(test)] mod tests { - use libp2p_tcp::TcpConfig; - use futures::future; + use super::DnsConfig; + use futures::{future::BoxFuture, prelude::*, stream::BoxStream}; use libp2p_core::{ Transport, multiaddr::{Protocol, Multiaddr}, - transport::TransportError + transport::ListenerEvent, + transport::TransportError, }; - use super::DnsConfig; #[test] fn basic_resolve() { @@ -322,11 +255,11 @@ mod tests { struct CustomTransport; impl Transport for CustomTransport { - type Output = ::Output; - type Error = ::Error; - type Listener = ::Listener; - type ListenerUpgrade = ::ListenerUpgrade; - type Dial = future::Empty; + type Output = (); + type Error = std::io::Error; + type Listener = BoxStream<'static, Result, Self::Error>>; + type ListenerUpgrade = BoxFuture<'static, Result>; + type Dial = BoxFuture<'static, Result>; fn listen_on(self, _: Multiaddr) -> Result> { unreachable!() @@ -340,22 +273,36 @@ mod tests { _ => panic!(), }; match addr[0] { - Protocol::Dns4(_) => (), - Protocol::Dns6(_) => (), + Protocol::Ip4(_) => (), + Protocol::Ip6(_) => (), _ => panic!(), }; - Ok(future::empty()) + Ok(Box::pin(future::ready(Ok(())))) } } - let transport = DnsConfig::new(CustomTransport); - - let _ = transport - .clone() - .dial("/dns4/example.com/tcp/20000".parse().unwrap()) - .unwrap(); - let _ = transport - .dial("/dns6/example.com/tcp/20000".parse().unwrap()) - .unwrap(); + futures::executor::block_on(async move { + let transport = DnsConfig::new(CustomTransport).unwrap(); + + let _ = transport + .clone() + .dial("/dns4/example.com/tcp/20000".parse().unwrap()) + .unwrap() + .await + .unwrap(); + + let _ = transport + .clone() + .dial("/dns6/example.com/tcp/20000".parse().unwrap()) + .unwrap() + .await + .unwrap(); + + let _ = transport + .dial("/ip4/1.2.3.4/tcp/20000".parse().unwrap()) + .unwrap() + .await + .unwrap(); + }); } } diff --git a/transports/tcp/Cargo.toml b/transports/tcp/Cargo.toml index 82d84c382ae..a9e2a2aabcc 100644 --- a/transports/tcp/Cargo.toml +++ b/transports/tcp/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-tcp" edition = "2018" description = "TCP/IP transport protocol for libp2p" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,15 +10,10 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4" +async-std = "1.0" +futures = "0.3.1" +futures-timer = "2.0" get_if_addrs = "0.5.3" ipnet = "2.0.0" -libp2p-core = { version = "0.13.0", path = "../../core" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } log = "0.4.1" -futures = "0.1" -tokio-io = "0.1" -tokio-timer = "0.2" -tokio-tcp = "0.1" - -[dev-dependencies] -tokio = "0.1" diff --git a/transports/tcp/src/lib.rs b/transports/tcp/src/lib.rs index d42b4f44da8..99ebad02f65 100644 --- a/transports/tcp/src/lib.rs +++ b/transports/tcp/src/lib.rs @@ -20,8 +20,6 @@ //! Implementation of the libp2p `Transport` trait for TCP/IP. //! -//! Uses [the *tokio* library](https://tokio.rs). -//! //! # Usage //! //! Example: @@ -38,11 +36,9 @@ //! The `TcpConfig` structs implements the `Transport` trait of the `swarm` library. See the //! documentation of `swarm` and of libp2p in general to learn how to use the `Transport` trait. -use futures::{ - future::{self, Either, FutureResult}, - prelude::*, - stream::{self, Chain, IterOk, Once} -}; +use async_std::net::TcpStream; +use futures::{future::{self, Ready}, prelude::*}; +use futures_timer::Delay; use get_if_addrs::{IfAddr, get_if_addrs}; use ipnet::{IpNet, Ipv4Net, Ipv6Net}; use libp2p_core::{ @@ -53,15 +49,13 @@ use libp2p_core::{ use log::{debug, trace}; use std::{ collections::VecDeque, - io::{self, Read, Write}, + io, iter::{self, FromIterator}, net::{IpAddr, SocketAddr}, - time::{Duration, Instant}, - vec::IntoIter + pin::Pin, + task::{Context, Poll}, + time::Duration }; -use tokio_io::{AsyncRead, AsyncWrite}; -use tokio_timer::Delay; -use tokio_tcp::{ConnectFuture, Incoming, TcpStream}; /// Represents the configuration for a TCP/IP transport capability for libp2p. /// @@ -71,14 +65,8 @@ use tokio_tcp::{ConnectFuture, Incoming, TcpStream}; pub struct TcpConfig { /// How long a listener should sleep after receiving an error, before trying again. sleep_on_error: Duration, - /// Size of the recv buffer size to set for opened sockets, or `None` to keep default. - recv_buffer_size: Option, - /// Size of the send buffer size to set for opened sockets, or `None` to keep default. - send_buffer_size: Option, /// TTL to set for opened sockets, or `None` to keep default. ttl: Option, - /// Keep alive duration to set for opened sockets, or `None` to keep default. - keepalive: Option>, /// `TCP_NODELAY` to set for opened sockets, or `None` to keep default. nodelay: Option, } @@ -88,38 +76,17 @@ impl TcpConfig { pub fn new() -> TcpConfig { TcpConfig { sleep_on_error: Duration::from_millis(100), - recv_buffer_size: None, - send_buffer_size: None, ttl: None, - keepalive: None, nodelay: None, } } - /// Sets the size of the recv buffer size to set for opened sockets. - pub fn recv_buffer_size(mut self, value: usize) -> Self { - self.recv_buffer_size = Some(value); - self - } - - /// Sets the size of the send buffer size to set for opened sockets. - pub fn send_buffer_size(mut self, value: usize) -> Self { - self.send_buffer_size = Some(value); - self - } - /// Sets the TTL to set for opened sockets. pub fn ttl(mut self, value: u32) -> Self { self.ttl = Some(value); self } - /// Sets the keep alive pinging duration to set for opened sockets. - pub fn keepalive(mut self, value: Option) -> Self { - self.keepalive = Some(value); - self - } - /// Sets the `TCP_NODELAY` to set for opened sockets. pub fn nodelay(mut self, value: bool) -> Self { self.nodelay = Some(value); @@ -130,9 +97,9 @@ impl TcpConfig { impl Transport for TcpConfig { type Output = TcpTransStream; type Error = io::Error; - type Listener = TcpListener; - type ListenerUpgrade = FutureResult; - type Dial = TcpDialFut; + type Listener = Pin, io::Error>> + Send>>; + type ListenerUpgrade = Ready>; + type Dial = Pin> + Send>>; fn listen_on(self, addr: Multiaddr) -> Result> { let socket_addr = @@ -142,54 +109,59 @@ impl Transport for TcpConfig { return Err(TransportError::MultiaddrNotSupported(addr)) }; - let listener = tokio_tcp::TcpListener::bind(&socket_addr).map_err(TransportError::Other)?; - let local_addr = listener.local_addr().map_err(TransportError::Other)?; - let port = local_addr.port(); - - // Determine all our listen addresses which is either a single local IP address - // or (if a wildcard IP address was used) the addresses of all our interfaces, - // as reported by `get_if_addrs`. - let addrs = - if socket_addr.ip().is_unspecified() { - let addrs = host_addresses(port).map_err(TransportError::Other)?; - debug!("Listening on {:?}", addrs.iter().map(|(_, _, ma)| ma).collect::>()); - Addresses::Many(addrs) - } else { - let ma = ip_to_multiaddr(local_addr.ip(), port); - debug!("Listening on {:?}", ma); - Addresses::One(ma) + async fn do_listen(cfg: TcpConfig, socket_addr: SocketAddr) + -> Result>>, io::Error>>, io::Error> + { + let listener = async_std::net::TcpListener::bind(&socket_addr).await?; + let local_addr = listener.local_addr()?; + let port = local_addr.port(); + + // Determine all our listen addresses which is either a single local IP address + // or (if a wildcard IP address was used) the addresses of all our interfaces, + // as reported by `get_if_addrs`. + let addrs = + if socket_addr.ip().is_unspecified() { + let addrs = host_addresses(port)?; + debug!("Listening on {:?}", addrs.iter().map(|(_, _, ma)| ma).collect::>()); + Addresses::Many(addrs) + } else { + let ma = ip_to_multiaddr(local_addr.ip(), port); + debug!("Listening on {:?}", ma); + Addresses::One(ma) + }; + + // Generate `NewAddress` events for each new `Multiaddr`. + let pending = match addrs { + Addresses::One(ref ma) => { + let event = ListenerEvent::NewAddress(ma.clone()); + let mut list = VecDeque::new(); + list.push_back(Ok(event)); + list + } + Addresses::Many(ref aa) => { + aa.iter() + .map(|(_, _, ma)| ma) + .cloned() + .map(ListenerEvent::NewAddress) + .map(Result::Ok) + .collect::>() + } }; - // Generate `NewAddress` events for each new `Multiaddr`. - let events = match addrs { - Addresses::One(ref ma) => { - let event = ListenerEvent::NewAddress(ma.clone()); - Either::A(stream::once(Ok(event))) - } - Addresses::Many(ref aa) => { - let events = aa.iter() - .map(|(_, _, ma)| ma) - .cloned() - .map(ListenerEvent::NewAddress) - .collect::>(); - Either::B(stream::iter_ok(events)) - } - }; + let listen_stream = TcpListenStream { + stream: listener, + pause: None, + pause_duration: cfg.sleep_on_error, + port, + addrs, + pending, + config: cfg + }; - let stream = TcpListenStream { - inner: Listener::new(listener.incoming(), self.sleep_on_error), - port, - addrs, - pending: VecDeque::new(), - config: self - }; + Ok(stream::unfold(listen_stream, |s| s.next().map(Some))) + } - Ok(TcpListener { - inner: match events { - Either::A(e) => Either::A(e.chain(stream)), - Either::B(e) => Either::B(e.chain(stream)) - } - }) + Ok(Box::pin(do_listen(self, socket_addr).try_flatten_stream())) } fn dial(self, addr: Multiaddr) -> Result> { @@ -206,12 +178,13 @@ impl Transport for TcpConfig { debug!("Dialing {}", addr); - let future = TcpDialFut { - inner: TcpStream::connect(&socket_addr), - config: self - }; + async fn do_dial(cfg: TcpConfig, socket_addr: SocketAddr) -> Result { + let stream = TcpStream::connect(&socket_addr).await?; + apply_config(&cfg, &stream)?; + Ok(TcpTransStream { inner: stream }) + } - Ok(future) + Ok(Box::pin(do_dial(self, socket_addr))) } } @@ -269,22 +242,10 @@ fn host_addresses(port: u16) -> io::Result> { /// Applies the socket configuration parameters to a socket. fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), io::Error> { - if let Some(recv_buffer_size) = config.recv_buffer_size { - socket.set_recv_buffer_size(recv_buffer_size)?; - } - - if let Some(send_buffer_size) = config.send_buffer_size { - socket.set_send_buffer_size(send_buffer_size)?; - } - if let Some(ttl) = config.ttl { socket.set_ttl(ttl)?; } - if let Some(keepalive) = config.keepalive { - socket.set_keepalive(keepalive)?; - } - if let Some(nodelay) = config.nodelay { socket.set_nodelay(nodelay)?; } @@ -292,55 +253,6 @@ fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), io::Error> Ok(()) } -/// Future that dials a TCP/IP address. -#[derive(Debug)] -#[must_use = "futures do nothing unless polled"] -pub struct TcpDialFut { - inner: ConnectFuture, - /// Original configuration. - config: TcpConfig, -} - -impl Future for TcpDialFut { - type Item = TcpTransStream; - type Error = io::Error; - - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(stream)) => { - apply_config(&self.config, &stream)?; - Ok(Async::Ready(TcpTransStream { inner: stream })) - } - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => { - debug!("Error while dialing => {:?}", err); - Err(err) - } - } - } -} - -/// Stream of `ListenerEvent`s. -#[derive(Debug)] -pub struct TcpListener { - inner: Either< - Chain>, io::Error>, TcpListenStream>, - Chain>>, io::Error>, TcpListenStream> - > -} - -impl Stream for TcpListener { - type Item = ListenerEvent>; - type Error = io::Error; - - fn poll(&mut self) -> Poll, Self::Error> { - match self.inner { - Either::A(ref mut it) => it.poll(), - Either::B(ref mut it) => it.poll() - } - } -} - /// Listen address information. #[derive(Debug)] enum Addresses { @@ -350,61 +262,16 @@ enum Addresses { Many(Vec<(IpAddr, IpNet, Multiaddr)>) } -type Buffer = VecDeque>>; +type Buffer = VecDeque>>, io::Error>>; -/// Incoming connection stream which pauses after errors. -#[derive(Debug)] -struct Listener { +/// Stream that listens on an TCP/IP address. +pub struct TcpListenStream { /// The incoming connections. - stream: S, + stream: async_std::net::TcpListener, /// The current pause if any. pause: Option, /// How long to pause after an error. - pause_duration: Duration -} - -impl Listener -where - S: Stream, - S::Error: std::fmt::Display -{ - fn new(stream: S, duration: Duration) -> Self { - Listener { stream, pause: None, pause_duration: duration } - } -} - -impl Stream for Listener -where - S: Stream, - S::Error: std::fmt::Display -{ - type Item = S::Item; - type Error = S::Error; - - /// Polls for incoming connections, pausing if an error is encountered. - fn poll(&mut self) -> Poll, S::Error> { - match self.pause.as_mut().map(|p| p.poll()) { - Some(Ok(Async::NotReady)) => return Ok(Async::NotReady), - Some(Ok(Async::Ready(()))) | Some(Err(_)) => { self.pause.take(); } - None => () - } - - match self.stream.poll() { - Ok(x) => Ok(x), - Err(e) => { - debug!("error accepting incoming connection: {}", e); - self.pause = Some(Delay::new(Instant::now() + self.pause_duration)); - Err(e) - } - } - } -} - -/// Stream that listens on an TCP/IP address. -#[derive(Debug)] -pub struct TcpListenStream { - /// Stream of incoming sockets. - inner: Listener, + pause_duration: Duration, /// The port which we use as our listen port in listener event addresses. port: u16, /// The set of known addresses. @@ -445,7 +312,7 @@ fn check_for_interface_changes( for (ip, _, ma) in old_listen_addrs.iter() { if listen_addrs.iter().find(|(i, ..)| i == ip).is_none() { debug!("Expired listen address: {}", ma); - pending.push_back(ListenerEvent::AddressExpired(ma.clone())); + pending.push_back(Ok(ListenerEvent::AddressExpired(ma.clone()))); } } @@ -453,7 +320,7 @@ fn check_for_interface_changes( for (ip, _, ma) in listen_addrs.iter() { if old_listen_addrs.iter().find(|(i, ..)| i == ip).is_none() { debug!("New listen address: {}", ma); - pending.push_back(ListenerEvent::NewAddress(ma.clone())); + pending.push_back(Ok(ListenerEvent::NewAddress(ma.clone()))); } } @@ -470,21 +337,26 @@ fn check_for_interface_changes( Ok(()) } -impl Stream for TcpListenStream { - type Item = ListenerEvent>; - type Error = io::Error; - - fn poll(&mut self) -> Poll, io::Error> { +impl TcpListenStream { + /// Takes ownership of the listener, and returns the next incoming event and the listener. + async fn next(mut self) -> (Result>>, io::Error>, Self) { loop { if let Some(event) = self.pending.pop_front() { - return Ok(Async::Ready(Some(event))) + return (event, self); } - let sock = match self.inner.poll() { - Ok(Async::Ready(Some(sock))) => sock, - Ok(Async::Ready(None)) => return Ok(Async::Ready(None)), - Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(e) => return Err(e) + if let Some(pause) = self.pause.take() { + let _ = pause.await; + } + + // TODO: do we get the peer_addr at the same time? + let (sock, _) = match self.stream.accept().await { + Ok(s) => s, + Err(e) => { + debug!("error accepting incoming connection: {}", e); + self.pause = Some(Delay::new(self.pause_duration)); + return (Err(e), self); + } }; let sock_addr = match sock.peer_addr() { @@ -498,7 +370,9 @@ impl Stream for TcpListenStream { let local_addr = match sock.local_addr() { Ok(sock_addr) => { if let Addresses::Many(ref mut addrs) = self.addrs { - check_for_interface_changes(&sock_addr, self.port, addrs, &mut self.pending)? + if let Err(err) = check_for_interface_changes(&sock_addr, self.port, addrs, &mut self.pending) { + return (Err(err), self); + } } ip_to_multiaddr(sock_addr.ip(), sock_addr.port()) } @@ -513,19 +387,19 @@ impl Stream for TcpListenStream { match apply_config(&self.config, &sock) { Ok(()) => { trace!("Incoming connection from {} at {}", remote_addr, local_addr); - self.pending.push_back(ListenerEvent::Upgrade { + self.pending.push_back(Ok(ListenerEvent::Upgrade { upgrade: future::ok(TcpTransStream { inner: sock }), local_addr, remote_addr - }) + })) } Err(err) => { debug!("Error upgrading incoming connection from {}: {:?}", remote_addr, err); - self.pending.push_back(ListenerEvent::Upgrade { + self.pending.push_back(Ok(ListenerEvent::Upgrade { upgrade: future::err(err), local_addr, remote_addr - }) + })) } } } @@ -538,35 +412,23 @@ pub struct TcpTransStream { inner: TcpStream, } -impl Read for TcpTransStream { - fn read(&mut self, buf: &mut [u8]) -> Result { - self.inner.read(buf) - } -} - impl AsyncRead for TcpTransStream { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.inner.prepare_uninitialized_buffer(buf) - } - - fn read_buf(&mut self, buf: &mut B) -> Poll { - self.inner.read_buf(buf) + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + AsyncRead::poll_read(Pin::new(&mut self.inner), cx, buf) } } -impl Write for TcpTransStream { - fn write(&mut self, buf: &[u8]) -> Result { - self.inner.write(buf) +impl AsyncWrite for TcpTransStream { + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll> { + AsyncWrite::poll_write(Pin::new(&mut self.inner), cx, buf) } - fn flush(&mut self) -> Result<(), io::Error> { - self.inner.flush() + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + AsyncWrite::poll_flush(Pin::new(&mut self.inner), cx) } -} -impl AsyncWrite for TcpTransStream { - fn shutdown(&mut self) -> Poll<(), io::Error> { - AsyncWrite::shutdown(&mut self.inner) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + AsyncWrite::poll_close(Pin::new(&mut self.inner), cx) } } @@ -582,31 +444,10 @@ impl Drop for TcpTransStream { #[cfg(test)] mod tests { - use futures::{prelude::*, future::{self, Loop}, stream}; + use futures::prelude::*; use libp2p_core::{Transport, multiaddr::{Multiaddr, Protocol}, transport::ListenerEvent}; - use std::{net::{IpAddr, Ipv4Addr, SocketAddr}, time::Duration}; - use super::{multiaddr_to_socketaddr, TcpConfig, Listener}; - use tokio::runtime::current_thread::{self, Runtime}; - use tokio_io; - - #[test] - fn pause_on_error() { - // We create a stream of values and errors and continue polling even after errors - // have been encountered. We count the number of items (including errors) and assert - // that no item has been missed. - let rs = stream::iter_result(vec![Ok(1), Err(1), Ok(1), Err(1)]); - let ls = Listener::new(rs, Duration::from_secs(1)); - let sum = future::loop_fn((0, ls), |(acc, ls)| { - ls.into_future().then(move |item| { - match item { - Ok((None, _)) => Ok::<_, std::convert::Infallible>(Loop::Break(acc)), - Ok((Some(n), rest)) => Ok(Loop::Continue((acc + n, rest))), - Err((n, rest)) => Ok(Loop::Continue((acc + n, rest))) - } - }) - }); - assert_eq!(4, current_thread::block_on_all(sum).unwrap()) - } + use std::net::{IpAddr, Ipv4Addr, SocketAddr}; + use super::{multiaddr_to_socketaddr, TcpConfig}; #[test] fn wildcard_expansion() { @@ -615,8 +456,7 @@ mod tests { .expect("listener"); // Get the first address. - let addr = listener.by_ref() - .wait() + let addr = futures::executor::block_on_stream(listener.by_ref()) .next() .expect("some event") .expect("no error") @@ -626,7 +466,7 @@ mod tests { // Process all initial `NewAddress` events and make sure they // do not contain wildcard address or port. let server = listener - .take_while(|event| match event { + .take_while(|event| match event.as_ref().unwrap() { ListenerEvent::NewAddress(a) => { let mut iter = a.iter(); match iter.next().expect("ip address") { @@ -639,14 +479,14 @@ mod tests { } else { panic!("No TCP port in address: {}", a) } - Ok(true) + futures::future::ready(true) } - _ => Ok(false) + _ => futures::future::ready(false) }) - .for_each(|_| Ok(())); + .for_each(|_| futures::future::ready(())); let client = TcpConfig::new().dial(addr).expect("dialer"); - tokio::run(server.join(client).map(|_| ()).map_err(|e| panic!("error: {}", e))) + async_std::task::block_on(futures::future::join(server, client)).1.unwrap(); } #[test] @@ -700,46 +540,43 @@ mod tests { #[test] fn communicating_between_dialer_and_listener() { - use std::io::Write; + let (ready_tx, ready_rx) = futures::channel::oneshot::channel(); + let mut ready_tx = Some(ready_tx); - std::thread::spawn(move || { - let addr = "/ip4/127.0.0.1/tcp/12345".parse::().unwrap(); + async_std::task::spawn(async move { + let addr = "/ip4/127.0.0.1/tcp/0".parse::().unwrap(); let tcp = TcpConfig::new(); - let mut rt = Runtime::new().unwrap(); - let handle = rt.handle(); - let listener = tcp.listen_on(addr).unwrap() - .filter_map(ListenerEvent::into_upgrade) - .for_each(|(sock, _)| { - sock.and_then(|sock| { - // Define what to do with the socket that just connected to us - // Which in this case is read 3 bytes - let handle_conn = tokio_io::io::read_exact(sock, [0; 3]) - .map(|(_, buf)| assert_eq!(buf, [1, 2, 3])) - .map_err(|err| panic!("IO error {:?}", err)); - - // Spawn the future as a concurrent task - handle.spawn(handle_conn).unwrap(); - - Ok(()) - }) - }); - - rt.block_on(listener).unwrap(); - rt.run().unwrap(); + let mut listener = tcp.listen_on(addr).unwrap(); + + loop { + match listener.next().await.unwrap().unwrap() { + ListenerEvent::NewAddress(listen_addr) => { + ready_tx.take().unwrap().send(listen_addr).unwrap(); + }, + ListenerEvent::Upgrade { upgrade, .. } => { + let mut upgrade = upgrade.await.unwrap(); + let mut buf = [0u8; 3]; + upgrade.read_exact(&mut buf).await.unwrap(); + assert_eq!(buf, [1, 2, 3]); + upgrade.write_all(&[4, 5, 6]).await.unwrap(); + }, + _ => unreachable!() + } + } }); - std::thread::sleep(std::time::Duration::from_millis(100)); - let addr = "/ip4/127.0.0.1/tcp/12345".parse::().unwrap(); - let tcp = TcpConfig::new(); - // Obtain a future socket through dialing - let socket = tcp.dial(addr.clone()).unwrap(); - // Define what to do with the socket once it's obtained - let action = socket.then(|sock| -> Result<(), ()> { - sock.unwrap().write(&[0x1, 0x2, 0x3]).unwrap(); - Ok(()) + + async_std::task::block_on(async move { + let addr = ready_rx.await.unwrap(); + let tcp = TcpConfig::new(); + + // Obtain a future socket through dialing + let mut socket = tcp.dial(addr.clone()).unwrap().await.unwrap(); + socket.write_all(&[0x1, 0x2, 0x3]).await.unwrap(); + + let mut buf = [0u8; 3]; + socket.read_exact(&mut buf).await.unwrap(); + assert_eq!(buf, [4, 5, 6]); }); - // Execute the future in our event loop - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(action).unwrap(); } #[test] @@ -749,7 +586,7 @@ mod tests { let addr = "/ip4/127.0.0.1/tcp/0".parse::().unwrap(); assert!(addr.to_string().contains("tcp/0")); - let new_addr = tcp.listen_on(addr).unwrap().wait() + let new_addr = futures::executor::block_on_stream(tcp.listen_on(addr).unwrap()) .next() .expect("some event") .expect("no error") @@ -766,7 +603,7 @@ mod tests { let addr: Multiaddr = "/ip6/::1/tcp/0".parse().unwrap(); assert!(addr.to_string().contains("tcp/0")); - let new_addr = tcp.listen_on(addr).unwrap().wait() + let new_addr = futures::executor::block_on_stream(tcp.listen_on(addr).unwrap()) .next() .expect("some event") .expect("no error") diff --git a/transports/uds/Cargo.toml b/transports/uds/Cargo.toml index 508df75d634..ad4f8ff7e8a 100644 --- a/transports/uds/Cargo.toml +++ b/transports/uds/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-uds" edition = "2018" description = "Unix domain sockets transport for libp2p" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,12 +10,10 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [target.'cfg(all(unix, not(any(target_os = "emscripten", target_os = "unknown"))))'.dependencies] -libp2p-core = { version = "0.13.0", path = "../../core" } +async-std = "1.0" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } log = "0.4.1" -futures = "0.1" -tokio-uds = "0.2" +futures = "0.3.1" [target.'cfg(all(unix, not(any(target_os = "emscripten", target_os = "unknown"))))'.dev-dependencies] tempfile = "3.0" -tokio = "0.1" -tokio-io = "0.1" diff --git a/transports/uds/src/lib.rs b/transports/uds/src/lib.rs index 4be826cae1f..dccee6229c2 100644 --- a/transports/uds/src/lib.rs +++ b/transports/uds/src/lib.rs @@ -20,8 +20,6 @@ //! Implementation of the libp2p `Transport` trait for Unix domain sockets. //! -//! Uses [the *tokio* library](https://tokio.rs). -//! //! # Platform support //! //! This transport only works on Unix platforms. @@ -46,28 +44,24 @@ #![cfg(all(unix, not(any(target_os = "emscripten", target_os = "unknown"))))] -use futures::{future::{self, FutureResult}, prelude::*, try_ready}; -use futures::stream::Stream; -use log::debug; -use std::{io, path::PathBuf}; +use async_std::os::unix::net::{UnixListener, UnixStream}; +use futures::{prelude::*, future::{BoxFuture, Ready}}; +use futures::stream::BoxStream; use libp2p_core::{ Transport, multiaddr::{Protocol, Multiaddr}, transport::{ListenerEvent, TransportError} }; -use tokio_uds::{UnixListener, UnixStream}; +use log::debug; +use std::{io, path::PathBuf}; /// Represents the configuration for a Unix domain sockets transport capability for libp2p. -/// -/// The Unixs sockets created by libp2p will need to be progressed by running the futures and -/// streams obtained by libp2p through the tokio reactor. #[derive(Debug, Clone)] pub struct UdsConfig { } impl UdsConfig { - /// Creates a new configuration object for TCP/IP. - #[inline] + /// Creates a new configuration object for Unix domain sockets. pub fn new() -> UdsConfig { UdsConfig {} } @@ -76,27 +70,39 @@ impl UdsConfig { impl Transport for UdsConfig { type Output = UnixStream; type Error = io::Error; - type Listener = ListenerStream; - type ListenerUpgrade = FutureResult; - type Dial = tokio_uds::ConnectFuture; + type Listener = BoxStream<'static, Result, Self::Error>>; + type ListenerUpgrade = Ready>; + type Dial = BoxFuture<'static, Result>; fn listen_on(self, addr: Multiaddr) -> Result> { if let Ok(path) = multiaddr_to_path(&addr) { - let listener = UnixListener::bind(&path); - // We need to build the `Multiaddr` to return from this function. If an error happened, - // just return the original multiaddr. - match listener { - Ok(listener) => { - debug!("Now listening on {}", addr); - let future = ListenerStream { - stream: listener.incoming(), - addr: addr.clone(), - tell_new_addr: true - }; - Ok(future) - } - Err(_) => return Err(TransportError::MultiaddrNotSupported(addr)), - } + Ok(async move { UnixListener::bind(&path).await } + .map_ok(move |listener| { + stream::once({ + let addr = addr.clone(); + async move { + debug!("Now listening on {}", addr); + Ok(ListenerEvent::NewAddress(addr)) + } + }).chain(stream::unfold(listener, move |listener| { + let addr = addr.clone(); + async move { + let (stream, _) = match listener.accept().await { + Ok(v) => v, + Err(err) => return Some((Err(err), listener)) + }; + debug!("incoming connection on {}", addr); + let event = ListenerEvent::Upgrade { + upgrade: future::ok(stream), + local_addr: addr.clone(), + remote_addr: addr.clone() + }; + Some((Ok(event), listener)) + } + })) + }) + .try_flatten_stream() + .boxed()) } else { Err(TransportError::MultiaddrNotSupported(addr)) } @@ -105,7 +111,7 @@ impl Transport for UdsConfig { fn dial(self, addr: Multiaddr) -> Result> { if let Ok(path) = multiaddr_to_path(&addr) { debug!("Dialing {}", addr); - Ok(UnixStream::connect(&path)) + Ok(async move { UnixStream::connect(&path).await }.boxed()) } else { Err(TransportError::MultiaddrNotSupported(addr)) } @@ -137,51 +143,13 @@ fn multiaddr_to_path(addr: &Multiaddr) -> Result { Ok(out) } -pub struct ListenerStream { - stream: T, - addr: Multiaddr, - tell_new_addr: bool -} - -impl Stream for ListenerStream -where - T: Stream -{ - type Item = ListenerEvent>; - type Error = T::Error; - - fn poll(&mut self) -> Poll, Self::Error> { - if self.tell_new_addr { - self.tell_new_addr = false; - return Ok(Async::Ready(Some(ListenerEvent::NewAddress(self.addr.clone())))) - } - match try_ready!(self.stream.poll()) { - Some(item) => { - debug!("incoming connection on {}", self.addr); - Ok(Async::Ready(Some(ListenerEvent::Upgrade { - upgrade: future::ok(item), - local_addr: self.addr.clone(), - remote_addr: self.addr.clone() - }))) - } - None => Ok(Async::Ready(None)) - } - } -} - #[cfg(test)] mod tests { - use tokio::runtime::current_thread::Runtime; use super::{multiaddr_to_path, UdsConfig}; - use futures::prelude::*; + use futures::{channel::oneshot, prelude::*}; use std::{self, borrow::Cow, path::Path}; - use libp2p_core::{ - Transport, - multiaddr::{Protocol, Multiaddr}, - transport::ListenerEvent - }; + use libp2p_core::{Transport, multiaddr::{Protocol, Multiaddr}}; use tempfile; - use tokio_io; #[test] fn multiaddr_to_path_conversion() { @@ -202,64 +170,56 @@ mod tests { #[test] fn communicating_between_dialer_and_listener() { - use std::io::Write; let temp_dir = tempfile::tempdir().unwrap(); let socket = temp_dir.path().join("socket"); let addr = Multiaddr::from(Protocol::Unix(Cow::Owned(socket.to_string_lossy().into_owned()))); - let addr2 = addr.clone(); - std::thread::spawn(move || { - let tcp = UdsConfig::new(); + let (tx, rx) = oneshot::channel(); + + async_std::task::spawn(async move { + let mut listener = UdsConfig::new().listen_on(addr).unwrap(); - let mut rt = Runtime::new().unwrap(); - let handle = rt.handle(); - let listener = tcp.listen_on(addr2).unwrap() - .filter_map(ListenerEvent::into_upgrade) - .for_each(|(sock, _)| { - sock.and_then(|sock| { - // Define what to do with the socket that just connected to us - // Which in this case is read 3 bytes - let handle_conn = tokio_io::io::read_exact(sock, [0; 3]) - .map(|(_, buf)| assert_eq!(buf, [1, 2, 3])) - .map_err(|err| panic!("IO error {:?}", err)); + let listen_addr = listener.try_next().await.unwrap() + .expect("some event") + .into_new_address() + .expect("listen address"); - // Spawn the future as a concurrent task - handle.spawn(handle_conn).unwrap(); - Ok(()) - }) - }); + tx.send(listen_addr).unwrap(); - rt.block_on(listener).unwrap(); - rt.run().unwrap(); + let (sock, _addr) = listener.try_filter_map(|e| future::ok(e.into_upgrade())) + .try_next() + .await + .unwrap() + .expect("some event"); + + let mut sock = sock.await.unwrap(); + let mut buf = [0u8; 3]; + sock.read_exact(&mut buf).await.unwrap(); + assert_eq!(buf, [1, 2, 3]); }); - std::thread::sleep(std::time::Duration::from_millis(100)); - let tcp = UdsConfig::new(); - // Obtain a future socket through dialing - let socket = tcp.dial(addr.clone()).unwrap(); - // Define what to do with the socket once it's obtained - let action = socket.then(|sock| -> Result<(), ()> { - sock.unwrap().write(&[0x1, 0x2, 0x3]).unwrap(); - Ok(()) + + async_std::task::block_on(async move { + let uds = UdsConfig::new(); + let addr = rx.await.unwrap(); + let mut socket = uds.dial(addr).unwrap().await.unwrap(); + socket.write(&[1, 2, 3]).await.unwrap(); }); - // Execute the future in our event loop - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(action).unwrap(); } #[test] #[ignore] // TODO: for the moment unix addresses fail to parse fn larger_addr_denied() { - let tcp = UdsConfig::new(); + let uds = UdsConfig::new(); - let addr = "/ip4/127.0.0.1/tcp/12345/unix//foo/bar" + let addr = "/unix//foo/bar" .parse::() .unwrap(); - assert!(tcp.listen_on(addr).is_err()); + assert!(uds.listen_on(addr).is_err()); } #[test] #[ignore] // TODO: for the moment unix addresses fail to parse fn relative_addr_denied() { - assert!("/ip4/127.0.0.1/tcp/12345/unix/./foo/bar".parse::().is_err()); + assert!("/unix/./foo/bar".parse::().is_err()); } } diff --git a/transports/wasm-ext/Cargo.toml b/transports/wasm-ext/Cargo.toml index 1bc934fff8d..e47daa0b087 100644 --- a/transports/wasm-ext/Cargo.toml +++ b/transports/wasm-ext/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "libp2p-wasm-ext" -version = "0.6.0" +version = "0.7.0-alpha.1" authors = ["Pierre Krieger "] edition = "2018" description = "Allows passing in an external transport in a WASM environment" @@ -10,10 +10,9 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -futures = "0.1" +futures = "0.3.1" js-sys = "0.3.19" -libp2p-core = { version = "0.13.0", path = "../../core" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } parity-send-wrapper = "0.1.0" -tokio-io = "0.1" wasm-bindgen = "0.2.42" -wasm-bindgen-futures = "0.3.19" +wasm-bindgen-futures = "0.4.4" diff --git a/transports/wasm-ext/src/lib.rs b/transports/wasm-ext/src/lib.rs index a577294b661..9b788a8db2a 100644 --- a/transports/wasm-ext/src/lib.rs +++ b/transports/wasm-ext/src/lib.rs @@ -32,11 +32,12 @@ //! module. //! -use futures::{future::FutureResult, prelude::*, stream::Stream, try_ready}; +use futures::{prelude::*, future::Ready}; use libp2p_core::{transport::ListenerEvent, transport::TransportError, Multiaddr, Transport}; use parity_send_wrapper::SendWrapper; -use std::{collections::VecDeque, error, fmt, io, mem}; +use std::{collections::VecDeque, error, fmt, io, mem, pin::Pin, task::Context, task::Poll}; use wasm_bindgen::{JsCast, prelude::*}; +use wasm_bindgen_futures::JsFuture; /// Contains the definition that one must match on the JavaScript side. pub mod ffi { @@ -156,7 +157,7 @@ impl Transport for ExtTransport { type Output = Connection; type Error = JsErr; type Listener = Listen; - type ListenerUpgrade = FutureResult; + type ListenerUpgrade = Ready>; type Dial = Dial; fn listen_on(self, addr: Multiaddr) -> Result> { @@ -200,7 +201,7 @@ impl Transport for ExtTransport { #[must_use = "futures do nothing unless polled"] pub struct Dial { /// A promise that will resolve to a `ffi::Connection` on success. - inner: SendWrapper, + inner: SendWrapper, } impl fmt::Debug for Dial { @@ -210,14 +211,13 @@ impl fmt::Debug for Dial { } impl Future for Dial { - type Item = Connection; - type Error = JsErr; + type Output = Result; - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(connec)) => Ok(Async::Ready(Connection::new(connec.into()))), - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => Err(JsErr::from(err)), + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + match Future::poll(Pin::new(&mut *self.inner), cx) { + Poll::Ready(Ok(connec)) => Poll::Ready(Ok(Connection::new(connec.into()))), + Poll::Pending => Poll::Pending, + Poll::Ready(Err(err)) => Poll::Ready(Err(JsErr::from(err))), } } } @@ -228,9 +228,9 @@ pub struct Listen { /// Iterator of `ListenEvent`s. iterator: SendWrapper, /// Promise that will yield the next `ListenEvent`. - next_event: Option>, + next_event: Option>, /// List of events that we are waiting to propagate. - pending_events: VecDeque>>, + pending_events: VecDeque>>>, } impl fmt::Debug for Listen { @@ -240,13 +240,12 @@ impl fmt::Debug for Listen { } impl Stream for Listen { - type Item = ListenerEvent>; - type Error = JsErr; + type Item = Result>>, JsErr>; - fn poll(&mut self) -> Poll, Self::Error> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { loop { if let Some(ev) = self.pending_events.pop_front() { - return Ok(Async::Ready(Some(ev))); + return Poll::Ready(Some(Ok(ev))); } if self.next_event.is_none() { @@ -258,11 +257,15 @@ impl Stream for Listen { } let event = if let Some(next_event) = self.next_event.as_mut() { - let e = ffi::ListenEvent::from(try_ready!(next_event.poll())); + let e = match Future::poll(Pin::new(&mut **next_event), cx) { + Poll::Ready(Ok(ev)) => ffi::ListenEvent::from(ev), + Poll::Pending => return Poll::Pending, + Poll::Ready(Err(err)) => return Poll::Ready(Some(Err(err.into()))), + }; self.next_event = None; e } else { - return Ok(Async::Ready(None)); + return Poll::Ready(None); }; for addr in event @@ -319,7 +322,7 @@ pub struct Connection { /// When we write data using the FFI, a promise is returned containing the moment when the /// underlying transport is ready to accept data again. This promise is stored here. /// If this is `Some`, we must wait until the contained promise is resolved to write again. - previous_write_promise: Option>, + previous_write_promise: Option>, } impl Connection { @@ -341,7 +344,7 @@ enum ConnectionReadState { /// Some data have been read and are waiting to be transferred. Can be empty. PendingData(Vec), /// Waiting for a `Promise` containing the next data. - Waiting(SendWrapper), + Waiting(SendWrapper), /// An error occurred or an earlier read yielded EOF. Finished, } @@ -352,11 +355,11 @@ impl fmt::Debug for Connection { } } -impl io::Read for Connection { - fn read(&mut self, buf: &mut [u8]) -> Result { +impl AsyncRead for Connection { + fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { loop { match mem::replace(&mut self.read_state, ConnectionReadState::Finished) { - ConnectionReadState::Finished => break Err(io::ErrorKind::BrokenPipe.into()), + ConnectionReadState::Finished => break Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())), ConnectionReadState::PendingData(ref data) if data.is_empty() => { let iter_next = self.read_iterator.next().map_err(JsErr::from)?; @@ -376,22 +379,23 @@ impl io::Read for Connection { buf.copy_from_slice(&data[..buf.len()]); self.read_state = ConnectionReadState::PendingData(data.split_off(buf.len())); - break Ok(buf.len()); + break Poll::Ready(Ok(buf.len())); } else { let len = data.len(); buf[..len].copy_from_slice(&data); self.read_state = ConnectionReadState::PendingData(Vec::new()); - break Ok(len); + break Poll::Ready(Ok(len)); } } ConnectionReadState::Waiting(mut promise) => { - let data = match promise.poll().map_err(JsErr::from)? { - Async::Ready(ref data) if data.is_null() => break Ok(0), - Async::Ready(data) => data, - Async::NotReady => { + let data = match Future::poll(Pin::new(&mut *promise), cx) { + Poll::Ready(Ok(ref data)) if data.is_null() => break Poll::Ready(Ok(0)), + Poll::Ready(Ok(data)) => data, + Poll::Ready(Err(err)) => break Poll::Ready(Err(io::Error::from(JsErr::from(err)))), + Poll::Pending => { self.read_state = ConnectionReadState::Waiting(promise); - break Err(io::ErrorKind::WouldBlock.into()); + break Poll::Ready(Err(io::ErrorKind::WouldBlock.into())); } }; @@ -402,7 +406,7 @@ impl io::Read for Connection { if data_len <= buf.len() { data.copy_to(&mut buf[..data_len]); self.read_state = ConnectionReadState::PendingData(Vec::new()); - break Ok(data_len); + break Poll::Ready(Ok(data_len)); } else { let mut tmp_buf = vec![0; data_len]; data.copy_to(&mut tmp_buf[..]); @@ -415,23 +419,18 @@ impl io::Read for Connection { } } -impl tokio_io::AsyncRead for Connection { - unsafe fn prepare_uninitialized_buffer(&self, _: &mut [u8]) -> bool { - false - } -} - -impl io::Write for Connection { - fn write(&mut self, buf: &[u8]) -> Result { +impl AsyncWrite for Connection { + fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll> { // Note: as explained in the doc-comments of `Connection`, each call to this function must // map to exactly one call to `self.inner.write()`. if let Some(mut promise) = self.previous_write_promise.take() { - match promise.poll().map_err(JsErr::from)? { - Async::Ready(_) => (), - Async::NotReady => { + match Future::poll(Pin::new(&mut *promise), cx) { + Poll::Ready(Ok(_)) => (), + Poll::Ready(Err(err)) => return Poll::Ready(Err(io::Error::from(JsErr::from(err)))), + Poll::Pending => { self.previous_write_promise = Some(promise); - return Err(io::ErrorKind::WouldBlock.into()); + return Poll::Pending; } } } @@ -440,20 +439,20 @@ impl io::Write for Connection { self.previous_write_promise = Some(SendWrapper::new( self.inner.write(buf).map_err(JsErr::from)?.into(), )); - Ok(buf.len()) + Poll::Ready(Ok(buf.len())) } - fn flush(&mut self) -> Result<(), io::Error> { + fn poll_flush(self: Pin<&mut Self>, _: &mut Context) -> Poll> { // There's no flushing mechanism. In the FFI we consider that writing implicitly flushes. - Ok(()) + Poll::Ready(Ok(())) } -} -impl tokio_io::AsyncWrite for Connection { - fn shutdown(&mut self) -> Poll<(), io::Error> { + fn poll_close(self: Pin<&mut Self>, _: &mut Context) -> Poll> { // Shutting down is considered instantaneous. - self.inner.shutdown().map_err(JsErr::from)?; - Ok(Async::Ready(())) + match self.inner.shutdown() { + Ok(()) => Poll::Ready(Ok(())), + Err(err) => Poll::Ready(Err(io::Error::from(JsErr::from(err)))), + } } } diff --git a/transports/websocket/Cargo.toml b/transports/websocket/Cargo.toml index 74313b9f76c..d3d7ed9958a 100644 --- a/transports/websocket/Cargo.toml +++ b/transports/websocket/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-websocket" edition = "2018" description = "WebSocket transport for libp2p" -version = "0.13.0" +version = "0.14.0-alpha.1" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -10,18 +10,19 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -bytes = "0.4.6" -futures = "0.1" -libp2p-core = { version = "0.13.0", path = "../../core" } -log = "0.4.1" -rw-stream-sink = { version = "0.1.1", path = "../../misc/rw-stream-sink" } -tokio-codec = "0.1.1" -tokio-io = "0.1.12" -tokio-rustls = "0.10.1" -soketto = { version = "0.2.3", features = ["deflate"] } -url = "2.1.0" -webpki-roots = "0.18.0" +async-tls = "0.6" +bytes = "0.5" +either = "1.5.3" +futures = "0.3.1" +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } +log = "0.4.8" +quicksink = "0.1" +rustls = "0.16" +rw-stream-sink = { version = "0.2.0", path = "../../misc/rw-stream-sink" } +soketto = { version = "0.3", features = ["deflate"] } +url = "2.1" +webpki = "0.21" +webpki-roots = "0.18" [dev-dependencies] -libp2p-tcp = { version = "0.13.0", path = "../tcp" } -tokio = "0.1.20" +libp2p-tcp = { version = "0.14.0-alpha.1", path = "../tcp" } diff --git a/transports/websocket/src/framed.rs b/transports/websocket/src/framed.rs index b82720a1566..c3966da4a10 100644 --- a/transports/websocket/src/framed.rs +++ b/transports/websocket/src/framed.rs @@ -18,9 +18,11 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use async_tls::{client, server}; use bytes::BytesMut; use crate::{error::Error, tls}; -use futures::{future::{self, Either, Loop}, prelude::*, try_ready}; +use either::Either; +use futures::{future::BoxFuture, prelude::*, ready, stream::BoxStream}; use libp2p_core::{ Transport, either::EitherOutput, @@ -28,21 +30,12 @@ use libp2p_core::{ transport::{ListenerEvent, TransportError} }; use log::{debug, trace}; -use tokio_rustls::{client, server}; -use soketto::{ - base, - connection::{Connection, Mode}, - extension::deflate::Deflate, - handshake::{self, Redirect, Response} -}; -use std::{convert::TryFrom, io}; -use tokio_codec::{Framed, FramedParts}; -use tokio_io::{AsyncRead, AsyncWrite}; -use tokio_rustls::webpki; +use soketto::{connection, data, extension::deflate::Deflate, handshake}; +use std::{convert::TryInto, fmt, io, pin::Pin, task::Context, task::Poll}; use url::Url; /// Max. number of payload bytes of a single frame. -const MAX_DATA_SIZE: u64 = 256 * 1024 * 1024; +const MAX_DATA_SIZE: usize = 256 * 1024 * 1024; /// A Websocket transport whose output type is a [`Stream`] and [`Sink`] of /// frame payloads which does not implement [`AsyncRead`] or @@ -50,7 +43,7 @@ const MAX_DATA_SIZE: u64 = 256 * 1024 * 1024; #[derive(Debug, Clone)] pub struct WsConfig { transport: T, - max_data_size: u64, + max_data_size: usize, tls_config: tls::Config, max_redirects: u8, use_deflate: bool @@ -80,12 +73,12 @@ impl WsConfig { } /// Get the max. frame data size we support. - pub fn max_data_size(&self) -> u64 { + pub fn max_data_size(&self) -> usize { self.max_data_size } /// Set the max. frame data size we support. - pub fn set_max_data_size(&mut self, size: u64) -> &mut Self { + pub fn set_max_data_size(&mut self, size: usize) -> &mut Self { self.max_data_size = size; self } @@ -103,6 +96,8 @@ impl WsConfig { } } +type TlsOrPlain = EitherOutput, server::TlsStream>, T>; + impl Transport for WsConfig where T: Transport + Send + Clone + 'static, @@ -110,13 +105,13 @@ where T::Dial: Send + 'static, T::Listener: Send + 'static, T::ListenerUpgrade: Send + 'static, - T::Output: AsyncRead + AsyncWrite + Send + 'static + T::Output: AsyncRead + AsyncWrite + Unpin + Send + 'static { - type Output = BytesConnection; + type Output = Connection; type Error = Error; - type Listener = Box, Error = Self::Error> + Send>; - type ListenerUpgrade = Box + Send>; - type Dial = Box + Send>; + type Listener = BoxStream<'static, Result, Self::Error>>; + type ListenerUpgrade = BoxFuture<'static, Result>; + type Dial = BoxFuture<'static, Result>; fn listen_on(self, addr: Multiaddr) -> Result> { let mut inner_addr = addr.clone(); @@ -139,10 +134,10 @@ where let tls_config = self.tls_config; let max_size = self.max_data_size; let use_deflate = self.use_deflate; - let listen = self.transport.listen_on(inner_addr) - .map_err(|e| e.map(Error::Transport))? + let transport = self.transport.listen_on(inner_addr).map_err(|e| e.map(Error::Transport))?; + let listen = transport .map_err(Error::Transport) - .map(move |event| match event { + .map_ok(move |event| match event { ListenerEvent::NewAddress(mut a) => { a = a.with(proto.clone()); debug!("Listening on {}", a); @@ -158,60 +153,79 @@ where let remote1 = remote_addr.clone(); // used for logging let remote2 = remote_addr.clone(); // used for logging let tls_config = tls_config.clone(); - let upgraded = upgrade.map_err(Error::Transport) - .and_then(move |stream| { - trace!("incoming connection from {}", remote1); + + let upgrade = async move { + let stream = upgrade.map_err(Error::Transport).await?; + trace!("incoming connection from {}", remote1); + + let stream = if use_tls { // begin TLS session - let server = tls_config.server.expect("for use_tls we checked server"); + let server = tls_config + .server + .expect("for use_tls we checked server is not none"); + trace!("awaiting TLS handshake with {}", remote1); - let future = server.accept(stream) + + let stream = server.accept(stream) .map_err(move |e| { debug!("TLS handshake with {} failed: {}", remote1, e); Error::Tls(tls::Error::from(e)) }) - .map(|s| EitherOutput::First(EitherOutput::Second(s))); - Either::A(future) + .await?; + + let stream: TlsOrPlain<_> = + EitherOutput::First(EitherOutput::Second(stream)); + + stream } else { // continue with plain stream - Either::B(future::ok(EitherOutput::Second(stream))) - } - }) - .and_then(move |stream| { - trace!("receiving websocket handshake request from {}", remote2); - let mut s = handshake::Server::new(); - if use_deflate { - s.add_extension(Box::new(Deflate::new(Mode::Server))); - } - Framed::new(stream, s) - .into_future() - .map_err(|(e, _framed)| Error::Handshake(Box::new(e))) - .and_then(move |(request, framed)| { - if let Some(r) = request { - trace!("accepting websocket handshake request from {}", remote2); - let key = Vec::from(r.key()); - Either::A(framed.send(Ok(handshake::Accept::new(key))) - .map_err(|e| Error::Base(Box::new(e))) - .map(move |f| { - trace!("websocket handshake with {} successful", remote2); - let (mut handshake, mut c) = - new_connection(f, max_size, Mode::Server); - c.add_extensions(handshake.drain_extensions()); - BytesConnection { inner: c } - })) - } else { - debug!("connection to {} terminated during handshake", remote2); - let e: io::Error = io::ErrorKind::ConnectionAborted.into(); - Either::B(future::err(Error::Handshake(Box::new(e)))) - } - }) - }); + EitherOutput::Second(stream) + }; + + trace!("receiving websocket handshake request from {}", remote2); + + let mut server = handshake::Server::new(stream); + + if use_deflate { + server.add_extension(Box::new(Deflate::new(connection::Mode::Server))); + } + + let ws_key = { + let request = server.receive_request() + .map_err(|e| Error::Handshake(Box::new(e))) + .await?; + request.into_key() + }; + + trace!("accepting websocket handshake request from {}", remote2); + + let response = + handshake::server::Response::Accept { + key: &ws_key, + protocol: None + }; + + server.send_response(&response) + .map_err(|e| Error::Handshake(Box::new(e))) + .await?; + + let conn = { + let mut builder = server.into_builder(); + builder.set_max_message_size(max_size); + builder.set_max_frame_size(max_size); + Connection::new(builder) + }; + + Ok(conn) + }; + ListenerEvent::Upgrade { - upgrade: Box::new(upgraded) as Box + Send>, + upgrade: Box::pin(upgrade) as BoxFuture<'static, _>, local_addr, remote_addr } } }); - Ok(Box::new(listen) as Box<_>) + Ok(Box::pin(listen)) } fn dial(self, addr: Multiaddr) -> Result> { @@ -222,121 +236,115 @@ where debug!("{} is not a websocket multiaddr", addr); return Err(TransportError::MultiaddrNotSupported(addr)) } + // We are looping here in order to follow redirects (if any): - let max_redirects = self.max_redirects; - let future = future::loop_fn((addr, self, max_redirects), |(addr, cfg, remaining)| { - dial(addr, cfg.clone()).and_then(move |result| match result { - Either::A(redirect) => { - if remaining == 0 { - debug!("too many redirects"); - return Err(Error::TooManyRedirects) + let mut remaining_redirects = self.max_redirects; + let mut addr = addr; + let future = async move { + loop { + let this = self.clone(); + match this.dial_once(addr).await { + Ok(Either::Left(redirect)) => { + if remaining_redirects == 0 { + debug!("too many redirects"); + return Err(Error::TooManyRedirects) + } + remaining_redirects -= 1; + addr = location_to_multiaddr(&redirect)? } - let a = location_to_multiaddr(redirect.location())?; - Ok(Loop::Continue((a, cfg, remaining - 1))) + Ok(Either::Right(conn)) => return Ok(conn), + Err(e) => return Err(e) } - Either::B(conn) => Ok(Loop::Break(conn)) - }) - }); - Ok(Box::new(future) as Box<_>) + } + }; + + Ok(Box::pin(future)) } } -/// Attempty to dial the given address and perform a websocket handshake. -fn dial(address: Multiaddr, config: WsConfig) - -> impl Future>, Error = Error> +impl WsConfig where T: Transport, - T::Output: AsyncRead + AsyncWrite + T::Output: AsyncRead + AsyncWrite + Send + Unpin + 'static { - trace!("dial address: {}", address); + /// Attempty to dial the given address and perform a websocket handshake. + async fn dial_once(self, address: Multiaddr) -> Result>, Error> { + trace!("dial address: {}", address); - let WsConfig { transport, max_data_size, tls_config, .. } = config; + let (host_port, dns_name) = host_and_dnsname(&address)?; - let (host_port, dns_name) = match host_and_dnsname(&address) { - Ok(x) => x, - Err(e) => return Either::A(future::err(e)) - }; + let mut inner_addr = address.clone(); - let mut inner_addr = address.clone(); + let (use_tls, path) = + match inner_addr.pop() { + Some(Protocol::Ws(path)) => (false, path), + Some(Protocol::Wss(path)) => { + if dns_name.is_none() { + debug!("no DNS name in {}", address); + return Err(Error::InvalidMultiaddr(address)) + } + (true, path) + } + _ => { + debug!("{} is not a websocket multiaddr", address); + return Err(Error::InvalidMultiaddr(address)) + } + }; - let (use_tls, path) = match inner_addr.pop() { - Some(Protocol::Ws(path)) => (false, path), - Some(Protocol::Wss(path)) => { - if dns_name.is_none() { - debug!("no DNS name in {}", address); - return Either::A(future::err(Error::InvalidMultiaddr(address))) - } - (true, path) - } - _ => { - debug!("{} is not a websocket multiaddr", address); - return Either::A(future::err(Error::InvalidMultiaddr(address))) - } - }; - - let dial = match transport.dial(inner_addr) { - Ok(dial) => dial, - Err(TransportError::MultiaddrNotSupported(a)) => - return Either::A(future::err(Error::InvalidMultiaddr(a))), - Err(TransportError::Other(e)) => - return Either::A(future::err(Error::Transport(e))) - }; - - let address1 = address.clone(); // used for logging - let address2 = address.clone(); // used for logging - let use_deflate = config.use_deflate; - let future = dial.map_err(Error::Transport) - .and_then(move |stream| { - trace!("connected to {}", address); + let dial = self.transport.dial(inner_addr) + .map_err(|e| match e { + TransportError::MultiaddrNotSupported(a) => Error::InvalidMultiaddr(a), + TransportError::Other(e) => Error::Transport(e) + })?; + + let stream = dial.map_err(Error::Transport).await?; + trace!("connected to {}", address); + + let stream = if use_tls { // begin TLS session let dns_name = dns_name.expect("for use_tls we have checked that dns_name is some"); trace!("starting TLS handshake with {}", address); - let future = tls_config.client.connect(dns_name.as_ref(), stream) - .map_err(move |e| { + let stream = self.tls_config.client.connect(&dns_name, stream) + .map_err(|e| { + // We should never enter here as we passed a `DNSNameRef` to `connect`. + debug!("invalid domain name: {:?}", dns_name); + Error::Tls(e.into()) + })? + .map_err(|e| { debug!("TLS handshake with {} failed: {}", address, e); Error::Tls(tls::Error::from(e)) }) - .map(|s| EitherOutput::First(EitherOutput::First(s))); - return Either::A(future) + .await?; + + let stream: TlsOrPlain<_> = EitherOutput::First(EitherOutput::First(stream)); + stream + } else { // continue with plain stream + EitherOutput::Second(stream) + }; + + trace!("sending websocket handshake request to {}", address); + + let mut client = handshake::Client::new(stream, &host_port, path.as_ref()); + + if self.use_deflate { + client.add_extension(Box::new(Deflate::new(connection::Mode::Client))); + } + + match client.handshake().map_err(|e| Error::Handshake(Box::new(e))).await? { + handshake::ServerResponse::Redirect { status_code, location } => { + debug!("received redirect ({}); location: {}", status_code, location); + Ok(Either::Left(location)) } - // continue with plain stream - Either::B(future::ok(EitherOutput::Second(stream))) - }) - .and_then(move |stream| { - trace!("sending websocket handshake request to {}", address1); - let mut client = handshake::Client::new(host_port, path); - if use_deflate { - client.add_extension(Box::new(Deflate::new(Mode::Client))); + handshake::ServerResponse::Rejected { status_code } => { + let msg = format!("server rejected handshake; status code = {}", status_code); + Err(Error::Handshake(msg.into())) } - Framed::new(stream, client) - .send(()) - .map_err(|e| Error::Handshake(Box::new(e))) - .and_then(move |framed| { - trace!("awaiting websocket handshake response form {}", address2); - framed.into_future().map_err(|(e, _)| Error::Base(Box::new(e))) - }) - .and_then(move |(response, framed)| { - match response { - None => { - debug!("connection to {} terminated during handshake", address1); - let e: io::Error = io::ErrorKind::ConnectionAborted.into(); - return Err(Error::Handshake(Box::new(e))) - } - Some(Response::Redirect(r)) => { - debug!("received {}", r); - return Ok(Either::A(r)) - } - Some(Response::Accepted(_)) => { - trace!("websocket handshake with {} successful", address1) - } - } - let (mut handshake, mut c) = new_connection(framed, max_data_size, Mode::Client); - c.add_extensions(handshake.drain_extensions()); - Ok(Either::B(BytesConnection { inner: c })) - }) - }); - - Either::B(future) + handshake::ServerResponse::Accepted { .. } => { + trace!("websocket handshake with {} successful", address); + Ok(Either::Right(Connection::new(client.into_builder()))) + } + } + } } // Extract host, port and optionally the DNS name from the given [`Multiaddr`]. @@ -396,63 +404,153 @@ fn location_to_multiaddr(location: &str) -> Result> { } } -/// Create a `Connection` from an existing `Framed` value. -fn new_connection(framed: Framed, max_size: u64, mode: Mode) -> (C, Connection) +/// The websocket connection. +pub struct Connection { + receiver: BoxStream<'static, Result>, + sender: Pin + Send>>, + _marker: std::marker::PhantomData +} + +/// Data received over the websocket connection. +#[derive(Debug, Clone)] +pub struct IncomingData(data::Incoming); + +impl IncomingData { + pub fn is_binary(&self) -> bool { + self.0.is_binary() + } + + pub fn is_text(&self) -> bool { + self.0.is_text() + } + + pub fn is_data(&self) -> bool { + self.0.is_data() + } + + pub fn is_pong(&self) -> bool { + self.0.is_pong() + } +} + +impl AsRef<[u8]> for IncomingData { + fn as_ref(&self) -> &[u8] { + self.0.as_ref() + } +} + +/// Data sent over the websocket connection. +#[derive(Debug, Clone)] +pub enum OutgoingData { + /// Send some bytes. + Binary(BytesMut), + /// Send a PING message. + Ping(BytesMut), + /// Send an unsolicited PONG message. + /// (Incoming PINGs are answered automatically.) + Pong(BytesMut) +} + +impl fmt::Debug for Connection { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.write_str("Connection") + } +} + +impl Connection where - T: AsyncRead + AsyncWrite + T: AsyncRead + AsyncWrite + Send + Unpin + 'static { - let mut codec = base::Codec::new(); - codec.set_max_data_size(max_size); - let old = framed.into_parts(); - let mut new = FramedParts::new(old.io, codec); - new.read_buf = old.read_buf; - new.write_buf = old.write_buf; - let framed = Framed::from_parts(new); - let mut conn = Connection::from_framed(framed, mode); - conn.set_max_buffer_size(usize::try_from(max_size).unwrap_or(std::usize::MAX)); - (old.codec, conn) -} + fn new(builder: connection::Builder>) -> Self { + let (sender, receiver) = builder.finish(); + let sink = quicksink::make_sink(sender, |mut sender, action| async move { + match action { + quicksink::Action::Send(OutgoingData::Binary(x)) => { + sender.send_binary_mut(x).await? + } + quicksink::Action::Send(OutgoingData::Ping(x)) => { + let data = x.as_ref().try_into().map_err(|_| { + io::Error::new(io::ErrorKind::InvalidInput, "PING data must be < 126 bytes") + })?; + sender.send_ping(data).await? + } + quicksink::Action::Send(OutgoingData::Pong(x)) => { + let data = x.as_ref().try_into().map_err(|_| { + io::Error::new(io::ErrorKind::InvalidInput, "PONG data must be < 126 bytes") + })?; + sender.send_pong(data).await? + } + quicksink::Action::Flush => sender.flush().await?, + quicksink::Action::Close => sender.close().await? + } + Ok(sender) + }); + Connection { + receiver: connection::into_stream(receiver).boxed(), + sender: Box::pin(sink), + _marker: std::marker::PhantomData + } + } -// BytesConnection //////////////////////////////////////////////////////////////////////////////// + /// Send binary application data to the remote. + pub fn send_data(&mut self, data: impl Into) -> sink::Send<'_, Self, OutgoingData> { + self.send(OutgoingData::Binary(data.into())) + } + + /// Send a PING to the remote. + pub fn send_ping(&mut self, data: impl Into) -> sink::Send<'_, Self, OutgoingData> { + self.send(OutgoingData::Ping(data.into())) + } -/// A [`Stream`] and [`Sink`] that produces and consumes [`BytesMut`] values -/// which correspond to the payload data of websocket frames. -#[derive(Debug)] -pub struct BytesConnection { - inner: Connection, server::TlsStream>, T>> + /// Send an unsolicited PONG to the remote. + pub fn send_pong(&mut self, data: impl Into) -> sink::Send<'_, Self, OutgoingData> { + self.send(OutgoingData::Pong(data.into())) + } } -impl Stream for BytesConnection { - type Item = BytesMut; - type Error = io::Error; +impl Stream for Connection +where + T: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + type Item = io::Result; - fn poll(&mut self) -> Poll, Self::Error> { - let data = try_ready!(self.inner.poll().map_err(|e| io::Error::new(io::ErrorKind::Other, e))); - Ok(Async::Ready(data.map(base::Data::into_bytes))) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let item = ready!(self.receiver.poll_next_unpin(cx)); + let item = item.map(|result| { + result.map(IncomingData).map_err(|e| io::Error::new(io::ErrorKind::Other, e)) + }); + Poll::Ready(item) } } -impl Sink for BytesConnection { - type SinkItem = BytesMut; - type SinkError = io::Error; +impl Sink for Connection +where + T: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + type Error = io::Error; - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - let result = self.inner.start_send(base::Data::Binary(item)) - .map_err(|e| io::Error::new(io::ErrorKind::Other, e)); + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.sender) + .poll_ready(cx) + .map_err(|e| io::Error::new(io::ErrorKind::Other, e)) + } - if let AsyncSink::NotReady(data) = result? { - Ok(AsyncSink::NotReady(data.into_bytes())) - } else { - Ok(AsyncSink::Ready) - } + fn start_send(mut self: Pin<&mut Self>, item: OutgoingData) -> io::Result<()> { + Pin::new(&mut self.sender) + .start_send(item) + .map_err(|e| io::Error::new(io::ErrorKind::Other, e)) } - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.inner.poll_complete().map_err(|e| io::Error::new(io::ErrorKind::Other, e)) + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.sender) + .poll_flush(cx) + .map_err(|e| io::Error::new(io::ErrorKind::Other, e)) } - fn close(&mut self) -> Poll<(), Self::SinkError> { - self.inner.close().map_err(|e| io::Error::new(io::ErrorKind::Other, e)) + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.sender) + .poll_close(cx) + .map_err(|e| io::Error::new(io::ErrorKind::Other, e)) } } diff --git a/transports/websocket/src/lib.rs b/transports/websocket/src/lib.rs index 533e1b78706..856be7e5328 100644 --- a/transports/websocket/src/lib.rs +++ b/transports/websocket/src/lib.rs @@ -24,9 +24,10 @@ pub mod error; pub mod framed; pub mod tls; +use bytes::BytesMut; use error::Error; -use framed::BytesConnection; -use futures::prelude::*; +use framed::Connection; +use futures::{future::BoxFuture, prelude::*, stream::BoxStream, ready}; use libp2p_core::{ ConnectedPoint, Transport, @@ -34,7 +35,7 @@ use libp2p_core::{ transport::{map::{MapFuture, MapStream}, ListenerEvent, TransportError} }; use rw_stream_sink::RwStreamSink; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{io, pin::Pin, task::{Context, Poll}}; /// A Websocket transport. #[derive(Debug, Clone)] @@ -60,12 +61,12 @@ impl WsConfig { } /// Get the max. frame data size we support. - pub fn max_data_size(&self) -> u64 { + pub fn max_data_size(&self) -> usize { self.transport.max_data_size() } /// Set the max. frame data size we support. - pub fn set_max_data_size(&mut self, size: u64) -> &mut Self { + pub fn set_max_data_size(&mut self, size: usize) -> &mut Self { self.transport.set_max_data_size(size); self } @@ -98,7 +99,7 @@ where T::Dial: Send + 'static, T::Listener: Send + 'static, T::ListenerUpgrade: Send + 'static, - T::Output: AsyncRead + AsyncWrite + Send + 'static + T::Output: AsyncRead + AsyncWrite + Unpin + Send + 'static { type Output = RwStreamSink>; type Error = Error; @@ -116,84 +117,99 @@ where } /// Type alias corresponding to `framed::WsConfig::Listener`. -pub type InnerStream = - Box<(dyn Stream, Item = ListenerEvent>> + Send)>; +pub type InnerStream = BoxStream<'static, Result>, Error>>; /// Type alias corresponding to `framed::WsConfig::Dial` and `framed::WsConfig::ListenerUpgrade`. -pub type InnerFuture = - Box<(dyn Future, Error = Error> + Send)>; +pub type InnerFuture = BoxFuture<'static, Result, Error>>; /// Function type that wraps a websocket connection (see. `wrap_connection`). -pub type WrapperFn = - fn(BytesConnection, ConnectedPoint) -> RwStreamSink>; +pub type WrapperFn = fn(Connection, ConnectedPoint) -> RwStreamSink>; /// Wrap a websocket connection producing data frames into a `RwStreamSink` /// implementing `AsyncRead` + `AsyncWrite`. -fn wrap_connection(c: BytesConnection, _: ConnectedPoint) -> RwStreamSink> +fn wrap_connection(c: Connection, _: ConnectedPoint) -> RwStreamSink> where - T: AsyncRead + AsyncWrite + T: AsyncRead + AsyncWrite + Send + Unpin + 'static { - RwStreamSink::new(c) + RwStreamSink::new(BytesConnection(c)) +} + +/// The websocket connection. +#[derive(Debug)] +pub struct BytesConnection(Connection); + +impl Stream for BytesConnection +where + T: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + type Item = io::Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + loop { + if let Some(item) = ready!(self.0.try_poll_next_unpin(cx)?) { + if item.is_data() { + return Poll::Ready(Some(Ok(BytesMut::from(item.as_ref())))) + } + } else { + return Poll::Ready(None) + } + } + } +} + +impl Sink for BytesConnection +where + T: AsyncRead + AsyncWrite + Send + Unpin + 'static +{ + type Error = io::Error; + + fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.0).poll_ready(cx) + } + + fn start_send(mut self: Pin<&mut Self>, item: BytesMut) -> io::Result<()> { + Pin::new(&mut self.0).start_send(framed::OutgoingData::Binary(item)) + } + + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.0).poll_flush(cx) + } + + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.0).poll_close(cx) + } } // Tests ////////////////////////////////////////////////////////////////////////////////////////// #[cfg(test)] mod tests { + use libp2p_core::Multiaddr; use libp2p_tcp as tcp; - use tokio::runtime::current_thread::Runtime; - use futures::{Future, Stream}; - use libp2p_core::{ - Transport, - multiaddr::Protocol, - transport::ListenerEvent - }; + use futures::prelude::*; + use libp2p_core::{Transport, multiaddr::Protocol}; use super::WsConfig; #[test] fn dialer_connects_to_listener_ipv4() { - let ws_config = WsConfig::new(tcp::TcpConfig::new()); - - let mut listener = ws_config.clone() - .listen_on("/ip4/127.0.0.1/tcp/0/ws".parse().unwrap()) - .unwrap(); - - let addr = listener.by_ref().wait() - .next() - .expect("some event") - .expect("no error") - .into_new_address() - .expect("listen address"); - - assert_eq!(Some(Protocol::Ws("/".into())), addr.iter().nth(2)); - assert_ne!(Some(Protocol::Tcp(0)), addr.iter().nth(1)); - - let listener = listener - .filter_map(ListenerEvent::into_upgrade) - .into_future() - .map_err(|(e, _)| e) - .and_then(|(c, _)| c.unwrap().0); - - let dialer = ws_config.clone().dial(addr.clone()).unwrap(); - - let future = listener - .select(dialer) - .map_err(|(e, _)| e) - .and_then(|(_, n)| n); - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); + let a = "/ip4/127.0.0.1/tcp/0/ws".parse().unwrap(); + futures::executor::block_on(connect(a)) } #[test] fn dialer_connects_to_listener_ipv6() { + let a = "/ip6/::1/tcp/0/ws".parse().unwrap(); + futures::executor::block_on(connect(a)) + } + + async fn connect(listen_addr: Multiaddr) { let ws_config = WsConfig::new(tcp::TcpConfig::new()); let mut listener = ws_config.clone() - .listen_on("/ip6/::1/tcp/0/ws".parse().unwrap()) - .unwrap(); + .listen_on(listen_addr) + .expect("listener"); - let addr = listener.by_ref().wait() - .next() + let addr = listener.try_next().await .expect("some event") .expect("no error") .into_new_address() @@ -202,20 +218,18 @@ mod tests { assert_eq!(Some(Protocol::Ws("/".into())), addr.iter().nth(2)); assert_ne!(Some(Protocol::Tcp(0)), addr.iter().nth(1)); - let listener = listener - .filter_map(ListenerEvent::into_upgrade) - .into_future() - .map_err(|(e, _)| e) - .and_then(|(c, _)| c.unwrap().0); - - let dialer = ws_config.clone().dial(addr.clone()).unwrap(); + let inbound = async move { + let (conn, _addr) = listener.try_filter_map(|e| future::ready(Ok(e.into_upgrade()))) + .try_next() + .await + .unwrap() + .unwrap(); + conn.await + }; - let future = listener - .select(dialer) - .map_err(|(e, _)| e) - .and_then(|(_, n)| n); + let outbound = ws_config.dial(addr).unwrap(); - let mut rt = Runtime::new().unwrap(); - let _ = rt.block_on(future).unwrap(); + let (a, b) = futures::join!(inbound, outbound); + a.and(b).unwrap(); } } diff --git a/transports/websocket/src/tls.rs b/transports/websocket/src/tls.rs index 08c0158084d..18dfb8bce15 100644 --- a/transports/websocket/src/tls.rs +++ b/transports/websocket/src/tls.rs @@ -18,13 +18,8 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use async_tls::{TlsConnector, TlsAcceptor}; use std::{fmt, io, sync::Arc}; -use tokio_rustls::{ - TlsConnector, - TlsAcceptor, - rustls, - webpki -}; /// TLS configuration. #[derive(Clone)] From 4d969efda09ea6705e9d1a5f4ea6d77b6530d87d Mon Sep 17 00:00:00 2001 From: Max Inden Date: Thu, 2 Jan 2020 17:15:52 +0100 Subject: [PATCH 83/91] protocols/gossipsub: Move to stable futures --- protocols/gossipsub/Cargo.toml | 17 ++-- protocols/gossipsub/src/behaviour.rs | 41 +++++----- protocols/gossipsub/src/handler.rs | 117 ++++++++++++++++----------- protocols/gossipsub/src/protocol.rs | 34 ++++---- 4 files changed, 119 insertions(+), 90 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index b483254106c..ffed7f051b0 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -9,16 +9,15 @@ license = "MIT" libp2p-swarm = { path = "../../swarm" } libp2p-core = { path = "../../core" } bs58 = "0.2.2" -bytes = "0.4.11" +bytes = "0.5" byteorder = "1.3.1" fnv = "1.0.6" -futures = "0.1.25" -protobuf = "2.3.0" +futures = { version = "0.3.1" } +protobuf = "= 2.8.1" rand = "0.6.5" -tokio-codec = "0.1.1" -tokio-io = "0.1.11" -tokio-timer = "0.2.8" -unsigned-varint = "0.2.2" +futures_codec = "0.3.4" +wasm-timer = "0.2" +unsigned-varint = { version = "0.3", features = ["futures-codec"] } log = "0.4.6" sha2 = "0.8.0" base64 = "0.10.1" @@ -28,8 +27,8 @@ smallvec = "1.0.0" [dev-dependencies] env_logger = "0.6.0" libp2p = { path = "../../" } -libp2p-plaintext = { path = "../plaintext" } -libp2p-yamux = { path = "../../muxers/yamux" } +libp2p-plaintext = { version = "0.14.0-alpha.1", path = "../plaintext" } +libp2p-yamux = { version = "0.14.0-alpha.1", path = "../../muxers/yamux" } quickcheck = "0.9.0" rand = "0.7.2" tokio = "0.1" diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 23c7b89e5a4..b5a5882531a 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -33,13 +33,17 @@ use log::{debug, error, info, trace, warn}; use lru::LruCache; use rand; use rand::{seq::SliceRandom, thread_rng}; -use std::collections::hash_map::HashMap; -use std::collections::HashSet; -use std::sync::Arc; -use std::time::Instant; -use std::{collections::VecDeque, iter, marker::PhantomData}; -use tokio_io::{AsyncRead, AsyncWrite}; -use tokio_timer::Interval; +use std::{ + collections::HashSet, + collections::VecDeque, + collections::hash_map::HashMap, + iter, + marker::PhantomData, + sync::Arc, + task::{Context, Poll}, + time::Instant, +}; +use wasm_timer::Interval; mod tests; @@ -112,7 +116,7 @@ impl Gossipsub { gs_config.message_id_fn, ), received: LruCache::new(256), // keep track of the last 256 messages - heartbeat: Interval::new( + heartbeat: Interval::new_at( Instant::now() + gs_config.heartbeat_initial_delay, gs_config.heartbeat_interval, ), @@ -987,7 +991,7 @@ impl Gossipsub { impl NetworkBehaviour for Gossipsub where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static, { type ProtocolsHandler = GossipsubHandler; type OutEvent = GossipsubEvent; @@ -1123,8 +1127,9 @@ where fn poll( &mut self, + cx: &mut Context, _: &mut impl PollParameters, - ) -> Async< + ) -> Poll< NetworkBehaviourAction< ::InEvent, Self::OutEvent, @@ -1138,35 +1143,35 @@ where event: send_event, } => match Arc::try_unwrap(send_event) { Ok(event) => { - return Async::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }); + return Poll::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }); } Err(event) => { - return Async::Ready(NetworkBehaviourAction::SendEvent { + return Poll::Ready(NetworkBehaviourAction::SendEvent { peer_id, event: (*event).clone(), }); } }, NetworkBehaviourAction::GenerateEvent(e) => { - return Async::Ready(NetworkBehaviourAction::GenerateEvent(e)); + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(e)); } NetworkBehaviourAction::DialAddress { address } => { - return Async::Ready(NetworkBehaviourAction::DialAddress { address }); + return Poll::Ready(NetworkBehaviourAction::DialAddress { address }); } NetworkBehaviourAction::DialPeer { peer_id } => { - return Async::Ready(NetworkBehaviourAction::DialPeer { peer_id }); + return Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id }); } NetworkBehaviourAction::ReportObservedAddr { address } => { - return Async::Ready(NetworkBehaviourAction::ReportObservedAddr { address }); + return Poll::Ready(NetworkBehaviourAction::ReportObservedAddr { address }); } } } - while let Ok(Async::Ready(Some(_))) = self.heartbeat.poll() { + while let Poll::Ready(Some(())) = self.heartbeat.poll_next_unpin(cx) { self.heartbeat(); } - Async::NotReady + Poll::Pending } } diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 7498efaeeee..907265c0550 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -27,10 +27,13 @@ use libp2p_swarm::protocols_handler::{ }; use log::{trace, warn}; use smallvec::SmallVec; -use std::borrow::Cow; -use std::io; -use tokio_codec::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{ + borrow::Cow, + io, + pin::Pin, + task::{Context, Poll}, +}; +use futures_codec::Framed; /// Protocol Handler that manages a single long-lived substream with a peer. pub struct GossipsubHandler @@ -119,7 +122,7 @@ where impl ProtocolsHandler for GossipsubHandler where - TSubstream: AsyncRead + AsyncWrite, + TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static, { type InEvent = GossipsubRpc; type OutEvent = GossipsubRpc; @@ -135,7 +138,7 @@ where fn inject_fully_negotiated_inbound( &mut self, - substream: >::Output, + substream: >>::Output, ) { // new inbound substream. Replace the current one, if it exists. trace!("New inbound substream request"); @@ -144,7 +147,7 @@ where fn inject_fully_negotiated_outbound( &mut self, - substream: >::Output, + substream: >>::Output, message: Self::OutboundOpenInfo, ) { // Should never establish a new outbound substream if one already exists. @@ -180,20 +183,20 @@ where fn poll( &mut self, - ) -> Poll< - ProtocolsHandlerEvent, - io::Error, - > { + cx: &mut Context, + ) -> Poll> { // determine if we need to create the stream if !self.send_queue.is_empty() && self.outbound_substream.is_none() { let message = self.send_queue.remove(0); self.send_queue.shrink_to_fit(); - return Ok(Async::Ready( + return Poll::Ready( ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol: self.listen_protocol.clone(), info: message, }, - )); + ); } loop { @@ -203,44 +206,46 @@ where ) { // inbound idle state Some(InboundSubstreamState::WaitingInput(mut substream)) => { - match substream.poll() { - Ok(Async::Ready(Some(message))) => { + match substream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(message))) => { self.inbound_substream = Some(InboundSubstreamState::WaitingInput(substream)); - return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(message))); + return Poll::Ready(ProtocolsHandlerEvent::Custom(message)); + } + Poll::Ready(Some(Err(e))) => { + // TODO: Should we really just close here? Not even log the error? + self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); } // peer closed the stream - Ok(Async::Ready(None)) => { + Poll::Ready(None) => { self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); } - Ok(Async::NotReady) => { + Poll::Pending => { self.inbound_substream = Some(InboundSubstreamState::WaitingInput(substream)); break; } - Err(_) => { - self.inbound_substream = Some(InboundSubstreamState::Closing(substream)) - } } } - Some(InboundSubstreamState::Closing(mut substream)) => match substream.close() { - Ok(Async::Ready(())) => { + Some(InboundSubstreamState::Closing(mut substream)) => match Sink::poll_close(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { self.inbound_substream = None; if self.outbound_substream.is_none() { self.keep_alive = KeepAlive::No; } break; } - Ok(Async::NotReady) => { - self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); - break; - } - Err(_) => { - return Err(io::Error::new( + // TODO: Shouldn't we at least log the error in debug mode? + Poll::Ready(Err(_)) => { + return Poll::Ready(ProtocolsHandlerEvent::Close(io::Error::new( io::ErrorKind::BrokenPipe, "Failed to close stream", - )) + ))) + } + Poll::Pending => { + self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); + break; } }, None => { @@ -272,12 +277,23 @@ where } } Some(OutboundSubstreamState::PendingSend(mut substream, message)) => { - match substream.start_send(message)? { - AsyncSink::Ready => { - self.outbound_substream = - Some(OutboundSubstreamState::PendingFlush(substream)) + match Sink::poll_ready(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { + match Sink::start_send(Pin::new(&mut substream), message) { + Ok(()) => { + self.outbound_substream = + Some(OutboundSubstreamState::PendingFlush(substream)) + } + Err(e) => { + return Poll::Ready(ProtocolsHandlerEvent::Close(e)); + } + } + } + Poll::Ready(Err(e)) => { + // TODO: Is this what we want? + return Poll::Ready(ProtocolsHandlerEvent::Close(e)); } - AsyncSink::NotReady(message) => { + Poll::Pending => { self.outbound_substream = Some(OutboundSubstreamState::PendingSend(substream, message)); break; @@ -285,12 +301,16 @@ where } } Some(OutboundSubstreamState::PendingFlush(mut substream)) => { - match substream.poll_complete()? { - Async::Ready(()) => { + match Sink::poll_flush(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { self.outbound_substream = Some(OutboundSubstreamState::WaitingOutput(substream)) } - Async::NotReady => { + Poll::Ready(Err(e)) => { + // TODO: Is this what we want? + return Poll::Ready(ProtocolsHandlerEvent::Close(e)) + } + Poll::Pending => { self.outbound_substream = Some(OutboundSubstreamState::PendingFlush(substream)); break; @@ -298,23 +318,24 @@ where } } // Currently never used - manual shutdown may implement this in the future - Some(OutboundSubstreamState::_Closing(mut substream)) => match substream.close() { - Ok(Async::Ready(())) => { + Some(OutboundSubstreamState::_Closing(mut substream)) => match Sink::poll_close(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(()) )=> { self.outbound_substream = None; if self.inbound_substream.is_none() { self.keep_alive = KeepAlive::No; } break; } - Ok(Async::NotReady) => { - self.outbound_substream = Some(OutboundSubstreamState::_Closing(substream)); - break; - } - Err(_) => { - return Err(io::Error::new( + Poll::Ready(Err(_)) => { + // TODO: Why do we not at least log the error (`_`)? + return Poll::Ready(ProtocolsHandlerEvent::Close(io::Error::new( io::ErrorKind::BrokenPipe, "Failed to close outbound substream", - )) + ))) + } + Poll::Pending => { + self.outbound_substream = Some(OutboundSubstreamState::_Closing(substream)); + break; } }, None => { @@ -327,6 +348,6 @@ where } } - Ok(Async::NotReady) + Poll::Pending } } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 00f94689fec..11dfb3971a5 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -4,13 +4,17 @@ use crate::topic::TopicHash; use byteorder::{BigEndian, ByteOrder}; use bytes::Bytes; use bytes::BytesMut; +use futures::prelude::*; use futures::future; -use libp2p_core::{upgrade, InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; +use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; use protobuf::Message as ProtobufMessage; -use std::borrow::Cow; -use std::{io, iter}; -use tokio_codec::{Decoder, Encoder, Framed}; -use tokio_io::{AsyncRead, AsyncWrite}; +use std::{ + borrow::Cow, + io, + iter, + pin::Pin, +}; +use futures_codec::{Decoder, Encoder, Framed}; use unsigned_varint::codec; /// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. @@ -54,31 +58,31 @@ impl UpgradeInfo for ProtocolConfig { impl InboundUpgrade for ProtocolConfig where - TSocket: AsyncRead + AsyncWrite, + TSocket: AsyncRead + AsyncWrite + Unpin + Send + 'static, { - type Output = Framed, GossipsubCodec>; + type Output = Framed; type Error = io::Error; - type Future = future::FutureResult; + type Future = Pin> + Send>>; - fn upgrade_inbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future { let mut length_codec = codec::UviBytes::default(); length_codec.set_max_len(self.max_transmit_size); - future::ok(Framed::new(socket, GossipsubCodec { length_codec })) + Box::pin(future::ok(Framed::new(socket, GossipsubCodec { length_codec }))) } } impl OutboundUpgrade for ProtocolConfig where - TSocket: AsyncWrite + AsyncRead, + TSocket: AsyncWrite + AsyncRead + Unpin + Send + 'static, { - type Output = Framed, GossipsubCodec>; + type Output = Framed; type Error = io::Error; - type Future = future::FutureResult; + type Future = Pin> + Send>>; - fn upgrade_outbound(self, socket: upgrade::Negotiated, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future { let mut length_codec = codec::UviBytes::default(); length_codec.set_max_len(self.max_transmit_size); - future::ok(Framed::new(socket, GossipsubCodec { length_codec })) + Box::pin(future::ok(Framed::new(socket, GossipsubCodec { length_codec }))) } } From 63751300dbd17c4e06e5893015ecd8ef3094c5dc Mon Sep 17 00:00:00 2001 From: Max Inden Date: Thu, 2 Jan 2020 18:10:27 +0100 Subject: [PATCH 84/91] examples/gossipsub-chat.rs: Move to stable futures --- examples/gossipsub-chat.rs | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/examples/gossipsub-chat.rs b/examples/gossipsub-chat.rs index 6d6efc1a768..4cbbb7c2673 100644 --- a/examples/gossipsub-chat.rs +++ b/examples/gossipsub-chat.rs @@ -46,20 +46,21 @@ //! //! The two nodes should then connect. +use async_std::{io, task}; use env_logger::{Builder, Env}; use futures::prelude::*; use libp2p::gossipsub::protocol::MessageId; use libp2p::gossipsub::{GossipsubEvent, GossipsubMessage, Topic}; use libp2p::{ gossipsub, identity, - tokio_codec::{FramedRead, LinesCodec}, PeerId, }; use std::collections::hash_map::DefaultHasher; use std::hash::{Hash, Hasher}; use std::time::Duration; +use std::{error::Error, task::{Context, Poll}}; -fn main() { +fn main() -> Result<(), Box> { Builder::from_env(Env::default().default_filter_or("info")).init(); // Create a random PeerId @@ -68,7 +69,7 @@ fn main() { println!("Local peer id: {:?}", local_peer_id); // Set up an encrypted TCP Transport over the Mplex and Yamux protocols - let transport = libp2p::build_development_transport(local_key); + let transport = libp2p::build_development_transport(local_key)?; // Create a Gossipsub topic let topic = Topic::new("test-net".into()); @@ -113,23 +114,22 @@ fn main() { } // Read full lines from stdin - let stdin = tokio_stdin_stdout::stdin(0); - let mut framed_stdin = FramedRead::new(stdin, LinesCodec::new()); + let mut stdin = io::BufReader::new(io::stdin()).lines(); // Kick it off let mut listening = false; - tokio::run(futures::future::poll_fn(move || -> Result<_, ()> { + task::block_on(future::poll_fn(move |cx: &mut Context| { loop { - match framed_stdin.poll().expect("Error while polling stdin") { - Async::Ready(Some(line)) => swarm.publish(&topic, line.as_bytes()), - Async::Ready(None) => panic!("Stdin closed"), - Async::NotReady => break, + match stdin.try_poll_next_unpin(cx)? { + Poll::Ready(Some(line)) => swarm.publish(&topic, line.as_bytes()), + Poll::Ready(None) => panic!("Stdin closed"), + Poll::Pending => break, }; } loop { - match swarm.poll().expect("Error while polling swarm") { - Async::Ready(Some(gossip_event)) => match gossip_event { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(gossip_event)) => match gossip_event { GossipsubEvent::Message(peer_id, id, message) => println!( "Got message: {} with id: {} from peer: {:?}", String::from_utf8_lossy(&message.data), @@ -138,7 +138,7 @@ fn main() { ), _ => {} }, - Async::Ready(None) | Async::NotReady => break, + Poll::Ready(None) | Poll::Pending => break, } } @@ -149,6 +149,6 @@ fn main() { } } - Ok(Async::NotReady) - })); + Poll::Pending + })) } From 1395205df948091112308217c88277ec07487496 Mon Sep 17 00:00:00 2001 From: Max Inden Date: Thu, 2 Jan 2020 18:25:19 +0100 Subject: [PATCH 85/91] protocols/gossipsub/src/behaviour/tests: Update to stable futures --- protocols/gossipsub/Cargo.toml | 1 + protocols/gossipsub/src/behaviour/tests.rs | 7 ++++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index ffed7f051b0..8d90ce9aec7 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -26,6 +26,7 @@ smallvec = "1.0.0" [dev-dependencies] env_logger = "0.6.0" +async-std = "1.0" libp2p = { path = "../../" } libp2p-plaintext = { version = "0.14.0-alpha.1", path = "../plaintext" } libp2p-yamux = { version = "0.14.0-alpha.1", path = "../../muxers/yamux" } diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 92fbed96b53..4780e605426 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -3,6 +3,7 @@ #[cfg(test)] mod tests { use super::super::*; + use async_std::net::TcpStream; // helper functions for testing @@ -13,14 +14,14 @@ mod tests { topics: Vec, to_subscribe: bool, ) -> ( - Gossipsub, + Gossipsub, Vec, Vec, ) { // generate a default GossipsubConfig let gs_config = GossipsubConfig::default(); // create a gossipsub struct - let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); + let mut gs: Gossipsub = Gossipsub::new(PeerId::random(), gs_config); let mut topic_hashes = vec![]; @@ -40,7 +41,7 @@ mod tests { for _ in 0..peer_no { let peer = PeerId::random(); peers.push(peer.clone()); - as NetworkBehaviour>::inject_connected( + as NetworkBehaviour>::inject_connected( &mut gs, peer.clone(), dummy_connected_point.clone(), From 542b100c04860d7ebb6baa91cc64b2153861214e Mon Sep 17 00:00:00 2001 From: Max Inden Date: Wed, 15 Jan 2020 10:18:50 +0100 Subject: [PATCH 86/91] protocols/gossipsub/tests: Update to stable futures --- protocols/gossipsub/Cargo.toml | 12 ++--- protocols/gossipsub/tests/smoke.rs | 78 +++++++++++++++--------------- 2 files changed, 42 insertions(+), 48 deletions(-) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 8d90ce9aec7..ef3fbf5b28d 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -6,15 +6,15 @@ authors = ["Age Manning "] license = "MIT" [dependencies] -libp2p-swarm = { path = "../../swarm" } -libp2p-core = { path = "../../core" } +libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } +libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } bs58 = "0.2.2" bytes = "0.5" byteorder = "1.3.1" fnv = "1.0.6" futures = { version = "0.3.1" } protobuf = "= 2.8.1" -rand = "0.6.5" +rand = "0.7.2" futures_codec = "0.3.4" wasm-timer = "0.2" unsigned-varint = { version = "0.3", features = ["futures-codec"] } @@ -25,12 +25,8 @@ lru = "0.1.17" smallvec = "1.0.0" [dev-dependencies] -env_logger = "0.6.0" async-std = "1.0" -libp2p = { path = "../../" } +env_logger = "0.7.1" libp2p-plaintext = { version = "0.14.0-alpha.1", path = "../plaintext" } libp2p-yamux = { version = "0.14.0-alpha.1", path = "../../muxers/yamux" } quickcheck = "0.9.0" -rand = "0.7.2" -tokio = "0.1" -tokio-stdin-stdout = "0.1" diff --git a/protocols/gossipsub/tests/smoke.rs b/protocols/gossipsub/tests/smoke.rs index 65cc7d0c22b..fc1185dd87c 100644 --- a/protocols/gossipsub/tests/smoke.rs +++ b/protocols/gossipsub/tests/smoke.rs @@ -18,9 +18,17 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -#![cfg(test)] - use futures::prelude::*; +use log::debug; +use quickcheck::{QuickCheck, TestResult}; +use rand::{random, seq::SliceRandom, SeedableRng}; +use std::{ + io::Error, + pin::Pin, + task::{Context, Poll}, + time::Duration, +}; + use libp2p_core::{ identity, multiaddr::Protocol, @@ -29,16 +37,10 @@ use libp2p_core::{ transport::{boxed::Boxed, MemoryTransport}, upgrade, Multiaddr, PeerId, Transport, }; +use libp2p_gossipsub::{Gossipsub, GossipsubConfig, GossipsubEvent, Topic}; use libp2p_plaintext::PlainText2Config; use libp2p_swarm::Swarm; use libp2p_yamux as yamux; -use log::debug; -use quickcheck::{QuickCheck, TestResult}; -use rand::{random, seq::SliceRandom, SeedableRng}; -use std::{io::Error, time::Duration}; -use tokio::{runtime::current_thread::Runtime, util::FutureExt}; - -use libp2p_gossipsub::{Gossipsub, GossipsubConfig, GossipsubEvent, Topic}; type TestSwarm = Swarm, Gossipsub>>; @@ -48,19 +50,18 @@ struct Graph { } impl Future for Graph { - type Item = (Multiaddr, GossipsubEvent); - type Error = (); + type Output = (Multiaddr, GossipsubEvent); - fn poll(&mut self) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { for (addr, node) in &mut self.nodes { - match node.poll().expect("no error while polling") { - Async::Ready(Some(event)) => return Ok(Async::Ready((addr.clone(), event))), - Async::Ready(None) => panic!("unexpected None when polling nodes"), - Async::NotReady => {} + match node.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => return Poll::Ready((addr.clone(), event)), + Poll::Ready(None) => panic!("unexpected None when polling nodes"), + Poll::Pending => {} } } - Ok(Async::NotReady) + Poll::Pending } } @@ -106,34 +107,33 @@ impl Graph { nodes: connected_nodes, } } + /// Polls the graph and passes each event into the provided FnMut until it returns `true`. - fn wait_for(self, rt: &mut Runtime, mut f: F) -> Self + fn wait_for(self, mut f: F) -> Self where F: FnMut(GossipsubEvent) -> bool, { // The future below should return self. Given that it is a FnMut and not a FnOnce, one needs - // to wrap `self` in an Option, leaving a `None` behind after the final `Async::Ready`. + // to wrap `self` in an Option, leaving a `None` behind after the final `Poll::Ready`. let mut this = Some(self); - let fut = futures::future::poll_fn(move || -> Result<_, ()> { - match &mut this { - Some(graph) => loop { - match graph.poll().expect("no error while polling") { - Async::Ready((_addr, ev)) => { - if f(ev) { - return Ok(Async::Ready(this.take().unwrap())); - } + let fut = futures::future::poll_fn(move |cx| match &mut this { + Some(graph) => loop { + match graph.poll_unpin(cx) { + Poll::Ready((_addr, ev)) => { + if f(ev) { + return Poll::Ready(this.take().unwrap()); } - Async::NotReady => return Ok(Async::NotReady), } - }, - None => panic!("future called after final return"), - } - }) - .timeout(Duration::from_secs(10)) - .map_err(|e| panic!("{:?}", e)); + Poll::Pending => return Poll::Pending, + } + }, + None => panic!("future called after final return"), + }); - rt.block_on(fut).unwrap() + let fut = async_std::future::timeout(Duration::from_secs(10), fut); + + futures::executor::block_on(fut).unwrap() } } @@ -159,7 +159,7 @@ fn build_node() -> (Multiaddr, TestSwarm) { let mut addr: Multiaddr = Protocol::Memory(port).into(); Swarm::listen_on(&mut swarm, addr.clone()).unwrap(); - addr = addr.with(libp2p::core::multiaddr::Protocol::P2p( + addr = addr.with(libp2p_core::multiaddr::Protocol::P2p( public_key.into_peer_id().into(), )); @@ -177,8 +177,6 @@ fn multi_hop_propagation() { debug!("number nodes: {:?}, seed: {:?}", num_nodes, seed); - let mut rt = Runtime::new().unwrap(); - let mut graph = Graph::new_connected(num_nodes, seed); let number_nodes = graph.nodes.len(); @@ -190,7 +188,7 @@ fn multi_hop_propagation() { // Wait for all nodes to be subscribed. let mut subscribed = 0; - graph = graph.wait_for(&mut rt, move |ev| { + graph = graph.wait_for(move |ev| { if let GossipsubEvent::Subscribed { .. } = ev { subscribed += 1; if subscribed == (number_nodes - 1) * 2 { @@ -206,7 +204,7 @@ fn multi_hop_propagation() { // Wait for all nodes to receive the published message. let mut received_msgs = 0; - graph.wait_for(&mut rt, move |ev| { + graph.wait_for(move |ev| { if let GossipsubEvent::Message(..) = ev { received_msgs += 1; if received_msgs == number_nodes - 1 { From 0c054dfacf185b312dff61f887c4fc9e1ebe40fc Mon Sep 17 00:00:00 2001 From: Max Inden Date: Mon, 20 Jan 2020 11:30:24 +0100 Subject: [PATCH 87/91] protocols/gossipsub: Log substream errors --- protocols/gossipsub/src/handler.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 907265c0550..478711d109a 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -25,7 +25,7 @@ use libp2p_core::upgrade::{InboundUpgrade, Negotiated, OutboundUpgrade}; use libp2p_swarm::protocols_handler::{ KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, }; -use log::{trace, warn}; +use log::{debug, trace, warn}; use smallvec::SmallVec; use std::{ borrow::Cow, @@ -213,7 +213,7 @@ where return Poll::Ready(ProtocolsHandlerEvent::Custom(message)); } Poll::Ready(Some(Err(e))) => { - // TODO: Should we really just close here? Not even log the error? + debug!("Inbound substream error while awaiting input: {:?}", e); self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); } // peer closed the stream @@ -236,8 +236,8 @@ where } break; } - // TODO: Shouldn't we at least log the error in debug mode? - Poll::Ready(Err(_)) => { + Poll::Ready(Err(e)) => { + debug!("Inbound substream error while closing: {:?}", e); return Poll::Ready(ProtocolsHandlerEvent::Close(io::Error::new( io::ErrorKind::BrokenPipe, "Failed to close stream", @@ -290,7 +290,6 @@ where } } Poll::Ready(Err(e)) => { - // TODO: Is this what we want? return Poll::Ready(ProtocolsHandlerEvent::Close(e)); } Poll::Pending => { From 93b5bb22a0195754253287657ee1d54df25d17dc Mon Sep 17 00:00:00 2001 From: Max Inden Date: Mon, 20 Jan 2020 12:02:16 +0100 Subject: [PATCH 88/91] protocols/gossipsub: Log outbound substream errors --- protocols/gossipsub/src/handler.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 478711d109a..44b594e5dd9 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -306,7 +306,6 @@ where Some(OutboundSubstreamState::WaitingOutput(substream)) } Poll::Ready(Err(e)) => { - // TODO: Is this what we want? return Poll::Ready(ProtocolsHandlerEvent::Close(e)) } Poll::Pending => { @@ -325,8 +324,8 @@ where } break; } - Poll::Ready(Err(_)) => { - // TODO: Why do we not at least log the error (`_`)? + 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", From 684968c33f3003de73f3db3bfb4330d832b9a24c Mon Sep 17 00:00:00 2001 From: Age Manning Date: Thu, 23 Jan 2020 20:05:44 +1100 Subject: [PATCH 89/91] Remove rust-fmt formatting --- core/build.rs | 2 +- core/src/lib.rs | 21 +- protocols/plaintext/regen_structs_proto.sh | 4 - protocols/plaintext/src/structs_proto.rs | 289 --------------------- src/lib.rs | 96 ++----- 5 files changed, 36 insertions(+), 376 deletions(-) delete mode 100755 protocols/plaintext/regen_structs_proto.sh delete mode 100644 protocols/plaintext/src/structs_proto.rs diff --git a/core/build.rs b/core/build.rs index 9692abd9c81..c08517dee58 100644 --- a/core/build.rs +++ b/core/build.rs @@ -19,5 +19,5 @@ // DEALINGS IN THE SOFTWARE. fn main() { - prost_build::compile_protos(&["src/keys.proto"], &["src"]).unwrap(); + prost_build::compile_protos(&["src/keys.proto"], &["src"]).unwrap(); } diff --git a/core/src/lib.rs b/core/src/lib.rs index a3780ee64f4..beb0ffbbf9a 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -41,8 +41,7 @@ mod keys_proto { /// Multi-address re-export. pub use multiaddr; -pub type Negotiated = - futures::compat::Compat01As03>>; +pub type Negotiated = futures::compat::Compat01As03>>; use std::{future::Future, pin::Pin}; @@ -56,14 +55,14 @@ pub mod nodes; pub mod transport; pub mod upgrade; -pub use identity::PublicKey; pub use multiaddr::Multiaddr; pub use muxing::StreamMuxer; -pub use nodes::ConnectionInfo; pub use peer_id::PeerId; -pub use translation::address_translation; +pub use identity::PublicKey; pub use transport::Transport; -pub use upgrade::{InboundUpgrade, OutboundUpgrade, ProtocolName, UpgradeError, UpgradeInfo}; +pub use translation::address_translation; +pub use upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, UpgradeError, ProtocolName}; +pub use nodes::ConnectionInfo; #[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] pub enum Endpoint { @@ -79,7 +78,7 @@ impl std::ops::Not for Endpoint { fn not(self) -> Self::Output { match self { Endpoint::Dialer => Endpoint::Listener, - Endpoint::Listener => Endpoint::Dialer, + Endpoint::Listener => Endpoint::Dialer } } } @@ -118,7 +117,7 @@ pub enum ConnectedPoint { local_addr: Multiaddr, /// Stack of protocols used to send back data to the remote. send_back_addr: Multiaddr, - }, + } } impl From<&'_ ConnectedPoint> for Endpoint { @@ -138,7 +137,7 @@ impl ConnectedPoint { pub fn to_endpoint(&self) -> Endpoint { match self { ConnectedPoint::Dialer { .. } => Endpoint::Dialer, - ConnectedPoint::Listener { .. } => Endpoint::Listener, + ConnectedPoint::Listener { .. } => Endpoint::Listener } } @@ -146,7 +145,7 @@ impl ConnectedPoint { pub fn is_dialer(&self) -> bool { match self { ConnectedPoint::Dialer { .. } => true, - ConnectedPoint::Listener { .. } => false, + ConnectedPoint::Listener { .. } => false } } @@ -154,7 +153,7 @@ impl ConnectedPoint { pub fn is_listener(&self) -> bool { match self { ConnectedPoint::Dialer { .. } => false, - ConnectedPoint::Listener { .. } => true, + ConnectedPoint::Listener { .. } => true } } } diff --git a/protocols/plaintext/regen_structs_proto.sh b/protocols/plaintext/regen_structs_proto.sh deleted file mode 100755 index 42ecc3ecc43..00000000000 --- a/protocols/plaintext/regen_structs_proto.sh +++ /dev/null @@ -1,4 +0,0 @@ -#!/bin/sh - -../../scripts/protobuf/gen.sh ./src/structs.proto - diff --git a/protocols/plaintext/src/structs_proto.rs b/protocols/plaintext/src/structs_proto.rs deleted file mode 100644 index 4063daea33e..00000000000 --- a/protocols/plaintext/src/structs_proto.rs +++ /dev/null @@ -1,289 +0,0 @@ -// This file is generated by rust-protobuf 2.8.1. Do not edit -// @generated - -// https://github.com/Manishearth/rust-clippy/issues/702 -#![allow(unknown_lints)] -#![allow(clippy::all)] - -#![cfg_attr(rustfmt, rustfmt_skip)] - -#![allow(box_pointers)] -#![allow(dead_code)] -#![allow(missing_docs)] -#![allow(non_camel_case_types)] -#![allow(non_snake_case)] -#![allow(non_upper_case_globals)] -#![allow(trivial_casts)] -#![allow(unsafe_code)] -#![allow(unused_imports)] -#![allow(unused_results)] -//! Generated file from `src/structs.proto` - -use protobuf::Message as Message_imported_for_functions; -use protobuf::ProtobufEnum as ProtobufEnum_imported_for_functions; - -/// Generated files are compatible only with the same version -/// of protobuf runtime. -const _PROTOBUF_VERSION_CHECK: () = ::protobuf::VERSION_2_8_1; - -#[derive(PartialEq,Clone,Default)] -pub struct Exchange { - // message fields - id: ::protobuf::SingularField<::std::vec::Vec>, - pubkey: ::protobuf::SingularField<::std::vec::Vec>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a Exchange { - fn default() -> &'a Exchange { - ::default_instance() - } -} - -impl Exchange { - pub fn new() -> Exchange { - ::std::default::Default::default() - } - - // optional bytes id = 1; - - - pub fn get_id(&self) -> &[u8] { - match self.id.as_ref() { - Some(v) => &v, - None => &[], - } - } - pub fn clear_id(&mut self) { - self.id.clear(); - } - - pub fn has_id(&self) -> bool { - self.id.is_some() - } - - // Param is passed by value, moved - pub fn set_id(&mut self, v: ::std::vec::Vec) { - self.id = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_id(&mut self) -> &mut ::std::vec::Vec { - if self.id.is_none() { - self.id.set_default(); - } - self.id.as_mut().unwrap() - } - - // Take field - pub fn take_id(&mut self) -> ::std::vec::Vec { - self.id.take().unwrap_or_else(|| ::std::vec::Vec::new()) - } - - // optional bytes pubkey = 2; - - - pub fn get_pubkey(&self) -> &[u8] { - match self.pubkey.as_ref() { - Some(v) => &v, - None => &[], - } - } - pub fn clear_pubkey(&mut self) { - self.pubkey.clear(); - } - - pub fn has_pubkey(&self) -> bool { - self.pubkey.is_some() - } - - // Param is passed by value, moved - pub fn set_pubkey(&mut self, v: ::std::vec::Vec) { - self.pubkey = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_pubkey(&mut self) -> &mut ::std::vec::Vec { - if self.pubkey.is_none() { - self.pubkey.set_default(); - } - self.pubkey.as_mut().unwrap() - } - - // Take field - pub fn take_pubkey(&mut self) -> ::std::vec::Vec { - self.pubkey.take().unwrap_or_else(|| ::std::vec::Vec::new()) - } -} - -impl ::protobuf::Message for Exchange { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.id)?; - }, - 2 => { - ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.pubkey)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(ref v) = self.id.as_ref() { - my_size += ::protobuf::rt::bytes_size(1, &v); - } - if let Some(ref v) = self.pubkey.as_ref() { - my_size += ::protobuf::rt::bytes_size(2, &v); - } - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(ref v) = self.id.as_ref() { - os.write_bytes(1, &v)?; - } - if let Some(ref v) = self.pubkey.as_ref() { - os.write_bytes(2, &v)?; - } - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> Exchange { - Exchange::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "id", - |m: &Exchange| { &m.id }, - |m: &mut Exchange| { &mut m.id }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "pubkey", - |m: &Exchange| { &m.pubkey }, - |m: &mut Exchange| { &mut m.pubkey }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "Exchange", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static Exchange { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const Exchange, - }; - unsafe { - instance.get(Exchange::new) - } - } -} - -impl ::protobuf::Clear for Exchange { - fn clear(&mut self) { - self.id.clear(); - self.pubkey.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for Exchange { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for Exchange { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -static file_descriptor_proto_data: &'static [u8] = b"\ - \n\x11src/structs.proto\"2\n\x08Exchange\x12\x0e\n\x02id\x18\x01\x20\x01\ - (\x0cR\x02id\x12\x16\n\x06pubkey\x18\x02\x20\x01(\x0cR\x06pubkeyJ\xb4\ - \x01\n\x06\x12\x04\0\0\x05\x01\n\x08\n\x01\x0c\x12\x03\0\0\x12\n\n\n\x02\ - \x04\0\x12\x04\x02\0\x05\x01\n\n\n\x03\x04\0\x01\x12\x03\x02\x08\x10\n\ - \x0b\n\x04\x04\0\x02\0\x12\x03\x03\x02\x18\n\x0c\n\x05\x04\0\x02\0\x04\ - \x12\x03\x03\x02\n\n\x0c\n\x05\x04\0\x02\0\x05\x12\x03\x03\x0b\x10\n\x0c\ - \n\x05\x04\0\x02\0\x01\x12\x03\x03\x11\x13\n\x0c\n\x05\x04\0\x02\0\x03\ - \x12\x03\x03\x16\x17\n\x0b\n\x04\x04\0\x02\x01\x12\x03\x04\x02\x1c\n\x0c\ - \n\x05\x04\0\x02\x01\x04\x12\x03\x04\x02\n\n\x0c\n\x05\x04\0\x02\x01\x05\ - \x12\x03\x04\x0b\x10\n\x0c\n\x05\x04\0\x02\x01\x01\x12\x03\x04\x11\x17\n\ - \x0c\n\x05\x04\0\x02\x01\x03\x12\x03\x04\x1a\x1b\ -"; - -static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::descriptor::FileDescriptorProto, -}; - -fn parse_descriptor_proto() -> ::protobuf::descriptor::FileDescriptorProto { - ::protobuf::parse_from_bytes(file_descriptor_proto_data).unwrap() -} - -pub fn file_descriptor_proto() -> &'static ::protobuf::descriptor::FileDescriptorProto { - unsafe { - file_descriptor_proto_lazy.get(|| { - parse_descriptor_proto() - }) - } -} diff --git a/src/lib.rs b/src/lib.rs index df01b2c6936..2d762a80070 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -168,18 +168,18 @@ pub use libp2p_deflate as deflate; #[doc(inline)] pub use libp2p_dns as dns; #[doc(inline)] +pub use libp2p_identify as identify; +#[doc(inline)] +pub use libp2p_kad as kad; +#[doc(inline)] pub use libp2p_floodsub as floodsub; #[doc(inline)] pub use libp2p_gossipsub as gossipsub; #[doc(inline)] -pub use libp2p_identify as identify; -#[doc(inline)] -pub use libp2p_kad as kad; +pub use libp2p_mplex as mplex; #[cfg(not(any(target_os = "emscripten", target_os = "unknown")))] #[doc(inline)] pub use libp2p_mdns as mdns; -#[doc(inline)] -pub use libp2p_mplex as mplex; #[cfg(not(any(target_os = "emscripten", target_os = "unknown")))] #[doc(inline)] pub use libp2p_noise as noise; @@ -198,10 +198,7 @@ pub use libp2p_tcp as tcp; pub use libp2p_uds as uds; #[doc(inline)] pub use libp2p_wasm_ext as wasm_ext; -#[cfg(all( - feature = "libp2p-websocket", - not(any(target_os = "emscripten", target_os = "unknown")) -))] +#[cfg(all(feature = "libp2p-websocket", not(any(target_os = "emscripten", target_os = "unknown"))))] #[doc(inline)] pub use libp2p_websocket as websocket; #[doc(inline)] @@ -214,15 +211,16 @@ pub mod simple; pub use self::core::{ identity, + PeerId, + Transport, transport::TransportError, - upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt}, - PeerId, Transport, + upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt} }; -pub use self::multiaddr::{multiaddr as build_multiaddr, Multiaddr}; +pub use libp2p_core_derive::NetworkBehaviour; +pub use self::multiaddr::{Multiaddr, multiaddr as build_multiaddr}; pub use self::simple::SimpleProtocol; pub use self::swarm::Swarm; pub use self::transport_ext::TransportExt; -pub use libp2p_core_derive::NetworkBehaviour; use std::{error, io, time::Duration}; @@ -230,26 +228,10 @@ use std::{error, io, time::Duration}; /// /// > **Note**: This `Transport` is not suitable for production usage, as its implementation /// > reserves the right to support additional protocols or remove deprecated protocols. -pub fn build_development_transport( - keypair: identity::Keypair, -) -> io::Result< - impl Transport< - Output = ( - PeerId, - impl core::muxing::StreamMuxer< - OutboundSubstream = impl Send, - Substream = impl Send, - Error = impl Into, - > + Send - + Sync, - ), - Error = impl error::Error + Send, - Listener = impl Send, - Dial = impl Send, - ListenerUpgrade = impl Send, - > + Clone, -> { - build_tcp_ws_secio_mplex_yamux(keypair) +pub fn build_development_transport(keypair: identity::Keypair) + -> io::Result> + Send + Sync), Error = impl error::Error + Send, Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone> +{ + build_tcp_ws_secio_mplex_yamux(keypair) } /// Builds an implementation of `Transport` that is suitable for usage with the `Swarm`. @@ -258,32 +240,13 @@ pub fn build_development_transport( /// and mplex or yamux as the multiplexing layer. /// /// > **Note**: If you ever need to express the type of this `Transport`. -pub fn build_tcp_ws_secio_mplex_yamux( - keypair: identity::Keypair, -) -> io::Result< - impl Transport< - Output = ( - PeerId, - impl core::muxing::StreamMuxer< - OutboundSubstream = impl Send, - Substream = impl Send, - Error = impl Into, - > + Send - + Sync, - ), - Error = impl error::Error + Send, - Listener = impl Send, - Dial = impl Send, - ListenerUpgrade = impl Send, - > + Clone, -> { +pub fn build_tcp_ws_secio_mplex_yamux(keypair: identity::Keypair) + -> io::Result> + Send + Sync), Error = impl error::Error + Send, Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone> +{ Ok(CommonTransport::new()? .upgrade(core::upgrade::Version::V1) .authenticate(secio::SecioConfig::new(keypair)) - .multiplex(core::upgrade::SelectUpgrade::new( - yamux::Config::default(), - mplex::MplexConfig::new(), - )) + .multiplex(core::upgrade::SelectUpgrade::new(yamux::Config::default(), mplex::MplexConfig::new())) .map(|(peer, muxer), _| (peer, core::muxing::StreamMuxerBox::new(muxer))) .timeout(Duration::from_secs(20))) } @@ -295,21 +258,12 @@ pub fn build_tcp_ws_secio_mplex_yamux( #[derive(Debug, Clone)] struct CommonTransport { // The actual implementation of everything. - inner: CommonTransportInner, + inner: CommonTransportInner } -#[cfg(all( - not(any(target_os = "emscripten", target_os = "unknown")), - feature = "libp2p-websocket" -))] -type InnerImplementation = core::transport::OrTransport< - dns::DnsConfig, - websocket::WsConfig>, ->; -#[cfg(all( - not(any(target_os = "emscripten", target_os = "unknown")), - not(feature = "libp2p-websocket") -))] +#[cfg(all(not(any(target_os = "emscripten", target_os = "unknown")), feature = "libp2p-websocket"))] +type InnerImplementation = core::transport::OrTransport, websocket::WsConfig>>; +#[cfg(all(not(any(target_os = "emscripten", target_os = "unknown")), not(feature = "libp2p-websocket")))] type InnerImplementation = dns::DnsConfig; #[cfg(any(target_os = "emscripten", target_os = "unknown"))] type InnerImplementation = core::transport::dummy::DummyTransport; @@ -332,7 +286,7 @@ impl CommonTransport { }; Ok(CommonTransport { - inner: CommonTransportInner { inner: transport }, + inner: CommonTransportInner { inner: transport } }) } @@ -341,7 +295,7 @@ impl CommonTransport { pub fn new() -> io::Result { let inner = core::transport::dummy::DummyTransport::new(); Ok(CommonTransport { - inner: CommonTransportInner { inner }, + inner: CommonTransportInner { inner } }) } } From 1ca966b2733ce4e1e4e8fc226dea4d8455d7da2d Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 24 Jan 2020 13:40:23 +1100 Subject: [PATCH 90/91] Shift to prost for protobuf compiling --- protocols/gossipsub/Cargo.toml | 33 +- protocols/gossipsub/build.rs | 24 + protocols/gossipsub/regen_structs_proto.sh | 3 - protocols/gossipsub/src/behaviour.rs | 4 +- protocols/gossipsub/src/behaviour/tests.rs | 21 + protocols/gossipsub/src/config.rs | 2 +- protocols/gossipsub/src/handler.rs | 110 +- protocols/gossipsub/src/lib.rs | 7 +- protocols/gossipsub/src/mcache.rs | 2 +- protocols/gossipsub/src/protocol.rs | 269 +- protocols/gossipsub/{ => src}/rpc.proto | 16 +- protocols/gossipsub/src/rpc_proto.rs | 2911 -------------------- protocols/gossipsub/src/topic.rs | 24 +- 13 files changed, 307 insertions(+), 3119 deletions(-) create mode 100644 protocols/gossipsub/build.rs delete mode 100755 protocols/gossipsub/regen_structs_proto.sh rename protocols/gossipsub/{ => src}/rpc.proto (81%) delete mode 100644 protocols/gossipsub/src/rpc_proto.rs diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index ef3fbf5b28d..fab3d9ee01e 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -8,25 +8,28 @@ license = "MIT" [dependencies] libp2p-swarm = { version = "0.4.0-alpha.1", path = "../../swarm" } libp2p-core = { version = "0.14.0-alpha.1", path = "../../core" } -bs58 = "0.2.2" -bytes = "0.5" -byteorder = "1.3.1" +bs58 = "0.3.0" +bytes = "0.5.4" +byteorder = "1.3.2" fnv = "1.0.6" -futures = { version = "0.3.1" } -protobuf = "= 2.8.1" -rand = "0.7.2" +futures = "0.3.1" +rand = "0.7.3" futures_codec = "0.3.4" -wasm-timer = "0.2" -unsigned-varint = { version = "0.3", features = ["futures-codec"] } -log = "0.4.6" -sha2 = "0.8.0" -base64 = "0.10.1" -lru = "0.1.17" -smallvec = "1.0.0" +wasm-timer = "0.2.4" +unsigned-varint = { version = "0.3.0", features = ["futures-codec"] } +log = "0.4.8" +sha2 = "0.8.1" +base64 = "0.11.0" +lru = "0.4.3" +smallvec = "1.1.0" +prost = "0.6.1" [dev-dependencies] -async-std = "1.0" +async-std = "1.4.0" env_logger = "0.7.1" libp2p-plaintext = { version = "0.14.0-alpha.1", path = "../plaintext" } libp2p-yamux = { version = "0.14.0-alpha.1", path = "../../muxers/yamux" } -quickcheck = "0.9.0" +quickcheck = "0.9.2" + +[build-dependencies] +prost-build = "0.6" diff --git a/protocols/gossipsub/build.rs b/protocols/gossipsub/build.rs new file mode 100644 index 00000000000..3de5b750ca2 --- /dev/null +++ b/protocols/gossipsub/build.rs @@ -0,0 +1,24 @@ +// Copyright 2020 Parity Technologies (UK) 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. + +fn main() { + prost_build::compile_protos(&["src/rpc.proto"], &["src"]).unwrap(); +} + diff --git a/protocols/gossipsub/regen_structs_proto.sh b/protocols/gossipsub/regen_structs_proto.sh deleted file mode 100755 index 95a6bda2674..00000000000 --- a/protocols/gossipsub/regen_structs_proto.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/sh - -../../scripts/protobuf/gen.sh src/rpc.proto diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index b5a5882531a..103b2f2d00e 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -1,4 +1,4 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. +// 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"), @@ -34,9 +34,9 @@ use lru::LruCache; use rand; use rand::{seq::SliceRandom, thread_rng}; use std::{ + collections::hash_map::HashMap, collections::HashSet, collections::VecDeque, - collections::hash_map::HashMap, iter, marker::PhantomData, sync::Arc, diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 4780e605426..ac5ce5e1c57 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -1,3 +1,24 @@ +// 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. + + // collection of tests for the gossipsub network behaviour #[cfg(test)] diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index fd56042ab4b..5a715848c39 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -1,4 +1,4 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. +// 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"), diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 44b594e5dd9..adafe2dc0c1 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -1,4 +1,4 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. +// 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"), @@ -21,6 +21,7 @@ use crate::behaviour::GossipsubRpc; use crate::protocol::{GossipsubCodec, ProtocolConfig}; use futures::prelude::*; +use futures_codec::Framed; use libp2p_core::upgrade::{InboundUpgrade, Negotiated, OutboundUpgrade}; use libp2p_swarm::protocols_handler::{ KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, @@ -33,7 +34,6 @@ use std::{ pin::Pin, task::{Context, Poll}, }; -use futures_codec::Framed; /// Protocol Handler that manages a single long-lived substream with a peer. pub struct GossipsubHandler @@ -184,19 +184,22 @@ where fn poll( &mut self, cx: &mut Context, - ) -> Poll> { + ) -> Poll< + ProtocolsHandlerEvent< + Self::OutboundProtocol, + Self::OutboundOpenInfo, + Self::OutEvent, + Self::Error, + >, + > { // determine if we need to create the stream if !self.send_queue.is_empty() && self.outbound_substream.is_none() { let message = self.send_queue.remove(0); self.send_queue.shrink_to_fit(); - return Poll::Ready( - ProtocolsHandlerEvent::OutboundSubstreamRequest { - protocol: self.listen_protocol.clone(), - info: message, - }, - ); + return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + protocol: self.listen_protocol.clone(), + info: message, + }); } loop { @@ -214,7 +217,8 @@ where } Poll::Ready(Some(Err(e))) => { debug!("Inbound substream error while awaiting input: {:?}", e); - self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); + self.inbound_substream = + Some(InboundSubstreamState::Closing(substream)); } // peer closed the stream Poll::Ready(None) => { @@ -228,26 +232,29 @@ where } } } - Some(InboundSubstreamState::Closing(mut substream)) => match Sink::poll_close(Pin::new(&mut substream), cx) { - Poll::Ready(Ok(())) => { - self.inbound_substream = None; - if self.outbound_substream.is_none() { - self.keep_alive = KeepAlive::No; + Some(InboundSubstreamState::Closing(mut substream)) => { + match Sink::poll_close(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { + self.inbound_substream = None; + if self.outbound_substream.is_none() { + self.keep_alive = KeepAlive::No; + } + break; + } + Poll::Ready(Err(e)) => { + debug!("Inbound substream error while closing: {:?}", e); + return Poll::Ready(ProtocolsHandlerEvent::Close(io::Error::new( + io::ErrorKind::BrokenPipe, + "Failed to close stream", + ))); + } + Poll::Pending => { + self.inbound_substream = + Some(InboundSubstreamState::Closing(substream)); + break; } - break; - } - Poll::Ready(Err(e)) => { - debug!("Inbound substream error while closing: {:?}", e); - return Poll::Ready(ProtocolsHandlerEvent::Close(io::Error::new( - io::ErrorKind::BrokenPipe, - "Failed to close stream", - ))) - } - Poll::Pending => { - self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); - break; } - }, + } None => { self.inbound_substream = None; break; @@ -305,9 +312,7 @@ where self.outbound_substream = Some(OutboundSubstreamState::WaitingOutput(substream)) } - Poll::Ready(Err(e)) => { - return Poll::Ready(ProtocolsHandlerEvent::Close(e)) - } + Poll::Ready(Err(e)) => return Poll::Ready(ProtocolsHandlerEvent::Close(e)), Poll::Pending => { self.outbound_substream = Some(OutboundSubstreamState::PendingFlush(substream)); @@ -316,26 +321,29 @@ where } } // Currently never used - manual shutdown may implement this in the future - Some(OutboundSubstreamState::_Closing(mut substream)) => match Sink::poll_close(Pin::new(&mut substream), cx) { - Poll::Ready(Ok(()) )=> { - self.outbound_substream = None; - if self.inbound_substream.is_none() { - self.keep_alive = KeepAlive::No; + Some(OutboundSubstreamState::_Closing(mut substream)) => { + match Sink::poll_close(Pin::new(&mut substream), cx) { + Poll::Ready(Ok(())) => { + self.outbound_substream = None; + if self.inbound_substream.is_none() { + self.keep_alive = KeepAlive::No; + } + 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", + ))); + } + Poll::Pending => { + self.outbound_substream = + Some(OutboundSubstreamState::_Closing(substream)); + break; } - 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", - ))) - } - Poll::Pending => { - self.outbound_substream = Some(OutboundSubstreamState::_Closing(substream)); - break; } - }, + } None => { self.outbound_substream = None; break; diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 5586bf563a8..e0efa955714 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -1,4 +1,4 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. +// 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"), @@ -141,9 +141,12 @@ mod behaviour; mod config; mod handler; mod mcache; -mod rpc_proto; mod topic; +mod rpc_proto { + include!(concat!(env!("OUT_DIR"), "/gossipsub.pb.rs")); +} + pub use self::behaviour::{Gossipsub, GossipsubEvent, GossipsubRpc}; pub use self::config::{GossipsubConfig, GossipsubConfigBuilder}; pub use self::protocol::{GossipsubMessage, MessageId}; diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index b8d45755d78..8e74308ca88 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -1,4 +1,4 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. +// 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"), diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 11dfb3971a5..14a8c6ddd39 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -1,20 +1,35 @@ +// 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::behaviour::GossipsubRpc; use crate::rpc_proto; use crate::topic::TopicHash; use byteorder::{BigEndian, ByteOrder}; use bytes::Bytes; use bytes::BytesMut; -use futures::prelude::*; use futures::future; -use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; -use protobuf::Message as ProtobufMessage; -use std::{ - borrow::Cow, - io, - iter, - pin::Pin, -}; +use futures::prelude::*; use futures_codec::{Decoder, Encoder, Framed}; +use libp2p_core::{InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; +use prost::Message as ProtobufMessage; +use std::{borrow::Cow, io, iter, pin::Pin}; use unsigned_varint::codec; /// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. @@ -67,7 +82,10 @@ where fn upgrade_inbound(self, socket: TSocket, _: 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 { length_codec }))) + Box::pin(future::ok(Framed::new( + socket, + GossipsubCodec { length_codec }, + ))) } } @@ -82,7 +100,10 @@ where fn upgrade_outbound(self, socket: TSocket, _: 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 { length_codec }))) + Box::pin(future::ok(Framed::new( + socket, + GossipsubCodec { length_codec }, + ))) } } @@ -98,33 +119,41 @@ impl Encoder for GossipsubCodec { type Error = io::Error; fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { - let mut proto = rpc_proto::RPC::new(); - - for message in item.messages.into_iter() { - let mut msg = rpc_proto::Message::new(); - msg.set_from(message.source.into_bytes()); - msg.set_data(message.data); - msg.set_seqno(message.sequence_number.to_be_bytes().to_vec()); - msg.set_topicIDs( - message + // messages + let publish = item + .messages + .into_iter() + .map(|message| rpc_proto::Message { + from: Some(message.source.into_bytes()), + data: Some(message.data), + seqno: Some(message.sequence_number.to_be_bytes().to_vec()), + topic_ids: message .topics .into_iter() .map(TopicHash::into_string) .collect(), - ); - proto.mut_publish().push(msg); - } + }) + .collect::>(); - for subscription in item.subscriptions.into_iter() { - let mut rpc_subscription = rpc_proto::RPC_SubOpts::new(); - rpc_subscription - .set_subscribe(subscription.action == GossipsubSubscriptionAction::Subscribe); - rpc_subscription.set_topicid(subscription.topic_hash.into_string()); - proto.mut_subscriptions().push(rpc_subscription); - } + // 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_string()), + }) + .collect::>(); + + // control messages + let mut control = rpc_proto::ControlMessage { + ihave: Vec::new(), + iwant: Vec::new(), + graft: Vec::new(), + prune: Vec::new(), + }; - // gossipsub control messages - let mut control_msg = rpc_proto::ControlMessage::new(); + let empty_control_msg = item.control_msgs.is_empty(); for action in item.control_msgs { match action { @@ -133,41 +162,50 @@ impl Encoder for GossipsubCodec { topic_hash, message_ids, } => { - let mut rpc_ihave = rpc_proto::ControlIHave::new(); - rpc_ihave.set_topicID(topic_hash.into_string()); - for msg_id in message_ids { - rpc_ihave.mut_messageIDs().push(msg_id.0); - } - control_msg.mut_ihave().push(rpc_ihave); + 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 mut rpc_iwant = rpc_proto::ControlIWant::new(); - for msg_id in message_ids { - rpc_iwant.mut_messageIDs().push(msg_id.0); - } - control_msg.mut_iwant().push(rpc_iwant); + 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 mut rpc_graft = rpc_proto::ControlGraft::new(); - rpc_graft.set_topicID(topic_hash.into_string()); - control_msg.mut_graft().push(rpc_graft); + let rpc_graft = rpc_proto::ControlGraft { + topic_id: Some(topic_hash.into_string()), + }; + control.graft.push(rpc_graft); } GossipsubControlAction::Prune { topic_hash } => { - let mut rpc_prune = rpc_proto::ControlPrune::new(); - rpc_prune.set_topicID(topic_hash.into_string()); - control_msg.mut_prune().push(rpc_prune); + let rpc_prune = rpc_proto::ControlPrune { + topic_id: Some(topic_hash.into_string()), + }; + control.prune.push(rpc_prune); } } } - proto.set_control(control_msg); + 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()); - let bytes = proto - .write_to_bytes() - .expect("there is no situation in which the protobuf message can be invalid"); + rpc.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(bytes), dst) + self.length_codec.encode(Bytes::from(buf), dst) } } @@ -181,94 +219,99 @@ impl Decoder for GossipsubCodec { None => return Ok(None), }; - let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?; + let rpc = rpc_proto::Rpc::decode(&packet[..])?; - let mut messages = Vec::with_capacity(rpc.get_publish().len()); - for mut publish in rpc.take_publish().into_iter() { + let mut messages = Vec::with_capacity(rpc.publish.len()); + for publish in rpc.publish.into_iter() { // ensure the sequence number is a u64 - let raw_seq = publish.take_seqno(); - if raw_seq.len() != 8 { + let seq_no = publish.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", )); } messages.push(GossipsubMessage { - source: PeerId::from_bytes(publish.take_from()) + source: PeerId::from_bytes(publish.from.unwrap_or_default()) .map_err(|_| io::Error::new(io::ErrorKind::InvalidData, "Invalid Peer Id"))?, - data: publish.take_data(), - sequence_number: BigEndian::read_u64(&raw_seq), + data: publish.data.unwrap_or_default(), + sequence_number: BigEndian::read_u64(&seq_no), topics: publish - .take_topicIDs() + .topic_ids .into_iter() .map(TopicHash::from_raw) .collect(), }); } - let mut rpc_control = rpc.take_control(); - let mut control_msgs = vec![]; - // Collect the gossipsub control messages - let ihave_msgs: Vec = rpc_control - .take_ihave() - .into_iter() - .map(|mut ihave| GossipsubControlAction::IHave { - topic_hash: TopicHash::from_raw(ihave.take_topicID()), - message_ids: ihave - .take_messageIDs() - .into_vec() - .into_iter() - .map(|x| MessageId(x)) - .collect::>(), - }) - .collect(); + let mut control_msgs = Vec::new(); - let iwant_msgs: Vec = rpc_control - .take_iwant() - .into_iter() - .map(|mut iwant| GossipsubControlAction::IWant { - message_ids: iwant - .take_messageIDs() - .into_vec() - .into_iter() - .map(|x| MessageId(x)) - .collect::>(), - }) - .collect(); + if let Some(rpc_control) = rpc.control { + // Collect the gossipsub control messages + let ihave_msgs: Vec = rpc_control + .ihave + .into_iter() + .map(|ihave| GossipsubControlAction::IHave { + topic_hash: TopicHash::from_raw(ihave.topic_id.unwrap_or_default()), + message_ids: ihave + .message_ids + .into_iter() + .map(|x| MessageId(x)) + .collect::>(), + }) + .collect(); - let graft_msgs: Vec = rpc_control - .take_graft() - .into_iter() - .map(|mut graft| GossipsubControlAction::Graft { - topic_hash: TopicHash::from_raw(graft.take_topicID()), - }) - .collect(); + let iwant_msgs: Vec = rpc_control + .iwant + .into_iter() + .map(|iwant| GossipsubControlAction::IWant { + message_ids: iwant + .message_ids + .into_iter() + .map(|x| MessageId(x)) + .collect::>(), + }) + .collect(); - let prune_msgs: Vec = rpc_control - .take_prune() - .into_iter() - .map(|mut prune| GossipsubControlAction::Prune { - topic_hash: TopicHash::from_raw(prune.take_topicID()), - }) - .collect(); + let graft_msgs: Vec = rpc_control + .graft + .into_iter() + .map(|graft| GossipsubControlAction::Graft { + topic_hash: TopicHash::from_raw(graft.topic_id.unwrap_or_default()), + }) + .collect(); - control_msgs.extend(ihave_msgs); - control_msgs.extend(iwant_msgs); - control_msgs.extend(graft_msgs); - control_msgs.extend(prune_msgs); + let prune_msgs: Vec = rpc_control + .prune + .into_iter() + .map(|prune| GossipsubControlAction::Prune { + topic_hash: TopicHash::from_raw(prune.topic_id.unwrap_or_default()), + }) + .collect(); + + control_msgs.extend(ihave_msgs); + control_msgs.extend(iwant_msgs); + control_msgs.extend(graft_msgs); + control_msgs.extend(prune_msgs); + } Ok(Some(GossipsubRpc { messages, subscriptions: rpc - .take_subscriptions() + .subscriptions .into_iter() - .map(|mut sub| GossipsubSubscription { - action: if sub.get_subscribe() { + .map(|sub| GossipsubSubscription { + action: if Some(true) == sub.subscribe { GossipsubSubscriptionAction::Subscribe } else { GossipsubSubscriptionAction::Unsubscribe }, - topic_hash: TopicHash::from_raw(sub.take_topicid()), + topic_hash: TopicHash::from_raw(sub.topic_id.unwrap_or_default()), }) .collect(), control_msgs, diff --git a/protocols/gossipsub/rpc.proto b/protocols/gossipsub/src/rpc.proto similarity index 81% rename from protocols/gossipsub/rpc.proto rename to protocols/gossipsub/src/rpc.proto index 2e65c1d8460..1aa19430aa2 100644 --- a/protocols/gossipsub/rpc.proto +++ b/protocols/gossipsub/src/rpc.proto @@ -8,7 +8,7 @@ message RPC { message SubOpts { optional bool subscribe = 1; // subscribe or unsubscribe - optional string topicid = 2; + optional string topic_id = 2; } optional ControlMessage control = 3; @@ -18,7 +18,7 @@ message Message { optional bytes from = 1; optional bytes data = 2; optional bytes seqno = 3; - repeated string topicIDs = 4; + repeated string topic_ids = 4; } message ControlMessage { @@ -29,20 +29,20 @@ message ControlMessage { } message ControlIHave { - optional string topicID = 1; - repeated string messageIDs = 2; + optional string topic_id = 1; + repeated string message_ids = 2; } message ControlIWant { - repeated string messageIDs = 1; + repeated string message_ids= 1; } message ControlGraft { - optional string topicID = 1; + optional string topic_id = 1; } message ControlPrune { - optional string topicID = 1; + optional string topic_id = 1; } // topicID = hash(topicDescriptor); (not the topic.name) @@ -64,7 +64,7 @@ message TopicDescriptor { message EncOpts { optional EncMode mode = 1; - repeated bytes keyHashes = 2; // the hashes of the shared keys used (salted) + repeated bytes key_hashes = 2; // the hashes of the shared keys used (salted) enum EncMode { NONE = 0; // no encryption, anyone can read diff --git a/protocols/gossipsub/src/rpc_proto.rs b/protocols/gossipsub/src/rpc_proto.rs deleted file mode 100644 index 8f128e10c4f..00000000000 --- a/protocols/gossipsub/src/rpc_proto.rs +++ /dev/null @@ -1,2911 +0,0 @@ -// This file is generated by rust-protobuf 2.8.1. Do not edit -// @generated - -// https://github.com/Manishearth/rust-clippy/issues/702 -#![allow(unknown_lints)] -#![allow(clippy::all)] - -#![cfg_attr(rustfmt, rustfmt_skip)] - -#![allow(box_pointers)] -#![allow(dead_code)] -#![allow(missing_docs)] -#![allow(non_camel_case_types)] -#![allow(non_snake_case)] -#![allow(non_upper_case_globals)] -#![allow(trivial_casts)] -#![allow(unsafe_code)] -#![allow(unused_imports)] -#![allow(unused_results)] -//! Generated file from `rpc.proto` - -use protobuf::Message as Message_imported_for_functions; -use protobuf::ProtobufEnum as ProtobufEnum_imported_for_functions; - -/// Generated files are compatible only with the same version -/// of protobuf runtime. -const _PROTOBUF_VERSION_CHECK: () = ::protobuf::VERSION_2_8_1; - -#[derive(PartialEq,Clone,Default)] -pub struct RPC { - // message fields - subscriptions: ::protobuf::RepeatedField, - publish: ::protobuf::RepeatedField, - control: ::protobuf::SingularPtrField, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a RPC { - fn default() -> &'a RPC { - ::default_instance() - } -} - -impl RPC { - pub fn new() -> RPC { - ::std::default::Default::default() - } - - // repeated .gossipsub.pb.RPC.SubOpts subscriptions = 1; - - - pub fn get_subscriptions(&self) -> &[RPC_SubOpts] { - &self.subscriptions - } - pub fn clear_subscriptions(&mut self) { - self.subscriptions.clear(); - } - - // Param is passed by value, moved - pub fn set_subscriptions(&mut self, v: ::protobuf::RepeatedField) { - self.subscriptions = v; - } - - // Mutable pointer to the field. - pub fn mut_subscriptions(&mut self) -> &mut ::protobuf::RepeatedField { - &mut self.subscriptions - } - - // Take field - pub fn take_subscriptions(&mut self) -> ::protobuf::RepeatedField { - ::std::mem::replace(&mut self.subscriptions, ::protobuf::RepeatedField::new()) - } - - // repeated .gossipsub.pb.Message publish = 2; - - - pub fn get_publish(&self) -> &[Message] { - &self.publish - } - pub fn clear_publish(&mut self) { - self.publish.clear(); - } - - // Param is passed by value, moved - pub fn set_publish(&mut self, v: ::protobuf::RepeatedField) { - self.publish = v; - } - - // Mutable pointer to the field. - pub fn mut_publish(&mut self) -> &mut ::protobuf::RepeatedField { - &mut self.publish - } - - // Take field - pub fn take_publish(&mut self) -> ::protobuf::RepeatedField { - ::std::mem::replace(&mut self.publish, ::protobuf::RepeatedField::new()) - } - - // optional .gossipsub.pb.ControlMessage control = 3; - - - pub fn get_control(&self) -> &ControlMessage { - self.control.as_ref().unwrap_or_else(|| ControlMessage::default_instance()) - } - pub fn clear_control(&mut self) { - self.control.clear(); - } - - pub fn has_control(&self) -> bool { - self.control.is_some() - } - - // Param is passed by value, moved - pub fn set_control(&mut self, v: ControlMessage) { - self.control = ::protobuf::SingularPtrField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_control(&mut self) -> &mut ControlMessage { - if self.control.is_none() { - self.control.set_default(); - } - self.control.as_mut().unwrap() - } - - // Take field - pub fn take_control(&mut self) -> ControlMessage { - self.control.take().unwrap_or_else(|| ControlMessage::new()) - } -} - -impl ::protobuf::Message for RPC { - fn is_initialized(&self) -> bool { - for v in &self.subscriptions { - if !v.is_initialized() { - return false; - } - }; - for v in &self.publish { - if !v.is_initialized() { - return false; - } - }; - for v in &self.control { - if !v.is_initialized() { - return false; - } - }; - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.subscriptions)?; - }, - 2 => { - ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.publish)?; - }, - 3 => { - ::protobuf::rt::read_singular_message_into(wire_type, is, &mut self.control)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - for value in &self.subscriptions { - let len = value.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - }; - for value in &self.publish { - let len = value.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - }; - if let Some(ref v) = self.control.as_ref() { - let len = v.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - } - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - for v in &self.subscriptions { - os.write_tag(1, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - }; - for v in &self.publish { - os.write_tag(2, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - }; - if let Some(ref v) = self.control.as_ref() { - os.write_tag(3, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - } - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> RPC { - RPC::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "subscriptions", - |m: &RPC| { &m.subscriptions }, - |m: &mut RPC| { &mut m.subscriptions }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "publish", - |m: &RPC| { &m.publish }, - |m: &mut RPC| { &mut m.publish }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_ptr_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "control", - |m: &RPC| { &m.control }, - |m: &mut RPC| { &mut m.control }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "RPC", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static RPC { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const RPC, - }; - unsafe { - instance.get(RPC::new) - } - } -} - -impl ::protobuf::Clear for RPC { - fn clear(&mut self) { - self.subscriptions.clear(); - self.publish.clear(); - self.control.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for RPC { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for RPC { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct RPC_SubOpts { - // message fields - subscribe: ::std::option::Option, - topicid: ::protobuf::SingularField<::std::string::String>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a RPC_SubOpts { - fn default() -> &'a RPC_SubOpts { - ::default_instance() - } -} - -impl RPC_SubOpts { - pub fn new() -> RPC_SubOpts { - ::std::default::Default::default() - } - - // optional bool subscribe = 1; - - - pub fn get_subscribe(&self) -> bool { - self.subscribe.unwrap_or(false) - } - pub fn clear_subscribe(&mut self) { - self.subscribe = ::std::option::Option::None; - } - - pub fn has_subscribe(&self) -> bool { - self.subscribe.is_some() - } - - // Param is passed by value, moved - pub fn set_subscribe(&mut self, v: bool) { - self.subscribe = ::std::option::Option::Some(v); - } - - // optional string topicid = 2; - - - pub fn get_topicid(&self) -> &str { - match self.topicid.as_ref() { - Some(v) => &v, - None => "", - } - } - pub fn clear_topicid(&mut self) { - self.topicid.clear(); - } - - pub fn has_topicid(&self) -> bool { - self.topicid.is_some() - } - - // Param is passed by value, moved - pub fn set_topicid(&mut self, v: ::std::string::String) { - self.topicid = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_topicid(&mut self) -> &mut ::std::string::String { - if self.topicid.is_none() { - self.topicid.set_default(); - } - self.topicid.as_mut().unwrap() - } - - // Take field - pub fn take_topicid(&mut self) -> ::std::string::String { - self.topicid.take().unwrap_or_else(|| ::std::string::String::new()) - } -} - -impl ::protobuf::Message for RPC_SubOpts { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - if wire_type != ::protobuf::wire_format::WireTypeVarint { - return ::std::result::Result::Err(::protobuf::rt::unexpected_wire_type(wire_type)); - } - let tmp = is.read_bool()?; - self.subscribe = ::std::option::Option::Some(tmp); - }, - 2 => { - ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicid)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(v) = self.subscribe { - my_size += 2; - } - if let Some(ref v) = self.topicid.as_ref() { - my_size += ::protobuf::rt::string_size(2, &v); - } - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(v) = self.subscribe { - os.write_bool(1, v)?; - } - if let Some(ref v) = self.topicid.as_ref() { - os.write_string(2, &v)?; - } - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> RPC_SubOpts { - RPC_SubOpts::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeBool>( - "subscribe", - |m: &RPC_SubOpts| { &m.subscribe }, - |m: &mut RPC_SubOpts| { &mut m.subscribe }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "topicid", - |m: &RPC_SubOpts| { &m.topicid }, - |m: &mut RPC_SubOpts| { &mut m.topicid }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "RPC_SubOpts", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static RPC_SubOpts { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const RPC_SubOpts, - }; - unsafe { - instance.get(RPC_SubOpts::new) - } - } -} - -impl ::protobuf::Clear for RPC_SubOpts { - fn clear(&mut self) { - self.subscribe = ::std::option::Option::None; - self.topicid.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for RPC_SubOpts { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for RPC_SubOpts { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct Message { - // message fields - from: ::protobuf::SingularField<::std::vec::Vec>, - data: ::protobuf::SingularField<::std::vec::Vec>, - seqno: ::protobuf::SingularField<::std::vec::Vec>, - topicIDs: ::protobuf::RepeatedField<::std::string::String>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a Message { - fn default() -> &'a Message { - ::default_instance() - } -} - -impl Message { - pub fn new() -> Message { - ::std::default::Default::default() - } - - // optional bytes from = 1; - - - pub fn get_from(&self) -> &[u8] { - match self.from.as_ref() { - Some(v) => &v, - None => &[], - } - } - pub fn clear_from(&mut self) { - self.from.clear(); - } - - pub fn has_from(&self) -> bool { - self.from.is_some() - } - - // Param is passed by value, moved - pub fn set_from(&mut self, v: ::std::vec::Vec) { - self.from = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_from(&mut self) -> &mut ::std::vec::Vec { - if self.from.is_none() { - self.from.set_default(); - } - self.from.as_mut().unwrap() - } - - // Take field - pub fn take_from(&mut self) -> ::std::vec::Vec { - self.from.take().unwrap_or_else(|| ::std::vec::Vec::new()) - } - - // optional bytes data = 2; - - - pub fn get_data(&self) -> &[u8] { - match self.data.as_ref() { - Some(v) => &v, - None => &[], - } - } - pub fn clear_data(&mut self) { - self.data.clear(); - } - - pub fn has_data(&self) -> bool { - self.data.is_some() - } - - // Param is passed by value, moved - pub fn set_data(&mut self, v: ::std::vec::Vec) { - self.data = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_data(&mut self) -> &mut ::std::vec::Vec { - if self.data.is_none() { - self.data.set_default(); - } - self.data.as_mut().unwrap() - } - - // Take field - pub fn take_data(&mut self) -> ::std::vec::Vec { - self.data.take().unwrap_or_else(|| ::std::vec::Vec::new()) - } - - // optional bytes seqno = 3; - - - pub fn get_seqno(&self) -> &[u8] { - match self.seqno.as_ref() { - Some(v) => &v, - None => &[], - } - } - pub fn clear_seqno(&mut self) { - self.seqno.clear(); - } - - pub fn has_seqno(&self) -> bool { - self.seqno.is_some() - } - - // Param is passed by value, moved - pub fn set_seqno(&mut self, v: ::std::vec::Vec) { - self.seqno = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_seqno(&mut self) -> &mut ::std::vec::Vec { - if self.seqno.is_none() { - self.seqno.set_default(); - } - self.seqno.as_mut().unwrap() - } - - // Take field - pub fn take_seqno(&mut self) -> ::std::vec::Vec { - self.seqno.take().unwrap_or_else(|| ::std::vec::Vec::new()) - } - - // repeated string topicIDs = 4; - - - pub fn get_topicIDs(&self) -> &[::std::string::String] { - &self.topicIDs - } - pub fn clear_topicIDs(&mut self) { - self.topicIDs.clear(); - } - - // Param is passed by value, moved - pub fn set_topicIDs(&mut self, v: ::protobuf::RepeatedField<::std::string::String>) { - self.topicIDs = v; - } - - // Mutable pointer to the field. - pub fn mut_topicIDs(&mut self) -> &mut ::protobuf::RepeatedField<::std::string::String> { - &mut self.topicIDs - } - - // Take field - pub fn take_topicIDs(&mut self) -> ::protobuf::RepeatedField<::std::string::String> { - ::std::mem::replace(&mut self.topicIDs, ::protobuf::RepeatedField::new()) - } -} - -impl ::protobuf::Message for Message { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.from)?; - }, - 2 => { - ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.data)?; - }, - 3 => { - ::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.seqno)?; - }, - 4 => { - ::protobuf::rt::read_repeated_string_into(wire_type, is, &mut self.topicIDs)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(ref v) = self.from.as_ref() { - my_size += ::protobuf::rt::bytes_size(1, &v); - } - if let Some(ref v) = self.data.as_ref() { - my_size += ::protobuf::rt::bytes_size(2, &v); - } - if let Some(ref v) = self.seqno.as_ref() { - my_size += ::protobuf::rt::bytes_size(3, &v); - } - for value in &self.topicIDs { - my_size += ::protobuf::rt::string_size(4, &value); - }; - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(ref v) = self.from.as_ref() { - os.write_bytes(1, &v)?; - } - if let Some(ref v) = self.data.as_ref() { - os.write_bytes(2, &v)?; - } - if let Some(ref v) = self.seqno.as_ref() { - os.write_bytes(3, &v)?; - } - for v in &self.topicIDs { - os.write_string(4, &v)?; - }; - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> Message { - Message::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "from", - |m: &Message| { &m.from }, - |m: &mut Message| { &mut m.from }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "data", - |m: &Message| { &m.data }, - |m: &mut Message| { &mut m.data }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "seqno", - |m: &Message| { &m.seqno }, - |m: &mut Message| { &mut m.seqno }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "topicIDs", - |m: &Message| { &m.topicIDs }, - |m: &mut Message| { &mut m.topicIDs }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "Message", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static Message { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const Message, - }; - unsafe { - instance.get(Message::new) - } - } -} - -impl ::protobuf::Clear for Message { - fn clear(&mut self) { - self.from.clear(); - self.data.clear(); - self.seqno.clear(); - self.topicIDs.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for Message { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for Message { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct ControlMessage { - // message fields - ihave: ::protobuf::RepeatedField, - iwant: ::protobuf::RepeatedField, - graft: ::protobuf::RepeatedField, - prune: ::protobuf::RepeatedField, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a ControlMessage { - fn default() -> &'a ControlMessage { - ::default_instance() - } -} - -impl ControlMessage { - pub fn new() -> ControlMessage { - ::std::default::Default::default() - } - - // repeated .gossipsub.pb.ControlIHave ihave = 1; - - - pub fn get_ihave(&self) -> &[ControlIHave] { - &self.ihave - } - pub fn clear_ihave(&mut self) { - self.ihave.clear(); - } - - // Param is passed by value, moved - pub fn set_ihave(&mut self, v: ::protobuf::RepeatedField) { - self.ihave = v; - } - - // Mutable pointer to the field. - pub fn mut_ihave(&mut self) -> &mut ::protobuf::RepeatedField { - &mut self.ihave - } - - // Take field - pub fn take_ihave(&mut self) -> ::protobuf::RepeatedField { - ::std::mem::replace(&mut self.ihave, ::protobuf::RepeatedField::new()) - } - - // repeated .gossipsub.pb.ControlIWant iwant = 2; - - - pub fn get_iwant(&self) -> &[ControlIWant] { - &self.iwant - } - pub fn clear_iwant(&mut self) { - self.iwant.clear(); - } - - // Param is passed by value, moved - pub fn set_iwant(&mut self, v: ::protobuf::RepeatedField) { - self.iwant = v; - } - - // Mutable pointer to the field. - pub fn mut_iwant(&mut self) -> &mut ::protobuf::RepeatedField { - &mut self.iwant - } - - // Take field - pub fn take_iwant(&mut self) -> ::protobuf::RepeatedField { - ::std::mem::replace(&mut self.iwant, ::protobuf::RepeatedField::new()) - } - - // repeated .gossipsub.pb.ControlGraft graft = 3; - - - pub fn get_graft(&self) -> &[ControlGraft] { - &self.graft - } - pub fn clear_graft(&mut self) { - self.graft.clear(); - } - - // Param is passed by value, moved - pub fn set_graft(&mut self, v: ::protobuf::RepeatedField) { - self.graft = v; - } - - // Mutable pointer to the field. - pub fn mut_graft(&mut self) -> &mut ::protobuf::RepeatedField { - &mut self.graft - } - - // Take field - pub fn take_graft(&mut self) -> ::protobuf::RepeatedField { - ::std::mem::replace(&mut self.graft, ::protobuf::RepeatedField::new()) - } - - // repeated .gossipsub.pb.ControlPrune prune = 4; - - - pub fn get_prune(&self) -> &[ControlPrune] { - &self.prune - } - pub fn clear_prune(&mut self) { - self.prune.clear(); - } - - // Param is passed by value, moved - pub fn set_prune(&mut self, v: ::protobuf::RepeatedField) { - self.prune = v; - } - - // Mutable pointer to the field. - pub fn mut_prune(&mut self) -> &mut ::protobuf::RepeatedField { - &mut self.prune - } - - // Take field - pub fn take_prune(&mut self) -> ::protobuf::RepeatedField { - ::std::mem::replace(&mut self.prune, ::protobuf::RepeatedField::new()) - } -} - -impl ::protobuf::Message for ControlMessage { - fn is_initialized(&self) -> bool { - for v in &self.ihave { - if !v.is_initialized() { - return false; - } - }; - for v in &self.iwant { - if !v.is_initialized() { - return false; - } - }; - for v in &self.graft { - if !v.is_initialized() { - return false; - } - }; - for v in &self.prune { - if !v.is_initialized() { - return false; - } - }; - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.ihave)?; - }, - 2 => { - ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.iwant)?; - }, - 3 => { - ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.graft)?; - }, - 4 => { - ::protobuf::rt::read_repeated_message_into(wire_type, is, &mut self.prune)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - for value in &self.ihave { - let len = value.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - }; - for value in &self.iwant { - let len = value.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - }; - for value in &self.graft { - let len = value.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - }; - for value in &self.prune { - let len = value.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - }; - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - for v in &self.ihave { - os.write_tag(1, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - }; - for v in &self.iwant { - os.write_tag(2, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - }; - for v in &self.graft { - os.write_tag(3, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - }; - for v in &self.prune { - os.write_tag(4, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - }; - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> ControlMessage { - ControlMessage::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "ihave", - |m: &ControlMessage| { &m.ihave }, - |m: &mut ControlMessage| { &mut m.ihave }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "iwant", - |m: &ControlMessage| { &m.iwant }, - |m: &mut ControlMessage| { &mut m.iwant }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "graft", - |m: &ControlMessage| { &m.graft }, - |m: &mut ControlMessage| { &mut m.graft }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "prune", - |m: &ControlMessage| { &m.prune }, - |m: &mut ControlMessage| { &mut m.prune }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "ControlMessage", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static ControlMessage { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ControlMessage, - }; - unsafe { - instance.get(ControlMessage::new) - } - } -} - -impl ::protobuf::Clear for ControlMessage { - fn clear(&mut self) { - self.ihave.clear(); - self.iwant.clear(); - self.graft.clear(); - self.prune.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for ControlMessage { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for ControlMessage { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct ControlIHave { - // message fields - topicID: ::protobuf::SingularField<::std::string::String>, - messageIDs: ::protobuf::RepeatedField<::std::string::String>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a ControlIHave { - fn default() -> &'a ControlIHave { - ::default_instance() - } -} - -impl ControlIHave { - pub fn new() -> ControlIHave { - ::std::default::Default::default() - } - - // optional string topicID = 1; - - - pub fn get_topicID(&self) -> &str { - match self.topicID.as_ref() { - Some(v) => &v, - None => "", - } - } - pub fn clear_topicID(&mut self) { - self.topicID.clear(); - } - - pub fn has_topicID(&self) -> bool { - self.topicID.is_some() - } - - // Param is passed by value, moved - pub fn set_topicID(&mut self, v: ::std::string::String) { - self.topicID = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_topicID(&mut self) -> &mut ::std::string::String { - if self.topicID.is_none() { - self.topicID.set_default(); - } - self.topicID.as_mut().unwrap() - } - - // Take field - pub fn take_topicID(&mut self) -> ::std::string::String { - self.topicID.take().unwrap_or_else(|| ::std::string::String::new()) - } - - // repeated string messageIDs = 2; - - - pub fn get_messageIDs(&self) -> &[::std::string::String] { - &self.messageIDs - } - pub fn clear_messageIDs(&mut self) { - self.messageIDs.clear(); - } - - // Param is passed by value, moved - pub fn set_messageIDs(&mut self, v: ::protobuf::RepeatedField<::std::string::String>) { - self.messageIDs = v; - } - - // Mutable pointer to the field. - pub fn mut_messageIDs(&mut self) -> &mut ::protobuf::RepeatedField<::std::string::String> { - &mut self.messageIDs - } - - // Take field - pub fn take_messageIDs(&mut self) -> ::protobuf::RepeatedField<::std::string::String> { - ::std::mem::replace(&mut self.messageIDs, ::protobuf::RepeatedField::new()) - } -} - -impl ::protobuf::Message for ControlIHave { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicID)?; - }, - 2 => { - ::protobuf::rt::read_repeated_string_into(wire_type, is, &mut self.messageIDs)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(ref v) = self.topicID.as_ref() { - my_size += ::protobuf::rt::string_size(1, &v); - } - for value in &self.messageIDs { - my_size += ::protobuf::rt::string_size(2, &value); - }; - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(ref v) = self.topicID.as_ref() { - os.write_string(1, &v)?; - } - for v in &self.messageIDs { - os.write_string(2, &v)?; - }; - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> ControlIHave { - ControlIHave::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "topicID", - |m: &ControlIHave| { &m.topicID }, - |m: &mut ControlIHave| { &mut m.topicID }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "messageIDs", - |m: &ControlIHave| { &m.messageIDs }, - |m: &mut ControlIHave| { &mut m.messageIDs }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "ControlIHave", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static ControlIHave { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ControlIHave, - }; - unsafe { - instance.get(ControlIHave::new) - } - } -} - -impl ::protobuf::Clear for ControlIHave { - fn clear(&mut self) { - self.topicID.clear(); - self.messageIDs.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for ControlIHave { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for ControlIHave { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct ControlIWant { - // message fields - messageIDs: ::protobuf::RepeatedField<::std::string::String>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a ControlIWant { - fn default() -> &'a ControlIWant { - ::default_instance() - } -} - -impl ControlIWant { - pub fn new() -> ControlIWant { - ::std::default::Default::default() - } - - // repeated string messageIDs = 1; - - - pub fn get_messageIDs(&self) -> &[::std::string::String] { - &self.messageIDs - } - pub fn clear_messageIDs(&mut self) { - self.messageIDs.clear(); - } - - // Param is passed by value, moved - pub fn set_messageIDs(&mut self, v: ::protobuf::RepeatedField<::std::string::String>) { - self.messageIDs = v; - } - - // Mutable pointer to the field. - pub fn mut_messageIDs(&mut self) -> &mut ::protobuf::RepeatedField<::std::string::String> { - &mut self.messageIDs - } - - // Take field - pub fn take_messageIDs(&mut self) -> ::protobuf::RepeatedField<::std::string::String> { - ::std::mem::replace(&mut self.messageIDs, ::protobuf::RepeatedField::new()) - } -} - -impl ::protobuf::Message for ControlIWant { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_repeated_string_into(wire_type, is, &mut self.messageIDs)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - for value in &self.messageIDs { - my_size += ::protobuf::rt::string_size(1, &value); - }; - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - for v in &self.messageIDs { - os.write_string(1, &v)?; - }; - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> ControlIWant { - ControlIWant::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "messageIDs", - |m: &ControlIWant| { &m.messageIDs }, - |m: &mut ControlIWant| { &mut m.messageIDs }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "ControlIWant", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static ControlIWant { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ControlIWant, - }; - unsafe { - instance.get(ControlIWant::new) - } - } -} - -impl ::protobuf::Clear for ControlIWant { - fn clear(&mut self) { - self.messageIDs.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for ControlIWant { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for ControlIWant { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct ControlGraft { - // message fields - topicID: ::protobuf::SingularField<::std::string::String>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a ControlGraft { - fn default() -> &'a ControlGraft { - ::default_instance() - } -} - -impl ControlGraft { - pub fn new() -> ControlGraft { - ::std::default::Default::default() - } - - // optional string topicID = 1; - - - pub fn get_topicID(&self) -> &str { - match self.topicID.as_ref() { - Some(v) => &v, - None => "", - } - } - pub fn clear_topicID(&mut self) { - self.topicID.clear(); - } - - pub fn has_topicID(&self) -> bool { - self.topicID.is_some() - } - - // Param is passed by value, moved - pub fn set_topicID(&mut self, v: ::std::string::String) { - self.topicID = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_topicID(&mut self) -> &mut ::std::string::String { - if self.topicID.is_none() { - self.topicID.set_default(); - } - self.topicID.as_mut().unwrap() - } - - // Take field - pub fn take_topicID(&mut self) -> ::std::string::String { - self.topicID.take().unwrap_or_else(|| ::std::string::String::new()) - } -} - -impl ::protobuf::Message for ControlGraft { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicID)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(ref v) = self.topicID.as_ref() { - my_size += ::protobuf::rt::string_size(1, &v); - } - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(ref v) = self.topicID.as_ref() { - os.write_string(1, &v)?; - } - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> ControlGraft { - ControlGraft::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "topicID", - |m: &ControlGraft| { &m.topicID }, - |m: &mut ControlGraft| { &mut m.topicID }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "ControlGraft", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static ControlGraft { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ControlGraft, - }; - unsafe { - instance.get(ControlGraft::new) - } - } -} - -impl ::protobuf::Clear for ControlGraft { - fn clear(&mut self) { - self.topicID.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for ControlGraft { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for ControlGraft { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct ControlPrune { - // message fields - topicID: ::protobuf::SingularField<::std::string::String>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a ControlPrune { - fn default() -> &'a ControlPrune { - ::default_instance() - } -} - -impl ControlPrune { - pub fn new() -> ControlPrune { - ::std::default::Default::default() - } - - // optional string topicID = 1; - - - pub fn get_topicID(&self) -> &str { - match self.topicID.as_ref() { - Some(v) => &v, - None => "", - } - } - pub fn clear_topicID(&mut self) { - self.topicID.clear(); - } - - pub fn has_topicID(&self) -> bool { - self.topicID.is_some() - } - - // Param is passed by value, moved - pub fn set_topicID(&mut self, v: ::std::string::String) { - self.topicID = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_topicID(&mut self) -> &mut ::std::string::String { - if self.topicID.is_none() { - self.topicID.set_default(); - } - self.topicID.as_mut().unwrap() - } - - // Take field - pub fn take_topicID(&mut self) -> ::std::string::String { - self.topicID.take().unwrap_or_else(|| ::std::string::String::new()) - } -} - -impl ::protobuf::Message for ControlPrune { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.topicID)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(ref v) = self.topicID.as_ref() { - my_size += ::protobuf::rt::string_size(1, &v); - } - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(ref v) = self.topicID.as_ref() { - os.write_string(1, &v)?; - } - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> ControlPrune { - ControlPrune::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "topicID", - |m: &ControlPrune| { &m.topicID }, - |m: &mut ControlPrune| { &mut m.topicID }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "ControlPrune", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static ControlPrune { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ControlPrune, - }; - unsafe { - instance.get(ControlPrune::new) - } - } -} - -impl ::protobuf::Clear for ControlPrune { - fn clear(&mut self) { - self.topicID.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for ControlPrune { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for ControlPrune { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct TopicDescriptor { - // message fields - name: ::protobuf::SingularField<::std::string::String>, - auth: ::protobuf::SingularPtrField, - enc: ::protobuf::SingularPtrField, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a TopicDescriptor { - fn default() -> &'a TopicDescriptor { - ::default_instance() - } -} - -impl TopicDescriptor { - pub fn new() -> TopicDescriptor { - ::std::default::Default::default() - } - - // optional string name = 1; - - - pub fn get_name(&self) -> &str { - match self.name.as_ref() { - Some(v) => &v, - None => "", - } - } - pub fn clear_name(&mut self) { - self.name.clear(); - } - - pub fn has_name(&self) -> bool { - self.name.is_some() - } - - // Param is passed by value, moved - pub fn set_name(&mut self, v: ::std::string::String) { - self.name = ::protobuf::SingularField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_name(&mut self) -> &mut ::std::string::String { - if self.name.is_none() { - self.name.set_default(); - } - self.name.as_mut().unwrap() - } - - // Take field - pub fn take_name(&mut self) -> ::std::string::String { - self.name.take().unwrap_or_else(|| ::std::string::String::new()) - } - - // optional .gossipsub.pb.TopicDescriptor.AuthOpts auth = 2; - - - pub fn get_auth(&self) -> &TopicDescriptor_AuthOpts { - self.auth.as_ref().unwrap_or_else(|| TopicDescriptor_AuthOpts::default_instance()) - } - pub fn clear_auth(&mut self) { - self.auth.clear(); - } - - pub fn has_auth(&self) -> bool { - self.auth.is_some() - } - - // Param is passed by value, moved - pub fn set_auth(&mut self, v: TopicDescriptor_AuthOpts) { - self.auth = ::protobuf::SingularPtrField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_auth(&mut self) -> &mut TopicDescriptor_AuthOpts { - if self.auth.is_none() { - self.auth.set_default(); - } - self.auth.as_mut().unwrap() - } - - // Take field - pub fn take_auth(&mut self) -> TopicDescriptor_AuthOpts { - self.auth.take().unwrap_or_else(|| TopicDescriptor_AuthOpts::new()) - } - - // optional .gossipsub.pb.TopicDescriptor.EncOpts enc = 3; - - - pub fn get_enc(&self) -> &TopicDescriptor_EncOpts { - self.enc.as_ref().unwrap_or_else(|| TopicDescriptor_EncOpts::default_instance()) - } - pub fn clear_enc(&mut self) { - self.enc.clear(); - } - - pub fn has_enc(&self) -> bool { - self.enc.is_some() - } - - // Param is passed by value, moved - pub fn set_enc(&mut self, v: TopicDescriptor_EncOpts) { - self.enc = ::protobuf::SingularPtrField::some(v); - } - - // Mutable pointer to the field. - // If field is not initialized, it is initialized with default value first. - pub fn mut_enc(&mut self) -> &mut TopicDescriptor_EncOpts { - if self.enc.is_none() { - self.enc.set_default(); - } - self.enc.as_mut().unwrap() - } - - // Take field - pub fn take_enc(&mut self) -> TopicDescriptor_EncOpts { - self.enc.take().unwrap_or_else(|| TopicDescriptor_EncOpts::new()) - } -} - -impl ::protobuf::Message for TopicDescriptor { - fn is_initialized(&self) -> bool { - for v in &self.auth { - if !v.is_initialized() { - return false; - } - }; - for v in &self.enc { - if !v.is_initialized() { - return false; - } - }; - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.name)?; - }, - 2 => { - ::protobuf::rt::read_singular_message_into(wire_type, is, &mut self.auth)?; - }, - 3 => { - ::protobuf::rt::read_singular_message_into(wire_type, is, &mut self.enc)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(ref v) = self.name.as_ref() { - my_size += ::protobuf::rt::string_size(1, &v); - } - if let Some(ref v) = self.auth.as_ref() { - let len = v.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - } - if let Some(ref v) = self.enc.as_ref() { - let len = v.compute_size(); - my_size += 1 + ::protobuf::rt::compute_raw_varint32_size(len) + len; - } - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(ref v) = self.name.as_ref() { - os.write_string(1, &v)?; - } - if let Some(ref v) = self.auth.as_ref() { - os.write_tag(2, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - } - if let Some(ref v) = self.enc.as_ref() { - os.write_tag(3, ::protobuf::wire_format::WireTypeLengthDelimited)?; - os.write_raw_varint32(v.get_cached_size())?; - v.write_to_with_cached_sizes(os)?; - } - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> TopicDescriptor { - TopicDescriptor::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>( - "name", - |m: &TopicDescriptor| { &m.name }, - |m: &mut TopicDescriptor| { &mut m.name }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_ptr_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "auth", - |m: &TopicDescriptor| { &m.auth }, - |m: &mut TopicDescriptor| { &mut m.auth }, - )); - fields.push(::protobuf::reflect::accessor::make_singular_ptr_field_accessor::<_, ::protobuf::types::ProtobufTypeMessage>( - "enc", - |m: &TopicDescriptor| { &m.enc }, - |m: &mut TopicDescriptor| { &mut m.enc }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "TopicDescriptor", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static TopicDescriptor { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const TopicDescriptor, - }; - unsafe { - instance.get(TopicDescriptor::new) - } - } -} - -impl ::protobuf::Clear for TopicDescriptor { - fn clear(&mut self) { - self.name.clear(); - self.auth.clear(); - self.enc.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for TopicDescriptor { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for TopicDescriptor { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct TopicDescriptor_AuthOpts { - // message fields - mode: ::std::option::Option, - keys: ::protobuf::RepeatedField<::std::vec::Vec>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a TopicDescriptor_AuthOpts { - fn default() -> &'a TopicDescriptor_AuthOpts { - ::default_instance() - } -} - -impl TopicDescriptor_AuthOpts { - pub fn new() -> TopicDescriptor_AuthOpts { - ::std::default::Default::default() - } - - // optional .gossipsub.pb.TopicDescriptor.AuthOpts.AuthMode mode = 1; - - - pub fn get_mode(&self) -> TopicDescriptor_AuthOpts_AuthMode { - self.mode.unwrap_or(TopicDescriptor_AuthOpts_AuthMode::NONE) - } - pub fn clear_mode(&mut self) { - self.mode = ::std::option::Option::None; - } - - pub fn has_mode(&self) -> bool { - self.mode.is_some() - } - - // Param is passed by value, moved - pub fn set_mode(&mut self, v: TopicDescriptor_AuthOpts_AuthMode) { - self.mode = ::std::option::Option::Some(v); - } - - // repeated bytes keys = 2; - - - pub fn get_keys(&self) -> &[::std::vec::Vec] { - &self.keys - } - pub fn clear_keys(&mut self) { - self.keys.clear(); - } - - // Param is passed by value, moved - pub fn set_keys(&mut self, v: ::protobuf::RepeatedField<::std::vec::Vec>) { - self.keys = v; - } - - // Mutable pointer to the field. - pub fn mut_keys(&mut self) -> &mut ::protobuf::RepeatedField<::std::vec::Vec> { - &mut self.keys - } - - // Take field - pub fn take_keys(&mut self) -> ::protobuf::RepeatedField<::std::vec::Vec> { - ::std::mem::replace(&mut self.keys, ::protobuf::RepeatedField::new()) - } -} - -impl ::protobuf::Message for TopicDescriptor_AuthOpts { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_proto2_enum_with_unknown_fields_into(wire_type, is, &mut self.mode, 1, &mut self.unknown_fields)? - }, - 2 => { - ::protobuf::rt::read_repeated_bytes_into(wire_type, is, &mut self.keys)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(v) = self.mode { - my_size += ::protobuf::rt::enum_size(1, v); - } - for value in &self.keys { - my_size += ::protobuf::rt::bytes_size(2, &value); - }; - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(v) = self.mode { - os.write_enum(1, v.value())?; - } - for v in &self.keys { - os.write_bytes(2, &v)?; - }; - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> TopicDescriptor_AuthOpts { - TopicDescriptor_AuthOpts::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeEnum>( - "mode", - |m: &TopicDescriptor_AuthOpts| { &m.mode }, - |m: &mut TopicDescriptor_AuthOpts| { &mut m.mode }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "keys", - |m: &TopicDescriptor_AuthOpts| { &m.keys }, - |m: &mut TopicDescriptor_AuthOpts| { &mut m.keys }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "TopicDescriptor_AuthOpts", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static TopicDescriptor_AuthOpts { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const TopicDescriptor_AuthOpts, - }; - unsafe { - instance.get(TopicDescriptor_AuthOpts::new) - } - } -} - -impl ::protobuf::Clear for TopicDescriptor_AuthOpts { - fn clear(&mut self) { - self.mode = ::std::option::Option::None; - self.keys.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for TopicDescriptor_AuthOpts { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_AuthOpts { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(Clone,PartialEq,Eq,Debug,Hash)] -pub enum TopicDescriptor_AuthOpts_AuthMode { - NONE = 0, - KEY = 1, - WOT = 2, -} - -impl ::protobuf::ProtobufEnum for TopicDescriptor_AuthOpts_AuthMode { - fn value(&self) -> i32 { - *self as i32 - } - - fn from_i32(value: i32) -> ::std::option::Option { - match value { - 0 => ::std::option::Option::Some(TopicDescriptor_AuthOpts_AuthMode::NONE), - 1 => ::std::option::Option::Some(TopicDescriptor_AuthOpts_AuthMode::KEY), - 2 => ::std::option::Option::Some(TopicDescriptor_AuthOpts_AuthMode::WOT), - _ => ::std::option::Option::None - } - } - - fn values() -> &'static [Self] { - static values: &'static [TopicDescriptor_AuthOpts_AuthMode] = &[ - TopicDescriptor_AuthOpts_AuthMode::NONE, - TopicDescriptor_AuthOpts_AuthMode::KEY, - TopicDescriptor_AuthOpts_AuthMode::WOT, - ]; - values - } - - fn enum_descriptor_static() -> &'static ::protobuf::reflect::EnumDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::EnumDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::EnumDescriptor, - }; - unsafe { - descriptor.get(|| { - ::protobuf::reflect::EnumDescriptor::new("TopicDescriptor_AuthOpts_AuthMode", file_descriptor_proto()) - }) - } - } -} - -impl ::std::marker::Copy for TopicDescriptor_AuthOpts_AuthMode { -} - -impl ::std::default::Default for TopicDescriptor_AuthOpts_AuthMode { - fn default() -> Self { - TopicDescriptor_AuthOpts_AuthMode::NONE - } -} - -impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_AuthOpts_AuthMode { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Enum(self.descriptor()) - } -} - -#[derive(PartialEq,Clone,Default)] -pub struct TopicDescriptor_EncOpts { - // message fields - mode: ::std::option::Option, - keyHashes: ::protobuf::RepeatedField<::std::vec::Vec>, - // special fields - pub unknown_fields: ::protobuf::UnknownFields, - pub cached_size: ::protobuf::CachedSize, -} - -impl<'a> ::std::default::Default for &'a TopicDescriptor_EncOpts { - fn default() -> &'a TopicDescriptor_EncOpts { - ::default_instance() - } -} - -impl TopicDescriptor_EncOpts { - pub fn new() -> TopicDescriptor_EncOpts { - ::std::default::Default::default() - } - - // optional .gossipsub.pb.TopicDescriptor.EncOpts.EncMode mode = 1; - - - pub fn get_mode(&self) -> TopicDescriptor_EncOpts_EncMode { - self.mode.unwrap_or(TopicDescriptor_EncOpts_EncMode::NONE) - } - pub fn clear_mode(&mut self) { - self.mode = ::std::option::Option::None; - } - - pub fn has_mode(&self) -> bool { - self.mode.is_some() - } - - // Param is passed by value, moved - pub fn set_mode(&mut self, v: TopicDescriptor_EncOpts_EncMode) { - self.mode = ::std::option::Option::Some(v); - } - - // repeated bytes keyHashes = 2; - - - pub fn get_keyHashes(&self) -> &[::std::vec::Vec] { - &self.keyHashes - } - pub fn clear_keyHashes(&mut self) { - self.keyHashes.clear(); - } - - // Param is passed by value, moved - pub fn set_keyHashes(&mut self, v: ::protobuf::RepeatedField<::std::vec::Vec>) { - self.keyHashes = v; - } - - // Mutable pointer to the field. - pub fn mut_keyHashes(&mut self) -> &mut ::protobuf::RepeatedField<::std::vec::Vec> { - &mut self.keyHashes - } - - // Take field - pub fn take_keyHashes(&mut self) -> ::protobuf::RepeatedField<::std::vec::Vec> { - ::std::mem::replace(&mut self.keyHashes, ::protobuf::RepeatedField::new()) - } -} - -impl ::protobuf::Message for TopicDescriptor_EncOpts { - fn is_initialized(&self) -> bool { - true - } - - fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream<'_>) -> ::protobuf::ProtobufResult<()> { - while !is.eof()? { - let (field_number, wire_type) = is.read_tag_unpack()?; - match field_number { - 1 => { - ::protobuf::rt::read_proto2_enum_with_unknown_fields_into(wire_type, is, &mut self.mode, 1, &mut self.unknown_fields)? - }, - 2 => { - ::protobuf::rt::read_repeated_bytes_into(wire_type, is, &mut self.keyHashes)?; - }, - _ => { - ::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?; - }, - }; - } - ::std::result::Result::Ok(()) - } - - // Compute sizes of nested messages - #[allow(unused_variables)] - fn compute_size(&self) -> u32 { - let mut my_size = 0; - if let Some(v) = self.mode { - my_size += ::protobuf::rt::enum_size(1, v); - } - for value in &self.keyHashes { - my_size += ::protobuf::rt::bytes_size(2, &value); - }; - my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields()); - self.cached_size.set(my_size); - my_size - } - - fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream<'_>) -> ::protobuf::ProtobufResult<()> { - if let Some(v) = self.mode { - os.write_enum(1, v.value())?; - } - for v in &self.keyHashes { - os.write_bytes(2, &v)?; - }; - os.write_unknown_fields(self.get_unknown_fields())?; - ::std::result::Result::Ok(()) - } - - fn get_cached_size(&self) -> u32 { - self.cached_size.get() - } - - fn get_unknown_fields(&self) -> &::protobuf::UnknownFields { - &self.unknown_fields - } - - fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields { - &mut self.unknown_fields - } - - fn as_any(&self) -> &dyn (::std::any::Any) { - self as &dyn (::std::any::Any) - } - fn as_any_mut(&mut self) -> &mut dyn (::std::any::Any) { - self as &mut dyn (::std::any::Any) - } - fn into_any(self: Box) -> ::std::boxed::Box { - self - } - - fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { - Self::descriptor_static() - } - - fn new() -> TopicDescriptor_EncOpts { - TopicDescriptor_EncOpts::new() - } - - fn descriptor_static() -> &'static ::protobuf::reflect::MessageDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::MessageDescriptor, - }; - unsafe { - descriptor.get(|| { - let mut fields = ::std::vec::Vec::new(); - fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeEnum>( - "mode", - |m: &TopicDescriptor_EncOpts| { &m.mode }, - |m: &mut TopicDescriptor_EncOpts| { &mut m.mode }, - )); - fields.push(::protobuf::reflect::accessor::make_repeated_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>( - "keyHashes", - |m: &TopicDescriptor_EncOpts| { &m.keyHashes }, - |m: &mut TopicDescriptor_EncOpts| { &mut m.keyHashes }, - )); - ::protobuf::reflect::MessageDescriptor::new::( - "TopicDescriptor_EncOpts", - fields, - file_descriptor_proto() - ) - }) - } - } - - fn default_instance() -> &'static TopicDescriptor_EncOpts { - static mut instance: ::protobuf::lazy::Lazy = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const TopicDescriptor_EncOpts, - }; - unsafe { - instance.get(TopicDescriptor_EncOpts::new) - } - } -} - -impl ::protobuf::Clear for TopicDescriptor_EncOpts { - fn clear(&mut self) { - self.mode = ::std::option::Option::None; - self.keyHashes.clear(); - self.unknown_fields.clear(); - } -} - -impl ::std::fmt::Debug for TopicDescriptor_EncOpts { - fn fmt(&self, f: &mut ::std::fmt::Formatter<'_>) -> ::std::fmt::Result { - ::protobuf::text_format::fmt(self, f) - } -} - -impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_EncOpts { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Message(self) - } -} - -#[derive(Clone,PartialEq,Eq,Debug,Hash)] -pub enum TopicDescriptor_EncOpts_EncMode { - NONE = 0, - SHAREDKEY = 1, - WOT = 2, -} - -impl ::protobuf::ProtobufEnum for TopicDescriptor_EncOpts_EncMode { - fn value(&self) -> i32 { - *self as i32 - } - - fn from_i32(value: i32) -> ::std::option::Option { - match value { - 0 => ::std::option::Option::Some(TopicDescriptor_EncOpts_EncMode::NONE), - 1 => ::std::option::Option::Some(TopicDescriptor_EncOpts_EncMode::SHAREDKEY), - 2 => ::std::option::Option::Some(TopicDescriptor_EncOpts_EncMode::WOT), - _ => ::std::option::Option::None - } - } - - fn values() -> &'static [Self] { - static values: &'static [TopicDescriptor_EncOpts_EncMode] = &[ - TopicDescriptor_EncOpts_EncMode::NONE, - TopicDescriptor_EncOpts_EncMode::SHAREDKEY, - TopicDescriptor_EncOpts_EncMode::WOT, - ]; - values - } - - fn enum_descriptor_static() -> &'static ::protobuf::reflect::EnumDescriptor { - static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::EnumDescriptor> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::reflect::EnumDescriptor, - }; - unsafe { - descriptor.get(|| { - ::protobuf::reflect::EnumDescriptor::new("TopicDescriptor_EncOpts_EncMode", file_descriptor_proto()) - }) - } - } -} - -impl ::std::marker::Copy for TopicDescriptor_EncOpts_EncMode { -} - -impl ::std::default::Default for TopicDescriptor_EncOpts_EncMode { - fn default() -> Self { - TopicDescriptor_EncOpts_EncMode::NONE - } -} - -impl ::protobuf::reflect::ProtobufValue for TopicDescriptor_EncOpts_EncMode { - fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef { - ::protobuf::reflect::ProtobufValueRef::Enum(self.descriptor()) - } -} - -static file_descriptor_proto_data: &'static [u8] = b"\ - \n\trpc.proto\x12\x0cgossipsub.pb\"\xf2\x01\n\x03RPC\x12?\n\rsubscriptio\ - ns\x18\x01\x20\x03(\x0b2\x19.gossipsub.pb.RPC.SubOptsR\rsubscriptions\ - \x12/\n\x07publish\x18\x02\x20\x03(\x0b2\x15.gossipsub.pb.MessageR\x07pu\ - blish\x126\n\x07control\x18\x03\x20\x01(\x0b2\x1c.gossipsub.pb.ControlMe\ - ssageR\x07control\x1aA\n\x07SubOpts\x12\x1c\n\tsubscribe\x18\x01\x20\x01\ - (\x08R\tsubscribe\x12\x18\n\x07topicid\x18\x02\x20\x01(\tR\x07topicid\"c\ - \n\x07Message\x12\x12\n\x04from\x18\x01\x20\x01(\x0cR\x04from\x12\x12\n\ - \x04data\x18\x02\x20\x01(\x0cR\x04data\x12\x14\n\x05seqno\x18\x03\x20\ - \x01(\x0cR\x05seqno\x12\x1a\n\x08topicIDs\x18\x04\x20\x03(\tR\x08topicID\ - s\"\xd8\x01\n\x0eControlMessage\x120\n\x05ihave\x18\x01\x20\x03(\x0b2\ - \x1a.gossipsub.pb.ControlIHaveR\x05ihave\x120\n\x05iwant\x18\x02\x20\x03\ - (\x0b2\x1a.gossipsub.pb.ControlIWantR\x05iwant\x120\n\x05graft\x18\x03\ - \x20\x03(\x0b2\x1a.gossipsub.pb.ControlGraftR\x05graft\x120\n\x05prune\ - \x18\x04\x20\x03(\x0b2\x1a.gossipsub.pb.ControlPruneR\x05prune\"H\n\x0cC\ - ontrolIHave\x12\x18\n\x07topicID\x18\x01\x20\x01(\tR\x07topicID\x12\x1e\ - \n\nmessageIDs\x18\x02\x20\x03(\tR\nmessageIDs\".\n\x0cControlIWant\x12\ - \x1e\n\nmessageIDs\x18\x01\x20\x03(\tR\nmessageIDs\"(\n\x0cControlGraft\ - \x12\x18\n\x07topicID\x18\x01\x20\x01(\tR\x07topicID\"(\n\x0cControlPrun\ - e\x12\x18\n\x07topicID\x18\x01\x20\x01(\tR\x07topicID\"\xc2\x03\n\x0fTop\ - icDescriptor\x12\x12\n\x04name\x18\x01\x20\x01(\tR\x04name\x12:\n\x04aut\ - h\x18\x02\x20\x01(\x0b2&.gossipsub.pb.TopicDescriptor.AuthOptsR\x04auth\ - \x127\n\x03enc\x18\x03\x20\x01(\x0b2%.gossipsub.pb.TopicDescriptor.EncOp\ - tsR\x03enc\x1a\x8b\x01\n\x08AuthOpts\x12C\n\x04mode\x18\x01\x20\x01(\x0e\ - 2/.gossipsub.pb.TopicDescriptor.AuthOpts.AuthModeR\x04mode\x12\x12\n\x04\ - keys\x18\x02\x20\x03(\x0cR\x04keys\"&\n\x08AuthMode\x12\x08\n\x04NONE\ - \x10\0\x12\x07\n\x03KEY\x10\x01\x12\x07\n\x03WOT\x10\x02\x1a\x97\x01\n\ - \x07EncOpts\x12A\n\x04mode\x18\x01\x20\x01(\x0e2-.gossipsub.pb.TopicDesc\ - riptor.EncOpts.EncModeR\x04mode\x12\x1c\n\tkeyHashes\x18\x02\x20\x03(\ - \x0cR\tkeyHashes\"+\n\x07EncMode\x12\x08\n\x04NONE\x10\0\x12\r\n\tSHARED\ - KEY\x10\x01\x12\x07\n\x03WOT\x10\x02J\xf7\x16\n\x06\x12\x04\0\0J\x01\n\ - \x08\n\x01\x0c\x12\x03\0\0\x12\n\x08\n\x01\x02\x12\x03\x02\x08\x14\n\n\n\ - \x02\x04\0\x12\x04\x04\0\x0e\x01\n\n\n\x03\x04\0\x01\x12\x03\x04\x08\x0b\ - \n\x0b\n\x04\x04\0\x02\0\x12\x03\x05\x08+\n\x0c\n\x05\x04\0\x02\0\x04\ - \x12\x03\x05\x08\x10\n\x0c\n\x05\x04\0\x02\0\x06\x12\x03\x05\x11\x18\n\ - \x0c\n\x05\x04\0\x02\0\x01\x12\x03\x05\x19&\n\x0c\n\x05\x04\0\x02\0\x03\ - \x12\x03\x05)*\n\x0b\n\x04\x04\0\x02\x01\x12\x03\x06\x08%\n\x0c\n\x05\ - \x04\0\x02\x01\x04\x12\x03\x06\x08\x10\n\x0c\n\x05\x04\0\x02\x01\x06\x12\ - \x03\x06\x11\x18\n\x0c\n\x05\x04\0\x02\x01\x01\x12\x03\x06\x19\x20\n\x0c\ - \n\x05\x04\0\x02\x01\x03\x12\x03\x06#$\n\x0c\n\x04\x04\0\x03\0\x12\x04\ - \x08\x08\x0b\t\n\x0c\n\x05\x04\0\x03\0\x01\x12\x03\x08\x10\x17\n)\n\x06\ - \x04\0\x03\0\x02\0\x12\x03\t\x10,\"\x1a\x20subscribe\x20or\x20unsubscrib\ - e\n\n\x0e\n\x07\x04\0\x03\0\x02\0\x04\x12\x03\t\x10\x18\n\x0e\n\x07\x04\ - \0\x03\0\x02\0\x05\x12\x03\t\x19\x1d\n\x0e\n\x07\x04\0\x03\0\x02\0\x01\ - \x12\x03\t\x1e'\n\x0e\n\x07\x04\0\x03\0\x02\0\x03\x12\x03\t*+\n\r\n\x06\ - \x04\0\x03\0\x02\x01\x12\x03\n\x10,\n\x0e\n\x07\x04\0\x03\0\x02\x01\x04\ - \x12\x03\n\x10\x18\n\x0e\n\x07\x04\0\x03\0\x02\x01\x05\x12\x03\n\x19\x1f\ - \n\x0e\n\x07\x04\0\x03\0\x02\x01\x01\x12\x03\n\x20'\n\x0e\n\x07\x04\0\ - \x03\0\x02\x01\x03\x12\x03\n*+\n\x0b\n\x04\x04\0\x02\x02\x12\x03\r\x08,\ - \n\x0c\n\x05\x04\0\x02\x02\x04\x12\x03\r\x08\x10\n\x0c\n\x05\x04\0\x02\ - \x02\x06\x12\x03\r\x11\x1f\n\x0c\n\x05\x04\0\x02\x02\x01\x12\x03\r\x20'\ - \n\x0c\n\x05\x04\0\x02\x02\x03\x12\x03\r*+\n\n\n\x02\x04\x01\x12\x04\x10\ - \0\x15\x01\n\n\n\x03\x04\x01\x01\x12\x03\x10\x08\x0f\n\x0b\n\x04\x04\x01\ - \x02\0\x12\x03\x11\x08\x20\n\x0c\n\x05\x04\x01\x02\0\x04\x12\x03\x11\x08\ - \x10\n\x0c\n\x05\x04\x01\x02\0\x05\x12\x03\x11\x11\x16\n\x0c\n\x05\x04\ - \x01\x02\0\x01\x12\x03\x11\x17\x1b\n\x0c\n\x05\x04\x01\x02\0\x03\x12\x03\ - \x11\x1e\x1f\n\x0b\n\x04\x04\x01\x02\x01\x12\x03\x12\x08\x20\n\x0c\n\x05\ - \x04\x01\x02\x01\x04\x12\x03\x12\x08\x10\n\x0c\n\x05\x04\x01\x02\x01\x05\ - \x12\x03\x12\x11\x16\n\x0c\n\x05\x04\x01\x02\x01\x01\x12\x03\x12\x17\x1b\ - \n\x0c\n\x05\x04\x01\x02\x01\x03\x12\x03\x12\x1e\x1f\n\x0b\n\x04\x04\x01\ - \x02\x02\x12\x03\x13\x08!\n\x0c\n\x05\x04\x01\x02\x02\x04\x12\x03\x13\ - \x08\x10\n\x0c\n\x05\x04\x01\x02\x02\x05\x12\x03\x13\x11\x16\n\x0c\n\x05\ - \x04\x01\x02\x02\x01\x12\x03\x13\x17\x1c\n\x0c\n\x05\x04\x01\x02\x02\x03\ - \x12\x03\x13\x1f\x20\n\x0b\n\x04\x04\x01\x02\x03\x12\x03\x14\x08%\n\x0c\ - \n\x05\x04\x01\x02\x03\x04\x12\x03\x14\x08\x10\n\x0c\n\x05\x04\x01\x02\ - \x03\x05\x12\x03\x14\x11\x17\n\x0c\n\x05\x04\x01\x02\x03\x01\x12\x03\x14\ - \x18\x20\n\x0c\n\x05\x04\x01\x02\x03\x03\x12\x03\x14#$\n\n\n\x02\x04\x02\ - \x12\x04\x17\0\x1c\x01\n\n\n\x03\x04\x02\x01\x12\x03\x17\x08\x16\n\x0b\n\ - \x04\x04\x02\x02\0\x12\x03\x18\x08(\n\x0c\n\x05\x04\x02\x02\0\x04\x12\ - \x03\x18\x08\x10\n\x0c\n\x05\x04\x02\x02\0\x06\x12\x03\x18\x11\x1d\n\x0c\ - \n\x05\x04\x02\x02\0\x01\x12\x03\x18\x1e#\n\x0c\n\x05\x04\x02\x02\0\x03\ - \x12\x03\x18&'\n\x0b\n\x04\x04\x02\x02\x01\x12\x03\x19\x08(\n\x0c\n\x05\ - \x04\x02\x02\x01\x04\x12\x03\x19\x08\x10\n\x0c\n\x05\x04\x02\x02\x01\x06\ - \x12\x03\x19\x11\x1d\n\x0c\n\x05\x04\x02\x02\x01\x01\x12\x03\x19\x1e#\n\ - \x0c\n\x05\x04\x02\x02\x01\x03\x12\x03\x19&'\n\x0b\n\x04\x04\x02\x02\x02\ - \x12\x03\x1a\x08(\n\x0c\n\x05\x04\x02\x02\x02\x04\x12\x03\x1a\x08\x10\n\ - \x0c\n\x05\x04\x02\x02\x02\x06\x12\x03\x1a\x11\x1d\n\x0c\n\x05\x04\x02\ - \x02\x02\x01\x12\x03\x1a\x1e#\n\x0c\n\x05\x04\x02\x02\x02\x03\x12\x03\ - \x1a&'\n\x0b\n\x04\x04\x02\x02\x03\x12\x03\x1b\x08(\n\x0c\n\x05\x04\x02\ - \x02\x03\x04\x12\x03\x1b\x08\x10\n\x0c\n\x05\x04\x02\x02\x03\x06\x12\x03\ - \x1b\x11\x1d\n\x0c\n\x05\x04\x02\x02\x03\x01\x12\x03\x1b\x1e#\n\x0c\n\ - \x05\x04\x02\x02\x03\x03\x12\x03\x1b&'\n\n\n\x02\x04\x03\x12\x04\x1e\0!\ - \x01\n\n\n\x03\x04\x03\x01\x12\x03\x1e\x08\x14\n\x0b\n\x04\x04\x03\x02\0\ - \x12\x03\x1f\x08$\n\x0c\n\x05\x04\x03\x02\0\x04\x12\x03\x1f\x08\x10\n\ - \x0c\n\x05\x04\x03\x02\0\x05\x12\x03\x1f\x11\x17\n\x0c\n\x05\x04\x03\x02\ - \0\x01\x12\x03\x1f\x18\x1f\n\x0c\n\x05\x04\x03\x02\0\x03\x12\x03\x1f\"#\ - \n\x0b\n\x04\x04\x03\x02\x01\x12\x03\x20\x08'\n\x0c\n\x05\x04\x03\x02\ - \x01\x04\x12\x03\x20\x08\x10\n\x0c\n\x05\x04\x03\x02\x01\x05\x12\x03\x20\ - \x11\x17\n\x0c\n\x05\x04\x03\x02\x01\x01\x12\x03\x20\x18\"\n\x0c\n\x05\ - \x04\x03\x02\x01\x03\x12\x03\x20%&\n\n\n\x02\x04\x04\x12\x04#\0%\x01\n\n\ - \n\x03\x04\x04\x01\x12\x03#\x08\x14\n\x0b\n\x04\x04\x04\x02\0\x12\x03$\ - \x08'\n\x0c\n\x05\x04\x04\x02\0\x04\x12\x03$\x08\x10\n\x0c\n\x05\x04\x04\ - \x02\0\x05\x12\x03$\x11\x17\n\x0c\n\x05\x04\x04\x02\0\x01\x12\x03$\x18\"\ - \n\x0c\n\x05\x04\x04\x02\0\x03\x12\x03$%&\n\n\n\x02\x04\x05\x12\x04'\0)\ - \x01\n\n\n\x03\x04\x05\x01\x12\x03'\x08\x14\n\x0b\n\x04\x04\x05\x02\0\ - \x12\x03(\x08$\n\x0c\n\x05\x04\x05\x02\0\x04\x12\x03(\x08\x10\n\x0c\n\ - \x05\x04\x05\x02\0\x05\x12\x03(\x11\x17\n\x0c\n\x05\x04\x05\x02\0\x01\ - \x12\x03(\x18\x1f\n\x0c\n\x05\x04\x05\x02\0\x03\x12\x03(\"#\n\n\n\x02\ - \x04\x06\x12\x04+\0-\x01\n\n\n\x03\x04\x06\x01\x12\x03+\x08\x14\n\x0b\n\ - \x04\x04\x06\x02\0\x12\x03,\x08$\n\x0c\n\x05\x04\x06\x02\0\x04\x12\x03,\ - \x08\x10\n\x0c\n\x05\x04\x06\x02\0\x05\x12\x03,\x11\x17\n\x0c\n\x05\x04\ - \x06\x02\0\x01\x12\x03,\x18\x1f\n\x0c\n\x05\x04\x06\x02\0\x03\x12\x03,\"\ - #\nC\n\x02\x04\x07\x12\x040\0J\x01\x1a7\x20topicID\x20=\x20hash(topicDes\ - criptor);\x20(not\x20the\x20topic.name)\n\n\n\n\x03\x04\x07\x01\x12\x030\ - \x08\x17\n\x0b\n\x04\x04\x07\x02\0\x12\x031\x08!\n\x0c\n\x05\x04\x07\x02\ - \0\x04\x12\x031\x08\x10\n\x0c\n\x05\x04\x07\x02\0\x05\x12\x031\x11\x17\n\ - \x0c\n\x05\x04\x07\x02\0\x01\x12\x031\x18\x1c\n\x0c\n\x05\x04\x07\x02\0\ - \x03\x12\x031\x1f\x20\n\x0b\n\x04\x04\x07\x02\x01\x12\x032\x08#\n\x0c\n\ - \x05\x04\x07\x02\x01\x04\x12\x032\x08\x10\n\x0c\n\x05\x04\x07\x02\x01\ - \x06\x12\x032\x11\x19\n\x0c\n\x05\x04\x07\x02\x01\x01\x12\x032\x1a\x1e\n\ - \x0c\n\x05\x04\x07\x02\x01\x03\x12\x032!\"\n\x0b\n\x04\x04\x07\x02\x02\ - \x12\x033\x08!\n\x0c\n\x05\x04\x07\x02\x02\x04\x12\x033\x08\x10\n\x0c\n\ - \x05\x04\x07\x02\x02\x06\x12\x033\x11\x18\n\x0c\n\x05\x04\x07\x02\x02\ - \x01\x12\x033\x19\x1c\n\x0c\n\x05\x04\x07\x02\x02\x03\x12\x033\x1f\x20\n\ - \x0c\n\x04\x04\x07\x03\0\x12\x045\x08>\t\n\x0c\n\x05\x04\x07\x03\0\x01\ - \x12\x035\x10\x18\n\r\n\x06\x04\x07\x03\0\x02\0\x12\x036\x10+\n\x0e\n\ - \x07\x04\x07\x03\0\x02\0\x04\x12\x036\x10\x18\n\x0e\n\x07\x04\x07\x03\0\ - \x02\0\x06\x12\x036\x19!\n\x0e\n\x07\x04\x07\x03\0\x02\0\x01\x12\x036\"&\ - \n\x0e\n\x07\x04\x07\x03\0\x02\0\x03\x12\x036)*\n#\n\x06\x04\x07\x03\0\ - \x02\x01\x12\x037\x10(\"\x14\x20root\x20keys\x20to\x20trust\n\n\x0e\n\ - \x07\x04\x07\x03\0\x02\x01\x04\x12\x037\x10\x18\n\x0e\n\x07\x04\x07\x03\ - \0\x02\x01\x05\x12\x037\x19\x1e\n\x0e\n\x07\x04\x07\x03\0\x02\x01\x01\ - \x12\x037\x1f#\n\x0e\n\x07\x04\x07\x03\0\x02\x01\x03\x12\x037&'\n\x0e\n\ - \x06\x04\x07\x03\0\x04\0\x12\x049\x10=\x11\n\x0e\n\x07\x04\x07\x03\0\x04\ - \0\x01\x12\x039\x15\x1d\n8\n\x08\x04\x07\x03\0\x04\0\x02\0\x12\x03:\x18!\ - \"'\x20no\x20authentication,\x20anyone\x20can\x20publish\n\n\x10\n\t\x04\ - \x07\x03\0\x04\0\x02\0\x01\x12\x03:\x18\x1c\n\x10\n\t\x04\x07\x03\0\x04\ - \0\x02\0\x02\x12\x03:\x1f\x20\nT\n\x08\x04\x07\x03\0\x04\0\x02\x01\x12\ - \x03;\x18\x20\"C\x20only\x20messages\x20signed\x20by\x20keys\x20in\x20th\ - e\x20topic\x20descriptor\x20are\x20accepted\n\n\x10\n\t\x04\x07\x03\0\ - \x04\0\x02\x01\x01\x12\x03;\x18\x1b\n\x10\n\t\x04\x07\x03\0\x04\0\x02\ - \x01\x02\x12\x03;\x1e\x1f\nM\n\x08\x04\x07\x03\0\x04\0\x02\x02\x12\x03<\ - \x18\x20\"<\x20web\x20of\x20trust,\x20certificates\x20can\x20allow\x20pu\ - blisher\x20set\x20to\x20grow\n\n\x10\n\t\x04\x07\x03\0\x04\0\x02\x02\x01\ - \x12\x03<\x18\x1b\n\x10\n\t\x04\x07\x03\0\x04\0\x02\x02\x02\x12\x03<\x1e\ - \x1f\n\x0c\n\x04\x04\x07\x03\x01\x12\x04@\x08I\t\n\x0c\n\x05\x04\x07\x03\ - \x01\x01\x12\x03@\x10\x17\n\r\n\x06\x04\x07\x03\x01\x02\0\x12\x03A\x10*\ - \n\x0e\n\x07\x04\x07\x03\x01\x02\0\x04\x12\x03A\x10\x18\n\x0e\n\x07\x04\ - \x07\x03\x01\x02\0\x06\x12\x03A\x19\x20\n\x0e\n\x07\x04\x07\x03\x01\x02\ - \0\x01\x12\x03A!%\n\x0e\n\x07\x04\x07\x03\x01\x02\0\x03\x12\x03A()\n<\n\ - \x06\x04\x07\x03\x01\x02\x01\x12\x03B\x10-\"-\x20the\x20hashes\x20of\x20\ - the\x20shared\x20keys\x20used\x20(salted)\n\n\x0e\n\x07\x04\x07\x03\x01\ - \x02\x01\x04\x12\x03B\x10\x18\n\x0e\n\x07\x04\x07\x03\x01\x02\x01\x05\ - \x12\x03B\x19\x1e\n\x0e\n\x07\x04\x07\x03\x01\x02\x01\x01\x12\x03B\x1f(\ - \n\x0e\n\x07\x04\x07\x03\x01\x02\x01\x03\x12\x03B+,\n\x0e\n\x06\x04\x07\ - \x03\x01\x04\0\x12\x04D\x10H\x11\n\x0e\n\x07\x04\x07\x03\x01\x04\0\x01\ - \x12\x03D\x15\x1c\n1\n\x08\x04\x07\x03\x01\x04\0\x02\0\x12\x03E\x18!\"\ - \x20\x20no\x20encryption,\x20anyone\x20can\x20read\n\n\x10\n\t\x04\x07\ - \x03\x01\x04\0\x02\0\x01\x12\x03E\x18\x1c\n\x10\n\t\x04\x07\x03\x01\x04\ - \0\x02\0\x02\x12\x03E\x1f\x20\n9\n\x08\x04\x07\x03\x01\x04\0\x02\x01\x12\ - \x03F\x18&\"(\x20messages\x20are\x20encrypted\x20with\x20shared\x20key\n\ - \n\x10\n\t\x04\x07\x03\x01\x04\0\x02\x01\x01\x12\x03F\x18!\n\x10\n\t\x04\ - \x07\x03\x01\x04\0\x02\x01\x02\x12\x03F$%\nM\n\x08\x04\x07\x03\x01\x04\0\ - \x02\x02\x12\x03G\x18\x20\"<\x20web\x20of\x20trust,\x20certificates\x20c\ - an\x20allow\x20publisher\x20set\x20to\x20grow\n\n\x10\n\t\x04\x07\x03\ - \x01\x04\0\x02\x02\x01\x12\x03G\x18\x1b\n\x10\n\t\x04\x07\x03\x01\x04\0\ - \x02\x02\x02\x12\x03G\x1e\x1f\ -"; - -static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy { - lock: ::protobuf::lazy::ONCE_INIT, - ptr: 0 as *const ::protobuf::descriptor::FileDescriptorProto, -}; - -fn parse_descriptor_proto() -> ::protobuf::descriptor::FileDescriptorProto { - ::protobuf::parse_from_bytes(file_descriptor_proto_data).unwrap() -} - -pub fn file_descriptor_proto() -> &'static ::protobuf::descriptor::FileDescriptorProto { - unsafe { - file_descriptor_proto_lazy.get(|| { - parse_descriptor_proto() - }) - } -} diff --git a/protocols/gossipsub/src/topic.rs b/protocols/gossipsub/src/topic.rs index 7db5a9dd588..6eacb9b3265 100644 --- a/protocols/gossipsub/src/topic.rs +++ b/protocols/gossipsub/src/topic.rs @@ -1,4 +1,4 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. +// 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"), @@ -20,7 +20,7 @@ use crate::rpc_proto; use base64::encode; -use protobuf::Message; +use prost::Message; use sha2::{Digest, Sha256}; use std::fmt; @@ -58,16 +58,16 @@ impl Topic { /// Creates a `TopicHash` by SHA256 hashing the topic then base64 encoding the /// hash. pub fn sha256_hash(&self) -> TopicHash { - let mut topic_descripter = rpc_proto::TopicDescriptor::new(); - topic_descripter.set_name(self.topic.clone()); - let hash = encode( - Sha256::digest( - &topic_descripter - .write_to_bytes() - .expect("Message is always valid"), - ) - .as_slice(), - ); + 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 } } From 25a9636d7989da942c45509121587cd7233e81dc Mon Sep 17 00:00:00 2001 From: Age Manning Date: Fri, 24 Jan 2020 13:49:24 +1100 Subject: [PATCH 91/91] Use wasm_timer for wasm compatibility --- protocols/gossipsub/src/behaviour.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 103b2f2d00e..981cd178338 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -41,9 +41,8 @@ use std::{ marker::PhantomData, sync::Arc, task::{Context, Poll}, - time::Instant, }; -use wasm_timer::Interval; +use wasm_timer::{Instant, Interval}; mod tests;