protocols/relay: Implement circuit relay v2 protocol (#2059)

This commit adds an implementation for the circuit relay v2 protocol to be used
as a relay server, i.e. it supports incoming HOP requests and outgoing STOP
requests and used as a relay clients, i.e. outgoing HOP requests and incoming
STOP requests.

The existing circuit relay v1 protocol implementation is moved to
protocols/relay/src/v1.

Co-authored-by: ronzigelman <ronzigelman@gmail.com>
Co-authored-by: Marco Munizaga <git@marcopolo.io>
Co-authored-by: Thomas Eizinger <thomas@eizinger.io>
Co-authored-by: Elena Frank <57632201+elenaf9@users.noreply.github.com>
This commit is contained in:
Max Inden
2022-01-14 19:58:28 +01:00
committed by GitHub
parent 3e39cd74a9
commit 17ee5047e3
38 changed files with 5919 additions and 270 deletions

View File

@ -50,7 +50,7 @@ noise = ["libp2p-noise"]
ping = ["libp2p-ping", "libp2p-metrics/ping"]
plaintext = ["libp2p-plaintext"]
pnet = ["libp2p-pnet"]
relay = ["libp2p-relay"]
relay = ["libp2p-relay", "libp2p-metrics/relay"]
request-response = ["libp2p-request-response"]
rendezvous = ["libp2p-rendezvous"]
tcp-async-io = ["libp2p-tcp", "libp2p-tcp/async-io"]

View File

@ -15,6 +15,7 @@ gossipsub = ["libp2p-gossipsub"]
identify = ["libp2p-identify"]
kad = ["libp2p-kad"]
ping = ["libp2p-ping"]
relay = ["libp2p-relay"]
[dependencies]
libp2p-core = { version = "0.31.0", path = "../../core", default-features = false }
@ -22,6 +23,7 @@ libp2p-gossipsub = { version = "0.35.0", path = "../../protocols/gossipsub", op
libp2p-identify = { version = "0.33.0", path = "../../protocols/identify", optional = true }
libp2p-kad = { version = "0.34.0", path = "../../protocols/kad", optional = true }
libp2p-ping = { version = "0.33.0", path = "../../protocols/ping", optional = true }
libp2p-relay = { version = "0.6.0", path = "../../protocols/relay", optional = true }
libp2p-swarm = { version = "0.33.0", path = "../../swarm" }
open-metrics-client = "0.14.0"

View File

@ -33,6 +33,8 @@ mod identify;
mod kad;
#[cfg(feature = "ping")]
mod ping;
#[cfg(feature = "relay")]
mod relay;
mod swarm;
use open_metrics_client::registry::Registry;
@ -47,6 +49,8 @@ pub struct Metrics {
kad: kad::Metrics,
#[cfg(feature = "ping")]
ping: ping::Metrics,
#[cfg(feature = "relay")]
relay: relay::Metrics,
swarm: swarm::Metrics,
}
@ -70,6 +74,8 @@ impl Metrics {
kad: kad::Metrics::new(sub_registry),
#[cfg(feature = "ping")]
ping: ping::Metrics::new(sub_registry),
#[cfg(feature = "relay")]
relay: relay::Metrics::new(sub_registry),
swarm: swarm::Metrics::new(sub_registry),
}
}

114
misc/metrics/src/relay.rs Normal file
View File

@ -0,0 +1,114 @@
// Copyright 2021 Protocol Labs.
//
// 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 open_metrics_client::encoding::text::Encode;
use open_metrics_client::metrics::counter::Counter;
use open_metrics_client::metrics::family::Family;
use open_metrics_client::registry::Registry;
pub struct Metrics {
events: Family<EventLabels, Counter>,
}
impl Metrics {
pub fn new(registry: &mut Registry) -> Self {
let sub_registry = registry.sub_registry_with_prefix("relay");
let events = Family::default();
sub_registry.register(
"events",
"Events emitted by the relay NetworkBehaviour",
Box::new(events.clone()),
);
Self { events }
}
}
#[derive(Debug, Clone, Hash, PartialEq, Eq, Encode)]
struct EventLabels {
event: EventType,
}
#[derive(Debug, Clone, Hash, PartialEq, Eq, Encode)]
enum EventType {
ReservationReqAccepted,
ReservationReqAcceptFailed,
ReservationReqDenied,
ReservationReqDenyFailed,
ReservationTimedOut,
CircuitReqReceiveFailed,
CircuitReqDenied,
CircuitReqDenyFailed,
CircuitReqOutboundConnectFailed,
CircuitReqAccepted,
CircuitReqAcceptFailed,
CircuitClosed,
}
impl From<&libp2p_relay::v2::relay::Event> for EventType {
fn from(event: &libp2p_relay::v2::relay::Event) -> Self {
match event {
libp2p_relay::v2::relay::Event::ReservationReqAccepted { .. } => {
EventType::ReservationReqAccepted
}
libp2p_relay::v2::relay::Event::ReservationReqAcceptFailed { .. } => {
EventType::ReservationReqAcceptFailed
}
libp2p_relay::v2::relay::Event::ReservationReqDenied { .. } => {
EventType::ReservationReqDenied
}
libp2p_relay::v2::relay::Event::ReservationReqDenyFailed { .. } => {
EventType::ReservationReqDenyFailed
}
libp2p_relay::v2::relay::Event::ReservationTimedOut { .. } => {
EventType::ReservationTimedOut
}
libp2p_relay::v2::relay::Event::CircuitReqReceiveFailed { .. } => {
EventType::CircuitReqReceiveFailed
}
libp2p_relay::v2::relay::Event::CircuitReqDenied { .. } => EventType::CircuitReqDenied,
libp2p_relay::v2::relay::Event::CircuitReqOutboundConnectFailed { .. } => {
EventType::CircuitReqOutboundConnectFailed
}
libp2p_relay::v2::relay::Event::CircuitReqDenyFailed { .. } => {
EventType::CircuitReqDenyFailed
}
libp2p_relay::v2::relay::Event::CircuitReqAccepted { .. } => {
EventType::CircuitReqAccepted
}
libp2p_relay::v2::relay::Event::CircuitReqAcceptFailed { .. } => {
EventType::CircuitReqAcceptFailed
}
libp2p_relay::v2::relay::Event::CircuitClosed { .. } => EventType::CircuitClosed,
}
}
}
impl super::Recorder<libp2p_relay::v2::relay::Event> for super::Metrics {
fn record(&self, event: &libp2p_relay::v2::relay::Event) {
self.relay
.events
.get_or_create(&EventLabels {
event: event.into(),
})
.inc();
}
}

View File

@ -4,7 +4,7 @@ edition = "2021"
rust-version = "1.56.1"
description = "Communications relaying for libp2p"
version = "0.6.0"
authors = ["Parity Technologies <admin@parity.io>"]
authors = ["Parity Technologies <admin@parity.io>", "Max Inden <mail@max-inden.de>"]
license = "MIT"
repository = "https://github.com/libp2p/rust-libp2p"
keywords = ["peer-to-peer", "libp2p", "networking"]
@ -13,6 +13,7 @@ categories = ["network-programming", "asynchronous"]
[dependencies]
asynchronous-codec = "0.6"
bytes = "1"
either = "1.6.0"
futures = "0.3.1"
futures-timer = "3"
instant = "0.1.11"
@ -21,8 +22,10 @@ libp2p-swarm = { version = "0.33.0", path = "../../swarm" }
log = "0.4"
pin-project = "1"
prost = "0.9"
rand = "0.7"
rand = "0.8.4"
smallvec = "1.6.1"
static_assertions = "1"
thiserror = "1.0"
unsigned-varint = { version = "0.7", features = ["asynchronous_codec"] }
void = "1"
@ -31,10 +34,11 @@ prost-build = "0.9"
[dev-dependencies]
env_logger = "0.9.0"
structopt = "0.3.21"
libp2p = { path = "../.." }
libp2p-identify = { path = "../identify" }
libp2p-kad = { path = "../kad" }
libp2p-ping = { path = "../ping" }
libp2p-identify = { path = "../identify" }
libp2p-plaintext = { path = "../../transports/plaintext" }
libp2p-yamux = { path = "../../muxers/yamux" }
quickcheck = "1"
structopt = "0.3.21"

View File

@ -19,5 +19,6 @@
// DEALINGS IN THE SOFTWARE.
fn main() {
prost_build::compile_protos(&["src/message.proto"], &["src"]).unwrap();
prost_build::compile_protos(&["src/v1/message.proto"], &["src/v1"]).unwrap();
prost_build::compile_protos(&["src/v2/message.proto"], &["src/v2"]).unwrap();
}

View File

@ -61,7 +61,7 @@ use futures::executor::block_on;
use futures::stream::StreamExt;
use libp2p::dns::DnsConfig;
use libp2p::plaintext;
use libp2p::relay::{Relay, RelayConfig};
use libp2p::relay::v1::{Relay, RelayConfig};
use libp2p::swarm::SwarmEvent;
use libp2p::tcp::TcpConfig;
use libp2p::Transport;
@ -94,7 +94,7 @@ fn main() -> Result<(), Box<dyn Error>> {
..Default::default()
};
let (relay_wrapped_transport, relay_behaviour) =
libp2p_relay::new_transport_and_behaviour(relay_config, transport);
libp2p_relay::v1::new_transport_and_behaviour(relay_config, transport);
let behaviour = Behaviour {
relay: relay_behaviour,

View File

@ -0,0 +1,152 @@
// Copyright 2020 Parity Technologies (UK) Ltd.
// Copyright 2021 Protocol Labs.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::executor::block_on;
use futures::stream::StreamExt;
use libp2p::core::upgrade;
use libp2p::identify::{Identify, IdentifyConfig, IdentifyEvent};
use libp2p::multiaddr::Protocol;
use libp2p::ping::{Ping, PingConfig, PingEvent};
use libp2p::relay::v2::relay::{self, Relay};
use libp2p::swarm::{Swarm, SwarmEvent};
use libp2p::tcp::TcpConfig;
use libp2p::Transport;
use libp2p::{identity, NetworkBehaviour, PeerId};
use libp2p::{noise, Multiaddr};
use std::error::Error;
use std::net::{Ipv4Addr, Ipv6Addr};
use structopt::StructOpt;
fn main() -> Result<(), Box<dyn Error>> {
env_logger::init();
let opt = Opt::from_args();
println!("opt: {:?}", opt);
// Create a static known PeerId based on given secret
let local_key: identity::Keypair = generate_ed25519(opt.secret_key_seed);
let local_peer_id = PeerId::from(local_key.public());
println!("Local peer id: {:?}", local_peer_id);
let tcp_transport = TcpConfig::new();
let noise_keys = noise::Keypair::<noise::X25519Spec>::new()
.into_authentic(&local_key)
.expect("Signing libp2p-noise static DH keypair failed.");
let transport = tcp_transport
.upgrade(upgrade::Version::V1)
.authenticate(noise::NoiseConfig::xx(noise_keys).into_authenticated())
.multiplex(libp2p_yamux::YamuxConfig::default())
.boxed();
let behaviour = Behaviour {
relay: Relay::new(local_peer_id, Default::default()),
ping: Ping::new(PingConfig::new()),
identify: Identify::new(IdentifyConfig::new(
"/TODO/0.0.1".to_string(),
local_key.public(),
)),
};
let mut swarm = Swarm::new(transport, behaviour, local_peer_id);
// Listen on all interfaces
let listen_addr = Multiaddr::empty()
.with(match opt.use_ipv6 {
Some(true) => Protocol::from(Ipv6Addr::UNSPECIFIED),
_ => Protocol::from(Ipv4Addr::UNSPECIFIED),
})
.with(Protocol::Tcp(opt.port));
swarm.listen_on(listen_addr)?;
block_on(async {
loop {
match swarm.next().await.expect("Infinite Stream.") {
SwarmEvent::Behaviour(Event::Relay(event)) => {
println!("{:?}", event)
}
SwarmEvent::NewListenAddr { address, .. } => {
println!("Listening on {:?}", address);
}
_ => {}
}
}
})
}
#[derive(NetworkBehaviour)]
#[behaviour(out_event = "Event", event_process = false)]
struct Behaviour {
relay: Relay,
ping: Ping,
identify: Identify,
}
#[derive(Debug)]
enum Event {
Ping(PingEvent),
Identify(IdentifyEvent),
Relay(relay::Event),
}
impl From<PingEvent> for Event {
fn from(e: PingEvent) -> Self {
Event::Ping(e)
}
}
impl From<IdentifyEvent> for Event {
fn from(e: IdentifyEvent) -> Self {
Event::Identify(e)
}
}
impl From<relay::Event> for Event {
fn from(e: relay::Event) -> Self {
Event::Relay(e)
}
}
fn generate_ed25519(secret_key_seed: u8) -> identity::Keypair {
let mut bytes = [0u8; 32];
bytes[0] = secret_key_seed;
let secret_key = identity::ed25519::SecretKey::from_bytes(&mut bytes)
.expect("this returns `Err` only if the length is wrong; the length is correct; qed");
identity::Keypair::Ed25519(secret_key.into())
}
#[derive(Debug, StructOpt)]
#[structopt(name = "libp2p relay")]
struct Opt {
/// Determine if the relay listen on ipv6 or ipv4 loopback address. the default is ipv4
#[structopt(long)]
use_ipv6: Option<bool>,
/// Fixed value to generate deterministic peer id
#[structopt(long)]
secret_key_seed: u8,
/// The port used to listen on all interfaces
#[structopt(long)]
port: u16,
}

View File

@ -1,4 +1,5 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
// Copyright 2021 Protocol Labs.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
@ -18,107 +19,12 @@
// 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, dial_opts::DialOpts};
//! # 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.to_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(relay_addr).unwrap();
//!
//! // Dial node (5678) via relay node (1234).
//! swarm.dial(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*.
//! libp2p circuit relay implementations
mod behaviour;
pub mod v1;
pub mod v2;
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())
}
// Check that we can safely cast a `usize` to a `u64`.
static_assertions::const_assert! {
std::mem::size_of::<usize>() <= std::mem::size_of::<u64>()
}

126
protocols/relay/src/v1.rs Normal file
View File

@ -0,0 +1,126 @@
// 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 v1
//! specification](https://github.com/libp2p/specs/tree/master/relay).
//!
//! ## Example
//!
//! ```rust
//! # use libp2p_core::transport::memory::MemoryTransport;
//! # use libp2p_relay::v1::{RelayConfig, new_transport_and_behaviour};
//! # use libp2p_swarm::{Swarm, dial_opts::DialOpts};
//! # 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.to_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(relay_addr).unwrap();
//!
//! // Dial node (5678) via relay node (1234).
//! swarm.dial(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 connection;
mod copy_future;
mod handler;
mod protocol;
mod transport;
pub use behaviour::{Relay, RelayConfig};
pub use connection::Connection;
pub use transport::{RelayError, RelayListener, RelayTransport};
use libp2p_core::Transport;
mod message_proto {
include!(concat!(env!("OUT_DIR"), "/message_v1.pb.rs"));
}
/// Create both a [`RelayTransport`] wrapping the provided [`Transport`]
/// as well as a [`Relay`] [`NetworkBehaviour`](libp2p_swarm::NetworkBehaviour).
///
/// Interconnects the returned [`RelayTransport`] and [`Relay`].
pub fn new_transport_and_behaviour<T: Transport + Clone>(
config: RelayConfig,
transport: T,
) -> (RelayTransport<T>, Relay) {
let (transport, from_transport) = RelayTransport::new(transport);
let behaviour = Relay::new(config, from_transport);
(transport, behaviour)
}
/// The ID of an outgoing / incoming, relay / destination request.
#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)]
pub struct RequestId(u64);
impl RequestId {
fn new() -> RequestId {
RequestId(rand::random())
}
}

View File

@ -18,11 +18,12 @@
// 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 crate::v1::handler::{
RelayHandlerConfig, RelayHandlerEvent, RelayHandlerIn, RelayHandlerProto,
};
use crate::v1::message_proto::circuit_relay;
use crate::v1::transport::{OutgoingRelayReqError, TransportToBehaviourMsg};
use crate::v1::{protocol, Connection, RequestId};
use futures::channel::{mpsc, oneshot};
use futures::prelude::*;
use libp2p_core::connection::{ConnectedPoint, ConnectionId, ListenerId};
@ -40,10 +41,10 @@ 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`].
/// Channel receiver from [`crate::v1::RelayTransport`].
from_transport: mpsc::Receiver<TransportToBehaviourMsg>,
/// Events that need to be send to a [`RelayListener`](crate::transport::RelayListener) via
/// Events that need to be send to a [`RelayListener`](crate::v1::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.
@ -84,11 +85,11 @@ struct OutgoingDialingRelayReq {
relay_addr: Multiaddr,
dst_addr: Option<Multiaddr>,
dst_peer_id: PeerId,
send_back: oneshot::Sender<Result<protocol::Connection, OutgoingRelayReqError>>,
send_back: oneshot::Sender<Result<Connection, OutgoingRelayReqError>>,
}
struct OutgoingUpgradingRelayReq {
send_back: oneshot::Sender<Result<protocol::Connection, OutgoingRelayReqError>>,
send_back: oneshot::Sender<Result<Connection, OutgoingRelayReqError>>,
}
enum IncomingRelayReq {
@ -772,18 +773,6 @@ impl NetworkBehaviour for Relay {
}
}
#[derive(Debug)]
#[allow(clippy::large_enum_variant)]
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,
@ -797,7 +786,7 @@ enum RelayListener {
impl RelayListener {
/// Returns whether the channel to the
/// [`RelayListener`](crate::transport::RelayListener) is closed.
/// [`RelayListener`](crate::v1::RelayListener) is closed.
fn is_closed(&self) -> bool {
match self {
RelayListener::Connecting { to_listener, .. }
@ -806,7 +795,14 @@ impl RelayListener {
}
}
#[derive(Debug, Eq, PartialEq)]
pub enum OutgoingRelayReqError {
DialingRelay,
#[derive(Debug)]
#[allow(clippy::large_enum_variant)]
pub enum BehaviourToListenerMsg {
ConnectionToRelayEstablished,
IncomingRelayedConnection {
stream: Connection,
src_peer_id: PeerId,
relay_peer_id: PeerId,
relay_addr: Multiaddr,
},
}

View File

@ -0,0 +1,95 @@
// 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 bytes::Bytes;
use futures::channel::oneshot;
use futures::io::{AsyncRead, AsyncWrite};
use libp2p_swarm::NegotiatedSubstream;
use std::io::{Error, IoSlice};
use std::pin::Pin;
use std::task::{Context, Poll};
/// 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 {
pub fn new(
initial_data: Bytes,
stream: NegotiatedSubstream,
notifier: oneshot::Sender<()>,
) -> Self {
Connection {
initial_data,
stream,
_notifier: notifier,
}
}
}
impl AsyncWrite for Connection {
fn poll_write(
mut self: Pin<&mut Self>,
cx: &mut Context,
buf: &[u8],
) -> Poll<Result<usize, Error>> {
Pin::new(&mut self.stream).poll_write(cx, buf)
}
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
Pin::new(&mut self.stream).poll_flush(cx)
}
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
Pin::new(&mut self.stream).poll_close(cx)
}
fn poll_write_vectored(
mut self: Pin<&mut Self>,
cx: &mut Context,
bufs: &[IoSlice],
) -> Poll<Result<usize, Error>> {
Pin::new(&mut self.stream).poll_write_vectored(cx, bufs)
}
}
impl AsyncRead for Connection {
fn poll_read(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
buf: &mut [u8],
) -> Poll<Result<usize, Error>> {
if !self.initial_data.is_empty() {
let n = std::cmp::min(self.initial_data.len(), buf.len());
let data = self.initial_data.split_to(n);
buf[0..n].copy_from_slice(&data[..]);
return Poll::Ready(Ok(n));
}
Pin::new(&mut self.stream).poll_read(cx, buf)
}
}

View File

@ -18,9 +18,8 @@
// 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 crate::v1::message_proto::circuit_relay;
use crate::v1::{protocol, Connection, RequestId};
use futures::channel::oneshot::{self, Canceled};
use futures::future::BoxFuture;
use futures::prelude::*;
@ -93,10 +92,7 @@ pub struct RelayHandler {
accept_dst_futures: FuturesUnordered<
BoxFuture<
'static,
Result<
(PeerId, protocol::Connection, oneshot::Receiver<()>),
protocol::IncomingDstReqError,
>,
Result<(PeerId, Connection, oneshot::Receiver<()>), protocol::IncomingDstReqError>,
>,
>,
/// Futures that copy from a source to a destination.
@ -108,8 +104,8 @@ pub struct RelayHandler {
/// 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).
/// [`Connection`](Connection) to the
/// [`RelayTransport`](crate::v1::RelayTransport).
///
/// For each substream to the peer of this handler, there is a future in here that resolves once
/// the given substream is dropped.
@ -169,7 +165,7 @@ pub enum RelayHandlerEvent {
/// > **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),
OutgoingRelayReqSuccess(PeerId, RequestId, Connection),
/// The local node has accepted an incoming destination request. Contains a substream that
/// communicates with the source.
@ -178,7 +174,7 @@ pub enum RelayHandlerEvent {
/// > encryption handshake has to be performed on top of this substream in order to
/// > avoid MITM attacks.
IncomingDstReqSuccess {
stream: protocol::Connection,
stream: Connection,
src_peer_id: PeerId,
relay_peer_id: PeerId,
relay_addr: Multiaddr,
@ -429,6 +425,7 @@ impl ProtocolsHandler for RelayHandler {
"Can not successfully dial a destination when actually dialing a relay."
),
};
// TODO: Should this not be driven by the src handler?
self.copy_futures
.push(incoming_relay_req.fulfill(to_dest_substream, from_dst_read_buffer));
}

View File

@ -1,5 +1,5 @@
syntax = "proto2";
package message.pb;
package message_v1.pb;
message CircuitRelay {

View File

@ -18,28 +18,12 @@
// 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::v1::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;
@ -55,7 +39,14 @@ pub use self::outgoing_dst_req::{OutgoingDstReq, OutgoingDstReqError};
mod listen;
pub use self::listen::{RelayListen, RelayListenError, RelayRemoteReq};
pub mod copy_future;
/// 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";
/// Representation of a `CircuitRelay_Peer` protobuf message with refined field types.
///
@ -112,71 +103,3 @@ impl error::Error for PeerParseError {
}
}
}
/// A [`NegotiatedSubstream`] acting as a relayed [`Connection`].
#[derive(Debug)]
pub struct Connection {
/// [`Connection`] might at first return data, that was already read during relay negotiation.
initial_data: Bytes,
stream: NegotiatedSubstream,
/// Notifies the other side of the channel of this [`Connection`] being dropped.
_notifier: oneshot::Sender<()>,
}
impl Unpin for Connection {}
impl Connection {
fn new(
initial_data: Bytes,
stream: NegotiatedSubstream,
notifier: oneshot::Sender<()>,
) -> Self {
Connection {
initial_data,
stream,
_notifier: notifier,
}
}
}
impl AsyncWrite for Connection {
fn poll_write(
mut self: Pin<&mut Self>,
cx: &mut Context,
buf: &[u8],
) -> Poll<Result<usize, Error>> {
Pin::new(&mut self.stream).poll_write(cx, buf)
}
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
Pin::new(&mut self.stream).poll_flush(cx)
}
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
Pin::new(&mut self.stream).poll_close(cx)
}
fn poll_write_vectored(
mut self: Pin<&mut Self>,
cx: &mut Context,
bufs: &[IoSlice],
) -> Poll<Result<usize, Error>> {
Pin::new(&mut self.stream).poll_write_vectored(cx, bufs)
}
}
impl AsyncRead for Connection {
fn poll_read(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
buf: &mut [u8],
) -> Poll<Result<usize, Error>> {
if !self.initial_data.is_empty() {
let n = std::cmp::min(self.initial_data.len(), buf.len());
let data = self.initial_data.split_to(n);
buf[0..n].copy_from_slice(&data[..]);
return Poll::Ready(Ok(n));
}
Pin::new(&mut self.stream).poll_read(cx, buf)
}
}

View File

@ -18,8 +18,9 @@
// 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 crate::v1::message_proto::{circuit_relay, CircuitRelay};
use crate::v1::protocol::Peer;
use crate::v1::Connection;
use asynchronous_codec::{Framed, FramedParts};
use bytes::BytesMut;
@ -69,10 +70,8 @@ impl IncomingDstReq {
/// 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>,
> {
) -> BoxFuture<'static, Result<(PeerId, Connection, oneshot::Receiver<()>), IncomingDstReqError>>
{
let IncomingDstReq { mut stream, src } = self;
let msg = CircuitRelay {
r#type: Some(circuit_relay::Type::Status.into()),
@ -102,7 +101,7 @@ impl IncomingDstReq {
Ok((
src.peer_id,
super::Connection::new(read_buffer.freeze(), io, tx),
Connection::new(read_buffer.freeze(), io, tx),
rx,
))
}

View File

@ -18,9 +18,9 @@
// 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 crate::v1::copy_future::CopyFuture;
use crate::v1::message_proto::{circuit_relay, circuit_relay::Status, CircuitRelay};
use crate::v1::protocol::Peer;
use asynchronous_codec::{Framed, FramedParts};
use bytes::{Bytes, BytesMut};

View File

@ -18,10 +18,11 @@
// 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 crate::v1::message_proto::{circuit_relay, CircuitRelay};
use crate::v1::protocol::incoming_dst_req::IncomingDstReq;
use crate::v1::protocol::incoming_relay_req::IncomingRelayReq;
use crate::v1::protocol::{Peer, PeerParseError};
use crate::v1::protocol::{MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME};
use asynchronous_codec::Framed;
use futures::channel::oneshot;
use futures::{future::BoxFuture, prelude::*};

View File

@ -18,8 +18,9 @@
// 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 crate::v1::message_proto::{circuit_relay, CircuitRelay};
use crate::v1::protocol::Peer;
use crate::v1::protocol::{MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME};
use asynchronous_codec::{Framed, FramedParts};
use bytes::Bytes;
use futures::future::BoxFuture;

View File

@ -18,8 +18,9 @@
// 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 crate::v1::message_proto::{circuit_relay, CircuitRelay};
use crate::v1::protocol::{MAX_ACCEPTED_MESSAGE_LEN, PROTOCOL_NAME};
use crate::v1::Connection;
use asynchronous_codec::{Framed, FramedParts};
use futures::channel::oneshot;
use futures::future::BoxFuture;
@ -68,7 +69,7 @@ impl upgrade::UpgradeInfo for OutgoingRelayReq {
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for OutgoingRelayReq {
type Output = (super::Connection, oneshot::Receiver<()>);
type Output = (Connection, oneshot::Receiver<()>);
type Error = OutgoingRelayReqError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
@ -156,7 +157,7 @@ impl upgrade::OutboundUpgrade<NegotiatedSubstream> for OutgoingRelayReq {
let (tx, rx) = oneshot::channel();
Ok((super::Connection::new(read_buffer.freeze(), io, tx), rx))
Ok((Connection::new(read_buffer.freeze(), io, tx), rx))
}
.boxed()
}

View File

@ -18,9 +18,8 @@
// 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 crate::v1::behaviour::BehaviourToListenerMsg;
use crate::v1::{Connection, RequestId};
use futures::channel::mpsc;
use futures::channel::oneshot;
use futures::future::{BoxFuture, Future, FutureExt};
@ -43,7 +42,7 @@ use std::task::{Context, Poll};
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, Transport};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour};
/// # use libp2p_relay::v1::{RelayConfig, new_transport_and_behaviour};
/// # let inner_transport = MemoryTransport::default();
/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour(
/// # RelayConfig::default(),
@ -57,7 +56,7 @@ use std::task::{Context, Poll};
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, PeerId, Transport};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour};
/// # use libp2p_relay::v1::{RelayConfig, new_transport_and_behaviour};
/// # let inner_transport = MemoryTransport::default();
/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour(
/// # RelayConfig::default(),
@ -77,7 +76,7 @@ use std::task::{Context, Poll};
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, PeerId, Transport};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour};
/// # use libp2p_relay::v1::{RelayConfig, new_transport_and_behaviour};
/// # let inner_transport = MemoryTransport::default();
/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour(
/// # RelayConfig::default(),
@ -98,7 +97,7 @@ use std::task::{Context, Poll};
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, PeerId, Transport};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour};
/// # use libp2p_relay::v1::{RelayConfig, new_transport_and_behaviour};
/// # let inner_transport = MemoryTransport::default();
/// # let (relay_transport, relay_behaviour) = new_transport_and_behaviour(
/// # RelayConfig::default(),
@ -121,7 +120,7 @@ impl<T: Clone> RelayTransport<T> {
///
///```
/// # use libp2p_core::transport::dummy::DummyTransport;
/// # use libp2p_relay::{RelayConfig, new_transport_and_behaviour};
/// # use libp2p_relay::v1::{RelayConfig, new_transport_and_behaviour};
///
/// let inner_transport = DummyTransport::<()>::new();
/// let (relay_transport, relay_behaviour) = new_transport_and_behaviour(
@ -143,7 +142,7 @@ impl<T: Clone> RelayTransport<T> {
}
impl<T: Transport + Clone> Transport for RelayTransport<T> {
type Output = EitherOutput<<T as Transport>::Output, protocol::Connection>;
type Output = EitherOutput<<T as Transport>::Output, Connection>;
type Error = EitherError<<T as Transport>::Error, RelayError>;
type Listener = RelayListener<T>;
type ListenerUpgrade = RelayedListenerUpgrade<T>;
@ -427,17 +426,17 @@ impl<T: Transport> Stream for RelayListener<T> {
}
}
pub type RelayedDial = BoxFuture<'static, Result<protocol::Connection, RelayError>>;
pub type RelayedDial = BoxFuture<'static, Result<Connection, RelayError>>;
#[pin_project(project = RelayedListenerUpgradeProj)]
pub enum RelayedListenerUpgrade<T: Transport> {
Inner(#[pin] <T as Transport>::ListenerUpgrade),
Relayed(Option<protocol::Connection>),
Relayed(Option<Connection>),
}
impl<T: Transport> Future for RelayedListenerUpgrade<T> {
type Output = Result<
EitherOutput<<T as Transport>::Output, protocol::Connection>,
EitherOutput<<T as Transport>::Output, Connection>,
EitherError<<T as Transport>::Error, RelayError>,
>;
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
@ -534,7 +533,7 @@ impl std::fmt::Display for RelayError {
impl std::error::Error for RelayError {}
/// Message from the [`RelayTransport`] to the [`Relay`](crate::Relay)
/// Message from the [`RelayTransport`] to the [`Relay`](crate::v1::Relay)
/// [`NetworkBehaviour`](libp2p_swarm::NetworkBehaviour).
pub enum TransportToBehaviourMsg {
/// Dial destination node via relay node.
@ -544,7 +543,7 @@ pub enum TransportToBehaviourMsg {
relay_peer_id: PeerId,
dst_addr: Option<Multiaddr>,
dst_peer_id: PeerId,
send_back: oneshot::Sender<Result<protocol::Connection, OutgoingRelayReqError>>,
send_back: oneshot::Sender<Result<Connection, OutgoingRelayReqError>>,
},
/// Listen for incoming relayed connections via relay node.
ListenReq {
@ -555,3 +554,8 @@ pub enum TransportToBehaviourMsg {
to_listener: mpsc::Sender<BehaviourToListenerMsg>,
},
}
#[derive(Debug, Eq, PartialEq)]
pub enum OutgoingRelayReqError {
DialingRelay,
}

41
protocols/relay/src/v2.rs Normal file
View File

@ -0,0 +1,41 @@
// Copyright 2021 Protocol Labs.
//
// 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 v2
//! specification](https://github.com/libp2p/specs/issues/314).
mod message_proto {
include!(concat!(env!("OUT_DIR"), "/message_v2.pb.rs"));
}
pub mod client;
mod copy_future;
mod protocol;
pub mod relay;
/// The ID of an outgoing / incoming, relay / destination request.
#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)]
pub struct RequestId(u64);
impl RequestId {
fn new() -> RequestId {
RequestId(rand::random())
}
}

View File

@ -0,0 +1,451 @@
// Copyright 2021 Protocol Labs.
//
// 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.
//! [`NetworkBehaviour`] to act as a circuit relay v2 **client**.
mod handler;
pub mod transport;
use crate::v2::protocol::{self, inbound_stop, outbound_hop};
use bytes::Bytes;
use either::Either;
use futures::channel::mpsc::Receiver;
use futures::channel::oneshot;
use futures::future::{BoxFuture, FutureExt};
use futures::io::{AsyncRead, AsyncWrite};
use futures::ready;
use futures::stream::StreamExt;
use libp2p_core::connection::{ConnectedPoint, ConnectionId};
use libp2p_core::{Multiaddr, PeerId};
use libp2p_swarm::dial_opts::DialOpts;
use libp2p_swarm::protocols_handler::DummyProtocolsHandler;
use libp2p_swarm::{
NegotiatedSubstream, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters,
ProtocolsHandlerUpgrErr,
};
use std::collections::{hash_map, HashMap, VecDeque};
use std::io::{Error, IoSlice};
use std::ops::DerefMut;
use std::pin::Pin;
use std::task::{Context, Poll};
/// The events produced by the [`Client`] behaviour.
#[derive(Debug)]
pub enum Event {
/// An outbound reservation has been accepted.
ReservationReqAccepted {
relay_peer_id: PeerId,
/// Indicates whether the request replaces an existing reservation.
renewal: bool,
limit: Option<protocol::Limit>,
},
ReservationReqFailed {
relay_peer_id: PeerId,
/// Indicates whether the request replaces an existing reservation.
renewal: bool,
error: ProtocolsHandlerUpgrErr<outbound_hop::ReservationFailedReason>,
},
OutboundCircuitEstablished {
relay_peer_id: PeerId,
limit: Option<protocol::Limit>,
},
OutboundCircuitReqFailed {
relay_peer_id: PeerId,
error: ProtocolsHandlerUpgrErr<outbound_hop::CircuitFailedReason>,
},
/// An inbound circuit has been established.
InboundCircuitEstablished {
src_peer_id: PeerId,
limit: Option<protocol::Limit>,
},
InboundCircuitReqFailed {
relay_peer_id: PeerId,
error: ProtocolsHandlerUpgrErr<void::Void>,
},
/// An inbound circuit request has been denied.
InboundCircuitReqDenied { src_peer_id: PeerId },
/// Denying an inbound circuit request failed.
InboundCircuitReqDenyFailed {
src_peer_id: PeerId,
error: std::io::Error,
},
}
pub struct Client {
local_peer_id: PeerId,
from_transport: Receiver<transport::TransportToBehaviourMsg>,
/// Set of directly connected peers, i.e. not connected via a relayed
/// connection.
directly_connected_peers: HashMap<PeerId, Vec<ConnectionId>>,
/// Queue of actions to return when polled.
queued_actions: VecDeque<Event>,
}
impl Client {
pub fn new_transport_and_behaviour(
local_peer_id: PeerId,
) -> (transport::ClientTransport, Self) {
let (transport, from_transport) = transport::ClientTransport::new();
let behaviour = Client {
local_peer_id,
from_transport,
directly_connected_peers: Default::default(),
queued_actions: Default::default(),
};
(transport, behaviour)
}
}
impl NetworkBehaviour for Client {
type ProtocolsHandler = handler::Prototype;
type OutEvent = Event;
fn new_handler(&mut self) -> Self::ProtocolsHandler {
handler::Prototype::new(self.local_peer_id, None)
}
fn inject_connection_established(
&mut self,
peer_id: &PeerId,
connection_id: &ConnectionId,
endpoint: &ConnectedPoint,
_failed_addresses: Option<&Vec<Multiaddr>>,
) {
if !endpoint.is_relayed() {
self.directly_connected_peers
.entry(*peer_id)
.or_default()
.push(*connection_id);
}
}
fn inject_connection_closed(
&mut self,
peer_id: &PeerId,
connection_id: &ConnectionId,
endpoint: &ConnectedPoint,
_handler: Either<handler::Handler, DummyProtocolsHandler>,
) {
if !endpoint.is_relayed() {
match self.directly_connected_peers.entry(*peer_id) {
hash_map::Entry::Occupied(mut connections) => {
let position = connections
.get()
.iter()
.position(|c| c == connection_id)
.expect("Connection to be known.");
connections.get_mut().remove(position);
if connections.get().is_empty() {
connections.remove();
}
}
hash_map::Entry::Vacant(_) => {
unreachable!("`inject_connection_closed` for unconnected peer.")
}
};
}
}
fn inject_event(
&mut self,
event_source: PeerId,
_connection: ConnectionId,
handler_event: Either<handler::Event, void::Void>,
) {
let handler_event = match handler_event {
Either::Left(e) => e,
Either::Right(v) => void::unreachable(v),
};
match handler_event {
handler::Event::ReservationReqAccepted { renewal, limit } => self
.queued_actions
.push_back(Event::ReservationReqAccepted {
relay_peer_id: event_source,
renewal,
limit,
}),
handler::Event::ReservationReqFailed { renewal, error } => {
self.queued_actions.push_back(Event::ReservationReqFailed {
relay_peer_id: event_source,
renewal,
error,
})
}
handler::Event::OutboundCircuitEstablished { limit } => {
self.queued_actions
.push_back(Event::OutboundCircuitEstablished {
relay_peer_id: event_source,
limit,
})
}
handler::Event::OutboundCircuitReqFailed { error } => {
self.queued_actions
.push_back(Event::OutboundCircuitReqFailed {
relay_peer_id: event_source,
error,
})
}
handler::Event::InboundCircuitEstablished { src_peer_id, limit } => self
.queued_actions
.push_back(Event::InboundCircuitEstablished { src_peer_id, limit }),
handler::Event::InboundCircuitReqFailed { error } => {
self.queued_actions
.push_back(Event::InboundCircuitReqFailed {
relay_peer_id: event_source,
error,
})
}
handler::Event::InboundCircuitReqDenied { src_peer_id } => self
.queued_actions
.push_back(Event::InboundCircuitReqDenied { src_peer_id }),
handler::Event::InboundCircuitReqDenyFailed { src_peer_id, error } => self
.queued_actions
.push_back(Event::InboundCircuitReqDenyFailed { src_peer_id, error }),
}
}
fn poll(
&mut self,
cx: &mut Context<'_>,
_poll_parameters: &mut impl PollParameters,
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
if let Some(event) = self.queued_actions.pop_front() {
return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event));
}
let action = match ready!(self.from_transport.poll_next_unpin(cx)) {
Some(transport::TransportToBehaviourMsg::ListenReq {
relay_peer_id,
relay_addr,
to_listener,
}) => {
match self
.directly_connected_peers
.get(&relay_peer_id)
.and_then(|cs| cs.get(0))
{
Some(connection_id) => NetworkBehaviourAction::NotifyHandler {
peer_id: relay_peer_id,
handler: NotifyHandler::One(*connection_id),
event: Either::Left(handler::In::Reserve { to_listener }),
},
None => {
let handler = handler::Prototype::new(
self.local_peer_id,
Some(handler::In::Reserve { to_listener }),
);
NetworkBehaviourAction::Dial {
opts: DialOpts::peer_id(relay_peer_id)
.addresses(vec![relay_addr])
.extend_addresses_through_behaviour()
.build(),
handler,
}
}
}
}
Some(transport::TransportToBehaviourMsg::DialReq {
relay_addr,
relay_peer_id,
dst_peer_id,
send_back,
..
}) => {
match self
.directly_connected_peers
.get(&relay_peer_id)
.and_then(|cs| cs.get(0))
{
Some(connection_id) => NetworkBehaviourAction::NotifyHandler {
peer_id: relay_peer_id,
handler: NotifyHandler::One(*connection_id),
event: Either::Left(handler::In::EstablishCircuit {
send_back,
dst_peer_id,
}),
},
None => {
let handler = handler::Prototype::new(
self.local_peer_id,
Some(handler::In::EstablishCircuit {
send_back,
dst_peer_id,
}),
);
NetworkBehaviourAction::Dial {
opts: DialOpts::peer_id(relay_peer_id)
.addresses(vec![relay_addr])
.extend_addresses_through_behaviour()
.build(),
handler,
}
}
}
}
None => unreachable!(
"`Relay` `NetworkBehaviour` polled after channel from \
`RelayTransport` has been closed. Unreachable under \
the assumption that the `Client` is never polled after \
`ClientTransport` is dropped.",
),
};
return Poll::Ready(action);
}
}
/// A [`NegotiatedSubstream`] acting as a [`RelayedConnection`].
pub enum RelayedConnection {
InboundAccepting {
accept: BoxFuture<'static, Result<RelayedConnection, std::io::Error>>,
},
Operational {
read_buffer: Bytes,
substream: NegotiatedSubstream,
drop_notifier: oneshot::Sender<void::Void>,
},
}
impl Unpin for RelayedConnection {}
impl RelayedConnection {
pub(crate) fn new_inbound(
circuit: inbound_stop::Circuit,
drop_notifier: oneshot::Sender<void::Void>,
) -> Self {
RelayedConnection::InboundAccepting {
accept: async {
let (substream, read_buffer) = circuit.accept().await?;
Ok(RelayedConnection::Operational {
read_buffer,
substream,
drop_notifier,
})
}
.boxed(),
}
}
pub(crate) fn new_outbound(
substream: NegotiatedSubstream,
read_buffer: Bytes,
drop_notifier: oneshot::Sender<void::Void>,
) -> Self {
RelayedConnection::Operational {
substream,
read_buffer,
drop_notifier,
}
}
}
impl AsyncWrite for RelayedConnection {
fn poll_write(
mut self: Pin<&mut Self>,
cx: &mut Context,
buf: &[u8],
) -> Poll<Result<usize, Error>> {
loop {
match self.deref_mut() {
RelayedConnection::InboundAccepting { accept } => {
*self = ready!(accept.poll_unpin(cx))?;
}
RelayedConnection::Operational { substream, .. } => {
return Pin::new(substream).poll_write(cx, buf);
}
}
}
}
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
loop {
match self.deref_mut() {
RelayedConnection::InboundAccepting { accept } => {
*self = ready!(accept.poll_unpin(cx))?;
}
RelayedConnection::Operational { substream, .. } => {
return Pin::new(substream).poll_flush(cx);
}
}
}
}
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Error>> {
loop {
match self.deref_mut() {
RelayedConnection::InboundAccepting { accept } => {
*self = ready!(accept.poll_unpin(cx))?;
}
RelayedConnection::Operational { substream, .. } => {
return Pin::new(substream).poll_close(cx);
}
}
}
}
fn poll_write_vectored(
mut self: Pin<&mut Self>,
cx: &mut Context,
bufs: &[IoSlice],
) -> Poll<Result<usize, Error>> {
loop {
match self.deref_mut() {
RelayedConnection::InboundAccepting { accept } => {
*self = ready!(accept.poll_unpin(cx))?;
}
RelayedConnection::Operational { substream, .. } => {
return Pin::new(substream).poll_write_vectored(cx, bufs);
}
}
}
}
}
impl AsyncRead for RelayedConnection {
fn poll_read(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
buf: &mut [u8],
) -> Poll<Result<usize, Error>> {
loop {
match self.deref_mut() {
RelayedConnection::InboundAccepting { accept } => {
*self = ready!(accept.poll_unpin(cx))?;
}
RelayedConnection::Operational {
read_buffer,
substream,
..
} => {
if !read_buffer.is_empty() {
let n = std::cmp::min(read_buffer.len(), buf.len());
let data = read_buffer.split_to(n);
buf[0..n].copy_from_slice(&data[..]);
return Poll::Ready(Ok(n));
}
return Pin::new(substream).poll_read(cx, buf);
}
}
}
}
}

View File

@ -0,0 +1,725 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::client::transport;
use crate::v2::message_proto::Status;
use crate::v2::protocol::{self, inbound_stop, outbound_hop};
use either::Either;
use futures::channel::{mpsc, oneshot};
use futures::future::{BoxFuture, FutureExt};
use futures::sink::SinkExt;
use futures::stream::{FuturesUnordered, StreamExt};
use futures_timer::Delay;
use instant::Instant;
use libp2p_core::either::EitherError;
use libp2p_core::multiaddr::Protocol;
use libp2p_core::{upgrade, ConnectedPoint, Multiaddr, PeerId};
use libp2p_swarm::protocols_handler::{
DummyProtocolsHandler, InboundUpgradeSend, OutboundUpgradeSend, SendWrapper,
};
use libp2p_swarm::{
IntoProtocolsHandler, KeepAlive, NegotiatedSubstream, ProtocolsHandler, ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr, SubstreamProtocol,
};
use log::debug;
use std::collections::VecDeque;
use std::fmt;
use std::task::{Context, Poll};
use std::time::Duration;
pub enum In {
Reserve {
to_listener: mpsc::Sender<transport::ToListenerMsg>,
},
EstablishCircuit {
dst_peer_id: PeerId,
send_back: oneshot::Sender<Result<super::RelayedConnection, ()>>,
},
}
impl fmt::Debug for In {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
In::Reserve { to_listener: _ } => f.debug_struct("In::Reserve").finish(),
In::EstablishCircuit {
dst_peer_id,
send_back: _,
} => f
.debug_struct("In::EstablishCircuit")
.field("dst_peer_id", dst_peer_id)
.finish(),
}
}
}
#[derive(Debug)]
pub enum Event {
ReservationReqAccepted {
/// Indicates whether the request replaces an existing reservation.
renewal: bool,
limit: Option<protocol::Limit>,
},
ReservationReqFailed {
/// Indicates whether the request replaces an existing reservation.
renewal: bool,
error: ProtocolsHandlerUpgrErr<outbound_hop::ReservationFailedReason>,
},
/// An outbound circuit has been established.
OutboundCircuitEstablished { limit: Option<protocol::Limit> },
OutboundCircuitReqFailed {
error: ProtocolsHandlerUpgrErr<outbound_hop::CircuitFailedReason>,
},
/// An inbound circuit has been established.
InboundCircuitEstablished {
src_peer_id: PeerId,
limit: Option<protocol::Limit>,
},
/// An inbound circuit request has failed.
InboundCircuitReqFailed {
error: ProtocolsHandlerUpgrErr<void::Void>,
},
/// An inbound circuit request has been denied.
InboundCircuitReqDenied { src_peer_id: PeerId },
/// Denying an inbound circuit request failed.
InboundCircuitReqDenyFailed {
src_peer_id: PeerId,
error: std::io::Error,
},
}
pub struct Prototype {
local_peer_id: PeerId,
/// Initial [`In`] event from [`super::Client`] provided at creation time.
initial_in: Option<In>,
}
impl Prototype {
pub(crate) fn new(local_peer_id: PeerId, initial_in: Option<In>) -> Self {
Self {
local_peer_id,
initial_in,
}
}
}
impl IntoProtocolsHandler for Prototype {
type Handler = Either<Handler, DummyProtocolsHandler>;
fn into_handler(self, remote_peer_id: &PeerId, endpoint: &ConnectedPoint) -> Self::Handler {
if endpoint.is_relayed() {
if let Some(event) = self.initial_in {
debug!(
"Established relayed instead of direct connection to {:?}, \
dropping initial in event {:?}.",
remote_peer_id, event
);
}
// Deny all substreams on relayed connection.
Either::Right(DummyProtocolsHandler::default())
} else {
let mut handler = Handler {
remote_peer_id: *remote_peer_id,
remote_addr: endpoint.get_remote_address().clone(),
local_peer_id: self.local_peer_id,
queued_events: Default::default(),
pending_error: Default::default(),
reservation: Reservation::None,
alive_lend_out_substreams: Default::default(),
circuit_deny_futs: Default::default(),
send_error_futs: Default::default(),
keep_alive: KeepAlive::Yes,
};
if let Some(event) = self.initial_in {
handler.inject_event(event)
}
Either::Left(handler)
}
}
fn inbound_protocol(&self) -> <Self::Handler as ProtocolsHandler>::InboundProtocol {
upgrade::EitherUpgrade::A(SendWrapper(inbound_stop::Upgrade {}))
}
}
pub struct Handler {
local_peer_id: PeerId,
remote_peer_id: PeerId,
remote_addr: Multiaddr,
/// A pending fatal error that results in the connection being closed.
pending_error: Option<
ProtocolsHandlerUpgrErr<
EitherError<inbound_stop::FatalUpgradeError, outbound_hop::FatalUpgradeError>,
>,
>,
/// Until when to keep the connection alive.
keep_alive: KeepAlive,
/// Queue of events to return when polled.
queued_events: VecDeque<
ProtocolsHandlerEvent<
<Self as ProtocolsHandler>::OutboundProtocol,
<Self as ProtocolsHandler>::OutboundOpenInfo,
<Self as ProtocolsHandler>::OutEvent,
<Self as ProtocolsHandler>::Error,
>,
>,
reservation: Reservation,
/// Tracks substreams lent out to the transport.
///
/// Contains a [`futures::future::Future`] for each lend out substream that
/// resolves once the 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<void::Void>>,
circuit_deny_futs: FuturesUnordered<BoxFuture<'static, (PeerId, Result<(), std::io::Error>)>>,
/// Futures that try to send errors to the transport.
///
/// We may drop errors if this handler ends up in a terminal state (by returning
/// [`ProtocolsHandlerEvent::Close`]).
send_error_futs: FuturesUnordered<BoxFuture<'static, ()>>,
}
impl ProtocolsHandler for Handler {
type InEvent = In;
type OutEvent = Event;
type Error = ProtocolsHandlerUpgrErr<
EitherError<inbound_stop::FatalUpgradeError, outbound_hop::FatalUpgradeError>,
>;
type InboundProtocol = inbound_stop::Upgrade;
type OutboundProtocol = outbound_hop::Upgrade;
type OutboundOpenInfo = OutboundOpenInfo;
type InboundOpenInfo = ();
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
SubstreamProtocol::new(inbound_stop::Upgrade {}, ())
}
fn inject_fully_negotiated_inbound(
&mut self,
inbound_circuit: inbound_stop::Circuit,
_: Self::InboundOpenInfo,
) {
match &mut self.reservation {
Reservation::Accepted { pending_msgs, .. }
| Reservation::Renewing { pending_msgs, .. } => {
let src_peer_id = inbound_circuit.src_peer_id();
let limit = inbound_circuit.limit();
let (tx, rx) = oneshot::channel();
self.alive_lend_out_substreams.push(rx);
let connection = super::RelayedConnection::new_inbound(inbound_circuit, tx);
pending_msgs.push_back(transport::ToListenerMsg::IncomingRelayedConnection {
stream: connection,
src_peer_id,
relay_peer_id: self.remote_peer_id,
relay_addr: self.remote_addr.clone(),
});
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::InboundCircuitEstablished {
src_peer_id: self.remote_peer_id,
limit,
},
));
}
Reservation::None => {
let src_peer_id = inbound_circuit.src_peer_id();
self.circuit_deny_futs.push(
inbound_circuit
.deny(Status::NoReservation)
.map(move |result| (src_peer_id, result))
.boxed(),
)
}
}
}
fn inject_fully_negotiated_outbound(
&mut self,
output: <Self::OutboundProtocol as upgrade::OutboundUpgrade<NegotiatedSubstream>>::Output,
info: Self::OutboundOpenInfo,
) {
match (output, info) {
// Outbound reservation
(
outbound_hop::Output::Reservation {
renewal_timeout,
addrs,
limit,
},
OutboundOpenInfo::Reserve { to_listener },
) => {
let event = self.reservation.accepted(
renewal_timeout,
addrs,
to_listener,
self.local_peer_id,
limit,
);
self.queued_events
.push_back(ProtocolsHandlerEvent::Custom(event));
}
// Outbound circuit
(
outbound_hop::Output::Circuit {
substream,
read_buffer,
limit,
},
OutboundOpenInfo::Connect { send_back },
) => {
let (tx, rx) = oneshot::channel();
match send_back.send(Ok(super::RelayedConnection::new_outbound(
substream,
read_buffer,
tx,
))) {
Ok(()) => {
self.alive_lend_out_substreams.push(rx);
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::OutboundCircuitEstablished { limit },
));
}
Err(_) => debug!(
"Oneshot to `RelayedDial` future dropped. \
Dropping established relayed connection to {:?}.",
self.remote_peer_id,
),
}
}
_ => unreachable!(),
}
}
fn inject_event(&mut self, event: Self::InEvent) {
match event {
In::Reserve { to_listener } => {
self.queued_events
.push_back(ProtocolsHandlerEvent::OutboundSubstreamRequest {
protocol: SubstreamProtocol::new(
outbound_hop::Upgrade::Reserve,
OutboundOpenInfo::Reserve { to_listener },
),
});
}
In::EstablishCircuit {
send_back,
dst_peer_id,
} => {
self.queued_events
.push_back(ProtocolsHandlerEvent::OutboundSubstreamRequest {
protocol: SubstreamProtocol::new(
outbound_hop::Upgrade::Connect { dst_peer_id },
OutboundOpenInfo::Connect { send_back },
),
});
}
}
}
fn inject_listen_upgrade_error(
&mut self,
_: Self::InboundOpenInfo,
error: ProtocolsHandlerUpgrErr<<Self::InboundProtocol as InboundUpgradeSend>::Error>,
) {
let non_fatal_error = match error {
ProtocolsHandlerUpgrErr::Timeout => ProtocolsHandlerUpgrErr::Timeout,
ProtocolsHandlerUpgrErr::Timer => ProtocolsHandlerUpgrErr::Timer,
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select(
upgrade::NegotiationError::Failed,
)) => 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)),
));
return;
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(
inbound_stop::UpgradeError::Fatal(error),
)) => {
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Apply(EitherError::A(error)),
));
return;
}
};
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::InboundCircuitReqFailed {
error: non_fatal_error,
},
));
}
fn inject_dial_upgrade_error(
&mut self,
open_info: Self::OutboundOpenInfo,
error: ProtocolsHandlerUpgrErr<<Self::OutboundProtocol as OutboundUpgradeSend>::Error>,
) {
match open_info {
OutboundOpenInfo::Reserve { mut to_listener } => {
let non_fatal_error = match error {
ProtocolsHandlerUpgrErr::Timeout => ProtocolsHandlerUpgrErr::Timeout,
ProtocolsHandlerUpgrErr::Timer => ProtocolsHandlerUpgrErr::Timer,
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select(
upgrade::NegotiationError::Failed,
)) => 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),
),
));
return;
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(error)) => {
match error {
outbound_hop::UpgradeError::Fatal(error) => {
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Apply(EitherError::B(error)),
));
return;
}
outbound_hop::UpgradeError::ReservationFailed(error) => {
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(
error,
))
}
outbound_hop::UpgradeError::CircuitFailed(_) => {
unreachable!(
"Do not emitt `CircuitFailed` for outgoing reservation."
)
}
}
}
};
if self.pending_error.is_none() {
self.send_error_futs.push(
async move {
let _ = to_listener
.send(transport::ToListenerMsg::Reservation(Err(())))
.await;
}
.boxed(),
);
} else {
// Fatal error occured, thus handler is closing as quickly as possible.
// Transport is notified through dropping `to_listener`.
}
let renewal = self.reservation.failed();
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::ReservationReqFailed {
renewal,
error: non_fatal_error,
},
));
}
OutboundOpenInfo::Connect { send_back } => {
let non_fatal_error = match error {
ProtocolsHandlerUpgrErr::Timeout => ProtocolsHandlerUpgrErr::Timeout,
ProtocolsHandlerUpgrErr::Timer => ProtocolsHandlerUpgrErr::Timer,
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select(
upgrade::NegotiationError::Failed,
)) => 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),
),
));
return;
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(error)) => {
match error {
outbound_hop::UpgradeError::Fatal(error) => {
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Apply(EitherError::B(error)),
));
return;
}
outbound_hop::UpgradeError::CircuitFailed(error) => {
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(
error,
))
}
outbound_hop::UpgradeError::ReservationFailed(_) => {
unreachable!(
"Do not emitt `ReservationFailed` for outgoing circuit."
)
}
}
}
};
let _ = send_back.send(Err(()));
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::OutboundCircuitReqFailed {
error: non_fatal_error,
},
));
}
}
}
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));
}
// Return queued events.
if let Some(event) = self.queued_events.pop_front() {
return Poll::Ready(event);
}
if let Poll::Ready(Some(protocol)) = self.reservation.poll(cx) {
return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol });
}
// Deny incoming circuit requests.
if let Poll::Ready(Some((src_peer_id, result))) = self.circuit_deny_futs.poll_next_unpin(cx)
{
match result {
Ok(()) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::InboundCircuitReqDenied { src_peer_id },
))
}
Err(error) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::InboundCircuitReqDenyFailed { src_peer_id, error },
))
}
}
}
// Send errors to transport.
while let Poll::Ready(Some(())) = self.send_error_futs.poll_next_unpin(cx) {}
// Check status of lend out substreams.
loop {
match self.alive_lend_out_substreams.poll_next_unpin(cx) {
Poll::Ready(Some(Err(oneshot::Canceled))) => {}
Poll::Ready(Some(Ok(v))) => void::unreachable(v),
Poll::Ready(None) | Poll::Pending => break,
}
}
// Update keep-alive handling.
if matches!(self.reservation, Reservation::None)
&& self.alive_lend_out_substreams.is_empty()
&& self.circuit_deny_futs.is_empty()
{
match self.keep_alive {
KeepAlive::Yes => {
self.keep_alive = KeepAlive::Until(Instant::now() + Duration::from_secs(10));
}
KeepAlive::Until(_) => {}
KeepAlive::No => panic!("Handler never sets KeepAlive::No."),
}
} else {
self.keep_alive = KeepAlive::Yes;
}
Poll::Pending
}
}
enum Reservation {
/// The Reservation is accepted by the relay.
Accepted {
renewal_timeout: Delay,
/// Buffer of messages to be send to the transport listener.
pending_msgs: VecDeque<transport::ToListenerMsg>,
to_listener: mpsc::Sender<transport::ToListenerMsg>,
},
/// The reservation is being renewed with the relay.
Renewing {
/// Buffer of messages to be send to the transport listener.
pending_msgs: VecDeque<transport::ToListenerMsg>,
},
None,
}
impl Reservation {
fn accepted(
&mut self,
renewal_timeout: Delay,
addrs: Vec<Multiaddr>,
to_listener: mpsc::Sender<transport::ToListenerMsg>,
local_peer_id: PeerId,
limit: Option<protocol::Limit>,
) -> Event {
let (renewal, mut pending_msgs) = match std::mem::replace(self, Self::None) {
Reservation::Accepted { pending_msgs, .. }
| Reservation::Renewing { pending_msgs, .. } => (true, pending_msgs),
Reservation::None => (false, VecDeque::new()),
};
pending_msgs.push_back(transport::ToListenerMsg::Reservation(Ok(
transport::Reservation {
addrs: addrs
.into_iter()
.map(|a| {
a.with(Protocol::P2pCircuit)
.with(Protocol::P2p(local_peer_id.into()))
})
.collect(),
},
)));
*self = Reservation::Accepted {
renewal_timeout,
pending_msgs,
to_listener,
};
Event::ReservationReqAccepted { renewal, limit }
}
/// Marks the current reservation as failed.
///
/// Returns whether the reservation request was a renewal.
fn failed(&mut self) -> bool {
let renewal = matches!(
self,
Reservation::Accepted { .. } | Reservation::Renewing { .. }
);
*self = Reservation::None;
renewal
}
fn forward_messages_to_transport_listener(&mut self, cx: &mut Context<'_>) {
if let Reservation::Accepted {
pending_msgs,
to_listener,
..
} = self
{
if !pending_msgs.is_empty() {
match to_listener.poll_ready(cx) {
Poll::Ready(Ok(())) => {
if let Err(e) = to_listener
.start_send(pending_msgs.pop_front().expect("Called !is_empty()."))
{
debug!("Failed to sent pending message to listener: {:?}", e);
*self = Reservation::None;
}
}
Poll::Ready(Err(e)) => {
debug!("Channel to listener failed: {:?}", e);
*self = Reservation::None;
}
Poll::Pending => {}
}
}
}
}
fn poll(
&mut self,
cx: &mut Context<'_>,
) -> Poll<Option<SubstreamProtocol<outbound_hop::Upgrade, OutboundOpenInfo>>> {
self.forward_messages_to_transport_listener(cx);
// Check renewal timeout if any.
let (next_reservation, poll_val) = match std::mem::replace(self, Reservation::None) {
Reservation::Accepted {
mut renewal_timeout,
pending_msgs,
to_listener,
} => match renewal_timeout.poll_unpin(cx) {
Poll::Ready(()) => (
Reservation::Renewing { pending_msgs },
Poll::Ready(Some(SubstreamProtocol::new(
outbound_hop::Upgrade::Reserve,
OutboundOpenInfo::Reserve { to_listener },
))),
),
Poll::Pending => (
Reservation::Accepted {
renewal_timeout,
pending_msgs,
to_listener,
},
Poll::Pending,
),
},
r => (r, Poll::Pending),
};
*self = next_reservation;
poll_val
}
}
pub enum OutboundOpenInfo {
Reserve {
to_listener: mpsc::Sender<transport::ToListenerMsg>,
},
Connect {
send_back: oneshot::Sender<Result<super::RelayedConnection, ()>>,
},
}

View File

@ -0,0 +1,398 @@
// Copyright 2020 Parity Technologies (UK) Ltd.
// Copyright 2021 Protocol Labs.
//
// 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::v2::client::RelayedConnection;
use crate::v2::RequestId;
use futures::channel::mpsc;
use futures::channel::oneshot;
use futures::future::{ready, BoxFuture, Future, FutureExt, Ready};
use futures::ready;
use futures::sink::SinkExt;
use futures::stream::{Stream, StreamExt};
use libp2p_core::multiaddr::{Multiaddr, Protocol};
use libp2p_core::transport::{ListenerEvent, TransportError};
use libp2p_core::{PeerId, Transport};
use std::collections::VecDeque;
use std::pin::Pin;
use std::task::{Context, Poll};
use thiserror::Error;
/// A [`Transport`] enabling client relay capabilities.
///
/// Note: The transport only handles listening and dialing on relayed [`Multiaddr`], and depends on
/// an other transport to do the actual transmission of data. They should be combined through the
/// [`OrTransport`](libp2p_core::transport::choice::OrTransport).
///
/// Allows the local node to:
///
/// 1. Establish relayed connections by dialing `/p2p-circuit` addresses.
///
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, Transport, PeerId};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_core::transport::choice::OrTransport;
/// # use libp2p_relay::v2::client;
/// let actual_transport = MemoryTransport::default();
/// let (relay_transport, behaviour) = client::Client::new_transport_and_behaviour(
/// PeerId::random(),
/// );
/// let transport = OrTransport::new(relay_transport, actual_transport);
/// # let relay_id = PeerId::random();
/// # let destination_id = PeerId::random();
/// let dst_addr_via_relay = Multiaddr::empty()
/// .with(Protocol::Memory(40)) // Relay address.
/// .with(Protocol::P2p(relay_id.into())) // Relay peer id.
/// .with(Protocol::P2pCircuit) // Signal to connect via relay and not directly.
/// .with(Protocol::P2p(destination_id.into())); // Destination peer id.
/// transport.dial(dst_addr_via_relay).unwrap();
/// ```
///
/// 3. Listen for incoming relayed connections via specific relay.
///
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, Transport, PeerId};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_core::transport::choice::OrTransport;
/// # use libp2p_relay::v2::client;
/// # let relay_id = PeerId::random();
/// # let local_peer_id = PeerId::random();
/// let actual_transport = MemoryTransport::default();
/// let (relay_transport, behaviour) = client::Client::new_transport_and_behaviour(
/// local_peer_id,
/// );
/// let transport = OrTransport::new(relay_transport, actual_transport);
/// let relay_addr = Multiaddr::empty()
/// .with(Protocol::Memory(40)) // Relay address.
/// .with(Protocol::P2p(relay_id.into())) // Relay peer id.
/// .with(Protocol::P2pCircuit); // Signal to listen via remote relay node.
/// transport.listen_on(relay_addr).unwrap();
/// ```
#[derive(Clone)]
pub struct ClientTransport {
to_behaviour: mpsc::Sender<TransportToBehaviourMsg>,
}
impl ClientTransport {
/// Create a new [`ClientTransport`].
///
/// Note: The transport only handles listening and dialing on relayed [`Multiaddr`], and depends on
/// an other transport to do the actual transmission of data. They should be combined through the
/// [`OrTransport`](libp2p_core::transport::choice::OrTransport).
///
/// ```
/// # use libp2p_core::{Multiaddr, multiaddr::{Protocol}, Transport, PeerId};
/// # use libp2p_core::transport::memory::MemoryTransport;
/// # use libp2p_core::transport::choice::OrTransport;
/// # use libp2p_relay::v2::client;
/// let actual_transport = MemoryTransport::default();
/// let (relay_transport, behaviour) = client::Client::new_transport_and_behaviour(
/// PeerId::random(),
/// );
///
/// // To reduce unnecessary connection attempts, put `relay_transport` first.
/// let transport = OrTransport::new(relay_transport, actual_transport);
/// ```
pub(crate) fn new() -> (Self, mpsc::Receiver<TransportToBehaviourMsg>) {
let (to_behaviour, from_transport) = mpsc::channel(0);
(ClientTransport { to_behaviour }, from_transport)
}
}
impl Transport for ClientTransport {
type Output = RelayedConnection;
type Error = RelayError;
type Listener = RelayListener;
type ListenerUpgrade = Ready<Result<Self::Output, Self::Error>>;
type Dial = RelayedDial;
fn listen_on(self, addr: Multiaddr) -> Result<Self::Listener, TransportError<Self::Error>> {
let (relay_peer_id, relay_addr) = match parse_relayed_multiaddr(addr)? {
RelayedMultiaddr {
relay_peer_id: None,
relay_addr: _,
..
} => return Err(RelayError::MissingDstPeerId.into()),
RelayedMultiaddr {
relay_peer_id: _,
relay_addr: None,
..
} => return Err(RelayError::MissingRelayAddr.into()),
RelayedMultiaddr {
relay_peer_id: Some(peer_id),
relay_addr: Some(addr),
..
} => (peer_id, addr),
};
let (to_listener, from_behaviour) = mpsc::channel(0);
let mut to_behaviour = self.to_behaviour;
let msg_to_behaviour = Some(
async move {
to_behaviour
.send(TransportToBehaviourMsg::ListenReq {
relay_peer_id,
relay_addr,
to_listener,
})
.await
}
.boxed(),
);
Ok(RelayListener {
queued_new_addresses: Default::default(),
from_behaviour,
msg_to_behaviour,
})
}
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let RelayedMultiaddr {
relay_peer_id,
relay_addr,
dst_peer_id,
dst_addr,
} = parse_relayed_multiaddr(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;
Ok(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?.map_err(|()| RelayError::Connect)?;
Ok(stream)
}
.boxed())
}
fn address_translation(&self, _server: &Multiaddr, _observed: &Multiaddr) -> Option<Multiaddr> {
None
}
}
#[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`].
fn parse_relayed_multiaddr(
addr: Multiaddr,
) -> Result<RelayedMultiaddr, TransportError<RelayError>> {
if !addr.iter().any(|p| matches!(p, Protocol::P2pCircuit)) {
return Err(TransportError::MultiaddrNotSupported(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 {
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() {
Err(RelayError::MalformedMultiaddr)?;
}
relayed_multiaddr.relay_peer_id = Some(peer_id)
} else {
if relayed_multiaddr.dst_peer_id.is_some() {
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(relayed_multiaddr)
}
pub struct RelayListener {
queued_new_addresses: VecDeque<Multiaddr>,
from_behaviour: mpsc::Receiver<ToListenerMsg>,
msg_to_behaviour: Option<BoxFuture<'static, Result<(), mpsc::SendError>>>,
}
impl Unpin for RelayListener {}
impl Stream for RelayListener {
type Item =
Result<ListenerEvent<Ready<Result<RelayedConnection, RelayError>>, RelayError>, RelayError>;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
loop {
if let Some(msg) = &mut self.msg_to_behaviour {
match Future::poll(msg.as_mut(), cx) {
Poll::Ready(Ok(())) => self.msg_to_behaviour = None,
Poll::Ready(Err(e)) => return Poll::Ready(Some(Err(e.into()))),
Poll::Pending => {}
}
}
if let Some(addr) = self.queued_new_addresses.pop_front() {
return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr))));
}
let msg = match ready!(self.from_behaviour.poll_next_unpin(cx)) {
Some(msg) => msg,
None => {
// Sender of `from_behaviour` has been dropped, signaling listener to close.
return Poll::Ready(None);
}
};
let result = match msg {
ToListenerMsg::Reservation(Ok(Reservation { addrs })) => {
debug_assert!(
self.queued_new_addresses.is_empty(),
"Assert empty due to previous `pop_front` attempt."
);
// Returned as [`ListenerEvent::NewAddress`] in next iteration of loop.
self.queued_new_addresses = addrs.into();
continue;
}
ToListenerMsg::IncomingRelayedConnection {
stream,
src_peer_id,
relay_addr,
relay_peer_id: _,
} => Ok(ListenerEvent::Upgrade {
upgrade: ready(Ok(stream)),
local_addr: relay_addr.with(Protocol::P2pCircuit),
remote_addr: Protocol::P2p(src_peer_id.into()).into(),
}),
ToListenerMsg::Reservation(Err(())) => Err(RelayError::Reservation),
};
return Poll::Ready(Some(result));
}
}
}
pub type RelayedDial = BoxFuture<'static, Result<RelayedConnection, RelayError>>;
/// Error that occurred during relay connection setup.
#[derive(Debug, Error)]
pub enum RelayError {
#[error("Missing relay peer id.")]
MissingRelayPeerId,
#[error("Missing relay address.")]
MissingRelayAddr,
#[error("Missing destination peer id.")]
MissingDstPeerId,
#[error("Invalid peer id hash.")]
InvalidHash,
#[error("Failed to send message to relay behaviour: {0:?}")]
SendingMessageToBehaviour(#[from] mpsc::SendError),
#[error("Response from behaviour was canceled")]
ResponseFromBehaviourCanceled(#[from] oneshot::Canceled),
#[error(
"Address contains multiple circuit relay protocols (`p2p-circuit`) which is not supported."
)]
MultipleCircuitRelayProtocolsUnsupported,
#[error("One of the provided multiaddresses is malformed.")]
MalformedMultiaddr,
#[error("Failed to get Reservation.")]
Reservation,
#[error("Failed to connect to destination.")]
Connect,
}
impl From<RelayError> for TransportError<RelayError> {
fn from(error: RelayError) -> Self {
TransportError::Other(error)
}
}
/// Message from the [`ClientTransport`] to the [`Relay`](crate::v2::relay::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<RelayedConnection, ()>>,
},
/// Listen for incoming relayed connections via relay node.
ListenReq {
relay_peer_id: PeerId,
relay_addr: Multiaddr,
to_listener: mpsc::Sender<ToListenerMsg>,
},
}
#[allow(clippy::large_enum_variant)]
pub enum ToListenerMsg {
Reservation(Result<Reservation, ()>),
IncomingRelayedConnection {
stream: RelayedConnection,
src_peer_id: PeerId,
relay_peer_id: PeerId,
relay_addr: Multiaddr,
},
}
pub struct Reservation {
pub(crate) addrs: Vec<Multiaddr>,
}

View File

@ -0,0 +1,291 @@
// Copyright 2020 Parity Technologies (UK) Ltd.
// Copyright 2021 Protocol Labs.
//
// 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::convert::TryInto;
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>,
max_circuit_duration: Delay,
max_circuit_bytes: u64,
bytes_sent: u64,
}
impl<S: AsyncRead, D: AsyncRead> CopyFuture<S, D> {
pub fn new(src: S, dst: D, max_circuit_duration: Duration, max_circuit_bytes: u64) -> Self {
CopyFuture {
src: BufReader::new(src),
dst: BufReader::new(dst),
max_circuit_duration: Delay::new(max_circuit_duration),
max_circuit_bytes,
bytes_sent: Default::default(),
}
}
}
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;
loop {
if this.bytes_sent > this.max_circuit_bytes {
return Poll::Ready(Err(io::Error::new(
io::ErrorKind::Other,
"Max circuit bytes reached.",
)));
}
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(0)) => Status::Done,
Poll::Ready(Ok(i)) => {
this.bytes_sent += i;
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(0)) => Status::Done,
Poll::Ready(Ok(i)) => {
this.bytes_sent += i;
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.
(Status::Progressed, _) | (_, Status::Progressed) => {}
// Both are pending. Check if max circuit duration 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 let Poll::Ready(()) = this.max_circuit_duration.poll_unpin(cx) {
return Poll::Ready(Err(io::ErrorKind::TimedOut.into()));
}
Poll::Pending
}
}
/// Forwards data from `source` to `destination`.
///
/// Returns `0` when done, i.e. `source` having reached EOF, returns number of bytes sent 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<u64>> {
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(0));
}
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(i.try_into().expect("usize to fit into u64.")))
}
#[cfg(test)]
mod tests {
use super::CopyFuture;
use futures::executor::block_on;
use futures::io::{AsyncRead, AsyncWrite};
use quickcheck::QuickCheck;
use std::io::ErrorKind;
use std::pin::Pin;
use std::task::{Context, Poll};
use std::time::Duration;
struct Connection {
read: Vec<u8>,
write: Vec<u8>,
}
impl AsyncWrite for Connection {
fn poll_write(
mut self: std::pin::Pin<&mut Self>,
cx: &mut Context<'_>,
buf: &[u8],
) -> Poll<std::io::Result<usize>> {
Pin::new(&mut self.write).poll_write(cx, buf)
}
fn poll_flush(
mut self: std::pin::Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<std::io::Result<()>> {
Pin::new(&mut self.write).poll_flush(cx)
}
fn poll_close(
mut self: std::pin::Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<std::io::Result<()>> {
Pin::new(&mut self.write).poll_close(cx)
}
}
impl AsyncRead for Connection {
fn poll_read(
mut self: Pin<&mut Self>,
_cx: &mut Context<'_>,
buf: &mut [u8],
) -> Poll<std::io::Result<usize>> {
let n = std::cmp::min(self.read.len(), buf.len());
buf[0..n].copy_from_slice(&self.read[0..n]);
self.read = self.read.split_off(n);
return Poll::Ready(Ok(n));
}
}
struct PendingConnection {}
impl AsyncWrite for PendingConnection {
fn poll_write(
self: std::pin::Pin<&mut Self>,
_cx: &mut Context<'_>,
_buf: &[u8],
) -> Poll<std::io::Result<usize>> {
Poll::Pending
}
fn poll_flush(
self: std::pin::Pin<&mut Self>,
_cx: &mut Context<'_>,
) -> Poll<std::io::Result<()>> {
Poll::Pending
}
fn poll_close(
self: std::pin::Pin<&mut Self>,
_cx: &mut Context<'_>,
) -> Poll<std::io::Result<()>> {
Poll::Pending
}
}
impl AsyncRead for PendingConnection {
fn poll_read(
self: Pin<&mut Self>,
_cx: &mut Context<'_>,
_buf: &mut [u8],
) -> Poll<std::io::Result<usize>> {
Poll::Pending
}
}
#[test]
fn quickcheck() {
fn prop(a: Vec<u8>, b: Vec<u8>, max_circuit_bytes: u64) {
let connection_a = Connection {
read: a.clone(),
write: Vec::new(),
};
let connection_b = Connection {
read: b.clone(),
write: Vec::new(),
};
let mut copy_future = CopyFuture::new(
connection_a,
connection_b,
Duration::from_secs(60),
max_circuit_bytes,
);
match block_on(&mut copy_future) {
Ok(()) => {
assert_eq!(copy_future.src.into_inner().write, b);
assert_eq!(copy_future.dst.into_inner().write, a);
}
Err(error) => {
assert_eq!(error.kind(), ErrorKind::Other);
assert_eq!(error.to_string(), "Max circuit bytes reached.");
assert!(a.len() + b.len() > max_circuit_bytes as usize);
}
}
}
QuickCheck::new().quickcheck(prop as fn(_, _, _))
}
#[test]
fn max_circuit_duration() {
let copy_future = CopyFuture::new(
PendingConnection {},
PendingConnection {},
Duration::from_millis(1),
u64::MAX,
);
std::thread::sleep(Duration::from_millis(2));
let error =
block_on(copy_future).expect_err("Expect maximum circuit duration to be reached.");
assert_eq!(error.kind(), ErrorKind::TimedOut);
}
}

View File

@ -0,0 +1,59 @@
syntax = "proto2";
package message_v2.pb;
message HopMessage {
enum Type {
RESERVE = 0;
CONNECT = 1;
STATUS = 2;
}
required Type type = 1;
optional Peer peer = 2;
optional Reservation reservation = 3;
optional Limit limit = 4;
optional Status status = 5;
}
message StopMessage {
enum Type {
CONNECT = 0;
STATUS = 1;
}
required Type type = 1;
optional Peer peer = 2;
optional Limit limit = 3;
optional Status status = 4;
}
message Peer {
required bytes id = 1;
repeated bytes addrs = 2;
}
message Reservation {
required uint64 expire = 1; // Unix expiration time (UTC)
repeated bytes addrs = 2; // relay addrs for reserving peer
optional bytes voucher = 3; // reservation voucher
}
message Limit {
optional uint32 duration = 1; // seconds
optional uint64 data = 2; // bytes
}
enum Status {
OK = 100;
RESERVATION_REFUSED = 200;
RESOURCE_LIMIT_EXCEEDED = 201;
PERMISSION_DENIED = 202;
CONNECTION_FAILED = 203;
NO_RESERVATION = 204;
MALFORMED_MESSAGE = 400;
UNEXPECTED_MESSAGE = 401;
}

View File

@ -0,0 +1,47 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::message_proto;
use std::time::Duration;
pub mod inbound_hop;
pub mod inbound_stop;
pub mod outbound_hop;
pub mod outbound_stop;
const HOP_PROTOCOL_NAME: &[u8; 31] = b"/libp2p/circuit/relay/0.2.0/hop";
const STOP_PROTOCOL_NAME: &[u8; 32] = b"/libp2p/circuit/relay/0.2.0/stop";
const MAX_MESSAGE_SIZE: usize = 4096;
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct Limit {
duration: Option<Duration>,
data_in_bytes: Option<u64>,
}
impl From<message_proto::Limit> for Limit {
fn from(limit: message_proto::Limit) -> Self {
Limit {
duration: limit.duration.map(|d| Duration::from_secs(d.into())),
data_in_bytes: limit.data,
}
}
}

View File

@ -0,0 +1,256 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::message_proto::{hop_message, HopMessage, Limit, Reservation, Status};
use crate::v2::protocol::{HOP_PROTOCOL_NAME, MAX_MESSAGE_SIZE};
use asynchronous_codec::{Framed, FramedParts};
use bytes::Bytes;
use futures::{future::BoxFuture, prelude::*};
use libp2p_core::{upgrade, Multiaddr, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::convert::TryInto;
use std::io::Cursor;
use std::iter;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use thiserror::Error;
use unsigned_varint::codec::UviBytes;
pub struct Upgrade {
pub reservation_duration: Duration,
pub max_circuit_duration: Duration,
pub max_circuit_bytes: u64,
}
impl upgrade::UpgradeInfo for Upgrade {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(HOP_PROTOCOL_NAME)
}
}
impl upgrade::InboundUpgrade<NegotiatedSubstream> for Upgrade {
type Output = Req;
type Error = UpgradeError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
fn upgrade_inbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
let mut codec = UviBytes::default();
codec.set_max_len(MAX_MESSAGE_SIZE);
let mut substream = Framed::new(substream, codec);
async move {
let msg: bytes::BytesMut = substream
.next()
.await
.ok_or_else(|| std::io::Error::new(std::io::ErrorKind::UnexpectedEof, ""))??;
let HopMessage {
r#type,
peer,
reservation: _,
limit: _,
status: _,
} = HopMessage::decode(Cursor::new(msg))?;
let r#type =
hop_message::Type::from_i32(r#type).ok_or(FatalUpgradeError::ParseTypeField)?;
let req = match r#type {
hop_message::Type::Reserve => Req::Reserve(ReservationReq {
substream,
reservation_duration: self.reservation_duration,
max_circuit_duration: self.max_circuit_duration,
max_circuit_bytes: self.max_circuit_bytes,
}),
hop_message::Type::Connect => {
let dst = PeerId::from_bytes(&peer.ok_or(FatalUpgradeError::MissingPeer)?.id)
.map_err(|_| FatalUpgradeError::ParsePeerId)?;
Req::Connect(CircuitReq { dst, substream })
}
hop_message::Type::Status => Err(FatalUpgradeError::UnexpectedTypeStatus)?,
};
Ok(req)
}
.boxed()
}
}
#[derive(Debug, Error)]
pub enum UpgradeError {
#[error("Fatal")]
Fatal(#[from] FatalUpgradeError),
}
impl From<prost::DecodeError> for UpgradeError {
fn from(error: prost::DecodeError) -> Self {
Self::Fatal(error.into())
}
}
impl From<std::io::Error> for UpgradeError {
fn from(error: std::io::Error) -> Self {
Self::Fatal(error.into())
}
}
#[derive(Debug, Error)]
pub enum FatalUpgradeError {
#[error("Failed to decode message: {0}.")]
Decode(
#[from]
#[source]
prost::DecodeError,
),
#[error(transparent)]
Io(#[from] std::io::Error),
#[error("Failed to parse response type field.")]
ParseTypeField,
#[error("Failed to parse peer id.")]
ParsePeerId,
#[error("Expected 'peer' field to be set.")]
MissingPeer,
#[error("Unexpected message type 'status'")]
UnexpectedTypeStatus,
}
pub enum Req {
Reserve(ReservationReq),
Connect(CircuitReq),
}
pub struct ReservationReq {
substream: Framed<NegotiatedSubstream, UviBytes<Cursor<Vec<u8>>>>,
reservation_duration: Duration,
max_circuit_duration: Duration,
max_circuit_bytes: u64,
}
impl ReservationReq {
pub async fn accept(self, addrs: Vec<Multiaddr>) -> Result<(), std::io::Error> {
let msg = HopMessage {
r#type: hop_message::Type::Status.into(),
peer: None,
reservation: Some(Reservation {
addrs: addrs.into_iter().map(|a| a.to_vec()).collect(),
expire: (SystemTime::now() + self.reservation_duration)
.duration_since(UNIX_EPOCH)
.unwrap()
.as_secs(),
voucher: None,
}),
limit: Some(Limit {
duration: Some(
self.max_circuit_duration
.as_secs()
.try_into()
.expect("`max_circuit_duration` not to exceed `u32::MAX`."),
),
data: Some(self.max_circuit_bytes),
}),
status: Some(Status::Ok.into()),
};
self.send(msg).await
}
pub async fn deny(self, status: Status) -> Result<(), std::io::Error> {
let msg = HopMessage {
r#type: hop_message::Type::Status.into(),
peer: None,
reservation: None,
limit: None,
status: Some(status.into()),
};
self.send(msg).await
}
async fn send(mut self, msg: HopMessage) -> Result<(), std::io::Error> {
let mut encoded_msg = Vec::with_capacity(msg.encoded_len());
msg.encode(&mut encoded_msg)
.expect("Vec to have sufficient capacity.");
self.substream.send(Cursor::new(encoded_msg)).await?;
self.substream.flush().await?;
self.substream.close().await?;
Ok(())
}
}
pub struct CircuitReq {
dst: PeerId,
substream: Framed<NegotiatedSubstream, UviBytes<Cursor<Vec<u8>>>>,
}
impl CircuitReq {
pub fn dst(&self) -> PeerId {
self.dst
}
pub async fn accept(mut self) -> Result<(NegotiatedSubstream, Bytes), std::io::Error> {
let msg = HopMessage {
r#type: hop_message::Type::Status.into(),
peer: None,
reservation: None,
limit: None,
status: Some(Status::Ok.into()),
};
self.send(msg).await?;
let FramedParts {
io,
read_buffer,
write_buffer,
..
} = self.substream.into_parts();
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have an empty write buffer."
);
Ok((io, read_buffer.freeze()))
}
pub async fn deny(mut self, status: Status) -> Result<(), std::io::Error> {
let msg = HopMessage {
r#type: hop_message::Type::Status.into(),
peer: None,
reservation: None,
limit: None,
status: Some(status.into()),
};
self.send(msg).await?;
self.substream.close().await
}
async fn send(&mut self, msg: HopMessage) -> Result<(), std::io::Error> {
let mut encoded_msg = Vec::with_capacity(msg.encoded_len());
msg.encode(&mut encoded_msg)
.expect("Vec to have sufficient capacity.");
self.substream.send(Cursor::new(encoded_msg)).await?;
self.substream.flush().await?;
Ok(())
}
}

View File

@ -0,0 +1,185 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::message_proto::{stop_message, Status, StopMessage};
use crate::v2::protocol::{self, MAX_MESSAGE_SIZE, STOP_PROTOCOL_NAME};
use asynchronous_codec::{Framed, FramedParts};
use bytes::Bytes;
use futures::{future::BoxFuture, prelude::*};
use libp2p_core::{upgrade, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::io::Cursor;
use std::iter;
use thiserror::Error;
use unsigned_varint::codec::UviBytes;
pub struct Upgrade {}
impl upgrade::UpgradeInfo for Upgrade {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(STOP_PROTOCOL_NAME)
}
}
impl upgrade::InboundUpgrade<NegotiatedSubstream> for Upgrade {
type Output = Circuit;
type Error = UpgradeError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
fn upgrade_inbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
let mut codec = UviBytes::default();
codec.set_max_len(MAX_MESSAGE_SIZE);
let mut substream = Framed::new(substream, codec);
async move {
let msg: bytes::BytesMut = substream
.next()
.await
.ok_or_else(|| std::io::Error::new(std::io::ErrorKind::UnexpectedEof, ""))??;
let StopMessage {
r#type,
peer,
limit,
status: _,
} = StopMessage::decode(Cursor::new(msg))?;
let r#type =
stop_message::Type::from_i32(r#type).ok_or(FatalUpgradeError::ParseTypeField)?;
match r#type {
stop_message::Type::Connect => {
let src_peer_id =
PeerId::from_bytes(&peer.ok_or(FatalUpgradeError::MissingPeer)?.id)
.map_err(|_| FatalUpgradeError::ParsePeerId)?;
Ok(Circuit {
substream,
src_peer_id,
limit: limit.map(Into::into),
})
}
stop_message::Type::Status => Err(FatalUpgradeError::UnexpectedTypeStatus)?,
}
}
.boxed()
}
}
#[derive(Debug, Error)]
pub enum UpgradeError {
#[error("Fatal")]
Fatal(#[from] FatalUpgradeError),
}
impl From<prost::DecodeError> for UpgradeError {
fn from(error: prost::DecodeError) -> Self {
Self::Fatal(error.into())
}
}
impl From<std::io::Error> for UpgradeError {
fn from(error: std::io::Error) -> Self {
Self::Fatal(error.into())
}
}
#[derive(Debug, Error)]
pub enum FatalUpgradeError {
#[error("Failed to decode message: {0}.")]
Decode(
#[from]
#[source]
prost::DecodeError,
),
#[error(transparent)]
Io(#[from] std::io::Error),
#[error("Failed to parse response type field.")]
ParseTypeField,
#[error("Failed to parse peer id.")]
ParsePeerId,
#[error("Expected 'peer' field to be set.")]
MissingPeer,
#[error("Unexpected message type 'status'")]
UnexpectedTypeStatus,
}
pub struct Circuit {
substream: Framed<NegotiatedSubstream, UviBytes<Cursor<Vec<u8>>>>,
src_peer_id: PeerId,
limit: Option<protocol::Limit>,
}
impl Circuit {
pub fn src_peer_id(&self) -> PeerId {
self.src_peer_id
}
pub fn limit(&self) -> Option<protocol::Limit> {
self.limit
}
pub async fn accept(mut self) -> Result<(NegotiatedSubstream, Bytes), std::io::Error> {
let msg = StopMessage {
r#type: stop_message::Type::Status.into(),
peer: None,
limit: None,
status: Some(Status::Ok.into()),
};
self.send(msg).await?;
let FramedParts {
io,
read_buffer,
write_buffer,
..
} = self.substream.into_parts();
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have an empty write buffer."
);
Ok((io, read_buffer.freeze()))
}
pub async fn deny(mut self, status: Status) -> Result<(), std::io::Error> {
let msg = StopMessage {
r#type: stop_message::Type::Status.into(),
peer: None,
limit: None,
status: Some(status.into()),
};
self.send(msg).await
}
async fn send(&mut self, msg: StopMessage) -> Result<(), std::io::Error> {
let mut encoded_msg = Vec::with_capacity(msg.encoded_len());
msg.encode(&mut encoded_msg)
.expect("Vec to have sufficient capacity.");
self.substream.send(Cursor::new(encoded_msg)).await?;
self.substream.flush().await?;
Ok(())
}
}

View File

@ -0,0 +1,282 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::message_proto::{hop_message, HopMessage, Peer, Status};
use crate::v2::protocol::{Limit, HOP_PROTOCOL_NAME, MAX_MESSAGE_SIZE};
use asynchronous_codec::{Framed, FramedParts};
use bytes::Bytes;
use futures::{future::BoxFuture, prelude::*};
use futures_timer::Delay;
use libp2p_core::{upgrade, Multiaddr, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::convert::TryFrom;
use std::io::Cursor;
use std::iter;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use thiserror::Error;
use unsigned_varint::codec::UviBytes;
pub enum Upgrade {
Reserve,
Connect { dst_peer_id: PeerId },
}
impl upgrade::UpgradeInfo for Upgrade {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(HOP_PROTOCOL_NAME)
}
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for Upgrade {
type Output = Output;
type Error = UpgradeError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
fn upgrade_outbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
let msg = match self {
Upgrade::Reserve => HopMessage {
r#type: hop_message::Type::Reserve.into(),
peer: None,
reservation: None,
limit: None,
status: None,
},
Upgrade::Connect { dst_peer_id } => HopMessage {
r#type: hop_message::Type::Connect.into(),
peer: Some(Peer {
id: dst_peer_id.to_bytes(),
addrs: vec![],
}),
reservation: None,
limit: None,
status: None,
},
};
let mut encoded_msg = Vec::with_capacity(msg.encoded_len());
msg.encode(&mut encoded_msg)
.expect("Vec to have sufficient capacity.");
let mut codec = UviBytes::default();
codec.set_max_len(MAX_MESSAGE_SIZE);
let mut substream = Framed::new(substream, codec);
async move {
substream.send(Cursor::new(encoded_msg)).await?;
let msg: bytes::BytesMut = substream
.next()
.await
.ok_or_else(|| std::io::Error::new(std::io::ErrorKind::UnexpectedEof, ""))??;
let HopMessage {
r#type,
peer: _,
reservation,
limit,
status,
} = HopMessage::decode(Cursor::new(msg))?;
let r#type =
hop_message::Type::from_i32(r#type).ok_or(FatalUpgradeError::ParseTypeField)?;
match r#type {
hop_message::Type::Connect => Err(FatalUpgradeError::UnexpectedTypeConnect)?,
hop_message::Type::Reserve => Err(FatalUpgradeError::UnexpectedTypeReserve)?,
hop_message::Type::Status => {}
}
let status = Status::from_i32(status.ok_or(FatalUpgradeError::MissingStatusField)?)
.ok_or(FatalUpgradeError::ParseStatusField)?;
let limit = limit.map(Into::into);
let output = match self {
Upgrade::Reserve => {
match status {
Status::Ok => {}
Status::ReservationRefused => Err(ReservationFailedReason::Refused)?,
Status::ResourceLimitExceeded => {
Err(ReservationFailedReason::ResourceLimitExceeded)?
}
s => Err(FatalUpgradeError::UnexpectedStatus(s))?,
}
let reservation =
reservation.ok_or(FatalUpgradeError::MissingReservationField)?;
if reservation.addrs.is_empty() {
Err(FatalUpgradeError::NoAddressesInReservation)?;
}
let addrs = reservation
.addrs
.into_iter()
.map(TryFrom::try_from)
.collect::<Result<Vec<Multiaddr>, _>>()
.map_err(|_| FatalUpgradeError::InvalidReservationAddrs)?;
let renewal_timeout = reservation
.expire
.checked_sub(
SystemTime::now()
.duration_since(UNIX_EPOCH)
.unwrap()
.as_secs(),
)
// Renew the reservation after 3/4 of the reservation expiration timestamp.
.and_then(|duration| duration.checked_sub(duration / 4))
.map(Duration::from_secs)
.map(Delay::new)
.ok_or(FatalUpgradeError::InvalidReservationExpiration)?;
substream.close().await?;
Output::Reservation {
renewal_timeout,
addrs,
limit,
}
}
Upgrade::Connect { .. } => {
match status {
Status::Ok => {}
Status::ResourceLimitExceeded => {
Err(CircuitFailedReason::ResourceLimitExceeded)?
}
Status::ConnectionFailed => Err(CircuitFailedReason::ConnectionFailed)?,
Status::NoReservation => Err(CircuitFailedReason::NoReservation)?,
Status::PermissionDenied => Err(CircuitFailedReason::PermissionDenied)?,
s => Err(FatalUpgradeError::UnexpectedStatus(s))?,
}
let FramedParts {
io,
read_buffer,
write_buffer,
..
} = substream.into_parts();
assert!(
write_buffer.is_empty(),
"Expect a flushed Framed to have empty write buffer."
);
Output::Circuit {
substream: io,
read_buffer: read_buffer.freeze(),
limit,
}
}
};
Ok(output)
}
.boxed()
}
}
#[derive(Debug, Error)]
pub enum UpgradeError {
#[error("Reservation failed")]
ReservationFailed(#[from] ReservationFailedReason),
#[error("Circuit failed")]
CircuitFailed(#[from] CircuitFailedReason),
#[error("Fatal")]
Fatal(#[from] FatalUpgradeError),
}
impl From<std::io::Error> for UpgradeError {
fn from(error: std::io::Error) -> Self {
Self::Fatal(error.into())
}
}
impl From<prost::DecodeError> for UpgradeError {
fn from(error: prost::DecodeError) -> Self {
Self::Fatal(error.into())
}
}
#[derive(Debug, Error)]
pub enum CircuitFailedReason {
#[error("Remote reported resource limit exceeded.")]
ResourceLimitExceeded,
#[error("Relay failed to connect to destination.")]
ConnectionFailed,
#[error("Relay has no reservation for destination.")]
NoReservation,
#[error("Remote denied permission.")]
PermissionDenied,
}
#[derive(Debug, Error)]
pub enum ReservationFailedReason {
#[error("Reservation refused.")]
Refused,
#[error("Remote reported resource limit exceeded.")]
ResourceLimitExceeded,
}
#[derive(Debug, Error)]
pub enum FatalUpgradeError {
#[error("Failed to decode message: {0}.")]
Decode(
#[from]
#[source]
prost::DecodeError,
),
#[error(transparent)]
Io(#[from] std::io::Error),
#[error("Expected 'status' field to be set.")]
MissingStatusField,
#[error("Expected 'reservation' field to be set.")]
MissingReservationField,
#[error("Expected at least one address in reservation.")]
NoAddressesInReservation,
#[error("Invalid expiration timestamp in reservation.")]
InvalidReservationExpiration,
#[error("Invalid addresses in reservation.")]
InvalidReservationAddrs,
#[error("Failed to parse response type field.")]
ParseTypeField,
#[error("Unexpected message type 'connect'")]
UnexpectedTypeConnect,
#[error("Unexpected message type 'reserve'")]
UnexpectedTypeReserve,
#[error("Failed to parse response type field.")]
ParseStatusField,
#[error("Unexpected message status '{0:?}'")]
UnexpectedStatus(Status),
}
pub enum Output {
Reservation {
renewal_timeout: Delay,
addrs: Vec<Multiaddr>,
limit: Option<Limit>,
},
Circuit {
substream: NegotiatedSubstream,
read_buffer: Bytes,
limit: Option<Limit>,
},
}

View File

@ -0,0 +1,178 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::message_proto::{stop_message, Limit, Peer, Status, StopMessage};
use crate::v2::protocol::{MAX_MESSAGE_SIZE, STOP_PROTOCOL_NAME};
use asynchronous_codec::{Framed, FramedParts};
use bytes::Bytes;
use futures::{future::BoxFuture, prelude::*};
use libp2p_core::{upgrade, PeerId};
use libp2p_swarm::NegotiatedSubstream;
use prost::Message;
use std::convert::TryInto;
use std::io::Cursor;
use std::iter;
use std::time::Duration;
use thiserror::Error;
use unsigned_varint::codec::UviBytes;
pub struct Upgrade {
pub relay_peer_id: PeerId,
pub max_circuit_duration: Duration,
pub max_circuit_bytes: u64,
}
impl upgrade::UpgradeInfo for Upgrade {
type Info = &'static [u8];
type InfoIter = iter::Once<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(STOP_PROTOCOL_NAME)
}
}
impl upgrade::OutboundUpgrade<NegotiatedSubstream> for Upgrade {
type Output = (NegotiatedSubstream, Bytes);
type Error = UpgradeError;
type Future = BoxFuture<'static, Result<Self::Output, Self::Error>>;
fn upgrade_outbound(self, substream: NegotiatedSubstream, _: Self::Info) -> Self::Future {
let msg = StopMessage {
r#type: stop_message::Type::Connect.into(),
peer: Some(Peer {
id: self.relay_peer_id.to_bytes(),
addrs: vec![],
}),
limit: Some(Limit {
duration: Some(
self.max_circuit_duration
.as_secs()
.try_into()
.expect("`max_circuit_duration` not to exceed `u32::MAX`."),
),
data: Some(self.max_circuit_bytes),
}),
status: None,
};
let mut encoded_msg = Vec::with_capacity(msg.encoded_len());
msg.encode(&mut encoded_msg)
.expect("Vec to have sufficient capacity.");
let mut codec = UviBytes::default();
codec.set_max_len(MAX_MESSAGE_SIZE);
let mut substream = Framed::new(substream, codec);
async move {
substream.send(std::io::Cursor::new(encoded_msg)).await?;
let msg: bytes::BytesMut = substream
.next()
.await
.ok_or_else(|| std::io::Error::new(std::io::ErrorKind::UnexpectedEof, ""))??;
let StopMessage {
r#type,
peer: _,
limit: _,
status,
} = StopMessage::decode(Cursor::new(msg))?;
let r#type =
stop_message::Type::from_i32(r#type).ok_or(FatalUpgradeError::ParseTypeField)?;
match r#type {
stop_message::Type::Connect => Err(FatalUpgradeError::UnexpectedTypeConnect)?,
stop_message::Type::Status => {}
}
let status = Status::from_i32(status.ok_or(FatalUpgradeError::MissingStatusField)?)
.ok_or(FatalUpgradeError::ParseStatusField)?;
match status {
Status::Ok => {}
Status::ResourceLimitExceeded => Err(CircuitFailedReason::ResourceLimitExceeded)?,
Status::PermissionDenied => Err(CircuitFailedReason::PermissionDenied)?,
s => Err(FatalUpgradeError::UnexpectedStatus(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, Error)]
pub enum UpgradeError {
#[error("Circuit failed")]
CircuitFailed(#[from] CircuitFailedReason),
#[error("Fatal")]
Fatal(#[from] FatalUpgradeError),
}
impl From<std::io::Error> for UpgradeError {
fn from(error: std::io::Error) -> Self {
Self::Fatal(error.into())
}
}
impl From<prost::DecodeError> for UpgradeError {
fn from(error: prost::DecodeError) -> Self {
Self::Fatal(error.into())
}
}
#[derive(Debug, Error)]
pub enum CircuitFailedReason {
#[error("Remote reported resource limit exceeded.")]
ResourceLimitExceeded,
#[error("Remote reported permission denied.")]
PermissionDenied,
}
#[derive(Debug, Error)]
pub enum FatalUpgradeError {
#[error("Failed to decode message: {0}.")]
Decode(
#[from]
#[source]
prost::DecodeError,
),
#[error(transparent)]
Io(#[from] std::io::Error),
#[error("Expected 'status' field to be set.")]
MissingStatusField,
#[error("Failed to parse response type field.")]
ParseTypeField,
#[error("Unexpected message type 'connect'")]
UnexpectedTypeConnect,
#[error("Failed to parse response type field.")]
ParseStatusField,
#[error("Unexpected message status '{0:?}'")]
UnexpectedStatus(Status),
}

View File

@ -0,0 +1,731 @@
// Copyright 2021 Protocol Labs.
//
// 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.
//! [`NetworkBehaviour`] to act as a circuit relay v2 **relay**.
mod handler;
pub mod rate_limiter;
use crate::v2::message_proto;
use crate::v2::protocol::inbound_hop;
use either::Either;
use instant::Instant;
use libp2p_core::connection::{ConnectedPoint, ConnectionId};
use libp2p_core::multiaddr::Protocol;
use libp2p_core::PeerId;
use libp2p_swarm::protocols_handler::DummyProtocolsHandler;
use libp2p_swarm::{
NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters,
ProtocolsHandlerUpgrErr,
};
use std::collections::{HashMap, HashSet, VecDeque};
use std::num::NonZeroU32;
use std::ops::Add;
use std::task::{Context, Poll};
use std::time::Duration;
use super::protocol::outbound_stop;
/// Configuration for the [`Relay`] [`NetworkBehaviour`].
///
/// # Panics
///
/// [`Config::max_circuit_duration`] may not exceed [`u32::MAX`].
pub struct Config {
pub max_reservations: usize,
pub max_reservations_per_peer: usize,
pub reservation_duration: Duration,
pub reservation_rate_limiters: Vec<Box<dyn rate_limiter::RateLimiter>>,
pub max_circuits: usize,
pub max_circuits_per_peer: usize,
pub max_circuit_duration: Duration,
pub max_circuit_bytes: u64,
pub circuit_src_rate_limiters: Vec<Box<dyn rate_limiter::RateLimiter>>,
}
impl Default for Config {
fn default() -> Self {
let reservation_rate_limiters = vec![
// For each peer ID one reservation every 2 minutes with up to 30 reservations per hour.
rate_limiter::new_per_peer(rate_limiter::GenericRateLimiterConfig {
limit: NonZeroU32::new(30).expect("30 > 0"),
interval: Duration::from_secs(60 * 2),
}),
// For each IP address one reservation every minute with up to 60 reservations per hour.
rate_limiter::new_per_ip(rate_limiter::GenericRateLimiterConfig {
limit: NonZeroU32::new(60).expect("60 > 0"),
interval: Duration::from_secs(60),
}),
];
let circuit_src_rate_limiters = vec![
// For each source peer ID one circuit every 2 minute with up to 30 circuits per hour.
rate_limiter::new_per_peer(rate_limiter::GenericRateLimiterConfig {
limit: NonZeroU32::new(30).expect("30 > 0"),
interval: Duration::from_secs(60 * 2),
}),
// For each source IP address one circuit every minute with up to 60 circuits per hour.
rate_limiter::new_per_ip(rate_limiter::GenericRateLimiterConfig {
limit: NonZeroU32::new(60).expect("60 > 0"),
interval: Duration::from_secs(60),
}),
];
Config {
max_reservations: 128,
max_reservations_per_peer: 4,
reservation_duration: Duration::from_secs(60 * 60),
reservation_rate_limiters,
max_circuits: 16,
max_circuits_per_peer: 4,
max_circuit_duration: Duration::from_secs(2 * 60),
max_circuit_bytes: 1 << 17, // 128 kibibyte
circuit_src_rate_limiters,
}
}
}
/// The events produced by the [`Relay`] behaviour.
#[derive(Debug)]
pub enum Event {
/// An inbound reservation request has been accepted.
ReservationReqAccepted {
src_peer_id: PeerId,
/// Indicates whether the request replaces an existing reservation.
renewed: bool,
},
/// Accepting an inbound reservation request failed.
ReservationReqAcceptFailed {
src_peer_id: PeerId,
error: std::io::Error,
},
/// An inbound reservation request has been denied.
ReservationReqDenied { src_peer_id: PeerId },
/// Denying an inbound reservation request has failed.
ReservationReqDenyFailed {
src_peer_id: PeerId,
error: std::io::Error,
},
/// An inbound reservation has timed out.
ReservationTimedOut { src_peer_id: PeerId },
CircuitReqReceiveFailed {
src_peer_id: PeerId,
error: ProtocolsHandlerUpgrErr<void::Void>,
},
/// An inbound circuit request has been denied.
CircuitReqDenied {
src_peer_id: PeerId,
dst_peer_id: PeerId,
},
/// Denying an inbound circuit request failed.
CircuitReqDenyFailed {
src_peer_id: PeerId,
dst_peer_id: PeerId,
error: std::io::Error,
},
/// An inbound cirucit request has been accepted.
CircuitReqAccepted {
src_peer_id: PeerId,
dst_peer_id: PeerId,
},
/// An outbound connect for an inbound cirucit request failed.
CircuitReqOutboundConnectFailed {
src_peer_id: PeerId,
dst_peer_id: PeerId,
error: ProtocolsHandlerUpgrErr<outbound_stop::CircuitFailedReason>,
},
/// Accepting an inbound circuit request failed.
CircuitReqAcceptFailed {
src_peer_id: PeerId,
dst_peer_id: PeerId,
error: std::io::Error,
},
/// An inbound circuit has closed.
CircuitClosed {
src_peer_id: PeerId,
dst_peer_id: PeerId,
error: Option<std::io::Error>,
},
}
/// [`Relay`] is a [`NetworkBehaviour`] that implements the relay server
/// functionality of the circuit relay v2 protocol.
pub struct Relay {
config: Config,
local_peer_id: PeerId,
reservations: HashMap<PeerId, HashSet<ConnectionId>>,
circuits: CircuitsTracker,
/// Queue of actions to return when polled.
queued_actions: VecDeque<Action>,
}
impl Relay {
pub fn new(local_peer_id: PeerId, config: Config) -> Self {
Self {
config,
local_peer_id,
reservations: Default::default(),
circuits: Default::default(),
queued_actions: Default::default(),
}
}
}
impl NetworkBehaviour for Relay {
type ProtocolsHandler = handler::Prototype;
type OutEvent = Event;
fn new_handler(&mut self) -> Self::ProtocolsHandler {
handler::Prototype {
config: handler::Config {
reservation_duration: self.config.reservation_duration,
max_circuit_duration: self.config.max_circuit_duration,
max_circuit_bytes: self.config.max_circuit_bytes,
},
}
}
fn inject_connection_closed(
&mut self,
peer: &PeerId,
connection: &ConnectionId,
_: &ConnectedPoint,
_handler: Either<handler::Handler, DummyProtocolsHandler>,
) {
if let Some(connections) = self.reservations.get_mut(peer) {
connections.remove(&connection);
}
for circuit in self
.circuits
.remove_by_connection(*peer, *connection)
.iter()
// Only emit [`CircuitClosed`] for accepted requests.
.filter(|c| matches!(c.status, CircuitStatus::Accepted))
{
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitClosed {
src_peer_id: circuit.src_peer_id,
dst_peer_id: circuit.dst_peer_id,
error: Some(std::io::ErrorKind::ConnectionAborted.into()),
})
.into(),
);
}
}
fn inject_event(
&mut self,
event_source: PeerId,
connection: ConnectionId,
event: Either<handler::Event, void::Void>,
) {
let event = match event {
Either::Left(e) => e,
Either::Right(v) => void::unreachable(v),
};
match event {
handler::Event::ReservationReqReceived {
inbound_reservation_req,
endpoint,
renewed,
} => {
let now = Instant::now();
assert!(
!endpoint.is_relayed(),
"`DummyProtocolsHandler` handles relayed connections. It \
denies all inbound substreams."
);
let action = if
// Deny if it is a new reservation and exceeds `max_reservations_per_peer`.
(!renewed
&& self
.reservations
.get(&event_source)
.map(|cs| cs.len())
.unwrap_or(0)
> self.config.max_reservations_per_peer)
// Deny if it exceeds `max_reservations`.
|| self
.reservations
.iter()
.map(|(_, cs)| cs.len())
.sum::<usize>()
>= self.config.max_reservations
// Deny if it exceeds the allowed rate of reservations.
|| !self
.config
.reservation_rate_limiters
.iter_mut()
.all(|limiter| {
limiter.try_next(event_source, endpoint.get_remote_address(), now)
}) {
NetworkBehaviourAction::NotifyHandler {
handler: NotifyHandler::One(connection),
peer_id: event_source,
event: Either::Left(handler::In::DenyReservationReq {
inbound_reservation_req,
status: message_proto::Status::ResourceLimitExceeded,
}),
}
.into()
} else {
// Accept reservation.
self.reservations
.entry(event_source)
.or_default()
.insert(connection);
Action::AcceptReservationPrototype {
handler: NotifyHandler::One(connection),
peer_id: event_source,
inbound_reservation_req,
}
};
self.queued_actions.push_back(action);
}
handler::Event::ReservationReqAccepted { renewed } => {
// Ensure local eventual consistent reservation state matches handler (source of
// truth).
self.reservations
.entry(event_source)
.or_default()
.insert(connection);
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::ReservationReqAccepted {
src_peer_id: event_source,
renewed,
})
.into(),
);
}
handler::Event::ReservationReqAcceptFailed { error } => {
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::ReservationReqAcceptFailed {
src_peer_id: event_source,
error,
})
.into(),
);
}
handler::Event::ReservationReqDenied {} => {
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::ReservationReqDenied {
src_peer_id: event_source,
})
.into(),
);
}
handler::Event::ReservationReqDenyFailed { error } => {
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::ReservationReqDenyFailed {
src_peer_id: event_source,
error,
})
.into(),
);
}
handler::Event::ReservationTimedOut {} => {
self.reservations
.get_mut(&event_source)
.map(|cs| cs.remove(&connection));
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::ReservationTimedOut {
src_peer_id: event_source,
})
.into(),
);
}
handler::Event::CircuitReqReceived {
inbound_circuit_req,
endpoint,
} => {
let now = Instant::now();
assert!(
!endpoint.is_relayed(),
"`DummyProtocolsHandler` handles relayed connections. It \
denies all inbound substreams."
);
let action = if self.circuits.num_circuits_of_peer(event_source)
> self.config.max_circuits_per_peer
|| self.circuits.len() >= self.config.max_circuits
|| !self
.config
.circuit_src_rate_limiters
.iter_mut()
.all(|limiter| {
limiter.try_next(event_source, endpoint.get_remote_address(), now)
}) {
// Deny circuit exceeding limits.
NetworkBehaviourAction::NotifyHandler {
handler: NotifyHandler::One(connection),
peer_id: event_source,
event: Either::Left(handler::In::DenyCircuitReq {
circuit_id: None,
inbound_circuit_req,
status: message_proto::Status::ResourceLimitExceeded,
}),
}
} else if let Some(dst_conn) = self
.reservations
.get(&inbound_circuit_req.dst())
.map(|cs| cs.iter().next())
.flatten()
{
// Accept circuit request if reservation present.
let circuit_id = self.circuits.insert(Circuit {
status: CircuitStatus::Accepting,
src_peer_id: event_source,
src_connection_id: connection,
dst_peer_id: inbound_circuit_req.dst(),
dst_connection_id: *dst_conn,
});
NetworkBehaviourAction::NotifyHandler {
handler: NotifyHandler::One(*dst_conn),
peer_id: event_source,
event: Either::Left(handler::In::NegotiateOutboundConnect {
circuit_id,
inbound_circuit_req,
relay_peer_id: self.local_peer_id,
src_peer_id: event_source,
src_connection_id: connection,
}),
}
} else {
// Deny circuit request if no reservation present.
NetworkBehaviourAction::NotifyHandler {
handler: NotifyHandler::One(connection),
peer_id: event_source,
event: Either::Left(handler::In::DenyCircuitReq {
circuit_id: None,
inbound_circuit_req,
status: message_proto::Status::NoReservation,
}),
}
};
self.queued_actions.push_back(action.into());
}
handler::Event::CircuitReqReceiveFailed { error } => {
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitReqReceiveFailed {
src_peer_id: event_source,
error,
})
.into(),
);
}
handler::Event::CircuitReqDenied {
circuit_id,
dst_peer_id,
} => {
if let Some(circuit_id) = circuit_id {
self.circuits.remove(circuit_id);
}
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitReqDenied {
src_peer_id: event_source,
dst_peer_id,
})
.into(),
);
}
handler::Event::CircuitReqDenyFailed {
circuit_id,
dst_peer_id,
error,
} => {
if let Some(circuit_id) = circuit_id {
self.circuits.remove(circuit_id);
}
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitReqDenyFailed {
src_peer_id: event_source,
dst_peer_id,
error,
})
.into(),
);
}
handler::Event::OutboundConnectNegotiated {
circuit_id,
src_peer_id,
src_connection_id,
inbound_circuit_req,
dst_handler_notifier,
dst_stream,
dst_pending_data,
} => {
self.queued_actions.push_back(
NetworkBehaviourAction::NotifyHandler {
handler: NotifyHandler::One(src_connection_id),
peer_id: src_peer_id,
event: Either::Left(handler::In::AcceptAndDriveCircuit {
circuit_id,
dst_peer_id: event_source,
inbound_circuit_req,
dst_handler_notifier,
dst_stream,
dst_pending_data,
}),
}
.into(),
);
}
handler::Event::OutboundConnectNegotiationFailed {
circuit_id,
src_peer_id,
src_connection_id,
inbound_circuit_req,
status,
error,
} => {
self.queued_actions.push_back(
NetworkBehaviourAction::NotifyHandler {
handler: NotifyHandler::One(src_connection_id),
peer_id: src_peer_id,
event: Either::Left(handler::In::DenyCircuitReq {
circuit_id: Some(circuit_id),
inbound_circuit_req,
status,
}),
}
.into(),
);
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitReqOutboundConnectFailed {
src_peer_id,
dst_peer_id: event_source,
error,
})
.into(),
);
}
handler::Event::CircuitReqAccepted {
dst_peer_id,
circuit_id,
} => {
self.circuits.accepted(circuit_id);
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitReqAccepted {
src_peer_id: event_source,
dst_peer_id,
})
.into(),
);
}
handler::Event::CircuitReqAcceptFailed {
dst_peer_id,
circuit_id,
error,
} => {
self.circuits.remove(circuit_id);
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitReqAcceptFailed {
src_peer_id: event_source,
dst_peer_id,
error,
})
.into(),
);
}
handler::Event::CircuitClosed {
dst_peer_id,
circuit_id,
error,
} => {
self.circuits.remove(circuit_id);
self.queued_actions.push_back(
NetworkBehaviourAction::GenerateEvent(Event::CircuitClosed {
src_peer_id: event_source,
dst_peer_id,
error,
})
.into(),
);
}
}
}
fn poll(
&mut self,
_cx: &mut Context<'_>,
poll_parameters: &mut impl PollParameters,
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
if let Some(action) = self.queued_actions.pop_front() {
return Poll::Ready(action.build(poll_parameters));
}
Poll::Pending
}
}
#[derive(Default)]
struct CircuitsTracker {
next_id: CircuitId,
circuits: HashMap<CircuitId, Circuit>,
}
impl CircuitsTracker {
fn len(&self) -> usize {
self.circuits.len()
}
fn insert(&mut self, circuit: Circuit) -> CircuitId {
let id = self.next_id;
self.next_id = self.next_id + 1;
self.circuits.insert(id, circuit);
id
}
fn accepted(&mut self, circuit_id: CircuitId) {
if let Some(c) = self.circuits.get_mut(&circuit_id) {
c.status = CircuitStatus::Accepted;
};
}
fn remove(&mut self, circuit_id: CircuitId) -> Option<Circuit> {
self.circuits.remove(&circuit_id)
}
fn remove_by_connection(
&mut self,
peer_id: PeerId,
connection_id: ConnectionId,
) -> Vec<Circuit> {
let mut removed = vec![];
self.circuits.retain(|_circuit_id, circuit| {
let is_src =
circuit.src_peer_id == peer_id && circuit.src_connection_id == connection_id;
let is_dst =
circuit.dst_peer_id == peer_id && circuit.dst_connection_id == connection_id;
if is_src || is_dst {
removed.push(circuit.clone());
// Remove circuit from HashMap.
false
} else {
// Retain circuit in HashMap.
true
}
});
removed
}
fn num_circuits_of_peer(&self, peer: PeerId) -> usize {
self.circuits
.iter()
.filter(|(_, c)| c.src_peer_id == peer || c.dst_peer_id == peer)
.count()
}
}
#[derive(Clone)]
struct Circuit {
src_peer_id: PeerId,
src_connection_id: ConnectionId,
dst_peer_id: PeerId,
dst_connection_id: ConnectionId,
status: CircuitStatus,
}
#[derive(Clone)]
enum CircuitStatus {
Accepting,
Accepted,
}
#[derive(Default, Clone, Copy, Debug, Hash, Eq, PartialEq)]
pub struct CircuitId(u64);
impl Add<u64> for CircuitId {
type Output = CircuitId;
fn add(self, rhs: u64) -> Self {
CircuitId(self.0 + rhs)
}
}
/// A [`NetworkBehaviourAction`], either complete, or still requiring data from [`PollParameters`]
/// before being returned in [`Relay::poll`].
#[allow(clippy::large_enum_variant)]
enum Action {
Done(NetworkBehaviourAction<Event, handler::Prototype>),
AcceptReservationPrototype {
inbound_reservation_req: inbound_hop::ReservationReq,
handler: NotifyHandler,
peer_id: PeerId,
},
}
impl From<NetworkBehaviourAction<Event, handler::Prototype>> for Action {
fn from(action: NetworkBehaviourAction<Event, handler::Prototype>) -> Self {
Self::Done(action)
}
}
impl Action {
fn build(
self,
poll_parameters: &mut impl PollParameters,
) -> NetworkBehaviourAction<Event, handler::Prototype> {
match self {
Action::Done(action) => action,
Action::AcceptReservationPrototype {
inbound_reservation_req,
handler,
peer_id,
} => NetworkBehaviourAction::NotifyHandler {
handler,
peer_id,
event: Either::Left(handler::In::AcceptReservationReq {
inbound_reservation_req,
addrs: poll_parameters
.external_addresses()
.map(|a| {
a.addr
.with(Protocol::P2p((*poll_parameters.local_peer_id()).into()))
})
.collect(),
}),
},
}
}
}

View File

@ -0,0 +1,911 @@
// Copyright 2021 Protocol Labs.
//
// 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::v2::copy_future::CopyFuture;
use crate::v2::message_proto::Status;
use crate::v2::protocol::{inbound_hop, outbound_stop};
use crate::v2::relay::CircuitId;
use bytes::Bytes;
use either::Either;
use futures::channel::oneshot::{self, Canceled};
use futures::future::{BoxFuture, FutureExt, TryFutureExt};
use futures::io::AsyncWriteExt;
use futures::stream::{FuturesUnordered, StreamExt};
use futures_timer::Delay;
use instant::Instant;
use libp2p_core::connection::ConnectionId;
use libp2p_core::either::EitherError;
use libp2p_core::{upgrade, ConnectedPoint, Multiaddr, PeerId};
use libp2p_swarm::protocols_handler::{DummyProtocolsHandler, SendWrapper};
use libp2p_swarm::protocols_handler::{InboundUpgradeSend, OutboundUpgradeSend};
use libp2p_swarm::{
IntoProtocolsHandler, KeepAlive, NegotiatedSubstream, ProtocolsHandler, ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr, SubstreamProtocol,
};
use std::collections::VecDeque;
use std::fmt;
use std::task::{Context, Poll};
use std::time::Duration;
pub struct Config {
pub reservation_duration: Duration,
pub max_circuit_duration: Duration,
pub max_circuit_bytes: u64,
}
pub enum In {
AcceptReservationReq {
inbound_reservation_req: inbound_hop::ReservationReq,
addrs: Vec<Multiaddr>,
},
DenyReservationReq {
inbound_reservation_req: inbound_hop::ReservationReq,
status: Status,
},
DenyCircuitReq {
circuit_id: Option<CircuitId>,
inbound_circuit_req: inbound_hop::CircuitReq,
status: Status,
},
NegotiateOutboundConnect {
circuit_id: CircuitId,
inbound_circuit_req: inbound_hop::CircuitReq,
relay_peer_id: PeerId,
src_peer_id: PeerId,
src_connection_id: ConnectionId,
},
AcceptAndDriveCircuit {
circuit_id: CircuitId,
dst_peer_id: PeerId,
inbound_circuit_req: inbound_hop::CircuitReq,
dst_handler_notifier: oneshot::Sender<()>,
dst_stream: NegotiatedSubstream,
dst_pending_data: Bytes,
},
}
impl fmt::Debug for In {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
In::AcceptReservationReq {
inbound_reservation_req: _,
addrs,
} => f
.debug_struct("In::AcceptReservationReq")
.field("addrs", addrs)
.finish(),
In::DenyReservationReq {
inbound_reservation_req: _,
status,
} => f
.debug_struct("In::DenyReservationReq")
.field("status", status)
.finish(),
In::DenyCircuitReq {
circuit_id,
inbound_circuit_req: _,
status,
} => f
.debug_struct("In::DenyCircuitReq")
.field("circuit_id", circuit_id)
.field("status", status)
.finish(),
In::NegotiateOutboundConnect {
circuit_id,
inbound_circuit_req: _,
relay_peer_id,
src_peer_id,
src_connection_id,
} => f
.debug_struct("In::NegotiateOutboundConnect")
.field("circuit_id", circuit_id)
.field("relay_peer_id", relay_peer_id)
.field("src_peer_id", src_peer_id)
.field("src_connection_id", src_connection_id)
.finish(),
In::AcceptAndDriveCircuit {
circuit_id,
inbound_circuit_req: _,
dst_peer_id,
dst_handler_notifier: _,
dst_stream: _,
dst_pending_data: _,
} => f
.debug_struct("In::AcceptAndDriveCircuit")
.field("circuit_id", circuit_id)
.field("dst_peer_id", dst_peer_id)
.finish(),
}
}
}
/// The events produced by the [`Handler`].
#[allow(clippy::large_enum_variant)]
pub enum Event {
/// An inbound reservation request has been received.
ReservationReqReceived {
inbound_reservation_req: inbound_hop::ReservationReq,
endpoint: ConnectedPoint,
/// Indicates whether the request replaces an existing reservation.
renewed: bool,
},
/// An inbound reservation request has been accepted.
ReservationReqAccepted {
/// Indicates whether the request replaces an existing reservation.
renewed: bool,
},
/// Accepting an inbound reservation request failed.
ReservationReqAcceptFailed { error: std::io::Error },
/// An inbound reservation request has been denied.
ReservationReqDenied {},
/// Denying an inbound reservation request has failed.
ReservationReqDenyFailed { error: std::io::Error },
/// An inbound reservation has timed out.
ReservationTimedOut {},
/// An inbound circuit request has been received.
CircuitReqReceived {
inbound_circuit_req: inbound_hop::CircuitReq,
endpoint: ConnectedPoint,
},
/// Receiving an inbound circuit request failed.
CircuitReqReceiveFailed {
error: ProtocolsHandlerUpgrErr<void::Void>,
},
/// An inbound circuit request has been denied.
CircuitReqDenied {
circuit_id: Option<CircuitId>,
dst_peer_id: PeerId,
},
/// Denying an inbound circuit request failed.
CircuitReqDenyFailed {
circuit_id: Option<CircuitId>,
dst_peer_id: PeerId,
error: std::io::Error,
},
/// An inbound cirucit request has been accepted.
CircuitReqAccepted {
circuit_id: CircuitId,
dst_peer_id: PeerId,
},
/// Accepting an inbound circuit request failed.
CircuitReqAcceptFailed {
circuit_id: CircuitId,
dst_peer_id: PeerId,
error: std::io::Error,
},
/// An outbound substream for an inbound circuit request has been
/// negotiated.
OutboundConnectNegotiated {
circuit_id: CircuitId,
src_peer_id: PeerId,
src_connection_id: ConnectionId,
inbound_circuit_req: inbound_hop::CircuitReq,
dst_handler_notifier: oneshot::Sender<()>,
dst_stream: NegotiatedSubstream,
dst_pending_data: Bytes,
},
/// Negotiating an outbound substream for an inbound circuit request failed.
OutboundConnectNegotiationFailed {
circuit_id: CircuitId,
src_peer_id: PeerId,
src_connection_id: ConnectionId,
inbound_circuit_req: inbound_hop::CircuitReq,
status: Status,
error: ProtocolsHandlerUpgrErr<outbound_stop::CircuitFailedReason>,
},
/// An inbound circuit has closed.
CircuitClosed {
circuit_id: CircuitId,
dst_peer_id: PeerId,
error: Option<std::io::Error>,
},
}
impl fmt::Debug for Event {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
Event::ReservationReqReceived {
inbound_reservation_req: _,
endpoint,
renewed,
} => f
.debug_struct("Event::ReservationReqReceived")
.field("endpoint", endpoint)
.field("renewed", renewed)
.finish(),
Event::ReservationReqAccepted { renewed } => f
.debug_struct("Event::ReservationReqAccepted")
.field("renewed", renewed)
.finish(),
Event::ReservationReqAcceptFailed { error } => f
.debug_struct("Event::ReservationReqAcceptFailed")
.field("error", error)
.finish(),
Event::ReservationReqDenied {} => {
f.debug_struct("Event::ReservationReqDenied").finish()
}
Event::ReservationReqDenyFailed { error } => f
.debug_struct("Event::ReservationReqDenyFailed")
.field("error", error)
.finish(),
Event::ReservationTimedOut {} => f.debug_struct("Event::ReservationTimedOut").finish(),
Event::CircuitReqReceived {
endpoint,
inbound_circuit_req: _,
} => f
.debug_struct("Event::CircuitReqReceived")
.field("endpoint", endpoint)
.finish(),
Event::CircuitReqReceiveFailed { error } => f
.debug_struct("Event::CircuitReqReceiveFailed")
.field("error", error)
.finish(),
Event::CircuitReqDenied {
circuit_id,
dst_peer_id,
} => f
.debug_struct("Event::CircuitReqDenied")
.field("circuit_id", circuit_id)
.field("dst_peer_id", dst_peer_id)
.finish(),
Event::CircuitReqDenyFailed {
circuit_id,
dst_peer_id,
error,
} => f
.debug_struct("Event::CircuitReqDenyFailed")
.field("circuit_id", circuit_id)
.field("dst_peer_id", dst_peer_id)
.field("error", error)
.finish(),
Event::CircuitReqAccepted {
circuit_id,
dst_peer_id,
} => f
.debug_struct("Event::CircuitReqAccepted")
.field("circuit_id", circuit_id)
.field("dst_peer_id", dst_peer_id)
.finish(),
Event::CircuitReqAcceptFailed {
circuit_id,
dst_peer_id,
error,
} => f
.debug_struct("Event::CircuitReqAcceptFailed")
.field("circuit_id", circuit_id)
.field("dst_peer_id", dst_peer_id)
.field("error", error)
.finish(),
Event::OutboundConnectNegotiated {
circuit_id,
src_peer_id,
src_connection_id,
inbound_circuit_req: _,
dst_handler_notifier: _,
dst_stream: _,
dst_pending_data: _,
} => f
.debug_struct("Event::OutboundConnectNegotiated")
.field("circuit_id", circuit_id)
.field("src_peer_id", src_peer_id)
.field("src_connection_id", src_connection_id)
.finish(),
Event::OutboundConnectNegotiationFailed {
circuit_id,
src_peer_id,
src_connection_id,
inbound_circuit_req: _,
status,
error,
} => f
.debug_struct("Event::OutboundConnectNegotiationFailed")
.field("circuit_id", circuit_id)
.field("src_peer_id", src_peer_id)
.field("src_connection_id", src_connection_id)
.field("status", status)
.field("error", error)
.finish(),
Event::CircuitClosed {
circuit_id,
dst_peer_id,
error,
} => f
.debug_struct("Event::CircuitClosed")
.field("circuit_id", circuit_id)
.field("dst_peer_id", dst_peer_id)
.field("error", error)
.finish(),
}
}
}
pub struct Prototype {
pub config: Config,
}
impl IntoProtocolsHandler for Prototype {
type Handler = Either<Handler, DummyProtocolsHandler>;
fn into_handler(self, _remote_peer_id: &PeerId, endpoint: &ConnectedPoint) -> Self::Handler {
if endpoint.is_relayed() {
// Deny all substreams on relayed connection.
Either::Right(DummyProtocolsHandler::default())
} else {
Either::Left(Handler {
endpoint: endpoint.clone(),
config: self.config,
queued_events: Default::default(),
pending_error: Default::default(),
reservation_accept_futures: Default::default(),
reservation_deny_futures: Default::default(),
circuit_accept_futures: Default::default(),
circuit_deny_futures: Default::default(),
alive_lend_out_substreams: Default::default(),
circuits: Default::default(),
active_reservation: Default::default(),
keep_alive: KeepAlive::Yes,
})
}
}
fn inbound_protocol(&self) -> <Self::Handler as ProtocolsHandler>::InboundProtocol {
upgrade::EitherUpgrade::A(SendWrapper(inbound_hop::Upgrade {
reservation_duration: self.config.reservation_duration,
max_circuit_duration: self.config.max_circuit_duration,
max_circuit_bytes: self.config.max_circuit_bytes,
}))
}
}
/// [`ProtocolsHandler`] that manages substreams for a relay on a single
/// connection with a peer.
pub struct Handler {
endpoint: ConnectedPoint,
/// Static [`Handler`] [`Config`].
config: Config,
/// Queue of events to return when polled.
queued_events: VecDeque<
ProtocolsHandlerEvent<
<Self as ProtocolsHandler>::OutboundProtocol,
<Self as ProtocolsHandler>::OutboundOpenInfo,
<Self as ProtocolsHandler>::OutEvent,
<Self as ProtocolsHandler>::Error,
>,
>,
/// A pending fatal error that results in the connection being closed.
pending_error: Option<
ProtocolsHandlerUpgrErr<
EitherError<inbound_hop::FatalUpgradeError, outbound_stop::FatalUpgradeError>,
>,
>,
/// Until when to keep the connection alive.
keep_alive: KeepAlive,
/// Futures accepting an inbound reservation request.
reservation_accept_futures: Futures<Result<(), std::io::Error>>,
/// Futures denying an inbound reservation request.
reservation_deny_futures: Futures<Result<(), std::io::Error>>,
/// Timeout for the currently active reservation.
active_reservation: Option<Delay>,
/// Futures accepting an inbound circuit request.
circuit_accept_futures: Futures<Result<CircuitParts, (CircuitId, PeerId, std::io::Error)>>,
/// Futures deying an inbound circuit request.
circuit_deny_futures: Futures<(Option<CircuitId>, PeerId, Result<(), std::io::Error>)>,
/// Tracks substreams lend out to other [`Handler`]s.
///
/// Contains a [`futures::future::Future`] for each lend out substream that
/// resolves once the 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<()>>,
/// Futures relaying data for circuit between two peers.
circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>,
}
type Futures<T> = FuturesUnordered<BoxFuture<'static, T>>;
impl ProtocolsHandler for Handler {
type InEvent = In;
type OutEvent = Event;
type Error = ProtocolsHandlerUpgrErr<
EitherError<inbound_hop::FatalUpgradeError, outbound_stop::FatalUpgradeError>,
>;
type InboundProtocol = inbound_hop::Upgrade;
type OutboundProtocol = outbound_stop::Upgrade;
type OutboundOpenInfo = OutboundOpenInfo;
type InboundOpenInfo = ();
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
SubstreamProtocol::new(
inbound_hop::Upgrade {
reservation_duration: self.config.reservation_duration,
max_circuit_duration: self.config.max_circuit_duration,
max_circuit_bytes: self.config.max_circuit_bytes,
},
(),
)
}
fn inject_fully_negotiated_inbound(
&mut self,
request: <Self::InboundProtocol as upgrade::InboundUpgrade<NegotiatedSubstream>>::Output,
_: Self::InboundOpenInfo,
) {
match request {
inbound_hop::Req::Reserve(inbound_reservation_req) => {
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::ReservationReqReceived {
inbound_reservation_req,
endpoint: self.endpoint.clone(),
renewed: self.active_reservation.is_some(),
},
));
}
inbound_hop::Req::Connect(inbound_circuit_req) => {
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::CircuitReqReceived {
inbound_circuit_req,
endpoint: self.endpoint.clone(),
},
));
}
}
}
fn inject_fully_negotiated_outbound(
&mut self,
(dst_stream, dst_pending_data): <Self::OutboundProtocol as upgrade::OutboundUpgrade<
NegotiatedSubstream,
>>::Output,
outbound_open_info: Self::OutboundOpenInfo,
) {
let OutboundOpenInfo {
circuit_id,
inbound_circuit_req,
src_peer_id,
src_connection_id,
} = outbound_open_info;
let (tx, rx) = oneshot::channel();
self.alive_lend_out_substreams.push(rx);
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::OutboundConnectNegotiated {
circuit_id,
src_peer_id,
src_connection_id,
inbound_circuit_req,
dst_handler_notifier: tx,
dst_stream,
dst_pending_data,
},
));
}
fn inject_event(&mut self, event: Self::InEvent) {
match event {
In::AcceptReservationReq {
inbound_reservation_req,
addrs,
} => {
self.reservation_accept_futures
.push(inbound_reservation_req.accept(addrs).boxed());
}
In::DenyReservationReq {
inbound_reservation_req,
status,
} => {
self.reservation_deny_futures
.push(inbound_reservation_req.deny(status).boxed());
}
In::NegotiateOutboundConnect {
circuit_id,
inbound_circuit_req,
relay_peer_id,
src_peer_id,
src_connection_id,
} => {
self.queued_events
.push_back(ProtocolsHandlerEvent::OutboundSubstreamRequest {
protocol: SubstreamProtocol::new(
outbound_stop::Upgrade {
relay_peer_id,
max_circuit_duration: self.config.max_circuit_duration,
max_circuit_bytes: self.config.max_circuit_bytes,
},
OutboundOpenInfo {
circuit_id,
inbound_circuit_req,
src_peer_id,
src_connection_id,
},
),
});
}
In::DenyCircuitReq {
circuit_id,
inbound_circuit_req,
status,
} => {
let dst_peer_id = inbound_circuit_req.dst();
self.circuit_deny_futures.push(
inbound_circuit_req
.deny(status)
.map(move |result| (circuit_id, dst_peer_id, result))
.boxed(),
);
}
In::AcceptAndDriveCircuit {
circuit_id,
dst_peer_id,
inbound_circuit_req,
dst_handler_notifier,
dst_stream,
dst_pending_data,
} => {
self.circuit_accept_futures.push(
inbound_circuit_req
.accept()
.map_ok(move |(src_stream, src_pending_data)| CircuitParts {
circuit_id,
src_stream,
src_pending_data,
dst_peer_id,
dst_handler_notifier,
dst_stream,
dst_pending_data,
})
.map_err(move |e| (circuit_id, dst_peer_id, e))
.boxed(),
);
}
}
}
fn inject_listen_upgrade_error(
&mut self,
_: Self::InboundOpenInfo,
error: ProtocolsHandlerUpgrErr<<Self::InboundProtocol as InboundUpgradeSend>::Error>,
) {
let non_fatal_error = match error {
ProtocolsHandlerUpgrErr::Timeout => ProtocolsHandlerUpgrErr::Timeout,
ProtocolsHandlerUpgrErr::Timer => ProtocolsHandlerUpgrErr::Timer,
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select(
upgrade::NegotiationError::Failed,
)) => 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)),
));
return;
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(
inbound_hop::UpgradeError::Fatal(error),
)) => {
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Apply(EitherError::A(error)),
));
return;
}
};
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::CircuitReqReceiveFailed {
error: non_fatal_error,
},
));
}
fn inject_dial_upgrade_error(
&mut self,
open_info: Self::OutboundOpenInfo,
error: ProtocolsHandlerUpgrErr<<Self::OutboundProtocol as OutboundUpgradeSend>::Error>,
) {
let (non_fatal_error, status) = match error {
ProtocolsHandlerUpgrErr::Timeout => {
(ProtocolsHandlerUpgrErr::Timeout, Status::ConnectionFailed)
}
ProtocolsHandlerUpgrErr::Timer => {
(ProtocolsHandlerUpgrErr::Timer, Status::ConnectionFailed)
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select(
upgrade::NegotiationError::Failed,
)) => {
// The remote has previously done a reservation. Doing a reservation but not
// supporting the stop protocol is pointless, thus disconnecting.
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Select(upgrade::NegotiationError::Failed),
));
return;
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Select(
upgrade::NegotiationError::ProtocolError(e),
)) => {
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Select(upgrade::NegotiationError::ProtocolError(e)),
));
return;
}
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(error)) => match error {
outbound_stop::UpgradeError::Fatal(error) => {
self.pending_error = Some(ProtocolsHandlerUpgrErr::Upgrade(
upgrade::UpgradeError::Apply(EitherError::B(error)),
));
return;
}
outbound_stop::UpgradeError::CircuitFailed(error) => {
let status = match error {
outbound_stop::CircuitFailedReason::ResourceLimitExceeded => {
Status::ResourceLimitExceeded
}
outbound_stop::CircuitFailedReason::PermissionDenied => {
Status::PermissionDenied
}
};
(
ProtocolsHandlerUpgrErr::Upgrade(upgrade::UpgradeError::Apply(error)),
status,
)
}
},
};
let OutboundOpenInfo {
circuit_id,
inbound_circuit_req,
src_peer_id,
src_connection_id,
} = open_info;
self.queued_events.push_back(ProtocolsHandlerEvent::Custom(
Event::OutboundConnectNegotiationFailed {
circuit_id,
src_peer_id,
src_connection_id,
inbound_circuit_req,
status,
error: non_fatal_error,
},
));
}
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));
}
// Return queued events.
if let Some(event) = self.queued_events.pop_front() {
return Poll::Ready(event);
}
if let Poll::Ready(Some((circuit_id, dst_peer_id, result))) =
self.circuits.poll_next_unpin(cx)
{
match result {
Ok(()) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(Event::CircuitClosed {
circuit_id,
dst_peer_id,
error: None,
}))
}
Err(e) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(Event::CircuitClosed {
circuit_id,
dst_peer_id,
error: Some(e),
}))
}
}
}
if let Poll::Ready(Some(result)) = self.reservation_accept_futures.poll_next_unpin(cx) {
match result {
Ok(()) => {
let renewed = self
.active_reservation
.replace(Delay::new(self.config.reservation_duration))
.is_some();
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::ReservationReqAccepted { renewed },
));
}
Err(error) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::ReservationReqAcceptFailed { error },
));
}
}
}
if let Poll::Ready(Some(result)) = self.reservation_deny_futures.poll_next_unpin(cx) {
match result {
Ok(()) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::ReservationReqDenied {},
))
}
Err(error) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::ReservationReqDenyFailed { error },
));
}
}
}
if let Poll::Ready(Some(result)) = self.circuit_accept_futures.poll_next_unpin(cx) {
match result {
Ok(parts) => {
let CircuitParts {
circuit_id,
mut src_stream,
src_pending_data,
dst_peer_id,
dst_handler_notifier,
mut dst_stream,
dst_pending_data,
} = parts;
let max_circuit_duration = self.config.max_circuit_duration;
let max_circuit_bytes = self.config.max_circuit_bytes;
let circuit = async move {
let (result_1, result_2) = futures::future::join(
src_stream.write_all(&dst_pending_data),
dst_stream.write_all(&src_pending_data),
)
.await;
result_1?;
result_2?;
CopyFuture::new(
src_stream,
dst_stream,
max_circuit_duration,
max_circuit_bytes,
)
.await?;
// Inform destination handler that the stream to the destination is dropped.
drop(dst_handler_notifier);
Ok(())
}
.map(move |r| (circuit_id, dst_peer_id, r))
.boxed();
self.circuits.push(circuit);
return Poll::Ready(ProtocolsHandlerEvent::Custom(Event::CircuitReqAccepted {
circuit_id,
dst_peer_id,
}));
}
Err((circuit_id, dst_peer_id, error)) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::CircuitReqAcceptFailed {
circuit_id,
dst_peer_id,
error,
},
));
}
}
}
if let Poll::Ready(Some((circuit_id, dst_peer_id, result))) =
self.circuit_deny_futures.poll_next_unpin(cx)
{
match result {
Ok(()) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(Event::CircuitReqDenied {
circuit_id,
dst_peer_id,
}));
}
Err(error) => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
Event::CircuitReqDenyFailed {
circuit_id,
dst_peer_id,
error,
},
));
}
}
}
while let Poll::Ready(Some(Err(Canceled))) =
self.alive_lend_out_substreams.poll_next_unpin(cx)
{}
if let Some(Poll::Ready(())) = self
.active_reservation
.as_mut()
.map(|fut| fut.poll_unpin(cx))
{
self.active_reservation = None;
return Poll::Ready(ProtocolsHandlerEvent::Custom(Event::ReservationTimedOut {}));
}
if self.reservation_accept_futures.is_empty()
&& self.reservation_deny_futures.is_empty()
&& self.circuit_accept_futures.is_empty()
&& self.circuit_deny_futures.is_empty()
&& self.alive_lend_out_substreams.is_empty()
&& self.circuits.is_empty()
&& self.active_reservation.is_none()
{
match self.keep_alive {
KeepAlive::Yes => {
self.keep_alive = KeepAlive::Until(Instant::now() + Duration::from_secs(10));
}
KeepAlive::Until(_) => {}
KeepAlive::No => panic!("Handler never sets KeepAlive::No."),
}
} else {
self.keep_alive = KeepAlive::Yes;
}
Poll::Pending
}
}
pub struct OutboundOpenInfo {
circuit_id: CircuitId,
inbound_circuit_req: inbound_hop::CircuitReq,
src_peer_id: PeerId,
src_connection_id: ConnectionId,
}
pub struct CircuitParts {
circuit_id: CircuitId,
src_stream: NegotiatedSubstream,
src_pending_data: Bytes,
dst_peer_id: PeerId,
dst_handler_notifier: oneshot::Sender<()>,
dst_stream: NegotiatedSubstream,
dst_pending_data: Bytes,
}

View File

@ -0,0 +1,315 @@
// Copyright 2021 Protocol Labs.
//
// 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.
pub use generic::{
RateLimiter as GenericRateLimiter, RateLimiterConfig as GenericRateLimiterConfig,
};
use instant::Instant;
use libp2p_core::multiaddr::{Multiaddr, Protocol};
use libp2p_core::PeerId;
use std::net::IpAddr;
/// Allows rate limiting access to some resource based on the [`PeerId`] and
/// [`Multiaddr`] of a remote peer.
///
/// See [`new_per_peer`] and [`new_per_ip`] for precast implementations. Use
/// [`GenericRateLimiter`] to build your own, e.g. based on the autonomous system
/// number of a peers IP address.
pub trait RateLimiter: Send {
fn try_next(&mut self, peer: PeerId, addr: &Multiaddr, now: Instant) -> bool;
}
pub fn new_per_peer(config: GenericRateLimiterConfig) -> Box<dyn RateLimiter> {
let mut limiter = GenericRateLimiter::new(config);
Box::new(move |peer_id, _addr: &Multiaddr, now| limiter.try_next(peer_id, now))
}
pub fn new_per_ip(config: GenericRateLimiterConfig) -> Box<dyn RateLimiter> {
let mut limiter = GenericRateLimiter::new(config);
Box::new(move |_peer_id, addr: &Multiaddr, now| {
multiaddr_to_ip(addr)
.map(|a| limiter.try_next(a, now))
.unwrap_or(true)
})
}
impl<T: FnMut(PeerId, &Multiaddr, Instant) -> bool + Send> RateLimiter for T {
fn try_next(&mut self, peer: PeerId, addr: &Multiaddr, now: Instant) -> bool {
self(peer, addr, now)
}
}
fn multiaddr_to_ip(addr: &Multiaddr) -> Option<IpAddr> {
addr.iter().find_map(|p| match p {
Protocol::Ip4(addr) => Some(addr.into()),
Protocol::Ip6(addr) => Some(addr.into()),
_ => None,
})
}
mod generic {
use instant::Instant;
use std::collections::{HashMap, VecDeque};
use std::convert::TryInto;
use std::hash::Hash;
use std::num::NonZeroU32;
use std::time::Duration;
/// Rate limiter using the [Token Bucket] algorithm.
///
/// [Token Bucket]: https://en.wikipedia.org/wiki/Token_bucket
pub struct RateLimiter<Id> {
limit: u32,
interval: Duration,
refill_schedule: VecDeque<(Instant, Id)>,
buckets: HashMap<Id, u32>,
}
/// Configuration for a [`RateLimiter`].
#[derive(Clone, Copy)]
pub struct RateLimiterConfig {
/// The maximum number of tokens in the bucket at any point in time.
pub limit: NonZeroU32,
/// The interval at which a single token is added to the bucket.
pub interval: Duration,
}
impl<Id: Eq + PartialEq + Hash + Clone> RateLimiter<Id> {
pub(crate) fn new(config: RateLimiterConfig) -> Self {
assert!(!config.interval.is_zero());
Self {
limit: config.limit.into(),
interval: config.interval,
refill_schedule: Default::default(),
buckets: Default::default(),
}
}
pub(crate) fn try_next(&mut self, id: Id, now: Instant) -> bool {
self.refill(now);
match self.buckets.get_mut(&id) {
// If the bucket exists, try to take a token.
Some(balance) => match balance.checked_sub(1) {
Some(a) => {
*balance = a;
true
}
None => false,
},
// If the bucket is missing, act like the bucket has `limit` number of tokens. Take one
// token and track the new bucket balance.
None => {
self.buckets.insert(id.clone(), self.limit - 1);
self.refill_schedule.push_back((now, id));
true
}
}
}
fn refill(&mut self, now: Instant) {
// Note when used with a high number of buckets: This loop refills all the to-be-refilled
// buckets at once, thus potentially delaying the parent call to `try_next`.
loop {
match self.refill_schedule.get(0) {
// Only continue if (a) there is a bucket and (b) the bucket has not already been
// refilled recently.
Some((last_refill, _)) if now.duration_since(*last_refill) >= self.interval => {
}
// Otherwise stop refilling. Items in `refill_schedule` are sorted, thus, if the
// first ain't ready, none of them are.
_ => return,
};
let (last_refill, id) = self
.refill_schedule
.pop_front()
.expect("Queue not to be empty.");
// Get the current balance of the bucket.
let balance = self
.buckets
.get(&id)
.expect("Entry can only be removed via refill.");
// Calculate the new balance.
let duration_since = now.duration_since(last_refill);
let new_tokens = duration_since
.as_micros()
// Note that the use of `as_micros` limits the number of tokens to 10^6 per second.
.checked_div(self.interval.as_micros())
.and_then(|i| i.try_into().ok())
.unwrap_or(u32::MAX);
let new_balance = balance.checked_add(new_tokens).unwrap_or(u32::MAX);
// If the new balance is below the limit, update the bucket.
if new_balance < self.limit {
self.buckets
.insert(id.clone(), new_balance)
.expect("To override value.");
self.refill_schedule.push_back((now, id));
} else {
// If the balance is above the limit, the bucket can be removed, given that a
// non-existing bucket is equivalent to a bucket with `limit` tokens.
self.buckets.remove(&id);
}
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use quickcheck::{QuickCheck, TestResult};
use std::num::NonZeroU32;
#[test]
fn first() {
let id = 1;
let mut l = RateLimiter::new(RateLimiterConfig {
limit: NonZeroU32::new(10).unwrap(),
interval: Duration::from_secs(1),
});
assert!(l.try_next(id, Instant::now()));
}
#[test]
fn limits() {
let id = 1;
let now = Instant::now();
let mut l = RateLimiter::new(RateLimiterConfig {
limit: NonZeroU32::new(10).unwrap(),
interval: Duration::from_secs(1),
});
for _ in 0..10 {
assert!(l.try_next(id, now));
}
assert!(!l.try_next(id, now));
}
#[test]
fn refills() {
let id = 1;
let now = Instant::now();
let mut l = RateLimiter::new(RateLimiterConfig {
limit: NonZeroU32::new(10).unwrap(),
interval: Duration::from_secs(1),
});
for _ in 0..10 {
assert!(l.try_next(id, now));
}
assert!(!l.try_next(id, now));
let now = now + Duration::from_secs(1);
assert!(l.try_next(id, now));
assert!(!l.try_next(id, now));
let now = now + Duration::from_secs(10);
for _ in 0..10 {
assert!(l.try_next(id, now));
}
}
#[test]
fn move_at_half_interval_steps() {
let id = 1;
let now = Instant::now();
let mut l = RateLimiter::new(RateLimiterConfig {
limit: NonZeroU32::new(1).unwrap(),
interval: Duration::from_secs(2),
});
assert!(l.try_next(id, now));
assert!(!l.try_next(id, now));
let now = now + Duration::from_secs(1);
assert!(!l.try_next(id, now));
let now = now + Duration::from_secs(1);
assert!(l.try_next(id, now));
}
#[test]
fn garbage_collects() {
let now = Instant::now();
let mut l = RateLimiter::new(RateLimiterConfig {
limit: NonZeroU32::new(1).unwrap(),
interval: Duration::from_secs(1),
});
assert!(l.try_next(1, now));
let now = now + Duration::from_secs(1);
assert!(l.try_next(2, now));
assert_eq!(l.buckets.len(), 1);
assert_eq!(l.refill_schedule.len(), 1);
}
#[test]
fn quick_check() {
fn prop(
limit: NonZeroU32,
interval: Duration,
events: Vec<(u32, Duration)>,
) -> TestResult {
if interval.is_zero() {
return TestResult::discard();
}
let mut now = Instant::now();
let mut l = RateLimiter::new(RateLimiterConfig {
limit: limit.try_into().unwrap(),
interval,
});
for (id, d) in events {
now = if let Some(now) = now.checked_add(d) {
now
} else {
return TestResult::discard();
};
l.try_next(id, now);
}
now = if let Some(now) = interval
.checked_mul(limit.into())
.and_then(|full_interval| now.checked_add(full_interval))
{
now
} else {
return TestResult::discard();
};
assert!(l.try_next(1, now));
assert_eq!(l.buckets.len(), 1);
assert_eq!(l.refill_schedule.len(), 1);
TestResult::passed()
}
QuickCheck::new().quickcheck(prop as fn(_, _, _) -> _)
}
}
}

View File

@ -33,10 +33,10 @@ use libp2p_identify::{Identify, IdentifyConfig, IdentifyEvent, IdentifyInfo};
use libp2p_kad::{GetClosestPeersOk, Kademlia, KademliaEvent, QueryResult};
use libp2p_ping as ping;
use libp2p_plaintext::PlainText2Config;
use libp2p_relay::{Relay, RelayConfig};
use libp2p_relay::v1::{new_transport_and_behaviour, Relay, RelayConfig};
use libp2p_swarm::protocols_handler::KeepAlive;
use libp2p_swarm::{
dial_opts::DialOpts, DialError, DummyBehaviour, NetworkBehaviour, NetworkBehaviourAction,
DialError, DummyBehaviour, NetworkBehaviour, NetworkBehaviourAction,
NetworkBehaviourEventProcess, PollParameters, Swarm, SwarmEvent,
};
use std::task::{Context, Poll};
@ -1284,7 +1284,7 @@ fn build_swarm(reachability: Reachability, relay_mode: RelayMode) -> Swarm<Combi
Reachability::Routable => EitherTransport::Right(transport),
};
let (transport, relay_behaviour) = libp2p_relay::new_transport_and_behaviour(
let (transport, relay_behaviour) = new_transport_and_behaviour(
RelayConfig {
actively_connect_to_dst_nodes: relay_mode.into(),
..Default::default()
@ -1326,7 +1326,7 @@ fn build_keep_alive_swarm() -> Swarm<CombinedKeepAliveBehaviour> {
let transport = MemoryTransport::default();
let (transport, relay_behaviour) =
libp2p_relay::new_transport_and_behaviour(RelayConfig::default(), transport);
new_transport_and_behaviour(RelayConfig::default(), transport);
let transport = transport
.upgrade(upgrade::Version::V1)

451
protocols/relay/tests/v2.rs Normal file
View File

@ -0,0 +1,451 @@
// Copyright 2021 Protocol Labs.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::executor::LocalPool;
use futures::future::FutureExt;
use futures::io::{AsyncRead, AsyncWrite};
use futures::stream::StreamExt;
use futures::task::Spawn;
use libp2p::core::multiaddr::{Multiaddr, Protocol};
use libp2p::core::muxing::StreamMuxerBox;
use libp2p::core::transport::choice::OrTransport;
use libp2p::core::transport::{Boxed, MemoryTransport, Transport};
use libp2p::core::PublicKey;
use libp2p::core::{identity, upgrade, PeerId};
use libp2p::ping::{Ping, PingConfig, PingEvent};
use libp2p::plaintext::PlainText2Config;
use libp2p::relay::v2::client;
use libp2p::relay::v2::relay;
use libp2p::NetworkBehaviour;
use libp2p_swarm::{AddressScore, NetworkBehaviour, Swarm, SwarmEvent};
use std::time::Duration;
#[test]
fn reservation() {
let _ = env_logger::try_init();
let mut pool = LocalPool::new();
let relay_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::<u64>()));
let mut relay = build_relay();
let relay_peer_id = *relay.local_peer_id();
relay.listen_on(relay_addr.clone()).unwrap();
relay.add_external_address(relay_addr.clone(), AddressScore::Infinite);
spawn_swarm_on_pool(&pool, relay);
let client_addr = relay_addr
.clone()
.with(Protocol::P2p(relay_peer_id.into()))
.with(Protocol::P2pCircuit);
let mut client = build_client();
let client_peer_id = *client.local_peer_id();
client.listen_on(client_addr.clone()).unwrap();
// Wait for connection to relay.
assert!(pool.run_until(wait_for_dial(&mut client, relay_peer_id)));
// Wait for initial reservation.
pool.run_until(wait_for_reservation(
&mut client,
client_addr
.clone()
.with(Protocol::P2p(client_peer_id.into())),
relay_peer_id,
false, // No renewal.
));
// Wait for renewal.
pool.run_until(wait_for_reservation(
&mut client,
client_addr.with(Protocol::P2p(client_peer_id.into())),
relay_peer_id,
true, // Renewal.
));
}
#[test]
fn new_reservation_to_same_relay_replaces_old() {
let _ = env_logger::try_init();
let mut pool = LocalPool::new();
let relay_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::<u64>()));
let mut relay = build_relay();
let relay_peer_id = *relay.local_peer_id();
relay.listen_on(relay_addr.clone()).unwrap();
relay.add_external_address(relay_addr.clone(), AddressScore::Infinite);
spawn_swarm_on_pool(&pool, relay);
let mut client = build_client();
let client_peer_id = *client.local_peer_id();
let client_addr = relay_addr
.clone()
.with(Protocol::P2p(relay_peer_id.into()))
.with(Protocol::P2pCircuit);
let client_addr_with_peer_id = client_addr
.clone()
.with(Protocol::P2p(client_peer_id.into()));
let old_listener = client.listen_on(client_addr.clone()).unwrap();
// Wait for connection to relay.
assert!(pool.run_until(wait_for_dial(&mut client, relay_peer_id)));
// Wait for first (old) reservation.
pool.run_until(wait_for_reservation(
&mut client,
client_addr_with_peer_id.clone(),
relay_peer_id,
false, // No renewal.
));
// Trigger new reservation.
let new_listener = client.listen_on(client_addr.clone()).unwrap();
// Wait for
// - listener of old reservation to close
// - new reservation to be accepted
// - new listener address to be reported
pool.run_until(async {
let mut old_listener_closed = false;
let mut new_reservation_accepted = false;
let mut new_listener_address_reported = false;
loop {
match client.select_next_some().await {
SwarmEvent::ListenerClosed {
addresses,
listener_id,
..
} => {
assert_eq!(addresses, vec![client_addr_with_peer_id.clone()]);
assert_eq!(listener_id, old_listener);
old_listener_closed = true;
if new_reservation_accepted && new_listener_address_reported {
break;
}
}
SwarmEvent::Behaviour(ClientEvent::Relay(
client::Event::ReservationReqAccepted {
relay_peer_id: peer_id,
..
},
)) => {
assert_eq!(relay_peer_id, peer_id);
new_reservation_accepted = true;
if old_listener_closed && new_listener_address_reported {
break;
}
}
SwarmEvent::NewListenAddr {
address,
listener_id,
} => {
assert_eq!(address, client_addr_with_peer_id);
assert_eq!(listener_id, new_listener);
new_listener_address_reported = true;
if old_listener_closed && new_reservation_accepted {
break;
}
}
SwarmEvent::Behaviour(ClientEvent::Ping(_)) => {}
e => panic!("{:?}", e),
}
}
});
}
#[test]
fn connect() {
let _ = env_logger::try_init();
let mut pool = LocalPool::new();
let relay_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::<u64>()));
let mut relay = build_relay();
let relay_peer_id = *relay.local_peer_id();
relay.listen_on(relay_addr.clone()).unwrap();
relay.add_external_address(relay_addr.clone(), AddressScore::Infinite);
spawn_swarm_on_pool(&pool, relay);
let mut dst = build_client();
let dst_peer_id = *dst.local_peer_id();
let dst_addr = relay_addr
.clone()
.with(Protocol::P2p(relay_peer_id.into()))
.with(Protocol::P2pCircuit)
.with(Protocol::P2p(dst_peer_id.into()));
dst.listen_on(dst_addr.clone()).unwrap();
assert!(pool.run_until(wait_for_dial(&mut dst, relay_peer_id)));
pool.run_until(wait_for_reservation(
&mut dst,
dst_addr.clone(),
relay_peer_id,
false, // No renewal.
));
spawn_swarm_on_pool(&pool, dst);
let mut src = build_client();
src.dial(dst_addr).unwrap();
pool.run_until(async {
loop {
match src.select_next_some().await {
SwarmEvent::Dialing(peer_id) if peer_id == relay_peer_id => {}
SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == relay_peer_id => {}
SwarmEvent::Behaviour(ClientEvent::Ping(PingEvent { peer, .. }))
if peer == dst_peer_id =>
{
break
}
SwarmEvent::Behaviour(ClientEvent::Relay(
client::Event::OutboundCircuitEstablished { .. },
)) => {}
SwarmEvent::Behaviour(ClientEvent::Ping(PingEvent { peer, .. }))
if peer == relay_peer_id => {}
SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == dst_peer_id => {
break
}
e => panic!("{:?}", e),
}
}
})
}
#[test]
fn handle_dial_failure() {
let _ = env_logger::try_init();
let mut pool = LocalPool::new();
let relay_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::<u64>()));
let relay_peer_id = PeerId::random();
let mut client = build_client();
let client_peer_id = *client.local_peer_id();
let client_addr = relay_addr
.clone()
.with(Protocol::P2p(relay_peer_id.into()))
.with(Protocol::P2pCircuit)
.with(Protocol::P2p(client_peer_id.into()));
client.listen_on(client_addr.clone()).unwrap();
assert!(!pool.run_until(wait_for_dial(&mut client, relay_peer_id)));
}
#[test]
fn reuse_connection() {
let _ = env_logger::try_init();
let mut pool = LocalPool::new();
let relay_addr = Multiaddr::empty().with(Protocol::Memory(rand::random::<u64>()));
let mut relay = build_relay();
let relay_peer_id = *relay.local_peer_id();
relay.listen_on(relay_addr.clone()).unwrap();
relay.add_external_address(relay_addr.clone(), AddressScore::Infinite);
spawn_swarm_on_pool(&pool, relay);
let client_addr = relay_addr
.clone()
.with(Protocol::P2p(relay_peer_id.into()))
.with(Protocol::P2pCircuit);
let mut client = build_client();
let client_peer_id = *client.local_peer_id();
client.dial(relay_addr).unwrap();
assert!(pool.run_until(wait_for_dial(&mut client, relay_peer_id)));
client.listen_on(client_addr.clone()).unwrap();
pool.run_until(wait_for_reservation(
&mut client,
client_addr.with(Protocol::P2p(client_peer_id.into())),
relay_peer_id,
false, // No renewal.
));
}
fn build_relay() -> Swarm<Relay> {
let local_key = identity::Keypair::generate_ed25519();
let local_public_key = local_key.public();
let local_peer_id = local_public_key.clone().to_peer_id();
let transport = upgrade_transport(MemoryTransport::default().boxed(), local_public_key);
Swarm::new(
transport,
Relay {
ping: Ping::new(PingConfig::new()),
relay: relay::Relay::new(
local_peer_id,
relay::Config {
reservation_duration: Duration::from_secs(2),
..Default::default()
},
),
},
local_peer_id,
)
}
fn build_client() -> Swarm<Client> {
let local_key = identity::Keypair::generate_ed25519();
let local_public_key = local_key.public();
let local_peer_id = local_public_key.clone().to_peer_id();
let (relay_transport, behaviour) = client::Client::new_transport_and_behaviour(local_peer_id);
let transport = upgrade_transport(
OrTransport::new(relay_transport, MemoryTransport::default()).boxed(),
local_public_key,
);
Swarm::new(
transport,
Client {
ping: Ping::new(PingConfig::new()),
relay: behaviour,
},
local_peer_id,
)
}
fn upgrade_transport<StreamSink>(
transport: Boxed<StreamSink>,
local_public_key: PublicKey,
) -> Boxed<(PeerId, StreamMuxerBox)>
where
StreamSink: AsyncRead + AsyncWrite + Send + Unpin + 'static,
{
transport
.upgrade(upgrade::Version::V1)
.authenticate(PlainText2Config { local_public_key })
.multiplex(libp2p_yamux::YamuxConfig::default())
.boxed()
}
#[derive(NetworkBehaviour)]
#[behaviour(out_event = "RelayEvent", event_process = false)]
struct Relay {
relay: relay::Relay,
ping: Ping,
}
#[derive(Debug)]
enum RelayEvent {
Relay(relay::Event),
Ping(PingEvent),
}
impl From<relay::Event> for RelayEvent {
fn from(event: relay::Event) -> Self {
RelayEvent::Relay(event)
}
}
impl From<PingEvent> for RelayEvent {
fn from(event: PingEvent) -> Self {
RelayEvent::Ping(event)
}
}
#[derive(NetworkBehaviour)]
#[behaviour(out_event = "ClientEvent", event_process = false)]
struct Client {
relay: client::Client,
ping: Ping,
}
#[derive(Debug)]
enum ClientEvent {
Relay(client::Event),
Ping(PingEvent),
}
impl From<client::Event> for ClientEvent {
fn from(event: client::Event) -> Self {
ClientEvent::Relay(event)
}
}
impl From<PingEvent> for ClientEvent {
fn from(event: PingEvent) -> Self {
ClientEvent::Ping(event)
}
}
fn spawn_swarm_on_pool<B: NetworkBehaviour>(pool: &LocalPool, swarm: Swarm<B>) {
pool.spawner()
.spawn_obj(swarm.collect::<Vec<_>>().map(|_| ()).boxed().into())
.unwrap();
}
async fn wait_for_reservation(
client: &mut Swarm<Client>,
client_addr: Multiaddr,
relay_peer_id: PeerId,
is_renewal: bool,
) {
let mut new_listen_addr = false;
let mut reservation_req_accepted = false;
loop {
match client.select_next_some().await {
SwarmEvent::Behaviour(ClientEvent::Relay(client::Event::ReservationReqAccepted {
relay_peer_id: peer_id,
renewal,
..
})) if relay_peer_id == peer_id && renewal == is_renewal => {
reservation_req_accepted = true;
if new_listen_addr {
break;
}
}
SwarmEvent::NewListenAddr { address, .. } if address == client_addr => {
new_listen_addr = true;
if reservation_req_accepted {
break;
}
}
SwarmEvent::Behaviour(ClientEvent::Ping(_)) => {}
e => panic!("{:?}", e),
}
}
}
async fn wait_for_dial(client: &mut Swarm<Client>, remote: PeerId) -> bool {
loop {
match client.select_next_some().await {
SwarmEvent::Dialing(peer_id) if peer_id == remote => {}
SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == remote => return true,
SwarmEvent::OutgoingConnectionError { peer_id, .. } if peer_id == Some(remote) => {
return false
}
SwarmEvent::Behaviour(ClientEvent::Ping(_)) => {}
e => panic!("{:?}", e),
}
}
}