mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-13 01:51:23 +00:00
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:
793
protocols/relay/src/behaviour.rs
Normal file
793
protocols/relay/src/behaviour.rs
Normal 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,
|
||||
}
|
747
protocols/relay/src/handler.rs
Normal file
747
protocols/relay/src/handler.rs
Normal 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
124
protocols/relay/src/lib.rs
Normal 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())
|
||||
}
|
||||
}
|
43
protocols/relay/src/message.proto
Normal file
43
protocols/relay/src/message.proto
Normal 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
|
||||
}
|
182
protocols/relay/src/protocol.rs
Normal file
182
protocols/relay/src/protocol.rs
Normal 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)
|
||||
}
|
||||
}
|
137
protocols/relay/src/protocol/copy_future.rs
Normal file
137
protocols/relay/src/protocol/copy_future.rs
Normal 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))
|
||||
}
|
140
protocols/relay/src/protocol/incoming_dst_req.rs
Normal file
140
protocols/relay/src/protocol/incoming_dst_req.rs
Normal 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)
|
||||
}
|
||||
}
|
156
protocols/relay/src/protocol/incoming_relay_req.rs
Normal file
156
protocols/relay/src/protocol/incoming_relay_req.rs
Normal 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)
|
||||
}
|
||||
}
|
177
protocols/relay/src/protocol/listen.rs
Normal file
177
protocols/relay/src/protocol/listen.rs
Normal 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,
|
||||
}
|
||||
}
|
||||
}
|
226
protocols/relay/src/protocol/outgoing_dst_req.rs
Normal file
226
protocols/relay/src/protocol/outgoing_dst_req.rs
Normal 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,
|
||||
}
|
||||
}
|
||||
}
|
235
protocols/relay/src/protocol/outgoing_relay_req.rs
Normal file
235
protocols/relay/src/protocol/outgoing_relay_req.rs
Normal 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,
|
||||
}
|
||||
}
|
||||
}
|
559
protocols/relay/src/transport.rs
Normal file
559
protocols/relay/src/transport.rs
Normal 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>,
|
||||
},
|
||||
}
|
Reference in New Issue
Block a user