protocols/relay: Implement circuit relay specification (#1838)

This commit implements the [libp2p circuit
relay](https://github.com/libp2p/specs/tree/master/relay) specification. It is
based on previous work from https://github.com/libp2p/rust-libp2p/pull/1134.

Instead of altering the `Transport` trait, the approach taken in this commit
is to wrap an existing implementation of `Transport` allowing one to:

- Intercept `dial` requests with a relayed address.

- Inject incoming relayed connections with the local node being the destination.

- Intercept `listen_on` requests pointing to a relay, ensuring to keep a
  constant connection to the relay, waiting for incoming requests with the local
  node being the destination.

More concretely one would wrap an existing `Transport` implementation as seen
below, allowing the `Relay` behaviour and the `RelayTransport` to communicate
via channels.

### Example

```rust
let (relay_transport, relay_behaviour) = new_transport_and_behaviour(
    RelayConfig::default(),
    MemoryTransport::default(),
);

let transport = relay_transport
    .upgrade(upgrade::Version::V1)
    .authenticate(plaintext)
    .multiplex(YamuxConfig::default())
    .boxed();

let mut swarm = Swarm::new(transport, relay_behaviour, local_peer_id);

let relay_addr = Multiaddr::from_str("/memory/1234").unwrap()
    .with(Protocol::P2p(PeerId::random().into()))
    .with(Protocol::P2pCircuit);
let dst_addr = relay_addr.clone().with(Protocol::Memory(5678));

// Listen for incoming connections via relay node (1234).
Swarm::listen_on(&mut swarm, relay_addr).unwrap();

// Dial node (5678) via relay node (1234).
Swarm::dial_addr(&mut swarm, dst_addr).unwrap();
```

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

View File

@ -0,0 +1,137 @@
// Copyright 2020 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! Helper to interconnect two substreams, connecting the receiver side of A with the sender side of
//! B and vice versa.
//!
//! Inspired by [`futures::io::Copy`].
use futures::future::Future;
use futures::future::FutureExt;
use futures::io::{AsyncBufRead, BufReader};
use futures::io::{AsyncRead, AsyncWrite};
use futures::ready;
use futures_timer::Delay;
use std::io;
use std::pin::Pin;
use std::task::{Context, Poll};
use std::time::Duration;
pub struct CopyFuture<S, D> {
src: BufReader<S>,
dst: BufReader<D>,
active_timeout: Delay,
configured_timeout: Duration,
}
impl<S: AsyncRead, D: AsyncRead> CopyFuture<S, D> {
pub fn new(src: S, dst: D, timeout: Duration) -> Self {
CopyFuture {
src: BufReader::new(src),
dst: BufReader::new(dst),
active_timeout: Delay::new(timeout),
configured_timeout: timeout,
}
}
}
impl<S, D> Future for CopyFuture<S, D>
where
S: AsyncRead + AsyncWrite + Unpin,
D: AsyncRead + AsyncWrite + Unpin,
{
type Output = io::Result<()>;
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = &mut *self;
let mut reset_timer = false;
loop {
enum Status {
Pending,
Done,
Progressed,
}
let src_status = match forward_data(&mut this.src, &mut this.dst, cx) {
Poll::Ready(Err(e)) => return Poll::Ready(Err(e)),
Poll::Ready(Ok(true)) => Status::Done,
Poll::Ready(Ok(false)) => Status::Progressed,
Poll::Pending => Status::Pending,
};
let dst_status = match forward_data(&mut this.dst, &mut this.src, cx) {
Poll::Ready(Err(e)) => return Poll::Ready(Err(e)),
Poll::Ready(Ok(true)) => Status::Done,
Poll::Ready(Ok(false)) => Status::Progressed,
Poll::Pending => Status::Pending,
};
match (src_status, dst_status) {
// Both source and destination are done sending data.
(Status::Done, Status::Done) => return Poll::Ready(Ok(())),
// Either source or destination made progress, thus reset timer.
(Status::Progressed, _) | (_, Status::Progressed) => reset_timer = true,
// Both are pending. Check if timer fired, otherwise return Poll::Pending.
(Status::Pending, Status::Pending) => break,
// One is done sending data, the other is pending. Check if timer fired, otherwise
// return Poll::Pending.
(Status::Pending, Status::Done) | (Status::Done, Status::Pending) => break,
}
}
if reset_timer {
this.active_timeout = Delay::new(this.configured_timeout);
}
if let Poll::Ready(()) = this.active_timeout.poll_unpin(cx) {
return Poll::Ready(Err(io::ErrorKind::TimedOut.into()));
}
Poll::Pending
}
}
/// Forwards data from `source` to `destination`.
///
/// Returns `true` when done, i.e. `source` having reached EOF, returns false otherwise, thus
/// indicating progress.
fn forward_data<S: AsyncBufRead + Unpin, D: AsyncWrite + Unpin>(
mut src: &mut S,
mut dst: &mut D,
cx: &mut Context<'_>,
) -> Poll<io::Result<bool>> {
let buffer = ready!(Pin::new(&mut src).poll_fill_buf(cx))?;
if buffer.is_empty() {
ready!(Pin::new(&mut dst).poll_flush(cx))?;
ready!(Pin::new(&mut dst).poll_close(cx))?;
return Poll::Ready(Ok(true));
}
let i = ready!(Pin::new(dst).poll_write(cx, buffer))?;
if i == 0 {
return Poll::Ready(Err(io::ErrorKind::WriteZero.into()));
}
Pin::new(src).consume(i);
Poll::Ready(Ok(false))
}

View File

@ -0,0 +1,140 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::message_proto::{circuit_relay, CircuitRelay};
use crate::protocol::Peer;
use asynchronous_codec::{Framed, FramedParts};
use bytes::BytesMut;
use futures::{future::BoxFuture, prelude::*};
use futures::channel::oneshot;
use libp2p_core::{Multiaddr, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::io;
use unsigned_varint::codec::UviBytes;
/// Request from a remote for us to become a destination.
///
/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*, and
/// we are the *destination*, this struct is a message that the *relay* sent to us. The
/// parameters passed to `IncomingDstReq::new()` are the information of the *source*.
///
/// If the upgrade succeeds, the substream is returned and we will receive data sent from the
/// source on it.
#[must_use = "An incoming destination request should be either accepted or denied"]
pub struct IncomingDstReq {
/// The stream to the source.
stream: Framed<NegotiatedSubstream, UviBytes>,
/// Source of the request.
src: Peer,
}
impl IncomingDstReq
{
/// Creates a `IncomingDstReq`.
pub(crate) fn new(stream: Framed<NegotiatedSubstream, UviBytes>, src: Peer) -> Self {
IncomingDstReq {
stream: stream,
src,
}
}
/// Returns the peer id of the source that is being relayed.
pub fn src_id(&self) -> &PeerId {
&self.src.peer_id
}
/// Returns the addresses of the source that is being relayed.
pub fn src_addrs(&self) -> impl Iterator<Item = &Multiaddr> {
self.src.addrs.iter()
}
/// Accepts the request.
///
/// The returned `Future` sends back a success message then returns the raw stream. This raw
/// stream then points to the source (as retreived with `src_id()` and `src_addrs()`).
pub fn accept(
self,
) -> BoxFuture<'static, Result<(PeerId, super::Connection, oneshot::Receiver<()>), IncomingDstReqError>> {
let IncomingDstReq { mut stream, src } = self;
let msg = CircuitRelay {
r#type: Some(circuit_relay::Type::Status.into()),
src_peer: None,
dst_peer: None,
code: Some(circuit_relay::Status::Success.into()),
};
let mut msg_bytes = BytesMut::new();
msg.encode(&mut msg_bytes)
.expect("all the mandatory fields are always filled; QED");
async move {
stream.send(msg_bytes.freeze()).await?;
let FramedParts {
io,
read_buffer,
write_buffer,
..
} = stream.into_parts();
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have empty write buffer."
);
let (tx, rx) = oneshot::channel();
Ok((src.peer_id, super::Connection::new(read_buffer.freeze(), io, tx), rx))
}
.boxed()
}
/// Refuses the request.
///
/// The returned `Future` gracefully shuts down the request.
pub fn deny(mut self) -> BoxFuture<'static, Result<(), io::Error>> {
let msg = CircuitRelay {
r#type: Some(circuit_relay::Type::Status.into()),
src_peer: None,
dst_peer: None,
code: Some(circuit_relay::Status::StopRelayRefused.into()),
};
let mut msg_bytes = BytesMut::new();
msg.encode(&mut msg_bytes)
.expect("all the mandatory fields are always filled; QED");
async move {
self.stream.send(msg_bytes.freeze()).await?;
Ok(())
}
.boxed()
}
}
#[derive(Debug)]
pub enum IncomingDstReqError {
Io(std::io::Error),
}
impl From<std::io::Error> for IncomingDstReqError {
fn from(e: std::io::Error) -> Self {
IncomingDstReqError::Io(e)
}
}

View File

@ -0,0 +1,156 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use super::copy_future::CopyFuture;
use crate::message_proto::{circuit_relay, circuit_relay::Status, CircuitRelay};
use crate::protocol::Peer;
use asynchronous_codec::{Framed, FramedParts};
use bytes::{BytesMut, Bytes};
use futures::channel::oneshot;
use futures::future::BoxFuture;
use futures::prelude::*;
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::time::Duration;
use unsigned_varint::codec::UviBytes;
/// Request from a remote for us to relay communications to another node.
///
/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*, and
/// we are the *relay*, this struct is a message that the *source* sent to us. The parameters
/// passed to `IncomingRelayReq::new()` are the information of the *destination*.
///
/// If the upgrade succeeds, the substream is returned and we will receive data sent from the
/// source on it. This data must be transmitted to the destination.
#[must_use = "An incoming relay request should be either accepted or denied."]
pub struct IncomingRelayReq {
/// The stream to the source.
stream: Framed<NegotiatedSubstream, UviBytes>,
/// Target of the request.
dest: Peer,
_notifier: oneshot::Sender<()>,
}
impl IncomingRelayReq
{
/// Creates a [`IncomingRelayReq`] as well as a Future that resolves once the
/// [`IncomingRelayReq`] is dropped.
pub(crate) fn new(
stream: Framed<NegotiatedSubstream, UviBytes>,
dest: Peer,
) -> (Self, oneshot::Receiver<()>) {
let (tx, rx) = oneshot::channel();
(
IncomingRelayReq {
stream,
dest,
_notifier: tx,
},
rx,
)
}
/// Peer id of the node we should relay communications to.
pub(crate) fn dst_peer(&self) -> &Peer {
&self.dest
}
/// Accepts the request by providing a stream to the destination.
pub fn fulfill<TDestSubstream>(
mut self,
dst_stream: TDestSubstream,
dst_read_buffer: Bytes,
) -> BoxFuture<'static, Result<(), IncomingRelayReqError>>
where
TDestSubstream: AsyncRead + AsyncWrite + Send + Unpin + 'static,
{
let msg = CircuitRelay {
r#type: Some(circuit_relay::Type::Status.into()),
src_peer: None,
dst_peer: None,
code: Some(circuit_relay::Status::Success.into()),
};
let mut msg_bytes = BytesMut::new();
msg.encode(&mut msg_bytes)
.expect("all the mandatory fields are always filled; QED");
async move {
self.stream.send(msg_bytes.freeze()).await?;
let FramedParts {
mut io,
read_buffer,
write_buffer,
..
} = self.stream.into_parts();
assert!(
read_buffer.is_empty(),
"Expect a Framed, that was never actively read from, not to read."
);
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have empty write buffer."
);
if !dst_read_buffer.is_empty() {
io.write_all(&dst_read_buffer).await?;
}
let copy_future = CopyFuture::new(io, dst_stream, Duration::from_secs(5));
copy_future.await.map_err(Into::into)
}
.boxed()
}
/// Refuses the request.
///
/// The returned `Future` gracefully shuts down the request.
pub fn deny(mut self, err_code: Status) -> BoxFuture<'static, Result<(), std::io::Error>> {
let msg = CircuitRelay {
r#type: Some(circuit_relay::Type::Status.into()),
code: Some(err_code.into()),
src_peer: None,
dst_peer: None,
};
let mut msg_bytes = BytesMut::new();
msg.encode(&mut msg_bytes)
.expect("all the mandatory fields are always filled; QED");
async move {
self.stream.send(msg_bytes.freeze()).await?;
Ok(())
}
.boxed()
}
}
#[derive(Debug)]
pub enum IncomingRelayReqError {
Io(std::io::Error),
}
impl From<std::io::Error> for IncomingRelayReqError {
fn from(e: std::io::Error) -> Self {
IncomingRelayReqError::Io(e)
}
}

View File

@ -0,0 +1,177 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::message_proto::{circuit_relay, CircuitRelay};
use crate::protocol::incoming_dst_req::IncomingDstReq;
use crate::protocol::incoming_relay_req::IncomingRelayReq;
use crate::protocol::{Peer, PeerParseError, MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME};
use asynchronous_codec::Framed;
use futures::channel::oneshot;
use futures::{future::BoxFuture, prelude::*};
use libp2p_core::upgrade;
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::io::Cursor;
use std::{convert::TryFrom, error, fmt, iter};
use unsigned_varint::codec::UviBytes;
/// Configuration for an inbound upgrade that handles requests from the remote for the relay
/// protocol.
#[derive(Debug, Clone)]
pub struct RelayListen {}
/// Outcome of the listening.
pub enum RelayRemoteReq {
/// We have been asked to become a destination.
DstReq(IncomingDstReq),
/// We have been asked to relay communications to another node.
RelayReq((IncomingRelayReq, oneshot::Receiver<()>)),
}
impl RelayListen {
/// Builds a new `RelayListen` with default options.
pub fn new() -> RelayListen {
RelayListen {}
}
}
impl upgrade::UpgradeInfo for RelayListen {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(PROTOCOL_NAME)
}
}
impl upgrade::InboundUpgrade<NegotiatedSubstream> for RelayListen {
type Output = RelayRemoteReq;
type Error = RelayListenError;
type Future = BoxFuture<'static, Result<RelayRemoteReq, RelayListenError>>;
fn upgrade_inbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
async move {
let mut codec = UviBytes::<bytes::Bytes>::default();
codec.set_max_len(MAX_ACCEPTED_MESSAGE_LEN);
let mut substream = Framed::new(substream, codec);
let msg: bytes::BytesMut = substream
.next()
.await
.ok_or(std::io::Error::new(std::io::ErrorKind::UnexpectedEof, ""))??;
let CircuitRelay {
r#type,
src_peer,
dst_peer,
code: _,
} = CircuitRelay::decode(Cursor::new(msg))?;
match circuit_relay::Type::from_i32(r#type.ok_or(RelayListenError::NoMessageType)?)
.ok_or(RelayListenError::InvalidMessageTy)?
{
circuit_relay::Type::Hop => {
let peer = Peer::try_from(dst_peer.ok_or(RelayListenError::NoDstPeer)?)?;
let (rq, notifyee) = IncomingRelayReq::new(substream, peer);
Ok(RelayRemoteReq::RelayReq((rq, notifyee)))
}
circuit_relay::Type::Stop => {
let peer = Peer::try_from(src_peer.ok_or(RelayListenError::NoSrcPeer)?)?;
let rq = IncomingDstReq::new(substream, peer);
Ok(RelayRemoteReq::DstReq(rq))
}
_ => Err(RelayListenError::InvalidMessageTy),
}
}
.boxed()
}
}
/// Error while upgrading with a [`RelayListen`].
#[derive(Debug)]
pub enum RelayListenError {
Decode(prost::DecodeError),
Io(std::io::Error),
NoSrcPeer,
NoDstPeer,
ParsePeer(PeerParseError),
NoMessageType,
InvalidMessageTy,
}
impl From<prost::DecodeError> for RelayListenError {
fn from(err: prost::DecodeError) -> Self {
RelayListenError::Decode(err)
}
}
impl From<std::io::Error> for RelayListenError {
fn from(e: std::io::Error) -> Self {
RelayListenError::Io(e)
}
}
impl From<PeerParseError> for RelayListenError {
fn from(err: PeerParseError) -> Self {
RelayListenError::ParsePeer(err)
}
}
impl fmt::Display for RelayListenError {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
RelayListenError::Decode(e) => {
write!(f, "Failed to decode response: {}.", e)
}
RelayListenError::Io(e) => {
write!(f, "Io error {}", e)
}
RelayListenError::NoSrcPeer => {
write!(f, "Expected source peer id")
}
RelayListenError::NoDstPeer => {
write!(f, "Expected destination peer id")
}
RelayListenError::ParsePeer(e) => {
write!(f, "Failed to parse peer field: {}", e)
}
RelayListenError::NoMessageType => {
write!(f, "Expected message type to be set.")
}
RelayListenError::InvalidMessageTy => {
write!(f, "Invalid message type")
}
}
}
}
impl error::Error for RelayListenError {
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
RelayListenError::Decode(e) => Some(e),
RelayListenError::Io(e) => Some(e),
RelayListenError::NoSrcPeer => None,
RelayListenError::NoDstPeer => None,
RelayListenError::ParsePeer(_) => None,
RelayListenError::NoMessageType => None,
RelayListenError::InvalidMessageTy => None,
}
}
}

View File

@ -0,0 +1,226 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::message_proto::{circuit_relay, CircuitRelay};
use crate::protocol::{Peer, MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME};
use asynchronous_codec::{Framed, FramedParts};
use bytes::Bytes;
use futures::future::BoxFuture;
use futures::prelude::*;
use libp2p_core::{upgrade, Multiaddr, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::{fmt, error, iter};
use unsigned_varint::codec::UviBytes;
/// Ask the remote to become a destination. The upgrade succeeds if the remote accepts, and fails
/// if the remote refuses.
///
/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*,
/// this struct is the message that the *relay* sends to the *destination* at initialization. The
/// parameters passed to `OutgoingDstReq::new()` are the information of the *source* and the
/// *destination* (not the information of the *relay*).
///
/// The upgrade should be performed on a substream to the *destination*.
///
/// If the upgrade succeeds, the substream is returned and we must link it with the data sent from
/// the source.
#[derive(Debug, Clone)]
pub struct OutgoingDstReq {
/// The message to send to the destination. Pre-computed.
message: Vec<u8>,
}
impl OutgoingDstReq {
/// Creates a `OutgoingDstReq`. Must pass the parameters of the message.
pub(crate) fn new(src_id: PeerId, src_addr: Multiaddr, dst_peer: Peer) -> Self {
let message = CircuitRelay {
r#type: Some(circuit_relay::Type::Stop.into()),
src_peer: Some(circuit_relay::Peer {
id: src_id.to_bytes(),
addrs: vec![src_addr.to_vec()],
}),
dst_peer: Some(circuit_relay::Peer {
id: dst_peer.peer_id.to_bytes(),
addrs: dst_peer.addrs.into_iter().map(|a| a.to_vec()).collect(),
}),
code: None,
};
let mut encoded_msg = Vec::new();
message
.encode(&mut encoded_msg)
.expect("all the mandatory fields are always filled; QED");
OutgoingDstReq {
message: encoded_msg,
}
}
}
impl upgrade::UpgradeInfo for OutgoingDstReq {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(PROTOCOL_NAME)
}
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for OutgoingDstReq {
type Output = (NegotiatedSubstream, Bytes);
type Error = OutgoingDstReqError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
fn upgrade_outbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
let mut codec = UviBytes::default();
codec.set_max_len(MAX_ACCEPTED_MESSAGE_LEN);
let mut substream = Framed::new(substream, codec);
async move {
substream.send(std::io::Cursor::new(self.message)).await?;
let msg =
substream
.next()
.await
.ok_or(OutgoingDstReqError::Io(std::io::Error::new(
std::io::ErrorKind::UnexpectedEof,
"",
)))??;
let msg = std::io::Cursor::new(msg);
let CircuitRelay {
r#type,
src_peer,
dst_peer,
code,
} = CircuitRelay::decode(msg)?;
match r#type
.map(circuit_relay::Type::from_i32)
.flatten()
.ok_or(OutgoingDstReqError::ParseTypeField)?
{
circuit_relay::Type::Status => {}
s => return Err(OutgoingDstReqError::ExpectedStatusType(s)),
}
if src_peer.is_some() {
return Err(OutgoingDstReqError::UnexpectedSrcPeerWithStatusType);
}
if dst_peer.is_some() {
return Err(OutgoingDstReqError::UnexpectedDstPeerWithStatusType);
}
match code
.map(circuit_relay::Status::from_i32)
.flatten()
.ok_or(OutgoingDstReqError::ParseStatusField)?
{
circuit_relay::Status::Success => {}
s => return Err(OutgoingDstReqError::ExpectedSuccessStatus(s)),
}
let FramedParts {
io,
read_buffer,
write_buffer,
..
} = substream.into_parts();
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have an empty write buffer."
);
Ok((io, read_buffer.freeze()))
}
.boxed()
}
}
#[derive(Debug)]
pub enum OutgoingDstReqError {
Decode(prost::DecodeError),
Io(std::io::Error),
ParseTypeField,
ParseStatusField,
ExpectedStatusType(circuit_relay::Type),
ExpectedSuccessStatus(circuit_relay::Status),
UnexpectedSrcPeerWithStatusType,
UnexpectedDstPeerWithStatusType,
}
impl From<std::io::Error> for OutgoingDstReqError {
fn from(e: std::io::Error) -> Self {
OutgoingDstReqError::Io(e)
}
}
impl From<prost::DecodeError> for OutgoingDstReqError {
fn from(e: prost::DecodeError) -> Self {
OutgoingDstReqError::Decode(e)
}
}
impl fmt::Display for OutgoingDstReqError {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
OutgoingDstReqError::Decode(e) => {
write!(f, "Failed to decode response: {}.", e)
}
OutgoingDstReqError::Io(e) => {
write!(f, "Io error {}", e)
}
OutgoingDstReqError::ParseTypeField => {
write!(f, "Failed to parse response type field.")
}
OutgoingDstReqError::ParseStatusField => {
write!(f, "Failed to parse response status field.")
}
OutgoingDstReqError::ExpectedStatusType(t) => {
write!(f, "Expected status message type, but got {:?}", t)
}
OutgoingDstReqError::UnexpectedSrcPeerWithStatusType => {
write!(f, "Unexpected source peer with status type.")
}
OutgoingDstReqError::UnexpectedDstPeerWithStatusType => {
write!(f, "Unexpected destination peer with status type.")
}
OutgoingDstReqError::ExpectedSuccessStatus(s) => {
write!(f, "Expected success status but got {:?}", s)
}
}
}
}
impl error::Error for OutgoingDstReqError {
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
OutgoingDstReqError::Decode(e) => Some(e),
OutgoingDstReqError::Io(e) => Some(e),
OutgoingDstReqError::ParseTypeField => None,
OutgoingDstReqError::ParseStatusField => None,
OutgoingDstReqError::ExpectedStatusType(_) => None,
OutgoingDstReqError::UnexpectedSrcPeerWithStatusType => None,
OutgoingDstReqError::UnexpectedDstPeerWithStatusType => None,
OutgoingDstReqError::ExpectedSuccessStatus(_) => None,
}
}
}

View File

@ -0,0 +1,235 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::message_proto::{circuit_relay, CircuitRelay};
use crate::protocol::{MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME};
use asynchronous_codec::{Framed, FramedParts};
use futures::channel::oneshot;
use futures::future::BoxFuture;
use futures::prelude::*;
use libp2p_core::{upgrade, Multiaddr, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::{error, fmt, iter};
use unsigned_varint::codec::UviBytes;
/// Ask a remote to act as a relay.
///
/// If we take a situation where a *source* wants to talk to a *destination* through a *relay*,
/// this struct is the message that the *source* sends to the *relay* at initialization. The
/// parameters passed to `OutgoingRelayReq::new()` are the information of the *destination*
/// (not the information of the *relay*).
///
/// The upgrade should be performed on a substream to the *relay*.
///
/// If the upgrade succeeds, the substream is returned and is now a brand new connection pointing
/// to the *destination*.
pub struct OutgoingRelayReq {
src_id: PeerId,
dst_id: PeerId,
dst_address: Option<Multiaddr>,
}
impl OutgoingRelayReq {
/// Builds a request for the target to act as a relay to a third party.
pub fn new(src_id: PeerId, dst_id: PeerId, dst_address: Option<Multiaddr>) -> Self {
OutgoingRelayReq {
src_id,
dst_id,
dst_address,
}
}
}
impl upgrade::UpgradeInfo for OutgoingRelayReq {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(PROTOCOL_NAME)
}
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for OutgoingRelayReq {
type Output = (super::Connection, oneshot::Receiver<()>);
type Error = OutgoingRelayReqError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
fn upgrade_outbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
let OutgoingRelayReq {
src_id,
dst_id,
dst_address,
} = self;
let message = CircuitRelay {
r#type: Some(circuit_relay::Type::Hop.into()),
src_peer: Some(circuit_relay::Peer {
id: src_id.to_bytes(),
addrs: vec![],
}),
dst_peer: Some(circuit_relay::Peer {
id: dst_id.to_bytes(),
addrs: vec![dst_address.unwrap_or(Multiaddr::empty()).to_vec()],
}),
code: None,
};
let mut encoded = Vec::new();
message
.encode(&mut encoded)
.expect("all the mandatory fields are always filled; QED");
let mut codec = UviBytes::default();
codec.set_max_len(MAX_ACCEPTED_MESSAGE_LEN);
let mut substream = Framed::new(substream, codec);
async move {
substream.send(std::io::Cursor::new(encoded)).await?;
let msg =
substream
.next()
.await
.ok_or(OutgoingRelayReqError::Io(std::io::Error::new(
std::io::ErrorKind::UnexpectedEof,
"",
)))??;
let msg = std::io::Cursor::new(msg);
let CircuitRelay {
r#type,
src_peer,
dst_peer,
code,
} = CircuitRelay::decode(msg)?;
match r#type
.map(circuit_relay::Type::from_i32)
.flatten()
.ok_or(OutgoingRelayReqError::ParseTypeField)?
{
circuit_relay::Type::Status => {}
s => return Err(OutgoingRelayReqError::ExpectedStatusType(s)),
}
match code
.map(circuit_relay::Status::from_i32)
.flatten()
.ok_or(OutgoingRelayReqError::ParseStatusField)?
{
circuit_relay::Status::Success => {}
e => return Err(OutgoingRelayReqError::ExpectedSuccessStatus(e)),
}
if src_peer.is_some() {
return Err(OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType);
}
if dst_peer.is_some() {
return Err(OutgoingRelayReqError::UnexpectedDstPeerWithStatusType);
}
let FramedParts {
io,
read_buffer,
write_buffer,
..
} = substream.into_parts();
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have empty write buffer."
);
let (tx, rx) = oneshot::channel();
Ok((super::Connection::new(read_buffer.freeze(), io, tx), rx))
}
.boxed()
}
}
#[derive(Debug)]
pub enum OutgoingRelayReqError {
Decode(prost::DecodeError),
Io(std::io::Error),
ParseTypeField,
ParseStatusField,
ExpectedStatusType(circuit_relay::Type),
UnexpectedSrcPeerWithStatusType,
UnexpectedDstPeerWithStatusType,
ExpectedSuccessStatus(circuit_relay::Status),
}
impl From<std::io::Error> for OutgoingRelayReqError {
fn from(e: std::io::Error) -> Self {
OutgoingRelayReqError::Io(e)
}
}
impl From<prost::DecodeError> for OutgoingRelayReqError {
fn from(e: prost::DecodeError) -> Self {
OutgoingRelayReqError::Decode(e)
}
}
impl fmt::Display for OutgoingRelayReqError {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
OutgoingRelayReqError::Decode(e) => {
write!(f, "Failed to decode response: {}.", e)
}
OutgoingRelayReqError::Io(e) => {
write!(f, "Io error {}", e)
}
OutgoingRelayReqError::ParseTypeField => {
write!(f, "Failed to parse response type field.")
}
OutgoingRelayReqError::ParseStatusField => {
write!(f, "Failed to parse response status field.")
}
OutgoingRelayReqError::ExpectedStatusType(t) => {
write!(f, "Expected status message type, but got {:?}", t)
}
OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType => {
write!(f, "Unexpected source peer with status type.")
}
OutgoingRelayReqError::UnexpectedDstPeerWithStatusType => {
write!(f, "Unexpected destination peer with status type.")
}
OutgoingRelayReqError::ExpectedSuccessStatus(s) => {
write!(f, "Expected success status but got {:?}", s)
}
}
}
}
impl error::Error for OutgoingRelayReqError {
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
OutgoingRelayReqError::Decode(e) => Some(e),
OutgoingRelayReqError::Io(e) => Some(e),
OutgoingRelayReqError::ParseTypeField => None,
OutgoingRelayReqError::ParseStatusField => None,
OutgoingRelayReqError::ExpectedStatusType(_) => None,
OutgoingRelayReqError::UnexpectedSrcPeerWithStatusType => None,
OutgoingRelayReqError::UnexpectedDstPeerWithStatusType => None,
OutgoingRelayReqError::ExpectedSuccessStatus(_) => None,
}
}
}