mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-20 21:36:31 +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:
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,
|
||||
}
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user