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 <pierre.krieger1708@gmail.com>
Co-authored-by: Roman Borschel <romanb@users.noreply.github.com>
Co-authored-by: David Craven <david@craven.ch>
This commit is contained in:
Max Inden
2021-03-11 16:07:59 +01:00
committed by GitHub
parent f48bb15fa7
commit 2f9c1759e6
20 changed files with 5139 additions and 0 deletions

View File

@ -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<TransportToBehaviourMsg>,
/// 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<NetworkBehaviourAction<RelayHandlerIn, ()>>,
/// List of peers the network is connected to.
connected_peers: HashMap<PeerId, HashSet<ConnectionId>>,
/// 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<PeerId, Vec<IncomingRelayReq>>,
/// 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<PeerId, RelayListener>,
/// 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<mpsc::Sender<BehaviourToListenerMsg>>,
}
#[derive(Default)]
struct OutgoingRelayReqs {
/// Indexed by relay peer id.
dialing: HashMap<PeerId, Vec<OutgoingDialingRelayReq>>,
upgrading: HashMap<RequestId, OutgoingUpgradingRelayReq>,
}
struct OutgoingDialingRelayReq {
request_id: RequestId,
src_peer_id: PeerId,
relay_addr: Multiaddr,
dst_addr: Option<Multiaddr>,
dst_peer_id: PeerId,
send_back: oneshot::Sender<Result<protocol::Connection, OutgoingRelayReqError>>,
}
struct OutgoingUpgradingRelayReq {
send_back: oneshot::Sender<Result<protocol::Connection, OutgoingRelayReqError>>,
}
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(<relay_server>/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<TransportToBehaviourMsg>,
) -> 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<Multiaddr> {
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<NetworkBehaviourAction<RelayHandlerIn, Self::OutEvent>> {
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<BehaviourToListenerMsg>,
},
Connected {
connection_id: ConnectionId,
to_listener: mpsc::Sender<BehaviourToListenerMsg>,
},
}
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,
}

View File

@ -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) -> <Self::Handler as ProtocolsHandler>::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<BoxFuture<'static, Result<(), std::io::Error>>>,
/// 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<BoxFuture<'static, Result<(), protocol::IncomingRelayReqError>>>,
/// Requests asking the remote to become a relay.
outgoing_relay_reqs: Vec<OutgoingRelayReq>,
/// Requests asking the remote to become a destination.
outgoing_dst_reqs: Vec<OutgoingDstReq>,
/// Queue of events to return when polled.
queued_events: Vec<RelayHandlerEvent>,
/// 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<oneshot::Receiver<()>>,
/// 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<protocol::OutgoingRelayReqError, protocol::OutgoingDstReqError>,
>,
>,
>,
}
struct OutgoingRelayReq {
src_peer_id: PeerId,
dst_peer_id: PeerId,
request_id: RequestId,
/// Addresses of the destination.
dst_addr: Option<Multiaddr>,
}
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<Multiaddr>,
},
/// 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<protocol::OutgoingRelayReqError, protocol::OutgoingDstReqError>,
>,
>;
type InboundProtocol = protocol::RelayListen;
type OutboundProtocol =
upgrade::EitherUpgrade<protocol::OutgoingRelayReq, protocol::OutgoingDstReq>;
type OutboundOpenInfo = RelayOutboundOpenInfo;
type InboundOpenInfo = RequestId;
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
SubstreamProtocol::new(protocol::RelayListen::new(), RequestId::new())
}
fn inject_fully_negotiated_inbound(
&mut self,
protocol: <Self::InboundProtocol as upgrade::InboundUpgrade<NegotiatedSubstream>>::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: <Self::OutboundProtocol as upgrade::OutboundUpgrade<NegotiatedSubstream>>::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<protocol::RelayListenError>,
) {
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<protocol::OutgoingRelayReqError, protocol::OutgoingDstReqError>,
>,
) {
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,
},
}

124
protocols/relay/src/lib.rs Normal file
View File

@ -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<T: Transport + Clone>(
config: RelayConfig,
transport: T,
) -> (RelayTransport<T>, 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())
}
}

View File

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

View File

@ -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<circuit_relay::Peer> for Peer {
type Error = PeerParseError;
fn try_from(peer: circuit_relay::Peer) -> Result<Peer, Self::Error> {
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<Result<usize, Error>> {
Pin::new(&mut self.stream).poll_write(cx, buf)
}
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
Pin::new(&mut self.stream).poll_flush(cx)
}
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
Pin::new(&mut self.stream).poll_close(cx)
}
fn poll_write_vectored(
mut self: Pin<&mut Self>,
cx: &mut Context,
bufs: &[IoSlice],
) -> Poll<Result<usize, Error>> {
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<Result<usize, Error>> {
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)
}
}

View File

@ -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<S, D> {
src: BufReader<S>,
dst: BufReader<D>,
active_timeout: Delay,
configured_timeout: Duration,
}
impl<S: AsyncRead, D: AsyncRead> CopyFuture<S, D> {
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<S, D> Future for CopyFuture<S, D>
where
S: AsyncRead + AsyncWrite + Unpin,
D: AsyncRead + AsyncWrite + Unpin,
{
type Output = io::Result<()>;
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
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<S: AsyncBufRead + Unpin, D: AsyncWrite + Unpin>(
mut src: &mut S,
mut dst: &mut D,
cx: &mut Context<'_>,
) -> Poll<io::Result<bool>> {
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))
}

View File

@ -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<NegotiatedSubstream, UviBytes>,
/// Source of the request.
src: Peer,
}
impl IncomingDstReq
{
/// Creates a `IncomingDstReq`.
pub(crate) fn new(stream: Framed<NegotiatedSubstream, UviBytes>, 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<Item = &Multiaddr> {
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<std::io::Error> for IncomingDstReqError {
fn from(e: std::io::Error) -> Self {
IncomingDstReqError::Io(e)
}
}

View File

@ -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<NegotiatedSubstream, UviBytes>,
/// 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<NegotiatedSubstream, UviBytes>,
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<TDestSubstream>(
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<std::io::Error> for IncomingRelayReqError {
fn from(e: std::io::Error) -> Self {
IncomingRelayReqError::Io(e)
}
}

View File

@ -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<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(PROTOCOL_NAME)
}
}
impl upgrade::InboundUpgrade<NegotiatedSubstream> for RelayListen {
type Output = RelayRemoteReq;
type Error = RelayListenError;
type Future = BoxFuture<'static, Result<RelayRemoteReq, RelayListenError>>;
fn upgrade_inbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
async move {
let mut codec = UviBytes::<bytes::Bytes>::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<prost::DecodeError> for RelayListenError {
fn from(err: prost::DecodeError) -> Self {
RelayListenError::Decode(err)
}
}
impl From<std::io::Error> for RelayListenError {
fn from(e: std::io::Error) -> Self {
RelayListenError::Io(e)
}
}
impl From<PeerParseError> 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,
}
}
}

View File

@ -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<u8>,
}
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<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(PROTOCOL_NAME)
}
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for OutgoingDstReq {
type Output = (NegotiatedSubstream, Bytes);
type Error = OutgoingDstReqError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
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<std::io::Error> for OutgoingDstReqError {
fn from(e: std::io::Error) -> Self {
OutgoingDstReqError::Io(e)
}
}
impl From<prost::DecodeError> 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,
}
}
}

View File

@ -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<Multiaddr>,
}
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<Multiaddr>) -> Self {
OutgoingRelayReq {
src_id,
dst_id,
dst_address,
}
}
}
impl upgrade::UpgradeInfo for OutgoingRelayReq {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(PROTOCOL_NAME)
}
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for OutgoingRelayReq {
type Output = (super::Connection, oneshot::Receiver<()>);
type Error = OutgoingRelayReqError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
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<std::io::Error> for OutgoingRelayReqError {
fn from(e: std::io::Error) -> Self {
OutgoingRelayReqError::Io(e)
}
}
impl From<prost::DecodeError> 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,
}
}
}

View File

@ -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<T: Clone> {
to_behaviour: mpsc::Sender<TransportToBehaviourMsg>,
inner_transport: T,
}
impl<T: Clone> RelayTransport<T> {
/// 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<TransportToBehaviourMsg>) {
let (to_behaviour, from_transport) = mpsc::channel(0);
let transport = RelayTransport {
to_behaviour,
inner_transport: t,
};
(transport, from_transport)
}
}
impl<T: Transport + Clone> Transport for RelayTransport<T> {
type Output = EitherOutput<<T as Transport>::Output, protocol::Connection>;
type Error = EitherError<<T as Transport>::Error, RelayError>;
type Listener = RelayListener<T>;
type ListenerUpgrade = RelayedListenerUpgrade<T>;
type Dial = EitherFuture<<T as Transport>::Dial, RelayedDial>;
fn listen_on(self, addr: Multiaddr) -> Result<Self::Listener, TransportError<Self::Error>> {
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<Self::Dial, TransportError<Self::Error>> {
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<Multiaddr> {
self.inner_transport.address_translation(server, observed)
}
}
#[derive(Default)]
struct RelayedMultiaddr {
relay_peer_id: Option<PeerId>,
relay_addr: Option<Multiaddr>,
dst_peer_id: Option<PeerId>,
dst_addr: Option<Multiaddr>,
}
/// 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<Result<RelayedMultiaddr, Multiaddr>, 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<T: Transport> {
Inner(#[pin] <T as Transport>::Listener),
Relayed {
from_behaviour: mpsc::Receiver<BehaviourToListenerMsg>,
msg_to_behaviour: Option<BoxFuture<'static, Result<(), mpsc::SendError>>>,
report_listen_addr: Option<Multiaddr>,
},
}
impl<T: Transport> Stream for RelayListener<T> {
type Item = Result<
ListenerEvent<RelayedListenerUpgrade<T>, EitherError<<T as Transport>::Error, RelayError>>,
EitherError<<T as Transport>::Error, RelayError>,
>;
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
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<protocol::Connection, RelayError>>;
#[pin_project(project = RelayedListenerUpgradeProj)]
pub enum RelayedListenerUpgrade<T: Transport> {
Inner(#[pin] <T as Transport>::ListenerUpgrade),
Relayed(Option<protocol::Connection>),
}
impl<T: Transport> Future for RelayedListenerUpgrade<T> {
type Output = Result<
EitherOutput<<T as Transport>::Output, protocol::Connection>,
EitherError<<T as Transport>::Error, RelayError>,
>;
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
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<E> From<RelayError> for TransportError<EitherError<E, RelayError>> {
fn from(error: RelayError) -> Self {
TransportError::Other(EitherError::B(error))
}
}
impl From<mpsc::SendError> for RelayError {
fn from(error: mpsc::SendError) -> Self {
RelayError::SendingMessageToBehaviour(error)
}
}
impl From<oneshot::Canceled> for RelayError {
fn from(_: oneshot::Canceled) -> Self {
RelayError::ResponseFromBehaviourCanceled
}
}
impl From<OutgoingRelayReqError> 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<Multiaddr>,
dst_peer_id: PeerId,
send_back: oneshot::Sender<Result<protocol::Connection, OutgoingRelayReqError>>,
},
/// 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<BehaviourToListenerMsg>,
},
}