From 2f9c1759e67f9abbf198e95825923b1dcd7c9c02 Mon Sep 17 00:00:00 2001 From: Max Inden Date: Thu, 11 Mar 2021 16:07:59 +0100 Subject: [PATCH] protocols/relay: Implement circuit relay specification (#1838) This commit implements the [libp2p circuit relay](https://github.com/libp2p/specs/tree/master/relay) specification. It is based on previous work from https://github.com/libp2p/rust-libp2p/pull/1134. Instead of altering the `Transport` trait, the approach taken in this commit is to wrap an existing implementation of `Transport` allowing one to: - Intercept `dial` requests with a relayed address. - Inject incoming relayed connections with the local node being the destination. - Intercept `listen_on` requests pointing to a relay, ensuring to keep a constant connection to the relay, waiting for incoming requests with the local node being the destination. More concretely one would wrap an existing `Transport` implementation as seen below, allowing the `Relay` behaviour and the `RelayTransport` to communicate via channels. ### Example ```rust let (relay_transport, relay_behaviour) = new_transport_and_behaviour( RelayConfig::default(), MemoryTransport::default(), ); let transport = relay_transport .upgrade(upgrade::Version::V1) .authenticate(plaintext) .multiplex(YamuxConfig::default()) .boxed(); let mut swarm = Swarm::new(transport, relay_behaviour, local_peer_id); let relay_addr = Multiaddr::from_str("/memory/1234").unwrap() .with(Protocol::P2p(PeerId::random().into())) .with(Protocol::P2pCircuit); let dst_addr = relay_addr.clone().with(Protocol::Memory(5678)); // Listen for incoming connections via relay node (1234). Swarm::listen_on(&mut swarm, relay_addr).unwrap(); // Dial node (5678) via relay node (1234). Swarm::dial_addr(&mut swarm, dst_addr).unwrap(); ``` Co-authored-by: Pierre Krieger Co-authored-by: Roman Borschel Co-authored-by: David Craven --- CHANGELOG.md | 3 + Cargo.toml | 4 + protocols/relay/CHANGELOG.md | 4 + protocols/relay/Cargo.toml | 38 + protocols/relay/build.rs | 23 + protocols/relay/examples/relay.rs | 84 + protocols/relay/src/behaviour.rs | 793 +++++++++ protocols/relay/src/handler.rs | 747 +++++++++ protocols/relay/src/lib.rs | 124 ++ protocols/relay/src/message.proto | 43 + protocols/relay/src/protocol.rs | 182 ++ protocols/relay/src/protocol/copy_future.rs | 137 ++ .../relay/src/protocol/incoming_dst_req.rs | 140 ++ .../relay/src/protocol/incoming_relay_req.rs | 156 ++ protocols/relay/src/protocol/listen.rs | 177 ++ .../relay/src/protocol/outgoing_dst_req.rs | 226 +++ .../relay/src/protocol/outgoing_relay_req.rs | 235 +++ protocols/relay/src/transport.rs | 559 +++++++ protocols/relay/tests/lib.rs | 1460 +++++++++++++++++ src/lib.rs | 4 + 20 files changed, 5139 insertions(+) create mode 100644 protocols/relay/CHANGELOG.md create mode 100644 protocols/relay/Cargo.toml create mode 100644 protocols/relay/build.rs create mode 100644 protocols/relay/examples/relay.rs create mode 100644 protocols/relay/src/behaviour.rs create mode 100644 protocols/relay/src/handler.rs create mode 100644 protocols/relay/src/lib.rs create mode 100644 protocols/relay/src/message.proto create mode 100644 protocols/relay/src/protocol.rs create mode 100644 protocols/relay/src/protocol/copy_future.rs create mode 100644 protocols/relay/src/protocol/incoming_dst_req.rs create mode 100644 protocols/relay/src/protocol/incoming_relay_req.rs create mode 100644 protocols/relay/src/protocol/listen.rs create mode 100644 protocols/relay/src/protocol/outgoing_dst_req.rs create mode 100644 protocols/relay/src/protocol/outgoing_relay_req.rs create mode 100644 protocols/relay/src/transport.rs create mode 100644 protocols/relay/tests/lib.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index ab53d41f996..a76a962d9c0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,6 +15,7 @@ - [`libp2p-kad` CHANGELOG](protocols/kad/CHANGELOG.md) - [`libp2p-mdns` CHANGELOG](protocols/mdns/CHANGELOG.md) - [`libp2p-ping` CHANGELOG](protocols/ping/CHANGELOG.md) +- [`libp2p-relay` CHANGELOG](protocols/relay/CHANGELOG.md) - [`libp2p-request-response` CHANGELOG](protocols/request-response/CHANGELOG.md) ## Transport Protocols & Upgrades @@ -48,6 +49,8 @@ - Do not leak default features from libp2p crates. [PR 1986](https://github.com/libp2p/rust-libp2p/pull/1986). +- Add `libp2p-relay` to `libp2p` facade crate. + ## Version 0.35.1 [2021-02-17] - Update `libp2p-yamux` to latest patch version. diff --git a/Cargo.toml b/Cargo.toml index 822e54dc1ae..9e521824191 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,6 +23,7 @@ default = [ "ping", "plaintext", "pnet", + "relay", "request-response", "secp256k1", "tcp-async-io", @@ -43,6 +44,7 @@ noise = ["libp2p-noise"] ping = ["libp2p-ping"] plaintext = ["libp2p-plaintext"] pnet = ["libp2p-pnet"] +relay = ["libp2p-relay"] request-response = ["libp2p-request-response"] tcp-async-io = ["libp2p-tcp", "libp2p-tcp/async-io"] tcp-tokio = ["libp2p-tcp", "libp2p-tcp/tokio"] @@ -71,6 +73,7 @@ libp2p-noise = { version = "0.29.0", path = "transports/noise", optional = true libp2p-ping = { version = "0.28.0", path = "protocols/ping", optional = true } libp2p-plaintext = { version = "0.27.1", path = "transports/plaintext", optional = true } libp2p-pnet = { version = "0.20.0", path = "transports/pnet", optional = true } +libp2p-relay = { version = "0.1.0", path = "protocols/relay", optional = true } libp2p-request-response = { version = "0.10.0", path = "protocols/request-response", optional = true } libp2p-swarm = { version = "0.28.0", path = "swarm" } libp2p-swarm-derive = { version = "0.22.0", path = "swarm-derive" } @@ -109,6 +112,7 @@ members = [ "protocols/kad", "protocols/mdns", "protocols/ping", + "protocols/relay", "protocols/request-response", "swarm", "swarm-derive", diff --git a/protocols/relay/CHANGELOG.md b/protocols/relay/CHANGELOG.md new file mode 100644 index 00000000000..16647c9a283 --- /dev/null +++ b/protocols/relay/CHANGELOG.md @@ -0,0 +1,4 @@ +# 0.1.0 [unreleased] + +- First release supporting all major features of the circuit relay v1 + specification. [PR 1838](https://github.com/libp2p/rust-libp2p/pull/1838). diff --git a/protocols/relay/Cargo.toml b/protocols/relay/Cargo.toml new file mode 100644 index 00000000000..e94123443cd --- /dev/null +++ b/protocols/relay/Cargo.toml @@ -0,0 +1,38 @@ +[package] +name = "libp2p-relay" +edition = "2018" +description = "Communications relaying for libp2p" +version = "0.1.0" +authors = ["Parity Technologies "] +license = "MIT" +repository = "https://github.com/libp2p/rust-libp2p" +keywords = ["peer-to-peer", "libp2p", "networking"] +categories = ["network-programming", "asynchronous"] + +[dependencies] +asynchronous-codec = "0.6" +bytes = "1" +futures = "0.3.1" +futures-timer = "3" +libp2p-core = { version = "0.27", path = "../../core" } +libp2p-swarm = { version = "0.28", path = "../../swarm" } +log = "0.4" +pin-project = "1" +prost = "0.7" +rand = "0.7" +smallvec = "0.6.9" +unsigned-varint = { version = "0.7", features = ["asynchronous_codec"] } +void = "1" +wasm-timer = "0.2" + +[build-dependencies] +prost-build = "0.7" + +[dev-dependencies] +env_logger = "0.7.1" +libp2p = { path = "../.." } +libp2p-kad = { path = "../kad" } +libp2p-ping = { path = "../ping" } +libp2p-identify = { path = "../identify" } +libp2p-plaintext = { path = "../../transports/plaintext" } +libp2p-yamux = { path = "../../muxers/yamux" } diff --git a/protocols/relay/build.rs b/protocols/relay/build.rs new file mode 100644 index 00000000000..cd7bd3deef6 --- /dev/null +++ b/protocols/relay/build.rs @@ -0,0 +1,23 @@ +// 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/message.proto"], &["src"]).unwrap(); +} diff --git a/protocols/relay/examples/relay.rs b/protocols/relay/examples/relay.rs new file mode 100644 index 00000000000..b52637e2cd3 --- /dev/null +++ b/protocols/relay/examples/relay.rs @@ -0,0 +1,84 @@ +// 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. + +use futures::executor::block_on; +use futures::stream::StreamExt; +use libp2p::core::upgrade; +use libp2p::plaintext; +use libp2p::relay::RelayConfig; +use libp2p::tcp::TcpConfig; +use libp2p::Transport; +use libp2p::{identity, PeerId, Swarm}; +use std::error::Error; +use std::task::{Context, Poll}; +use std::time::Duration; + +fn main() -> Result<(), Box> { + env_logger::init(); + + // Create a random PeerId + let local_key = identity::Keypair::generate_ed25519(); + let local_peer_id = PeerId::from(local_key.public()); + println!("Local peer id: {:?}", local_peer_id); + + let tcp_transport = TcpConfig::new(); + + let relay_config = RelayConfig { + connection_idle_timeout: Duration::from_secs(10 * 60), + ..Default::default() + }; + let (relay_wrapped_transport, relay_behaviour) = + libp2p_relay::new_transport_and_behaviour(relay_config, tcp_transport); + + let plaintext = plaintext::PlainText2Config { + local_public_key: local_key.public(), + }; + + let transport = relay_wrapped_transport + .upgrade(upgrade::Version::V1) + .authenticate(plaintext) + .multiplex(libp2p_yamux::YamuxConfig::default()) + .boxed(); + + let mut swarm = Swarm::new(transport, relay_behaviour, local_peer_id); + + // Listen on all interfaces and whatever port the OS assigns + Swarm::listen_on(&mut swarm, "/ip6/::/tcp/0".parse()?)?; + + let mut listening = false; + block_on(futures::future::poll_fn(move |cx: &mut Context<'_>| { + loop { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => println!("{:?}", event), + Poll::Ready(None) => return Poll::Ready(Ok(())), + Poll::Pending => { + if !listening { + for addr in Swarm::listeners(&swarm) { + println!("Listening on {:?}", addr); + listening = true; + } + } + break; + } + } + } + Poll::Pending + })) +} diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs new file mode 100644 index 00000000000..90858e072bb --- /dev/null +++ b/protocols/relay/src/behaviour.rs @@ -0,0 +1,793 @@ +// 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 crate::handler::{RelayHandlerConfig, RelayHandlerEvent, RelayHandlerIn, RelayHandlerProto}; +use crate::message_proto::circuit_relay; +use crate::protocol; +use crate::transport::TransportToBehaviourMsg; +use crate::RequestId; +use futures::channel::{mpsc, oneshot}; +use futures::prelude::*; +use libp2p_core::connection::{ConnectedPoint, ConnectionId, ListenerId}; +use libp2p_core::multiaddr::Multiaddr; +use libp2p_core::PeerId; +use libp2p_swarm::{ + DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, +}; +use std::collections::{hash_map::Entry, HashMap, HashSet, VecDeque}; +use std::task::{Context, Poll}; +use std::time::Duration; + +/// Network behaviour allowing the local node to act as a source, a relay and a destination. +pub struct Relay { + config: RelayConfig, + /// Channel receiver from [`crate::RelayTransport`]. + from_transport: mpsc::Receiver, + + /// Events that need to be send to a [`RelayListener`](crate::transport::RelayListener) via + /// [`Self::listeners`] or [`Self::listener_any_relay`]. + outbox_to_listeners: VecDeque<(PeerId, BehaviourToListenerMsg)>, + /// Events that need to be yielded to the outside when polling. + outbox_to_swarm: VecDeque>, + + /// List of peers the network is connected to. + connected_peers: HashMap>, + + /// Requests by the local node to a relay to relay a connection for the local node to a + /// destination. + outgoing_relay_reqs: OutgoingRelayReqs, + + /// Requests for the local node to act as a relay from a source to a destination indexed by + /// destination [`PeerId`]. + incoming_relay_reqs: HashMap>, + + /// List of relay nodes via which the local node is explicitly listening for incoming relayed + /// connections. + /// + /// Indexed by relay [`PeerId`]. Contains channel sender to listener. + listeners: HashMap, + + /// Channel sender to listener listening for incoming relayed connections from relay nodes via + /// which the local node is not explicitly listening. + listener_any_relay: Option>, +} + +#[derive(Default)] +struct OutgoingRelayReqs { + /// Indexed by relay peer id. + dialing: HashMap>, + upgrading: HashMap, +} + +struct OutgoingDialingRelayReq { + request_id: RequestId, + src_peer_id: PeerId, + relay_addr: Multiaddr, + dst_addr: Option, + dst_peer_id: PeerId, + send_back: oneshot::Sender>, +} + +struct OutgoingUpgradingRelayReq { + send_back: oneshot::Sender>, +} + +enum IncomingRelayReq { + DialingDst { + src_peer_id: PeerId, + src_addr: Multiaddr, + src_connection_id: ConnectionId, + request_id: RequestId, + incoming_relay_req: protocol::IncomingRelayReq, + }, +} + +pub struct RelayConfig { + /// How long to keep connections alive when they're idle. + /// + /// For a server, acting as a relay, allowing other nodes to listen for + /// incoming connections via oneself, this should likely be increased in + /// order not to force the peer to reconnect too regularly. + pub connection_idle_timeout: Duration, + /// Whether to actively establish an outgoing connection to a destination + /// node, when being asked by a source node to relay a connection to said + /// destination node. + /// + /// For security reasons this behaviour is disabled by default. Instead a + /// destination node should establish a connection to a relay node before + /// advertising their relayed address via that relay node to a source node. + pub actively_connect_to_dst_nodes: bool, +} + +impl Default for RelayConfig { + fn default() -> Self { + RelayConfig { + connection_idle_timeout: Duration::from_secs(10), + actively_connect_to_dst_nodes: false, + } + } +} + +// TODO: For now one is only able to specify relay servers via +// `Swarm::listen_on(Multiaddress(/p2p-circuit/))`. In the future +// we might want to support adding them via the Relay behaviour? The latter +// would allow other behaviours to manage ones relay listeners. +impl Relay { + /// Builds a new [`Relay`] [`NetworkBehaviour`]. + pub(crate) fn new( + config: RelayConfig, + from_transport: mpsc::Receiver, + ) -> Self { + Relay { + config, + from_transport, + outbox_to_listeners: Default::default(), + outbox_to_swarm: Default::default(), + connected_peers: Default::default(), + incoming_relay_reqs: Default::default(), + outgoing_relay_reqs: Default::default(), + listeners: Default::default(), + listener_any_relay: Default::default(), + } + } +} + +impl NetworkBehaviour for Relay { + type ProtocolsHandler = RelayHandlerProto; + type OutEvent = (); + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + RelayHandlerProto { + config: RelayHandlerConfig { + connection_idle_timeout: self.config.connection_idle_timeout, + }, + } + } + + fn addresses_of_peer(&mut self, remote_peer_id: &PeerId) -> Vec { + self.listeners + .iter() + .filter_map(|(peer_id, r)| { + if let RelayListener::Connecting { relay_addr, .. } = r { + if peer_id == remote_peer_id { + return Some(relay_addr.clone()); + } + } + None + }) + .chain( + self.outgoing_relay_reqs + .dialing + .get(remote_peer_id) + .into_iter() + .flatten() + .map(|OutgoingDialingRelayReq { relay_addr, .. }| relay_addr.clone()), + ) + .chain( + self.incoming_relay_reqs + .get(remote_peer_id) + .into_iter() + .flatten() + .map( + |IncomingRelayReq::DialingDst { + incoming_relay_req, .. + }| incoming_relay_req.dst_peer().addrs.clone(), + ) + .flatten(), + ) + .collect() + } + + fn inject_connection_established( + &mut self, + peer: &PeerId, + connection_id: &ConnectionId, + _: &ConnectedPoint, + ) { + let is_first = self + .connected_peers + .entry(*peer) + .or_default() + .insert(*connection_id); + assert!( + is_first, + "`inject_connection_established` called with known connection id" + ); + + if let Some(RelayListener::Connecting { .. }) = self.listeners.get(peer) { + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: *peer, + handler: NotifyHandler::One(*connection_id), + event: RelayHandlerIn::UsedForListening(true), + }); + let mut to_listener = match self.listeners.remove(peer) { + None | Some(RelayListener::Connected { .. }) => unreachable!("See outer match."), + Some(RelayListener::Connecting { to_listener, .. }) => to_listener, + }; + to_listener + .start_send(BehaviourToListenerMsg::ConnectionToRelayEstablished) + .expect("Channel to have at least capacity of 1."); + self.listeners.insert( + *peer, + RelayListener::Connected { + connection_id: *connection_id, + to_listener, + }, + ); + } + } + + fn inject_connected(&mut self, peer_id: &PeerId) { + assert!( + self.connected_peers + .get(peer_id) + .map(|cs| !cs.is_empty()) + .unwrap_or(false), + "Expect to be connected to peer with at least one connection." + ); + + if let Some(reqs) = self.outgoing_relay_reqs.dialing.remove(peer_id) { + for req in reqs { + let OutgoingDialingRelayReq { + request_id, + src_peer_id, + relay_addr: _, + dst_addr, + dst_peer_id, + send_back, + } = req; + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: *peer_id, + handler: NotifyHandler::Any, + event: RelayHandlerIn::OutgoingRelayReq { + src_peer_id, + request_id, + dst_peer_id, + dst_addr: dst_addr.clone(), + }, + }); + + self.outgoing_relay_reqs + .upgrading + .insert(request_id, OutgoingUpgradingRelayReq { send_back }); + } + } + + // Ask the newly-opened connection to be used as destination if relevant. + if let Some(reqs) = self.incoming_relay_reqs.remove(peer_id) { + for req in reqs { + let IncomingRelayReq::DialingDst { + src_peer_id, + src_addr, + src_connection_id, + request_id, + incoming_relay_req, + } = req; + let event = RelayHandlerIn::OutgoingDstReq { + src_peer_id, + src_addr, + src_connection_id, + request_id, + incoming_relay_req, + }; + + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: *peer_id, + handler: NotifyHandler::Any, + event: event, + }); + } + } + } + + fn inject_dial_failure(&mut self, peer_id: &PeerId) { + if let Entry::Occupied(o) = self.listeners.entry(*peer_id) { + if matches!(o.get(), RelayListener::Connecting{ .. }) { + // By removing the entry, the channel to the listener is dropped and thus the + // listener is notified that dialing the relay failed. + o.remove_entry(); + } + } + + if let Some(reqs) = self.outgoing_relay_reqs.dialing.remove(peer_id) { + for req in reqs { + let _ = req.send_back.send(Err(OutgoingRelayReqError::DialingRelay)); + } + } + + if let Some(reqs) = self.incoming_relay_reqs.remove(peer_id) { + for req in reqs { + let IncomingRelayReq::DialingDst { + src_peer_id, + incoming_relay_req, + .. + } = req; + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: src_peer_id, + handler: NotifyHandler::Any, + event: RelayHandlerIn::DenyIncomingRelayReq( + incoming_relay_req.deny(circuit_relay::Status::HopCantDialDst), + ), + }) + } + } + } + + fn inject_connection_closed( + &mut self, + peer: &PeerId, + connection: &ConnectionId, + _: &ConnectedPoint, + ) { + // Remove connection from the set of connections for the given peer. In case the set is + // empty it will be removed in `inject_disconnected`. + let was_present = self + .connected_peers + .get_mut(peer) + .expect("`inject_connection_closed` called for connected peer.") + .remove(connection); + assert!( + was_present, + "`inject_connection_closed` called for known connection" + ); + + match self.listeners.get(peer) { + None => {} + Some(RelayListener::Connecting { .. }) => unreachable!( + "State mismatch. Listener waiting for connection while \ + connection previously established.", + ), + Some(RelayListener::Connected { connection_id, .. }) => { + if connection_id == connection { + if let Some(new_primary) = self + .connected_peers + .get(peer) + .and_then(|cs| cs.iter().next()) + { + let to_listener = match self.listeners.remove(peer) { + None | Some(RelayListener::Connecting { .. }) => { + unreachable!("Due to outer match.") + } + Some(RelayListener::Connected { to_listener, .. }) => to_listener, + }; + self.listeners.insert( + *peer, + RelayListener::Connected { + connection_id: *new_primary, + to_listener, + }, + ); + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: *peer, + handler: NotifyHandler::One(*new_primary), + event: RelayHandlerIn::UsedForListening(true), + }); + } else { + // There are no more connections to the relay left that + // could be promoted as primary. Remove the listener, + // notifying the listener by dropping the channel to it. + self.listeners.remove(peer); + } + } + } + } + } + + fn inject_addr_reach_failure( + &mut self, + _peer_id: Option<&PeerId>, + _addr: &Multiaddr, + _error: &dyn std::error::Error, + ) { + // Handled in `inject_dial_failure`. + } + + fn inject_listener_error(&mut self, _id: ListenerId, _err: &(dyn std::error::Error + 'static)) { + } + + fn inject_listener_closed(&mut self, _id: ListenerId, _reason: Result<(), &std::io::Error>) {} + + fn inject_disconnected(&mut self, id: &PeerId) { + self.connected_peers.remove(id); + + if let Some(reqs) = self.incoming_relay_reqs.remove(id) { + for req in reqs { + let IncomingRelayReq::DialingDst { + src_peer_id, + incoming_relay_req, + .. + } = req; + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: src_peer_id, + handler: NotifyHandler::Any, + event: RelayHandlerIn::DenyIncomingRelayReq( + incoming_relay_req.deny(circuit_relay::Status::HopCantDialDst), + ), + }) + } + } + } + + fn inject_event( + &mut self, + event_source: PeerId, + connection: ConnectionId, + event: RelayHandlerEvent, + ) { + match event { + // Remote wants us to become a relay. + RelayHandlerEvent::IncomingRelayReq { + request_id, + src_addr, + req, + } => { + if self.connected_peers.get(&req.dst_peer().peer_id).is_some() { + let dest_id = req.dst_peer().peer_id; + let event = RelayHandlerIn::OutgoingDstReq { + src_peer_id: event_source, + src_addr, + src_connection_id: connection, + request_id, + incoming_relay_req: req, + }; + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: dest_id, + handler: NotifyHandler::Any, + event, + }); + } else { + if self.config.actively_connect_to_dst_nodes { + let dest_id = req.dst_peer().peer_id; + self.incoming_relay_reqs.entry(dest_id).or_default().push( + IncomingRelayReq::DialingDst { + request_id, + incoming_relay_req: req, + src_peer_id: event_source, + src_addr, + src_connection_id: connection, + }, + ); + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::DialPeer { + peer_id: dest_id, + condition: DialPeerCondition::NotDialing, + }); + } else { + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: event_source, + handler: NotifyHandler::One(connection), + event: RelayHandlerIn::DenyIncomingRelayReq( + req.deny(circuit_relay::Status::HopNoConnToDst), + ), + }); + } + } + } + // Remote wants us to become a destination. + RelayHandlerEvent::IncomingDstReq(request) => { + let got_explicit_listener = self + .listeners + .get(&event_source) + .map(|l| !l.is_closed()) + .unwrap_or(false); + let got_listener_for_any_relay = self + .listener_any_relay + .as_mut() + .map(|l| !l.is_closed()) + .unwrap_or(false); + + let send_back = if got_explicit_listener || got_listener_for_any_relay { + RelayHandlerIn::AcceptDstReq(request) + } else { + RelayHandlerIn::DenyDstReq(request) + }; + + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: event_source, + handler: NotifyHandler::One(connection), + event: send_back, + }); + } + RelayHandlerEvent::OutgoingRelayReqError(_dst_peer_id, request_id) => { + self.outgoing_relay_reqs + .upgrading + .remove(&request_id) + .expect("Outgoing relay request error for unknown request."); + } + RelayHandlerEvent::OutgoingRelayReqSuccess(_dst, request_id, stream) => { + let send_back = self + .outgoing_relay_reqs + .upgrading + .remove(&request_id) + .map(|OutgoingUpgradingRelayReq { send_back, .. }| send_back) + .expect("Outgoing relay request success for unknown request."); + let _ = send_back.send(Ok(stream)); + } + RelayHandlerEvent::IncomingDstReqSuccess { + stream, + src_peer_id, + relay_peer_id, + relay_addr, + } => self.outbox_to_listeners.push_back(( + relay_peer_id, + BehaviourToListenerMsg::IncomingRelayedConnection { + stream, + src_peer_id, + relay_peer_id, + relay_addr, + }, + )), + RelayHandlerEvent::OutgoingDstReqError { + src_connection_id, + incoming_relay_req_deny_fut, + } => { + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: event_source, + handler: NotifyHandler::One(src_connection_id), + event: RelayHandlerIn::DenyIncomingRelayReq(incoming_relay_req_deny_fut), + }); + } + } + } + + fn poll( + &mut self, + cx: &mut Context<'_>, + poll_parameters: &mut impl PollParameters, + ) -> Poll> { + if !self.outbox_to_listeners.is_empty() { + let relay_peer_id = self.outbox_to_listeners[0].0; + + let listeners = &mut self.listeners; + let listener_any_relay = self.listener_any_relay.as_mut(); + + // Get channel sender to the listener that is explicitly listening + // via this relay node, or, if registered, channel sender to + // listener listening via any relay. + let to_listener = listeners + .get_mut(&relay_peer_id) + .filter(|l| !l.is_closed()) + .and_then(|l| match l { + RelayListener::Connected { to_listener, .. } => Some(to_listener), + // State mismatch. Got relayed connection via relay, but + // local node is not connected to relay. + RelayListener::Connecting { .. } => None, + }) + .or_else(|| listener_any_relay) + .filter(|l| !l.is_closed()); + + match to_listener { + Some(to_listener) => match to_listener.poll_ready(cx) { + Poll::Ready(Ok(())) => { + if let Err(mpsc::SendError { .. }) = to_listener.start_send( + self.outbox_to_listeners + .pop_front() + .expect("Outbox is empty despite !is_empty().") + .1, + ) { + self.listeners.remove(&relay_peer_id); + } + } + Poll::Ready(Err(mpsc::SendError { .. })) => { + self.outbox_to_listeners.pop_front(); + self.listeners.remove(&relay_peer_id); + } + Poll::Pending => {} + }, + None => { + // No listener to send request to, thus dropping it. This + // case should be rare, as we check whether we have a + // listener before accepting an incoming destination + // request. + let event = self.outbox_to_listeners.pop_front(); + log::trace!("Dropping event for unknown listener: {:?}", event); + } + } + } + + loop { + match self.from_transport.poll_next_unpin(cx) { + Poll::Ready(Some(TransportToBehaviourMsg::DialReq { + request_id, + relay_addr, + relay_peer_id, + dst_addr, + dst_peer_id, + send_back, + })) => { + if let Some(_) = self.connected_peers.get(&relay_peer_id) { + // In case we are already listening via the relay, + // prefer the primary connection. + let handler = self + .listeners + .get(&relay_peer_id) + .and_then(|s| { + if let RelayListener::Connected { connection_id, .. } = s { + Some(NotifyHandler::One(*connection_id)) + } else { + None + } + }) + .unwrap_or(NotifyHandler::Any); + self.outbox_to_swarm + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: relay_peer_id, + handler, + event: RelayHandlerIn::OutgoingRelayReq { + request_id, + src_peer_id: *poll_parameters.local_peer_id(), + dst_peer_id, + dst_addr: dst_addr.clone(), + }, + }); + + self.outgoing_relay_reqs + .upgrading + .insert(request_id, OutgoingUpgradingRelayReq { send_back }); + } else { + self.outgoing_relay_reqs + .dialing + .entry(relay_peer_id) + .or_default() + .push(OutgoingDialingRelayReq { + src_peer_id: *poll_parameters.local_peer_id(), + request_id, + relay_addr, + dst_addr, + dst_peer_id, + send_back, + }); + return Poll::Ready(NetworkBehaviourAction::DialPeer { + peer_id: relay_peer_id, + condition: DialPeerCondition::Disconnected, + }); + } + } + Poll::Ready(Some(TransportToBehaviourMsg::ListenReq { + relay_peer_id_and_addr, + mut to_listener, + })) => { + match relay_peer_id_and_addr { + // Listener is listening for all incoming relayed + // connections from any relay + // node. + None => { + match self.listener_any_relay.as_mut() { + Some(sender) if !sender.is_closed() => { + // Already got listener listening for all + // incoming relayed connections. Signal to + // listener by dropping the channel sender + // to the listener. + } + _ => { + to_listener + .start_send( + BehaviourToListenerMsg::ConnectionToRelayEstablished, + ) + .expect("Channel to have at least capacity of 1."); + self.listener_any_relay = Some(to_listener); + } + } + } + // Listener is listening for incoming relayed + // connections from this relay only. + Some((relay_peer_id, relay_addr)) => { + if let Some(connections) = self.connected_peers.get(&relay_peer_id) { + to_listener + .start_send( + BehaviourToListenerMsg::ConnectionToRelayEstablished, + ) + .expect("Channel to have at least capacity of 1."); + let primary_connection = + connections.iter().next().expect("At least one connection."); + self.listeners.insert( + relay_peer_id, + RelayListener::Connected { + connection_id: *primary_connection, + to_listener, + }, + ); + + self.outbox_to_swarm.push_back( + NetworkBehaviourAction::NotifyHandler { + peer_id: relay_peer_id, + handler: NotifyHandler::One(*primary_connection), + event: RelayHandlerIn::UsedForListening(true), + }, + ); + } else { + self.listeners.insert( + relay_peer_id, + RelayListener::Connecting { + relay_addr, + to_listener, + }, + ); + return Poll::Ready(NetworkBehaviourAction::DialPeer { + peer_id: relay_peer_id, + condition: DialPeerCondition::Disconnected, + }); + } + } + } + } + Poll::Ready(None) => unreachable!( + "`Relay` `NetworkBehaviour` polled after channel from \ + `RelayTransport` has been closed.", + ), + Poll::Pending => break, + } + } + + if let Some(event) = self.outbox_to_swarm.pop_front() { + return Poll::Ready(event); + } + + Poll::Pending + } +} + +#[derive(Debug)] +pub enum BehaviourToListenerMsg { + ConnectionToRelayEstablished, + IncomingRelayedConnection { + stream: protocol::Connection, + src_peer_id: PeerId, + relay_peer_id: PeerId, + relay_addr: Multiaddr, + }, +} + +enum RelayListener { + Connecting { + relay_addr: Multiaddr, + to_listener: mpsc::Sender, + }, + Connected { + connection_id: ConnectionId, + to_listener: mpsc::Sender, + }, +} + +impl RelayListener { + /// Returns whether the channel to the + /// [`RelayListener`](crate::transport::RelayListener) is closed. + fn is_closed(&self) -> bool { + match self { + RelayListener::Connecting { to_listener, .. } + | RelayListener::Connected { to_listener, .. } => to_listener.is_closed(), + } + } +} + +#[derive(Debug, Eq, PartialEq)] +pub enum OutgoingRelayReqError { + DialingRelay, +} diff --git a/protocols/relay/src/handler.rs b/protocols/relay/src/handler.rs new file mode 100644 index 00000000000..7ea0d8731ac --- /dev/null +++ b/protocols/relay/src/handler.rs @@ -0,0 +1,747 @@ +// 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 crate::message_proto::circuit_relay; +use crate::protocol; +use crate::RequestId; +use futures::channel::oneshot::{self, Canceled}; +use futures::future::BoxFuture; +use futures::prelude::*; +use futures::stream::FuturesUnordered; +use libp2p_core::connection::ConnectionId; +use libp2p_core::either::{EitherError, EitherOutput}; +use libp2p_core::{upgrade, ConnectedPoint, Multiaddr, PeerId}; +use libp2p_swarm::{ + IntoProtocolsHandler, KeepAlive, NegotiatedSubstream, ProtocolsHandler, ProtocolsHandlerEvent, + ProtocolsHandlerUpgrErr, SubstreamProtocol, +}; +use log::warn; +use std::task::{Context, Poll}; +use std::time::Duration; +use wasm_timer::Instant; + +pub struct RelayHandlerConfig { + pub connection_idle_timeout: Duration, +} + +pub struct RelayHandlerProto { + pub config: RelayHandlerConfig, +} + +impl IntoProtocolsHandler for RelayHandlerProto { + type Handler = RelayHandler; + + fn into_handler(self, remote_peer_id: &PeerId, endpoint: &ConnectedPoint) -> Self::Handler { + RelayHandler::new( + self.config, + *remote_peer_id, + endpoint.get_remote_address().clone(), + ) + } + + fn inbound_protocol(&self) -> ::InboundProtocol { + protocol::RelayListen::new() + } +} + +/// Protocol handler that handles the relay protocol. +/// +/// There are four possible situations in play here: +/// +/// - The handler emits [`RelayHandlerEvent::IncomingRelayReq`] if the node we handle asks us to act +/// as a relay. You must send a [`RelayHandlerIn::OutgoingDstReq`] to another handler, or send +/// back a [`RelayHandlerIn::DenyIncomingRelayReq`]. +/// +/// - The handler emits [`RelayHandlerEvent::IncomingDstReq`] if the node we handle asks us to act +/// as a destination. You must either send back a [`RelayHandlerIn::AcceptDstReq`]`, or send back +/// a [`RelayHandlerIn::DenyDstReq`]. +/// +/// - Send a [`RelayHandlerIn::OutgoingRelayReq`] if the node we handle must act as a relay to a +/// destination. The handler will either send back a +/// [`RelayHandlerEvent::OutgoingRelayReqSuccess`] containing the stream to the destination, or a +/// [`RelayHandlerEvent::OutgoingRelayReqError`]. +/// +/// - Send a [`RelayHandlerIn::OutgoingDstReq`] if the node we handle must act as a destination. The +/// handler will automatically notify the source whether the request was accepted or denied. +pub struct RelayHandler { + config: RelayHandlerConfig, + /// Specifies whether the handled connection is used to listen for incoming relayed connections. + used_for_listening: bool, + remote_address: Multiaddr, + remote_peer_id: PeerId, + /// Futures that send back negative responses. + deny_futures: FuturesUnordered>>, + /// Futures that send back an accept response to a relay. + accept_dst_futures: FuturesUnordered< + BoxFuture< + 'static, + Result< + (PeerId, protocol::Connection, oneshot::Receiver<()>), + protocol::IncomingDstReqError, + >, + >, + >, + /// Futures that copy from a source to a destination. + copy_futures: FuturesUnordered>>, + /// Requests asking the remote to become a relay. + outgoing_relay_reqs: Vec, + /// Requests asking the remote to become a destination. + outgoing_dst_reqs: Vec, + /// Queue of events to return when polled. + queued_events: Vec, + /// Tracks substreams lend out to other [`RelayHandler`]s or as + /// [`Connection`](protocol::Connection) to the + /// [`RelayTransport`](crate::RelayTransport). + /// + /// For each substream to the peer of this handler, there is a future in here that resolves once + /// the given substream is dropped. + /// + /// Once all substreams are dropped and this handler has no other work, [`KeepAlive::Until`] can + /// be set, allowing the connection to be closed eventually. + alive_lend_out_substreams: FuturesUnordered>, + /// The current connection keep-alive. + keep_alive: KeepAlive, + /// A pending fatal error that results in the connection being closed. + pending_error: Option< + ProtocolsHandlerUpgrErr< + EitherError< + protocol::RelayListenError, + EitherError, + >, + >, + >, +} + +struct OutgoingRelayReq { + src_peer_id: PeerId, + dst_peer_id: PeerId, + request_id: RequestId, + /// Addresses of the destination. + dst_addr: Option, +} + +struct OutgoingDstReq { + src_peer_id: PeerId, + src_addr: Multiaddr, + src_connection_id: ConnectionId, + request_id: RequestId, + incoming_relay_req: protocol::IncomingRelayReq, +} + +/// Event produced by the relay handler. +pub enum RelayHandlerEvent { + /// The remote wants us to relay communications to a third party. You must either send back a + /// [`RelayHandlerIn::DenyIncomingRelayReq`], or an [`RelayHandlerIn::OutgoingDstReq`] to any + /// connection handler for the destination peer, providing the [`protocol::IncomingRelayReq`]. + IncomingRelayReq { + request_id: RequestId, + src_addr: Multiaddr, + req: protocol::IncomingRelayReq, + }, + + /// The remote is a relay and is relaying a connection to us. In other words, we are used as + /// a destination. The behaviour can accept or deny the request via + /// [`AcceptDstReq`](RelayHandlerIn::AcceptDstReq) or + /// [`DenyDstReq`](RelayHandlerIn::DenyDstReq). + IncomingDstReq(protocol::IncomingDstReq), + + /// A `RelayReq` that has previously been sent has been accepted by the remote. Contains + /// a substream that communicates with the requested destination. + /// + /// > **Note**: There is no proof that we are actually communicating with the destination. An + /// > encryption handshake has to be performed on top of this substream in order to + /// > avoid MITM attacks. + OutgoingRelayReqSuccess(PeerId, RequestId, protocol::Connection), + + /// The local node has accepted an incoming destination request. Contains a substream that + /// communicates with the source. + /// + /// > **Note**: There is no proof that we are actually communicating with the destination. An + /// > encryption handshake has to be performed on top of this substream in order to + /// > avoid MITM attacks. + IncomingDstReqSuccess { + stream: protocol::Connection, + src_peer_id: PeerId, + relay_peer_id: PeerId, + relay_addr: Multiaddr, + }, + + /// A `RelayReq` that has previously been sent by the local node has failed. + OutgoingRelayReqError(PeerId, RequestId), + + /// A destination request that has previously been sent by the local node has failed. + /// + /// Includes the incoming relay request, which is yet to be denied due to the failure. + OutgoingDstReqError { + src_connection_id: ConnectionId, + incoming_relay_req_deny_fut: BoxFuture<'static, Result<(), std::io::Error>>, + }, +} + +/// Event that can be sent to the relay handler. +pub enum RelayHandlerIn { + /// Tell the handler whether it is handling a connection used to listen for incoming relayed + /// connections. + UsedForListening(bool), + /// Denies a relay request sent by the node we talk to acting as a source. + DenyIncomingRelayReq(BoxFuture<'static, Result<(), std::io::Error>>), + + /// Accepts a destination request sent by the node we talk to. + AcceptDstReq(protocol::IncomingDstReq), + + /// Denies a destination request sent by the node we talk to. + DenyDstReq(protocol::IncomingDstReq), + + /// Opens a new substream to the remote and asks it to relay communications to a third party. + OutgoingRelayReq { + src_peer_id: PeerId, + dst_peer_id: PeerId, + request_id: RequestId, + /// Addresses known for this peer to transmit to the remote. + dst_addr: Option, + }, + + /// Asks the node to be used as a destination for a relayed connection. + /// + /// The positive or negative response will be written to `substream`. + OutgoingDstReq { + /// Peer id of the node whose communications are being relayed. + src_peer_id: PeerId, + /// Address of the node whose communications are being relayed. + src_addr: Multiaddr, + src_connection_id: ConnectionId, + request_id: RequestId, + /// Incoming relay request from the source node. + incoming_relay_req: protocol::IncomingRelayReq, + }, +} + +impl RelayHandler { + /// Builds a new `RelayHandler`. + pub fn new( + config: RelayHandlerConfig, + remote_peer_id: PeerId, + remote_address: Multiaddr, + ) -> Self { + RelayHandler { + config, + used_for_listening: false, + remote_address, + remote_peer_id, + deny_futures: Default::default(), + accept_dst_futures: Default::default(), + copy_futures: Default::default(), + outgoing_relay_reqs: Default::default(), + outgoing_dst_reqs: Default::default(), + queued_events: Default::default(), + alive_lend_out_substreams: Default::default(), + keep_alive: KeepAlive::Yes, + pending_error: None, + } + } +} + +impl ProtocolsHandler for RelayHandler { + type InEvent = RelayHandlerIn; + type OutEvent = RelayHandlerEvent; + type Error = ProtocolsHandlerUpgrErr< + EitherError< + protocol::RelayListenError, + EitherError, + >, + >; + type InboundProtocol = protocol::RelayListen; + type OutboundProtocol = + upgrade::EitherUpgrade; + type OutboundOpenInfo = RelayOutboundOpenInfo; + type InboundOpenInfo = RequestId; + + fn listen_protocol(&self) -> SubstreamProtocol { + SubstreamProtocol::new(protocol::RelayListen::new(), RequestId::new()) + } + + fn inject_fully_negotiated_inbound( + &mut self, + protocol: >::Output, + request_id: Self::InboundOpenInfo, + ) { + match protocol { + // We have been asked to act as a relay. + protocol::RelayRemoteReq::RelayReq((incoming_relay_request, notifyee)) => { + self.alive_lend_out_substreams.push(notifyee); + self.queued_events + .push(RelayHandlerEvent::IncomingRelayReq { + request_id, + src_addr: self.remote_address.clone(), + req: incoming_relay_request, + }); + } + // We have been asked to become a destination. + protocol::RelayRemoteReq::DstReq(dst_request) => { + self.queued_events + .push(RelayHandlerEvent::IncomingDstReq(dst_request)); + } + } + } + + fn inject_fully_negotiated_outbound( + &mut self, + protocol: >::Output, + open_info: Self::OutboundOpenInfo, + ) { + match protocol { + // We have successfully negotiated a substream towards a relay. + EitherOutput::First((substream_to_dest, notifyee)) => { + let (dst_peer_id, request_id) = match open_info { + RelayOutboundOpenInfo::Relay { + dst_peer_id, + request_id, + } => (dst_peer_id, request_id), + RelayOutboundOpenInfo::Destination { .. } => unreachable!( + "Can not successfully dial a relay when actually dialing a destination." + ), + }; + + self.alive_lend_out_substreams.push(notifyee); + self.queued_events + .push(RelayHandlerEvent::OutgoingRelayReqSuccess( + dst_peer_id, + request_id, + substream_to_dest, + )); + } + // We have successfully asked the node to be a destination. + EitherOutput::Second((to_dest_substream, from_dst_read_buffer)) => { + let incoming_relay_req = match open_info { + RelayOutboundOpenInfo::Destination { + incoming_relay_req, .. + } => incoming_relay_req, + RelayOutboundOpenInfo::Relay { .. } => unreachable!( + "Can not successfully dial a destination when actually dialing a relay." + ), + }; + self.copy_futures + .push(incoming_relay_req.fulfill(to_dest_substream, from_dst_read_buffer)); + } + } + } + + fn inject_event(&mut self, event: Self::InEvent) { + match event { + RelayHandlerIn::UsedForListening(s) => self.used_for_listening = s, + // Deny a relay request from the node we handle. + RelayHandlerIn::DenyIncomingRelayReq(req) => { + self.deny_futures.push(req); + } + RelayHandlerIn::AcceptDstReq(request) => self.accept_dst_futures.push(request.accept()), + RelayHandlerIn::DenyDstReq(request) => self.deny_futures.push(request.deny()), + // Ask the node we handle to act as a relay. + RelayHandlerIn::OutgoingRelayReq { + src_peer_id, + dst_peer_id, + request_id, + dst_addr, + } => { + self.outgoing_relay_reqs.push(OutgoingRelayReq { + src_peer_id, + dst_peer_id, + request_id, + dst_addr, + }); + } + // Ask the node we handle to act as a destination. + RelayHandlerIn::OutgoingDstReq { + src_peer_id, + src_addr, + src_connection_id, + request_id, + incoming_relay_req, + } => { + self.outgoing_dst_reqs.push(OutgoingDstReq { + src_peer_id, + src_addr, + src_connection_id, + request_id, + incoming_relay_req, + }); + } + } + } + + fn inject_listen_upgrade_error( + &mut self, + _: Self::InboundOpenInfo, + error: ProtocolsHandlerUpgrErr, + ) { + match error { + ProtocolsHandlerUpgrErr::Timeout | ProtocolsHandlerUpgrErr::Timer => {} + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select( + upgrade::NegotiationError::Failed, + )) => {} + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select( + upgrade::NegotiationError::ProtocolError(e), + )) => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Select(upgrade::NegotiationError::ProtocolError(e)), + )); + } + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(error)) => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Apply(EitherError::A(error)), + )) + } + } + } + + fn inject_dial_upgrade_error( + &mut self, + open_info: Self::OutboundOpenInfo, + error: ProtocolsHandlerUpgrErr< + EitherError, + >, + ) { + match open_info { + RelayOutboundOpenInfo::Relay { + dst_peer_id, + request_id, + } => { + match error { + ProtocolsHandlerUpgrErr::Timeout | ProtocolsHandlerUpgrErr::Timer => {} + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select( + upgrade::NegotiationError::Failed, + )) => {} + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select( + upgrade::NegotiationError::ProtocolError(e), + )) => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Select( + upgrade::NegotiationError::ProtocolError(e), + ), + )); + } + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply( + EitherError::A(error), + )) => match error { + protocol::OutgoingRelayReqError::Decode(_) + | protocol::OutgoingRelayReqError::Io(_) + | protocol::OutgoingRelayReqError::ParseTypeField + | protocol::OutgoingRelayReqError::ParseStatusField + | protocol::OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType + | protocol::OutgoingRelayReqError::UnexpectedDstPeerWithStatusType + | protocol::OutgoingRelayReqError::ExpectedStatusType(_) => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Apply(EitherError::B(EitherError::A(error))), + )); + } + protocol::OutgoingRelayReqError::ExpectedSuccessStatus(status) => { + match status { + circuit_relay::Status::Success => { + unreachable!("Status success was explicitly expected earlier.") + } + // With either status below there is no reason to stay connected. + // Thus terminate the connection. + circuit_relay::Status::HopSrcAddrTooLong + | circuit_relay::Status::HopSrcMultiaddrInvalid + | circuit_relay::Status::MalformedMessage => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Apply(EitherError::B( + EitherError::A(error), + )), + )); + } + // While useless for reaching this particular destination, the + // connection to the relay might still proof helpful for other + // destinations. Thus do not terminate the connection. + circuit_relay::Status::StopSrcAddrTooLong + | circuit_relay::Status::StopDstAddrTooLong + | circuit_relay::Status::StopSrcMultiaddrInvalid + | circuit_relay::Status::StopDstMultiaddrInvalid + | circuit_relay::Status::StopRelayRefused + | circuit_relay::Status::HopDstAddrTooLong + | circuit_relay::Status::HopDstMultiaddrInvalid + | circuit_relay::Status::HopNoConnToDst + | circuit_relay::Status::HopCantDialDst + | circuit_relay::Status::HopCantOpenDstStream + | circuit_relay::Status::HopCantSpeakRelay + | circuit_relay::Status::HopCantRelayToSelf => {} + } + } + }, + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply( + EitherError::B(_), + )) => { + unreachable!("Can not receive an OutgoingDstReqError when dialing a relay.") + } + } + + self.queued_events + .push(RelayHandlerEvent::OutgoingRelayReqError( + dst_peer_id, + request_id, + )); + } + RelayOutboundOpenInfo::Destination { + src_connection_id, + incoming_relay_req, + .. + } => { + let err_code = match error { + ProtocolsHandlerUpgrErr::Timeout | ProtocolsHandlerUpgrErr::Timer => { + circuit_relay::Status::HopCantOpenDstStream + } + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select( + upgrade::NegotiationError::Failed, + )) => circuit_relay::Status::HopCantSpeakRelay, + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select( + upgrade::NegotiationError::ProtocolError(e), + )) => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Select( + upgrade::NegotiationError::ProtocolError(e), + ), + )); + circuit_relay::Status::HopCantSpeakRelay + } + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply( + EitherError::A(_), + )) => unreachable!( + "Can not receive an OutgoingRelayReqError when dialing a destination." + ), + ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply( + EitherError::B(error), + )) => { + match error { + protocol::OutgoingDstReqError::Decode(_) + | protocol::OutgoingDstReqError::Io(_) + | protocol::OutgoingDstReqError::ParseTypeField + | protocol::OutgoingDstReqError::ParseStatusField + | protocol::OutgoingDstReqError::UnexpectedSrcPeerWithStatusType + | protocol::OutgoingDstReqError::UnexpectedDstPeerWithStatusType + | protocol::OutgoingDstReqError::ExpectedStatusType(_) => { + self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Apply(EitherError::B(EitherError::B( + error, + ))), + )); + circuit_relay::Status::HopCantOpenDstStream + } + protocol::OutgoingDstReqError::ExpectedSuccessStatus(status) => { + match status { + circuit_relay::Status::Success => { + unreachable!( + "Status success was explicitly expected earlier." + ) + } + // A destination node returning `Hop.*` status is a protocol + // violation. Thus terminate the connection. + circuit_relay::Status::HopDstAddrTooLong + | circuit_relay::Status::HopDstMultiaddrInvalid + | circuit_relay::Status::HopNoConnToDst + | circuit_relay::Status::HopCantDialDst + | circuit_relay::Status::HopCantOpenDstStream + | circuit_relay::Status::HopCantSpeakRelay + | circuit_relay::Status::HopCantRelayToSelf + | circuit_relay::Status::HopSrcAddrTooLong + | circuit_relay::Status::HopSrcMultiaddrInvalid => { + self.pending_error = + Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Apply(EitherError::B( + EitherError::B(error), + )), + )); + } + // With either status below there is no reason to stay connected. + // Thus terminate the connection. + circuit_relay::Status::StopDstAddrTooLong + | circuit_relay::Status::StopDstMultiaddrInvalid + | circuit_relay::Status::MalformedMessage => { + self.pending_error = + Some(ProtocolsHandlerUpgrErr::Upgrade( + upgrade::UpgradeError::Apply(EitherError::B( + EitherError::B(error), + )), + )); + } + // While useless for reaching this particular destination, the + // connection to the relay might still proof helpful for other + // destinations. Thus do not terminate the connection. + circuit_relay::Status::StopSrcAddrTooLong + | circuit_relay::Status::StopSrcMultiaddrInvalid + | circuit_relay::Status::StopRelayRefused => {} + } + status + } + } + } + }; + + self.queued_events + .push(RelayHandlerEvent::OutgoingDstReqError { + src_connection_id, + incoming_relay_req_deny_fut: incoming_relay_req.deny(err_code), + }); + } + } + } + + fn connection_keep_alive(&self) -> KeepAlive { + self.keep_alive + } + + fn poll( + &mut self, + cx: &mut Context<'_>, + ) -> Poll< + ProtocolsHandlerEvent< + Self::OutboundProtocol, + Self::OutboundOpenInfo, + Self::OutEvent, + Self::Error, + >, + > { + // Check for a pending (fatal) error. + if let Some(err) = self.pending_error.take() { + // The handler will not be polled again by the `Swarm`. + return Poll::Ready(ProtocolsHandlerEvent::Close(err)); + } + + // Request the remote to act as a relay. + if !self.outgoing_relay_reqs.is_empty() { + let OutgoingRelayReq { + src_peer_id, + dst_peer_id, + request_id, + dst_addr, + } = self.outgoing_relay_reqs.remove(0); + self.outgoing_relay_reqs.shrink_to_fit(); + return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + protocol: SubstreamProtocol::new( + upgrade::EitherUpgrade::A(protocol::OutgoingRelayReq::new( + src_peer_id, + dst_peer_id, + dst_addr, + )), + RelayOutboundOpenInfo::Relay { + dst_peer_id, + request_id, + }, + ), + }); + } + + // Request the remote to act as destination. + if !self.outgoing_dst_reqs.is_empty() { + let OutgoingDstReq { + src_peer_id, + src_addr, + src_connection_id, + request_id, + incoming_relay_req, + } = self.outgoing_dst_reqs.remove(0); + self.outgoing_dst_reqs.shrink_to_fit(); + return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + protocol: SubstreamProtocol::new( + upgrade::EitherUpgrade::B(protocol::OutgoingDstReq::new( + src_peer_id, + src_addr, + incoming_relay_req.dst_peer().clone(), + )), + RelayOutboundOpenInfo::Destination { + src_peer_id, + request_id, + src_connection_id, + incoming_relay_req, + }, + ), + }); + } + + match self.accept_dst_futures.poll_next_unpin(cx) { + Poll::Ready(Some(Ok((src_peer_id, substream, notifyee)))) => { + self.alive_lend_out_substreams.push(notifyee); + let event = RelayHandlerEvent::IncomingDstReqSuccess { + stream: substream, + src_peer_id, + relay_peer_id: self.remote_peer_id, + relay_addr: self.remote_address.clone(), + }; + return Poll::Ready(ProtocolsHandlerEvent::Custom(event)); + } + Poll::Ready(Some(Err(e))) => { + log::debug!("Failed to accept destination future: {:?}", e); + } + Poll::Ready(None) => {} + Poll::Pending => {} + } + + while let Poll::Ready(Some(result)) = self.copy_futures.poll_next_unpin(cx) { + if let Err(e) = result { + warn!("Incoming relay request failed: {:?}", e); + } + } + + while let Poll::Ready(Some(result)) = self.deny_futures.poll_next_unpin(cx) { + if let Err(e) = result { + warn!("Denying request failed: {:?}", e); + } + } + + // Report the queued events. + if !self.queued_events.is_empty() { + let event = self.queued_events.remove(0); + return Poll::Ready(ProtocolsHandlerEvent::Custom(event)); + } + + while let Poll::Ready(Some(Err(Canceled))) = + self.alive_lend_out_substreams.poll_next_unpin(cx) + {} + + if self.used_for_listening + || !self.deny_futures.is_empty() + || !self.accept_dst_futures.is_empty() + || !self.copy_futures.is_empty() + || !self.alive_lend_out_substreams.is_empty() + { + // Protocol handler is busy. + self.keep_alive = KeepAlive::Yes; + } else { + // Protocol handler is idle. + if matches!(self.keep_alive, KeepAlive::Yes) { + self.keep_alive = + KeepAlive::Until(Instant::now() + self.config.connection_idle_timeout); + } + } + + Poll::Pending + } +} + +pub enum RelayOutboundOpenInfo { + Relay { + dst_peer_id: PeerId, + request_id: RequestId, + }, + Destination { + src_peer_id: PeerId, + src_connection_id: ConnectionId, + request_id: RequestId, + incoming_relay_req: protocol::IncomingRelayReq, + }, +} diff --git a/protocols/relay/src/lib.rs b/protocols/relay/src/lib.rs new file mode 100644 index 00000000000..62a0645c432 --- /dev/null +++ b/protocols/relay/src/lib.rs @@ -0,0 +1,124 @@ +// 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. + +//! Implementation of the [libp2p circuit relay +//! specification](https://github.com/libp2p/specs/tree/master/relay). +//! +//! ## Example +//! +//! ```rust +//! # use libp2p_core::transport::memory::MemoryTransport; +//! # use libp2p_relay::{RelayConfig, new_transport_and_behaviour}; +//! # use libp2p_swarm::Swarm; +//! # use libp2p_core::{identity, Multiaddr, multiaddr::Protocol, PeerId, upgrade, Transport}; +//! # use libp2p_yamux::YamuxConfig; +//! # use libp2p_plaintext::PlainText2Config; +//! # use std::convert::TryInto; +//! # use std::str::FromStr; +//! # +//! # let local_key = identity::Keypair::generate_ed25519(); +//! # let local_public_key = local_key.public(); +//! # let local_peer_id = local_public_key.clone().into_peer_id(); +//! # let plaintext = PlainText2Config { +//! # local_public_key: local_public_key.clone(), +//! # }; +//! # +//! let (relay_transport, relay_behaviour) = new_transport_and_behaviour( +//! RelayConfig::default(), +//! MemoryTransport::default(), +//! ); +//! +//! let transport = relay_transport +//! .upgrade(upgrade::Version::V1) +//! .authenticate(plaintext) +//! .multiplex(YamuxConfig::default()) +//! .boxed(); +//! +//! let mut swarm = Swarm::new(transport, relay_behaviour, local_peer_id); +//! +//! let relay_addr = Multiaddr::from_str("/memory/1234").unwrap() +//! .with(Protocol::P2p(PeerId::random().into())) +//! .with(Protocol::P2pCircuit); +//! let dst_addr = relay_addr.clone().with(Protocol::Memory(5678)); +//! +//! // Listen for incoming connections via relay node (1234). +//! Swarm::listen_on(&mut swarm, relay_addr).unwrap(); +//! +//! // Dial node (5678) via relay node (1234). +//! Swarm::dial_addr(&mut swarm, dst_addr).unwrap(); +//! ``` +//! +//! ## Terminology +//! +//! ### Entities +//! +//! - **Source**: The node initiating a connection via a *relay* to a *destination*. +//! +//! - **Relay**: The node being asked by a *source* to relay to a *destination*. +//! +//! - **Destination**: The node contacted by the *source* via the *relay*. +//! +//! ### Messages +//! +//! - **Outgoing relay request**: The request sent by a *source* to a *relay*. +//! +//! - **Incoming relay request**: The request received by a *relay* from a *source*. +//! +//! - **Outgoing destination request**: The request sent by a *relay* to a *destination*. +//! +//! - **Incoming destination request**: The request received by a *destination* from a *relay*. + +mod behaviour; + +mod message_proto { + include!(concat!(env!("OUT_DIR"), "/message.pb.rs")); +} + +mod handler; +mod protocol; +mod transport; + +pub use behaviour::{Relay, RelayConfig}; +pub use transport::{RelayError, RelayTransport}; + +use libp2p_core::Transport; + +/// Create both a [`RelayTransport`] wrapping the provided [`Transport`] +/// as well as a [`Relay`] [`NetworkBehaviour`](libp2p_swarm::NetworkBehaviour). +/// +/// Interconnects the returned [`RelayTransport`] and [`Relay`]. +pub fn new_transport_and_behaviour( + config: RelayConfig, + transport: T, +) -> (RelayTransport, Relay) { + let (transport, from_transport) = RelayTransport::new(transport); + let behaviour = Relay::new(config, from_transport); + (transport, behaviour) +} + +/// The ID of an outgoing / incoming, relay / destination request. +#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +pub struct RequestId(u64); + +impl RequestId { + fn new() -> RequestId { + RequestId(rand::random()) + } +} diff --git a/protocols/relay/src/message.proto b/protocols/relay/src/message.proto new file mode 100644 index 00000000000..dfaf0411eea --- /dev/null +++ b/protocols/relay/src/message.proto @@ -0,0 +1,43 @@ +syntax = "proto2"; +package message.pb; + +message CircuitRelay { + + enum Status { + SUCCESS = 100; + HOP_SRC_ADDR_TOO_LONG = 220; + HOP_DST_ADDR_TOO_LONG = 221; + HOP_SRC_MULTIADDR_INVALID = 250; + HOP_DST_MULTIADDR_INVALID = 251; + HOP_NO_CONN_TO_DST = 260; + HOP_CANT_DIAL_DST = 261; + HOP_CANT_OPEN_DST_STREAM = 262; + HOP_CANT_SPEAK_RELAY = 270; + HOP_CANT_RELAY_TO_SELF = 280; + STOP_SRC_ADDR_TOO_LONG = 320; + STOP_DST_ADDR_TOO_LONG = 321; + STOP_SRC_MULTIADDR_INVALID = 350; + STOP_DST_MULTIADDR_INVALID = 351; + STOP_RELAY_REFUSED = 390; + MALFORMED_MESSAGE = 400; + } + + enum Type { // RPC identifier, either HOP, STOP or STATUS + HOP = 1; + STOP = 2; + STATUS = 3; + CAN_HOP = 4; // is peer a relay? + } + + message Peer { + required bytes id = 1; // peer id + repeated bytes addrs = 2; // peer's known addresses + } + + optional Type type = 1; // Type of the message + + optional Peer srcPeer = 2; // srcPeer and dstPeer are used when Type is HOP or STOP + optional Peer dstPeer = 3; + + optional Status code = 4; // Status code, used when Type is STATUS +} diff --git a/protocols/relay/src/protocol.rs b/protocols/relay/src/protocol.rs new file mode 100644 index 00000000000..900d16a6d74 --- /dev/null +++ b/protocols/relay/src/protocol.rs @@ -0,0 +1,182 @@ +// 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::message_proto::circuit_relay; + +use bytes::Bytes; +use futures::channel::oneshot; +use futures::io::{AsyncRead, AsyncWrite}; +use libp2p_core::{multiaddr::Error as MultiaddrError, Multiaddr, PeerId}; +use libp2p_swarm::NegotiatedSubstream; +use smallvec::SmallVec; +use std::io::{Error, IoSlice}; +use std::pin::Pin; +use std::task::{Context, Poll}; +use std::{convert::TryFrom, error, fmt}; + +/// Any message received on the wire whose length exceeds this value is refused. +// +// The circuit relay specification sets a maximum of 1024 bytes per multiaddr. A single message can +// contain multiple addresses for both the source and destination node. Setting the maximum message +// length to 10 times that limit is an unproven estimate. Feel free to refine this in the future. +const MAX_ACCEPTED_MESSAGE_LEN: usize = 10 * 1024; + +const PROTOCOL_NAME: &[u8; 27] = b"/libp2p/circuit/relay/0.1.0"; + +// Source -> Relay +mod incoming_relay_req; +mod outgoing_relay_req; +pub use self::incoming_relay_req::{IncomingRelayReq, IncomingRelayReqError}; +pub use self::outgoing_relay_req::{OutgoingRelayReq, OutgoingRelayReqError}; + +// Relay -> Destination +mod incoming_dst_req; +mod outgoing_dst_req; +pub use self::incoming_dst_req::{IncomingDstReq, IncomingDstReqError}; +pub use self::outgoing_dst_req::{OutgoingDstReq, OutgoingDstReqError}; + +mod listen; +pub use self::listen::{RelayListen, RelayListenError, RelayRemoteReq}; + +pub mod copy_future; + +/// Representation of a `CircuitRelay_Peer` protobuf message with refined field types. +/// +/// Can be parsed from a `CircuitRelay_Peer` using the `TryFrom` trait. +#[derive(Clone)] +pub(crate) struct Peer { + pub(crate) peer_id: PeerId, + pub(crate) addrs: SmallVec<[Multiaddr; 4]>, +} + +impl TryFrom for Peer { + type Error = PeerParseError; + + fn try_from(peer: circuit_relay::Peer) -> Result { + let circuit_relay::Peer { id, addrs } = peer; + let peer_id = PeerId::from_bytes(&id).map_err(|_| PeerParseError::PeerIdParseError)?; + let mut parsed_addrs = SmallVec::with_capacity(addrs.len()); + for addr in addrs.into_iter() { + let addr = Multiaddr::try_from(addr).map_err(PeerParseError::MultiaddrParseError)?; + parsed_addrs.push(addr); + } + Ok(Peer { + peer_id, + addrs: parsed_addrs, + }) + } +} + +/// Error while parsing information about a peer from a network message. +#[derive(Debug)] +pub enum PeerParseError { + /// Failed to parse the identity of the peer. + PeerIdParseError, + /// Failed to parse one of the multiaddresses for the peer. + MultiaddrParseError(MultiaddrError), +} + +impl fmt::Display for PeerParseError { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + PeerParseError::PeerIdParseError => write!(f, "Error while parsing the peer id"), + PeerParseError::MultiaddrParseError(ref err) => { + write!(f, "Error while parsing a multiaddress: {}", err) + } + } + } +} + +impl error::Error for PeerParseError { + fn source(&self) -> Option<&(dyn error::Error + 'static)> { + match self { + PeerParseError::PeerIdParseError => None, + PeerParseError::MultiaddrParseError(ref err) => Some(err), + } + } +} + +/// A [`NegotiatedSubstream`] acting as a relayed [`Connection`]. +#[derive(Debug)] +pub struct Connection { + /// [`Connection`] might at first return data, that was already read during relay negotiation. + initial_data: Bytes, + stream: NegotiatedSubstream, + /// Notifies the other side of the channel of this [`Connection`] being dropped. + _notifier: oneshot::Sender<()>, +} + +impl Unpin for Connection {} + +impl Connection { + fn new( + initial_data: Bytes, + stream: NegotiatedSubstream, + notifier: oneshot::Sender<()>, + ) -> Self { + Connection { + initial_data, + stream, + + _notifier: notifier, + } + } +} + +impl AsyncWrite for Connection { + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context, + buf: &[u8], + ) -> Poll> { + Pin::new(&mut self.stream).poll_write(cx, buf) + } + fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.stream).poll_flush(cx) + } + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + Pin::new(&mut self.stream).poll_close(cx) + } + + fn poll_write_vectored( + mut self: Pin<&mut Self>, + cx: &mut Context, + bufs: &[IoSlice], + ) -> Poll> { + Pin::new(&mut self.stream).poll_write_vectored(cx, bufs) + } +} + +impl AsyncRead for Connection { + fn poll_read( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &mut [u8], + ) -> Poll> { + if !self.initial_data.is_empty() { + let n = std::cmp::min(self.initial_data.len(), buf.len()); + let data = self.initial_data.split_to(n); + buf[0..n].copy_from_slice(&data[..]); + return Poll::Ready(Ok(n)); + } + + Pin::new(&mut self.stream).poll_read(cx, buf) + } +} diff --git a/protocols/relay/src/protocol/copy_future.rs b/protocols/relay/src/protocol/copy_future.rs new file mode 100644 index 00000000000..5b6b9a50c82 --- /dev/null +++ b/protocols/relay/src/protocol/copy_future.rs @@ -0,0 +1,137 @@ +// 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. + +//! Helper to interconnect two substreams, connecting the receiver side of A with the sender side of +//! B and vice versa. +//! +//! Inspired by [`futures::io::Copy`]. + +use futures::future::Future; +use futures::future::FutureExt; +use futures::io::{AsyncBufRead, BufReader}; +use futures::io::{AsyncRead, AsyncWrite}; +use futures::ready; +use futures_timer::Delay; +use std::io; +use std::pin::Pin; +use std::task::{Context, Poll}; +use std::time::Duration; + +pub struct CopyFuture { + src: BufReader, + dst: BufReader, + + active_timeout: Delay, + configured_timeout: Duration, +} + +impl CopyFuture { + pub fn new(src: S, dst: D, timeout: Duration) -> Self { + CopyFuture { + src: BufReader::new(src), + dst: BufReader::new(dst), + active_timeout: Delay::new(timeout), + configured_timeout: timeout, + } + } +} + +impl Future for CopyFuture +where + S: AsyncRead + AsyncWrite + Unpin, + D: AsyncRead + AsyncWrite + Unpin, +{ + type Output = io::Result<()>; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = &mut *self; + + let mut reset_timer = false; + + loop { + enum Status { + Pending, + Done, + Progressed, + } + + let src_status = match forward_data(&mut this.src, &mut this.dst, cx) { + Poll::Ready(Err(e)) => return Poll::Ready(Err(e)), + Poll::Ready(Ok(true)) => Status::Done, + Poll::Ready(Ok(false)) => Status::Progressed, + Poll::Pending => Status::Pending, + }; + + let dst_status = match forward_data(&mut this.dst, &mut this.src, cx) { + Poll::Ready(Err(e)) => return Poll::Ready(Err(e)), + Poll::Ready(Ok(true)) => Status::Done, + Poll::Ready(Ok(false)) => Status::Progressed, + Poll::Pending => Status::Pending, + }; + + match (src_status, dst_status) { + // Both source and destination are done sending data. + (Status::Done, Status::Done) => return Poll::Ready(Ok(())), + // Either source or destination made progress, thus reset timer. + (Status::Progressed, _) | (_, Status::Progressed) => reset_timer = true, + // Both are pending. Check if timer fired, otherwise return Poll::Pending. + (Status::Pending, Status::Pending) => break, + // One is done sending data, the other is pending. Check if timer fired, otherwise + // return Poll::Pending. + (Status::Pending, Status::Done) | (Status::Done, Status::Pending) => break, + } + } + + if reset_timer { + this.active_timeout = Delay::new(this.configured_timeout); + } + + if let Poll::Ready(()) = this.active_timeout.poll_unpin(cx) { + return Poll::Ready(Err(io::ErrorKind::TimedOut.into())); + } + + Poll::Pending + } +} + +/// Forwards data from `source` to `destination`. +/// +/// Returns `true` when done, i.e. `source` having reached EOF, returns false otherwise, thus +/// indicating progress. +fn forward_data( + mut src: &mut S, + mut dst: &mut D, + cx: &mut Context<'_>, +) -> Poll> { + let buffer = ready!(Pin::new(&mut src).poll_fill_buf(cx))?; + if buffer.is_empty() { + ready!(Pin::new(&mut dst).poll_flush(cx))?; + ready!(Pin::new(&mut dst).poll_close(cx))?; + return Poll::Ready(Ok(true)); + } + + let i = ready!(Pin::new(dst).poll_write(cx, buffer))?; + if i == 0 { + return Poll::Ready(Err(io::ErrorKind::WriteZero.into())); + } + Pin::new(src).consume(i); + + Poll::Ready(Ok(false)) +} diff --git a/protocols/relay/src/protocol/incoming_dst_req.rs b/protocols/relay/src/protocol/incoming_dst_req.rs new file mode 100644 index 00000000000..b3b0ded9de6 --- /dev/null +++ b/protocols/relay/src/protocol/incoming_dst_req.rs @@ -0,0 +1,140 @@ +// 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 crate::message_proto::{circuit_relay, CircuitRelay}; +use crate::protocol::Peer; + +use asynchronous_codec::{Framed, FramedParts}; +use bytes::BytesMut; +use futures::{future::BoxFuture, prelude::*}; +use futures::channel::oneshot; +use libp2p_core::{Multiaddr, PeerId}; +use libp2p_swarm::NegotiatedSubstream; +use prost::Message; +use std::io; +use unsigned_varint::codec::UviBytes; + +/// Request from a remote for us to become a destination. +/// +/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*, and +/// we are the *destination*, this struct is a message that the *relay* sent to us. The +/// parameters passed to `IncomingDstReq::new()` are the information of the *source*. +/// +/// If the upgrade succeeds, the substream is returned and we will receive data sent from the +/// source on it. +#[must_use = "An incoming destination request should be either accepted or denied"] +pub struct IncomingDstReq { + /// The stream to the source. + stream: Framed, + /// Source of the request. + src: Peer, +} + +impl IncomingDstReq +{ + /// Creates a `IncomingDstReq`. + pub(crate) fn new(stream: Framed, src: Peer) -> Self { + IncomingDstReq { + stream: stream, + src, + } + } + + /// Returns the peer id of the source that is being relayed. + pub fn src_id(&self) -> &PeerId { + &self.src.peer_id + } + + /// Returns the addresses of the source that is being relayed. + pub fn src_addrs(&self) -> impl Iterator { + self.src.addrs.iter() + } + + /// Accepts the request. + /// + /// The returned `Future` sends back a success message then returns the raw stream. This raw + /// stream then points to the source (as retreived with `src_id()` and `src_addrs()`). + pub fn accept( + self, + ) -> BoxFuture<'static, Result<(PeerId, super::Connection, oneshot::Receiver<()>), IncomingDstReqError>> { + let IncomingDstReq { mut stream, src } = self; + let msg = CircuitRelay { + r#type: Some(circuit_relay::Type::Status.into()), + src_peer: None, + dst_peer: None, + code: Some(circuit_relay::Status::Success.into()), + }; + let mut msg_bytes = BytesMut::new(); + msg.encode(&mut msg_bytes) + .expect("all the mandatory fields are always filled; QED"); + + async move { + stream.send(msg_bytes.freeze()).await?; + + let FramedParts { + io, + read_buffer, + write_buffer, + .. + } = stream.into_parts(); + assert!( + write_buffer.is_empty(), + "Expect a flushed Framed to have empty write buffer." + ); + + let (tx, rx) = oneshot::channel(); + + Ok((src.peer_id, super::Connection::new(read_buffer.freeze(), io, tx), rx)) + } + .boxed() + } + + /// Refuses the request. + /// + /// The returned `Future` gracefully shuts down the request. + pub fn deny(mut self) -> BoxFuture<'static, Result<(), io::Error>> { + let msg = CircuitRelay { + r#type: Some(circuit_relay::Type::Status.into()), + src_peer: None, + dst_peer: None, + code: Some(circuit_relay::Status::StopRelayRefused.into()), + }; + let mut msg_bytes = BytesMut::new(); + msg.encode(&mut msg_bytes) + .expect("all the mandatory fields are always filled; QED"); + + async move { + self.stream.send(msg_bytes.freeze()).await?; + Ok(()) + } + .boxed() + } +} + +#[derive(Debug)] +pub enum IncomingDstReqError { + Io(std::io::Error), +} + +impl From for IncomingDstReqError { + fn from(e: std::io::Error) -> Self { + IncomingDstReqError::Io(e) + } +} diff --git a/protocols/relay/src/protocol/incoming_relay_req.rs b/protocols/relay/src/protocol/incoming_relay_req.rs new file mode 100644 index 00000000000..6f585db2854 --- /dev/null +++ b/protocols/relay/src/protocol/incoming_relay_req.rs @@ -0,0 +1,156 @@ +// 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 super::copy_future::CopyFuture; +use crate::message_proto::{circuit_relay, circuit_relay::Status, CircuitRelay}; +use crate::protocol::Peer; + +use asynchronous_codec::{Framed, FramedParts}; +use bytes::{BytesMut, Bytes}; +use futures::channel::oneshot; +use futures::future::BoxFuture; +use futures::prelude::*; +use libp2p_swarm::NegotiatedSubstream; +use prost::Message; +use std::time::Duration; +use unsigned_varint::codec::UviBytes; + +/// Request from a remote for us to relay communications to another node. +/// +/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*, and +/// we are the *relay*, this struct is a message that the *source* sent to us. The parameters +/// passed to `IncomingRelayReq::new()` are the information of the *destination*. +/// +/// If the upgrade succeeds, the substream is returned and we will receive data sent from the +/// source on it. This data must be transmitted to the destination. +#[must_use = "An incoming relay request should be either accepted or denied."] +pub struct IncomingRelayReq { + /// The stream to the source. + stream: Framed, + /// Target of the request. + dest: Peer, + + _notifier: oneshot::Sender<()>, +} + +impl IncomingRelayReq +{ + /// Creates a [`IncomingRelayReq`] as well as a Future that resolves once the + /// [`IncomingRelayReq`] is dropped. + pub(crate) fn new( + stream: Framed, + dest: Peer, + ) -> (Self, oneshot::Receiver<()>) { + let (tx, rx) = oneshot::channel(); + ( + IncomingRelayReq { + stream, + dest, + _notifier: tx, + }, + rx, + ) + } + + /// Peer id of the node we should relay communications to. + pub(crate) fn dst_peer(&self) -> &Peer { + &self.dest + } + + /// Accepts the request by providing a stream to the destination. + pub fn fulfill( + mut self, + dst_stream: TDestSubstream, + dst_read_buffer: Bytes, + ) -> BoxFuture<'static, Result<(), IncomingRelayReqError>> + where + TDestSubstream: AsyncRead + AsyncWrite + Send + Unpin + 'static, + { + let msg = CircuitRelay { + r#type: Some(circuit_relay::Type::Status.into()), + src_peer: None, + dst_peer: None, + code: Some(circuit_relay::Status::Success.into()), + }; + let mut msg_bytes = BytesMut::new(); + msg.encode(&mut msg_bytes) + .expect("all the mandatory fields are always filled; QED"); + + async move { + self.stream.send(msg_bytes.freeze()).await?; + + let FramedParts { + mut io, + read_buffer, + write_buffer, + .. + } = self.stream.into_parts(); + assert!( + read_buffer.is_empty(), + "Expect a Framed, that was never actively read from, not to read." + ); + assert!( + write_buffer.is_empty(), + "Expect a flushed Framed to have empty write buffer." + ); + + if !dst_read_buffer.is_empty() { + io.write_all(&dst_read_buffer).await?; + } + + let copy_future = CopyFuture::new(io, dst_stream, Duration::from_secs(5)); + + copy_future.await.map_err(Into::into) + } + .boxed() + } + + /// Refuses the request. + /// + /// The returned `Future` gracefully shuts down the request. + pub fn deny(mut self, err_code: Status) -> BoxFuture<'static, Result<(), std::io::Error>> { + let msg = CircuitRelay { + r#type: Some(circuit_relay::Type::Status.into()), + code: Some(err_code.into()), + src_peer: None, + dst_peer: None, + }; + let mut msg_bytes = BytesMut::new(); + msg.encode(&mut msg_bytes) + .expect("all the mandatory fields are always filled; QED"); + + async move { + self.stream.send(msg_bytes.freeze()).await?; + Ok(()) + } + .boxed() + } +} + +#[derive(Debug)] +pub enum IncomingRelayReqError { + Io(std::io::Error), +} + +impl From for IncomingRelayReqError { + fn from(e: std::io::Error) -> Self { + IncomingRelayReqError::Io(e) + } +} diff --git a/protocols/relay/src/protocol/listen.rs b/protocols/relay/src/protocol/listen.rs new file mode 100644 index 00000000000..70a084640f8 --- /dev/null +++ b/protocols/relay/src/protocol/listen.rs @@ -0,0 +1,177 @@ +// 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 crate::message_proto::{circuit_relay, CircuitRelay}; +use crate::protocol::incoming_dst_req::IncomingDstReq; +use crate::protocol::incoming_relay_req::IncomingRelayReq; +use crate::protocol::{Peer, PeerParseError, MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME}; +use asynchronous_codec::Framed; +use futures::channel::oneshot; +use futures::{future::BoxFuture, prelude::*}; +use libp2p_core::upgrade; +use libp2p_swarm::NegotiatedSubstream; +use prost::Message; + +use std::io::Cursor; +use std::{convert::TryFrom, error, fmt, iter}; +use unsigned_varint::codec::UviBytes; + +/// Configuration for an inbound upgrade that handles requests from the remote for the relay +/// protocol. +#[derive(Debug, Clone)] +pub struct RelayListen {} + +/// Outcome of the listening. +pub enum RelayRemoteReq { + /// We have been asked to become a destination. + DstReq(IncomingDstReq), + /// We have been asked to relay communications to another node. + RelayReq((IncomingRelayReq, oneshot::Receiver<()>)), +} + +impl RelayListen { + /// Builds a new `RelayListen` with default options. + pub fn new() -> RelayListen { + RelayListen {} + } +} + +impl upgrade::UpgradeInfo for RelayListen { + type Info = &'static [u8]; + type InfoIter = iter::Once; + + fn protocol_info(&self) -> Self::InfoIter { + iter::once(PROTOCOL_NAME) + } +} + +impl upgrade::InboundUpgrade for RelayListen { + type Output = RelayRemoteReq; + type Error = RelayListenError; + type Future = BoxFuture<'static, Result>; + + fn upgrade_inbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future { + async move { + let mut codec = UviBytes::::default(); + codec.set_max_len(MAX_ACCEPTED_MESSAGE_LEN); + let mut substream = Framed::new(substream, codec); + + let msg: bytes::BytesMut = substream + .next() + .await + .ok_or(std::io::Error::new(std::io::ErrorKind::UnexpectedEof, ""))??; + let CircuitRelay { + r#type, + src_peer, + dst_peer, + code: _, + } = CircuitRelay::decode(Cursor::new(msg))?; + + match circuit_relay::Type::from_i32(r#type.ok_or(RelayListenError::NoMessageType)?) + .ok_or(RelayListenError::InvalidMessageTy)? + { + circuit_relay::Type::Hop => { + let peer = Peer::try_from(dst_peer.ok_or(RelayListenError::NoDstPeer)?)?; + let (rq, notifyee) = IncomingRelayReq::new(substream, peer); + Ok(RelayRemoteReq::RelayReq((rq, notifyee))) + } + circuit_relay::Type::Stop => { + let peer = Peer::try_from(src_peer.ok_or(RelayListenError::NoSrcPeer)?)?; + let rq = IncomingDstReq::new(substream, peer); + Ok(RelayRemoteReq::DstReq(rq)) + } + _ => Err(RelayListenError::InvalidMessageTy), + } + } + .boxed() + } +} + +/// Error while upgrading with a [`RelayListen`]. +#[derive(Debug)] +pub enum RelayListenError { + Decode(prost::DecodeError), + Io(std::io::Error), + NoSrcPeer, + NoDstPeer, + ParsePeer(PeerParseError), + NoMessageType, + InvalidMessageTy, +} + +impl From for RelayListenError { + fn from(err: prost::DecodeError) -> Self { + RelayListenError::Decode(err) + } +} + +impl From for RelayListenError { + fn from(e: std::io::Error) -> Self { + RelayListenError::Io(e) + } +} + +impl From for RelayListenError { + fn from(err: PeerParseError) -> Self { + RelayListenError::ParsePeer(err) + } +} + +impl fmt::Display for RelayListenError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + RelayListenError::Decode(e) => { + write!(f, "Failed to decode response: {}.", e) + } + RelayListenError::Io(e) => { + write!(f, "Io error {}", e) + } + RelayListenError::NoSrcPeer => { + write!(f, "Expected source peer id") + } + RelayListenError::NoDstPeer => { + write!(f, "Expected destination peer id") + } + RelayListenError::ParsePeer(e) => { + write!(f, "Failed to parse peer field: {}", e) + } + RelayListenError::NoMessageType => { + write!(f, "Expected message type to be set.") + } + RelayListenError::InvalidMessageTy => { + write!(f, "Invalid message type") + } + } + } +} + +impl error::Error for RelayListenError { + fn source(&self) -> Option<&(dyn error::Error + 'static)> { + match self { + RelayListenError::Decode(e) => Some(e), + RelayListenError::Io(e) => Some(e), + RelayListenError::NoSrcPeer => None, + RelayListenError::NoDstPeer => None, + RelayListenError::ParsePeer(_) => None, + RelayListenError::NoMessageType => None, + RelayListenError::InvalidMessageTy => None, + } + } +} diff --git a/protocols/relay/src/protocol/outgoing_dst_req.rs b/protocols/relay/src/protocol/outgoing_dst_req.rs new file mode 100644 index 00000000000..0a257da3bab --- /dev/null +++ b/protocols/relay/src/protocol/outgoing_dst_req.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. + +use crate::message_proto::{circuit_relay, CircuitRelay}; +use crate::protocol::{Peer, MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME}; +use asynchronous_codec::{Framed, FramedParts}; +use bytes::Bytes; +use futures::future::BoxFuture; +use futures::prelude::*; +use libp2p_core::{upgrade, Multiaddr, PeerId}; +use libp2p_swarm::NegotiatedSubstream; +use prost::Message; +use std::{fmt, error, iter}; +use unsigned_varint::codec::UviBytes; + +/// Ask the remote to become a destination. The upgrade succeeds if the remote accepts, and fails +/// if the remote refuses. +/// +/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*, +/// this struct is the message that the *relay* sends to the *destination* at initialization. The +/// parameters passed to `OutgoingDstReq::new()` are the information of the *source* and the +/// *destination* (not the information of the *relay*). +/// +/// The upgrade should be performed on a substream to the *destination*. +/// +/// If the upgrade succeeds, the substream is returned and we must link it with the data sent from +/// the source. +#[derive(Debug, Clone)] +pub struct OutgoingDstReq { + /// The message to send to the destination. Pre-computed. + message: Vec, +} + +impl OutgoingDstReq { + /// Creates a `OutgoingDstReq`. Must pass the parameters of the message. + pub(crate) fn new(src_id: PeerId, src_addr: Multiaddr, dst_peer: Peer) -> Self { + let message = CircuitRelay { + r#type: Some(circuit_relay::Type::Stop.into()), + src_peer: Some(circuit_relay::Peer { + id: src_id.to_bytes(), + addrs: vec![src_addr.to_vec()], + }), + dst_peer: Some(circuit_relay::Peer { + id: dst_peer.peer_id.to_bytes(), + addrs: dst_peer.addrs.into_iter().map(|a| a.to_vec()).collect(), + }), + code: None, + }; + let mut encoded_msg = Vec::new(); + message + .encode(&mut encoded_msg) + .expect("all the mandatory fields are always filled; QED"); + + OutgoingDstReq { + message: encoded_msg, + } + } +} + +impl upgrade::UpgradeInfo for OutgoingDstReq { + type Info = &'static [u8]; + type InfoIter = iter::Once; + + fn protocol_info(&self) -> Self::InfoIter { + iter::once(PROTOCOL_NAME) + } +} + +impl upgrade::OutboundUpgrade for OutgoingDstReq { + type Output = (NegotiatedSubstream, Bytes); + type Error = OutgoingDstReqError; + type Future = BoxFuture<'static, Result>; + + fn upgrade_outbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future { + let mut codec = UviBytes::default(); + codec.set_max_len(MAX_ACCEPTED_MESSAGE_LEN); + + let mut substream = Framed::new(substream, codec); + + async move { + substream.send(std::io::Cursor::new(self.message)).await?; + let msg = + substream + .next() + .await + .ok_or(OutgoingDstReqError::Io(std::io::Error::new( + std::io::ErrorKind::UnexpectedEof, + "", + )))??; + + let msg = std::io::Cursor::new(msg); + let CircuitRelay { + r#type, + src_peer, + dst_peer, + code, + } = CircuitRelay::decode(msg)?; + + match r#type + .map(circuit_relay::Type::from_i32) + .flatten() + .ok_or(OutgoingDstReqError::ParseTypeField)? + { + circuit_relay::Type::Status => {} + s => return Err(OutgoingDstReqError::ExpectedStatusType(s)), + } + + if src_peer.is_some() { + return Err(OutgoingDstReqError::UnexpectedSrcPeerWithStatusType); + } + if dst_peer.is_some() { + return Err(OutgoingDstReqError::UnexpectedDstPeerWithStatusType); + } + + match code + .map(circuit_relay::Status::from_i32) + .flatten() + .ok_or(OutgoingDstReqError::ParseStatusField)? + { + circuit_relay::Status::Success => {} + s => return Err(OutgoingDstReqError::ExpectedSuccessStatus(s)), + } + + let FramedParts { + io, + read_buffer, + write_buffer, + .. + } = substream.into_parts(); + assert!( + write_buffer.is_empty(), + "Expect a flushed Framed to have an empty write buffer." + ); + + Ok((io, read_buffer.freeze())) + } + .boxed() + } +} + +#[derive(Debug)] +pub enum OutgoingDstReqError { + Decode(prost::DecodeError), + Io(std::io::Error), + ParseTypeField, + ParseStatusField, + ExpectedStatusType(circuit_relay::Type), + ExpectedSuccessStatus(circuit_relay::Status), + UnexpectedSrcPeerWithStatusType, + UnexpectedDstPeerWithStatusType, +} + +impl From for OutgoingDstReqError { + fn from(e: std::io::Error) -> Self { + OutgoingDstReqError::Io(e) + } +} + +impl From for OutgoingDstReqError { + fn from(e: prost::DecodeError) -> Self { + OutgoingDstReqError::Decode(e) + } +} + +impl fmt::Display for OutgoingDstReqError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + OutgoingDstReqError::Decode(e) => { + write!(f, "Failed to decode response: {}.", e) + } + OutgoingDstReqError::Io(e) => { + write!(f, "Io error {}", e) + } + OutgoingDstReqError::ParseTypeField => { + write!(f, "Failed to parse response type field.") + } + OutgoingDstReqError::ParseStatusField => { + write!(f, "Failed to parse response status field.") + } + OutgoingDstReqError::ExpectedStatusType(t) => { + write!(f, "Expected status message type, but got {:?}", t) + } + OutgoingDstReqError::UnexpectedSrcPeerWithStatusType => { + write!(f, "Unexpected source peer with status type.") + } + OutgoingDstReqError::UnexpectedDstPeerWithStatusType => { + write!(f, "Unexpected destination peer with status type.") + } + OutgoingDstReqError::ExpectedSuccessStatus(s) => { + write!(f, "Expected success status but got {:?}", s) + } + } + } +} + +impl error::Error for OutgoingDstReqError { + fn source(&self) -> Option<&(dyn error::Error + 'static)> { + match self { + OutgoingDstReqError::Decode(e) => Some(e), + OutgoingDstReqError::Io(e) => Some(e), + OutgoingDstReqError::ParseTypeField => None, + OutgoingDstReqError::ParseStatusField => None, + OutgoingDstReqError::ExpectedStatusType(_) => None, + OutgoingDstReqError::UnexpectedSrcPeerWithStatusType => None, + OutgoingDstReqError::UnexpectedDstPeerWithStatusType => None, + OutgoingDstReqError::ExpectedSuccessStatus(_) => None, + } + } +} diff --git a/protocols/relay/src/protocol/outgoing_relay_req.rs b/protocols/relay/src/protocol/outgoing_relay_req.rs new file mode 100644 index 00000000000..8f6ed8b3aa3 --- /dev/null +++ b/protocols/relay/src/protocol/outgoing_relay_req.rs @@ -0,0 +1,235 @@ +// 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 crate::message_proto::{circuit_relay, CircuitRelay}; +use crate::protocol::{MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME}; +use asynchronous_codec::{Framed, FramedParts}; +use futures::channel::oneshot; +use futures::future::BoxFuture; +use futures::prelude::*; +use libp2p_core::{upgrade, Multiaddr, PeerId}; +use libp2p_swarm::NegotiatedSubstream; +use prost::Message; +use std::{error, fmt, iter}; +use unsigned_varint::codec::UviBytes; + +/// Ask a remote to act as a relay. +/// +/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*, +/// this struct is the message that the *source* sends to the *relay* at initialization. The +/// parameters passed to `OutgoingRelayReq::new()` are the information of the *destination* +/// (not the information of the *relay*). +/// +/// The upgrade should be performed on a substream to the *relay*. +/// +/// If the upgrade succeeds, the substream is returned and is now a brand new connection pointing +/// to the *destination*. +pub struct OutgoingRelayReq { + src_id: PeerId, + dst_id: PeerId, + dst_address: Option, +} + +impl OutgoingRelayReq { + /// Builds a request for the target to act as a relay to a third party. + pub fn new(src_id: PeerId, dst_id: PeerId, dst_address: Option) -> Self { + OutgoingRelayReq { + src_id, + dst_id, + dst_address, + } + } +} + +impl upgrade::UpgradeInfo for OutgoingRelayReq { + type Info = &'static [u8]; + type InfoIter = iter::Once; + + fn protocol_info(&self) -> Self::InfoIter { + iter::once(PROTOCOL_NAME) + } +} + +impl upgrade::OutboundUpgrade for OutgoingRelayReq { + type Output = (super::Connection, oneshot::Receiver<()>); + type Error = OutgoingRelayReqError; + type Future = BoxFuture<'static, Result>; + + fn upgrade_outbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future { + let OutgoingRelayReq { + src_id, + dst_id, + dst_address, + } = self; + + let message = CircuitRelay { + r#type: Some(circuit_relay::Type::Hop.into()), + src_peer: Some(circuit_relay::Peer { + id: src_id.to_bytes(), + addrs: vec![], + }), + dst_peer: Some(circuit_relay::Peer { + id: dst_id.to_bytes(), + addrs: vec![dst_address.unwrap_or(Multiaddr::empty()).to_vec()], + }), + code: None, + }; + let mut encoded = Vec::new(); + message + .encode(&mut encoded) + .expect("all the mandatory fields are always filled; QED"); + + let mut codec = UviBytes::default(); + codec.set_max_len(MAX_ACCEPTED_MESSAGE_LEN); + + let mut substream = Framed::new(substream, codec); + + async move { + substream.send(std::io::Cursor::new(encoded)).await?; + let msg = + substream + .next() + .await + .ok_or(OutgoingRelayReqError::Io(std::io::Error::new( + std::io::ErrorKind::UnexpectedEof, + "", + )))??; + + let msg = std::io::Cursor::new(msg); + let CircuitRelay { + r#type, + src_peer, + dst_peer, + code, + } = CircuitRelay::decode(msg)?; + + match r#type + .map(circuit_relay::Type::from_i32) + .flatten() + .ok_or(OutgoingRelayReqError::ParseTypeField)? + { + circuit_relay::Type::Status => {} + s => return Err(OutgoingRelayReqError::ExpectedStatusType(s)), + } + + match code + .map(circuit_relay::Status::from_i32) + .flatten() + .ok_or(OutgoingRelayReqError::ParseStatusField)? + { + circuit_relay::Status::Success => {} + e => return Err(OutgoingRelayReqError::ExpectedSuccessStatus(e)), + } + + if src_peer.is_some() { + return Err(OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType); + } + if dst_peer.is_some() { + return Err(OutgoingRelayReqError::UnexpectedDstPeerWithStatusType); + } + + let FramedParts { + io, + read_buffer, + write_buffer, + .. + } = substream.into_parts(); + assert!( + write_buffer.is_empty(), + "Expect a flushed Framed to have empty write buffer." + ); + + let (tx, rx) = oneshot::channel(); + + Ok((super::Connection::new(read_buffer.freeze(), io, tx), rx)) + } + .boxed() + } +} + +#[derive(Debug)] +pub enum OutgoingRelayReqError { + Decode(prost::DecodeError), + Io(std::io::Error), + ParseTypeField, + ParseStatusField, + ExpectedStatusType(circuit_relay::Type), + UnexpectedSrcPeerWithStatusType, + UnexpectedDstPeerWithStatusType, + ExpectedSuccessStatus(circuit_relay::Status), +} + +impl From for OutgoingRelayReqError { + fn from(e: std::io::Error) -> Self { + OutgoingRelayReqError::Io(e) + } +} + +impl From for OutgoingRelayReqError { + fn from(e: prost::DecodeError) -> Self { + OutgoingRelayReqError::Decode(e) + } +} + +impl fmt::Display for OutgoingRelayReqError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + OutgoingRelayReqError::Decode(e) => { + write!(f, "Failed to decode response: {}.", e) + } + OutgoingRelayReqError::Io(e) => { + write!(f, "Io error {}", e) + } + OutgoingRelayReqError::ParseTypeField => { + write!(f, "Failed to parse response type field.") + } + OutgoingRelayReqError::ParseStatusField => { + write!(f, "Failed to parse response status field.") + } + OutgoingRelayReqError::ExpectedStatusType(t) => { + write!(f, "Expected status message type, but got {:?}", t) + } + OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType => { + write!(f, "Unexpected source peer with status type.") + } + OutgoingRelayReqError::UnexpectedDstPeerWithStatusType => { + write!(f, "Unexpected destination peer with status type.") + } + OutgoingRelayReqError::ExpectedSuccessStatus(s) => { + write!(f, "Expected success status but got {:?}", s) + } + } + } +} + +impl error::Error for OutgoingRelayReqError { + fn source(&self) -> Option<&(dyn error::Error + 'static)> { + match self { + OutgoingRelayReqError::Decode(e) => Some(e), + OutgoingRelayReqError::Io(e) => Some(e), + OutgoingRelayReqError::ParseTypeField => None, + OutgoingRelayReqError::ParseStatusField => None, + OutgoingRelayReqError::ExpectedStatusType(_) => None, + OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType => None, + OutgoingRelayReqError::UnexpectedDstPeerWithStatusType => None, + OutgoingRelayReqError::ExpectedSuccessStatus(_) => None, + } + } +} diff --git a/protocols/relay/src/transport.rs b/protocols/relay/src/transport.rs new file mode 100644 index 00000000000..1d5ff1bf57b --- /dev/null +++ b/protocols/relay/src/transport.rs @@ -0,0 +1,559 @@ +// 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. + +use crate::behaviour::{BehaviourToListenerMsg, OutgoingRelayReqError}; +use crate::protocol; +use crate::RequestId; +use futures::channel::mpsc; +use futures::channel::oneshot; +use futures::future::{BoxFuture, Future, FutureExt}; +use futures::sink::SinkExt; +use futures::stream::{Stream, StreamExt}; +use libp2p_core::either::{EitherError, EitherFuture, EitherOutput}; +use libp2p_core::multiaddr::{Multiaddr, Protocol}; +use libp2p_core::transport::{ListenerEvent, TransportError}; +use libp2p_core::{PeerId, Transport}; +use pin_project::pin_project; +use std::pin::Pin; +use std::task::{Context, Poll}; + +/// A [`Transport`] wrapping another [`Transport`] enabling relay capabilities. +/// +/// Allows the local node to: +/// +/// 1. Use inner wrapped transport as before. +/// +/// ``` +/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, Transport}; +/// # use libp2p_core::transport::memory::MemoryTransport; +/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour}; +/// # let inner_transport = MemoryTransport::default(); +/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour( +/// # RelayConfig::default(), +/// # inner_transport, +/// # ); +/// relay_transport.dial(Multiaddr::empty().with(Protocol::Memory(42))); +/// ``` +/// +/// 2. Establish relayed connections by dialing `/p2p-circuit` addresses. +/// +/// ``` +/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, PeerId, Transport}; +/// # use libp2p_core::transport::memory::MemoryTransport; +/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour}; +/// # let inner_transport = MemoryTransport::default(); +/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour( +/// # RelayConfig::default(), +/// # inner_transport, +/// # ); +/// let dst_addr_via_relay = Multiaddr::empty() +/// .with(Protocol::Memory(40)) // Relay address. +/// .with(Protocol::P2p(PeerId::random().into())) // Relay peer id. +/// .with(Protocol::P2pCircuit) // Signal to connect via relay and not directly. +/// .with(Protocol::Memory(42)) // Destination address. +/// .with(Protocol::P2p(PeerId::random().into())); // Destination peer id. +/// relay_transport.dial(dst_addr_via_relay).unwrap(); +/// ``` +/// +/// 3. Listen for incoming relayed connections via specific relay. +/// +/// ``` +/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, PeerId, Transport}; +/// # use libp2p_core::transport::memory::MemoryTransport; +/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour}; +/// # let inner_transport = MemoryTransport::default(); +/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour( +/// # RelayConfig::default(), +/// # inner_transport, +/// # ); +/// let relay_addr = Multiaddr::empty() +/// .with(Protocol::Memory(40)) // Relay address. +/// .with(Protocol::P2p(PeerId::random().into())) // Relay peer id. +/// .with(Protocol::P2pCircuit); // Signal to listen via remote relay node. +/// relay_transport.listen_on(relay_addr).unwrap(); +/// ``` +/// +/// 4. Listen for incoming relayed connections via any relay. +/// +/// Note: Without this listener, incoming relayed connections from relays, that the local node is +/// not explicitly listening via, are dropped. +/// +/// ``` +/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, PeerId, Transport}; +/// # use libp2p_core::transport::memory::MemoryTransport; +/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour}; +/// # let inner_transport = MemoryTransport::default(); +/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour( +/// # RelayConfig::default(), +/// # inner_transport, +/// # ); +/// let addr = Multiaddr::empty() +/// .with(Protocol::P2pCircuit); // Signal to listen via any relay. +/// relay_transport.listen_on(addr).unwrap(); +/// ``` +#[derive(Clone)] +pub struct RelayTransport { + to_behaviour: mpsc::Sender, + + inner_transport: T, +} + +impl RelayTransport { + /// Create a new [`RelayTransport`] by wrapping an existing [`Transport`] in a + /// [`RelayTransport`]. + /// + ///``` + /// # use libp2p_core::transport::dummy::DummyTransport; + /// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour}; + /// + /// let inner_transport = DummyTransport::<()>::new(); + /// let (relay_transport, relay_behaviour) = new_transport_and_behaviour( + /// RelayConfig::default(), + /// inner_transport, + /// ); + ///``` + pub(crate) fn new(t: T) -> (Self, mpsc::Receiver) { + let (to_behaviour, from_transport) = mpsc::channel(0); + + let transport = RelayTransport { + to_behaviour, + + inner_transport: t, + }; + + (transport, from_transport) + } +} + +impl Transport for RelayTransport { + type Output = EitherOutput<::Output, protocol::Connection>; + type Error = EitherError<::Error, RelayError>; + type Listener = RelayListener; + type ListenerUpgrade = RelayedListenerUpgrade; + type Dial = EitherFuture<::Dial, RelayedDial>; + + fn listen_on(self, addr: Multiaddr) -> Result> { + let orig_addr = addr.clone(); + + match parse_relayed_multiaddr(addr)? { + // Address does not contain circuit relay protocol. Use inner transport. + Err(addr) => { + let inner_listener = match self.inner_transport.listen_on(addr) { + Ok(listener) => listener, + Err(TransportError::MultiaddrNotSupported(addr)) => { + return Err(TransportError::MultiaddrNotSupported(addr)) + } + Err(TransportError::Other(err)) => { + return Err(TransportError::Other(EitherError::A(err))) + } + }; + Ok(RelayListener::Inner(inner_listener)) + } + // Address does contain circuit relay protocol. Use relayed listener. + Ok(relayed_addr) => { + let relay_peer_id_and_addr = match relayed_addr { + // TODO: In the future we might want to support listening via a relay by its + // address only. + RelayedMultiaddr { + relay_peer_id: None, + relay_addr: Some(_), + .. + } => return Err(RelayError::MissingRelayPeerId.into()), + // TODO: In the future we might want to support listening via a relay by its + // peer_id only. + RelayedMultiaddr { + relay_peer_id: Some(_), + relay_addr: None, + .. + } => return Err(RelayError::MissingRelayAddr.into()), + // Listen for incoming relayed connections via specific relay. + RelayedMultiaddr { + relay_peer_id: Some(peer_id), + relay_addr: Some(addr), + .. + } => Some((peer_id, addr)), + // Listen for incoming relayed connections via any relay. + RelayedMultiaddr { + relay_peer_id: None, + relay_addr: None, + .. + } => None, + }; + + let (to_listener, from_behaviour) = mpsc::channel(0); + let mut to_behaviour = self.to_behaviour.clone(); + let msg_to_behaviour = Some( + async move { + to_behaviour + .send(TransportToBehaviourMsg::ListenReq { + relay_peer_id_and_addr, + to_listener, + }) + .await + } + .boxed(), + ); + + Ok(RelayListener::Relayed { + from_behaviour, + msg_to_behaviour, + report_listen_addr: Some(orig_addr), + }) + } + } + } + + fn dial(self, addr: Multiaddr) -> Result> { + match parse_relayed_multiaddr(addr)? { + // Address does not contain circuit relay protocol. Use inner transport. + Err(addr) => match self.inner_transport.dial(addr) { + Ok(dialer) => Ok(EitherFuture::First(dialer)), + Err(TransportError::MultiaddrNotSupported(addr)) => { + Err(TransportError::MultiaddrNotSupported(addr)) + } + Err(TransportError::Other(err)) => Err(TransportError::Other(EitherError::A(err))), + }, + // Address does contain circuit relay protocol. Dial destination via relay. + Ok(RelayedMultiaddr { + relay_peer_id, + relay_addr, + dst_peer_id, + dst_addr, + }) => { + // TODO: In the future we might want to support dialing a relay by its address only. + let relay_peer_id = relay_peer_id.ok_or(RelayError::MissingRelayPeerId)?; + let relay_addr = relay_addr.ok_or(RelayError::MissingRelayAddr)?; + let dst_peer_id = dst_peer_id.ok_or(RelayError::MissingDstPeerId)?; + + let mut to_behaviour = self.to_behaviour.clone(); + Ok(EitherFuture::Second( + async move { + let (tx, rx) = oneshot::channel(); + to_behaviour + .send(TransportToBehaviourMsg::DialReq { + request_id: RequestId::new(), + relay_addr, + relay_peer_id, + dst_addr, + dst_peer_id, + send_back: tx, + }) + .await?; + let stream = rx.await??; + Ok(stream) + } + .boxed(), + )) + } + } + } + + fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { + self.inner_transport.address_translation(server, observed) + } +} + +#[derive(Default)] +struct RelayedMultiaddr { + relay_peer_id: Option, + relay_addr: Option, + dst_peer_id: Option, + dst_addr: Option, +} + +/// Parse a [`Multiaddr`] containing a [`Protocol::P2pCircuit`]. +/// +/// Returns `Ok(Err(provided_addr))` when passed address contains no [`Protocol::P2pCircuit`]. +/// +/// Returns `Err(_)` when address is malformed. +fn parse_relayed_multiaddr( + addr: Multiaddr, +) -> Result, RelayError> { + if !addr.iter().any(|p| matches!(p, Protocol::P2pCircuit)) { + return Ok(Err(addr)); + } + + let mut relayed_multiaddr = RelayedMultiaddr::default(); + + let mut before_circuit = true; + for protocol in addr.into_iter() { + match protocol { + Protocol::P2pCircuit => { + if before_circuit { + before_circuit = false; + } else { + return Err(RelayError::MultipleCircuitRelayProtocolsUnsupported); + } + } + Protocol::P2p(hash) => { + let peer_id = PeerId::from_multihash(hash).map_err(|_| RelayError::InvalidHash)?; + + if before_circuit { + if relayed_multiaddr.relay_peer_id.is_some() { + return Err(RelayError::MalformedMultiaddr); + } + relayed_multiaddr.relay_peer_id = Some(peer_id) + } else { + if relayed_multiaddr.dst_peer_id.is_some() { + return Err(RelayError::MalformedMultiaddr); + } + relayed_multiaddr.dst_peer_id = Some(peer_id) + } + } + p => { + if before_circuit { + relayed_multiaddr + .relay_addr + .get_or_insert(Multiaddr::empty()) + .push(p); + } else { + relayed_multiaddr + .dst_addr + .get_or_insert(Multiaddr::empty()) + .push(p); + } + } + } + } + + Ok(Ok(relayed_multiaddr)) +} + +#[pin_project(project = RelayListenerProj)] +pub enum RelayListener { + Inner(#[pin] ::Listener), + Relayed { + from_behaviour: mpsc::Receiver, + + msg_to_behaviour: Option>>, + report_listen_addr: Option, + }, +} + +impl Stream for RelayListener { + type Item = Result< + ListenerEvent, EitherError<::Error, RelayError>>, + EitherError<::Error, RelayError>, + >; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.project(); + + match this { + RelayListenerProj::Inner(listener) => match listener.poll_next(cx) { + Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(EitherError::A(e)))), + Poll::Ready(Some(Ok(ListenerEvent::Upgrade { + upgrade, + local_addr, + remote_addr, + }))) => { + return Poll::Ready(Some(Ok(ListenerEvent::Upgrade { + upgrade: RelayedListenerUpgrade::Inner(upgrade), + local_addr, + remote_addr, + }))) + } + Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr)))) => { + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr)))) + } + Poll::Ready(Some(Ok(ListenerEvent::AddressExpired(addr)))) => { + return Poll::Ready(Some(Ok(ListenerEvent::AddressExpired(addr)))) + } + Poll::Ready(Some(Ok(ListenerEvent::Error(err)))) => { + return Poll::Ready(Some(Ok(ListenerEvent::Error(EitherError::A(err))))) + } + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => {} + }, + RelayListenerProj::Relayed { + from_behaviour, + msg_to_behaviour, + report_listen_addr, + } => { + if let Some(msg) = msg_to_behaviour { + match Future::poll(msg.as_mut(), cx) { + Poll::Ready(Ok(())) => *msg_to_behaviour = None, + Poll::Ready(Err(e)) => { + return Poll::Ready(Some(Err(EitherError::B(e.into())))) + } + Poll::Pending => {} + } + } + + match from_behaviour.poll_next_unpin(cx) { + Poll::Ready(Some(BehaviourToListenerMsg::IncomingRelayedConnection { + stream, + src_peer_id, + relay_peer_id, + relay_addr, + })) => { + return Poll::Ready(Some(Ok(ListenerEvent::Upgrade { + upgrade: RelayedListenerUpgrade::Relayed(Some(stream)), + local_addr: relay_addr + .with(Protocol::P2p(relay_peer_id.into())) + .with(Protocol::P2pCircuit), + remote_addr: Protocol::P2p(src_peer_id.into()).into(), + }))); + } + Poll::Ready(Some(BehaviourToListenerMsg::ConnectionToRelayEstablished)) => { + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress( + report_listen_addr + .take() + .expect("ConnectionToRelayEstablished to be send at most once"), + )))); + } + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => {} + } + } + } + + Poll::Pending + } +} + +pub type RelayedDial = BoxFuture<'static, Result>; + +#[pin_project(project = RelayedListenerUpgradeProj)] +pub enum RelayedListenerUpgrade { + Inner(#[pin] ::ListenerUpgrade), + Relayed(Option), +} + +impl Future for RelayedListenerUpgrade { + type Output = Result< + EitherOutput<::Output, protocol::Connection>, + EitherError<::Error, RelayError>, + >; + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match self.project() { + RelayedListenerUpgradeProj::Inner(upgrade) => match upgrade.poll(cx) { + Poll::Ready(Ok(out)) => return Poll::Ready(Ok(EitherOutput::First(out))), + Poll::Ready(Err(err)) => return Poll::Ready(Err(EitherError::A(err))), + Poll::Pending => {} + }, + RelayedListenerUpgradeProj::Relayed(substream) => { + return Poll::Ready(Ok(EitherOutput::Second( + substream.take().expect("Future polled after completion."), + ))) + } + } + + Poll::Pending + } +} + +/// Error that occurred during relay connection setup. +#[derive(Debug, Eq, PartialEq)] +pub enum RelayError { + MissingRelayPeerId, + MissingRelayAddr, + MissingDstPeerId, + InvalidHash, + SendingMessageToBehaviour(mpsc::SendError), + ResponseFromBehaviourCanceled, + DialingRelay, + MultipleCircuitRelayProtocolsUnsupported, + MalformedMultiaddr, +} + +impl From for TransportError> { + fn from(error: RelayError) -> Self { + TransportError::Other(EitherError::B(error)) + } +} + +impl From for RelayError { + fn from(error: mpsc::SendError) -> Self { + RelayError::SendingMessageToBehaviour(error) + } +} + +impl From for RelayError { + fn from(_: oneshot::Canceled) -> Self { + RelayError::ResponseFromBehaviourCanceled + } +} + +impl From for RelayError { + fn from(error: OutgoingRelayReqError) -> Self { + match error { + OutgoingRelayReqError::DialingRelay => RelayError::DialingRelay, + } + } +} + +impl std::fmt::Display for RelayError { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + RelayError::MissingRelayPeerId => { + write!(f, "Missing relay peer id.") + } + RelayError::MissingRelayAddr => { + write!(f, "Missing relay address.") + } + RelayError::MissingDstPeerId => { + write!(f, "Missing destination peer id.") + } + RelayError::InvalidHash => { + write!(f, "Invalid peer id hash.") + } + RelayError::SendingMessageToBehaviour(e) => { + write!(f, "Failed to send message to relay behaviour: {:?}", e) + } + RelayError::ResponseFromBehaviourCanceled => { + write!(f, "Response from behaviour was canceled") + } + RelayError::DialingRelay => { + write!(f, "Dialing relay failed") + } + RelayError::MultipleCircuitRelayProtocolsUnsupported => { + write!(f, "Address contains multiple circuit relay protocols (`p2p-circuit`) which is not supported.") + } + RelayError::MalformedMultiaddr => { + write!(f, "One of the provided multiaddresses is malformed.") + } + } + } +} + +impl std::error::Error for RelayError {} + +/// Message from the [`RelayTransport`] to the [`Relay`](crate::Relay) +/// [`NetworkBehaviour`](libp2p_swarm::NetworkBehaviour). +pub enum TransportToBehaviourMsg { + /// Dial destination node via relay node. + DialReq { + request_id: RequestId, + relay_addr: Multiaddr, + relay_peer_id: PeerId, + dst_addr: Option, + dst_peer_id: PeerId, + send_back: oneshot::Sender>, + }, + /// Listen for incoming relayed connections via relay node. + ListenReq { + /// [`PeerId`] and [`Multiaddr`] of relay node. + /// + /// When [`None`] listen for connections from any relay node. + relay_peer_id_and_addr: Option<(PeerId, Multiaddr)>, + to_listener: mpsc::Sender, + }, +} diff --git a/protocols/relay/tests/lib.rs b/protocols/relay/tests/lib.rs new file mode 100644 index 00000000000..a89c84fe5e9 --- /dev/null +++ b/protocols/relay/tests/lib.rs @@ -0,0 +1,1460 @@ +// Copyright 2021 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::executor::LocalPool; +use futures::future::{poll_fn, FutureExt}; +use futures::stream::Stream; +use futures::task::Spawn; +use libp2p::kad::record::store::MemoryStore; +use libp2p::NetworkBehaviour; +use libp2p_core::connection::{ConnectedPoint, ConnectionId}; +use libp2p_core::either::EitherTransport; +use libp2p_core::multiaddr::{Multiaddr, Protocol}; +use libp2p_core::transport::{MemoryTransport, Transport, TransportError}; +use libp2p_core::upgrade::{DeniedUpgrade, InboundUpgrade, OutboundUpgrade}; +use libp2p_core::{identity, upgrade, PeerId}; +use libp2p_identify::{Identify, IdentifyEvent, IdentifyInfo}; +use libp2p_kad::{GetClosestPeersOk, Kademlia, KademliaEvent, QueryResult}; +use libp2p_ping::{Ping, PingConfig, PingEvent}; +use libp2p_plaintext::PlainText2Config; +use libp2p_relay::{Relay, RelayConfig}; +use libp2p_swarm::protocols_handler::{ + KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, +}; +use libp2p_swarm::{ + AddressRecord, NegotiatedSubstream, NetworkBehaviour, NetworkBehaviourAction, + NetworkBehaviourEventProcess, PollParameters, Swarm, SwarmEvent, +}; +use std::iter; +use std::task::{Context, Poll}; +use std::time::Duration; +use void::Void; + +#[test] +fn src_connect_to_dst_listening_via_relay() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut dst_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + + let src_peer_id = Swarm::local_peer_id(&src_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + + let relay_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::())); + let dst_listen_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.into())) + .with(Protocol::P2pCircuit); + let dst_addr_via_relay = dst_listen_addr_via_relay + .clone() + .with(Protocol::P2p(dst_peer_id.into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::listen_on(&mut dst_swarm, dst_listen_addr_via_relay.clone()).unwrap(); + + pool.run_until(async { + // Destination Node dialing Relay. + match dst_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) => assert_eq!(peer_id, relay_peer_id), + e => panic!("{:?}", e), + } + + // Destination Node establishing connection to Relay. + match dst_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } => { + assert_eq!(peer_id, relay_peer_id); + } + e => panic!("{:?}", e), + } + + // Destination Node reporting listen address via relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::NewListenAddr(addr) if addr == dst_listen_addr_via_relay => break, + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::RoutingUpdated { + .. + })) => {} + e => panic!("{:?}", e), + } + } + + let dst = async move { + // Destination Node receiving connection from Source Node via Relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::IncomingConnection { send_back_addr, .. } => { + assert_eq!( + send_back_addr, + Protocol::P2p(src_peer_id.clone().into()).into() + ); + break; + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Destination Node establishing connection from Source Node via Relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == src_peer_id => { + break; + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Destination Node waiting for Ping from Source Node via Relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::ConnectionClosed { peer_id, .. } => { + assert_eq!(peer_id, src_peer_id); + break; + } + e => panic!("{:?}", e), + } + } + }; + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay).unwrap(); + let src = async move { + // Source Node dialing Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node establishing connection to Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node establishing connection to destination node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => { + break + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Source Node waiting for Ping from Destination Node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + peer, + result: Ok(_), + })) => { + if peer == dst_peer_id { + break; + } + } + e => panic!("{:?}", e), + } + } + }; + + futures::future::join(dst, src).await + }); +} + +#[test] +fn src_connect_to_dst_not_listening_via_active_relay() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut dst_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Active); + + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(dst_addr.into_iter().next().unwrap()) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::listen_on(&mut dst_swarm, dst_addr.clone()).unwrap(); + // Instruct destination node to listen for incoming relayed connections from unknown relay nodes. + Swarm::listen_on(&mut dst_swarm, Protocol::P2pCircuit.into()).unwrap(); + spawn_swarm_on_pool(&pool, dst_swarm); + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay).unwrap(); + pool.run_until(async move { + // Source Node dialing Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node establishing connection to Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node establishing connection to destination node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => { + break + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Source Node waiting for Ping from Destination Node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + peer, + result: Ok(_), + })) => { + if peer == dst_peer_id { + break; + } + } + e => panic!("{:?}", e), + } + } + }); +} + +#[test] +fn src_connect_to_dst_via_established_connection_to_relay() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut dst_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::listen_on(&mut dst_swarm, dst_addr_via_relay.clone()).unwrap(); + // Wait for destination to listen via relay. + pool.run_until(async { + loop { + match dst_swarm.next_event().await { + SwarmEvent::Dialing(_) => {} + SwarmEvent::ConnectionEstablished { .. } => {} + SwarmEvent::NewListenAddr(addr) if addr == dst_addr_via_relay => break, + e => panic!("{:?}", e), + } + } + }); + spawn_swarm_on_pool(&pool, dst_swarm); + + pool.run_until(async move { + Swarm::dial_addr(&mut src_swarm, relay_addr).unwrap(); + + // Source Node establishing connection to Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => { + break + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay).unwrap(); + + // Source Node establishing connection to destination node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => { + break + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Source Node waiting for Ping from Destination Node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + peer, + result: Ok(_), + })) => { + if peer == dst_peer_id { + break; + } + } + e => panic!("{:?}", e), + } + } + }); +} + +#[test] +fn src_try_connect_to_offline_dst() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + let dst_peer_id = PeerId::random(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(dst_addr.into_iter().next().unwrap()) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay.clone()).unwrap(); + pool.run_until(async move { + // Source Node dialing Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node establishing connection to Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + loop { + match src_swarm.next_event().await { + SwarmEvent::UnknownPeerUnreachableAddr { address, .. } + if address == dst_addr_via_relay => + { + break; + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + }); +} + +#[test] +fn src_try_connect_to_unsupported_dst() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut dst_swarm = build_keep_alive_only_swarm(); + + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(dst_addr.into_iter().next().unwrap()) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::listen_on(&mut dst_swarm, dst_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, dst_swarm); + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay.clone()).unwrap(); + pool.run_until(async move { + // Source Node dialing Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node establishing connection to Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + loop { + match src_swarm.next_event().await { + SwarmEvent::UnknownPeerUnreachableAddr { address, .. } + if address == dst_addr_via_relay => + { + break; + } + SwarmEvent::ConnectionClosed { peer_id, .. } if peer_id == relay_peer_id => {} + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + }); +} + +#[test] +fn src_try_connect_to_offline_dst_via_offline_relay() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + + let relay_peer_id = PeerId::random(); + let dst_peer_id = PeerId::random(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(dst_addr.into_iter().next().unwrap()) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay.clone()).unwrap(); + pool.run_until(async move { + // Source Node dialing Relay to connect to Destination Node. + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node fail to reach Relay. + match src_swarm.next_event().await { + SwarmEvent::UnreachableAddr { peer_id, .. } if peer_id == relay_peer_id => {} + e => panic!("{:?}", e), + } + + // Source Node fail to reach Destination Node due to failure reaching Relay. + match src_swarm.next_event().await { + SwarmEvent::UnknownPeerUnreachableAddr { address, .. } + if address == dst_addr_via_relay => {} + e => panic!("{:?}", e), + } + }); +} + +#[test] +fn firewalled_src_discover_firewalled_dst_via_kad_and_connect_to_dst_via_routable_relay() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut dst_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + + let src_peer_id = Swarm::local_peer_id(&src_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.into())) + .with(Protocol::P2pCircuit) + .with(Protocol::P2p(dst_peer_id.into())); + + src_swarm + .kad + .add_address(&relay_peer_id, relay_addr.clone()); + dst_swarm + .kad + .add_address(&relay_peer_id, relay_addr.clone()); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + // Destination Node listen via Relay. + Swarm::listen_on(&mut dst_swarm, dst_addr_via_relay.clone()).unwrap(); + + pool.run_until(async { + // Destination Node dialing Relay. + match dst_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) => assert_eq!(peer_id, relay_peer_id), + e => panic!("{:?}", e), + } + + // Destination Node establishing connection to Relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } => { + assert_eq!(peer_id, relay_peer_id); + break; + } + SwarmEvent::Behaviour(CombinedEvent::Kad(_)) => {} + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Destination Node reporting listen address via relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::NewListenAddr(addr) if addr == dst_addr_via_relay => break, + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::RoutingUpdated { + .. + })) => {} + e => panic!("{:?}", e), + } + } + + // Destination Node bootstrapping. + let query_id = dst_swarm.kad.bootstrap().unwrap(); + loop { + match dst_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::QueryResult { + id, + result: QueryResult::Bootstrap(Ok(_)), + .. + })) if query_id == id => { + if dst_swarm.kad.iter_queries().count() == 0 { + break; + } + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + let dst = async move { + // Destination Node receiving connection from Source Node via Relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::IncomingConnection { send_back_addr, .. } => { + assert_eq!(send_back_addr, Protocol::P2p(src_peer_id.into()).into()); + break; + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Destination Node establishing connection from Source Node via Relay. + loop { + match dst_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == src_peer_id => { + break; + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + + // Destination Node waiting for Source Node to close connection. + loop { + match dst_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::ConnectionClosed { peer_id, .. } if peer_id == relay_peer_id => {} + SwarmEvent::ConnectionClosed { peer_id, .. } if peer_id == src_peer_id => { + break; + } + SwarmEvent::Behaviour(CombinedEvent::Kad(_)) => {} + e => panic!("{:?}", e), + } + } + }; + + let src = async move { + // Source Node looking for Destination Node on the Kademlia DHT. + let mut query_id = src_swarm.kad.get_closest_peers(dst_peer_id); + // One has to retry multiple times to wait for Relay to receive Identify event from Node + // B. + let mut tries = 0; + + // Source Node establishing connection to Relay and, given that the DHT is small, Node + // B. + loop { + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } => { + if peer_id == relay_peer_id { + continue; + } else if peer_id == dst_peer_id { + break; + } else { + panic!("Unexpected peer id {:?}", peer_id); + } + } + SwarmEvent::Dialing(peer_id) + if peer_id == relay_peer_id || peer_id == dst_peer_id => {} + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::QueryResult { + id, + result: QueryResult::GetClosestPeers(Ok(GetClosestPeersOk { .. })), + .. + })) if id == query_id => { + tries += 1; + if tries > 300 { + panic!("Too many retries."); + } + + query_id = src_swarm.kad.get_closest_peers(dst_peer_id); + } + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::RoutingUpdated { + .. + })) => {} + e => panic!("{:?}", e), + } + } + + // Source Node waiting for Ping from Destination Node via Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + peer, + result: Ok(_), + })) => { + if peer == dst_peer_id { + break; + } + } + e => panic!("{:?}", e), + } + } + }; + + futures::future::join(dst, src).await + }); +} + +#[test] +fn inactive_connection_timeout() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_keep_alive_swarm(); + let mut dst_swarm = build_keep_alive_swarm(); + let mut relay_swarm = build_keep_alive_swarm(); + + // Connections only kept alive by Source Node and Destination Node. + relay_swarm.keep_alive.keep_alive = KeepAlive::No; + + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::listen_on(&mut dst_swarm, dst_addr_via_relay.clone()).unwrap(); + // Wait for destination to listen via relay. + pool.run_until(async { + loop { + match dst_swarm.next_event().await { + SwarmEvent::Dialing(_) => {} + SwarmEvent::ConnectionEstablished { .. } => {} + SwarmEvent::NewListenAddr(addr) if addr == dst_addr_via_relay => break, + e => panic!("{:?}", e), + } + } + }); + spawn_swarm_on_pool(&pool, dst_swarm); + + pool.run_until(async move { + Swarm::dial_addr(&mut src_swarm, relay_addr).unwrap(); + // Source Node dialing Relay. + loop { + match src_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {} + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => { + break; + } + e => panic!("{:?}", e), + } + } + + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay).unwrap(); + + // Source Node establishing connection to destination node via Relay. + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => {} + e => panic!("{:?}", e), + } + + // Relay should notice connection between Source Node and B to be idle. It should then close the + // relayed connection and eventually also close the connection to Source Node given that no + // connections are being relayed on the connection. + loop { + match src_swarm.next_event().await { + SwarmEvent::ConnectionClosed { peer_id, .. } => { + if peer_id == relay_peer_id { + break; + } + } + e => panic!("{:?}", e), + } + } + }); +} + +#[test] +fn concurrent_connection_same_relay_same_dst() { + let _ = env_logger::try_init(); + + let mut pool = LocalPool::new(); + + let mut src_swarm = build_swarm(Reachability::Firewalled, RelayMode::Passive); + let mut dst_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut relay_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + + let relay_peer_id = Swarm::local_peer_id(&relay_swarm).clone(); + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + + let relay_addr: Multiaddr = Protocol::Memory(rand::random::()).into(); + let dst_addr_via_relay = relay_addr + .clone() + .with(Protocol::P2p(relay_peer_id.clone().into())) + .with(Protocol::P2pCircuit) + .with(Protocol::P2p(dst_peer_id.clone().into())); + + Swarm::listen_on(&mut relay_swarm, relay_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_swarm); + + Swarm::listen_on(&mut dst_swarm, dst_addr_via_relay.clone()).unwrap(); + // Wait for destination to listen via relay. + pool.run_until(async { + loop { + match dst_swarm.next_event().await { + SwarmEvent::Dialing(_) => {} + SwarmEvent::ConnectionEstablished { .. } => {} + SwarmEvent::NewListenAddr(addr) if addr == dst_addr_via_relay => break, + e => panic!("{:?}", e), + } + } + }); + spawn_swarm_on_pool(&pool, dst_swarm); + + pool.run_until(async move { + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay.clone()).unwrap(); + Swarm::dial_addr(&mut src_swarm, dst_addr_via_relay).unwrap(); + + // Source Node establishing two connections to destination node via Relay. + let mut num_established = 0; + loop { + match src_swarm.next_event().await { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => {} + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => { + num_established += 1; + if num_established == 2 { + break; + } + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::Dialing(peer_id) => { + assert_eq!(peer_id, relay_peer_id); + } + e => panic!("{:?}", e), + } + } + }); +} + +/// Yield incoming connection through listener that listens via the relay node used by the +/// connection. In case the local node does not listen via the specific relay node, but has +/// registered a listener for all remaining incoming relayed connections, yield the connection via +/// that _catch-all_ listener. Drop the connection in all other cases. +/// +/// ## Nodes +/// +/// - Destination node explicitly listening via relay node 1 and 2. +/// +/// - 3 relay nodes +/// +/// - Source node 1 and 2 connecting to destination node via relay 1 and 2 respectively. +/// +/// - Source node 3 connecting to destination node via relay 3, expecting first connection attempt +/// to fail as destination node is not listening for incoming connections from any relay node. +/// Expecting second connection attempt to succeed after destination node registered listener for +/// any incoming relayed connection. +#[test] +fn yield_incoming_connection_through_correct_listener() { + let _ = env_logger::try_init(); + let mut pool = LocalPool::new(); + + let mut dst_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut src_1_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut src_2_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut src_3_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut relay_1_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut relay_2_swarm = build_swarm(Reachability::Routable, RelayMode::Passive); + let mut relay_3_swarm = build_swarm(Reachability::Routable, RelayMode::Active); + + let dst_peer_id = Swarm::local_peer_id(&dst_swarm).clone(); + let src_1_peer_id = Swarm::local_peer_id(&src_1_swarm).clone(); + let src_2_peer_id = Swarm::local_peer_id(&src_2_swarm).clone(); + let src_3_peer_id = Swarm::local_peer_id(&src_3_swarm).clone(); + let relay_1_peer_id = Swarm::local_peer_id(&relay_1_swarm).clone(); + let relay_2_peer_id = Swarm::local_peer_id(&relay_2_swarm).clone(); + let relay_3_peer_id = Swarm::local_peer_id(&relay_3_swarm).clone(); + + let dst_memory_port = Protocol::Memory(rand::random::()); + let dst_addr = Multiaddr::empty().with(dst_memory_port.clone()); + + let relay_1_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::())); + let relay_1_addr_incl_circuit = relay_1_addr + .clone() + .with(Protocol::P2p(relay_1_peer_id.into())) + .with(Protocol::P2pCircuit); + let dst_addr_via_relay_1 = relay_1_addr_incl_circuit + .clone() + .with(Protocol::P2p(dst_peer_id.into())); + let relay_2_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::())); + let relay_2_addr_incl_circuit = relay_2_addr + .clone() + .with(Protocol::P2p(relay_2_peer_id.into())) + .with(Protocol::P2pCircuit); + let dst_addr_via_relay_2 = relay_2_addr_incl_circuit + .clone() + .with(Protocol::P2p(dst_peer_id.into())); + let relay_3_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::())); + let relay_3_addr_incl_circuit = relay_3_addr + .clone() + .with(Protocol::P2p(relay_3_peer_id.into())) + .with(Protocol::P2pCircuit); + let dst_addr_via_relay_3 = relay_3_addr_incl_circuit + .clone() + .with(dst_memory_port) + .with(Protocol::P2p(dst_peer_id.into())); + + Swarm::listen_on(&mut relay_1_swarm, relay_1_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_1_swarm); + + Swarm::listen_on(&mut relay_2_swarm, relay_2_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_2_swarm); + + Swarm::listen_on(&mut relay_3_swarm, relay_3_addr.clone()).unwrap(); + spawn_swarm_on_pool(&pool, relay_3_swarm); + + Swarm::listen_on(&mut dst_swarm, relay_1_addr_incl_circuit.clone()).unwrap(); + Swarm::listen_on(&mut dst_swarm, relay_2_addr_incl_circuit.clone()).unwrap(); + // Listen on own address in order for relay 3 to be able to connect to destination node. + Swarm::listen_on(&mut dst_swarm, dst_addr.clone()).unwrap(); + + // Wait for destination node to establish connections to relay 1 and 2. + pool.run_until(async { + let mut established = 0; + loop { + match dst_swarm.next_event().await { + SwarmEvent::Dialing(peer_id) + if peer_id == relay_1_peer_id || peer_id == relay_2_peer_id => {} + SwarmEvent::ConnectionEstablished { peer_id, .. } + if peer_id == relay_1_peer_id || peer_id == relay_2_peer_id => + { + established += 1; + if established == 2 { + break; + } + } + SwarmEvent::NewListenAddr(addr) + if addr == relay_1_addr_incl_circuit + || addr == relay_2_addr_incl_circuit + || addr == dst_addr => {} + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + e => panic!("{:?}", e), + } + } + }); + + Swarm::dial_addr(&mut src_1_swarm, dst_addr_via_relay_1.clone()).unwrap(); + Swarm::dial_addr(&mut src_2_swarm, dst_addr_via_relay_2.clone()).unwrap(); + spawn_swarm_on_pool(&pool, src_1_swarm); + spawn_swarm_on_pool(&pool, src_2_swarm); + + // Expect source node 1 and 2 to reach destination node via relay 1 and 2 respectively. + pool.run_until(async { + let mut src_1_established = false; + let mut src_2_established = false; + let mut src_1_ping = false; + let mut src_2_ping = false; + loop { + match dst_swarm.next_event().await { + SwarmEvent::IncomingConnection { .. } => {} + SwarmEvent::ConnectionEstablished { + peer_id, endpoint, .. + } => { + let local_addr = match endpoint { + ConnectedPoint::Dialer { .. } => unreachable!(), + ConnectedPoint::Listener { local_addr, .. } => local_addr, + }; + + if peer_id == src_1_peer_id { + assert_eq!(local_addr, relay_1_addr_incl_circuit); + src_1_established = true; + } else if peer_id == src_2_peer_id { + assert_eq!(local_addr, relay_2_addr_incl_circuit); + src_2_established = true; + } else { + unreachable!(); + } + } + SwarmEvent::NewListenAddr(addr) + if addr == relay_1_addr_incl_circuit + || addr == relay_2_addr_incl_circuit + || addr == dst_addr => {} + SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + peer, + result: Ok(_), + })) => { + if peer == src_1_peer_id { + src_1_ping = true; + } else if peer == src_2_peer_id { + src_2_ping = true; + } + } + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::RoutingUpdated { + .. + })) => {} + e => panic!("{:?}", e), + } + + if src_1_established && src_2_established && src_1_ping && src_2_ping { + break; + } + } + }); + + // Expect destination node to reject incoming connection from unknown relay given that + // destination node is not listening for such connections. + Swarm::dial_addr(&mut src_3_swarm, dst_addr_via_relay_3.clone()).unwrap(); + pool.run_until(poll_fn(|cx| { + match dst_swarm.next_event().boxed().poll_unpin(cx) { + Poll::Ready(SwarmEvent::Behaviour(CombinedEvent::Ping(_))) => {} + Poll::Ready(SwarmEvent::IncomingConnection { .. }) => {} + Poll::Ready(SwarmEvent::ConnectionEstablished { peer_id, .. }) + if peer_id == relay_3_peer_id => {} + Poll::Ready(SwarmEvent::ConnectionEstablished { peer_id, .. }) + if peer_id == src_3_peer_id => + { + panic!( + "Expected destination node to reject incoming connection from unknown relay \ + without a catch-all listener", + ); + } + Poll::Pending => {} + e => panic!("{:?}", e), + } + + match src_3_swarm.next_event().boxed().poll_unpin(cx) { + Poll::Ready(SwarmEvent::UnknownPeerUnreachableAddr { address, .. }) + if address == dst_addr_via_relay_3 => + { + return Poll::Ready(()); + } + Poll::Ready(SwarmEvent::Dialing { .. }) => {} + Poll::Ready(SwarmEvent::ConnectionEstablished { peer_id, .. }) + if peer_id == relay_3_peer_id => {} + Poll::Ready(SwarmEvent::ConnectionEstablished { peer_id, .. }) + if peer_id == dst_peer_id => + { + panic!( + "Expected destination node to reject incoming connection from unknown relay \ + without a catch-all listener", + ); + } + Poll::Ready(SwarmEvent::Behaviour(CombinedEvent::Ping(_))) => {} + Poll::Pending => {} + e => panic!("{:?}", e), + } + + Poll::Pending + })); + + // Instruct destination node to listen for incoming relayed connections from unknown relay nodes. + Swarm::listen_on(&mut dst_swarm, Protocol::P2pCircuit.into()).unwrap(); + // Wait for destination node to report new listen address. + pool.run_until(async { + loop { + match dst_swarm.next_event().await { + SwarmEvent::NewListenAddr(addr) if addr == Protocol::P2pCircuit.into() => break, + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::RoutingUpdated { + .. + })) => {} + e => panic!("{:?}", e), + } + } + }); + spawn_swarm_on_pool(&pool, dst_swarm); + + // Expect destination node to accept incoming connection from "unknown" relay, i.e. the + // connection from source node 3 via relay 3. + Swarm::dial_addr(&mut src_3_swarm, dst_addr_via_relay_3.clone()).unwrap(); + pool.run_until(async move { + loop { + match src_3_swarm.next_event().await { + SwarmEvent::Dialing(_) => {} + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => { + break + } + SwarmEvent::Behaviour(CombinedEvent::Ping(_)) => {} + SwarmEvent::Behaviour(CombinedEvent::Kad(KademliaEvent::RoutingUpdated { + .. + })) => {} + e => panic!("{:?}", e), + } + } + }); +} + +#[derive(NetworkBehaviour)] +#[behaviour(out_event = "CombinedEvent", poll_method = "poll")] +struct CombinedBehaviour { + relay: Relay, + ping: Ping, + kad: Kademlia, + identify: Identify, + + #[behaviour(ignore)] + events: Vec, +} + +#[derive(Debug)] +enum CombinedEvent { + Kad(KademliaEvent), + Ping(PingEvent), +} + +impl CombinedBehaviour { + fn poll( + &mut self, + _: &mut Context, + _: &mut impl PollParameters, + ) -> Poll> { + if !self.events.is_empty() { + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(self.events.remove(0))); + } + + Poll::Pending + } +} + +impl NetworkBehaviourEventProcess for CombinedBehaviour { + fn inject_event(&mut self, event: PingEvent) { + self.events.push(CombinedEvent::Ping(event)); + } +} + +impl NetworkBehaviourEventProcess for CombinedBehaviour { + fn inject_event(&mut self, event: KademliaEvent) { + self.events.push(CombinedEvent::Kad(event)); + } +} + +impl NetworkBehaviourEventProcess for CombinedBehaviour { + fn inject_event(&mut self, event: IdentifyEvent) { + match event { + IdentifyEvent::Received { + peer_id, + info: IdentifyInfo { listen_addrs, .. }, + .. + } => { + for addr in listen_addrs { + self.kad.add_address(&peer_id, addr); + } + } + IdentifyEvent::Sent { .. } => {} + e => panic!("{:?}", e), + } + } +} + +impl NetworkBehaviourEventProcess<()> for CombinedBehaviour { + fn inject_event(&mut self, _event: ()) { + unreachable!(); + } +} + +#[derive(NetworkBehaviour)] +struct CombinedKeepAliveBehaviour { + relay: Relay, + keep_alive: KeepAliveBehaviour, +} + +impl NetworkBehaviourEventProcess<()> for CombinedKeepAliveBehaviour { + fn inject_event(&mut self, _event: ()) { + unreachable!(); + } +} + +impl NetworkBehaviourEventProcess for CombinedKeepAliveBehaviour { + fn inject_event(&mut self, _event: Void) { + unreachable!(); + } +} + +enum Reachability { + Firewalled, + Routable, +} + +enum RelayMode { + Active, + Passive, +} + +impl From for bool { + fn from(relay: RelayMode) -> Self { + match relay { + RelayMode::Active => true, + RelayMode::Passive => false, + } + } +} + +/// Wrapper around a [`Transport`] allowing outgoing but denying incoming connections. +/// +/// Meant for testing purposes only. +#[derive(Clone)] +pub struct Firewall(pub T); + +impl Transport for Firewall { + type Output = ::Output; + type Error = ::Error; + type Listener = futures::stream::Pending<<::Listener as Stream>::Item>; + type ListenerUpgrade = ::ListenerUpgrade; + type Dial = ::Dial; + + fn listen_on(self, _: Multiaddr) -> Result> { + Ok(futures::stream::pending()) + } + + fn dial(self, addr: Multiaddr) -> Result> { + self.0.dial(addr) + } + + fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { + self.0.address_translation(server, observed) + } +} + +fn build_swarm(reachability: Reachability, relay_mode: RelayMode) -> Swarm { + let local_key = identity::Keypair::generate_ed25519(); + let local_public_key = local_key.public(); + let plaintext = PlainText2Config { + local_public_key: local_public_key.clone(), + }; + let local_peer_id = local_public_key.clone().into_peer_id(); + + let transport = MemoryTransport::default(); + + let transport = match reachability { + Reachability::Firewalled => EitherTransport::Left(Firewall(transport)), + Reachability::Routable => EitherTransport::Right(transport), + }; + + let (transport, relay_behaviour) = libp2p_relay::new_transport_and_behaviour( + RelayConfig { + actively_connect_to_dst_nodes: relay_mode.into(), + ..Default::default() + }, + transport, + ); + + let transport = transport + .upgrade(upgrade::Version::V1) + .authenticate(plaintext) + .multiplex(libp2p_yamux::YamuxConfig::default()) + .boxed(); + + let combined_behaviour = CombinedBehaviour { + relay: relay_behaviour, + ping: Ping::new(PingConfig::new().with_interval(Duration::from_millis(100))), + kad: Kademlia::new( + local_peer_id.clone(), + MemoryStore::new(local_peer_id.clone()), + ), + identify: Identify::new( + "test".to_string(), + "test-agent".to_string(), + local_public_key.clone(), + ), + events: Default::default(), + }; + + Swarm::new(transport, combined_behaviour, local_peer_id) +} + +fn build_keep_alive_swarm() -> Swarm { + let local_key = identity::Keypair::generate_ed25519(); + let local_public_key = local_key.public(); + let plaintext = PlainText2Config { + local_public_key: local_public_key.clone(), + }; + let local_peer_id = local_public_key.clone().into_peer_id(); + + let transport = MemoryTransport::default(); + + let (transport, relay_behaviour) = + libp2p_relay::new_transport_and_behaviour(RelayConfig::default(), transport); + + let transport = transport + .upgrade(upgrade::Version::V1) + .authenticate(plaintext) + .multiplex(libp2p_yamux::YamuxConfig::default()) + .boxed(); + + let combined_behaviour = CombinedKeepAliveBehaviour { + relay: relay_behaviour, + keep_alive: KeepAliveBehaviour::default(), + }; + + Swarm::new(transport, combined_behaviour, local_peer_id) +} + +fn build_keep_alive_only_swarm() -> Swarm { + let local_key = identity::Keypair::generate_ed25519(); + let local_public_key = local_key.public(); + let plaintext = PlainText2Config { + local_public_key: local_public_key.clone(), + }; + let local_peer_id = local_public_key.clone().into_peer_id(); + + let transport = MemoryTransport::default(); + + let transport = transport + .upgrade(upgrade::Version::V1) + .authenticate(plaintext) + .multiplex(libp2p_yamux::YamuxConfig::default()) + .boxed(); + + Swarm::new(transport, KeepAliveBehaviour::default(), local_peer_id) +} + +#[derive(Clone)] +pub struct KeepAliveBehaviour { + keep_alive: KeepAlive, +} + +impl Default for KeepAliveBehaviour { + fn default() -> Self { + Self { + keep_alive: KeepAlive::Yes, + } + } +} + +impl libp2p_swarm::NetworkBehaviour for KeepAliveBehaviour { + type ProtocolsHandler = KeepAliveProtocolsHandler; + type OutEvent = void::Void; + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + KeepAliveProtocolsHandler { + keep_alive: self.keep_alive, + } + } + + fn addresses_of_peer(&mut self, _: &PeerId) -> Vec { + Vec::new() + } + + fn inject_connected(&mut self, _: &PeerId) {} + + fn inject_connection_established(&mut self, _: &PeerId, _: &ConnectionId, _: &ConnectedPoint) {} + + fn inject_disconnected(&mut self, _: &PeerId) {} + + fn inject_connection_closed(&mut self, _: &PeerId, _: &ConnectionId, _: &ConnectedPoint) {} + + fn inject_event( + &mut self, + _: PeerId, + _: ConnectionId, + _: ::OutEvent, + ) { + } + + fn poll( + &mut self, + _: &mut Context<'_>, + _: &mut impl PollParameters, + ) -> Poll< + NetworkBehaviourAction< + ::InEvent, + Self::OutEvent, + >, + > { + Poll::Pending + } +} + +/// Implementation of `ProtocolsHandler` that doesn't handle anything. +#[derive(Clone, Debug)] +pub struct KeepAliveProtocolsHandler { + pub keep_alive: KeepAlive, +} + +impl ProtocolsHandler for KeepAliveProtocolsHandler { + type InEvent = Void; + type OutEvent = Void; + type Error = Void; + type InboundProtocol = DeniedUpgrade; + type OutboundProtocol = DeniedUpgrade; + type OutboundOpenInfo = Void; + type InboundOpenInfo = (); + + fn listen_protocol(&self) -> SubstreamProtocol { + SubstreamProtocol::new(DeniedUpgrade, ()) + } + + fn inject_fully_negotiated_inbound( + &mut self, + _: >::Output, + _: Self::InboundOpenInfo, + ) { + } + + fn inject_fully_negotiated_outbound( + &mut self, + _: >::Output, + _: Self::OutboundOpenInfo, + ) { + } + + fn inject_event(&mut self, _: Self::InEvent) {} + + fn inject_address_change(&mut self, _: &Multiaddr) {} + + fn inject_dial_upgrade_error( + &mut self, + _: Self::OutboundOpenInfo, + _: ProtocolsHandlerUpgrErr< + >::Error, + >, + ) { + } + + fn inject_listen_upgrade_error( + &mut self, + _: Self::InboundOpenInfo, + _: ProtocolsHandlerUpgrErr< + >::Error, + >, + ) { + } + + fn connection_keep_alive(&self) -> KeepAlive { + self.keep_alive + } + + fn poll( + &mut self, + _: &mut Context<'_>, + ) -> Poll< + ProtocolsHandlerEvent< + Self::OutboundProtocol, + Self::OutboundOpenInfo, + Self::OutEvent, + Self::Error, + >, + > { + Poll::Pending + } +} + +struct DummyPollParameters {} + +impl PollParameters for DummyPollParameters { + type SupportedProtocolsIter = iter::Empty>; + type ListenedAddressesIter = iter::Empty; + type ExternalAddressesIter = iter::Empty; + + fn supported_protocols(&self) -> Self::SupportedProtocolsIter { + unimplemented!(); + } + + /// Returns the list of the addresses we're listening on. + fn listened_addresses(&self) -> Self::ListenedAddressesIter { + unimplemented!(); + } + + /// Returns the list of the addresses nodes can use to reach us. + fn external_addresses(&self) -> Self::ExternalAddressesIter { + unimplemented!(); + } + + /// Returns the peer id of the local node. + fn local_peer_id(&self) -> &PeerId { + unimplemented!(); + } +} + +fn spawn_swarm_on_pool(pool: &LocalPool, mut swarm: Swarm) { + pool.spawner() + .spawn_obj( + async move { + loop { + swarm.next_event().await; + } + } + .boxed() + .into(), + ) + .unwrap(); +} diff --git a/src/lib.rs b/src/lib.rs index eb965931c3d..18cb879b5e0 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -241,6 +241,10 @@ pub use libp2p_yamux as yamux; #[cfg_attr(docsrs, doc(cfg(feature = "pnet")))] #[doc(inline)] pub use libp2p_pnet as pnet; +#[cfg(feature = "relay")] +#[cfg_attr(docsrs, doc(cfg(feature = "relay")))] +#[doc(inline)] +pub use libp2p_relay as relay; #[cfg(feature = "request-response")] #[cfg_attr(docsrs, doc(cfg(feature = "request-response")))] #[doc(inline)]