From 79ee9235c737f26e17c35dd7b132b3e48c5b6699 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 24 Jul 2023 11:44:10 +0300 Subject: [PATCH 01/83] Refactoring of stop protocol --- protocols/relay/src/behaviour/handler.rs | 310 ++++++++++-------- protocols/relay/src/priv_client/handler.rs | 180 +++++----- protocols/relay/src/protocol.rs | 2 +- protocols/relay/src/protocol/inbound_stop.rs | 90 ++--- protocols/relay/src/protocol/outbound_stop.rs | 104 +----- 5 files changed, 300 insertions(+), 386 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 9c1b8524ec3..08cc9d69d83 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -20,25 +20,31 @@ use crate::behaviour::CircuitId; use crate::copy_future::CopyFuture; -use crate::proto; +use crate::proto::Status; +use crate::protocol::inbound_hop::CircuitReq; +use crate::protocol::outbound_stop::{CircuitFailedReason, FatalUpgradeError}; +use crate::protocol::MAX_MESSAGE_SIZE; use crate::protocol::{inbound_hop, outbound_stop}; +use crate::{proto, STOP_PROTOCOL_NAME}; +use asynchronous_codec::{Framed, FramedParts}; use bytes::Bytes; use either::Either; use futures::channel::oneshot::{self, Canceled}; use futures::future::{BoxFuture, FutureExt, TryFutureExt}; use futures::io::AsyncWriteExt; +use futures::prelude::*; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; +use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; use libp2p_swarm::handler::{ - ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, - ListenUpgradeError, + ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound, ListenUpgradeError, }; use libp2p_swarm::{ - ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamUpgradeError, - SubstreamProtocol, + ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, + StreamUpgradeError, SubstreamProtocol, }; use std::collections::VecDeque; use std::fmt; @@ -59,12 +65,12 @@ pub enum In { }, DenyReservationReq { inbound_reservation_req: inbound_hop::ReservationReq, - status: proto::Status, + status: Status, }, DenyCircuitReq { circuit_id: Option, inbound_circuit_req: inbound_hop::CircuitReq, - status: proto::Status, + status: Status, }, NegotiateOutboundConnect { circuit_id: CircuitId, @@ -380,6 +386,11 @@ pub struct Handler { alive_lend_out_substreams: FuturesUnordered>, /// Futures relaying data for circuit between two peers. circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, + + stop_requested_streams: VecDeque, + outbound_stop_futs: FuturesUnordered< + BoxFuture<'static, Result<(Event, Option>), FatalUpgradeError>>, + >, } impl Handler { @@ -396,7 +407,122 @@ impl Handler { circuits: Default::default(), active_reservation: Default::default(), keep_alive: KeepAlive::Yes, + stop_requested_streams: Default::default(), + outbound_stop_futs: Default::default(), + } + } + + fn send_stop_message_and_process_result( + &self, + io: Stream, + stop_command: StopCommand, + ) -> BoxFuture<'static, Result<(Event, Option>), FatalUpgradeError>> { + let msg = proto::StopMessage { + type_pb: proto::StopMessageType::CONNECT, + peer: Some(proto::Peer { + id: stop_command.src_peer_id.to_bytes(), + addrs: vec![], + }), + limit: Some(proto::Limit { + duration: Some( + stop_command + .max_circuit_duration + .as_secs() + .try_into() + .expect("`max_circuit_duration` not to exceed `u32::MAX`."), + ), + data: Some(stop_command.max_circuit_bytes), + }), + status: None, + }; + + let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + async move { + substream.send(msg).await?; + + let proto::StopMessage { + type_pb, + peer: _, + limit: _, + status, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::StopMessageType::CONNECT => { + return Err(FatalUpgradeError::UnexpectedTypeConnect); + } + proto::StopMessageType::STATUS => {} + } + + match status { + Some(proto_status) => match proto_status { + Status::OK => {} + Status::RESOURCE_LIMIT_EXCEEDED => { + return Ok(( + Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply( + CircuitFailedReason::ResourceLimitExceeded, + ), + }, + None, + )) + } + Status::PERMISSION_DENIED => { + return Ok(( + Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply( + CircuitFailedReason::PermissionDenied, + ), + }, + None, + )) + } + s => return Err(FatalUpgradeError::UnexpectedStatus(s)), + }, + None => { + return Err(FatalUpgradeError::MissingStatusField); + } + } + + 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." + ); + + let (tx, rx) = oneshot::channel(); + Ok(( + Event::OutboundConnectNegotiated { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + dst_handler_notifier: tx, + dst_stream: io, + dst_pending_data: read_buffer.freeze(), + }, + Some(rx), + )) } + .boxed() } fn on_fully_negotiated_inbound( @@ -431,39 +557,6 @@ impl Handler { } } - fn on_fully_negotiated_outbound( - &mut self, - FullyNegotiatedOutbound { - protocol: (dst_stream, dst_pending_data), - info: outbound_open_info, - }: FullyNegotiatedOutbound< - ::OutboundProtocol, - ::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(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundConnectNegotiated { - circuit_id, - src_peer_id, - src_connection_id, - inbound_circuit_req, - dst_handler_notifier: tx, - dst_stream, - dst_pending_data, - }, - )); - } - fn on_listen_upgrade_error( &mut self, ListenUpgradeError { @@ -476,70 +569,6 @@ impl Handler { ) { self.pending_error = Some(StreamUpgradeError::Apply(Either::Left(error))); } - - fn on_dial_upgrade_error( - &mut self, - DialUpgradeError { - info: open_info, - error, - }: DialUpgradeError< - ::OutboundOpenInfo, - ::OutboundProtocol, - >, - ) { - let (non_fatal_error, status) = match error { - StreamUpgradeError::Timeout => ( - StreamUpgradeError::Timeout, - proto::Status::CONNECTION_FAILED, - ), - StreamUpgradeError::NegotiationFailed => { - // The remote has previously done a reservation. Doing a reservation but not - // supporting the stop protocol is pointless, thus disconnecting. - self.pending_error = Some(StreamUpgradeError::NegotiationFailed); - return; - } - StreamUpgradeError::Io(e) => { - self.pending_error = Some(StreamUpgradeError::Io(e)); - return; - } - StreamUpgradeError::Apply(error) => match error { - outbound_stop::UpgradeError::Fatal(error) => { - self.pending_error = Some(StreamUpgradeError::Apply(Either::Right(error))); - return; - } - outbound_stop::UpgradeError::CircuitFailed(error) => { - let status = match error { - outbound_stop::CircuitFailedReason::ResourceLimitExceeded => { - proto::Status::RESOURCE_LIMIT_EXCEEDED - } - outbound_stop::CircuitFailedReason::PermissionDenied => { - proto::Status::PERMISSION_DENIED - } - }; - (StreamUpgradeError::Apply(error), status) - } - }, - }; - - let OutboundOpenInfo { - circuit_id, - inbound_circuit_req, - src_peer_id, - src_connection_id, - } = open_info; - - self.queued_events - .push_back(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundConnectNegotiationFailed { - circuit_id, - src_peer_id, - src_connection_id, - inbound_circuit_req, - status, - error: non_fatal_error, - }, - )); - } } enum ReservationRequestFuture { @@ -556,9 +585,9 @@ impl ConnectionHandler for Handler { Either, >; type InboundProtocol = inbound_hop::Upgrade; - type OutboundProtocol = outbound_stop::Upgrade; - type OutboundOpenInfo = OutboundOpenInfo; type InboundOpenInfo = (); + type OutboundProtocol = ReadyUpgrade; + type OutboundOpenInfo = (); fn listen_protocol(&self) -> SubstreamProtocol { SubstreamProtocol::new( @@ -607,21 +636,17 @@ impl ConnectionHandler for Handler { src_peer_id, src_connection_id, } => { + self.stop_requested_streams.push_back(StopCommand { + circuit_id, + inbound_circuit_req, + src_peer_id, + src_connection_id, + max_circuit_duration: self.config.max_circuit_duration, + max_circuit_bytes: self.config.max_circuit_bytes, + }); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { - protocol: SubstreamProtocol::new( - outbound_stop::Upgrade { - src_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, - }, - ), + protocol: SubstreamProtocol::new(ReadyUpgrade::new(STOP_PROTOCOL_NAME), ()), }); } In::DenyCircuitReq { @@ -716,6 +741,21 @@ impl ConnectionHandler for Handler { } } + // Send stop commands + if let Poll::Ready(Some(result)) = self.outbound_stop_futs.poll_next_unpin(cx) { + return match result { + Ok((event, receiver)) => { + if let Some(rx) = receiver { + self.alive_lend_out_substreams.push(rx); + } + Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)) + } + Err(e) => Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(e), + ))), + }; + } + // Deny new circuits. if let Poll::Ready(Some((circuit_id, dst_peer_id, result))) = self.circuit_deny_futures.poll_next_unpin(cx) @@ -899,29 +939,28 @@ impl ConnectionHandler for Handler { ConnectionEvent::FullyNegotiatedInbound(fully_negotiated_inbound) => { self.on_fully_negotiated_inbound(fully_negotiated_inbound) } - ConnectionEvent::FullyNegotiatedOutbound(fully_negotiated_outbound) => { - self.on_fully_negotiated_outbound(fully_negotiated_outbound) + ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { + protocol: stream, + .. + }) => { + let stop_command = self + .stop_requested_streams + .pop_front() + .expect("opened a stream without a pending stop command"); + self.outbound_stop_futs + .push(self.send_stop_message_and_process_result(stream, stop_command)); } ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => { self.on_listen_upgrade_error(listen_upgrade_error) } - ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { - self.on_dial_upgrade_error(dial_upgrade_error) - } ConnectionEvent::AddressChange(_) + | ConnectionEvent::DialUpgradeError(_) | ConnectionEvent::LocalProtocolsChange(_) | ConnectionEvent::RemoteProtocolsChange(_) => {} } } } -pub struct OutboundOpenInfo { - circuit_id: CircuitId, - inbound_circuit_req: inbound_hop::CircuitReq, - src_peer_id: PeerId, - src_connection_id: ConnectionId, -} - pub(crate) struct CircuitParts { circuit_id: CircuitId, src_stream: Stream, @@ -931,3 +970,12 @@ pub(crate) struct CircuitParts { dst_stream: Stream, dst_pending_data: Bytes, } + +pub(crate) struct StopCommand { + circuit_id: CircuitId, + inbound_circuit_req: CircuitReq, + src_peer_id: PeerId, + src_connection_id: ConnectionId, + max_circuit_duration: Duration, + max_circuit_bytes: u64, +} diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 9613d7d6b3e..0932b95aade 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -18,9 +18,11 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::priv_client::transport; -use crate::proto; -use crate::protocol::{self, inbound_stop, outbound_hop}; +use std::collections::{HashMap, VecDeque}; +use std::fmt; +use std::task::{Context, Poll}; +use std::time::Duration; + use either::Either; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; @@ -28,21 +30,24 @@ use futures::sink::SinkExt; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; +use log::debug; + use libp2p_core::multiaddr::Protocol; +use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::Multiaddr; use libp2p_identity::PeerId; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, - ListenUpgradeError, }; use libp2p_swarm::{ - ConnectionHandler, ConnectionHandlerEvent, KeepAlive, StreamUpgradeError, SubstreamProtocol, + ConnectionHandler, ConnectionHandlerEvent, KeepAlive, StreamProtocol, StreamUpgradeError, + SubstreamProtocol, }; -use log::debug; -use std::collections::{HashMap, VecDeque}; -use std::fmt; -use std::task::{Context, Poll}; -use std::time::Duration; + +use crate::priv_client::transport; +use crate::protocol::inbound_stop::{open_circuit, Circuit}; +use crate::protocol::{self, inbound_stop, outbound_hop}; +use crate::{proto, STOP_PROTOCOL_NAME}; /// The maximum number of circuits being denied concurrently. /// @@ -140,8 +145,10 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - circuit_deny_futs: - HashMap>>, + open_circuit_futs: + FuturesUnordered>>, + + circuit_deny_futs: HashMap>>, /// Futures that try to send errors to the transport. /// @@ -160,72 +167,13 @@ impl Handler { pending_error: Default::default(), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), + open_circuit_futs: Default::default(), circuit_deny_futs: Default::default(), send_error_futs: Default::default(), keep_alive: KeepAlive::Yes, } } - fn on_fully_negotiated_inbound( - &mut self, - FullyNegotiatedInbound { - protocol: inbound_circuit, - .. - }: FullyNegotiatedInbound< - ::InboundProtocol, - ::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::ConnectionState::new_inbound(inbound_circuit, tx); - - pending_msgs.push_back(transport::ToListenerMsg::IncomingRelayedConnection { - // stream: connection, - stream: super::Connection { state: connection }, - src_peer_id, - relay_peer_id: self.remote_peer_id, - relay_addr: self.remote_addr.clone(), - }); - - self.queued_events - .push_back(ConnectionHandlerEvent::NotifyBehaviour( - Event::InboundCircuitEstablished { src_peer_id, limit }, - )); - } - Reservation::None => { - let src_peer_id = inbound_circuit.src_peer_id(); - - if self.circuit_deny_futs.len() == MAX_NUMBER_DENYING_CIRCUIT - && !self.circuit_deny_futs.contains_key(&src_peer_id) - { - log::warn!( - "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", - src_peer_id, - ); - } else if self - .circuit_deny_futs - .insert( - src_peer_id, - inbound_circuit.deny(proto::Status::NO_RESERVATION).boxed(), - ) - .is_some() - { - log::warn!( - "Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", - src_peer_id - ) - } - } - } - } - fn on_fully_negotiated_outbound( &mut self, FullyNegotiatedOutbound { @@ -290,19 +238,6 @@ impl Handler { } } - fn on_listen_upgrade_error( - &mut self, - ListenUpgradeError { - error: inbound_stop::UpgradeError::Fatal(error), - .. - }: ListenUpgradeError< - ::InboundOpenInfo, - ::InboundProtocol, - >, - ) { - self.pending_error = Some(StreamUpgradeError::Apply(Either::Left(error))); - } - fn on_dial_upgrade_error( &mut self, DialUpgradeError { @@ -394,6 +329,28 @@ impl Handler { } } } + + fn insert_to_deny_futs(&mut self, circuit: Circuit) { + let src_peer_id = circuit.src_peer_id(); + + if self.circuit_deny_futs.len() == MAX_NUMBER_DENYING_CIRCUIT + && !self.circuit_deny_futs.contains_key(&src_peer_id) + { + log::warn!( + "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", + src_peer_id + ); + } else if self + .circuit_deny_futs + .insert( + src_peer_id, + circuit.deny(proto::Status::NO_RESERVATION).boxed(), + ) + .is_some() + { + log::warn!("Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", src_peer_id); + } + } } impl ConnectionHandler for Handler { @@ -402,13 +359,13 @@ impl ConnectionHandler for Handler { type Error = StreamUpgradeError< Either, >; - type InboundProtocol = inbound_stop::Upgrade; + type InboundProtocol = ReadyUpgrade; + type InboundOpenInfo = (); type OutboundProtocol = outbound_hop::Upgrade; type OutboundOpenInfo = OutboundOpenInfo; - type InboundOpenInfo = (); fn listen_protocol(&self) -> SubstreamProtocol { - SubstreamProtocol::new(inbound_stop::Upgrade {}, ()) + SubstreamProtocol::new(ReadyUpgrade::new(STOP_PROTOCOL_NAME), ()) } fn on_behaviour_event(&mut self, event: Self::FromBehaviour) { @@ -463,6 +420,42 @@ impl ConnectionHandler for Handler { return Poll::Ready(event); } + if let Poll::Ready(Some(circuit_res)) = self.open_circuit_futs.poll_next_unpin(cx) { + match circuit_res { + Ok(circuit) => match &mut self.reservation { + Reservation::Accepted { pending_msgs, .. } + | Reservation::Renewing { pending_msgs, .. } => { + let src_peer_id = circuit.src_peer_id(); + let limit = circuit.limit(); + + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); + let connection = super::ConnectionState::new_inbound(circuit, tx); + + pending_msgs.push_back( + transport::ToListenerMsg::IncomingRelayedConnection { + stream: super::Connection { state: connection }, + src_peer_id, + relay_peer_id: self.remote_peer_id, + relay_addr: self.remote_addr.clone(), + }, + ); + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::InboundCircuitEstablished { src_peer_id, limit }, + )); + } + Reservation::None => { + self.insert_to_deny_futs(circuit); + } + }, + Err(e) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Left(e), + ))); + } + } + } + if let Poll::Ready(Some(protocol)) = self.reservation.poll(cx) { return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol }); } @@ -533,19 +526,20 @@ impl ConnectionHandler for Handler { >, ) { match event { - ConnectionEvent::FullyNegotiatedInbound(fully_negotiated_inbound) => { - self.on_fully_negotiated_inbound(fully_negotiated_inbound) + ConnectionEvent::FullyNegotiatedInbound(FullyNegotiatedInbound { + protocol: stream, + .. + }) => { + self.open_circuit_futs.push(open_circuit(stream).boxed()); } ConnectionEvent::FullyNegotiatedOutbound(fully_negotiated_outbound) => { self.on_fully_negotiated_outbound(fully_negotiated_outbound) } - ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => { - self.on_listen_upgrade_error(listen_upgrade_error) - } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) } ConnectionEvent::AddressChange(_) + | ConnectionEvent::ListenUpgradeError(_) | ConnectionEvent::LocalProtocolsChange(_) | ConnectionEvent::RemoteProtocolsChange(_) => {} } diff --git a/protocols/relay/src/protocol.rs b/protocols/relay/src/protocol.rs index 4376f64cc0b..95debd464d0 100644 --- a/protocols/relay/src/protocol.rs +++ b/protocols/relay/src/protocol.rs @@ -31,7 +31,7 @@ pub const HOP_PROTOCOL_NAME: StreamProtocol = pub const STOP_PROTOCOL_NAME: StreamProtocol = StreamProtocol::new("/libp2p/circuit/relay/0.2.0/stop"); -const MAX_MESSAGE_SIZE: usize = 4096; +pub(crate) const MAX_MESSAGE_SIZE: usize = 4096; #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub struct Limit { diff --git a/protocols/relay/src/protocol/inbound_stop.rs b/protocols/relay/src/protocol/inbound_stop.rs index c279c8ee601..a3c74dbd7c8 100644 --- a/protocols/relay/src/protocol/inbound_stop.rs +++ b/protocols/relay/src/protocol/inbound_stop.rs @@ -19,66 +19,38 @@ // DEALINGS IN THE SOFTWARE. use crate::proto; -use crate::protocol::{self, MAX_MESSAGE_SIZE, STOP_PROTOCOL_NAME}; +use crate::protocol::{self, MAX_MESSAGE_SIZE}; use asynchronous_codec::{Framed, FramedParts}; use bytes::Bytes; -use futures::{future::BoxFuture, prelude::*}; -use libp2p_core::upgrade; +use futures::prelude::*; use libp2p_identity::PeerId; -use libp2p_swarm::{Stream, StreamProtocol}; -use std::iter; +use libp2p_swarm::Stream; use thiserror::Error; -pub struct Upgrade {} - -impl upgrade::UpgradeInfo for Upgrade { - type Info = StreamProtocol; - type InfoIter = iter::Once; - - fn protocol_info(&self) -> Self::InfoIter { - iter::once(STOP_PROTOCOL_NAME) - } -} - -impl upgrade::InboundUpgrade for Upgrade { - type Output = Circuit; - type Error = UpgradeError; - type Future = BoxFuture<'static, Result>; - - fn upgrade_inbound(self, substream: Stream, _: Self::Info) -> Self::Future { - let mut substream = Framed::new( - substream, - quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE), - ); - - async move { - let proto::StopMessage { - type_pb, - peer, - limit, - status: _, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; - - match type_pb { - proto::StopMessageType::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), - }) - } - proto::StopMessageType::STATUS => { - Err(FatalUpgradeError::UnexpectedTypeStatus.into()) - } - } +pub(crate) async fn open_circuit(io: Stream) -> Result { + let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + let proto::StopMessage { + type_pb, + peer, + limit, + status: _, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::StopMessageType::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), + }) } - .boxed() + proto::StopMessageType::STATUS => Err(FatalUpgradeError::UnexpectedTypeStatus), } } @@ -110,22 +82,22 @@ pub enum FatalUpgradeError { UnexpectedTypeStatus, } -pub struct Circuit { +pub(crate) struct Circuit { substream: Framed>, src_peer_id: PeerId, limit: Option, } impl Circuit { - pub fn src_peer_id(&self) -> PeerId { + pub(crate) fn src_peer_id(&self) -> PeerId { self.src_peer_id } - pub fn limit(&self) -> Option { + pub(crate) fn limit(&self) -> Option { self.limit } - pub async fn accept(mut self) -> Result<(Stream, Bytes), UpgradeError> { + pub(crate) async fn accept(mut self) -> Result<(Stream, Bytes), UpgradeError> { let msg = proto::StopMessage { type_pb: proto::StopMessageType::STATUS, peer: None, @@ -149,7 +121,7 @@ impl Circuit { Ok((io, read_buffer.freeze())) } - pub async fn deny(mut self, status: proto::Status) -> Result<(), UpgradeError> { + pub(crate) async fn deny(mut self, status: proto::Status) -> Result<(), UpgradeError> { let msg = proto::StopMessage { type_pb: proto::StopMessageType::STATUS, peer: None, diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 836468a8605..82728464403 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -18,112 +18,12 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::proto; -use crate::protocol::{MAX_MESSAGE_SIZE, STOP_PROTOCOL_NAME}; -use asynchronous_codec::{Framed, FramedParts}; -use bytes::Bytes; -use futures::{future::BoxFuture, prelude::*}; -use libp2p_core::upgrade; -use libp2p_identity::PeerId; -use libp2p_swarm::{Stream, StreamProtocol}; -use std::convert::TryInto; -use std::iter; -use std::time::Duration; use thiserror::Error; -pub struct Upgrade { - pub src_peer_id: PeerId, - pub max_circuit_duration: Duration, - pub max_circuit_bytes: u64, -} - -impl upgrade::UpgradeInfo for Upgrade { - type Info = StreamProtocol; - type InfoIter = iter::Once; - - fn protocol_info(&self) -> Self::InfoIter { - iter::once(STOP_PROTOCOL_NAME) - } -} - -impl upgrade::OutboundUpgrade for Upgrade { - type Output = (Stream, Bytes); - type Error = UpgradeError; - type Future = BoxFuture<'static, Result>; - - fn upgrade_outbound(self, substream: Stream, _: Self::Info) -> Self::Future { - let msg = proto::StopMessage { - type_pb: proto::StopMessageType::CONNECT, - peer: Some(proto::Peer { - id: self.src_peer_id.to_bytes(), - addrs: vec![], - }), - limit: Some(proto::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 substream = Framed::new( - substream, - quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE), - ); - - async move { - substream.send(msg).await?; - let proto::StopMessage { - type_pb, - peer: _, - limit: _, - status, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; - - match type_pb { - proto::StopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect.into()) - } - proto::StopMessageType::STATUS => {} - } - - match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { - proto::Status::OK => {} - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(CircuitFailedReason::ResourceLimitExceeded.into()) - } - proto::Status::PERMISSION_DENIED => { - return Err(CircuitFailedReason::PermissionDenied.into()) - } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), - } - - 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() - } -} +use crate::proto; #[derive(Debug, Error)] -pub enum UpgradeError { +pub(crate) enum UpgradeError { #[error("Circuit failed")] CircuitFailed(#[from] CircuitFailedReason), #[error("Fatal")] From 1b39aaf480eeae4799143e7c3a53d9f23a31aa4a Mon Sep 17 00:00:00 2001 From: dgarus Date: Tue, 1 Aug 2023 15:56:08 +0300 Subject: [PATCH 02/83] fix review comment --- protocols/relay/src/behaviour/handler.rs | 82 ++++++++++-------------- 1 file changed, 33 insertions(+), 49 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 08cc9d69d83..09579dbc038 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -388,9 +388,7 @@ pub struct Handler { circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, stop_requested_streams: VecDeque, - outbound_stop_futs: FuturesUnordered< - BoxFuture<'static, Result<(Event, Option>), FatalUpgradeError>>, - >, + outbound_stop_futs: FuturesUnordered>>, } impl Handler { @@ -416,7 +414,7 @@ impl Handler { &self, io: Stream, stop_command: StopCommand, - ) -> BoxFuture<'static, Result<(Event, Option>), FatalUpgradeError>> { + ) -> BoxFuture<'static, Result> { let msg = proto::StopMessage { type_pb: proto::StopMessageType::CONNECT, peer: Some(proto::Peer { @@ -436,6 +434,9 @@ impl Handler { status: None, }; + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); + let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); async move { substream.send(msg).await?; @@ -461,34 +462,26 @@ impl Handler { Some(proto_status) => match proto_status { Status::OK => {} Status::RESOURCE_LIMIT_EXCEEDED => { - return Ok(( - Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply( - CircuitFailedReason::ResourceLimitExceeded, - ), - }, - None, - )) + return Ok(Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply( + CircuitFailedReason::ResourceLimitExceeded, + ), + }) } Status::PERMISSION_DENIED => { - return Ok(( - Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply( - CircuitFailedReason::PermissionDenied, - ), - }, - None, - )) + return Ok(Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), + }) } s => return Err(FatalUpgradeError::UnexpectedStatus(s)), }, @@ -508,19 +501,15 @@ impl Handler { "Expect a flushed Framed to have an empty write buffer." ); - let (tx, rx) = oneshot::channel(); - Ok(( - Event::OutboundConnectNegotiated { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - dst_handler_notifier: tx, - dst_stream: io, - dst_pending_data: read_buffer.freeze(), - }, - Some(rx), - )) + Ok(Event::OutboundConnectNegotiated { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + dst_handler_notifier: tx, + dst_stream: io, + dst_pending_data: read_buffer.freeze(), + }) } .boxed() } @@ -744,12 +733,7 @@ impl ConnectionHandler for Handler { // Send stop commands if let Poll::Ready(Some(result)) = self.outbound_stop_futs.poll_next_unpin(cx) { return match result { - Ok((event, receiver)) => { - if let Some(rx) = receiver { - self.alive_lend_out_substreams.push(rx); - } - Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)) - } + Ok(event) => Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)), Err(e) => Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( Either::Right(e), ))), From de33b72f66bbbf393022069135eabb851966a4bb Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 2 Aug 2023 16:20:34 +0300 Subject: [PATCH 03/83] Refactoring of hop protocol --- protocols/relay/src/behaviour.rs | 4 +- protocols/relay/src/behaviour/handler.rs | 337 +++++++---- protocols/relay/src/priv_client/handler.rs | 557 +++++++++++++------ protocols/relay/src/protocol/inbound_hop.rs | 195 ------- protocols/relay/src/protocol/outbound_hop.rs | 187 +------ 5 files changed, 625 insertions(+), 655 deletions(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 8fcfa103c4f..daebcadf1d9 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -22,7 +22,7 @@ mod handler; pub(crate) mod rate_limiter; -use crate::behaviour::handler::Handler; +use crate::behaviour::handler::{Handler, ReservationReq}; use crate::multiaddr_ext::MultiaddrExt; use crate::proto; use crate::protocol::{inbound_hop, outbound_stop}; @@ -811,7 +811,7 @@ impl Add for CircuitId { enum Action { Done(ToSwarm>), AcceptReservationPrototype { - inbound_reservation_req: inbound_hop::ReservationReq, + inbound_reservation_req: ReservationReq, handler: NotifyHandler, peer_id: PeerId, }, diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 09579dbc038..ebb19b6ddee 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -18,14 +18,11 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::behaviour::CircuitId; -use crate::copy_future::CopyFuture; -use crate::proto::Status; -use crate::protocol::inbound_hop::CircuitReq; -use crate::protocol::outbound_stop::{CircuitFailedReason, FatalUpgradeError}; -use crate::protocol::MAX_MESSAGE_SIZE; -use crate::protocol::{inbound_hop, outbound_stop}; -use crate::{proto, STOP_PROTOCOL_NAME}; +use std::collections::VecDeque; +use std::fmt; +use std::task::{Context, Poll}; +use std::time::{Duration, SystemTime}; + use asynchronous_codec::{Framed, FramedParts}; use bytes::Bytes; use either::Either; @@ -36,20 +33,24 @@ use futures::prelude::*; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; + use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; -use libp2p_swarm::handler::{ - ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound, ListenUpgradeError, -}; +use libp2p_swarm::handler::{ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound}; use libp2p_swarm::{ ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, StreamUpgradeError, SubstreamProtocol, }; -use std::collections::VecDeque; -use std::fmt; -use std::task::{Context, Poll}; -use std::time::Duration; + +use crate::behaviour::CircuitId; +use crate::copy_future::CopyFuture; +use crate::proto::message_v2::pb::mod_HopMessage::Type; +use crate::proto::Status; +use crate::protocol::outbound_stop::{CircuitFailedReason, FatalUpgradeError}; +use crate::protocol::MAX_MESSAGE_SIZE; +use crate::protocol::{inbound_hop, outbound_stop}; +use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; #[derive(Debug, Clone)] pub struct Config { @@ -60,28 +61,28 @@ pub struct Config { pub enum In { AcceptReservationReq { - inbound_reservation_req: inbound_hop::ReservationReq, + inbound_reservation_req: ReservationReq, addrs: Vec, }, DenyReservationReq { - inbound_reservation_req: inbound_hop::ReservationReq, + inbound_reservation_req: ReservationReq, status: Status, }, DenyCircuitReq { circuit_id: Option, - inbound_circuit_req: inbound_hop::CircuitReq, + inbound_circuit_req: CircuitReq, status: Status, }, NegotiateOutboundConnect { circuit_id: CircuitId, - inbound_circuit_req: inbound_hop::CircuitReq, + inbound_circuit_req: CircuitReq, src_peer_id: PeerId, src_connection_id: ConnectionId, }, AcceptAndDriveCircuit { circuit_id: CircuitId, dst_peer_id: PeerId, - inbound_circuit_req: inbound_hop::CircuitReq, + inbound_circuit_req: CircuitReq, dst_handler_notifier: oneshot::Sender<()>, dst_stream: Stream, dst_pending_data: Bytes, @@ -146,7 +147,7 @@ impl fmt::Debug for In { pub enum Event { /// An inbound reservation request has been received. ReservationReqReceived { - inbound_reservation_req: inbound_hop::ReservationReq, + inbound_reservation_req: ReservationReq, endpoint: ConnectedPoint, /// Indicates whether the request replaces an existing reservation. renewed: bool, @@ -166,7 +167,7 @@ pub enum Event { ReservationTimedOut {}, /// An inbound circuit request has been received. CircuitReqReceived { - inbound_circuit_req: inbound_hop::CircuitReq, + inbound_circuit_req: CircuitReq, endpoint: ConnectedPoint, }, /// An inbound circuit request has been denied. @@ -180,7 +181,7 @@ pub enum Event { dst_peer_id: PeerId, error: inbound_hop::UpgradeError, }, - /// An inbound cirucit request has been accepted. + /// An inbound circuit request has been accepted. CircuitReqAccepted { circuit_id: CircuitId, dst_peer_id: PeerId, @@ -197,7 +198,7 @@ pub enum Event { circuit_id: CircuitId, src_peer_id: PeerId, src_connection_id: ConnectionId, - inbound_circuit_req: inbound_hop::CircuitReq, + inbound_circuit_req: CircuitReq, dst_handler_notifier: oneshot::Sender<()>, dst_stream: Stream, dst_pending_data: Bytes, @@ -207,9 +208,9 @@ pub enum Event { circuit_id: CircuitId, src_peer_id: PeerId, src_connection_id: ConnectionId, - inbound_circuit_req: inbound_hop::CircuitReq, - status: proto::Status, - error: StreamUpgradeError, + inbound_circuit_req: CircuitReq, + status: Status, + error: StreamUpgradeError, }, /// An inbound circuit has closed. CircuitClosed { @@ -369,7 +370,7 @@ pub struct Handler { /// Futures accepting an inbound circuit request. circuit_accept_futures: Futures>, - /// Futures deying an inbound circuit request. + /// Futures denying an inbound circuit request. circuit_deny_futures: Futures<( Option, PeerId, @@ -387,6 +388,8 @@ pub struct Handler { /// Futures relaying data for circuit between two peers. circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, + inbound_requests_futs: + FuturesUnordered>>, stop_requested_streams: VecDeque, outbound_stop_futs: FuturesUnordered>>, } @@ -405,11 +408,71 @@ impl Handler { circuits: Default::default(), active_reservation: Default::default(), keep_alive: KeepAlive::Yes, + inbound_requests_futs: Default::default(), stop_requested_streams: Default::default(), outbound_stop_futs: Default::default(), } } + fn process_inbound_request( + &self, + io: Stream, + ) -> BoxFuture<'static, Result> { + let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + let reservation_duration = self.config.reservation_duration; + let max_circuit_duration = self.config.max_circuit_duration; + let max_circuit_bytes = self.config.max_circuit_bytes; + let endpoint = self.endpoint.clone(); + let renewed = self.active_reservation.is_some(); + + async move { + let proto::HopMessage { + type_pb, + peer, + reservation: _, + limit: _, + status: _, + } = substream + .next() + .await + .ok_or(inbound_hop::FatalUpgradeError::StreamClosed)??; + + let event = match type_pb { + Type::RESERVE => { + let req = ReservationReq { + substream, + reservation_duration, + max_circuit_duration, + max_circuit_bytes, + }; + + Event::ReservationReqReceived { + inbound_reservation_req: req, + endpoint, + renewed, + } + } + Type::CONNECT => { + let dst = PeerId::from_bytes( + &peer.ok_or(inbound_hop::FatalUpgradeError::MissingPeer)?.id, + ) + .map_err(|_| inbound_hop::FatalUpgradeError::ParsePeerId)?; + let req = CircuitReq { dst, substream }; + + Event::CircuitReqReceived { + inbound_circuit_req: req, + endpoint, + } + } + Type::STATUS => return Err(inbound_hop::FatalUpgradeError::UnexpectedTypeStatus), + }; + + Ok(event) + } + .boxed() + } + fn send_stop_message_and_process_result( &self, io: Stream, @@ -513,51 +576,6 @@ impl Handler { } .boxed() } - - fn on_fully_negotiated_inbound( - &mut self, - FullyNegotiatedInbound { - protocol: request, .. - }: FullyNegotiatedInbound< - ::InboundProtocol, - ::InboundOpenInfo, - >, - ) { - match request { - inbound_hop::Req::Reserve(inbound_reservation_req) => { - self.queued_events - .push_back(ConnectionHandlerEvent::NotifyBehaviour( - 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(ConnectionHandlerEvent::NotifyBehaviour( - Event::CircuitReqReceived { - inbound_circuit_req, - endpoint: self.endpoint.clone(), - }, - )); - } - } - } - - fn on_listen_upgrade_error( - &mut self, - ListenUpgradeError { - error: inbound_hop::UpgradeError::Fatal(error), - .. - }: ListenUpgradeError< - ::InboundOpenInfo, - ::InboundProtocol, - >, - ) { - self.pending_error = Some(StreamUpgradeError::Apply(Either::Left(error))); - } } enum ReservationRequestFuture { @@ -570,23 +588,14 @@ type Futures = FuturesUnordered>; impl ConnectionHandler for Handler { type FromBehaviour = In; type ToBehaviour = Event; - type Error = StreamUpgradeError< - Either, - >; - type InboundProtocol = inbound_hop::Upgrade; + type Error = StreamUpgradeError>; + type InboundProtocol = ReadyUpgrade; type InboundOpenInfo = (); type OutboundProtocol = ReadyUpgrade; type OutboundOpenInfo = (); fn listen_protocol(&self) -> SubstreamProtocol { - 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, - }, - (), - ) + SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()) } fn on_behaviour_event(&mut self, event: Self::FromBehaviour) { @@ -730,14 +739,28 @@ impl ConnectionHandler for Handler { } } + // Process inbound requests + if let Poll::Ready(Some(result)) = self.inbound_requests_futs.poll_next_unpin(cx) { + let event = match result { + Ok(event) => ConnectionHandlerEvent::NotifyBehaviour(event), + Err(err) => { + ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left(err))) + } + }; + + return Poll::Ready(event); + } + // Send stop commands if let Poll::Ready(Some(result)) = self.outbound_stop_futs.poll_next_unpin(cx) { - return match result { - Ok(event) => Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)), - Err(e) => Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(e), - ))), + let event = match result { + Ok(event) => ConnectionHandlerEvent::NotifyBehaviour(event), + Err(e) => { + ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right(e))) + } }; + + return Poll::Ready(event); } // Deny new circuits. @@ -920,8 +943,12 @@ impl ConnectionHandler for Handler { >, ) { match event { - ConnectionEvent::FullyNegotiatedInbound(fully_negotiated_inbound) => { - self.on_fully_negotiated_inbound(fully_negotiated_inbound) + ConnectionEvent::FullyNegotiatedInbound(FullyNegotiatedInbound { + protocol: stream, + .. + }) => { + self.inbound_requests_futs + .push(self.process_inbound_request(stream)); } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, @@ -934,10 +961,8 @@ impl ConnectionHandler for Handler { self.outbound_stop_futs .push(self.send_stop_message_and_process_result(stream, stop_command)); } - ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => { - self.on_listen_upgrade_error(listen_upgrade_error) - } ConnectionEvent::AddressChange(_) + | ConnectionEvent::ListenUpgradeError(_) | ConnectionEvent::DialUpgradeError(_) | ConnectionEvent::LocalProtocolsChange(_) | ConnectionEvent::RemoteProtocolsChange(_) => {} @@ -945,7 +970,7 @@ impl ConnectionHandler for Handler { } } -pub(crate) struct CircuitParts { +struct CircuitParts { circuit_id: CircuitId, src_stream: Stream, src_pending_data: Bytes, @@ -955,7 +980,7 @@ pub(crate) struct CircuitParts { dst_pending_data: Bytes, } -pub(crate) struct StopCommand { +struct StopCommand { circuit_id: CircuitId, inbound_circuit_req: CircuitReq, src_peer_id: PeerId, @@ -963,3 +988,121 @@ pub(crate) struct StopCommand { max_circuit_duration: Duration, max_circuit_bytes: u64, } + +pub struct ReservationReq { + substream: Framed>, + reservation_duration: Duration, + max_circuit_duration: Duration, + max_circuit_bytes: u64, +} + +impl ReservationReq { + pub async fn accept(self, addrs: Vec) -> Result<(), inbound_hop::UpgradeError> { + if addrs.is_empty() { + log::debug!( + "Accepting relay reservation without providing external addresses of local node. \ + Thus the remote node might not be able to advertise its relayed address." + ) + } + + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: Some(proto::Reservation { + addrs: addrs.into_iter().map(|a| a.to_vec()).collect(), + expire: (SystemTime::now() + self.reservation_duration) + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs(), + voucher: None, + }), + limit: Some(proto::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(proto::Status::OK), + }; + + self.send(msg).await + } + + pub async fn deny(self, status: Status) -> Result<(), inbound_hop::UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: None, + limit: None, + status: Some(status), + }; + + self.send(msg).await + } + + async fn send(mut self, msg: proto::HopMessage) -> Result<(), inbound_hop::UpgradeError> { + self.substream.send(msg).await?; + self.substream.flush().await?; + self.substream.close().await?; + + Ok(()) + } +} + +pub struct CircuitReq { + dst: PeerId, + substream: Framed>, +} + +impl CircuitReq { + pub fn dst(&self) -> PeerId { + self.dst + } + + pub async fn accept(mut self) -> Result<(Stream, Bytes), inbound_hop::UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: None, + limit: None, + status: Some(Status::OK), + }; + + 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<(), inbound_hop::UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: None, + limit: None, + status: Some(status), + }; + self.send(msg).await?; + self.substream.close().await.map_err(Into::into) + } + + async fn send(&mut self, msg: proto::HopMessage) -> Result<(), quick_protobuf_codec::Error> { + self.substream.send(msg).await?; + self.substream.flush().await?; + + Ok(()) + } +} diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 0932b95aade..d156e9db775 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -21,8 +21,9 @@ use std::collections::{HashMap, VecDeque}; use std::fmt; use std::task::{Context, Poll}; -use std::time::Duration; +use std::time::{Duration, SystemTime}; +use asynchronous_codec::{Framed, FramedParts}; use either::Either; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; @@ -40,14 +41,15 @@ use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, }; use libp2p_swarm::{ - ConnectionHandler, ConnectionHandlerEvent, KeepAlive, StreamProtocol, StreamUpgradeError, - SubstreamProtocol, + ConnectionHandler, ConnectionHandlerEvent, KeepAlive, Stream, StreamProtocol, + StreamUpgradeError, SubstreamProtocol, }; use crate::priv_client::transport; use crate::protocol::inbound_stop::{open_circuit, Circuit}; -use crate::protocol::{self, inbound_stop, outbound_hop}; -use crate::{proto, STOP_PROTOCOL_NAME}; +use crate::protocol::outbound_hop::{FatalUpgradeError, Output, UpgradeError}; +use crate::protocol::{self, inbound_stop, outbound_hop, MAX_MESSAGE_SIZE}; +use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; /// The maximum number of circuits being denied concurrently. /// @@ -115,11 +117,8 @@ pub struct Handler { remote_peer_id: PeerId, remote_addr: Multiaddr, /// A pending fatal error that results in the connection being closed. - pending_error: Option< - StreamUpgradeError< - Either, - >, - >, + pending_error: + Option>>, /// Until when to keep the connection alive. keep_alive: KeepAlive, @@ -133,6 +132,13 @@ pub struct Handler { >, >, + wait_for_reserve_outbound_stream: VecDeque>, + reserve_futs: FuturesUnordered>>, + + wait_for_connection_outbound_stream: VecDeque, + circuit_connection_futs: + FuturesUnordered, UpgradeError>>>, + reservation: Reservation, /// Tracks substreams lent out to the transport. @@ -165,6 +171,10 @@ impl Handler { remote_addr, queued_events: Default::default(), pending_error: Default::default(), + wait_for_reserve_outbound_stream: Default::default(), + reserve_futs: Default::default(), + wait_for_connection_outbound_stream: Default::default(), + circuit_connection_futs: Default::default(), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), open_circuit_futs: Default::default(), @@ -174,160 +184,74 @@ impl Handler { } } - fn on_fully_negotiated_outbound( + fn on_dial_upgrade_error( &mut self, - FullyNegotiatedOutbound { - protocol: output, - info, - }: FullyNegotiatedOutbound< - ::OutboundProtocol, + DialUpgradeError { error, .. }: DialUpgradeError< ::OutboundOpenInfo, + ::OutboundProtocol, >, ) { - 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(ConnectionHandlerEvent::NotifyBehaviour(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::Connection { - state: super::ConnectionState::new_outbound(substream, read_buffer, tx), - })) { - Ok(()) => { - self.alive_lend_out_substreams.push(rx); - self.queued_events - .push_back(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitEstablished { limit }, - )); - } - Err(_) => debug!( - "Oneshot to `client::transport::Dial` future dropped. \ - Dropping established relayed connection to {:?}.", - self.remote_peer_id, - ), + // Try to process the error for a reservation + if let Some(mut to_listener) = self.wait_for_reserve_outbound_stream.pop_front() { + let non_fatal_error = match error { + StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, + StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, + StreamUpgradeError::Io(e) => { + self.pending_error = Some(StreamUpgradeError::Io(e)); + return; } + StreamUpgradeError::Apply(_) => unreachable!("should not update"), + }; + + 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 occurred, thus handler is closing as quickly as possible. + // Transport is notified through dropping `to_listener`. } - _ => unreachable!(), - } - } + let renewal = self.reservation.failed(); - fn on_dial_upgrade_error( - &mut self, - DialUpgradeError { - info: open_info, - error, - }: DialUpgradeError< - ::OutboundOpenInfo, - ::OutboundProtocol, - >, - ) { - match open_info { - OutboundOpenInfo::Reserve { mut to_listener } => { - let non_fatal_error = match error { - StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, - StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, - StreamUpgradeError::Io(e) => { - self.pending_error = Some(StreamUpgradeError::Io(e)); - return; - } - StreamUpgradeError::Apply(error) => match error { - outbound_hop::UpgradeError::Fatal(error) => { - self.pending_error = - Some(StreamUpgradeError::Apply(Either::Right(error))); - return; - } - outbound_hop::UpgradeError::ReservationFailed(error) => { - StreamUpgradeError::Apply(error) - } - outbound_hop::UpgradeError::CircuitFailed(_) => { - unreachable!("Do not emitt `CircuitFailed` for outgoing reservation.") - } + self.queued_events + .push_back(ConnectionHandlerEvent::NotifyBehaviour( + Event::ReservationReqFailed { + renewal, + error: non_fatal_error, }, - }; - - 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(ConnectionHandlerEvent::NotifyBehaviour( - Event::ReservationReqFailed { - renewal, - error: non_fatal_error, - }, - )); + return; + } + // Try to process the error for a connection + let cmd = self.wait_for_connection_outbound_stream.pop_front().expect( + "got a stream error without a pending connection command or a reserve listener", + ); + + let non_fatal_error = match error { + StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, + StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, + StreamUpgradeError::Io(e) => { + self.pending_error = Some(StreamUpgradeError::Io(e)); + return; } - OutboundOpenInfo::Connect { send_back } => { - let non_fatal_error = match error { - StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, - StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, - StreamUpgradeError::Io(e) => { - self.pending_error = Some(StreamUpgradeError::Io(e)); - return; - } - StreamUpgradeError::Apply(error) => match error { - outbound_hop::UpgradeError::Fatal(error) => { - self.pending_error = - Some(StreamUpgradeError::Apply(Either::Right(error))); - return; - } - outbound_hop::UpgradeError::CircuitFailed(error) => { - StreamUpgradeError::Apply(error) - } - outbound_hop::UpgradeError::ReservationFailed(_) => { - unreachable!("Do not emitt `ReservationFailed` for outgoing circuit.") - } - }, - }; + StreamUpgradeError::Apply(_) => unreachable!("should not update"), + }; - let _ = send_back.send(Err(())); + let _ = cmd.send_back.send(Err(())); - self.queued_events - .push_back(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: non_fatal_error, - }, - )); - } - } + self.queued_events + .push_back(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: non_fatal_error, + }, + )); } fn insert_to_deny_futs(&mut self, circuit: Circuit) { @@ -351,18 +275,205 @@ impl Handler { log::warn!("Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", src_peer_id); } } + + fn send_reserve_message_and_process_response( + &self, + protocol: Stream, + to_listener: mpsc::Sender, + ) -> BoxFuture<'static, Result> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::RESERVE, + peer: None, + reservation: None, + limit: None, + status: None, + }; + let mut substream = + Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + async move { + substream.send(msg).await?; + + let proto::HopMessage { + type_pb, + peer: _, + reservation, + limit, + status, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::HopMessageType::CONNECT => { + return Err(FatalUpgradeError::UnexpectedTypeConnect.into()); + } + proto::HopMessageType::RESERVE => { + return Err(FatalUpgradeError::UnexpectedTypeReserve.into()); + } + proto::HopMessageType::STATUS => {} + } + + let limit = limit.map(Into::into); + + match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + proto::Status::OK => {} + proto::Status::RESERVATION_REFUSED => { + return Err(outbound_hop::ReservationFailedReason::Refused.into()); + } + proto::Status::RESOURCE_LIMIT_EXCEEDED => { + return Err(outbound_hop::ReservationFailedReason::ResourceLimitExceeded.into()); + } + s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + } + + let reservation = reservation.ok_or(FatalUpgradeError::MissingReservationField)?; + + if reservation.addrs.is_empty() { + return Err(FatalUpgradeError::NoAddressesInReservation.into()); + } + + let addrs = reservation + .addrs + .into_iter() + .map(|b| Multiaddr::try_from(b.to_vec())) + .collect::, _>>() + .map_err(|_| FatalUpgradeError::InvalidReservationAddrs)?; + + let renewal_timeout = reservation + .expire + .checked_sub( + SystemTime::now() + .duration_since(SystemTime::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?; + + let output = Output::Reservation { + renewal_timeout, + addrs, + limit, + to_listener, + }; + + Ok(output) + } + .boxed() + } + + fn send_connection_message_and_process_response( + &self, + protocol: Stream, + con_command: ConnectionCommand, + ) -> BoxFuture<'static, Result, UpgradeError>> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::CONNECT, + peer: Some(proto::Peer { + id: con_command.dst_peer_id.to_bytes(), + addrs: vec![], + }), + reservation: None, + limit: None, + status: None, + }; + + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); + + let mut substream = + Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + let remote_peer_id = self.remote_peer_id; + + async move { + substream.send(msg).await?; + let proto::HopMessage { + type_pb, + peer: _, + reservation: _, + limit, + status, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::HopMessageType::CONNECT => { + return Err(FatalUpgradeError::UnexpectedTypeConnect.into()) + } + proto::HopMessageType::RESERVE => { + return Err(FatalUpgradeError::UnexpectedTypeReserve.into()) + } + proto::HopMessageType::STATUS => {} + } + + let limit = limit.map(Into::into); + + match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + proto::Status::OK => {} + proto::Status::RESOURCE_LIMIT_EXCEEDED => { + return Err(outbound_hop::CircuitFailedReason::ResourceLimitExceeded.into()) + } + proto::Status::CONNECTION_FAILED => { + return Err(outbound_hop::CircuitFailedReason::ConnectionFailed.into()) + } + proto::Status::NO_RESERVATION => { + return Err(outbound_hop::CircuitFailedReason::NoReservation.into()) + } + proto::Status::PERMISSION_DENIED => { + return Err(outbound_hop::CircuitFailedReason::PermissionDenied.into()) + } + s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + } + + let FramedParts { + io, + read_buffer, + write_buffer, + .. + } = substream.into_parts(); + assert!( + write_buffer.is_empty(), + "Expect a flushed Framed to have empty write buffer." + ); + + let output = match con_command.send_back.send(Ok(super::Connection { + state: super::ConnectionState::new_outbound(io, read_buffer.freeze(), tx), + })) { + Ok(()) => Some(Output::Circuit { limit }), + Err(_) => { + debug!( + "Oneshot to `client::transport::Dial` future dropped. \ + Dropping established relayed connection to {:?}.", + remote_peer_id, + ); + + None + } + }; + + Ok(output) + } + .boxed() + } } impl ConnectionHandler for Handler { type FromBehaviour = In; type ToBehaviour = Event; - type Error = StreamUpgradeError< - Either, - >; + type Error = StreamUpgradeError>; type InboundProtocol = ReadyUpgrade; type InboundOpenInfo = (); - type OutboundProtocol = outbound_hop::Upgrade; - type OutboundOpenInfo = OutboundOpenInfo; + type OutboundProtocol = ReadyUpgrade; + type OutboundOpenInfo = (); fn listen_protocol(&self) -> SubstreamProtocol { SubstreamProtocol::new(ReadyUpgrade::new(STOP_PROTOCOL_NAME), ()) @@ -371,24 +482,24 @@ impl ConnectionHandler for Handler { fn on_behaviour_event(&mut self, event: Self::FromBehaviour) { match event { In::Reserve { to_listener } => { + self.wait_for_reserve_outbound_stream.push_back(to_listener); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { - protocol: SubstreamProtocol::new( - outbound_hop::Upgrade::Reserve, - OutboundOpenInfo::Reserve { to_listener }, - ), + protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), }); } In::EstablishCircuit { send_back, dst_peer_id, } => { + self.wait_for_connection_outbound_stream + .push_back(ConnectionCommand { + dst_peer_id, + send_back, + }); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { - protocol: SubstreamProtocol::new( - outbound_hop::Upgrade::Connect { dst_peer_id }, - OutboundOpenInfo::Connect { send_back }, - ), + protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), }); } } @@ -415,6 +526,78 @@ impl ConnectionHandler for Handler { return Poll::Ready(ConnectionHandlerEvent::Close(err)); } + // Reservations + if let Poll::Ready(Some(result)) = self.reserve_futs.poll_next_unpin(cx) { + let event = match result { + Ok(Output::Reservation { + renewal_timeout, + addrs, + limit, + to_listener, + }) => ConnectionHandlerEvent::NotifyBehaviour(self.reservation.accepted( + renewal_timeout, + addrs, + to_listener, + self.local_peer_id, + limit, + )), + Err(err) => match err { + UpgradeError::ReservationFailed(e) => { + let renewal = self.reservation.failed(); + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::ReservationReqFailed { + renewal, + error: StreamUpgradeError::Apply(e), + }, + )); + } + UpgradeError::Fatal(e) => { + ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right(e))) + } + UpgradeError::CircuitFailed(_) => { + unreachable!("do not emit `CircuitFailed` for reservation") + } + }, + _ => unreachable!("do not emit 'Output::Circuit' for reservation"), + }; + + return Poll::Ready(event); + } + + // Circuit connections + if let Poll::Ready(Some(res)) = self.circuit_connection_futs.poll_next_unpin(cx) { + let opt = match res { + Ok(Some(Output::Circuit { limit })) => { + Some(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitEstablished { limit }, + )) + } + Ok(None) => None, + Err(err) => { + let res = match err { + UpgradeError::CircuitFailed(e) => ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }, + ), + UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( + StreamUpgradeError::Apply(Either::Right(e)), + ), + UpgradeError::ReservationFailed(_) => { + unreachable!("do not emit `ReservationFailed` for connection") + } + }; + + Some(res) + } + _ => unreachable!("do not emit 'Output::Reservation' for connection"), + }; + + if let Some(event) = opt { + return Poll::Ready(event); + } + } + // Return queued events. if let Some(event) = self.queued_events.pop_front() { return Poll::Ready(event); @@ -456,8 +639,12 @@ impl ConnectionHandler for Handler { } } - if let Poll::Ready(Some(protocol)) = self.reservation.poll(cx) { - return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol }); + if let Poll::Ready(Some(to_listener)) = self.reservation.poll(cx) { + self.wait_for_reserve_outbound_stream.push_back(to_listener); + + return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { + protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), + }); } // Deny incoming circuit requests. @@ -532,8 +719,21 @@ impl ConnectionHandler for Handler { }) => { self.open_circuit_futs.push(open_circuit(stream).boxed()); } - ConnectionEvent::FullyNegotiatedOutbound(fully_negotiated_outbound) => { - self.on_fully_negotiated_outbound(fully_negotiated_outbound) + ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { + protocol: stream, + .. + }) => { + if let Some(to_listener) = self.wait_for_reserve_outbound_stream.pop_front() { + self.reserve_futs + .push(self.send_reserve_message_and_process_response(stream, to_listener)); + return; + } + + let con_command = self.wait_for_connection_outbound_stream.pop_front().expect( + "opened a stream without a pending connection command or a reserve listener", + ); + self.circuit_connection_futs + .push(self.send_connection_message_and_process_response(stream, con_command)) } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) @@ -546,6 +746,11 @@ impl ConnectionHandler for Handler { } } +struct ConnectionCommand { + dst_peer_id: PeerId, + send_back: oneshot::Sender>, +} + enum Reservation { /// The Reservation is accepted by the relay. Accepted { @@ -642,7 +847,7 @@ impl Reservation { fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll>> { + ) -> Poll>> { self.forward_messages_to_transport_listener(cx); // Check renewal timeout if any. @@ -654,10 +859,7 @@ impl Reservation { } => 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::Ready(Some(to_listener)), ), Poll::Pending => ( Reservation::Accepted { @@ -675,12 +877,3 @@ impl Reservation { poll_val } } - -pub enum OutboundOpenInfo { - Reserve { - to_listener: mpsc::Sender, - }, - Connect { - send_back: oneshot::Sender>, - }, -} diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index 27f2572a636..754d9bc2b7f 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -18,80 +18,8 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::proto; -use crate::protocol::{HOP_PROTOCOL_NAME, MAX_MESSAGE_SIZE}; -use asynchronous_codec::{Framed, FramedParts}; -use bytes::Bytes; -use futures::{future::BoxFuture, prelude::*}; -use instant::{Duration, SystemTime}; -use libp2p_core::{upgrade, Multiaddr}; -use libp2p_identity::PeerId; -use libp2p_swarm::{Stream, StreamProtocol}; -use std::convert::TryInto; -use std::iter; use thiserror::Error; -pub struct Upgrade { - pub reservation_duration: Duration, - pub max_circuit_duration: Duration, - pub max_circuit_bytes: u64, -} - -impl upgrade::UpgradeInfo for Upgrade { - type Info = StreamProtocol; - type InfoIter = iter::Once; - - fn protocol_info(&self) -> Self::InfoIter { - iter::once(HOP_PROTOCOL_NAME) - } -} - -impl upgrade::InboundUpgrade for Upgrade { - type Output = Req; - type Error = UpgradeError; - type Future = BoxFuture<'static, Result>; - - fn upgrade_inbound(self, substream: Stream, _: Self::Info) -> Self::Future { - let mut substream = Framed::new( - substream, - quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE), - ); - - async move { - let proto::HopMessage { - type_pb, - peer, - reservation: _, - limit: _, - status: _, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; - - let req = match type_pb { - proto::HopMessageType::RESERVE => Req::Reserve(ReservationReq { - substream, - reservation_duration: self.reservation_duration, - max_circuit_duration: self.max_circuit_duration, - max_circuit_bytes: self.max_circuit_bytes, - }), - proto::HopMessageType::CONNECT => { - let dst = PeerId::from_bytes(&peer.ok_or(FatalUpgradeError::MissingPeer)?.id) - .map_err(|_| FatalUpgradeError::ParsePeerId)?; - Req::Connect(CircuitReq { dst, substream }) - } - proto::HopMessageType::STATUS => { - return Err(FatalUpgradeError::UnexpectedTypeStatus.into()) - } - }; - - Ok(req) - } - .boxed() - } -} - #[derive(Debug, Error)] pub enum UpgradeError { #[error("Fatal")] @@ -119,126 +47,3 @@ pub enum FatalUpgradeError { #[error("Unexpected message type 'status'")] UnexpectedTypeStatus, } - -pub enum Req { - Reserve(ReservationReq), - Connect(CircuitReq), -} - -pub struct ReservationReq { - substream: Framed>, - reservation_duration: Duration, - max_circuit_duration: Duration, - max_circuit_bytes: u64, -} - -impl ReservationReq { - pub async fn accept(self, addrs: Vec) -> Result<(), UpgradeError> { - if addrs.is_empty() { - log::debug!( - "Accepting relay reservation without providing external addresses of local node. \ - Thus the remote node might not be able to advertise its relayed address." - ) - } - - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: Some(proto::Reservation { - addrs: addrs.into_iter().map(|a| a.to_vec()).collect(), - expire: (SystemTime::now() + self.reservation_duration) - .duration_since(SystemTime::UNIX_EPOCH) - .unwrap() - .as_secs(), - voucher: None, - }), - limit: Some(proto::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(proto::Status::OK), - }; - - self.send(msg).await - } - - pub async fn deny(self, status: proto::Status) -> Result<(), UpgradeError> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: None, - limit: None, - status: Some(status), - }; - - self.send(msg).await - } - - async fn send(mut self, msg: proto::HopMessage) -> Result<(), UpgradeError> { - self.substream.send(msg).await?; - self.substream.flush().await?; - self.substream.close().await?; - - Ok(()) - } -} - -pub struct CircuitReq { - dst: PeerId, - substream: Framed>, -} - -impl CircuitReq { - pub fn dst(&self) -> PeerId { - self.dst - } - - pub async fn accept(mut self) -> Result<(Stream, Bytes), UpgradeError> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: None, - limit: None, - status: Some(proto::Status::OK), - }; - - 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: proto::Status) -> Result<(), UpgradeError> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: None, - limit: None, - status: Some(status), - }; - self.send(msg).await?; - self.substream.close().await.map_err(Into::into) - } - - async fn send(&mut self, msg: proto::HopMessage) -> Result<(), quick_protobuf_codec::Error> { - self.substream.send(msg).await?; - self.substream.flush().await?; - - Ok(()) - } -} diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index bec348e87db..0e3b89789e6 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -18,188 +18,18 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::proto; -use crate::protocol::{Limit, HOP_PROTOCOL_NAME, MAX_MESSAGE_SIZE}; -use asynchronous_codec::{Framed, FramedParts}; -use bytes::Bytes; -use futures::{future::BoxFuture, prelude::*}; +use futures::channel::mpsc; use futures_timer::Delay; -use instant::{Duration, SystemTime}; -use libp2p_core::{upgrade, Multiaddr}; -use libp2p_identity::PeerId; -use libp2p_swarm::{Stream, StreamProtocol}; -use std::convert::TryFrom; -use std::iter; use thiserror::Error; -pub enum Upgrade { - Reserve, - Connect { dst_peer_id: PeerId }, -} - -impl upgrade::UpgradeInfo for Upgrade { - type Info = StreamProtocol; - type InfoIter = iter::Once; - - fn protocol_info(&self) -> Self::InfoIter { - iter::once(HOP_PROTOCOL_NAME) - } -} - -impl upgrade::OutboundUpgrade for Upgrade { - type Output = Output; - type Error = UpgradeError; - type Future = BoxFuture<'static, Result>; - - fn upgrade_outbound(self, substream: Stream, _: Self::Info) -> Self::Future { - let msg = match self { - Upgrade::Reserve => proto::HopMessage { - type_pb: proto::HopMessageType::RESERVE, - peer: None, - reservation: None, - limit: None, - status: None, - }, - Upgrade::Connect { dst_peer_id } => proto::HopMessage { - type_pb: proto::HopMessageType::CONNECT, - peer: Some(proto::Peer { - id: dst_peer_id.to_bytes(), - addrs: vec![], - }), - reservation: None, - limit: None, - status: None, - }, - }; - - let mut substream = Framed::new( - substream, - quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE), - ); - - async move { - substream.send(msg).await?; - let proto::HopMessage { - type_pb, - peer: _, - reservation, - limit, - status, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; - - match type_pb { - proto::HopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect.into()) - } - proto::HopMessageType::RESERVE => { - return Err(FatalUpgradeError::UnexpectedTypeReserve.into()) - } - proto::HopMessageType::STATUS => {} - } - - let limit = limit.map(Into::into); +use libp2p_core::Multiaddr; - let output = match self { - Upgrade::Reserve => { - match status - .ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? - { - proto::Status::OK => {} - proto::Status::RESERVATION_REFUSED => { - return Err(ReservationFailedReason::Refused.into()) - } - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(ReservationFailedReason::ResourceLimitExceeded.into()) - } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), - } - - let reservation = - reservation.ok_or(FatalUpgradeError::MissingReservationField)?; - - if reservation.addrs.is_empty() { - return Err(FatalUpgradeError::NoAddressesInReservation.into()); - } - - let addrs = reservation - .addrs - .into_iter() - .map(|b| Multiaddr::try_from(b.to_vec())) - .collect::, _>>() - .map_err(|_| FatalUpgradeError::InvalidReservationAddrs)?; - - let renewal_timeout = reservation - .expire - .checked_sub( - SystemTime::now() - .duration_since(SystemTime::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 - .ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? - { - proto::Status::OK => {} - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(CircuitFailedReason::ResourceLimitExceeded.into()) - } - proto::Status::CONNECTION_FAILED => { - return Err(CircuitFailedReason::ConnectionFailed.into()) - } - proto::Status::NO_RESERVATION => { - return Err(CircuitFailedReason::NoReservation.into()) - } - proto::Status::PERMISSION_DENIED => { - return Err(CircuitFailedReason::PermissionDenied.into()) - } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), - } - - 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() - } -} +use crate::priv_client::transport; +use crate::proto; +use crate::protocol::Limit; #[derive(Debug, Error)] -pub enum UpgradeError { +pub(crate) enum UpgradeError { #[error("Reservation failed")] ReservationFailed(#[from] ReservationFailedReason), #[error("Circuit failed")] @@ -262,15 +92,14 @@ pub enum FatalUpgradeError { UnexpectedStatus(proto::Status), } -pub enum Output { +pub(crate) enum Output { Reservation { renewal_timeout: Delay, addrs: Vec, limit: Option, + to_listener: mpsc::Sender, }, Circuit { - substream: Stream, - read_buffer: Bytes, limit: Option, }, } From 0b629da8720902db31e63a12201744117700c10d Mon Sep 17 00:00:00 2001 From: dgarus Date: Thu, 3 Aug 2023 16:30:44 +0300 Subject: [PATCH 04/83] fix review comments --- protocols/relay/src/behaviour.rs | 4 +- protocols/relay/src/behaviour/handler.rs | 163 +++----------------- protocols/relay/src/priv_client/handler.rs | 46 +++--- protocols/relay/src/protocol/inbound_hop.rs | 130 ++++++++++++++++ 4 files changed, 184 insertions(+), 159 deletions(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index daebcadf1d9..8fcfa103c4f 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -22,7 +22,7 @@ mod handler; pub(crate) mod rate_limiter; -use crate::behaviour::handler::{Handler, ReservationReq}; +use crate::behaviour::handler::Handler; use crate::multiaddr_ext::MultiaddrExt; use crate::proto; use crate::protocol::{inbound_hop, outbound_stop}; @@ -811,7 +811,7 @@ impl Add for CircuitId { enum Action { Done(ToSwarm>), AcceptReservationPrototype { - inbound_reservation_req: ReservationReq, + inbound_reservation_req: inbound_hop::ReservationReq, handler: NotifyHandler, peer_id: PeerId, }, diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index ebb19b6ddee..c401ca4fe4c 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -21,7 +21,7 @@ use std::collections::VecDeque; use std::fmt; use std::task::{Context, Poll}; -use std::time::{Duration, SystemTime}; +use std::time::Duration; use asynchronous_codec::{Framed, FramedParts}; use bytes::Bytes; @@ -47,7 +47,7 @@ use crate::behaviour::CircuitId; use crate::copy_future::CopyFuture; use crate::proto::message_v2::pb::mod_HopMessage::Type; use crate::proto::Status; -use crate::protocol::outbound_stop::{CircuitFailedReason, FatalUpgradeError}; +use crate::protocol::outbound_stop::CircuitFailedReason; use crate::protocol::MAX_MESSAGE_SIZE; use crate::protocol::{inbound_hop, outbound_stop}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; @@ -61,28 +61,28 @@ pub struct Config { pub enum In { AcceptReservationReq { - inbound_reservation_req: ReservationReq, + inbound_reservation_req: inbound_hop::ReservationReq, addrs: Vec, }, DenyReservationReq { - inbound_reservation_req: ReservationReq, + inbound_reservation_req: inbound_hop::ReservationReq, status: Status, }, DenyCircuitReq { circuit_id: Option, - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, status: Status, }, NegotiateOutboundConnect { circuit_id: CircuitId, - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, src_peer_id: PeerId, src_connection_id: ConnectionId, }, AcceptAndDriveCircuit { circuit_id: CircuitId, dst_peer_id: PeerId, - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, dst_handler_notifier: oneshot::Sender<()>, dst_stream: Stream, dst_pending_data: Bytes, @@ -147,7 +147,7 @@ impl fmt::Debug for In { pub enum Event { /// An inbound reservation request has been received. ReservationReqReceived { - inbound_reservation_req: ReservationReq, + inbound_reservation_req: inbound_hop::ReservationReq, endpoint: ConnectedPoint, /// Indicates whether the request replaces an existing reservation. renewed: bool, @@ -167,7 +167,7 @@ pub enum Event { ReservationTimedOut {}, /// An inbound circuit request has been received. CircuitReqReceived { - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, endpoint: ConnectedPoint, }, /// An inbound circuit request has been denied. @@ -198,7 +198,7 @@ pub enum Event { circuit_id: CircuitId, src_peer_id: PeerId, src_connection_id: ConnectionId, - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, dst_handler_notifier: oneshot::Sender<()>, dst_stream: Stream, dst_pending_data: Bytes, @@ -208,7 +208,7 @@ pub enum Event { circuit_id: CircuitId, src_peer_id: PeerId, src_connection_id: ConnectionId, - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, status: Status, error: StreamUpgradeError, }, @@ -391,7 +391,8 @@ pub struct Handler { inbound_requests_futs: FuturesUnordered>>, stop_requested_streams: VecDeque, - outbound_stop_futs: FuturesUnordered>>, + outbound_stop_futs: + FuturesUnordered>>, } impl Handler { @@ -440,7 +441,7 @@ impl Handler { let event = match type_pb { Type::RESERVE => { - let req = ReservationReq { + let req = inbound_hop::ReservationReq { substream, reservation_duration, max_circuit_duration, @@ -458,7 +459,7 @@ impl Handler { &peer.ok_or(inbound_hop::FatalUpgradeError::MissingPeer)?.id, ) .map_err(|_| inbound_hop::FatalUpgradeError::ParsePeerId)?; - let req = CircuitReq { dst, substream }; + let req = inbound_hop::CircuitReq { dst, substream }; Event::CircuitReqReceived { inbound_circuit_req: req, @@ -477,7 +478,7 @@ impl Handler { &self, io: Stream, stop_command: StopCommand, - ) -> BoxFuture<'static, Result> { + ) -> BoxFuture<'static, Result> { let msg = proto::StopMessage { type_pb: proto::StopMessageType::CONNECT, peer: Some(proto::Peer { @@ -512,11 +513,11 @@ impl Handler { } = substream .next() .await - .ok_or(FatalUpgradeError::StreamClosed)??; + .ok_or(outbound_stop::FatalUpgradeError::StreamClosed)??; match type_pb { proto::StopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect); + return Err(outbound_stop::FatalUpgradeError::UnexpectedTypeConnect); } proto::StopMessageType::STATUS => {} } @@ -546,10 +547,10 @@ impl Handler { error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), }) } - s => return Err(FatalUpgradeError::UnexpectedStatus(s)), + s => return Err(outbound_stop::FatalUpgradeError::UnexpectedStatus(s)), }, None => { - return Err(FatalUpgradeError::MissingStatusField); + return Err(outbound_stop::FatalUpgradeError::MissingStatusField); } } @@ -588,7 +589,9 @@ type Futures = FuturesUnordered>; impl ConnectionHandler for Handler { type FromBehaviour = In; type ToBehaviour = Event; - type Error = StreamUpgradeError>; + type Error = StreamUpgradeError< + Either, + >; type InboundProtocol = ReadyUpgrade; type InboundOpenInfo = (); type OutboundProtocol = ReadyUpgrade; @@ -982,127 +985,9 @@ struct CircuitParts { struct StopCommand { circuit_id: CircuitId, - inbound_circuit_req: CircuitReq, + inbound_circuit_req: inbound_hop::CircuitReq, src_peer_id: PeerId, src_connection_id: ConnectionId, max_circuit_duration: Duration, max_circuit_bytes: u64, } - -pub struct ReservationReq { - substream: Framed>, - reservation_duration: Duration, - max_circuit_duration: Duration, - max_circuit_bytes: u64, -} - -impl ReservationReq { - pub async fn accept(self, addrs: Vec) -> Result<(), inbound_hop::UpgradeError> { - if addrs.is_empty() { - log::debug!( - "Accepting relay reservation without providing external addresses of local node. \ - Thus the remote node might not be able to advertise its relayed address." - ) - } - - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: Some(proto::Reservation { - addrs: addrs.into_iter().map(|a| a.to_vec()).collect(), - expire: (SystemTime::now() + self.reservation_duration) - .duration_since(SystemTime::UNIX_EPOCH) - .unwrap() - .as_secs(), - voucher: None, - }), - limit: Some(proto::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(proto::Status::OK), - }; - - self.send(msg).await - } - - pub async fn deny(self, status: Status) -> Result<(), inbound_hop::UpgradeError> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: None, - limit: None, - status: Some(status), - }; - - self.send(msg).await - } - - async fn send(mut self, msg: proto::HopMessage) -> Result<(), inbound_hop::UpgradeError> { - self.substream.send(msg).await?; - self.substream.flush().await?; - self.substream.close().await?; - - Ok(()) - } -} - -pub struct CircuitReq { - dst: PeerId, - substream: Framed>, -} - -impl CircuitReq { - pub fn dst(&self) -> PeerId { - self.dst - } - - pub async fn accept(mut self) -> Result<(Stream, Bytes), inbound_hop::UpgradeError> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: None, - limit: None, - status: Some(Status::OK), - }; - - 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<(), inbound_hop::UpgradeError> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::STATUS, - peer: None, - reservation: None, - limit: None, - status: Some(status), - }; - self.send(msg).await?; - self.substream.close().await.map_err(Into::into) - } - - async fn send(&mut self, msg: proto::HopMessage) -> Result<(), quick_protobuf_codec::Error> { - self.substream.send(msg).await?; - self.substream.flush().await?; - - Ok(()) - } -} diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index d156e9db775..62bfbba8999 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -47,7 +47,7 @@ use libp2p_swarm::{ use crate::priv_client::transport; use crate::protocol::inbound_stop::{open_circuit, Circuit}; -use crate::protocol::outbound_hop::{FatalUpgradeError, Output, UpgradeError}; +use crate::protocol::outbound_hop::{Output, UpgradeError}; use crate::protocol::{self, inbound_stop, outbound_hop, MAX_MESSAGE_SIZE}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; @@ -117,8 +117,11 @@ pub struct Handler { remote_peer_id: PeerId, remote_addr: Multiaddr, /// A pending fatal error that results in the connection being closed. - pending_error: - Option>>, + pending_error: Option< + StreamUpgradeError< + Either, + >, + >, /// Until when to keep the connection alive. keep_alive: KeepAlive, @@ -303,21 +306,23 @@ impl Handler { } = substream .next() .await - .ok_or(FatalUpgradeError::StreamClosed)??; + .ok_or(outbound_hop::FatalUpgradeError::StreamClosed)??; match type_pb { proto::HopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect.into()); + return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeConnect.into()); } proto::HopMessageType::RESERVE => { - return Err(FatalUpgradeError::UnexpectedTypeReserve.into()); + return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeReserve.into()); } proto::HopMessageType::STATUS => {} } let limit = limit.map(Into::into); - match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + match status.ok_or(UpgradeError::Fatal( + outbound_hop::FatalUpgradeError::MissingStatusField, + ))? { proto::Status::OK => {} proto::Status::RESERVATION_REFUSED => { return Err(outbound_hop::ReservationFailedReason::Refused.into()); @@ -325,13 +330,14 @@ impl Handler { proto::Status::RESOURCE_LIMIT_EXCEEDED => { return Err(outbound_hop::ReservationFailedReason::ResourceLimitExceeded.into()); } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + s => return Err(outbound_hop::FatalUpgradeError::UnexpectedStatus(s).into()), } - let reservation = reservation.ok_or(FatalUpgradeError::MissingReservationField)?; + let reservation = + reservation.ok_or(outbound_hop::FatalUpgradeError::MissingReservationField)?; if reservation.addrs.is_empty() { - return Err(FatalUpgradeError::NoAddressesInReservation.into()); + return Err(outbound_hop::FatalUpgradeError::NoAddressesInReservation.into()); } let addrs = reservation @@ -339,7 +345,7 @@ impl Handler { .into_iter() .map(|b| Multiaddr::try_from(b.to_vec())) .collect::, _>>() - .map_err(|_| FatalUpgradeError::InvalidReservationAddrs)?; + .map_err(|_| outbound_hop::FatalUpgradeError::InvalidReservationAddrs)?; let renewal_timeout = reservation .expire @@ -353,7 +359,7 @@ impl Handler { .and_then(|duration| duration.checked_sub(duration / 4)) .map(Duration::from_secs) .map(Delay::new) - .ok_or(FatalUpgradeError::InvalidReservationExpiration)?; + .ok_or(outbound_hop::FatalUpgradeError::InvalidReservationExpiration)?; substream.close().await?; @@ -403,21 +409,23 @@ impl Handler { } = substream .next() .await - .ok_or(FatalUpgradeError::StreamClosed)??; + .ok_or(outbound_hop::FatalUpgradeError::StreamClosed)??; match type_pb { proto::HopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect.into()) + return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeConnect.into()) } proto::HopMessageType::RESERVE => { - return Err(FatalUpgradeError::UnexpectedTypeReserve.into()) + return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeReserve.into()) } proto::HopMessageType::STATUS => {} } let limit = limit.map(Into::into); - match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + match status.ok_or(UpgradeError::Fatal( + outbound_hop::FatalUpgradeError::MissingStatusField, + ))? { proto::Status::OK => {} proto::Status::RESOURCE_LIMIT_EXCEEDED => { return Err(outbound_hop::CircuitFailedReason::ResourceLimitExceeded.into()) @@ -431,7 +439,7 @@ impl Handler { proto::Status::PERMISSION_DENIED => { return Err(outbound_hop::CircuitFailedReason::PermissionDenied.into()) } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + s => return Err(outbound_hop::FatalUpgradeError::UnexpectedStatus(s).into()), } let FramedParts { @@ -469,7 +477,9 @@ impl Handler { impl ConnectionHandler for Handler { type FromBehaviour = In; type ToBehaviour = Event; - type Error = StreamUpgradeError>; + type Error = StreamUpgradeError< + Either, + >; type InboundProtocol = ReadyUpgrade; type InboundOpenInfo = (); type OutboundProtocol = ReadyUpgrade; diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index 754d9bc2b7f..237c370de0c 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -18,8 +18,20 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use std::time::{Duration, SystemTime}; + +use asynchronous_codec::{Framed, FramedParts}; +use bytes::Bytes; +use futures::prelude::*; use thiserror::Error; +use libp2p_core::Multiaddr; +use libp2p_identity::PeerId; +use libp2p_swarm::Stream; + +use crate::proto; +use crate::proto::Status; + #[derive(Debug, Error)] pub enum UpgradeError { #[error("Fatal")] @@ -47,3 +59,121 @@ pub enum FatalUpgradeError { #[error("Unexpected message type 'status'")] UnexpectedTypeStatus, } + +pub struct ReservationReq { + pub(crate) substream: Framed>, + pub(crate) reservation_duration: Duration, + pub(crate) max_circuit_duration: Duration, + pub(crate) max_circuit_bytes: u64, +} + +impl ReservationReq { + pub async fn accept(self, addrs: Vec) -> Result<(), UpgradeError> { + if addrs.is_empty() { + log::debug!( + "Accepting relay reservation without providing external addresses of local node. \ + Thus the remote node might not be able to advertise its relayed address." + ) + } + + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: Some(proto::Reservation { + addrs: addrs.into_iter().map(|a| a.to_vec()).collect(), + expire: (SystemTime::now() + self.reservation_duration) + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs(), + voucher: None, + }), + limit: Some(proto::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(proto::Status::OK), + }; + + self.send(msg).await + } + + pub async fn deny(self, status: Status) -> Result<(), UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: None, + limit: None, + status: Some(status), + }; + + self.send(msg).await + } + + async fn send(mut self, msg: proto::HopMessage) -> Result<(), UpgradeError> { + self.substream.send(msg).await?; + self.substream.flush().await?; + self.substream.close().await?; + + Ok(()) + } +} + +pub struct CircuitReq { + pub(crate) dst: PeerId, + pub(crate) substream: Framed>, +} + +impl CircuitReq { + pub fn dst(&self) -> PeerId { + self.dst + } + + pub async fn accept(mut self) -> Result<(Stream, Bytes), UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: None, + limit: None, + status: Some(Status::OK), + }; + + 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<(), UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::STATUS, + peer: None, + reservation: None, + limit: None, + status: Some(status), + }; + self.send(msg).await?; + self.substream.close().await.map_err(Into::into) + } + + async fn send(&mut self, msg: proto::HopMessage) -> Result<(), quick_protobuf_codec::Error> { + self.substream.send(msg).await?; + self.substream.flush().await?; + + Ok(()) + } +} From dde7f89c69ef59f66c84706474f89d4394c42dce Mon Sep 17 00:00:00 2001 From: dgarus Date: Thu, 3 Aug 2023 16:59:29 +0300 Subject: [PATCH 05/83] fix review comments --- protocols/relay/src/behaviour.rs | 2 +- protocols/relay/src/behaviour/handler.rs | 73 +++----------------- protocols/relay/src/protocol/inbound_hop.rs | 76 ++++++++++++++++++--- 3 files changed, 77 insertions(+), 74 deletions(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 8fcfa103c4f..a7c3d81e54c 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -20,7 +20,7 @@ //! [`NetworkBehaviour`] to act as a circuit relay v2 **relay**. -mod handler; +pub(crate) mod handler; pub(crate) mod rate_limiter; use crate::behaviour::handler::Handler; use crate::multiaddr_ext::MultiaddrExt; diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index c401ca4fe4c..e533e2ea34d 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -45,7 +45,6 @@ use libp2p_swarm::{ use crate::behaviour::CircuitId; use crate::copy_future::CopyFuture; -use crate::proto::message_v2::pb::mod_HopMessage::Type; use crate::proto::Status; use crate::protocol::outbound_stop::CircuitFailedReason; use crate::protocol::MAX_MESSAGE_SIZE; @@ -415,65 +414,6 @@ impl Handler { } } - fn process_inbound_request( - &self, - io: Stream, - ) -> BoxFuture<'static, Result> { - let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - - let reservation_duration = self.config.reservation_duration; - let max_circuit_duration = self.config.max_circuit_duration; - let max_circuit_bytes = self.config.max_circuit_bytes; - let endpoint = self.endpoint.clone(); - let renewed = self.active_reservation.is_some(); - - async move { - let proto::HopMessage { - type_pb, - peer, - reservation: _, - limit: _, - status: _, - } = substream - .next() - .await - .ok_or(inbound_hop::FatalUpgradeError::StreamClosed)??; - - let event = match type_pb { - Type::RESERVE => { - let req = inbound_hop::ReservationReq { - substream, - reservation_duration, - max_circuit_duration, - max_circuit_bytes, - }; - - Event::ReservationReqReceived { - inbound_reservation_req: req, - endpoint, - renewed, - } - } - Type::CONNECT => { - let dst = PeerId::from_bytes( - &peer.ok_or(inbound_hop::FatalUpgradeError::MissingPeer)?.id, - ) - .map_err(|_| inbound_hop::FatalUpgradeError::ParsePeerId)?; - let req = inbound_hop::CircuitReq { dst, substream }; - - Event::CircuitReqReceived { - inbound_circuit_req: req, - endpoint, - } - } - Type::STATUS => return Err(inbound_hop::FatalUpgradeError::UnexpectedTypeStatus), - }; - - Ok(event) - } - .boxed() - } - fn send_stop_message_and_process_result( &self, io: Stream, @@ -950,8 +890,17 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - self.inbound_requests_futs - .push(self.process_inbound_request(stream)); + self.inbound_requests_futs.push( + inbound_hop::process_inbound_request( + stream, + self.config.reservation_duration, + self.config.max_circuit_duration, + self.config.max_circuit_bytes, + self.endpoint.clone(), + self.active_reservation.is_some(), + ) + .boxed(), + ); } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index 237c370de0c..7bc7fb887d2 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -25,12 +25,14 @@ use bytes::Bytes; use futures::prelude::*; use thiserror::Error; -use libp2p_core::Multiaddr; +use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; use libp2p_swarm::Stream; +use crate::behaviour::handler; use crate::proto; -use crate::proto::Status; +use crate::proto::message_v2::pb::mod_HopMessage::Type; +use crate::protocol::MAX_MESSAGE_SIZE; #[derive(Debug, Error)] pub enum UpgradeError { @@ -61,10 +63,10 @@ pub enum FatalUpgradeError { } pub struct ReservationReq { - pub(crate) substream: Framed>, - pub(crate) reservation_duration: Duration, - pub(crate) max_circuit_duration: Duration, - pub(crate) max_circuit_bytes: u64, + substream: Framed>, + reservation_duration: Duration, + max_circuit_duration: Duration, + max_circuit_bytes: u64, } impl ReservationReq { @@ -102,7 +104,7 @@ impl ReservationReq { self.send(msg).await } - pub async fn deny(self, status: Status) -> Result<(), UpgradeError> { + pub async fn deny(self, status: proto::Status) -> Result<(), UpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::STATUS, peer: None, @@ -124,8 +126,8 @@ impl ReservationReq { } pub struct CircuitReq { - pub(crate) dst: PeerId, - pub(crate) substream: Framed>, + dst: PeerId, + substream: Framed>, } impl CircuitReq { @@ -139,7 +141,7 @@ impl CircuitReq { peer: None, reservation: None, limit: None, - status: Some(Status::OK), + status: Some(proto::Status::OK), }; self.send(msg).await?; @@ -158,7 +160,7 @@ impl CircuitReq { Ok((io, read_buffer.freeze())) } - pub async fn deny(mut self, status: Status) -> Result<(), UpgradeError> { + pub async fn deny(mut self, status: proto::Status) -> Result<(), UpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::STATUS, peer: None, @@ -177,3 +179,55 @@ impl CircuitReq { Ok(()) } } + +pub(crate) async fn process_inbound_request( + io: Stream, + reservation_duration: Duration, + max_circuit_duration: Duration, + max_circuit_bytes: u64, + endpoint: ConnectedPoint, + renewed: bool, +) -> Result { + let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + let proto::HopMessage { + type_pb, + peer, + reservation: _, + limit: _, + status: _, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + let event = match type_pb { + Type::RESERVE => { + let req = ReservationReq { + substream, + reservation_duration, + max_circuit_duration, + max_circuit_bytes, + }; + + handler::Event::ReservationReqReceived { + inbound_reservation_req: req, + endpoint, + renewed, + } + } + Type::CONNECT => { + let dst = PeerId::from_bytes(&peer.ok_or(FatalUpgradeError::MissingPeer)?.id) + .map_err(|_| FatalUpgradeError::ParsePeerId)?; + let req = CircuitReq { dst, substream }; + + handler::Event::CircuitReqReceived { + inbound_circuit_req: req, + endpoint, + } + } + Type::STATUS => return Err(FatalUpgradeError::UnexpectedTypeStatus), + }; + + Ok(event) +} From 55fa4d6073c741cfd769c3b96a45482c8432c8fe Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 11:21:40 +0300 Subject: [PATCH 06/83] fix review comments --- protocols/relay/src/behaviour/handler.rs | 146 +++--------------- protocols/relay/src/protocol/outbound_stop.rs | 138 ++++++++++++++++- 2 files changed, 155 insertions(+), 129 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index e533e2ea34d..edb1e408b0c 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -23,13 +23,11 @@ use std::fmt; use std::task::{Context, Poll}; use std::time::Duration; -use asynchronous_codec::{Framed, FramedParts}; use bytes::Bytes; use either::Either; use futures::channel::oneshot::{self, Canceled}; use futures::future::{BoxFuture, FutureExt, TryFutureExt}; use futures::io::AsyncWriteExt; -use futures::prelude::*; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; @@ -47,9 +45,8 @@ use crate::behaviour::CircuitId; use crate::copy_future::CopyFuture; use crate::proto::Status; use crate::protocol::outbound_stop::CircuitFailedReason; -use crate::protocol::MAX_MESSAGE_SIZE; use crate::protocol::{inbound_hop, outbound_stop}; -use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; +use crate::{HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; #[derive(Debug, Clone)] pub struct Config { @@ -389,7 +386,7 @@ pub struct Handler { inbound_requests_futs: FuturesUnordered>>, - stop_requested_streams: VecDeque, + stop_requested_streams: VecDeque, outbound_stop_futs: FuturesUnordered>>, } @@ -413,110 +410,6 @@ impl Handler { outbound_stop_futs: Default::default(), } } - - fn send_stop_message_and_process_result( - &self, - io: Stream, - stop_command: StopCommand, - ) -> BoxFuture<'static, Result> { - let msg = proto::StopMessage { - type_pb: proto::StopMessageType::CONNECT, - peer: Some(proto::Peer { - id: stop_command.src_peer_id.to_bytes(), - addrs: vec![], - }), - limit: Some(proto::Limit { - duration: Some( - stop_command - .max_circuit_duration - .as_secs() - .try_into() - .expect("`max_circuit_duration` not to exceed `u32::MAX`."), - ), - data: Some(stop_command.max_circuit_bytes), - }), - status: None, - }; - - let (tx, rx) = oneshot::channel(); - self.alive_lend_out_substreams.push(rx); - - let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - async move { - substream.send(msg).await?; - - let proto::StopMessage { - type_pb, - peer: _, - limit: _, - status, - } = substream - .next() - .await - .ok_or(outbound_stop::FatalUpgradeError::StreamClosed)??; - - match type_pb { - proto::StopMessageType::CONNECT => { - return Err(outbound_stop::FatalUpgradeError::UnexpectedTypeConnect); - } - proto::StopMessageType::STATUS => {} - } - - match status { - Some(proto_status) => match proto_status { - Status::OK => {} - Status::RESOURCE_LIMIT_EXCEEDED => { - return Ok(Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply( - CircuitFailedReason::ResourceLimitExceeded, - ), - }) - } - Status::PERMISSION_DENIED => { - return Ok(Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), - }) - } - s => return Err(outbound_stop::FatalUpgradeError::UnexpectedStatus(s)), - }, - None => { - return Err(outbound_stop::FatalUpgradeError::MissingStatusField); - } - } - - 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(Event::OutboundConnectNegotiated { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - dst_handler_notifier: tx, - dst_stream: io, - dst_pending_data: read_buffer.freeze(), - }) - } - .boxed() - } } enum ReservationRequestFuture { @@ -577,14 +470,14 @@ impl ConnectionHandler for Handler { src_peer_id, src_connection_id, } => { - self.stop_requested_streams.push_back(StopCommand { - circuit_id, - inbound_circuit_req, - src_peer_id, - src_connection_id, - max_circuit_duration: self.config.max_circuit_duration, - max_circuit_bytes: self.config.max_circuit_bytes, - }); + self.stop_requested_streams + .push_back(outbound_stop::StopCommand::new( + circuit_id, + inbound_circuit_req, + src_peer_id, + src_connection_id, + &self.config, + )); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(STOP_PROTOCOL_NAME), ()), @@ -910,8 +803,14 @@ impl ConnectionHandler for Handler { .stop_requested_streams .pop_front() .expect("opened a stream without a pending stop command"); - self.outbound_stop_futs - .push(self.send_stop_message_and_process_result(stream, stop_command)); + + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); + + self.outbound_stop_futs.push( + outbound_stop::send_stop_message_and_process_result(stream, stop_command, tx) + .boxed(), + ); } ConnectionEvent::AddressChange(_) | ConnectionEvent::ListenUpgradeError(_) @@ -931,12 +830,3 @@ struct CircuitParts { dst_stream: Stream, dst_pending_data: Bytes, } - -struct StopCommand { - circuit_id: CircuitId, - inbound_circuit_req: inbound_hop::CircuitReq, - src_peer_id: PeerId, - src_connection_id: ConnectionId, - max_circuit_duration: Duration, - max_circuit_bytes: u64, -} diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 82728464403..91fb53e131b 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -18,9 +18,20 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use std::time::Duration; + +use asynchronous_codec::{Framed, FramedParts}; +use futures::channel::oneshot::{self}; +use futures::prelude::*; use thiserror::Error; -use crate::proto; +use libp2p_identity::PeerId; +use libp2p_swarm::{ConnectionId, Stream, StreamUpgradeError}; + +use crate::behaviour::handler; +use crate::behaviour::handler::Config; +use crate::protocol::{inbound_hop, MAX_MESSAGE_SIZE}; +use crate::{proto, CircuitId}; #[derive(Debug, Error)] pub(crate) enum UpgradeError { @@ -61,3 +72,128 @@ pub enum FatalUpgradeError { #[error("Unexpected message status '{0:?}'")] UnexpectedStatus(proto::Status), } + +pub(crate) async fn send_stop_message_and_process_result( + io: Stream, + stop_command: StopCommand, + tx: oneshot::Sender<()>, +) -> Result { + let msg = proto::StopMessage { + type_pb: proto::StopMessageType::CONNECT, + peer: Some(proto::Peer { + id: stop_command.src_peer_id.to_bytes(), + addrs: vec![], + }), + limit: Some(proto::Limit { + duration: Some( + stop_command + .max_circuit_duration + .as_secs() + .try_into() + .expect("`max_circuit_duration` not to exceed `u32::MAX`."), + ), + data: Some(stop_command.max_circuit_bytes), + }), + status: None, + }; + + let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + substream.send(msg).await?; + + let proto::StopMessage { + type_pb, + peer: _, + limit: _, + status, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::StopMessageType::CONNECT => { + return Err(FatalUpgradeError::UnexpectedTypeConnect); + } + proto::StopMessageType::STATUS => {} + } + + match status { + Some(proto_status) => match proto_status { + proto::Status::OK => {} + proto::Status::RESOURCE_LIMIT_EXCEEDED => { + return Ok(handler::Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), + }) + } + proto::Status::PERMISSION_DENIED => { + return Ok(handler::Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), + }) + } + s => return Err(FatalUpgradeError::UnexpectedStatus(s)), + }, + None => { + return Err(FatalUpgradeError::MissingStatusField); + } + } + + 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(handler::Event::OutboundConnectNegotiated { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + dst_handler_notifier: tx, + dst_stream: io, + dst_pending_data: read_buffer.freeze(), + }) +} + +pub(crate) struct StopCommand { + circuit_id: CircuitId, + inbound_circuit_req: inbound_hop::CircuitReq, + src_peer_id: PeerId, + src_connection_id: ConnectionId, + max_circuit_duration: Duration, + max_circuit_bytes: u64, +} + +impl StopCommand { + pub(crate) fn new( + circuit_id: CircuitId, + inbound_circuit_req: inbound_hop::CircuitReq, + src_peer_id: PeerId, + src_connection_id: ConnectionId, + config: &Config, + ) -> Self { + Self { + circuit_id, + inbound_circuit_req, + src_peer_id, + src_connection_id, + max_circuit_duration: config.max_circuit_duration, + max_circuit_bytes: config.max_circuit_bytes, + } + } +} From 3fb008c58b3e7a7e44e9a5c12ae39701861095cd Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 12:38:54 +0300 Subject: [PATCH 07/83] fix review comments --- protocols/relay/src/behaviour/handler.rs | 42 +++------- protocols/relay/src/protocol/inbound_hop.rs | 46 ++++++++--- protocols/relay/src/protocol/outbound_stop.rs | 79 ++++++++++++------- 3 files changed, 101 insertions(+), 66 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index edb1e408b0c..91160276c69 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -384,11 +384,8 @@ pub struct Handler { /// Futures relaying data for circuit between two peers. circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, - inbound_requests_futs: - FuturesUnordered>>, stop_requested_streams: VecDeque, - outbound_stop_futs: - FuturesUnordered>>, + protocol_futs: FuturesUnordered>, } impl Handler { @@ -405,9 +402,8 @@ impl Handler { circuits: Default::default(), active_reservation: Default::default(), keep_alive: KeepAlive::Yes, - inbound_requests_futs: Default::default(), stop_requested_streams: Default::default(), - outbound_stop_futs: Default::default(), + protocol_futs: Default::default(), } } } @@ -419,6 +415,13 @@ enum ReservationRequestFuture { type Futures = FuturesUnordered>; +pub(crate) type CHEvent = ConnectionHandlerEvent< + ::OutboundProtocol, + ::OutboundOpenInfo, + ::ToBehaviour, + ::Error, +>; + impl ConnectionHandler for Handler { type FromBehaviour = In; type ToBehaviour = Event; @@ -575,27 +578,8 @@ impl ConnectionHandler for Handler { } } - // Process inbound requests - if let Poll::Ready(Some(result)) = self.inbound_requests_futs.poll_next_unpin(cx) { - let event = match result { - Ok(event) => ConnectionHandlerEvent::NotifyBehaviour(event), - Err(err) => { - ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left(err))) - } - }; - - return Poll::Ready(event); - } - - // Send stop commands - if let Poll::Ready(Some(result)) = self.outbound_stop_futs.poll_next_unpin(cx) { - let event = match result { - Ok(event) => ConnectionHandlerEvent::NotifyBehaviour(event), - Err(e) => { - ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right(e))) - } - }; - + // Process protocol requests + if let Poll::Ready(Some(event)) = self.protocol_futs.poll_next_unpin(cx) { return Poll::Ready(event); } @@ -783,7 +767,7 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - self.inbound_requests_futs.push( + self.protocol_futs.push( inbound_hop::process_inbound_request( stream, self.config.reservation_duration, @@ -807,7 +791,7 @@ impl ConnectionHandler for Handler { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); - self.outbound_stop_futs.push( + self.protocol_futs.push( outbound_stop::send_stop_message_and_process_result(stream, stop_command, tx) .boxed(), ); diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index 7bc7fb887d2..21c8d85007f 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -22,12 +22,13 @@ use std::time::{Duration, SystemTime}; use asynchronous_codec::{Framed, FramedParts}; use bytes::Bytes; +use either::Either; use futures::prelude::*; use thiserror::Error; use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; -use libp2p_swarm::Stream; +use libp2p_swarm::{ConnectionHandlerEvent, Stream, StreamUpgradeError}; use crate::behaviour::handler; use crate::proto; @@ -187,19 +188,24 @@ pub(crate) async fn process_inbound_request( max_circuit_bytes: u64, endpoint: ConnectedPoint, renewed: bool, -) -> Result { +) -> handler::CHEvent { let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + let res = substream.next().await; + + if let None | Some(Err(_)) = res { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( + FatalUpgradeError::StreamClosed, + ))); + } + let proto::HopMessage { type_pb, peer, reservation: _, limit: _, status: _, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; + } = res.unwrap().expect("should be ok"); let event = match type_pb { Type::RESERVE => { @@ -217,8 +223,24 @@ pub(crate) async fn process_inbound_request( } } Type::CONNECT => { - let dst = PeerId::from_bytes(&peer.ok_or(FatalUpgradeError::MissingPeer)?.id) - .map_err(|_| FatalUpgradeError::ParsePeerId)?; + let peer_id_res = match peer { + Some(r) => PeerId::from_bytes(&r.id), + None => { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( + FatalUpgradeError::MissingPeer, + ))) + } + }; + + let dst = match peer_id_res { + Ok(res) => res, + Err(_) => { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( + FatalUpgradeError::ParsePeerId, + ))) + } + }; + let req = CircuitReq { dst, substream }; handler::Event::CircuitReqReceived { @@ -226,8 +248,12 @@ pub(crate) async fn process_inbound_request( endpoint, } } - Type::STATUS => return Err(FatalUpgradeError::UnexpectedTypeStatus), + Type::STATUS => { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( + FatalUpgradeError::UnexpectedTypeStatus, + ))) + } }; - Ok(event) + ConnectionHandlerEvent::NotifyBehaviour(event) } diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 91fb53e131b..93f6554c7f4 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -21,12 +21,13 @@ use std::time::Duration; use asynchronous_codec::{Framed, FramedParts}; +use either::Either; use futures::channel::oneshot::{self}; use futures::prelude::*; use thiserror::Error; use libp2p_identity::PeerId; -use libp2p_swarm::{ConnectionId, Stream, StreamUpgradeError}; +use libp2p_swarm::{ConnectionHandlerEvent, ConnectionId, Stream, StreamUpgradeError}; use crate::behaviour::handler; use crate::behaviour::handler::Config; @@ -77,7 +78,7 @@ pub(crate) async fn send_stop_message_and_process_result( io: Stream, stop_command: StopCommand, tx: oneshot::Sender<()>, -) -> Result { +) -> handler::CHEvent { let msg = proto::StopMessage { type_pb: proto::StopMessageType::CONNECT, peer: Some(proto::Peer { @@ -99,21 +100,33 @@ pub(crate) async fn send_stop_message_and_process_result( let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - substream.send(msg).await?; + let send_res = substream.send(msg).await; + if send_res.is_err() { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( + FatalUpgradeError::StreamClosed, + ))); + } + + let res = substream.next().await; + + if let None | Some(Err(_)) = res { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( + FatalUpgradeError::StreamClosed, + ))); + } let proto::StopMessage { type_pb, peer: _, limit: _, status, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; + } = res.unwrap().expect("should be ok"); match type_pb { proto::StopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect); + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( + FatalUpgradeError::UnexpectedTypeConnect, + ))) } proto::StopMessageType::STATUS => {} } @@ -122,29 +135,41 @@ pub(crate) async fn send_stop_message_and_process_result( Some(proto_status) => match proto_status { proto::Status::OK => {} proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Ok(handler::Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), - }) + return ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply( + CircuitFailedReason::ResourceLimitExceeded, + ), + }, + ) } proto::Status::PERMISSION_DENIED => { - return Ok(handler::Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), - }) + return ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto_status, + error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), + }, + ) + } + s => { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( + FatalUpgradeError::UnexpectedStatus(s), + ))) } - s => return Err(FatalUpgradeError::UnexpectedStatus(s)), }, None => { - return Err(FatalUpgradeError::MissingStatusField); + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( + FatalUpgradeError::MissingStatusField, + ))) } } @@ -159,7 +184,7 @@ pub(crate) async fn send_stop_message_and_process_result( "Expect a flushed Framed to have an empty write buffer." ); - Ok(handler::Event::OutboundConnectNegotiated { + ConnectionHandlerEvent::NotifyBehaviour(handler::Event::OutboundConnectNegotiated { circuit_id: stop_command.circuit_id, src_peer_id: stop_command.src_peer_id, src_connection_id: stop_command.src_connection_id, From 4aac352d636fb458c8761ff18927e8cd32b5853a Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 12:50:06 +0300 Subject: [PATCH 08/83] fix review comments --- protocols/relay/src/behaviour/handler.rs | 53 ++++++++++++++---------- 1 file changed, 30 insertions(+), 23 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 91160276c69..c3b2be920e6 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -406,6 +406,34 @@ impl Handler { protocol_futs: Default::default(), } } + + fn on_fully_negotiated_inbound(&self, stream: Stream) { + self.protocol_futs.push( + inbound_hop::process_inbound_request( + stream, + self.config.reservation_duration, + self.config.max_circuit_duration, + self.config.max_circuit_bytes, + self.endpoint.clone(), + self.active_reservation.is_some(), + ) + .boxed(), + ); + } + + fn on_fully_negotiated_outbound(&mut self, stream: Stream) { + let stop_command = self + .stop_requested_streams + .pop_front() + .expect("opened a stream without a pending stop command"); + + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); + + self.protocol_futs.push( + outbound_stop::send_stop_message_and_process_result(stream, stop_command, tx).boxed(), + ); + } } enum ReservationRequestFuture { @@ -767,34 +795,13 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - self.protocol_futs.push( - inbound_hop::process_inbound_request( - stream, - self.config.reservation_duration, - self.config.max_circuit_duration, - self.config.max_circuit_bytes, - self.endpoint.clone(), - self.active_reservation.is_some(), - ) - .boxed(), - ); + self.on_fully_negotiated_inbound(stream); } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, .. }) => { - let stop_command = self - .stop_requested_streams - .pop_front() - .expect("opened a stream without a pending stop command"); - - let (tx, rx) = oneshot::channel(); - self.alive_lend_out_substreams.push(rx); - - self.protocol_futs.push( - outbound_stop::send_stop_message_and_process_result(stream, stop_command, tx) - .boxed(), - ); + self.on_fully_negotiated_outbound(stream); } ConnectionEvent::AddressChange(_) | ConnectionEvent::ListenUpgradeError(_) From 1a1f912baac12854b9fbaca03461e0f46bb6180c Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 13:04:02 +0300 Subject: [PATCH 09/83] fix review comments --- protocols/relay/src/behaviour/handler.rs | 38 ++++++++++-------------- 1 file changed, 16 insertions(+), 22 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index c3b2be920e6..05499b96736 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -18,11 +18,10 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use std::collections::VecDeque; -use std::fmt; -use std::task::{Context, Poll}; -use std::time::Duration; - +use crate::behaviour::CircuitId; +use crate::copy_future::CopyFuture; +use crate::{HOP_PROTOCOL_NAME, proto, STOP_PROTOCOL_NAME}; +use crate::protocol::{inbound_hop, outbound_stop}; use bytes::Bytes; use either::Either; use futures::channel::oneshot::{self, Canceled}; @@ -31,22 +30,17 @@ use futures::io::AsyncWriteExt; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; - -use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; -use libp2p_swarm::handler::{ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound}; -use libp2p_swarm::{ - ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, - StreamUpgradeError, SubstreamProtocol, +use libp2p_swarm::handler::{ + ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound, }; - -use crate::behaviour::CircuitId; -use crate::copy_future::CopyFuture; -use crate::proto::Status; -use crate::protocol::outbound_stop::CircuitFailedReason; -use crate::protocol::{inbound_hop, outbound_stop}; -use crate::{HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; +use libp2p_swarm::{ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, StreamUpgradeError, SubstreamProtocol}; +use std::collections::VecDeque; +use std::fmt; +use std::task::{Context, Poll}; +use std::time::Duration; +use libp2p_core::upgrade::ReadyUpgrade; #[derive(Debug, Clone)] pub struct Config { @@ -62,12 +56,12 @@ pub enum In { }, DenyReservationReq { inbound_reservation_req: inbound_hop::ReservationReq, - status: Status, + status: proto::Status, }, DenyCircuitReq { circuit_id: Option, inbound_circuit_req: inbound_hop::CircuitReq, - status: Status, + status: proto::Status, }, NegotiateOutboundConnect { circuit_id: CircuitId, @@ -205,8 +199,8 @@ pub enum Event { src_peer_id: PeerId, src_connection_id: ConnectionId, inbound_circuit_req: inbound_hop::CircuitReq, - status: Status, - error: StreamUpgradeError, + status: proto::Status, + error: StreamUpgradeError, }, /// An inbound circuit has closed. CircuitClosed { From 4e699dd85263f1694fb8ed4225ef538996ee9229 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 13:26:15 +0300 Subject: [PATCH 10/83] fix review comments --- protocols/relay/src/priv_client/handler.rs | 68 ++++++++++------------ 1 file changed, 30 insertions(+), 38 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 62bfbba8999..7627a86101b 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -18,12 +18,9 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use std::collections::{HashMap, VecDeque}; -use std::fmt; -use std::task::{Context, Poll}; -use std::time::{Duration, SystemTime}; - -use asynchronous_codec::{Framed, FramedParts}; +use crate::priv_client::transport; +use crate::{HOP_PROTOCOL_NAME, proto, STOP_PROTOCOL_NAME}; +use crate::protocol::{self, inbound_stop, MAX_MESSAGE_SIZE, outbound_hop}; use either::Either; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; @@ -31,25 +28,20 @@ use futures::sink::SinkExt; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; -use log::debug; - use libp2p_core::multiaddr::Protocol; -use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::Multiaddr; use libp2p_identity::PeerId; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, }; -use libp2p_swarm::{ - ConnectionHandler, ConnectionHandlerEvent, KeepAlive, Stream, StreamProtocol, - StreamUpgradeError, SubstreamProtocol, -}; - -use crate::priv_client::transport; -use crate::protocol::inbound_stop::{open_circuit, Circuit}; -use crate::protocol::outbound_hop::{Output, UpgradeError}; -use crate::protocol::{self, inbound_stop, outbound_hop, MAX_MESSAGE_SIZE}; -use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; +use libp2p_swarm::{ConnectionHandler, ConnectionHandlerEvent, KeepAlive, Stream, StreamProtocol, StreamUpgradeError, SubstreamProtocol}; +use log::debug; +use std::collections::{HashMap, VecDeque}; +use std::fmt; +use std::task::{Context, Poll}; +use std::time::{Duration, SystemTime}; +use asynchronous_codec::{Framed, FramedParts}; +use libp2p_core::upgrade::ReadyUpgrade; /// The maximum number of circuits being denied concurrently. /// @@ -136,11 +128,11 @@ pub struct Handler { >, wait_for_reserve_outbound_stream: VecDeque>, - reserve_futs: FuturesUnordered>>, + reserve_futs: FuturesUnordered>>, wait_for_connection_outbound_stream: VecDeque, circuit_connection_futs: - FuturesUnordered, UpgradeError>>>, + FuturesUnordered, outbound_hop::UpgradeError>>>, reservation: Reservation, @@ -155,7 +147,7 @@ pub struct Handler { alive_lend_out_substreams: FuturesUnordered>, open_circuit_futs: - FuturesUnordered>>, + FuturesUnordered>>, circuit_deny_futs: HashMap>>, @@ -257,7 +249,7 @@ impl Handler { )); } - fn insert_to_deny_futs(&mut self, circuit: Circuit) { + fn insert_to_deny_futs(&mut self, circuit: inbound_stop::Circuit) { let src_peer_id = circuit.src_peer_id(); if self.circuit_deny_futs.len() == MAX_NUMBER_DENYING_CIRCUIT @@ -283,7 +275,7 @@ impl Handler { &self, protocol: Stream, to_listener: mpsc::Sender, - ) -> BoxFuture<'static, Result> { + ) -> BoxFuture<'static, Result> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::RESERVE, peer: None, @@ -320,7 +312,7 @@ impl Handler { let limit = limit.map(Into::into); - match status.ok_or(UpgradeError::Fatal( + match status.ok_or(outbound_hop::UpgradeError::Fatal( outbound_hop::FatalUpgradeError::MissingStatusField, ))? { proto::Status::OK => {} @@ -363,7 +355,7 @@ impl Handler { substream.close().await?; - let output = Output::Reservation { + let output = outbound_hop::Output::Reservation { renewal_timeout, addrs, limit, @@ -379,7 +371,7 @@ impl Handler { &self, protocol: Stream, con_command: ConnectionCommand, - ) -> BoxFuture<'static, Result, UpgradeError>> { + ) -> BoxFuture<'static, Result, outbound_hop::UpgradeError>> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::CONNECT, peer: Some(proto::Peer { @@ -423,7 +415,7 @@ impl Handler { let limit = limit.map(Into::into); - match status.ok_or(UpgradeError::Fatal( + match status.ok_or(outbound_hop::UpgradeError::Fatal( outbound_hop::FatalUpgradeError::MissingStatusField, ))? { proto::Status::OK => {} @@ -456,7 +448,7 @@ impl Handler { let output = match con_command.send_back.send(Ok(super::Connection { state: super::ConnectionState::new_outbound(io, read_buffer.freeze(), tx), })) { - Ok(()) => Some(Output::Circuit { limit }), + Ok(()) => Some(outbound_hop::Output::Circuit { limit }), Err(_) => { debug!( "Oneshot to `client::transport::Dial` future dropped. \ @@ -539,7 +531,7 @@ impl ConnectionHandler for Handler { // Reservations if let Poll::Ready(Some(result)) = self.reserve_futs.poll_next_unpin(cx) { let event = match result { - Ok(Output::Reservation { + Ok(outbound_hop::Output::Reservation { renewal_timeout, addrs, limit, @@ -552,7 +544,7 @@ impl ConnectionHandler for Handler { limit, )), Err(err) => match err { - UpgradeError::ReservationFailed(e) => { + outbound_hop::UpgradeError::ReservationFailed(e) => { let renewal = self.reservation.failed(); return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( Event::ReservationReqFailed { @@ -561,10 +553,10 @@ impl ConnectionHandler for Handler { }, )); } - UpgradeError::Fatal(e) => { + outbound_hop::UpgradeError::Fatal(e) => { ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right(e))) } - UpgradeError::CircuitFailed(_) => { + outbound_hop::UpgradeError::CircuitFailed(_) => { unreachable!("do not emit `CircuitFailed` for reservation") } }, @@ -577,7 +569,7 @@ impl ConnectionHandler for Handler { // Circuit connections if let Poll::Ready(Some(res)) = self.circuit_connection_futs.poll_next_unpin(cx) { let opt = match res { - Ok(Some(Output::Circuit { limit })) => { + Ok(Some(outbound_hop::Output::Circuit { limit })) => { Some(ConnectionHandlerEvent::NotifyBehaviour( Event::OutboundCircuitEstablished { limit }, )) @@ -585,15 +577,15 @@ impl ConnectionHandler for Handler { Ok(None) => None, Err(err) => { let res = match err { - UpgradeError::CircuitFailed(e) => ConnectionHandlerEvent::NotifyBehaviour( + outbound_hop::UpgradeError::CircuitFailed(e) => ConnectionHandlerEvent::NotifyBehaviour( Event::OutboundCircuitReqFailed { error: StreamUpgradeError::Apply(e), }, ), - UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( + outbound_hop::UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( StreamUpgradeError::Apply(Either::Right(e)), ), - UpgradeError::ReservationFailed(_) => { + outbound_hop::UpgradeError::ReservationFailed(_) => { unreachable!("do not emit `ReservationFailed` for connection") } }; @@ -727,7 +719,7 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - self.open_circuit_futs.push(open_circuit(stream).boxed()); + self.open_circuit_futs.push(inbound_stop::open_circuit(stream).boxed()); } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, From 16257048520826b72fda6090885081ca8d54f68f Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 14:06:20 +0300 Subject: [PATCH 11/83] fix rustfmt --- protocols/relay/src/behaviour/handler.rs | 11 +++--- protocols/relay/src/priv_client/handler.rs | 42 +++++++++++++--------- 2 files changed, 32 insertions(+), 21 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 05499b96736..f7bda521c89 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -20,8 +20,8 @@ use crate::behaviour::CircuitId; use crate::copy_future::CopyFuture; -use crate::{HOP_PROTOCOL_NAME, proto, STOP_PROTOCOL_NAME}; use crate::protocol::{inbound_hop, outbound_stop}; +use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; use bytes::Bytes; use either::Either; use futures::channel::oneshot::{self, Canceled}; @@ -30,17 +30,18 @@ use futures::io::AsyncWriteExt; use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; +use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; -use libp2p_swarm::handler::{ - ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound, +use libp2p_swarm::handler::{ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound}; +use libp2p_swarm::{ + ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, + StreamUpgradeError, SubstreamProtocol, }; -use libp2p_swarm::{ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, StreamUpgradeError, SubstreamProtocol}; use std::collections::VecDeque; use std::fmt; use std::task::{Context, Poll}; use std::time::Duration; -use libp2p_core::upgrade::ReadyUpgrade; #[derive(Debug, Clone)] pub struct Config { diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 7627a86101b..2c9b4f7c23d 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -19,8 +19,9 @@ // DEALINGS IN THE SOFTWARE. use crate::priv_client::transport; -use crate::{HOP_PROTOCOL_NAME, proto, STOP_PROTOCOL_NAME}; -use crate::protocol::{self, inbound_stop, MAX_MESSAGE_SIZE, outbound_hop}; +use crate::protocol::{self, inbound_stop, outbound_hop, MAX_MESSAGE_SIZE}; +use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; +use asynchronous_codec::{Framed, FramedParts}; use either::Either; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; @@ -29,19 +30,21 @@ use futures::stream::{FuturesUnordered, StreamExt}; use futures_timer::Delay; use instant::Instant; use libp2p_core::multiaddr::Protocol; +use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::Multiaddr; use libp2p_identity::PeerId; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, }; -use libp2p_swarm::{ConnectionHandler, ConnectionHandlerEvent, KeepAlive, Stream, StreamProtocol, StreamUpgradeError, SubstreamProtocol}; +use libp2p_swarm::{ + ConnectionHandler, ConnectionHandlerEvent, KeepAlive, Stream, StreamProtocol, + StreamUpgradeError, SubstreamProtocol, +}; use log::debug; use std::collections::{HashMap, VecDeque}; use std::fmt; use std::task::{Context, Poll}; use std::time::{Duration, SystemTime}; -use asynchronous_codec::{Framed, FramedParts}; -use libp2p_core::upgrade::ReadyUpgrade; /// The maximum number of circuits being denied concurrently. /// @@ -128,11 +131,14 @@ pub struct Handler { >, wait_for_reserve_outbound_stream: VecDeque>, - reserve_futs: FuturesUnordered>>, + reserve_futs: FuturesUnordered< + BoxFuture<'static, Result>, + >, wait_for_connection_outbound_stream: VecDeque, - circuit_connection_futs: - FuturesUnordered, outbound_hop::UpgradeError>>>, + circuit_connection_futs: FuturesUnordered< + BoxFuture<'static, Result, outbound_hop::UpgradeError>>, + >, reservation: Reservation, @@ -146,8 +152,9 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - open_circuit_futs: - FuturesUnordered>>, + open_circuit_futs: FuturesUnordered< + BoxFuture<'static, Result>, + >, circuit_deny_futs: HashMap>>, @@ -577,11 +584,13 @@ impl ConnectionHandler for Handler { Ok(None) => None, Err(err) => { let res = match err { - outbound_hop::UpgradeError::CircuitFailed(e) => ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(e), - }, - ), + outbound_hop::UpgradeError::CircuitFailed(e) => { + ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }, + ) + } outbound_hop::UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( StreamUpgradeError::Apply(Either::Right(e)), ), @@ -719,7 +728,8 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - self.open_circuit_futs.push(inbound_stop::open_circuit(stream).boxed()); + self.open_circuit_futs + .push(inbound_stop::open_circuit(stream).boxed()); } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, From 77f4aaae2f0a1f19db0b5d889e0a2a97fc1cd369 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 14:09:47 +0300 Subject: [PATCH 12/83] fix review comments --- protocols/relay/src/priv_client/handler.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 2c9b4f7c23d..93a032c6d84 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -266,7 +266,10 @@ impl Handler { "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", src_peer_id ); - } else if self + return; + } + + if self .circuit_deny_futs .insert( src_peer_id, From 4be941ffea829a71a8cc4c71fe98151d495725e7 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 14:55:08 +0300 Subject: [PATCH 13/83] fix review comments --- protocols/relay/src/priv_client/handler.rs | 107 ++----------------- protocols/relay/src/protocol/outbound_hop.rs | 94 +++++++++++++++- 2 files changed, 101 insertions(+), 100 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 93a032c6d84..425db03b6ab 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -44,7 +44,7 @@ use log::debug; use std::collections::{HashMap, VecDeque}; use std::fmt; use std::task::{Context, Poll}; -use std::time::{Duration, SystemTime}; +use std::time::Duration; /// The maximum number of circuits being denied concurrently. /// @@ -281,102 +281,6 @@ impl Handler { } } - fn send_reserve_message_and_process_response( - &self, - protocol: Stream, - to_listener: mpsc::Sender, - ) -> BoxFuture<'static, Result> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::RESERVE, - peer: None, - reservation: None, - limit: None, - status: None, - }; - let mut substream = - Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - - async move { - substream.send(msg).await?; - - let proto::HopMessage { - type_pb, - peer: _, - reservation, - limit, - status, - } = substream - .next() - .await - .ok_or(outbound_hop::FatalUpgradeError::StreamClosed)??; - - match type_pb { - proto::HopMessageType::CONNECT => { - return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeConnect.into()); - } - proto::HopMessageType::RESERVE => { - return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeReserve.into()); - } - proto::HopMessageType::STATUS => {} - } - - let limit = limit.map(Into::into); - - match status.ok_or(outbound_hop::UpgradeError::Fatal( - outbound_hop::FatalUpgradeError::MissingStatusField, - ))? { - proto::Status::OK => {} - proto::Status::RESERVATION_REFUSED => { - return Err(outbound_hop::ReservationFailedReason::Refused.into()); - } - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(outbound_hop::ReservationFailedReason::ResourceLimitExceeded.into()); - } - s => return Err(outbound_hop::FatalUpgradeError::UnexpectedStatus(s).into()), - } - - let reservation = - reservation.ok_or(outbound_hop::FatalUpgradeError::MissingReservationField)?; - - if reservation.addrs.is_empty() { - return Err(outbound_hop::FatalUpgradeError::NoAddressesInReservation.into()); - } - - let addrs = reservation - .addrs - .into_iter() - .map(|b| Multiaddr::try_from(b.to_vec())) - .collect::, _>>() - .map_err(|_| outbound_hop::FatalUpgradeError::InvalidReservationAddrs)?; - - let renewal_timeout = reservation - .expire - .checked_sub( - SystemTime::now() - .duration_since(SystemTime::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(outbound_hop::FatalUpgradeError::InvalidReservationExpiration)?; - - substream.close().await?; - - let output = outbound_hop::Output::Reservation { - renewal_timeout, - addrs, - limit, - to_listener, - }; - - Ok(output) - } - .boxed() - } - fn send_connection_message_and_process_response( &self, protocol: Stream, @@ -739,8 +643,13 @@ impl ConnectionHandler for Handler { .. }) => { if let Some(to_listener) = self.wait_for_reserve_outbound_stream.pop_front() { - self.reserve_futs - .push(self.send_reserve_message_and_process_response(stream, to_listener)); + self.reserve_futs.push( + outbound_hop::send_reserve_message_and_process_response( + stream, + to_listener, + ) + .boxed(), + ); return; } diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 0e3b89789e6..991f29c7469 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -18,15 +18,19 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use asynchronous_codec::Framed; use futures::channel::mpsc; +use futures::prelude::*; use futures_timer::Delay; +use std::time::{Duration, SystemTime}; use thiserror::Error; use libp2p_core::Multiaddr; +use libp2p_swarm::Stream; use crate::priv_client::transport; use crate::proto; -use crate::protocol::Limit; +use crate::protocol::{Limit, MAX_MESSAGE_SIZE}; #[derive(Debug, Error)] pub(crate) enum UpgradeError { @@ -103,3 +107,91 @@ pub(crate) enum Output { limit: Option, }, } + +pub(crate) async fn send_reserve_message_and_process_response( + protocol: Stream, + to_listener: mpsc::Sender, +) -> Result { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::RESERVE, + peer: None, + reservation: None, + limit: None, + status: None, + }; + let mut substream = Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + substream.send(msg).await?; + + let proto::HopMessage { + type_pb, + peer: _, + reservation, + limit, + status, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::HopMessageType::CONNECT => { + return Err(FatalUpgradeError::UnexpectedTypeConnect.into()); + } + proto::HopMessageType::RESERVE => { + return Err(FatalUpgradeError::UnexpectedTypeReserve.into()); + } + proto::HopMessageType::STATUS => {} + } + + let limit = limit.map(Into::into); + + match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + proto::Status::OK => {} + proto::Status::RESERVATION_REFUSED => { + return Err(ReservationFailedReason::Refused.into()); + } + proto::Status::RESOURCE_LIMIT_EXCEEDED => { + return Err(ReservationFailedReason::ResourceLimitExceeded.into()); + } + s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + } + + let reservation = reservation.ok_or(FatalUpgradeError::MissingReservationField)?; + + if reservation.addrs.is_empty() { + return Err(FatalUpgradeError::NoAddressesInReservation.into()); + } + + let addrs = reservation + .addrs + .into_iter() + .map(|b| Multiaddr::try_from(b.to_vec())) + .collect::, _>>() + .map_err(|_| FatalUpgradeError::InvalidReservationAddrs)?; + + let renewal_timeout = reservation + .expire + .checked_sub( + SystemTime::now() + .duration_since(SystemTime::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?; + + let output = Output::Reservation { + renewal_timeout, + addrs, + limit, + to_listener, + }; + + Ok(output) +} From 0f79f76fa527fd3ffc15a681c049089031ab76e1 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 15:21:46 +0300 Subject: [PATCH 14/83] fix review comments --- protocols/relay/src/priv_client.rs | 4 +- protocols/relay/src/priv_client/handler.rs | 132 +++---------------- protocols/relay/src/protocol/outbound_hop.rs | 115 +++++++++++++++- 3 files changed, 132 insertions(+), 119 deletions(-) diff --git a/protocols/relay/src/priv_client.rs b/protocols/relay/src/priv_client.rs index c3c80c5b504..e0129c3ad80 100644 --- a/protocols/relay/src/priv_client.rs +++ b/protocols/relay/src/priv_client.rs @@ -378,10 +378,10 @@ impl NetworkBehaviour for Behaviour { /// /// Internally, this uses a stream to the relay. pub struct Connection { - state: ConnectionState, + pub(crate) state: ConnectionState, } -enum ConnectionState { +pub(crate) enum ConnectionState { InboundAccepting { accept: BoxFuture<'static, Result>, }, diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 425db03b6ab..87faed31515 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -19,9 +19,8 @@ // DEALINGS IN THE SOFTWARE. use crate::priv_client::transport; -use crate::protocol::{self, inbound_stop, outbound_hop, MAX_MESSAGE_SIZE}; +use crate::protocol::{self, inbound_stop, outbound_hop}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; -use asynchronous_codec::{Framed, FramedParts}; use either::Either; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; @@ -37,8 +36,8 @@ use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, }; use libp2p_swarm::{ - ConnectionHandler, ConnectionHandlerEvent, KeepAlive, Stream, StreamProtocol, - StreamUpgradeError, SubstreamProtocol, + ConnectionHandler, ConnectionHandlerEvent, KeepAlive, StreamProtocol, StreamUpgradeError, + SubstreamProtocol, }; use log::debug; use std::collections::{HashMap, VecDeque}; @@ -135,7 +134,7 @@ pub struct Handler { BoxFuture<'static, Result>, >, - wait_for_connection_outbound_stream: VecDeque, + wait_for_connection_outbound_stream: VecDeque, circuit_connection_futs: FuturesUnordered< BoxFuture<'static, Result, outbound_hop::UpgradeError>>, >, @@ -280,104 +279,6 @@ impl Handler { log::warn!("Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", src_peer_id); } } - - fn send_connection_message_and_process_response( - &self, - protocol: Stream, - con_command: ConnectionCommand, - ) -> BoxFuture<'static, Result, outbound_hop::UpgradeError>> { - let msg = proto::HopMessage { - type_pb: proto::HopMessageType::CONNECT, - peer: Some(proto::Peer { - id: con_command.dst_peer_id.to_bytes(), - addrs: vec![], - }), - reservation: None, - limit: None, - status: None, - }; - - let (tx, rx) = oneshot::channel(); - self.alive_lend_out_substreams.push(rx); - - let mut substream = - Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - let remote_peer_id = self.remote_peer_id; - - async move { - substream.send(msg).await?; - let proto::HopMessage { - type_pb, - peer: _, - reservation: _, - limit, - status, - } = substream - .next() - .await - .ok_or(outbound_hop::FatalUpgradeError::StreamClosed)??; - - match type_pb { - proto::HopMessageType::CONNECT => { - return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeConnect.into()) - } - proto::HopMessageType::RESERVE => { - return Err(outbound_hop::FatalUpgradeError::UnexpectedTypeReserve.into()) - } - proto::HopMessageType::STATUS => {} - } - - let limit = limit.map(Into::into); - - match status.ok_or(outbound_hop::UpgradeError::Fatal( - outbound_hop::FatalUpgradeError::MissingStatusField, - ))? { - proto::Status::OK => {} - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(outbound_hop::CircuitFailedReason::ResourceLimitExceeded.into()) - } - proto::Status::CONNECTION_FAILED => { - return Err(outbound_hop::CircuitFailedReason::ConnectionFailed.into()) - } - proto::Status::NO_RESERVATION => { - return Err(outbound_hop::CircuitFailedReason::NoReservation.into()) - } - proto::Status::PERMISSION_DENIED => { - return Err(outbound_hop::CircuitFailedReason::PermissionDenied.into()) - } - s => return Err(outbound_hop::FatalUpgradeError::UnexpectedStatus(s).into()), - } - - let FramedParts { - io, - read_buffer, - write_buffer, - .. - } = substream.into_parts(); - assert!( - write_buffer.is_empty(), - "Expect a flushed Framed to have empty write buffer." - ); - - let output = match con_command.send_back.send(Ok(super::Connection { - state: super::ConnectionState::new_outbound(io, read_buffer.freeze(), tx), - })) { - Ok(()) => Some(outbound_hop::Output::Circuit { limit }), - Err(_) => { - debug!( - "Oneshot to `client::transport::Dial` future dropped. \ - Dropping established relayed connection to {:?}.", - remote_peer_id, - ); - - None - } - }; - - Ok(output) - } - .boxed() - } } impl ConnectionHandler for Handler { @@ -409,10 +310,7 @@ impl ConnectionHandler for Handler { dst_peer_id, } => { self.wait_for_connection_outbound_stream - .push_back(ConnectionCommand { - dst_peer_id, - send_back, - }); + .push_back(outbound_hop::ConnectionCommand::new(dst_peer_id, send_back)); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -656,8 +554,19 @@ impl ConnectionHandler for Handler { let con_command = self.wait_for_connection_outbound_stream.pop_front().expect( "opened a stream without a pending connection command or a reserve listener", ); - self.circuit_connection_futs - .push(self.send_connection_message_and_process_response(stream, con_command)) + + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); + + self.circuit_connection_futs.push( + outbound_hop::send_connection_message_and_process_response( + stream, + self.remote_peer_id, + con_command, + tx, + ) + .boxed(), + ) } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) @@ -670,11 +579,6 @@ impl ConnectionHandler for Handler { } } -struct ConnectionCommand { - dst_peer_id: PeerId, - send_back: oneshot::Sender>, -} - enum Reservation { /// The Reservation is accepted by the relay. Accepted { diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 991f29c7469..8f516c6bb05 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -18,19 +18,22 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use asynchronous_codec::Framed; -use futures::channel::mpsc; +use asynchronous_codec::{Framed, FramedParts}; +use futures::channel::{mpsc, oneshot}; use futures::prelude::*; use futures_timer::Delay; +use log::debug; use std::time::{Duration, SystemTime}; use thiserror::Error; +use void::Void; use libp2p_core::Multiaddr; +use libp2p_identity::PeerId; use libp2p_swarm::Stream; use crate::priv_client::transport; -use crate::proto; use crate::protocol::{Limit, MAX_MESSAGE_SIZE}; +use crate::{priv_client, proto}; #[derive(Debug, Error)] pub(crate) enum UpgradeError { @@ -195,3 +198,109 @@ pub(crate) async fn send_reserve_message_and_process_response( Ok(output) } + +pub(crate) async fn send_connection_message_and_process_response( + protocol: Stream, + remote_peer_id: PeerId, + con_command: ConnectionCommand, + tx: oneshot::Sender, +) -> Result, UpgradeError> { + let msg = proto::HopMessage { + type_pb: proto::HopMessageType::CONNECT, + peer: Some(proto::Peer { + id: con_command.dst_peer_id.to_bytes(), + addrs: vec![], + }), + reservation: None, + limit: None, + status: None, + }; + + let mut substream = Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); + + substream.send(msg).await?; + let proto::HopMessage { + type_pb, + peer: _, + reservation: _, + limit, + status, + } = substream + .next() + .await + .ok_or(FatalUpgradeError::StreamClosed)??; + + match type_pb { + proto::HopMessageType::CONNECT => { + return Err(FatalUpgradeError::UnexpectedTypeConnect.into()); + } + proto::HopMessageType::RESERVE => { + return Err(FatalUpgradeError::UnexpectedTypeReserve.into()); + } + proto::HopMessageType::STATUS => {} + } + + let limit = limit.map(Into::into); + + match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + proto::Status::OK => {} + proto::Status::RESOURCE_LIMIT_EXCEEDED => { + return Err(CircuitFailedReason::ResourceLimitExceeded.into()); + } + proto::Status::CONNECTION_FAILED => { + return Err(CircuitFailedReason::ConnectionFailed.into()); + } + proto::Status::NO_RESERVATION => { + return Err(CircuitFailedReason::NoReservation.into()); + } + proto::Status::PERMISSION_DENIED => { + return Err(CircuitFailedReason::PermissionDenied.into()); + } + s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + } + + let FramedParts { + io, + read_buffer, + write_buffer, + .. + } = substream.into_parts(); + assert!( + write_buffer.is_empty(), + "Expect a flushed Framed to have empty write buffer." + ); + + let output = match con_command.send_back.send(Ok(priv_client::Connection { + state: priv_client::ConnectionState::new_outbound(io, read_buffer.freeze(), tx), + })) { + Ok(()) => Some(Output::Circuit { limit }), + Err(_) => { + debug!( + "Oneshot to `client::transport::Dial` future dropped. \ + Dropping established relayed connection to {:?}.", + remote_peer_id, + ); + + None + } + }; + + Ok(output) +} + +pub(crate) struct ConnectionCommand { + dst_peer_id: PeerId, + pub(crate) send_back: oneshot::Sender>, +} + +impl ConnectionCommand { + pub(crate) fn new( + dst_peer_id: PeerId, + send_back: oneshot::Sender>, + ) -> Self { + Self { + dst_peer_id, + send_back, + } + } +} From 162c9c8a562dfdbc55511760f8265ce55355b9ae Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 15:23:02 +0300 Subject: [PATCH 15/83] ConnectionCommand to Command --- protocols/relay/src/priv_client/handler.rs | 4 ++-- protocols/relay/src/protocol/outbound_hop.rs | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 87faed31515..a120ffbcd5c 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -134,7 +134,7 @@ pub struct Handler { BoxFuture<'static, Result>, >, - wait_for_connection_outbound_stream: VecDeque, + wait_for_connection_outbound_stream: VecDeque, circuit_connection_futs: FuturesUnordered< BoxFuture<'static, Result, outbound_hop::UpgradeError>>, >, @@ -310,7 +310,7 @@ impl ConnectionHandler for Handler { dst_peer_id, } => { self.wait_for_connection_outbound_stream - .push_back(outbound_hop::ConnectionCommand::new(dst_peer_id, send_back)); + .push_back(outbound_hop::Command::new(dst_peer_id, send_back)); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 8f516c6bb05..2819f625436 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -202,7 +202,7 @@ pub(crate) async fn send_reserve_message_and_process_response( pub(crate) async fn send_connection_message_and_process_response( protocol: Stream, remote_peer_id: PeerId, - con_command: ConnectionCommand, + con_command: Command, tx: oneshot::Sender, ) -> Result, UpgradeError> { let msg = proto::HopMessage { @@ -288,12 +288,12 @@ pub(crate) async fn send_connection_message_and_process_response( Ok(output) } -pub(crate) struct ConnectionCommand { +pub(crate) struct Command { dst_peer_id: PeerId, pub(crate) send_back: oneshot::Sender>, } -impl ConnectionCommand { +impl Command { pub(crate) fn new( dst_peer_id: PeerId, send_back: oneshot::Sender>, From fd346a16f74b30533a98d840092101be3c838839 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 15:45:52 +0300 Subject: [PATCH 16/83] rename `fn`s --- protocols/relay/src/behaviour/handler.rs | 4 ++-- protocols/relay/src/priv_client/handler.rs | 4 ++-- protocols/relay/src/protocol/inbound_hop.rs | 2 +- protocols/relay/src/protocol/inbound_stop.rs | 2 +- protocols/relay/src/protocol/outbound_hop.rs | 2 +- protocols/relay/src/protocol/outbound_stop.rs | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index f7bda521c89..987cf43ad91 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -404,7 +404,7 @@ impl Handler { fn on_fully_negotiated_inbound(&self, stream: Stream) { self.protocol_futs.push( - inbound_hop::process_inbound_request( + inbound_hop::handle_inbound_request( stream, self.config.reservation_duration, self.config.max_circuit_duration, @@ -426,7 +426,7 @@ impl Handler { self.alive_lend_out_substreams.push(rx); self.protocol_futs.push( - outbound_stop::send_stop_message_and_process_result(stream, stop_command, tx).boxed(), + outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed(), ); } } diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index a120ffbcd5c..62f4dad5053 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -534,7 +534,7 @@ impl ConnectionHandler for Handler { .. }) => { self.open_circuit_futs - .push(inbound_stop::open_circuit(stream).boxed()); + .push(inbound_stop::handle_open_circuit(stream).boxed()); } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, @@ -559,7 +559,7 @@ impl ConnectionHandler for Handler { self.alive_lend_out_substreams.push(rx); self.circuit_connection_futs.push( - outbound_hop::send_connection_message_and_process_response( + outbound_hop::handle_connection_message_response( stream, self.remote_peer_id, con_command, diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index 21c8d85007f..f6f711aaec1 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -181,7 +181,7 @@ impl CircuitReq { } } -pub(crate) async fn process_inbound_request( +pub(crate) async fn handle_inbound_request( io: Stream, reservation_duration: Duration, max_circuit_duration: Duration, diff --git a/protocols/relay/src/protocol/inbound_stop.rs b/protocols/relay/src/protocol/inbound_stop.rs index a3c74dbd7c8..caaeee9cc53 100644 --- a/protocols/relay/src/protocol/inbound_stop.rs +++ b/protocols/relay/src/protocol/inbound_stop.rs @@ -27,7 +27,7 @@ use libp2p_identity::PeerId; use libp2p_swarm::Stream; use thiserror::Error; -pub(crate) async fn open_circuit(io: Stream) -> Result { +pub(crate) async fn handle_open_circuit(io: Stream) -> Result { let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); let proto::StopMessage { diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 2819f625436..8fbb99ab185 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -199,7 +199,7 @@ pub(crate) async fn send_reserve_message_and_process_response( Ok(output) } -pub(crate) async fn send_connection_message_and_process_response( +pub(crate) async fn handle_connection_message_response( protocol: Stream, remote_peer_id: PeerId, con_command: Command, diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 93f6554c7f4..f4e72191d4e 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -74,7 +74,7 @@ pub enum FatalUpgradeError { UnexpectedStatus(proto::Status), } -pub(crate) async fn send_stop_message_and_process_result( +pub(crate) async fn handle_stop_message_response( io: Stream, stop_command: StopCommand, tx: oneshot::Sender<()>, From cdc6bcfa4eecbef2a854261a656e8812464a337b Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 7 Aug 2023 16:21:27 +0300 Subject: [PATCH 17/83] fix rustfmt --- protocols/relay/src/behaviour/handler.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 987cf43ad91..5336d1a4411 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -425,9 +425,8 @@ impl Handler { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); - self.protocol_futs.push( - outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed(), - ); + self.protocol_futs + .push(outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed()); } } From f71b97eb8af42643ccfcb10b4f5c2895eb400a2f Mon Sep 17 00:00:00 2001 From: dgarus Date: Tue, 8 Aug 2023 14:06:57 +0300 Subject: [PATCH 18/83] fix review comments --- protocols/relay/src/behaviour/handler.rs | 51 ++++++++++++++++++- protocols/relay/src/protocol/outbound_stop.rs | 8 +-- 2 files changed, 53 insertions(+), 6 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 5336d1a4411..4701ab3e4ae 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -33,7 +33,9 @@ use instant::Instant; use libp2p_core::upgrade::ReadyUpgrade; use libp2p_core::{ConnectedPoint, Multiaddr}; use libp2p_identity::PeerId; -use libp2p_swarm::handler::{ConnectionEvent, FullyNegotiatedInbound, FullyNegotiatedOutbound}; +use libp2p_swarm::handler::{ + ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, +}; use libp2p_swarm::{ ConnectionHandler, ConnectionHandlerEvent, ConnectionId, KeepAlive, Stream, StreamProtocol, StreamUpgradeError, SubstreamProtocol, @@ -428,6 +430,49 @@ impl Handler { self.protocol_futs .push(outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed()); } + + fn on_dial_upgrade_error( + &mut self, + DialUpgradeError { error, .. }: DialUpgradeError< + ::OutboundOpenInfo, + ::OutboundProtocol, + >, + ) { + let (non_fatal_error, status) = match error { + StreamUpgradeError::Timeout => ( + StreamUpgradeError::Timeout, + proto::Status::CONNECTION_FAILED, + ), + StreamUpgradeError::NegotiationFailed => { + // The remote has previously done a reservation. Doing a reservation but not + // supporting the stop protocol is pointless, thus disconnecting. + self.pending_error = Some(StreamUpgradeError::NegotiationFailed); + return; + } + StreamUpgradeError::Io(e) => { + self.pending_error = Some(StreamUpgradeError::Io(e)); + return; + } + StreamUpgradeError::Apply(_) => unreachable!("Should not emit handle errors"), + }; + + let stop_command = self + .stop_requested_streams + .pop_front() + .expect("failed to open a stream without a pending stop command"); + + self.queued_events + .push_back(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status, + error: non_fatal_error, + }, + )); + } } enum ReservationRequestFuture { @@ -797,9 +842,11 @@ impl ConnectionHandler for Handler { }) => { self.on_fully_negotiated_outbound(stream); } + ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { + self.on_dial_upgrade_error(dial_upgrade_error); + } ConnectionEvent::AddressChange(_) | ConnectionEvent::ListenUpgradeError(_) - | ConnectionEvent::DialUpgradeError(_) | ConnectionEvent::LocalProtocolsChange(_) | ConnectionEvent::RemoteProtocolsChange(_) => {} } diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index f4e72191d4e..4695778aeb8 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -196,10 +196,10 @@ pub(crate) async fn handle_stop_message_response( } pub(crate) struct StopCommand { - circuit_id: CircuitId, - inbound_circuit_req: inbound_hop::CircuitReq, - src_peer_id: PeerId, - src_connection_id: ConnectionId, + pub(crate) circuit_id: CircuitId, + pub(crate) inbound_circuit_req: inbound_hop::CircuitReq, + pub(crate) src_peer_id: PeerId, + pub(crate) src_connection_id: ConnectionId, max_circuit_duration: Duration, max_circuit_bytes: u64, } From 6116e4dc4c996f72686e62b5ce8aaf8b48825f14 Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 9 Aug 2023 14:40:00 +0300 Subject: [PATCH 19/83] added futures-bounded struct --- Cargo.lock | 12 + Cargo.toml | 7 +- examples/dcutr/src/main.rs | 2 +- misc/futures-bounded/CHANGELOG.md | 3 + misc/futures-bounded/Cargo.toml | 19 ++ misc/futures-bounded/src/lib.rs | 193 ++++++++++++ protocols/autonat/src/behaviour.rs | 8 +- protocols/dcutr/tests/lib.rs | 2 +- protocols/perf/src/client.rs | 7 +- protocols/perf/src/server.rs | 8 +- protocols/relay/Cargo.toml | 1 + protocols/relay/src/priv_client.rs | 31 +- protocols/relay/src/priv_client/handler.rs | 44 ++- protocols/relay/src/protocol/outbound_hop.rs | 2 +- protocols/relay/tests/lib.rs | 2 +- protocols/request-response/CHANGELOG.md | 5 + protocols/request-response/Cargo.toml | 2 + protocols/request-response/src/handler.rs | 298 +++++++++++------- .../request-response/src/handler/protocol.rs | 117 ++----- protocols/request-response/src/lib.rs | 39 ++- swarm/CHANGELOG.md | 1 + 21 files changed, 554 insertions(+), 249 deletions(-) create mode 100644 misc/futures-bounded/CHANGELOG.md create mode 100644 misc/futures-bounded/Cargo.toml create mode 100644 misc/futures-bounded/src/lib.rs diff --git a/Cargo.lock b/Cargo.lock index 5f5026f1afc..cb24cdfb09e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1749,6 +1749,15 @@ dependencies = [ "futures-util", ] +[[package]] +name = "futures-bounded" +version = "0.1.0" +dependencies = [ + "futures-timer", + "futures-util", + "tokio", +] + [[package]] name = "futures-channel" version = "0.3.28" @@ -3093,6 +3102,7 @@ dependencies = [ "either", "env_logger 0.10.0", "futures", + "futures-bounded", "futures-timer", "instant", "libp2p-core", @@ -3150,6 +3160,8 @@ dependencies = [ "cbor4ii", "env_logger 0.10.0", "futures", + "futures-bounded", + "futures-timer", "futures_ringbuf", "instant", "libp2p-core", diff --git a/Cargo.toml b/Cargo.toml index 6b4bd5ebe09..9dbf1c444cc 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,6 +17,7 @@ members = [ "interop-tests", "misc/allow-block-list", "misc/connection-limits", + "misc/futures-bounded", "misc/keygen", "misc/memory-connection-limits", "misc/metrics", @@ -63,6 +64,7 @@ resolver = "2" rust-version = "1.65.0" [workspace.dependencies] +futures-bounded = { version = "0.1.0", path = "misc/futures-bounded" } libp2p-allow-block-list = { version = "0.2.0", path = "misc/allow-block-list" } libp2p-autonat = { version = "0.11.0", path = "protocols/autonat" } libp2p-connection-limits = { version = "0.2.1", path = "misc/connection-limits" } @@ -100,13 +102,12 @@ libp2p-webrtc = { version = "0.6.0-alpha", path = "transports/webrtc" } libp2p-websocket = { version = "0.42.0", path = "transports/websocket" } libp2p-webtransport-websys = { version = "0.1.0", path = "transports/webtransport-websys" } libp2p-yamux = { version = "0.44.1", path = "muxers/yamux" } +multiaddr = "0.18.0" +multihash = "0.19.0" multistream-select = { version = "0.13.0", path = "misc/multistream-select" } quick-protobuf-codec = { version = "0.2.0", path = "misc/quick-protobuf-codec" } quickcheck = { package = "quickcheck-ext", path = "misc/quickcheck-ext" } rw-stream-sink = { version = "0.4.0", path = "misc/rw-stream-sink" } -multiaddr = "0.18.0" -multihash = "0.19.0" - [patch.crates-io] diff --git a/examples/dcutr/src/main.rs b/examples/dcutr/src/main.rs index 8359bb1902a..16c7fb53b99 100644 --- a/examples/dcutr/src/main.rs +++ b/examples/dcutr/src/main.rs @@ -90,7 +90,7 @@ fn main() -> Result<(), Box> { let local_peer_id = PeerId::from(local_key.public()); info!("Local peer id: {:?}", local_peer_id); - let (relay_transport, client) = relay::client::new(local_peer_id); + let (relay_transport, client) = relay::client::new(local_peer_id, Default::default()); let transport = { let relay_tcp_quic_transport = relay_transport diff --git a/misc/futures-bounded/CHANGELOG.md b/misc/futures-bounded/CHANGELOG.md new file mode 100644 index 00000000000..712e5543386 --- /dev/null +++ b/misc/futures-bounded/CHANGELOG.md @@ -0,0 +1,3 @@ +## 0.1.0 - unreleased + +Initial release. diff --git a/misc/futures-bounded/Cargo.toml b/misc/futures-bounded/Cargo.toml new file mode 100644 index 00000000000..8c4af13be82 --- /dev/null +++ b/misc/futures-bounded/Cargo.toml @@ -0,0 +1,19 @@ +[package] +name = "futures-bounded" +version = "0.1.0" +edition = "2021" +rust-version.workspace = true +license = "MIT" +repository = "https://github.com/libp2p/rust-libp2p" +keywords = ["futures", "async", "backpressure"] +categories = ["data-structures", "asynchronous"] +description = "Utilities for bounding futures in size and time." + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +futures-util = { version = "0.3.28" } +futures-timer = "3.0.2" + +[dev-dependencies] +tokio = { version = "1.29.1", features = ["macros", "rt"] } diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs new file mode 100644 index 00000000000..576981e08a1 --- /dev/null +++ b/misc/futures-bounded/src/lib.rs @@ -0,0 +1,193 @@ +use std::future::Future; +use std::task::{Context, Poll, Waker}; +use std::time::Duration; + +use futures_timer::Delay; +use futures_util::future::{select, BoxFuture, Either}; +use futures_util::stream::FuturesUnordered; +use futures_util::{ready, FutureExt, StreamExt}; + +/// Represents a set of (Worker)-[Future]s. +/// +/// This wraps [FuturesUnordered] but bounds it by time and size. +/// In other words, each worker must finish within the specified time and the set never outgrows its capacity. +pub struct WorkerFutures { + timeout: Duration, + capacity: usize, + inner: FuturesUnordered)>>, + + empty_waker: Option, + full_waker: Option, +} + +impl WorkerFutures { + pub fn new(timeout: Duration, capacity: usize) -> Self { + Self { + timeout, + capacity, + inner: Default::default(), + empty_waker: None, + full_waker: None, + } + } +} + +impl WorkerFutures +where + ID: Send + 'static, +{ + pub fn try_push(&mut self, worker_id: ID, worker: F) -> Option + where + F: Future + Send + 'static + Unpin, + { + if self.inner.len() >= self.capacity { + return Some(worker); + } + let timeout = Delay::new(self.timeout); + + self.inner.push( + async move { + match select(worker, timeout).await { + Either::Left((out, _)) => (worker_id, Ok(out)), + Either::Right(((), _)) => (worker_id, Err(Timeout::new())), + } + } + .boxed(), + ); + + if let Some(waker) = self.empty_waker.take() { + waker.wake(); + } + + None + } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { + if self.inner.len() < self.capacity { + return Poll::Ready(()); + } + + self.full_waker = Some(cx.waker().clone()); + Poll::Pending + } + + pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll<(ID, Result)> { + match ready!(self.inner.poll_next_unpin(cx)) { + None => { + self.empty_waker = Some(cx.waker().clone()); + Poll::Pending + } + Some(result) => { + if let Some(waker) = self.full_waker.take() { + waker.wake(); + } + + Poll::Ready(result) + } + } + } +} + +#[derive(Debug)] +pub struct Timeout { + _priv: (), +} + +impl Timeout { + fn new() -> Self { + Self { _priv: () } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::future::{pending, poll_fn, ready}; + use std::pin::Pin; + use std::time::Instant; + + #[test] + fn cannot_push_more_than_capacity_tasks() { + let mut workers = WorkerFutures::new(Duration::from_secs(10), 1); + + assert!(workers.try_push((), ready(())).is_none()); + assert!(workers.try_push((), ready(())).is_some()); + } + + #[tokio::test] + async fn workers_timeout() { + let mut workers = WorkerFutures::new(Duration::from_millis(100), 1); + + let _ = workers.try_push((), pending::<()>()); + Delay::new(Duration::from_millis(150)).await; + let (_, result) = poll_fn(|cx| workers.poll_unpin(cx)).await; + + assert!(result.is_err()) + } + + // Each worker causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. + // We stop after NUM_WORKERS tasks, meaning the overall execution must at least take DELAY * NUM_WORKERS. + #[tokio::test] + async fn backpressure() { + const DELAY: Duration = Duration::from_millis(100); + const NUM_WORKERS: u32 = 10; + + let start = Instant::now(); + Task::new(DELAY, NUM_WORKERS, 1).await; + let duration = start.elapsed(); + + assert!(duration >= DELAY * NUM_WORKERS); + } + + struct Task { + worker: Duration, + num_workers: usize, + num_processed: usize, + inner: WorkerFutures<(), ()>, + } + + impl Task { + fn new(worker: Duration, num_workers: u32, capacity: usize) -> Self { + Self { + worker, + num_workers: num_workers as usize, + num_processed: 0, + inner: WorkerFutures::new(Duration::from_secs(60), capacity), + } + } + } + + impl Future for Task { + type Output = (); + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.get_mut(); + + while this.num_processed < this.num_workers { + if let Poll::Ready(((), result)) = this.inner.poll_unpin(cx) { + if result.is_err() { + panic!("Timeout is great than worker delay") + } + + this.num_processed += 1; + continue; + } + + if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { + let maybe_worker = this.inner.try_push((), Delay::new(this.worker)); + assert!(maybe_worker.is_none(), "we polled for readiness"); + + continue; + } + + return Poll::Pending; + } + + Poll::Ready(()) + } + } +} diff --git a/protocols/autonat/src/behaviour.rs b/protocols/autonat/src/behaviour.rs index 439543f8318..29c2036ff40 100644 --- a/protocols/autonat/src/behaviour.rs +++ b/protocols/autonat/src/behaviour.rs @@ -220,9 +220,11 @@ pub struct Behaviour { impl Behaviour { pub fn new(local_peer_id: PeerId, config: Config) -> Self { let protocols = iter::once((DEFAULT_PROTOCOL_NAME, ProtocolSupport::Full)); - let mut cfg = request_response::Config::default(); - cfg.set_request_timeout(config.timeout); - let inner = request_response::Behaviour::with_codec(AutoNatCodec, protocols, cfg); + let inner = request_response::Behaviour::with_codec( + AutoNatCodec, + protocols, + request_response::Config::default().with_request_timeout(config.timeout), + ); Self { local_peer_id, inner, diff --git a/protocols/dcutr/tests/lib.rs b/protocols/dcutr/tests/lib.rs index 6888e5914a0..90a2ac1a814 100644 --- a/protocols/dcutr/tests/lib.rs +++ b/protocols/dcutr/tests/lib.rs @@ -114,7 +114,7 @@ fn build_client() -> Swarm { let local_public_key = local_key.public(); let local_peer_id = local_public_key.to_peer_id(); - let (relay_transport, behaviour) = relay::client::new(local_peer_id); + let (relay_transport, behaviour) = relay::client::new(local_peer_id, Default::default()); let transport = relay_transport .or_transport(MemoryTransport::default()) diff --git a/protocols/perf/src/client.rs b/protocols/perf/src/client.rs index 93c2086a49e..9f7b80f4871 100644 --- a/protocols/perf/src/client.rs +++ b/protocols/perf/src/client.rs @@ -58,9 +58,6 @@ pub struct Behaviour { impl Default for Behaviour { fn default() -> Self { - let mut req_resp_config = request_response::Config::default(); - req_resp_config.set_connection_keep_alive(Duration::from_secs(60 * 5)); - req_resp_config.set_request_timeout(Duration::from_secs(60 * 5)); Self { connected: Default::default(), request_response: request_response::Behaviour::new( @@ -68,7 +65,9 @@ impl Default for Behaviour { crate::PROTOCOL_NAME, request_response::ProtocolSupport::Outbound, )), - req_resp_config, + request_response::Config::default() + .with_connection_keep_alive(Duration::from_secs(60 * 5)) + .with_request_timeout(Duration::from_secs(60 * 5)), ), } } diff --git a/protocols/perf/src/server.rs b/protocols/perf/src/server.rs index 79f77c74650..f34551f7cb9 100644 --- a/protocols/perf/src/server.rs +++ b/protocols/perf/src/server.rs @@ -37,17 +37,15 @@ pub struct Behaviour { impl Default for Behaviour { fn default() -> Self { - let mut req_resp_config = request_response::Config::default(); - req_resp_config.set_connection_keep_alive(Duration::from_secs(60 * 5)); - req_resp_config.set_request_timeout(Duration::from_secs(60 * 5)); - Self { request_response: request_response::Behaviour::new( std::iter::once(( crate::PROTOCOL_NAME, request_response::ProtocolSupport::Inbound, )), - req_resp_config, + request_response::Config::default() + .with_connection_keep_alive(Duration::from_secs(60 * 5)) + .with_request_timeout(Duration::from_secs(60 * 5)), ), } } diff --git a/protocols/relay/Cargo.toml b/protocols/relay/Cargo.toml index 31f6cc16d1e..a13ba2bb229 100644 --- a/protocols/relay/Cargo.toml +++ b/protocols/relay/Cargo.toml @@ -16,6 +16,7 @@ bytes = "1" either = "1.9.0" futures = "0.3.28" futures-timer = "3" +futures-bounded = { workspace = true } instant = "0.1.12" libp2p-core = { workspace = true } libp2p-swarm = { workspace = true } diff --git a/protocols/relay/src/priv_client.rs b/protocols/relay/src/priv_client.rs index e0129c3ad80..11e84af3214 100644 --- a/protocols/relay/src/priv_client.rs +++ b/protocols/relay/src/priv_client.rs @@ -47,6 +47,7 @@ use std::collections::{hash_map, HashMap, VecDeque}; use std::io::{Error, ErrorKind, IoSlice}; use std::pin::Pin; use std::task::{Context, Poll}; +use std::time::Duration; use transport::Transport; use void::Void; @@ -88,9 +89,26 @@ pub enum Event { }, } +#[derive(Debug, Clone)] +pub struct Config { + pub substream_timeout: Duration, + pub max_concurrent_streams: usize, +} + +impl Default for Config { + fn default() -> Self { + Self { + substream_timeout: Duration::from_secs(10), + max_concurrent_streams: 100, + } + } +} + /// [`NetworkBehaviour`] implementation of the relay client /// functionality of the circuit relay v2 protocol. pub struct Behaviour { + config: Config, + local_peer_id: PeerId, from_transport: Receiver, @@ -105,11 +123,12 @@ pub struct Behaviour { } /// Create a new client relay [`Behaviour`] with it's corresponding [`Transport`]. -pub fn new(local_peer_id: PeerId) -> (Transport, Behaviour) { +pub fn new(local_peer_id: PeerId, config: Config) -> (Transport, Behaviour) { let (transport, from_transport) = Transport::new(); let behaviour = Behaviour { local_peer_id, from_transport, + config, directly_connected_peers: Default::default(), queued_actions: Default::default(), pending_handler_commands: Default::default(), @@ -163,8 +182,12 @@ impl NetworkBehaviour for Behaviour { if local_addr.is_relayed() { return Ok(Either::Right(dummy::ConnectionHandler)); } - - let mut handler = Handler::new(self.local_peer_id, peer, remote_addr.clone()); + let mut handler = Handler::new( + self.config.clone(), + self.local_peer_id, + peer, + remote_addr.clone(), + ); if let Some(event) = self.pending_handler_commands.remove(&connection_id) { handler.on_behaviour_event(event) @@ -184,7 +207,7 @@ impl NetworkBehaviour for Behaviour { return Ok(Either::Right(dummy::ConnectionHandler)); } - let mut handler = Handler::new(self.local_peer_id, peer, addr.clone()); + let mut handler = Handler::new(self.config.clone(), self.local_peer_id, peer, addr.clone()); if let Some(event) = self.pending_handler_commands.remove(&connection_id) { handler.on_behaviour_event(event) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 62f4dad5053..a8dac5d3969 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::priv_client::transport; +use crate::priv_client::{transport, Config}; use crate::protocol::{self, inbound_stop, outbound_hop}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; use either::Either; @@ -130,8 +130,9 @@ pub struct Handler { >, wait_for_reserve_outbound_stream: VecDeque>, - reserve_futs: FuturesUnordered< - BoxFuture<'static, Result>, + reserve_futs: futures_bounded::WorkerFutures< + (), + Result, >, wait_for_connection_outbound_stream: VecDeque, @@ -165,7 +166,12 @@ pub struct Handler { } impl Handler { - pub fn new(local_peer_id: PeerId, remote_peer_id: PeerId, remote_addr: Multiaddr) -> Self { + pub fn new( + config: Config, + local_peer_id: PeerId, + remote_peer_id: PeerId, + remote_addr: Multiaddr, + ) -> Self { Self { local_peer_id, remote_peer_id, @@ -173,7 +179,10 @@ impl Handler { queued_events: Default::default(), pending_error: Default::default(), wait_for_reserve_outbound_stream: Default::default(), - reserve_futs: Default::default(), + reserve_futs: futures_bounded::WorkerFutures::new( + config.substream_timeout, + config.max_concurrent_streams, + ), wait_for_connection_outbound_stream: Default::default(), circuit_connection_futs: Default::default(), reservation: Reservation::None, @@ -341,7 +350,14 @@ impl ConnectionHandler for Handler { } // Reservations - if let Poll::Ready(Some(result)) = self.reserve_futs.poll_next_unpin(cx) { + if let Poll::Ready((_, worker_res)) = self.reserve_futs.poll_unpin(cx) { + let result = match worker_res { + Ok(r) => r, + Err(futures_bounded::Timeout { .. }) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + }; + let event = match result { Ok(outbound_hop::Output::Reservation { renewal_timeout, @@ -541,13 +557,17 @@ impl ConnectionHandler for Handler { .. }) => { if let Some(to_listener) = self.wait_for_reserve_outbound_stream.pop_front() { - self.reserve_futs.push( - outbound_hop::send_reserve_message_and_process_response( - stream, - to_listener, + if self + .reserve_futs + .try_push( + (), + outbound_hop::handle_reserve_message_response(stream, to_listener) + .boxed(), ) - .boxed(), - ); + .is_some() + { + log::warn!("Dropping outbound stream because we are at capacity") + } return; } diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 8fbb99ab185..86276f664cb 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -111,7 +111,7 @@ pub(crate) enum Output { }, } -pub(crate) async fn send_reserve_message_and_process_response( +pub(crate) async fn handle_reserve_message_response( protocol: Stream, to_listener: mpsc::Sender, ) -> Result { diff --git a/protocols/relay/tests/lib.rs b/protocols/relay/tests/lib.rs index fa79ab67f4b..9663cd4bf77 100644 --- a/protocols/relay/tests/lib.rs +++ b/protocols/relay/tests/lib.rs @@ -334,7 +334,7 @@ fn build_client() -> Swarm { let local_public_key = local_key.public(); let local_peer_id = local_public_key.to_peer_id(); - let (relay_transport, behaviour) = relay::client::new(local_peer_id); + let (relay_transport, behaviour) = relay::client::new(local_peer_id, Default::default()); let transport = upgrade_transport( OrTransport::new(relay_transport, MemoryTransport::default()).boxed(), local_public_key, diff --git a/protocols/request-response/CHANGELOG.md b/protocols/request-response/CHANGELOG.md index 292cb812103..eead6711f01 100644 --- a/protocols/request-response/CHANGELOG.md +++ b/protocols/request-response/CHANGELOG.md @@ -3,7 +3,12 @@ - Replace unmaintained `serde_cbor` dependency with `cbor4ii`. See [PR 4187]. +- Allow at most 100 concurrent inbound + outbound streams. + This limit is configurable via `Config::with_max_concurrent_streams`. + See [PR 3914]. + [PR 4187]: https://github.com/libp2p/rust-libp2p/pull/4187 +[PR 3914]: https://github.com/libp2p/rust-libp2p/pull/3914 ## 0.25.0 diff --git a/protocols/request-response/Cargo.toml b/protocols/request-response/Cargo.toml index ddb87316f86..b10e8e7951f 100644 --- a/protocols/request-response/Cargo.toml +++ b/protocols/request-response/Cargo.toml @@ -24,6 +24,8 @@ serde_json = { version = "1.0.100", optional = true } smallvec = "1.11.0" void = "1.0.2" log = "0.4.19" +futures-timer = "3.0.2" +futures-bounded = { workspace = true } [features] json = ["dep:serde", "dep:serde_json", "libp2p-swarm/macros"] diff --git a/protocols/request-response/src/handler.rs b/protocols/request-response/src/handler.rs index 35a2db98bdc..af40fb45f20 100644 --- a/protocols/request-response/src/handler.rs +++ b/protocols/request-response/src/handler.rs @@ -23,10 +23,11 @@ pub(crate) mod protocol; pub use protocol::ProtocolSupport; use crate::codec::Codec; -use crate::handler::protocol::{RequestProtocol, ResponseProtocol}; +use crate::handler::protocol::Protocol; use crate::{RequestId, EMPTY_QUEUE_SHRINK_THRESHOLD}; -use futures::{channel::oneshot, future::BoxFuture, prelude::*, stream::FuturesUnordered}; +use futures::channel::mpsc; +use futures::{channel::oneshot, prelude::*}; use instant::Instant; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, @@ -39,7 +40,7 @@ use libp2p_swarm::{ use smallvec::SmallVec; use std::{ collections::VecDeque, - fmt, + fmt, io, sync::{ atomic::{AtomicU64, Ordering}, Arc, @@ -68,21 +69,25 @@ where /// Queue of events to emit in `poll()`. pending_events: VecDeque>, /// Outbound upgrades waiting to be emitted as an `OutboundSubstreamRequest`. - outbound: VecDeque>, - /// Inbound upgrades waiting for the incoming request. - inbound: FuturesUnordered< - BoxFuture< - 'static, - Result< - ( - (RequestId, TCodec::Request), - oneshot::Sender, - ), - oneshot::Canceled, - >, - >, - >, + pending_outbound: VecDeque>, + + requested_outbound: VecDeque>, + /// A channel for receiving inbound requests. + inbound_receiver: mpsc::Receiver<( + RequestId, + TCodec::Request, + oneshot::Sender, + )>, + /// The [`mpsc::Sender`] for the above receiver. Cloned for each inbound request. + inbound_sender: mpsc::Sender<( + RequestId, + TCodec::Request, + oneshot::Sender, + )>, + inbound_request_id: Arc, + + worker_streams: futures_bounded::WorkerFutures, io::Error>>, } impl Handler @@ -95,49 +100,132 @@ where keep_alive_timeout: Duration, substream_timeout: Duration, inbound_request_id: Arc, + max_concurrent_streams: usize, ) -> Self { + let (inbound_sender, inbound_receiver) = mpsc::channel(0); Self { inbound_protocols, codec, keep_alive: KeepAlive::Yes, keep_alive_timeout, substream_timeout, - outbound: VecDeque::new(), - inbound: FuturesUnordered::new(), + pending_outbound: VecDeque::new(), + requested_outbound: Default::default(), + inbound_receiver, + inbound_sender, pending_events: VecDeque::new(), inbound_request_id, + worker_streams: futures_bounded::WorkerFutures::new( + substream_timeout, + max_concurrent_streams, + ), } } fn on_fully_negotiated_inbound( &mut self, FullyNegotiatedInbound { - protocol: sent, - info: request_id, + protocol: (mut stream, protocol), + info: (), }: FullyNegotiatedInbound< ::InboundProtocol, ::InboundOpenInfo, >, ) { - if sent { - self.pending_events - .push_back(Event::ResponseSent(request_id)) - } else { - self.pending_events - .push_back(Event::ResponseOmission(request_id)) + let mut codec = self.codec.clone(); + let request_id = RequestId(self.inbound_request_id.fetch_add(1, Ordering::Relaxed)); + let mut sender = self.inbound_sender.clone(); + + let recv = async move { + // A channel for notifying the inbound upgrade when the + // response is sent. + let (rs_send, rs_recv) = oneshot::channel(); + + let read = codec.read_request(&protocol, &mut stream); + let request = read.await?; + sender + .send((request_id, request, rs_send)) + .await + .expect("`ConnectionHandler` owns both ends of the channel"); + drop(sender); + + if let Ok(response) = rs_recv.await { + let write = codec.write_response(&protocol, &mut stream, response); + write.await?; + + stream.close().await?; + Ok(Event::ResponseSent(request_id)) + } else { + stream.close().await?; + Ok(Event::ResponseOmission(request_id)) + } + }; + + if self + .worker_streams + .try_push(request_id, recv.boxed()) + .is_some() + { + log::warn!("Dropping inbound stream because we are at capacity") + } + } + + fn on_fully_negotiated_outbound( + &mut self, + FullyNegotiatedOutbound { + protocol: (mut stream, protocol), + info: (), + }: FullyNegotiatedOutbound< + ::OutboundProtocol, + ::OutboundOpenInfo, + >, + ) { + let message = self + .requested_outbound + .pop_front() + .expect("negotiated a stream without a pending message"); + + let mut codec = self.codec.clone(); + let request_id = message.request_id; + + let send = async move { + let write = codec.write_request(&protocol, &mut stream, message.request); + write.await?; + stream.close().await?; + let read = codec.read_response(&protocol, &mut stream); + let response = read.await?; + + Ok(Event::Response { + request_id, + response, + }) + }; + + if self + .worker_streams + .try_push(request_id, send.boxed()) + .is_some() + { + log::warn!("Dropping outbound stream because we are at capacity") } } fn on_dial_upgrade_error( &mut self, - DialUpgradeError { info, error }: DialUpgradeError< + DialUpgradeError { error, info: () }: DialUpgradeError< ::OutboundOpenInfo, ::OutboundProtocol, >, ) { + let message = self + .requested_outbound + .pop_front() + .expect("negotiated a stream without a pending message"); + match error { StreamUpgradeError::Timeout => { - self.pending_events.push_back(Event::OutboundTimeout(info)); + self.pending_events + .push_back(Event::OutboundTimeout(message.request_id)); } StreamUpgradeError::NegotiationFailed => { // The remote merely doesn't support the protocol(s) we requested. @@ -146,24 +234,26 @@ where // An event is reported to permit user code to react to the fact that // the remote peer does not support the requested protocol(s). self.pending_events - .push_back(Event::OutboundUnsupportedProtocols(info)); - } - StreamUpgradeError::Apply(e) => { - log::debug!("outbound stream {info} failed: {e}"); + .push_back(Event::OutboundUnsupportedProtocols(message.request_id)); } + StreamUpgradeError::Apply(e) => void::unreachable(e), StreamUpgradeError::Io(e) => { - log::debug!("outbound stream {info} failed: {e}"); + log::debug!( + "outbound stream for request {} failed: {e}, retrying", + message.request_id + ); + self.requested_outbound.push_back(message); } } } fn on_listen_upgrade_error( &mut self, - ListenUpgradeError { error, info }: ListenUpgradeError< + ListenUpgradeError { error, .. }: ListenUpgradeError< ::InboundOpenInfo, ::InboundProtocol, >, ) { - log::debug!("inbound stream {info} failed: {error}"); + void::unreachable(error) } } @@ -233,55 +323,45 @@ impl fmt::Debug for Event { } } +pub struct OutboundMessage { + pub(crate) request_id: RequestId, + pub(crate) request: TCodec::Request, + pub(crate) protocols: SmallVec<[TCodec::Protocol; 2]>, +} + +impl fmt::Debug for OutboundMessage +where + TCodec: Codec, +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("OutboundMessage").finish_non_exhaustive() + } +} + impl ConnectionHandler for Handler where TCodec: Codec + Send + Clone + 'static, { - type FromBehaviour = RequestProtocol; + type FromBehaviour = OutboundMessage; type ToBehaviour = Event; type Error = void::Void; - type InboundProtocol = ResponseProtocol; - type OutboundProtocol = RequestProtocol; - type OutboundOpenInfo = RequestId; - type InboundOpenInfo = RequestId; + type InboundProtocol = Protocol; + type OutboundProtocol = Protocol; + type OutboundOpenInfo = (); + type InboundOpenInfo = (); fn listen_protocol(&self) -> SubstreamProtocol { - // A channel for notifying the handler when the inbound - // upgrade received the request. - let (rq_send, rq_recv) = oneshot::channel(); - - // A channel for notifying the inbound upgrade when the - // response is sent. - let (rs_send, rs_recv) = oneshot::channel(); - - let request_id = RequestId(self.inbound_request_id.fetch_add(1, Ordering::Relaxed)); - - // By keeping all I/O inside the `ResponseProtocol` and thus the - // inbound substream upgrade via above channels, we ensure that it - // is all subject to the configured timeout without extra bookkeeping - // for inbound substreams as well as their timeouts and also make the - // implementation of inbound and outbound upgrades symmetric in - // this sense. - let proto = ResponseProtocol { - protocols: self.inbound_protocols.clone(), - codec: self.codec.clone(), - request_sender: rq_send, - response_receiver: rs_recv, - request_id, - }; - - // The handler waits for the request to come in. It then emits - // `Event::Request` together with a - // `ResponseChannel`. - self.inbound - .push(rq_recv.map_ok(move |rq| (rq, rs_send)).boxed()); - - SubstreamProtocol::new(proto, request_id).with_timeout(self.substream_timeout) + SubstreamProtocol::new( + Protocol { + protocols: self.inbound_protocols.clone(), + }, + (), + ) } fn on_behaviour_event(&mut self, request: Self::FromBehaviour) { self.keep_alive = KeepAlive::Yes; - self.outbound.push_back(request); + self.pending_outbound.push_back(request); } fn connection_keep_alive(&self) -> KeepAlive { @@ -291,9 +371,23 @@ where fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll< - ConnectionHandlerEvent, RequestId, Self::ToBehaviour, Self::Error>, - > { + ) -> Poll, (), Self::ToBehaviour, Self::Error>> + { + loop { + match self.worker_streams.poll_unpin(cx) { + Poll::Ready((_, Ok(Ok(event)))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)) + } + Poll::Ready((id, Ok(Err(e)))) => { + log::debug!("Stream for request {id} failed: {e}"); + } + Poll::Ready((id, Err(futures_bounded::Timeout { .. }))) => { + log::debug!("Stream for request {id} timed out"); + } + Poll::Pending => break, + } + } + // Drain pending events. if let Some(event) = self.pending_events.pop_front() { return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)); @@ -302,41 +396,33 @@ where } // Check for inbound requests. - while let Poll::Ready(Some(result)) = self.inbound.poll_next_unpin(cx) { - match result { - Ok(((id, rq), rs_sender)) => { - // We received an inbound request. - self.keep_alive = KeepAlive::Yes; - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(Event::Request { - request_id: id, - request: rq, - sender: rs_sender, - })); - } - Err(oneshot::Canceled) => { - // The inbound upgrade has errored or timed out reading - // or waiting for the request. The handler is informed - // via `on_connection_event` call with `ConnectionEvent::ListenUpgradeError`. - } - } + if let Poll::Ready(Some((id, rq, rs_sender))) = self.inbound_receiver.poll_next_unpin(cx) { + // We received an inbound request. + self.keep_alive = KeepAlive::Yes; + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(Event::Request { + request_id: id, + request: rq, + sender: rs_sender, + })); } // Emit outbound requests. - if let Some(request) = self.outbound.pop_front() { - let info = request.request_id; + if let Some(request) = self.pending_outbound.pop_front() { + let protocols = request.protocols.clone(); + self.requested_outbound.push_back(request); + return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { - protocol: SubstreamProtocol::new(request, info) - .with_timeout(self.substream_timeout), + protocol: SubstreamProtocol::new(Protocol { protocols }, ()), }); } - debug_assert!(self.outbound.is_empty()); + debug_assert!(self.pending_outbound.is_empty()); - if self.outbound.capacity() > EMPTY_QUEUE_SHRINK_THRESHOLD { - self.outbound.shrink_to_fit(); + if self.pending_outbound.capacity() > EMPTY_QUEUE_SHRINK_THRESHOLD { + self.pending_outbound.shrink_to_fit(); } - if self.inbound.is_empty() && self.keep_alive.is_yes() { + if self.worker_streams.is_empty() && self.keep_alive.is_yes() { // No new inbound or outbound requests. However, we may just have // started the latest inbound or outbound upgrade(s), so make sure // the keep-alive timeout is preceded by the substream timeout. @@ -360,14 +446,8 @@ where ConnectionEvent::FullyNegotiatedInbound(fully_negotiated_inbound) => { self.on_fully_negotiated_inbound(fully_negotiated_inbound) } - ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { - protocol: response, - info: request_id, - }) => { - self.pending_events.push_back(Event::Response { - request_id, - response, - }); + ConnectionEvent::FullyNegotiatedOutbound(fully_negotiated_outbound) => { + self.on_fully_negotiated_outbound(fully_negotiated_outbound) } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) diff --git a/protocols/request-response/src/handler/protocol.rs b/protocols/request-response/src/handler/protocol.rs index 1368a3c1f98..833cacdd6ce 100644 --- a/protocols/request-response/src/handler/protocol.rs +++ b/protocols/request-response/src/handler/protocol.rs @@ -23,14 +23,10 @@ //! receives a request and sends a response, whereas the //! outbound upgrade send a request and receives a response. -use crate::codec::Codec; -use crate::RequestId; - -use futures::{channel::oneshot, future::BoxFuture, prelude::*}; +use futures::future::{ready, Ready}; use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use libp2p_swarm::Stream; use smallvec::SmallVec; -use std::{fmt, io}; /// The level of support for a particular protocol. #[derive(Debug, Clone)] @@ -65,22 +61,15 @@ impl ProtocolSupport { /// /// Receives a request and sends a response. #[derive(Debug)] -pub struct ResponseProtocol -where - TCodec: Codec, -{ - pub(crate) codec: TCodec, - pub(crate) protocols: SmallVec<[TCodec::Protocol; 2]>, - pub(crate) request_sender: oneshot::Sender<(RequestId, TCodec::Request)>, - pub(crate) response_receiver: oneshot::Receiver, - pub(crate) request_id: RequestId, +pub struct Protocol

{ + pub(crate) protocols: SmallVec<[P; 2]>, } -impl UpgradeInfo for ResponseProtocol +impl

UpgradeInfo for Protocol

where - TCodec: Codec, + P: AsRef + Clone, { - type Info = TCodec::Protocol; + type Info = P; type InfoIter = smallvec::IntoIter<[Self::Info; 2]>; fn protocol_info(&self) -> Self::InfoIter { @@ -88,94 +77,28 @@ where } } -impl InboundUpgrade for ResponseProtocol +impl

InboundUpgrade for Protocol

where - TCodec: Codec + Send + 'static, + P: AsRef + Clone, { - type Output = bool; - type Error = io::Error; - type Future = BoxFuture<'static, Result>; - - fn upgrade_inbound(mut self, mut io: Stream, protocol: Self::Info) -> Self::Future { - async move { - let read = self.codec.read_request(&protocol, &mut io); - let request = read.await?; - match self.request_sender.send((self.request_id, request)) { - Ok(()) => {}, - Err(_) => panic!( - "Expect request receiver to be alive i.e. protocol handler to be alive.", - ), - } + type Output = (Stream, P); + type Error = void::Void; + type Future = Ready>; - if let Ok(response) = self.response_receiver.await { - let write = self.codec.write_response(&protocol, &mut io, response); - write.await?; - - io.close().await?; - // Response was sent. Indicate to handler to emit a `ResponseSent` event. - Ok(true) - } else { - io.close().await?; - // No response was sent. Indicate to handler to emit a `ResponseOmission` event. - Ok(false) - } - }.boxed() + fn upgrade_inbound(self, io: Stream, protocol: Self::Info) -> Self::Future { + ready(Ok((io, protocol))) } } -/// Request substream upgrade protocol. -/// -/// Sends a request and receives a response. -pub struct RequestProtocol +impl

OutboundUpgrade for Protocol

where - TCodec: Codec, + P: AsRef + Clone, { - pub(crate) codec: TCodec, - pub(crate) protocols: SmallVec<[TCodec::Protocol; 2]>, - pub(crate) request_id: RequestId, - pub(crate) request: TCodec::Request, -} + type Output = (Stream, P); + type Error = void::Void; + type Future = Ready>; -impl fmt::Debug for RequestProtocol -where - TCodec: Codec, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("RequestProtocol") - .field("request_id", &self.request_id) - .finish() - } -} - -impl UpgradeInfo for RequestProtocol -where - TCodec: Codec, -{ - type Info = TCodec::Protocol; - type InfoIter = smallvec::IntoIter<[Self::Info; 2]>; - - fn protocol_info(&self) -> Self::InfoIter { - self.protocols.clone().into_iter() - } -} - -impl OutboundUpgrade for RequestProtocol -where - TCodec: Codec + Send + 'static, -{ - type Output = TCodec::Response; - type Error = io::Error; - type Future = BoxFuture<'static, Result>; - - fn upgrade_outbound(mut self, mut io: Stream, protocol: Self::Info) -> Self::Future { - async move { - let write = self.codec.write_request(&protocol, &mut io, self.request); - write.await?; - io.close().await?; - let read = self.codec.read_response(&protocol, &mut io); - let response = read.await?; - Ok(response) - } - .boxed() + fn upgrade_outbound(self, io: Stream, protocol: Self::Info) -> Self::Future { + ready(Ok((io, protocol))) } } diff --git a/protocols/request-response/src/lib.rs b/protocols/request-response/src/lib.rs index 7b1a8088443..7e81f6f5b5c 100644 --- a/protocols/request-response/src/lib.rs +++ b/protocols/request-response/src/lib.rs @@ -76,7 +76,7 @@ pub mod json; pub use codec::Codec; pub use handler::ProtocolSupport; -use crate::handler::protocol::RequestProtocol; +use crate::handler::OutboundMessage; use futures::channel::oneshot; use handler::Handler; use libp2p_core::{ConnectedPoint, Endpoint, Multiaddr}; @@ -285,6 +285,7 @@ impl fmt::Display for RequestId { pub struct Config { request_timeout: Duration, connection_keep_alive: Duration, + max_concurrent_streams: usize, } impl Default for Config { @@ -292,22 +293,43 @@ impl Default for Config { Self { connection_keep_alive: Duration::from_secs(10), request_timeout: Duration::from_secs(10), + max_concurrent_streams: 100, } } } impl Config { /// Sets the keep-alive timeout of idle connections. + #[deprecated(note = "Use `Config::with_connection_keep_alive` for one-liner constructions.")] pub fn set_connection_keep_alive(&mut self, v: Duration) -> &mut Self { self.connection_keep_alive = v; self } /// Sets the timeout for inbound and outbound requests. + #[deprecated(note = "Use `Config::with_request_timeout` for one-liner constructions.")] pub fn set_request_timeout(&mut self, v: Duration) -> &mut Self { self.request_timeout = v; self } + + /// Sets the keep-alive timeout of idle connections. + pub fn with_connection_keep_alive(mut self, v: Duration) -> Self { + self.connection_keep_alive = v; + self + } + + /// Sets the timeout for inbound and outbound requests. + pub fn with_request_timeout(mut self, v: Duration) -> Self { + self.request_timeout = v; + self + } + + /// Sets the upper bound for the number of concurrent inbound + outbound streams. + pub fn with_max_concurrent_streams(mut self, num_streams: usize) -> Self { + self.max_concurrent_streams = num_streams; + self + } } /// A request/response protocol for some message codec. @@ -329,7 +351,7 @@ where codec: TCodec, /// Pending events to return from `poll`. pending_events: - VecDeque, RequestProtocol>>, + VecDeque, OutboundMessage>>, /// The currently connected peers, their pending outbound and inbound responses and their known, /// reachable addresses, if any. connected: HashMap>, @@ -337,7 +359,7 @@ where addresses: HashMap>, /// Requests that have not yet been sent and are waiting for a connection /// to be established. - pending_outbound_requests: HashMap; 10]>>, + pending_outbound_requests: HashMap; 10]>>, } impl Behaviour @@ -401,11 +423,10 @@ where /// > [`Behaviour::remove_address`]. pub fn send_request(&mut self, peer: &PeerId, request: TCodec::Request) -> RequestId { let request_id = self.next_request_id(); - let request = RequestProtocol { + let request = OutboundMessage { request_id, - codec: self.codec.clone(), - protocols: self.outbound_protocols.clone(), request, + protocols: self.outbound_protocols.clone(), }; if let Some(request) = self.try_send_request(peer, request) { @@ -519,8 +540,8 @@ where fn try_send_request( &mut self, peer: &PeerId, - request: RequestProtocol, - ) -> Option> { + request: OutboundMessage, + ) -> Option> { if let Some(connections) = self.connected.get_mut(peer) { if connections.is_empty() { return Some(request); @@ -723,6 +744,7 @@ where self.config.connection_keep_alive, self.config.request_timeout, self.next_inbound_id.clone(), + self.config.max_concurrent_streams, )) } @@ -762,6 +784,7 @@ where self.config.connection_keep_alive, self.config.request_timeout, self.next_inbound_id.clone(), + self.config.max_concurrent_streams, )) } diff --git a/swarm/CHANGELOG.md b/swarm/CHANGELOG.md index 6a9808dd554..d703aef639a 100644 --- a/swarm/CHANGELOG.md +++ b/swarm/CHANGELOG.md @@ -22,6 +22,7 @@ ## 0.43.0 - Allow `NetworkBehaviours` to create and remove listeners. +- See [PR 3292]. - Raise MSRV to 1.65. From 13b690c2ac2c0211133d775046a54506d902856d Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 9 Aug 2023 14:50:37 +0300 Subject: [PATCH 20/83] added futures-bounded struct --- protocols/relay/src/priv_client/transport.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/protocols/relay/src/priv_client/transport.rs b/protocols/relay/src/priv_client/transport.rs index 45cc685aea0..8a63ca5a80a 100644 --- a/protocols/relay/src/priv_client/transport.rs +++ b/protocols/relay/src/priv_client/transport.rs @@ -55,7 +55,7 @@ use thiserror::Error; /// # use libp2p_identity::PeerId; /// let actual_transport = MemoryTransport::default(); /// let (relay_transport, behaviour) = relay::client::new( -/// PeerId::random(), +/// PeerId::random(), Default::default() /// ); /// let mut transport = OrTransport::new(relay_transport, actual_transport); /// # let relay_id = PeerId::random(); @@ -80,7 +80,7 @@ use thiserror::Error; /// # let local_peer_id = PeerId::random(); /// let actual_transport = MemoryTransport::default(); /// let (relay_transport, behaviour) = relay::client::new( -/// local_peer_id, +/// local_peer_id, Default::default() /// ); /// let mut transport = OrTransport::new(relay_transport, actual_transport); /// let relay_addr = Multiaddr::empty() From 4e1bee09e695526f1665cac9b1840c0408e8093f Mon Sep 17 00:00:00 2001 From: dgarus Date: Tue, 22 Aug 2023 15:54:40 +0300 Subject: [PATCH 21/83] Removed extra changes --- protocols/autonat/src/behaviour.rs | 8 +- protocols/perf/src/client.rs | 7 +- protocols/perf/src/server.rs | 8 +- protocols/request-response/CHANGELOG.md | 5 - protocols/request-response/Cargo.toml | 2 - protocols/request-response/src/handler.rs | 298 +++++++----------- .../request-response/src/handler/protocol.rs | 117 +++++-- protocols/request-response/src/lib.rs | 39 +-- swarm/CHANGELOG.md | 1 - 9 files changed, 226 insertions(+), 259 deletions(-) diff --git a/protocols/autonat/src/behaviour.rs b/protocols/autonat/src/behaviour.rs index 29c2036ff40..439543f8318 100644 --- a/protocols/autonat/src/behaviour.rs +++ b/protocols/autonat/src/behaviour.rs @@ -220,11 +220,9 @@ pub struct Behaviour { impl Behaviour { pub fn new(local_peer_id: PeerId, config: Config) -> Self { let protocols = iter::once((DEFAULT_PROTOCOL_NAME, ProtocolSupport::Full)); - let inner = request_response::Behaviour::with_codec( - AutoNatCodec, - protocols, - request_response::Config::default().with_request_timeout(config.timeout), - ); + let mut cfg = request_response::Config::default(); + cfg.set_request_timeout(config.timeout); + let inner = request_response::Behaviour::with_codec(AutoNatCodec, protocols, cfg); Self { local_peer_id, inner, diff --git a/protocols/perf/src/client.rs b/protocols/perf/src/client.rs index 9f7b80f4871..93c2086a49e 100644 --- a/protocols/perf/src/client.rs +++ b/protocols/perf/src/client.rs @@ -58,6 +58,9 @@ pub struct Behaviour { impl Default for Behaviour { fn default() -> Self { + let mut req_resp_config = request_response::Config::default(); + req_resp_config.set_connection_keep_alive(Duration::from_secs(60 * 5)); + req_resp_config.set_request_timeout(Duration::from_secs(60 * 5)); Self { connected: Default::default(), request_response: request_response::Behaviour::new( @@ -65,9 +68,7 @@ impl Default for Behaviour { crate::PROTOCOL_NAME, request_response::ProtocolSupport::Outbound, )), - request_response::Config::default() - .with_connection_keep_alive(Duration::from_secs(60 * 5)) - .with_request_timeout(Duration::from_secs(60 * 5)), + req_resp_config, ), } } diff --git a/protocols/perf/src/server.rs b/protocols/perf/src/server.rs index f34551f7cb9..79f77c74650 100644 --- a/protocols/perf/src/server.rs +++ b/protocols/perf/src/server.rs @@ -37,15 +37,17 @@ pub struct Behaviour { impl Default for Behaviour { fn default() -> Self { + let mut req_resp_config = request_response::Config::default(); + req_resp_config.set_connection_keep_alive(Duration::from_secs(60 * 5)); + req_resp_config.set_request_timeout(Duration::from_secs(60 * 5)); + Self { request_response: request_response::Behaviour::new( std::iter::once(( crate::PROTOCOL_NAME, request_response::ProtocolSupport::Inbound, )), - request_response::Config::default() - .with_connection_keep_alive(Duration::from_secs(60 * 5)) - .with_request_timeout(Duration::from_secs(60 * 5)), + req_resp_config, ), } } diff --git a/protocols/request-response/CHANGELOG.md b/protocols/request-response/CHANGELOG.md index eead6711f01..292cb812103 100644 --- a/protocols/request-response/CHANGELOG.md +++ b/protocols/request-response/CHANGELOG.md @@ -3,12 +3,7 @@ - Replace unmaintained `serde_cbor` dependency with `cbor4ii`. See [PR 4187]. -- Allow at most 100 concurrent inbound + outbound streams. - This limit is configurable via `Config::with_max_concurrent_streams`. - See [PR 3914]. - [PR 4187]: https://github.com/libp2p/rust-libp2p/pull/4187 -[PR 3914]: https://github.com/libp2p/rust-libp2p/pull/3914 ## 0.25.0 diff --git a/protocols/request-response/Cargo.toml b/protocols/request-response/Cargo.toml index b10e8e7951f..ddb87316f86 100644 --- a/protocols/request-response/Cargo.toml +++ b/protocols/request-response/Cargo.toml @@ -24,8 +24,6 @@ serde_json = { version = "1.0.100", optional = true } smallvec = "1.11.0" void = "1.0.2" log = "0.4.19" -futures-timer = "3.0.2" -futures-bounded = { workspace = true } [features] json = ["dep:serde", "dep:serde_json", "libp2p-swarm/macros"] diff --git a/protocols/request-response/src/handler.rs b/protocols/request-response/src/handler.rs index af40fb45f20..35a2db98bdc 100644 --- a/protocols/request-response/src/handler.rs +++ b/protocols/request-response/src/handler.rs @@ -23,11 +23,10 @@ pub(crate) mod protocol; pub use protocol::ProtocolSupport; use crate::codec::Codec; -use crate::handler::protocol::Protocol; +use crate::handler::protocol::{RequestProtocol, ResponseProtocol}; use crate::{RequestId, EMPTY_QUEUE_SHRINK_THRESHOLD}; -use futures::channel::mpsc; -use futures::{channel::oneshot, prelude::*}; +use futures::{channel::oneshot, future::BoxFuture, prelude::*, stream::FuturesUnordered}; use instant::Instant; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, @@ -40,7 +39,7 @@ use libp2p_swarm::{ use smallvec::SmallVec; use std::{ collections::VecDeque, - fmt, io, + fmt, sync::{ atomic::{AtomicU64, Ordering}, Arc, @@ -69,25 +68,21 @@ where /// Queue of events to emit in `poll()`. pending_events: VecDeque>, /// Outbound upgrades waiting to be emitted as an `OutboundSubstreamRequest`. - pending_outbound: VecDeque>, - - requested_outbound: VecDeque>, - /// A channel for receiving inbound requests. - inbound_receiver: mpsc::Receiver<( - RequestId, - TCodec::Request, - oneshot::Sender, - )>, - /// The [`mpsc::Sender`] for the above receiver. Cloned for each inbound request. - inbound_sender: mpsc::Sender<( - RequestId, - TCodec::Request, - oneshot::Sender, - )>, - + outbound: VecDeque>, + /// Inbound upgrades waiting for the incoming request. + inbound: FuturesUnordered< + BoxFuture< + 'static, + Result< + ( + (RequestId, TCodec::Request), + oneshot::Sender, + ), + oneshot::Canceled, + >, + >, + >, inbound_request_id: Arc, - - worker_streams: futures_bounded::WorkerFutures, io::Error>>, } impl Handler @@ -100,132 +95,49 @@ where keep_alive_timeout: Duration, substream_timeout: Duration, inbound_request_id: Arc, - max_concurrent_streams: usize, ) -> Self { - let (inbound_sender, inbound_receiver) = mpsc::channel(0); Self { inbound_protocols, codec, keep_alive: KeepAlive::Yes, keep_alive_timeout, substream_timeout, - pending_outbound: VecDeque::new(), - requested_outbound: Default::default(), - inbound_receiver, - inbound_sender, + outbound: VecDeque::new(), + inbound: FuturesUnordered::new(), pending_events: VecDeque::new(), inbound_request_id, - worker_streams: futures_bounded::WorkerFutures::new( - substream_timeout, - max_concurrent_streams, - ), } } fn on_fully_negotiated_inbound( &mut self, FullyNegotiatedInbound { - protocol: (mut stream, protocol), - info: (), + protocol: sent, + info: request_id, }: FullyNegotiatedInbound< ::InboundProtocol, ::InboundOpenInfo, >, ) { - let mut codec = self.codec.clone(); - let request_id = RequestId(self.inbound_request_id.fetch_add(1, Ordering::Relaxed)); - let mut sender = self.inbound_sender.clone(); - - let recv = async move { - // A channel for notifying the inbound upgrade when the - // response is sent. - let (rs_send, rs_recv) = oneshot::channel(); - - let read = codec.read_request(&protocol, &mut stream); - let request = read.await?; - sender - .send((request_id, request, rs_send)) - .await - .expect("`ConnectionHandler` owns both ends of the channel"); - drop(sender); - - if let Ok(response) = rs_recv.await { - let write = codec.write_response(&protocol, &mut stream, response); - write.await?; - - stream.close().await?; - Ok(Event::ResponseSent(request_id)) - } else { - stream.close().await?; - Ok(Event::ResponseOmission(request_id)) - } - }; - - if self - .worker_streams - .try_push(request_id, recv.boxed()) - .is_some() - { - log::warn!("Dropping inbound stream because we are at capacity") - } - } - - fn on_fully_negotiated_outbound( - &mut self, - FullyNegotiatedOutbound { - protocol: (mut stream, protocol), - info: (), - }: FullyNegotiatedOutbound< - ::OutboundProtocol, - ::OutboundOpenInfo, - >, - ) { - let message = self - .requested_outbound - .pop_front() - .expect("negotiated a stream without a pending message"); - - let mut codec = self.codec.clone(); - let request_id = message.request_id; - - let send = async move { - let write = codec.write_request(&protocol, &mut stream, message.request); - write.await?; - stream.close().await?; - let read = codec.read_response(&protocol, &mut stream); - let response = read.await?; - - Ok(Event::Response { - request_id, - response, - }) - }; - - if self - .worker_streams - .try_push(request_id, send.boxed()) - .is_some() - { - log::warn!("Dropping outbound stream because we are at capacity") + if sent { + self.pending_events + .push_back(Event::ResponseSent(request_id)) + } else { + self.pending_events + .push_back(Event::ResponseOmission(request_id)) } } fn on_dial_upgrade_error( &mut self, - DialUpgradeError { error, info: () }: DialUpgradeError< + DialUpgradeError { info, error }: DialUpgradeError< ::OutboundOpenInfo, ::OutboundProtocol, >, ) { - let message = self - .requested_outbound - .pop_front() - .expect("negotiated a stream without a pending message"); - match error { StreamUpgradeError::Timeout => { - self.pending_events - .push_back(Event::OutboundTimeout(message.request_id)); + self.pending_events.push_back(Event::OutboundTimeout(info)); } StreamUpgradeError::NegotiationFailed => { // The remote merely doesn't support the protocol(s) we requested. @@ -234,26 +146,24 @@ where // An event is reported to permit user code to react to the fact that // the remote peer does not support the requested protocol(s). self.pending_events - .push_back(Event::OutboundUnsupportedProtocols(message.request_id)); + .push_back(Event::OutboundUnsupportedProtocols(info)); + } + StreamUpgradeError::Apply(e) => { + log::debug!("outbound stream {info} failed: {e}"); } - StreamUpgradeError::Apply(e) => void::unreachable(e), StreamUpgradeError::Io(e) => { - log::debug!( - "outbound stream for request {} failed: {e}, retrying", - message.request_id - ); - self.requested_outbound.push_back(message); + log::debug!("outbound stream {info} failed: {e}"); } } } fn on_listen_upgrade_error( &mut self, - ListenUpgradeError { error, .. }: ListenUpgradeError< + ListenUpgradeError { error, info }: ListenUpgradeError< ::InboundOpenInfo, ::InboundProtocol, >, ) { - void::unreachable(error) + log::debug!("inbound stream {info} failed: {error}"); } } @@ -323,45 +233,55 @@ impl fmt::Debug for Event { } } -pub struct OutboundMessage { - pub(crate) request_id: RequestId, - pub(crate) request: TCodec::Request, - pub(crate) protocols: SmallVec<[TCodec::Protocol; 2]>, -} - -impl fmt::Debug for OutboundMessage -where - TCodec: Codec, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("OutboundMessage").finish_non_exhaustive() - } -} - impl ConnectionHandler for Handler where TCodec: Codec + Send + Clone + 'static, { - type FromBehaviour = OutboundMessage; + type FromBehaviour = RequestProtocol; type ToBehaviour = Event; type Error = void::Void; - type InboundProtocol = Protocol; - type OutboundProtocol = Protocol; - type OutboundOpenInfo = (); - type InboundOpenInfo = (); + type InboundProtocol = ResponseProtocol; + type OutboundProtocol = RequestProtocol; + type OutboundOpenInfo = RequestId; + type InboundOpenInfo = RequestId; fn listen_protocol(&self) -> SubstreamProtocol { - SubstreamProtocol::new( - Protocol { - protocols: self.inbound_protocols.clone(), - }, - (), - ) + // A channel for notifying the handler when the inbound + // upgrade received the request. + let (rq_send, rq_recv) = oneshot::channel(); + + // A channel for notifying the inbound upgrade when the + // response is sent. + let (rs_send, rs_recv) = oneshot::channel(); + + let request_id = RequestId(self.inbound_request_id.fetch_add(1, Ordering::Relaxed)); + + // By keeping all I/O inside the `ResponseProtocol` and thus the + // inbound substream upgrade via above channels, we ensure that it + // is all subject to the configured timeout without extra bookkeeping + // for inbound substreams as well as their timeouts and also make the + // implementation of inbound and outbound upgrades symmetric in + // this sense. + let proto = ResponseProtocol { + protocols: self.inbound_protocols.clone(), + codec: self.codec.clone(), + request_sender: rq_send, + response_receiver: rs_recv, + request_id, + }; + + // The handler waits for the request to come in. It then emits + // `Event::Request` together with a + // `ResponseChannel`. + self.inbound + .push(rq_recv.map_ok(move |rq| (rq, rs_send)).boxed()); + + SubstreamProtocol::new(proto, request_id).with_timeout(self.substream_timeout) } fn on_behaviour_event(&mut self, request: Self::FromBehaviour) { self.keep_alive = KeepAlive::Yes; - self.pending_outbound.push_back(request); + self.outbound.push_back(request); } fn connection_keep_alive(&self) -> KeepAlive { @@ -371,23 +291,9 @@ where fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll, (), Self::ToBehaviour, Self::Error>> - { - loop { - match self.worker_streams.poll_unpin(cx) { - Poll::Ready((_, Ok(Ok(event)))) => { - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)) - } - Poll::Ready((id, Ok(Err(e)))) => { - log::debug!("Stream for request {id} failed: {e}"); - } - Poll::Ready((id, Err(futures_bounded::Timeout { .. }))) => { - log::debug!("Stream for request {id} timed out"); - } - Poll::Pending => break, - } - } - + ) -> Poll< + ConnectionHandlerEvent, RequestId, Self::ToBehaviour, Self::Error>, + > { // Drain pending events. if let Some(event) = self.pending_events.pop_front() { return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)); @@ -396,33 +302,41 @@ where } // Check for inbound requests. - if let Poll::Ready(Some((id, rq, rs_sender))) = self.inbound_receiver.poll_next_unpin(cx) { - // We received an inbound request. - self.keep_alive = KeepAlive::Yes; - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(Event::Request { - request_id: id, - request: rq, - sender: rs_sender, - })); + while let Poll::Ready(Some(result)) = self.inbound.poll_next_unpin(cx) { + match result { + Ok(((id, rq), rs_sender)) => { + // We received an inbound request. + self.keep_alive = KeepAlive::Yes; + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(Event::Request { + request_id: id, + request: rq, + sender: rs_sender, + })); + } + Err(oneshot::Canceled) => { + // The inbound upgrade has errored or timed out reading + // or waiting for the request. The handler is informed + // via `on_connection_event` call with `ConnectionEvent::ListenUpgradeError`. + } + } } // Emit outbound requests. - if let Some(request) = self.pending_outbound.pop_front() { - let protocols = request.protocols.clone(); - self.requested_outbound.push_back(request); - + if let Some(request) = self.outbound.pop_front() { + let info = request.request_id; return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { - protocol: SubstreamProtocol::new(Protocol { protocols }, ()), + protocol: SubstreamProtocol::new(request, info) + .with_timeout(self.substream_timeout), }); } - debug_assert!(self.pending_outbound.is_empty()); + debug_assert!(self.outbound.is_empty()); - if self.pending_outbound.capacity() > EMPTY_QUEUE_SHRINK_THRESHOLD { - self.pending_outbound.shrink_to_fit(); + if self.outbound.capacity() > EMPTY_QUEUE_SHRINK_THRESHOLD { + self.outbound.shrink_to_fit(); } - if self.worker_streams.is_empty() && self.keep_alive.is_yes() { + if self.inbound.is_empty() && self.keep_alive.is_yes() { // No new inbound or outbound requests. However, we may just have // started the latest inbound or outbound upgrade(s), so make sure // the keep-alive timeout is preceded by the substream timeout. @@ -446,8 +360,14 @@ where ConnectionEvent::FullyNegotiatedInbound(fully_negotiated_inbound) => { self.on_fully_negotiated_inbound(fully_negotiated_inbound) } - ConnectionEvent::FullyNegotiatedOutbound(fully_negotiated_outbound) => { - self.on_fully_negotiated_outbound(fully_negotiated_outbound) + ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { + protocol: response, + info: request_id, + }) => { + self.pending_events.push_back(Event::Response { + request_id, + response, + }); } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) diff --git a/protocols/request-response/src/handler/protocol.rs b/protocols/request-response/src/handler/protocol.rs index 833cacdd6ce..1368a3c1f98 100644 --- a/protocols/request-response/src/handler/protocol.rs +++ b/protocols/request-response/src/handler/protocol.rs @@ -23,10 +23,14 @@ //! receives a request and sends a response, whereas the //! outbound upgrade send a request and receives a response. -use futures::future::{ready, Ready}; +use crate::codec::Codec; +use crate::RequestId; + +use futures::{channel::oneshot, future::BoxFuture, prelude::*}; use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use libp2p_swarm::Stream; use smallvec::SmallVec; +use std::{fmt, io}; /// The level of support for a particular protocol. #[derive(Debug, Clone)] @@ -61,15 +65,22 @@ impl ProtocolSupport { /// /// Receives a request and sends a response. #[derive(Debug)] -pub struct Protocol

{ - pub(crate) protocols: SmallVec<[P; 2]>, +pub struct ResponseProtocol +where + TCodec: Codec, +{ + pub(crate) codec: TCodec, + pub(crate) protocols: SmallVec<[TCodec::Protocol; 2]>, + pub(crate) request_sender: oneshot::Sender<(RequestId, TCodec::Request)>, + pub(crate) response_receiver: oneshot::Receiver, + pub(crate) request_id: RequestId, } -impl

UpgradeInfo for Protocol

+impl UpgradeInfo for ResponseProtocol where - P: AsRef + Clone, + TCodec: Codec, { - type Info = P; + type Info = TCodec::Protocol; type InfoIter = smallvec::IntoIter<[Self::Info; 2]>; fn protocol_info(&self) -> Self::InfoIter { @@ -77,28 +88,94 @@ where } } -impl

InboundUpgrade for Protocol

+impl InboundUpgrade for ResponseProtocol where - P: AsRef + Clone, + TCodec: Codec + Send + 'static, { - type Output = (Stream, P); - type Error = void::Void; - type Future = Ready>; + type Output = bool; + type Error = io::Error; + type Future = BoxFuture<'static, Result>; + + fn upgrade_inbound(mut self, mut io: Stream, protocol: Self::Info) -> Self::Future { + async move { + let read = self.codec.read_request(&protocol, &mut io); + let request = read.await?; + match self.request_sender.send((self.request_id, request)) { + Ok(()) => {}, + Err(_) => panic!( + "Expect request receiver to be alive i.e. protocol handler to be alive.", + ), + } - fn upgrade_inbound(self, io: Stream, protocol: Self::Info) -> Self::Future { - ready(Ok((io, protocol))) + if let Ok(response) = self.response_receiver.await { + let write = self.codec.write_response(&protocol, &mut io, response); + write.await?; + + io.close().await?; + // Response was sent. Indicate to handler to emit a `ResponseSent` event. + Ok(true) + } else { + io.close().await?; + // No response was sent. Indicate to handler to emit a `ResponseOmission` event. + Ok(false) + } + }.boxed() } } -impl

OutboundUpgrade for Protocol

+/// Request substream upgrade protocol. +/// +/// Sends a request and receives a response. +pub struct RequestProtocol where - P: AsRef + Clone, + TCodec: Codec, { - type Output = (Stream, P); - type Error = void::Void; - type Future = Ready>; + pub(crate) codec: TCodec, + pub(crate) protocols: SmallVec<[TCodec::Protocol; 2]>, + pub(crate) request_id: RequestId, + pub(crate) request: TCodec::Request, +} - fn upgrade_outbound(self, io: Stream, protocol: Self::Info) -> Self::Future { - ready(Ok((io, protocol))) +impl fmt::Debug for RequestProtocol +where + TCodec: Codec, +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("RequestProtocol") + .field("request_id", &self.request_id) + .finish() + } +} + +impl UpgradeInfo for RequestProtocol +where + TCodec: Codec, +{ + type Info = TCodec::Protocol; + type InfoIter = smallvec::IntoIter<[Self::Info; 2]>; + + fn protocol_info(&self) -> Self::InfoIter { + self.protocols.clone().into_iter() + } +} + +impl OutboundUpgrade for RequestProtocol +where + TCodec: Codec + Send + 'static, +{ + type Output = TCodec::Response; + type Error = io::Error; + type Future = BoxFuture<'static, Result>; + + fn upgrade_outbound(mut self, mut io: Stream, protocol: Self::Info) -> Self::Future { + async move { + let write = self.codec.write_request(&protocol, &mut io, self.request); + write.await?; + io.close().await?; + let read = self.codec.read_response(&protocol, &mut io); + let response = read.await?; + Ok(response) + } + .boxed() } } diff --git a/protocols/request-response/src/lib.rs b/protocols/request-response/src/lib.rs index 7e81f6f5b5c..7b1a8088443 100644 --- a/protocols/request-response/src/lib.rs +++ b/protocols/request-response/src/lib.rs @@ -76,7 +76,7 @@ pub mod json; pub use codec::Codec; pub use handler::ProtocolSupport; -use crate::handler::OutboundMessage; +use crate::handler::protocol::RequestProtocol; use futures::channel::oneshot; use handler::Handler; use libp2p_core::{ConnectedPoint, Endpoint, Multiaddr}; @@ -285,7 +285,6 @@ impl fmt::Display for RequestId { pub struct Config { request_timeout: Duration, connection_keep_alive: Duration, - max_concurrent_streams: usize, } impl Default for Config { @@ -293,43 +292,22 @@ impl Default for Config { Self { connection_keep_alive: Duration::from_secs(10), request_timeout: Duration::from_secs(10), - max_concurrent_streams: 100, } } } impl Config { /// Sets the keep-alive timeout of idle connections. - #[deprecated(note = "Use `Config::with_connection_keep_alive` for one-liner constructions.")] pub fn set_connection_keep_alive(&mut self, v: Duration) -> &mut Self { self.connection_keep_alive = v; self } /// Sets the timeout for inbound and outbound requests. - #[deprecated(note = "Use `Config::with_request_timeout` for one-liner constructions.")] pub fn set_request_timeout(&mut self, v: Duration) -> &mut Self { self.request_timeout = v; self } - - /// Sets the keep-alive timeout of idle connections. - pub fn with_connection_keep_alive(mut self, v: Duration) -> Self { - self.connection_keep_alive = v; - self - } - - /// Sets the timeout for inbound and outbound requests. - pub fn with_request_timeout(mut self, v: Duration) -> Self { - self.request_timeout = v; - self - } - - /// Sets the upper bound for the number of concurrent inbound + outbound streams. - pub fn with_max_concurrent_streams(mut self, num_streams: usize) -> Self { - self.max_concurrent_streams = num_streams; - self - } } /// A request/response protocol for some message codec. @@ -351,7 +329,7 @@ where codec: TCodec, /// Pending events to return from `poll`. pending_events: - VecDeque, OutboundMessage>>, + VecDeque, RequestProtocol>>, /// The currently connected peers, their pending outbound and inbound responses and their known, /// reachable addresses, if any. connected: HashMap>, @@ -359,7 +337,7 @@ where addresses: HashMap>, /// Requests that have not yet been sent and are waiting for a connection /// to be established. - pending_outbound_requests: HashMap; 10]>>, + pending_outbound_requests: HashMap; 10]>>, } impl Behaviour @@ -423,10 +401,11 @@ where /// > [`Behaviour::remove_address`]. pub fn send_request(&mut self, peer: &PeerId, request: TCodec::Request) -> RequestId { let request_id = self.next_request_id(); - let request = OutboundMessage { + let request = RequestProtocol { request_id, - request, + codec: self.codec.clone(), protocols: self.outbound_protocols.clone(), + request, }; if let Some(request) = self.try_send_request(peer, request) { @@ -540,8 +519,8 @@ where fn try_send_request( &mut self, peer: &PeerId, - request: OutboundMessage, - ) -> Option> { + request: RequestProtocol, + ) -> Option> { if let Some(connections) = self.connected.get_mut(peer) { if connections.is_empty() { return Some(request); @@ -744,7 +723,6 @@ where self.config.connection_keep_alive, self.config.request_timeout, self.next_inbound_id.clone(), - self.config.max_concurrent_streams, )) } @@ -784,7 +762,6 @@ where self.config.connection_keep_alive, self.config.request_timeout, self.next_inbound_id.clone(), - self.config.max_concurrent_streams, )) } diff --git a/swarm/CHANGELOG.md b/swarm/CHANGELOG.md index d703aef639a..6a9808dd554 100644 --- a/swarm/CHANGELOG.md +++ b/swarm/CHANGELOG.md @@ -22,7 +22,6 @@ ## 0.43.0 - Allow `NetworkBehaviours` to create and remove listeners. -- See [PR 3292]. - Raise MSRV to 1.65. From 67f382cb3863ad0d077704f601568a0ebecbe883 Mon Sep 17 00:00:00 2001 From: dgarus Date: Tue, 22 Aug 2023 15:58:08 +0300 Subject: [PATCH 22/83] Removed extra changes --- Cargo.lock | 2 -- 1 file changed, 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cb24cdfb09e..f7f44a6156b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3160,8 +3160,6 @@ dependencies = [ "cbor4ii", "env_logger 0.10.0", "futures", - "futures-bounded", - "futures-timer", "futures_ringbuf", "instant", "libp2p-core", From d6e09d38d180f1923181e8aae8a1f2c141279838 Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 23 Aug 2023 12:21:37 +0300 Subject: [PATCH 23/83] using futures-bounded struct --- protocols/relay/src/behaviour.rs | 8 ++++ protocols/relay/src/behaviour/handler.rs | 51 ++++++++++++++-------- protocols/relay/src/priv_client/handler.rs | 40 +++++++++++------ 3 files changed, 69 insertions(+), 30 deletions(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index a7c3d81e54c..b23f31adb07 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -54,6 +54,8 @@ pub struct Config { pub reservation_duration: Duration, pub reservation_rate_limiters: Vec>, + pub substream_timeout: Duration, + pub max_concurrent_streams: usize, pub max_circuits: usize, pub max_circuits_per_peer: usize, pub max_circuit_duration: Duration, @@ -151,6 +153,8 @@ impl Default for Config { reservation_duration: Duration::from_secs(60 * 60), reservation_rate_limiters, + substream_timeout: Duration::from_secs(10), + max_concurrent_streams: 100, max_circuits: 16, max_circuits_per_peer: 4, max_circuit_duration: Duration::from_secs(2 * 60), @@ -302,6 +306,8 @@ impl NetworkBehaviour for Behaviour { reservation_duration: self.config.reservation_duration, max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, + substream_timeout: self.config.substream_timeout, + max_concurrent_streams: self.config.max_concurrent_streams, }, ConnectedPoint::Listener { local_addr: local_addr.clone(), @@ -327,6 +333,8 @@ impl NetworkBehaviour for Behaviour { reservation_duration: self.config.reservation_duration, max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, + substream_timeout: self.config.substream_timeout, + max_concurrent_streams: self.config.max_concurrent_streams, }, ConnectedPoint::Dialer { address: addr.clone(), diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 4701ab3e4ae..b5e72cd291f 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -47,6 +47,8 @@ use std::time::Duration; #[derive(Debug, Clone)] pub struct Config { + pub substream_timeout: Duration, + pub max_concurrent_streams: usize, pub reservation_duration: Duration, pub max_circuit_duration: Duration, pub max_circuit_bytes: u64, @@ -382,12 +384,16 @@ pub struct Handler { circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, stop_requested_streams: VecDeque, - protocol_futs: FuturesUnordered>, + protocol_futs: futures_bounded::WorkerFutures<(), CHEvent>, } impl Handler { pub fn new(config: Config, endpoint: ConnectedPoint) -> Handler { Handler { + protocol_futs: futures_bounded::WorkerFutures::new( + config.substream_timeout, + config.max_concurrent_streams, + ), endpoint, config, queued_events: Default::default(), @@ -400,22 +406,25 @@ impl Handler { active_reservation: Default::default(), keep_alive: KeepAlive::Yes, stop_requested_streams: Default::default(), - protocol_futs: Default::default(), } } - fn on_fully_negotiated_inbound(&self, stream: Stream) { - self.protocol_futs.push( - inbound_hop::handle_inbound_request( - stream, - self.config.reservation_duration, - self.config.max_circuit_duration, - self.config.max_circuit_bytes, - self.endpoint.clone(), - self.active_reservation.is_some(), - ) - .boxed(), - ); + fn on_fully_negotiated_inbound(&mut self, stream: Stream) { + if self.protocol_futs + .try_push( + (), + inbound_hop::handle_inbound_request( + stream, + self.config.reservation_duration, + self.config.max_circuit_duration, + self.config.max_circuit_bytes, + self.endpoint.clone(), + self.active_reservation.is_some(), + ) + .boxed(), + ).is_some() { + log::warn!("Dropping inbound stream because we are at capacity") + } } fn on_fully_negotiated_outbound(&mut self, stream: Stream) { @@ -427,8 +436,13 @@ impl Handler { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); - self.protocol_futs - .push(outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed()); + if self.protocol_futs + .try_push( + (), + outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed(), + ).is_some() { + log::warn!("Dropping outbound stream because we are at capacity") + } } fn on_dial_upgrade_error( @@ -646,7 +660,10 @@ impl ConnectionHandler for Handler { } // Process protocol requests - if let Poll::Ready(Some(event)) = self.protocol_futs.poll_next_unpin(cx) { + if let Poll::Ready((_, worker_res)) = self.protocol_futs.poll_unpin(cx) { + let event = worker_res + .unwrap_or_else(|_|{ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)}); + return Poll::Ready(event); } diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index a8dac5d3969..a330a5d4bcc 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -136,8 +136,9 @@ pub struct Handler { >, wait_for_connection_outbound_stream: VecDeque, - circuit_connection_futs: FuturesUnordered< - BoxFuture<'static, Result, outbound_hop::UpgradeError>>, + circuit_connection_futs: futures_bounded::WorkerFutures< + (), + Result, outbound_hop::UpgradeError> >, reservation: Reservation, @@ -184,7 +185,10 @@ impl Handler { config.max_concurrent_streams, ), wait_for_connection_outbound_stream: Default::default(), - circuit_connection_futs: Default::default(), + circuit_connection_futs: futures_bounded::WorkerFutures::new( + config.substream_timeout, + config.max_concurrent_streams, + ), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), open_circuit_futs: Default::default(), @@ -395,7 +399,14 @@ impl ConnectionHandler for Handler { } // Circuit connections - if let Poll::Ready(Some(res)) = self.circuit_connection_futs.poll_next_unpin(cx) { + if let Poll::Ready((_, worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { + let res = match worker_res { + Ok(r) => r, + Err(futures_bounded::Timeout { .. }) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + }; + let opt = match res { Ok(Some(outbound_hop::Output::Circuit { limit })) => { Some(ConnectionHandlerEvent::NotifyBehaviour( @@ -578,15 +589,18 @@ impl ConnectionHandler for Handler { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); - self.circuit_connection_futs.push( - outbound_hop::handle_connection_message_response( - stream, - self.remote_peer_id, - con_command, - tx, - ) - .boxed(), - ) + if self.circuit_connection_futs + .try_push( + (), + outbound_hop::handle_connection_message_response( + stream, + self.remote_peer_id, + con_command, + tx, + ).boxed() + ).is_some() { + log::warn!("Dropping outbound stream because we are at capacity") + } } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) From 28e101daa3935309e03fff14d8f7b1a402d56a6d Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 23 Aug 2023 12:22:36 +0300 Subject: [PATCH 24/83] using futures-bounded struct --- protocols/relay/src/behaviour/handler.rs | 18 ++++++++++++------ protocols/relay/src/priv_client/handler.rs | 12 ++++++++---- 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index b5e72cd291f..1c7835d83aa 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -410,7 +410,8 @@ impl Handler { } fn on_fully_negotiated_inbound(&mut self, stream: Stream) { - if self.protocol_futs + if self + .protocol_futs .try_push( (), inbound_hop::handle_inbound_request( @@ -421,8 +422,10 @@ impl Handler { self.endpoint.clone(), self.active_reservation.is_some(), ) - .boxed(), - ).is_some() { + .boxed(), + ) + .is_some() + { log::warn!("Dropping inbound stream because we are at capacity") } } @@ -436,11 +439,14 @@ impl Handler { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); - if self.protocol_futs + if self + .protocol_futs .try_push( (), outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed(), - ).is_some() { + ) + .is_some() + { log::warn!("Dropping outbound stream because we are at capacity") } } @@ -662,7 +668,7 @@ impl ConnectionHandler for Handler { // Process protocol requests if let Poll::Ready((_, worker_res)) = self.protocol_futs.poll_unpin(cx) { let event = worker_res - .unwrap_or_else(|_|{ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)}); + .unwrap_or_else(|_| ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); return Poll::Ready(event); } diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index a330a5d4bcc..b94d14b53ce 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -138,7 +138,7 @@ pub struct Handler { wait_for_connection_outbound_stream: VecDeque, circuit_connection_futs: futures_bounded::WorkerFutures< (), - Result, outbound_hop::UpgradeError> + Result, outbound_hop::UpgradeError>, >, reservation: Reservation, @@ -589,7 +589,8 @@ impl ConnectionHandler for Handler { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); - if self.circuit_connection_futs + if self + .circuit_connection_futs .try_push( (), outbound_hop::handle_connection_message_response( @@ -597,8 +598,11 @@ impl ConnectionHandler for Handler { self.remote_peer_id, con_command, tx, - ).boxed() - ).is_some() { + ) + .boxed(), + ) + .is_some() + { log::warn!("Dropping outbound stream because we are at capacity") } } From 4f1b69de5bf19885d3d091d61465afbd4e9dbf70 Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 23 Aug 2023 14:29:49 +0300 Subject: [PATCH 25/83] using futures-bounded struct --- protocols/relay/src/priv_client/handler.rs | 50 +++++++++++++--------- 1 file changed, 29 insertions(+), 21 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index b94d14b53ce..7f861d034b1 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -153,8 +153,9 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - open_circuit_futs: FuturesUnordered< - BoxFuture<'static, Result>, + open_circuit_futs: futures_bounded::WorkerFutures< + (), + Result >, circuit_deny_futs: HashMap>>, @@ -191,7 +192,10 @@ impl Handler { ), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), - open_circuit_futs: Default::default(), + open_circuit_futs: futures_bounded::WorkerFutures::new( + config.substream_timeout, + config.max_concurrent_streams, + ), circuit_deny_futs: Default::default(), send_error_futs: Default::default(), keep_alive: KeepAlive::Yes, @@ -355,14 +359,11 @@ impl ConnectionHandler for Handler { // Reservations if let Poll::Ready((_, worker_res)) = self.reserve_futs.poll_unpin(cx) { - let result = match worker_res { - Ok(r) => r, - Err(futures_bounded::Timeout { .. }) => { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } - }; + if worker_res.is_err() { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } - let event = match result { + let event = match worker_res.unwrap() { Ok(outbound_hop::Output::Reservation { renewal_timeout, addrs, @@ -400,14 +401,11 @@ impl ConnectionHandler for Handler { // Circuit connections if let Poll::Ready((_, worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { - let res = match worker_res { - Ok(r) => r, - Err(futures_bounded::Timeout { .. }) => { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } - }; + if worker_res.is_err() { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)) + } - let opt = match res { + let opt = match worker_res.unwrap() { Ok(Some(outbound_hop::Output::Circuit { limit })) => { Some(ConnectionHandlerEvent::NotifyBehaviour( Event::OutboundCircuitEstablished { limit }, @@ -446,8 +444,12 @@ impl ConnectionHandler for Handler { return Poll::Ready(event); } - if let Poll::Ready(Some(circuit_res)) = self.open_circuit_futs.poll_next_unpin(cx) { - match circuit_res { + if let Poll::Ready((_, worker_res)) = self.open_circuit_futs.poll_unpin(cx) { + if worker_res.is_err() { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)) + } + + match worker_res.unwrap() { Ok(circuit) => match &mut self.reservation { Reservation::Accepted { pending_msgs, .. } | Reservation::Renewing { pending_msgs, .. } => { @@ -560,8 +562,14 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - self.open_circuit_futs - .push(inbound_stop::handle_open_circuit(stream).boxed()); + if self.open_circuit_futs + .try_push( + (), + inbound_stop::handle_open_circuit(stream).boxed() + ) + .is_some() { + log::warn!("Dropping inbound stream because we are at capacity") + } } ConnectionEvent::FullyNegotiatedOutbound(FullyNegotiatedOutbound { protocol: stream, From c0d1b23fa20b7b94dc826c3642b5fc26a6cf0f9d Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 23 Aug 2023 14:30:26 +0300 Subject: [PATCH 26/83] fix format --- protocols/relay/src/priv_client/handler.rs | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 7f861d034b1..3ba001fac6b 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -155,7 +155,7 @@ pub struct Handler { open_circuit_futs: futures_bounded::WorkerFutures< (), - Result + Result, >, circuit_deny_futs: HashMap>>, @@ -402,7 +402,7 @@ impl ConnectionHandler for Handler { // Circuit connections if let Poll::Ready((_, worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { if worker_res.is_err() { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)) + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); } let opt = match worker_res.unwrap() { @@ -446,7 +446,7 @@ impl ConnectionHandler for Handler { if let Poll::Ready((_, worker_res)) = self.open_circuit_futs.poll_unpin(cx) { if worker_res.is_err() { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)) + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); } match worker_res.unwrap() { @@ -562,12 +562,11 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - if self.open_circuit_futs - .try_push( - (), - inbound_stop::handle_open_circuit(stream).boxed() - ) - .is_some() { + if self + .open_circuit_futs + .try_push((), inbound_stop::handle_open_circuit(stream).boxed()) + .is_some() + { log::warn!("Dropping inbound stream because we are at capacity") } } From 31661ae96dbc5742da157f29bb61b66ed00b0f33 Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Sat, 26 Aug 2023 09:13:30 +0300 Subject: [PATCH 27/83] Update protocols/relay/src/behaviour.rs Co-authored-by: Max Inden --- protocols/relay/src/behaviour.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index b23f31adb07..2b10766c805 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -55,7 +55,7 @@ pub struct Config { pub reservation_rate_limiters: Vec>, pub substream_timeout: Duration, - pub max_concurrent_streams: usize, + pub max_concurrent_streams_per_connection: usize, pub max_circuits: usize, pub max_circuits_per_peer: usize, pub max_circuit_duration: Duration, From 1a83ed1d233cd9604b3917b0abf9a671b4ef901a Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 09:16:20 +0300 Subject: [PATCH 28/83] rename `max_concurrent_streams` to `max_concurrent_streams_per_connection` --- protocols/relay/src/behaviour.rs | 10 +++++++--- protocols/relay/src/behaviour/handler.rs | 4 ++-- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 2b10766c805..81d6c8c2539 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -154,7 +154,7 @@ impl Default for Config { reservation_rate_limiters, substream_timeout: Duration::from_secs(10), - max_concurrent_streams: 100, + max_concurrent_streams_per_connection: 100, max_circuits: 16, max_circuits_per_peer: 4, max_circuit_duration: Duration::from_secs(2 * 60), @@ -307,7 +307,9 @@ impl NetworkBehaviour for Behaviour { max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, substream_timeout: self.config.substream_timeout, - max_concurrent_streams: self.config.max_concurrent_streams, + max_concurrent_streams_per_connection: self + .config + .max_concurrent_streams_per_connection, }, ConnectedPoint::Listener { local_addr: local_addr.clone(), @@ -334,7 +336,9 @@ impl NetworkBehaviour for Behaviour { max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, substream_timeout: self.config.substream_timeout, - max_concurrent_streams: self.config.max_concurrent_streams, + max_concurrent_streams_per_connection: self + .config + .max_concurrent_streams_per_connection, }, ConnectedPoint::Dialer { address: addr.clone(), diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 1c7835d83aa..90590261977 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -48,7 +48,7 @@ use std::time::Duration; #[derive(Debug, Clone)] pub struct Config { pub substream_timeout: Duration, - pub max_concurrent_streams: usize, + pub max_concurrent_streams_per_connection: usize, pub reservation_duration: Duration, pub max_circuit_duration: Duration, pub max_circuit_bytes: u64, @@ -392,7 +392,7 @@ impl Handler { Handler { protocol_futs: futures_bounded::WorkerFutures::new( config.substream_timeout, - config.max_concurrent_streams, + config.max_concurrent_streams_per_connection, ), endpoint, config, From 22f676da64f160db11369ab20b932ebb0bf6e541 Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Sat, 26 Aug 2023 09:37:00 +0300 Subject: [PATCH 29/83] Update protocols/relay/src/behaviour.rs Co-authored-by: Max Inden --- protocols/relay/src/behaviour.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 2b10766c805..25f37691dd7 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -154,7 +154,7 @@ impl Default for Config { reservation_rate_limiters, substream_timeout: Duration::from_secs(10), - max_concurrent_streams: 100, + max_concurrent_streams: 10, max_circuits: 16, max_circuits_per_peer: 4, max_circuit_duration: Duration::from_secs(2 * 60), From 7c0076b2058327337feee3d436b429f07ea5c592 Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Sat, 26 Aug 2023 09:38:51 +0300 Subject: [PATCH 30/83] Update protocols/relay/src/behaviour/handler.rs Co-authored-by: Max Inden --- protocols/relay/src/behaviour/handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 1c7835d83aa..ed24e33672d 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -502,7 +502,7 @@ enum ReservationRequestFuture { type Futures = FuturesUnordered>; -pub(crate) type CHEvent = ConnectionHandlerEvent< +pub(crate) type ConnectionHandlerEvent = libp2p_swarm::ConnectionHandlerEvent< ::OutboundProtocol, ::OutboundOpenInfo, ::ToBehaviour, From 553a316c61fd03c7d7e5206ebe48889f3f155da2 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 09:54:26 +0300 Subject: [PATCH 31/83] fix review comments --- protocols/relay/src/behaviour.rs | 8 ++++++-- protocols/relay/src/behaviour/handler.rs | 4 ++-- protocols/relay/src/protocol/inbound_hop.rs | 2 +- protocols/relay/src/protocol/outbound_stop.rs | 2 +- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 4371c6b5b70..282f96d6fc2 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -307,7 +307,9 @@ impl NetworkBehaviour for Behaviour { max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, substream_timeout: self.config.substream_timeout, - max_concurrent_streams: self.config.max_concurrent_streams, + max_concurrent_streams_per_connection: self + .config + .max_concurrent_streams_per_connection, }, ConnectedPoint::Listener { local_addr: local_addr.clone(), @@ -334,7 +336,9 @@ impl NetworkBehaviour for Behaviour { max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, substream_timeout: self.config.substream_timeout, - max_concurrent_streams: self.config.max_concurrent_streams, + max_concurrent_streams_per_connection: self + .config + .max_concurrent_streams_per_connection, }, ConnectedPoint::Dialer { address: addr.clone(), diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 7d201f2c819..ec18277a3a2 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -384,7 +384,7 @@ pub struct Handler { circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, stop_requested_streams: VecDeque, - protocol_futs: futures_bounded::WorkerFutures<(), CHEvent>, + protocol_futs: futures_bounded::WorkerFutures<(), RelayConnectionHandlerEvent>, } impl Handler { @@ -502,7 +502,7 @@ enum ReservationRequestFuture { type Futures = FuturesUnordered>; -pub(crate) type ConnectionHandlerEvent = libp2p_swarm::ConnectionHandlerEvent< +pub(crate) type RelayConnectionHandlerEvent = libp2p_swarm::ConnectionHandlerEvent< ::OutboundProtocol, ::OutboundOpenInfo, ::ToBehaviour, diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index f6f711aaec1..a5f2d0b5db9 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -188,7 +188,7 @@ pub(crate) async fn handle_inbound_request( max_circuit_bytes: u64, endpoint: ConnectedPoint, renewed: bool, -) -> handler::CHEvent { +) -> handler::RelayConnectionHandlerEvent { let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); let res = substream.next().await; diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 4695778aeb8..7279dd63ac1 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -78,7 +78,7 @@ pub(crate) async fn handle_stop_message_response( io: Stream, stop_command: StopCommand, tx: oneshot::Sender<()>, -) -> handler::CHEvent { +) -> handler::RelayConnectionHandlerEvent { let msg = proto::StopMessage { type_pb: proto::StopMessageType::CONNECT, peer: Some(proto::Peer { From 18afe79f4e35452d5bbc03b462cc3257cd52fe22 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 10:51:54 +0300 Subject: [PATCH 32/83] removed config fields --- examples/dcutr/src/main.rs | 2 +- protocols/dcutr/tests/lib.rs | 2 +- protocols/relay/src/behaviour.rs | 12 --------- protocols/relay/src/behaviour/handler.rs | 9 ++++--- protocols/relay/src/priv_client.rs | 30 +++------------------- protocols/relay/src/priv_client/handler.rs | 24 ++++++++--------- protocols/relay/tests/lib.rs | 2 +- 7 files changed, 22 insertions(+), 59 deletions(-) diff --git a/examples/dcutr/src/main.rs b/examples/dcutr/src/main.rs index d943b377aea..8837df783b4 100644 --- a/examples/dcutr/src/main.rs +++ b/examples/dcutr/src/main.rs @@ -89,7 +89,7 @@ fn main() -> Result<(), Box> { let local_peer_id = PeerId::from(local_key.public()); info!("Local peer id: {:?}", local_peer_id); - let (relay_transport, client) = relay::client::new(local_peer_id, Default::default()); + let (relay_transport, client) = relay::client::new(local_peer_id); let transport = { let relay_tcp_quic_transport = relay_transport diff --git a/protocols/dcutr/tests/lib.rs b/protocols/dcutr/tests/lib.rs index 90a2ac1a814..6888e5914a0 100644 --- a/protocols/dcutr/tests/lib.rs +++ b/protocols/dcutr/tests/lib.rs @@ -114,7 +114,7 @@ fn build_client() -> Swarm { let local_public_key = local_key.public(); let local_peer_id = local_public_key.to_peer_id(); - let (relay_transport, behaviour) = relay::client::new(local_peer_id, Default::default()); + let (relay_transport, behaviour) = relay::client::new(local_peer_id); let transport = relay_transport .or_transport(MemoryTransport::default()) diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 282f96d6fc2..a7c3d81e54c 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -54,8 +54,6 @@ pub struct Config { pub reservation_duration: Duration, pub reservation_rate_limiters: Vec>, - pub substream_timeout: Duration, - pub max_concurrent_streams_per_connection: usize, pub max_circuits: usize, pub max_circuits_per_peer: usize, pub max_circuit_duration: Duration, @@ -153,8 +151,6 @@ impl Default for Config { reservation_duration: Duration::from_secs(60 * 60), reservation_rate_limiters, - substream_timeout: Duration::from_secs(10), - max_concurrent_streams_per_connection: 10, max_circuits: 16, max_circuits_per_peer: 4, max_circuit_duration: Duration::from_secs(2 * 60), @@ -306,10 +302,6 @@ impl NetworkBehaviour for Behaviour { reservation_duration: self.config.reservation_duration, max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, - substream_timeout: self.config.substream_timeout, - max_concurrent_streams_per_connection: self - .config - .max_concurrent_streams_per_connection, }, ConnectedPoint::Listener { local_addr: local_addr.clone(), @@ -335,10 +327,6 @@ impl NetworkBehaviour for Behaviour { reservation_duration: self.config.reservation_duration, max_circuit_duration: self.config.max_circuit_duration, max_circuit_bytes: self.config.max_circuit_bytes, - substream_timeout: self.config.substream_timeout, - max_concurrent_streams_per_connection: self - .config - .max_concurrent_streams_per_connection, }, ConnectedPoint::Dialer { address: addr.clone(), diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index ec18277a3a2..b87f4f793ce 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -45,10 +45,11 @@ use std::fmt; use std::task::{Context, Poll}; use std::time::Duration; +const STREAM_TIMEOUT: Duration = Duration::from_secs(60 * 60); +const MAX_CONCURRENT_STREAMS_PER_CONNECTION: usize = 10; + #[derive(Debug, Clone)] pub struct Config { - pub substream_timeout: Duration, - pub max_concurrent_streams_per_connection: usize, pub reservation_duration: Duration, pub max_circuit_duration: Duration, pub max_circuit_bytes: u64, @@ -391,8 +392,8 @@ impl Handler { pub fn new(config: Config, endpoint: ConnectedPoint) -> Handler { Handler { protocol_futs: futures_bounded::WorkerFutures::new( - config.substream_timeout, - config.max_concurrent_streams_per_connection, + STREAM_TIMEOUT, + MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), endpoint, config, diff --git a/protocols/relay/src/priv_client.rs b/protocols/relay/src/priv_client.rs index 11e84af3214..abc2fe13177 100644 --- a/protocols/relay/src/priv_client.rs +++ b/protocols/relay/src/priv_client.rs @@ -47,7 +47,6 @@ use std::collections::{hash_map, HashMap, VecDeque}; use std::io::{Error, ErrorKind, IoSlice}; use std::pin::Pin; use std::task::{Context, Poll}; -use std::time::Duration; use transport::Transport; use void::Void; @@ -89,26 +88,9 @@ pub enum Event { }, } -#[derive(Debug, Clone)] -pub struct Config { - pub substream_timeout: Duration, - pub max_concurrent_streams: usize, -} - -impl Default for Config { - fn default() -> Self { - Self { - substream_timeout: Duration::from_secs(10), - max_concurrent_streams: 100, - } - } -} - /// [`NetworkBehaviour`] implementation of the relay client /// functionality of the circuit relay v2 protocol. pub struct Behaviour { - config: Config, - local_peer_id: PeerId, from_transport: Receiver, @@ -123,12 +105,11 @@ pub struct Behaviour { } /// Create a new client relay [`Behaviour`] with it's corresponding [`Transport`]. -pub fn new(local_peer_id: PeerId, config: Config) -> (Transport, Behaviour) { +pub fn new(local_peer_id: PeerId) -> (Transport, Behaviour) { let (transport, from_transport) = Transport::new(); let behaviour = Behaviour { local_peer_id, from_transport, - config, directly_connected_peers: Default::default(), queued_actions: Default::default(), pending_handler_commands: Default::default(), @@ -182,12 +163,7 @@ impl NetworkBehaviour for Behaviour { if local_addr.is_relayed() { return Ok(Either::Right(dummy::ConnectionHandler)); } - let mut handler = Handler::new( - self.config.clone(), - self.local_peer_id, - peer, - remote_addr.clone(), - ); + let mut handler = Handler::new(self.local_peer_id, peer, remote_addr.clone()); if let Some(event) = self.pending_handler_commands.remove(&connection_id) { handler.on_behaviour_event(event) @@ -207,7 +183,7 @@ impl NetworkBehaviour for Behaviour { return Ok(Either::Right(dummy::ConnectionHandler)); } - let mut handler = Handler::new(self.config.clone(), self.local_peer_id, peer, addr.clone()); + let mut handler = Handler::new(self.local_peer_id, peer, addr.clone()); if let Some(event) = self.pending_handler_commands.remove(&connection_id) { handler.on_behaviour_event(event) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 3ba001fac6b..87c9f3558a4 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::priv_client::{transport, Config}; +use crate::priv_client::transport; use crate::protocol::{self, inbound_stop, outbound_hop}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; use either::Either; @@ -50,6 +50,9 @@ use std::time::Duration; /// Circuits to be denied exceeding the limit are dropped. const MAX_NUMBER_DENYING_CIRCUIT: usize = 8; +const STREAM_TIMEOUT: Duration = Duration::from_secs(60 * 60); +const MAX_CONCURRENT_STREAMS_PER_CONNECTION: usize = 10; + pub enum In { Reserve { to_listener: mpsc::Sender, @@ -168,12 +171,7 @@ pub struct Handler { } impl Handler { - pub fn new( - config: Config, - local_peer_id: PeerId, - remote_peer_id: PeerId, - remote_addr: Multiaddr, - ) -> Self { + pub fn new(local_peer_id: PeerId, remote_peer_id: PeerId, remote_addr: Multiaddr) -> Self { Self { local_peer_id, remote_peer_id, @@ -182,19 +180,19 @@ impl Handler { pending_error: Default::default(), wait_for_reserve_outbound_stream: Default::default(), reserve_futs: futures_bounded::WorkerFutures::new( - config.substream_timeout, - config.max_concurrent_streams, + STREAM_TIMEOUT, + MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), wait_for_connection_outbound_stream: Default::default(), circuit_connection_futs: futures_bounded::WorkerFutures::new( - config.substream_timeout, - config.max_concurrent_streams, + STREAM_TIMEOUT, + MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), open_circuit_futs: futures_bounded::WorkerFutures::new( - config.substream_timeout, - config.max_concurrent_streams, + STREAM_TIMEOUT, + MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), circuit_deny_futs: Default::default(), send_error_futs: Default::default(), diff --git a/protocols/relay/tests/lib.rs b/protocols/relay/tests/lib.rs index 9663cd4bf77..fa79ab67f4b 100644 --- a/protocols/relay/tests/lib.rs +++ b/protocols/relay/tests/lib.rs @@ -334,7 +334,7 @@ fn build_client() -> Swarm { let local_public_key = local_key.public(); let local_peer_id = local_public_key.to_peer_id(); - let (relay_transport, behaviour) = relay::client::new(local_peer_id, Default::default()); + let (relay_transport, behaviour) = relay::client::new(local_peer_id); let transport = upgrade_transport( OrTransport::new(relay_transport, MemoryTransport::default()).boxed(), local_public_key, From a3f2839282111046e5080222549c0840c5723545 Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Sat, 26 Aug 2023 10:55:30 +0300 Subject: [PATCH 33/83] Update protocols/relay/src/behaviour/handler.rs Co-authored-by: Max Inden --- protocols/relay/src/behaviour/handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index b87f4f793ce..c7df15f0a14 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -667,7 +667,7 @@ impl ConnectionHandler for Handler { } // Process protocol requests - if let Poll::Ready((_, worker_res)) = self.protocol_futs.poll_unpin(cx) { + if let Poll::Ready(((), worker_res)) = self.protocol_futs.poll_unpin(cx) { let event = worker_res .unwrap_or_else(|_| ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); From 3a265abcb9d4de582474193e5287dec63369de3e Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 11:00:30 +0300 Subject: [PATCH 34/83] fix review comments --- protocols/relay/src/priv_client/handler.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 87c9f3558a4..247cde30166 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -356,7 +356,7 @@ impl ConnectionHandler for Handler { } // Reservations - if let Poll::Ready((_, worker_res)) = self.reserve_futs.poll_unpin(cx) { + if let Poll::Ready(((), worker_res)) = self.reserve_futs.poll_unpin(cx) { if worker_res.is_err() { return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); } @@ -398,7 +398,7 @@ impl ConnectionHandler for Handler { } // Circuit connections - if let Poll::Ready((_, worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { + if let Poll::Ready(((), worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { if worker_res.is_err() { return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); } @@ -442,7 +442,7 @@ impl ConnectionHandler for Handler { return Poll::Ready(event); } - if let Poll::Ready((_, worker_res)) = self.open_circuit_futs.poll_unpin(cx) { + if let Poll::Ready(((), worker_res)) = self.open_circuit_futs.poll_unpin(cx) { if worker_res.is_err() { return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); } From 38f94c24225d09f4525ba82fdc19a6fa120d2bb4 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 11:56:37 +0300 Subject: [PATCH 35/83] fix toml file --- Cargo.toml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index 4aded393544..0a2fd0dc5f1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -109,6 +109,8 @@ multistream-select = { version = "0.13.0", path = "misc/multistream-select" } quick-protobuf-codec = { version = "0.2.0", path = "misc/quick-protobuf-codec" } quickcheck = { package = "quickcheck-ext", path = "misc/quickcheck-ext" } rw-stream-sink = { version = "0.4.0", path = "misc/rw-stream-sink" } +multiaddr = "0.18.0" +multihash = "0.19.0" [patch.crates-io] From 5ccf0d5f8358802ac791aec83a4c16da5e946bd2 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 12:10:23 +0300 Subject: [PATCH 36/83] fix review comments --- protocols/relay/src/priv_client/handler.rs | 80 +++++++++++----------- 1 file changed, 40 insertions(+), 40 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 247cde30166..ef199cca422 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -355,6 +355,46 @@ impl ConnectionHandler for Handler { return Poll::Ready(ConnectionHandlerEvent::Close(err)); } + // Circuit connections + if let Poll::Ready(((), worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { + if worker_res.is_err() { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + + let opt = match worker_res.unwrap() { + Ok(Some(outbound_hop::Output::Circuit { limit })) => { + Some(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitEstablished { limit }, + )) + } + Ok(None) => None, + Err(err) => { + let res = match err { + outbound_hop::UpgradeError::CircuitFailed(e) => { + ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }, + ) + } + outbound_hop::UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( + StreamUpgradeError::Apply(Either::Right(e)), + ), + outbound_hop::UpgradeError::ReservationFailed(_) => { + unreachable!("do not emit `ReservationFailed` for connection") + } + }; + + Some(res) + } + _ => unreachable!("do not emit 'Output::Reservation' for connection"), + }; + + if let Some(event) = opt { + return Poll::Ready(event); + } + } + // Reservations if let Poll::Ready(((), worker_res)) = self.reserve_futs.poll_unpin(cx) { if worker_res.is_err() { @@ -397,46 +437,6 @@ impl ConnectionHandler for Handler { return Poll::Ready(event); } - // Circuit connections - if let Poll::Ready(((), worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { - if worker_res.is_err() { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } - - let opt = match worker_res.unwrap() { - Ok(Some(outbound_hop::Output::Circuit { limit })) => { - Some(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitEstablished { limit }, - )) - } - Ok(None) => None, - Err(err) => { - let res = match err { - outbound_hop::UpgradeError::CircuitFailed(e) => { - ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(e), - }, - ) - } - outbound_hop::UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( - StreamUpgradeError::Apply(Either::Right(e)), - ), - outbound_hop::UpgradeError::ReservationFailed(_) => { - unreachable!("do not emit `ReservationFailed` for connection") - } - }; - - Some(res) - } - _ => unreachable!("do not emit 'Output::Reservation' for connection"), - }; - - if let Some(event) = opt { - return Poll::Ready(event); - } - } - // Return queued events. if let Some(event) = self.queued_events.pop_front() { return Poll::Ready(event); From 0a87fa6dab42e33415b0e02aaf5d9337be139bd9 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 12:23:40 +0300 Subject: [PATCH 37/83] fix review comments --- protocols/relay/src/priv_client/handler.rs | 34 ++++++++++++++-------- 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index ef199cca422..a91989729c6 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -26,6 +26,7 @@ use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; use futures::sink::SinkExt; use futures::stream::{FuturesUnordered, StreamExt}; +use futures_bounded::Timeout; use futures_timer::Delay; use instant::Instant; use libp2p_core::multiaddr::Protocol; @@ -357,11 +358,14 @@ impl ConnectionHandler for Handler { // Circuit connections if let Poll::Ready(((), worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { - if worker_res.is_err() { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } + let res = match worker_res { + Ok(r) => r, + Err(Timeout { .. }) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + }; - let opt = match worker_res.unwrap() { + let opt = match res { Ok(Some(outbound_hop::Output::Circuit { limit })) => { Some(ConnectionHandlerEvent::NotifyBehaviour( Event::OutboundCircuitEstablished { limit }, @@ -397,11 +401,14 @@ impl ConnectionHandler for Handler { // Reservations if let Poll::Ready(((), worker_res)) = self.reserve_futs.poll_unpin(cx) { - if worker_res.is_err() { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } + let res = match worker_res { + Ok(r) => r, + Err(Timeout { .. }) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + }; - let event = match worker_res.unwrap() { + let event = match res { Ok(outbound_hop::Output::Reservation { renewal_timeout, addrs, @@ -443,11 +450,14 @@ impl ConnectionHandler for Handler { } if let Poll::Ready(((), worker_res)) = self.open_circuit_futs.poll_unpin(cx) { - if worker_res.is_err() { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } + let res = match worker_res { + Ok(r) => r, + Err(Timeout { .. }) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + }; - match worker_res.unwrap() { + match res { Ok(circuit) => match &mut self.reservation { Reservation::Accepted { pending_msgs, .. } | Reservation::Renewing { pending_msgs, .. } => { From e68966d99d0ad5613e0af3596451689767e76bcd Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 14:46:25 +0300 Subject: [PATCH 38/83] refactoring --- protocols/relay/src/priv_client/handler.rs | 198 ++++++++++--------- protocols/relay/src/protocol/outbound_hop.rs | 5 + 2 files changed, 105 insertions(+), 98 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index a91989729c6..484f395eec9 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -19,6 +19,7 @@ // DEALINGS IN THE SOFTWARE. use crate::priv_client::transport; +use crate::protocol::outbound_hop::OutboundStreamInfo; use crate::protocol::{self, inbound_stop, outbound_hop}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; use either::Either; @@ -133,13 +134,11 @@ pub struct Handler { >, >, - wait_for_reserve_outbound_stream: VecDeque>, + wait_for_outbound_stream: VecDeque, reserve_futs: futures_bounded::WorkerFutures< (), Result, >, - - wait_for_connection_outbound_stream: VecDeque, circuit_connection_futs: futures_bounded::WorkerFutures< (), Result, outbound_hop::UpgradeError>, @@ -179,12 +178,11 @@ impl Handler { remote_addr, queued_events: Default::default(), pending_error: Default::default(), - wait_for_reserve_outbound_stream: Default::default(), + wait_for_outbound_stream: Default::default(), reserve_futs: futures_bounded::WorkerFutures::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), - wait_for_connection_outbound_stream: Default::default(), circuit_connection_futs: futures_bounded::WorkerFutures::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, @@ -208,67 +206,66 @@ impl Handler { ::OutboundProtocol, >, ) { - // Try to process the error for a reservation - if let Some(mut to_listener) = self.wait_for_reserve_outbound_stream.pop_front() { - let non_fatal_error = match error { - StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, - StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, - StreamUpgradeError::Io(e) => { - self.pending_error = Some(StreamUpgradeError::Io(e)); - return; - } - StreamUpgradeError::Apply(_) => unreachable!("should not update"), - }; - - 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 occurred, 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(ConnectionHandlerEvent::NotifyBehaviour( - Event::ReservationReqFailed { - renewal, - error: non_fatal_error, - }, - )); - - return; - } - // Try to process the error for a connection - let cmd = self.wait_for_connection_outbound_stream.pop_front().expect( + let outbound_info = self.wait_for_outbound_stream.pop_front().expect( "got a stream error without a pending connection command or a reserve listener", ); + match outbound_info { + OutboundStreamInfo::Reserve(mut to_listener) => { + let non_fatal_error = match error { + StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, + StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, + StreamUpgradeError::Io(e) => { + self.pending_error = Some(StreamUpgradeError::Io(e)); + return; + } + StreamUpgradeError::Apply(_) => unreachable!("should not update"), + }; + + 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 occurred, thus handler is closing as quickly as possible. + // Transport is notified through dropping `to_listener`. + } + + let renewal = self.reservation.failed(); - let non_fatal_error = match error { - StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, - StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, - StreamUpgradeError::Io(e) => { - self.pending_error = Some(StreamUpgradeError::Io(e)); - return; + self.queued_events + .push_back(ConnectionHandlerEvent::NotifyBehaviour( + Event::ReservationReqFailed { + renewal, + error: non_fatal_error, + }, + )); } - StreamUpgradeError::Apply(_) => unreachable!("should not update"), - }; + OutboundStreamInfo::CircuitConnection(cmd) => { + let non_fatal_error = match error { + StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, + StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, + StreamUpgradeError::Io(e) => { + self.pending_error = Some(StreamUpgradeError::Io(e)); + return; + } + StreamUpgradeError::Apply(_) => unreachable!("should not update"), + }; - let _ = cmd.send_back.send(Err(())); + let _ = cmd.send_back.send(Err(())); - self.queued_events - .push_back(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: non_fatal_error, - }, - )); + self.queued_events + .push_back(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: non_fatal_error, + }, + )); + } + } } fn insert_to_deny_futs(&mut self, circuit: inbound_stop::Circuit) { @@ -315,7 +312,8 @@ impl ConnectionHandler for Handler { fn on_behaviour_event(&mut self, event: Self::FromBehaviour) { match event { In::Reserve { to_listener } => { - self.wait_for_reserve_outbound_stream.push_back(to_listener); + self.wait_for_outbound_stream + .push_back(OutboundStreamInfo::Reserve(to_listener)); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -325,8 +323,10 @@ impl ConnectionHandler for Handler { send_back, dst_peer_id, } => { - self.wait_for_connection_outbound_stream - .push_back(outbound_hop::Command::new(dst_peer_id, send_back)); + self.wait_for_outbound_stream + .push_back(OutboundStreamInfo::CircuitConnection( + outbound_hop::Command::new(dst_peer_id, send_back), + )); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -493,7 +493,8 @@ impl ConnectionHandler for Handler { } if let Poll::Ready(Some(to_listener)) = self.reservation.poll(cx) { - self.wait_for_reserve_outbound_stream.push_back(to_listener); + self.wait_for_outbound_stream + .push_back(OutboundStreamInfo::Reserve(to_listener)); return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -582,43 +583,44 @@ impl ConnectionHandler for Handler { protocol: stream, .. }) => { - if let Some(to_listener) = self.wait_for_reserve_outbound_stream.pop_front() { - if self - .reserve_futs - .try_push( - (), - outbound_hop::handle_reserve_message_response(stream, to_listener) - .boxed(), - ) - .is_some() - { - log::warn!("Dropping outbound stream because we are at capacity") - } - return; - } - - let con_command = self.wait_for_connection_outbound_stream.pop_front().expect( + let outbound_info = self.wait_for_outbound_stream.pop_front().expect( "opened a stream without a pending connection command or a reserve listener", ); + match outbound_info { + OutboundStreamInfo::Reserve(to_listener) => { + if self + .reserve_futs + .try_push( + (), + outbound_hop::handle_reserve_message_response(stream, to_listener) + .boxed(), + ) + .is_some() + { + log::warn!("Dropping outbound stream because we are at capacity") + } + } + OutboundStreamInfo::CircuitConnection(cmd) => { + let (tx, rx) = oneshot::channel(); + self.alive_lend_out_substreams.push(rx); - let (tx, rx) = oneshot::channel(); - self.alive_lend_out_substreams.push(rx); - - if self - .circuit_connection_futs - .try_push( - (), - outbound_hop::handle_connection_message_response( - stream, - self.remote_peer_id, - con_command, - tx, - ) - .boxed(), - ) - .is_some() - { - log::warn!("Dropping outbound stream because we are at capacity") + if self + .circuit_connection_futs + .try_push( + (), + outbound_hop::handle_connection_message_response( + stream, + self.remote_peer_id, + cmd, + tx, + ) + .boxed(), + ) + .is_some() + { + log::warn!("Dropping outbound stream because we are at capacity") + } + } } } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 86276f664cb..32aacfbfd1f 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -288,6 +288,11 @@ pub(crate) async fn handle_connection_message_response( Ok(output) } +pub(crate) enum OutboundStreamInfo { + Reserve(mpsc::Sender), + CircuitConnection(Command), +} + pub(crate) struct Command { dst_peer_id: PeerId, pub(crate) send_back: oneshot::Sender>, From f5414100279301abba7bf5d7913f0693ed8f7991 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 26 Aug 2023 18:47:13 +0300 Subject: [PATCH 39/83] =?UTF-8?q?=D0=B0ix=20failed=20tests?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- protocols/relay/src/priv_client/transport.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/protocols/relay/src/priv_client/transport.rs b/protocols/relay/src/priv_client/transport.rs index 8a63ca5a80a..41114d0cdd5 100644 --- a/protocols/relay/src/priv_client/transport.rs +++ b/protocols/relay/src/priv_client/transport.rs @@ -55,7 +55,7 @@ use thiserror::Error; /// # use libp2p_identity::PeerId; /// let actual_transport = MemoryTransport::default(); /// let (relay_transport, behaviour) = relay::client::new( -/// PeerId::random(), Default::default() +/// PeerId::random() /// ); /// let mut transport = OrTransport::new(relay_transport, actual_transport); /// # let relay_id = PeerId::random(); @@ -80,7 +80,7 @@ use thiserror::Error; /// # let local_peer_id = PeerId::random(); /// let actual_transport = MemoryTransport::default(); /// let (relay_transport, behaviour) = relay::client::new( -/// local_peer_id, Default::default() +/// local_peer_id /// ); /// let mut transport = OrTransport::new(relay_transport, actual_transport); /// let relay_addr = Multiaddr::empty() From 07242771f43ced3d52ec6ee4be7cb83f4f3521c8 Mon Sep 17 00:00:00 2001 From: dgarus Date: Mon, 4 Sep 2023 14:39:45 +0300 Subject: [PATCH 40/83] fix review comments --- protocols/relay/src/behaviour/handler.rs | 2 +- protocols/relay/src/protocol/outbound_stop.rs | 67 +++++++++---------- 2 files changed, 32 insertions(+), 37 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index c7df15f0a14..720d0e4de18 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -45,7 +45,7 @@ use std::fmt; use std::task::{Context, Poll}; use std::time::Duration; -const STREAM_TIMEOUT: Duration = Duration::from_secs(60 * 60); +const STREAM_TIMEOUT: Duration = Duration::from_secs(60); const MAX_CONCURRENT_STREAMS_PER_CONNECTION: usize = 10; #[derive(Debug, Clone)] diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 7279dd63ac1..6f36801eee6 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -100,8 +100,7 @@ pub(crate) async fn handle_stop_message_response( let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - let send_res = substream.send(msg).await; - if send_res.is_err() { + if substream.send(msg).await.is_err() { return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( FatalUpgradeError::StreamClosed, ))); @@ -132,40 +131,36 @@ pub(crate) async fn handle_stop_message_response( } match status { - Some(proto_status) => match proto_status { - proto::Status::OK => {} - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply( - CircuitFailedReason::ResourceLimitExceeded, - ), - }, - ) - } - proto::Status::PERMISSION_DENIED => { - return ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto_status, - error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), - }, - ) - } - s => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( - FatalUpgradeError::UnexpectedStatus(s), - ))) - } - }, + Some(proto::Status::OK) => {} + Some(proto::Status::RESOURCE_LIMIT_EXCEEDED) => { + return ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto::Status::RESOURCE_LIMIT_EXCEEDED, + error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), + }, + ) + } + Some(proto::Status::PERMISSION_DENIED) => { + return ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundConnectNegotiationFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto::Status::PERMISSION_DENIED, + error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), + }, + ) + } + Some(s) => { + return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( + FatalUpgradeError::UnexpectedStatus(s), + ))) + } None => { return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( FatalUpgradeError::MissingStatusField, From bb789c2475141fc2b025e9946ed481451c4d0b47 Mon Sep 17 00:00:00 2001 From: dgarus Date: Tue, 5 Sep 2023 16:05:38 +0300 Subject: [PATCH 41/83] bounded and unique workers sets --- misc/futures-bounded/src/bounded.rs | 186 ++++++++++++++++++ misc/futures-bounded/src/lib.rs | 191 ++---------------- misc/futures-bounded/src/unique.rs | 218 +++++++++++++++++++++ protocols/relay/src/behaviour/handler.rs | 18 +- protocols/relay/src/priv_client/handler.rs | 114 +++++------ 5 files changed, 477 insertions(+), 250 deletions(-) create mode 100644 misc/futures-bounded/src/bounded.rs create mode 100644 misc/futures-bounded/src/unique.rs diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs new file mode 100644 index 00000000000..850f09c0545 --- /dev/null +++ b/misc/futures-bounded/src/bounded.rs @@ -0,0 +1,186 @@ +use std::future::Future; +use std::task::{Context, Poll, Waker}; +use std::time::Duration; + +use futures_timer::Delay; +use futures_util::future::{select, BoxFuture, Either}; +use futures_util::stream::FuturesUnordered; +use futures_util::{ready, FutureExt, StreamExt}; + +use crate::{PushResult, Timeout}; + +/// Represents a set of (Worker)-[Future]s. +/// +/// This wraps [FuturesUnordered] but bounds it by time and size. +/// In other words, each worker must finish within the specified time and the set never outgrows its capacity. +pub struct BoundedWorkers { + timeout: Duration, + capacity: usize, + inner: FuturesUnordered>>, + + empty_waker: Option, + full_waker: Option, +} + +impl BoundedWorkers { + pub fn new(timeout: Duration, capacity: usize) -> Self { + Self { + timeout, + capacity, + inner: Default::default(), + empty_waker: None, + full_waker: None, + } + } +} + +impl BoundedWorkers { + /// Push a worker into the set. + /// This method adds the given worker to the set and returns [PushResult::Ok]. + /// If length of the set is equal the capacity, this method returns [PushResult::BeyondCapacity] + /// and worker is ignored. + pub fn try_push(&mut self, worker: F) -> PushResult + where + F: Future + Send + 'static + Unpin, + { + if self.inner.len() >= self.capacity { + return PushResult::BeyondCapacity; + } + let timeout = Delay::new(self.timeout); + + self.inner.push( + async move { + match select(worker, timeout).await { + Either::Left((out, _)) => Ok(out), + Either::Right(((), _)) => Err(Timeout::new()), + } + } + .boxed(), + ); + + if let Some(waker) = self.empty_waker.take() { + waker.wake(); + } + + PushResult::Ok + } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { + if self.inner.len() < self.capacity { + return Poll::Ready(()); + } + + self.full_waker = Some(cx.waker().clone()); + Poll::Pending + } + + pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll> { + match ready!(self.inner.poll_next_unpin(cx)) { + None => { + self.empty_waker = Some(cx.waker().clone()); + Poll::Pending + } + Some(result) => { + if let Some(waker) = self.full_waker.take() { + waker.wake(); + } + + Poll::Ready(result) + } + } + } +} + +#[cfg(test)] +mod tests { + use std::future::{pending, poll_fn, ready}; + use std::pin::Pin; + use std::time::Instant; + + use super::*; + + #[test] + fn cannot_push_more_than_capacity_tasks() { + let mut workers = BoundedWorkers::new(Duration::from_secs(10), 1); + + assert!(workers.try_push(ready(())).is_ok()); + assert_eq!(workers.try_push(ready(())), PushResult::BeyondCapacity); + } + + #[tokio::test] + async fn workers_timeout() { + let mut workers = BoundedWorkers::new(Duration::from_millis(100), 1); + + let _ = workers.try_push(pending::<()>()); + Delay::new(Duration::from_millis(150)).await; + let result = poll_fn(|cx| workers.poll_unpin(cx)).await; + + assert!(result.is_err()) + } + + // Each worker causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. + // We stop after NUM_WORKERS tasks, meaning the overall execution must at least take DELAY * NUM_WORKERS. + #[tokio::test] + async fn backpressure() { + const DELAY: Duration = Duration::from_millis(100); + const NUM_WORKERS: u32 = 10; + + let start = Instant::now(); + Task::new(DELAY, NUM_WORKERS, 1).await; + let duration = start.elapsed(); + + assert!(duration >= DELAY * NUM_WORKERS); + } + + struct Task { + worker: Duration, + num_workers: usize, + num_processed: usize, + inner: BoundedWorkers<()>, + } + + impl Task { + fn new(worker: Duration, num_workers: u32, capacity: usize) -> Self { + Self { + worker, + num_workers: num_workers as usize, + num_processed: 0, + inner: BoundedWorkers::new(Duration::from_secs(60), capacity), + } + } + } + + impl Future for Task { + type Output = (); + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.get_mut(); + + while this.num_processed < this.num_workers { + if let Poll::Ready(result) = this.inner.poll_unpin(cx) { + if result.is_err() { + panic!("Timeout is great than worker delay") + } + + this.num_processed += 1; + continue; + } + + if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { + let maybe_worker = this.inner.try_push(Delay::new(this.worker)); + assert!(maybe_worker.is_ok(), "we polled for readiness"); + + continue; + } + + return Poll::Pending; + } + + Poll::Ready(()) + } + } +} diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs index 576981e08a1..311fc6f5961 100644 --- a/misc/futures-bounded/src/lib.rs +++ b/misc/futures-bounded/src/lib.rs @@ -1,96 +1,8 @@ -use std::future::Future; -use std::task::{Context, Poll, Waker}; -use std::time::Duration; +mod bounded; +mod unique; -use futures_timer::Delay; -use futures_util::future::{select, BoxFuture, Either}; -use futures_util::stream::FuturesUnordered; -use futures_util::{ready, FutureExt, StreamExt}; - -/// Represents a set of (Worker)-[Future]s. -/// -/// This wraps [FuturesUnordered] but bounds it by time and size. -/// In other words, each worker must finish within the specified time and the set never outgrows its capacity. -pub struct WorkerFutures { - timeout: Duration, - capacity: usize, - inner: FuturesUnordered)>>, - - empty_waker: Option, - full_waker: Option, -} - -impl WorkerFutures { - pub fn new(timeout: Duration, capacity: usize) -> Self { - Self { - timeout, - capacity, - inner: Default::default(), - empty_waker: None, - full_waker: None, - } - } -} - -impl WorkerFutures -where - ID: Send + 'static, -{ - pub fn try_push(&mut self, worker_id: ID, worker: F) -> Option - where - F: Future + Send + 'static + Unpin, - { - if self.inner.len() >= self.capacity { - return Some(worker); - } - let timeout = Delay::new(self.timeout); - - self.inner.push( - async move { - match select(worker, timeout).await { - Either::Left((out, _)) => (worker_id, Ok(out)), - Either::Right(((), _)) => (worker_id, Err(Timeout::new())), - } - } - .boxed(), - ); - - if let Some(waker) = self.empty_waker.take() { - waker.wake(); - } - - None - } - - pub fn is_empty(&self) -> bool { - self.inner.is_empty() - } - - pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { - if self.inner.len() < self.capacity { - return Poll::Ready(()); - } - - self.full_waker = Some(cx.waker().clone()); - Poll::Pending - } - - pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll<(ID, Result)> { - match ready!(self.inner.poll_next_unpin(cx)) { - None => { - self.empty_waker = Some(cx.waker().clone()); - Poll::Pending - } - Some(result) => { - if let Some(waker) = self.full_waker.take() { - waker.wake(); - } - - Poll::Ready(result) - } - } - } -} +pub use bounded::BoundedWorkers; +pub use unique::UniqueWorkers; #[derive(Debug)] pub struct Timeout { @@ -103,91 +15,20 @@ impl Timeout { } } -#[cfg(test)] -mod tests { - use super::*; - use std::future::{pending, poll_fn, ready}; - use std::pin::Pin; - use std::time::Instant; - - #[test] - fn cannot_push_more_than_capacity_tasks() { - let mut workers = WorkerFutures::new(Duration::from_secs(10), 1); - - assert!(workers.try_push((), ready(())).is_none()); - assert!(workers.try_push((), ready(())).is_some()); - } - - #[tokio::test] - async fn workers_timeout() { - let mut workers = WorkerFutures::new(Duration::from_millis(100), 1); - - let _ = workers.try_push((), pending::<()>()); - Delay::new(Duration::from_millis(150)).await; - let (_, result) = poll_fn(|cx| workers.poll_unpin(cx)).await; - - assert!(result.is_err()) - } - - // Each worker causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. - // We stop after NUM_WORKERS tasks, meaning the overall execution must at least take DELAY * NUM_WORKERS. - #[tokio::test] - async fn backpressure() { - const DELAY: Duration = Duration::from_millis(100); - const NUM_WORKERS: u32 = 10; - - let start = Instant::now(); - Task::new(DELAY, NUM_WORKERS, 1).await; - let duration = start.elapsed(); - - assert!(duration >= DELAY * NUM_WORKERS); - } - - struct Task { - worker: Duration, - num_workers: usize, - num_processed: usize, - inner: WorkerFutures<(), ()>, - } +/// Result of a worker pushing +#[derive(PartialEq, Debug)] +pub enum PushResult { + Ok, + BeyondCapacity, + ExistedID, +} - impl Task { - fn new(worker: Duration, num_workers: u32, capacity: usize) -> Self { - Self { - worker, - num_workers: num_workers as usize, - num_processed: 0, - inner: WorkerFutures::new(Duration::from_secs(60), capacity), - } - } +impl PushResult { + pub fn is_ok(&self) -> bool { + matches!(self, PushResult::Ok) } - impl Future for Task { - type Output = (); - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.get_mut(); - - while this.num_processed < this.num_workers { - if let Poll::Ready(((), result)) = this.inner.poll_unpin(cx) { - if result.is_err() { - panic!("Timeout is great than worker delay") - } - - this.num_processed += 1; - continue; - } - - if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { - let maybe_worker = this.inner.try_push((), Delay::new(this.worker)); - assert!(maybe_worker.is_none(), "we polled for readiness"); - - continue; - } - - return Poll::Pending; - } - - Poll::Ready(()) - } + pub fn is_failing(&self) -> bool { + !self.is_ok() } } diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/unique.rs new file mode 100644 index 00000000000..9925013cbff --- /dev/null +++ b/misc/futures-bounded/src/unique.rs @@ -0,0 +1,218 @@ +use std::collections::HashSet; +use std::future::Future; +use std::hash::Hash; +use std::task::{Context, Poll, Waker}; +use std::time::Duration; + +use futures_timer::Delay; +use futures_util::future::{select, BoxFuture, Either}; +use futures_util::stream::FuturesUnordered; +use futures_util::{ready, FutureExt, StreamExt}; + +use crate::{PushResult, Timeout}; + +/// Represents a set of (Worker)-[Future]s. +/// +/// This wraps [FuturesUnordered] but bounds it by time and size. Every worker has to have a unique ID. +/// In other words, each worker must finish within the specified time and the set never outgrows its capacity. +pub struct UniqueWorkers { + timeout: Duration, + capacity: usize, + keys: HashSet, + inner: FuturesUnordered)>>, + + empty_waker: Option, + full_waker: Option, +} + +impl UniqueWorkers { + pub fn new(timeout: Duration, capacity: usize) -> Self { + Self { + timeout, + capacity, + keys: HashSet::with_capacity(capacity), + inner: Default::default(), + empty_waker: None, + full_waker: None, + } + } +} + +impl UniqueWorkers +where + ID: Clone + Hash + Eq + Send + 'static, +{ + /// Push a worker into the set. + /// This method adds the given worker with defined `worker_id` to the set and returns [PushResult::Ok]. + /// The result [PushResult::ExistedID] says that the set already contains passed `worker_id`. + /// If length of the set is equal the capacity, this method returns [PushResult::BeyondCapacity]. + /// The result different from [PushResult::Ok] means that worker was not added to the set. + pub fn try_push(&mut self, worker_id: ID, worker: F) -> PushResult + where + F: Future + Send + 'static + Unpin, + { + // Check worker's ID + if !self.keys.insert(worker_id.clone()) { + return PushResult::ExistedID; + } + // Check capacity + if self.inner.len() >= self.capacity { + return PushResult::BeyondCapacity; + } + + let timeout = Delay::new(self.timeout); + + self.inner.push( + async move { + match select(worker, timeout).await { + Either::Left((out, _)) => (worker_id, Ok(out)), + Either::Right(((), _)) => (worker_id, Err(Timeout::new())), + } + } + .boxed(), + ); + + if let Some(waker) = self.empty_waker.take() { + waker.wake(); + } + + PushResult::Ok + } + + pub fn is_empty(&self) -> bool { + assert_eq!(self.keys.is_empty(), self.keys.is_empty()); + + self.inner.is_empty() + } + + pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { + if self.inner.len() < self.capacity { + return Poll::Ready(()); + } + + self.full_waker = Some(cx.waker().clone()); + Poll::Pending + } + + pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll<(ID, Result)> { + match ready!(self.inner.poll_next_unpin(cx)) { + None => { + self.empty_waker = Some(cx.waker().clone()); + Poll::Pending + } + Some(result) => { + if let Some(waker) = self.full_waker.take() { + waker.wake(); + } + + self.keys.remove(&result.0); + + Poll::Ready(result) + } + } + } +} + +#[cfg(test)] +mod tests { + use std::future::{pending, poll_fn, ready}; + use std::pin::Pin; + use std::time::Instant; + + use super::*; + + #[test] + fn cannot_push_more_than_capacity_tasks() { + let mut workers = UniqueWorkers::new(Duration::from_secs(10), 1); + + assert!(workers.try_push("ID_1", ready(())).is_ok()); + assert_eq!( + workers.try_push("ID_2", ready(())), + PushResult::BeyondCapacity + ); + } + + #[test] + fn cannot_push_the_same_id_few_times() { + let mut workers = UniqueWorkers::new(Duration::from_secs(10), 5); + + assert!(workers.try_push("ID", ready(())).is_ok()); + assert_eq!(workers.try_push("ID", ready(())), PushResult::ExistedID); + assert_eq!(workers.try_push("ID", ready(())), PushResult::ExistedID); + } + + #[tokio::test] + async fn workers_timeout() { + let mut workers = UniqueWorkers::new(Duration::from_millis(100), 1); + + let _ = workers.try_push("ID", pending::<()>()); + Delay::new(Duration::from_millis(150)).await; + let (_, result) = poll_fn(|cx| workers.poll_unpin(cx)).await; + + assert!(result.is_err()) + } + + // Each worker causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. + // We stop after NUM_WORKERS tasks, meaning the overall execution must at least take DELAY * NUM_WORKERS. + #[tokio::test] + async fn backpressure() { + const DELAY: Duration = Duration::from_millis(100); + const NUM_WORKERS: u32 = 10; + + let start = Instant::now(); + Task::new(DELAY, NUM_WORKERS, 1).await; + let duration = start.elapsed(); + + assert!(duration >= DELAY * NUM_WORKERS); + } + + struct Task { + worker: Duration, + num_workers: usize, + num_processed: usize, + inner: UniqueWorkers, + } + + impl Task { + fn new(worker: Duration, num_workers: u32, capacity: usize) -> Self { + Self { + worker, + num_workers: num_workers as usize, + num_processed: 0, + inner: UniqueWorkers::new(Duration::from_secs(60), capacity), + } + } + } + + impl Future for Task { + type Output = (); + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.get_mut(); + + while this.num_processed < this.num_workers { + if let Poll::Ready((_, result)) = this.inner.poll_unpin(cx) { + if result.is_err() { + panic!("Timeout is great than worker delay") + } + + this.num_processed += 1; + continue; + } + + if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { + // We push the constant worker's ID to prove that user can use the same ID + // if the worker's future was finished + let maybe_worker = this.inner.try_push(1u8, Delay::new(this.worker)); + assert!(maybe_worker.is_ok(), "we polled for readiness"); + + continue; + } + + return Poll::Pending; + } + + Poll::Ready(()) + } + } +} diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 720d0e4de18..6e1278cafe1 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -45,8 +45,8 @@ use std::fmt; use std::task::{Context, Poll}; use std::time::Duration; -const STREAM_TIMEOUT: Duration = Duration::from_secs(60); const MAX_CONCURRENT_STREAMS_PER_CONNECTION: usize = 10; +const STREAM_TIMEOUT: Duration = Duration::from_secs(60); #[derive(Debug, Clone)] pub struct Config { @@ -385,13 +385,13 @@ pub struct Handler { circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, stop_requested_streams: VecDeque, - protocol_futs: futures_bounded::WorkerFutures<(), RelayConnectionHandlerEvent>, + protocol_futs: futures_bounded::BoundedWorkers, } impl Handler { pub fn new(config: Config, endpoint: ConnectedPoint) -> Handler { Handler { - protocol_futs: futures_bounded::WorkerFutures::new( + protocol_futs: futures_bounded::BoundedWorkers::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), @@ -414,7 +414,6 @@ impl Handler { if self .protocol_futs .try_push( - (), inbound_hop::handle_inbound_request( stream, self.config.reservation_duration, @@ -425,7 +424,7 @@ impl Handler { ) .boxed(), ) - .is_some() + .is_failing() { log::warn!("Dropping inbound stream because we are at capacity") } @@ -442,11 +441,8 @@ impl Handler { if self .protocol_futs - .try_push( - (), - outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed(), - ) - .is_some() + .try_push(outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed()) + .is_failing() { log::warn!("Dropping outbound stream because we are at capacity") } @@ -667,7 +663,7 @@ impl ConnectionHandler for Handler { } // Process protocol requests - if let Poll::Ready(((), worker_res)) = self.protocol_futs.poll_unpin(cx) { + if let Poll::Ready(worker_res) = self.protocol_futs.poll_unpin(cx) { let event = worker_res .unwrap_or_else(|_| ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 484f395eec9..5ca0e62d594 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -27,7 +27,7 @@ use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; use futures::sink::SinkExt; use futures::stream::{FuturesUnordered, StreamExt}; -use futures_bounded::Timeout; +use futures_bounded::{PushResult, Timeout}; use futures_timer::Delay; use instant::Instant; use libp2p_core::multiaddr::Protocol; @@ -42,7 +42,7 @@ use libp2p_swarm::{ SubstreamProtocol, }; use log::debug; -use std::collections::{HashMap, VecDeque}; +use std::collections::VecDeque; use std::fmt; use std::task::{Context, Poll}; use std::time::Duration; @@ -51,9 +51,10 @@ use std::time::Duration; /// /// Circuits to be denied exceeding the limit are dropped. const MAX_NUMBER_DENYING_CIRCUIT: usize = 8; +const DENYING_CIRCUIT_TIMEOUT: Duration = Duration::from_secs(60); -const STREAM_TIMEOUT: Duration = Duration::from_secs(60 * 60); const MAX_CONCURRENT_STREAMS_PER_CONNECTION: usize = 10; +const STREAM_TIMEOUT: Duration = Duration::from_secs(60); pub enum In { Reserve { @@ -135,12 +136,9 @@ pub struct Handler { >, wait_for_outbound_stream: VecDeque, - reserve_futs: futures_bounded::WorkerFutures< - (), - Result, - >, - circuit_connection_futs: futures_bounded::WorkerFutures< - (), + reserve_futs: + futures_bounded::BoundedWorkers>, + circuit_connection_futs: futures_bounded::BoundedWorkers< Result, outbound_hop::UpgradeError>, >, @@ -156,12 +154,12 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - open_circuit_futs: futures_bounded::WorkerFutures< - (), + open_circuit_futs: futures_bounded::BoundedWorkers< Result, >, - circuit_deny_futs: HashMap>>, + circuit_deny_futs: + futures_bounded::UniqueWorkers>, /// Futures that try to send errors to the transport. /// @@ -179,21 +177,24 @@ impl Handler { queued_events: Default::default(), pending_error: Default::default(), wait_for_outbound_stream: Default::default(), - reserve_futs: futures_bounded::WorkerFutures::new( + reserve_futs: futures_bounded::BoundedWorkers::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), - circuit_connection_futs: futures_bounded::WorkerFutures::new( + circuit_connection_futs: futures_bounded::BoundedWorkers::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), - open_circuit_futs: futures_bounded::WorkerFutures::new( + open_circuit_futs: futures_bounded::BoundedWorkers::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), - circuit_deny_futs: Default::default(), + circuit_deny_futs: futures_bounded::UniqueWorkers::new( + DENYING_CIRCUIT_TIMEOUT, + MAX_NUMBER_DENYING_CIRCUIT, + ), send_error_futs: Default::default(), keep_alive: KeepAlive::Yes, } @@ -271,25 +272,19 @@ impl Handler { fn insert_to_deny_futs(&mut self, circuit: inbound_stop::Circuit) { let src_peer_id = circuit.src_peer_id(); - if self.circuit_deny_futs.len() == MAX_NUMBER_DENYING_CIRCUIT - && !self.circuit_deny_futs.contains_key(&src_peer_id) - { - log::warn!( + match self.circuit_deny_futs.try_push( + src_peer_id, + circuit.deny(proto::Status::NO_RESERVATION).boxed(), + ) { + PushResult::Ok => {} + PushResult::BeyondCapacity => log::warn!( "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", src_peer_id - ); - return; - } - - if self - .circuit_deny_futs - .insert( - src_peer_id, - circuit.deny(proto::Status::NO_RESERVATION).boxed(), - ) - .is_some() - { - log::warn!("Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", src_peer_id); + ), + PushResult::ExistedID => log::warn!( + "Dropping inbound circuit request to be denied from {:?} in favor of existing one.", + src_peer_id + ), } } } @@ -357,7 +352,7 @@ impl ConnectionHandler for Handler { } // Circuit connections - if let Poll::Ready(((), worker_res)) = self.circuit_connection_futs.poll_unpin(cx) { + if let Poll::Ready(worker_res) = self.circuit_connection_futs.poll_unpin(cx) { let res = match worker_res { Ok(r) => r, Err(Timeout { .. }) => { @@ -400,7 +395,7 @@ impl ConnectionHandler for Handler { } // Reservations - if let Poll::Ready(((), worker_res)) = self.reserve_futs.poll_unpin(cx) { + if let Poll::Ready(worker_res) = self.reserve_futs.poll_unpin(cx) { let res = match worker_res { Ok(r) => r, Err(Timeout { .. }) => { @@ -449,7 +444,7 @@ impl ConnectionHandler for Handler { return Poll::Ready(event); } - if let Poll::Ready(((), worker_res)) = self.open_circuit_futs.poll_unpin(cx) { + if let Poll::Ready(worker_res) = self.open_circuit_futs.poll_unpin(cx) { let res = match worker_res { Ok(r) => r, Err(Timeout { .. }) => { @@ -502,28 +497,21 @@ impl ConnectionHandler for Handler { } // Deny incoming circuit requests. - let maybe_event = - self.circuit_deny_futs - .iter_mut() - .find_map(|(src_peer_id, fut)| match fut.poll_unpin(cx) { - Poll::Ready(Ok(())) => Some(( - *src_peer_id, - Event::InboundCircuitReqDenied { - src_peer_id: *src_peer_id, - }, - )), - Poll::Ready(Err(error)) => Some(( - *src_peer_id, - Event::InboundCircuitReqDenyFailed { - src_peer_id: *src_peer_id, - error, - }, - )), - Poll::Pending => None, - }); - if let Some((src_peer_id, event)) = maybe_event { - self.circuit_deny_futs.remove(&src_peer_id); - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour(event)); + match self.circuit_deny_futs.poll_unpin(cx) { + Poll::Ready((src_peer_id, Ok(Ok(())))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::InboundCircuitReqDenied { src_peer_id }, + )); + } + Poll::Ready((src_peer_id, Ok(Err(error)))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::InboundCircuitReqDenyFailed { src_peer_id, error }, + )); + } + Poll::Ready((src_peer_id, Err(Timeout { .. }))) => { + log::warn!("Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", src_peer_id); + } + Poll::Pending => {} } // Send errors to transport. @@ -573,8 +561,8 @@ impl ConnectionHandler for Handler { }) => { if self .open_circuit_futs - .try_push((), inbound_stop::handle_open_circuit(stream).boxed()) - .is_some() + .try_push(inbound_stop::handle_open_circuit(stream).boxed()) + .is_failing() { log::warn!("Dropping inbound stream because we are at capacity") } @@ -591,11 +579,10 @@ impl ConnectionHandler for Handler { if self .reserve_futs .try_push( - (), outbound_hop::handle_reserve_message_response(stream, to_listener) .boxed(), ) - .is_some() + .is_failing() { log::warn!("Dropping outbound stream because we are at capacity") } @@ -607,7 +594,6 @@ impl ConnectionHandler for Handler { if self .circuit_connection_futs .try_push( - (), outbound_hop::handle_connection_message_response( stream, self.remote_peer_id, @@ -616,7 +602,7 @@ impl ConnectionHandler for Handler { ) .boxed(), ) - .is_some() + .is_failing() { log::warn!("Dropping outbound stream because we are at capacity") } From 13bb338a9d4e7f552d7f2fea0f331283ced1fde8 Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 6 Sep 2023 16:36:52 +0300 Subject: [PATCH 42/83] rewrite unique workers --- misc/futures-bounded/src/bounded.rs | 19 +++-- misc/futures-bounded/src/lib.rs | 20 +---- misc/futures-bounded/src/unique.rs | 99 ++++++++++++---------- protocols/relay/src/behaviour/handler.rs | 4 +- protocols/relay/src/priv_client/handler.rs | 16 ++-- 5 files changed, 75 insertions(+), 83 deletions(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index 850f09c0545..762e74aea37 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -7,7 +7,7 @@ use futures_util::future::{select, BoxFuture, Either}; use futures_util::stream::FuturesUnordered; use futures_util::{ready, FutureExt, StreamExt}; -use crate::{PushResult, Timeout}; +use crate::Timeout; /// Represents a set of (Worker)-[Future]s. /// @@ -36,18 +36,19 @@ impl BoundedWorkers { impl BoundedWorkers { /// Push a worker into the set. - /// This method adds the given worker to the set and returns [PushResult::Ok]. - /// If length of the set is equal the capacity, this method returns [PushResult::BeyondCapacity] - /// and worker is ignored. - pub fn try_push(&mut self, worker: F) -> PushResult + /// This method adds the given worker to the set. + /// If the length of the set is equal to the capacity, + /// this method returns a error that contains the passed worker. + /// In that case, the worker is not added to the set. + pub fn try_push(&mut self, worker: F) -> Result<(), F> where F: Future + Send + 'static + Unpin, { if self.inner.len() >= self.capacity { - return PushResult::BeyondCapacity; + return Err(worker); } - let timeout = Delay::new(self.timeout); + let timeout = Delay::new(self.timeout); self.inner.push( async move { match select(worker, timeout).await { @@ -62,7 +63,7 @@ impl BoundedWorkers { waker.wake(); } - PushResult::Ok + Ok(()) } pub fn is_empty(&self) -> bool { @@ -108,7 +109,7 @@ mod tests { let mut workers = BoundedWorkers::new(Duration::from_secs(10), 1); assert!(workers.try_push(ready(())).is_ok()); - assert_eq!(workers.try_push(ready(())), PushResult::BeyondCapacity); + matches!(workers.try_push(ready(())), Err(_)); } #[tokio::test] diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs index 311fc6f5961..1c6154072bd 100644 --- a/misc/futures-bounded/src/lib.rs +++ b/misc/futures-bounded/src/lib.rs @@ -2,7 +2,7 @@ mod bounded; mod unique; pub use bounded::BoundedWorkers; -pub use unique::UniqueWorkers; +pub use unique::{PushError, UniqueWorkers}; #[derive(Debug)] pub struct Timeout { @@ -14,21 +14,3 @@ impl Timeout { Self { _priv: () } } } - -/// Result of a worker pushing -#[derive(PartialEq, Debug)] -pub enum PushResult { - Ok, - BeyondCapacity, - ExistedID, -} - -impl PushResult { - pub fn is_ok(&self) -> bool { - matches!(self, PushResult::Ok) - } - - pub fn is_failing(&self) -> bool { - !self.is_ok() - } -} diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/unique.rs index 9925013cbff..3d3df5ccd25 100644 --- a/misc/futures-bounded/src/unique.rs +++ b/misc/futures-bounded/src/unique.rs @@ -1,15 +1,14 @@ -use std::collections::HashSet; +use std::collections::HashMap; use std::future::Future; use std::hash::Hash; use std::task::{Context, Poll, Waker}; use std::time::Duration; use futures_timer::Delay; -use futures_util::future::{select, BoxFuture, Either}; -use futures_util::stream::FuturesUnordered; -use futures_util::{ready, FutureExt, StreamExt}; +use futures_util::future::BoxFuture; +use futures_util::FutureExt; -use crate::{PushResult, Timeout}; +use crate::Timeout; /// Represents a set of (Worker)-[Future]s. /// @@ -18,19 +17,25 @@ use crate::{PushResult, Timeout}; pub struct UniqueWorkers { timeout: Duration, capacity: usize, - keys: HashSet, - inner: FuturesUnordered)>>, - + inner: HashMap, Delay)>, empty_waker: Option, full_waker: Option, } +/// Error of a worker pushing +#[derive(PartialEq, Debug)] +pub enum PushError { + /// The length of the set is equal the capacity + BeyondCapacity(F), + /// The set already contains the given worker's ID + ReplacedWorker(F), +} + impl UniqueWorkers { pub fn new(timeout: Duration, capacity: usize) -> Self { Self { timeout, capacity, - keys: HashSet::with_capacity(capacity), inner: Default::default(), empty_waker: None, full_waker: None, @@ -43,45 +48,35 @@ where ID: Clone + Hash + Eq + Send + 'static, { /// Push a worker into the set. - /// This method adds the given worker with defined `worker_id` to the set and returns [PushResult::Ok]. - /// The result [PushResult::ExistedID] says that the set already contains passed `worker_id`. - /// If length of the set is equal the capacity, this method returns [PushResult::BeyondCapacity]. - /// The result different from [PushResult::Ok] means that worker was not added to the set. - pub fn try_push(&mut self, worker_id: ID, worker: F) -> PushResult + /// This method adds the given worker with defined `worker_id` to the set. + /// If the length of the set is equal to the capacity, this method returns [PushError::BeyondCapacity], + /// that contains the passed worker. In that case, the worker is not added to the set. + /// If a worker with the given `worker_id` already exists, then the old worker will be replaced by a new one. + /// In that case, the returned error [PushError::ReplacedWorker] contains the old worker. + pub fn try_push(&mut self, worker_id: ID, worker: F) -> Result<(), PushError>> where F: Future + Send + 'static + Unpin, { - // Check worker's ID - if !self.keys.insert(worker_id.clone()) { - return PushResult::ExistedID; - } - // Check capacity if self.inner.len() >= self.capacity { - return PushResult::BeyondCapacity; + return Err(PushError::BeyondCapacity(worker.boxed())); } - let timeout = Delay::new(self.timeout); - - self.inner.push( - async move { - match select(worker, timeout).await { - Either::Left((out, _)) => (worker_id, Ok(out)), - Either::Right(((), _)) => (worker_id, Err(Timeout::new())), - } - } - .boxed(), + let val = self.inner.insert( + worker_id.clone(), + (worker.boxed(), Delay::new(self.timeout)), ); if let Some(waker) = self.empty_waker.take() { waker.wake(); } - PushResult::Ok + match val { + Some((old_worker, _)) => Err(PushError::ReplacedWorker(old_worker)), + None => Ok(()), + } } pub fn is_empty(&self) -> bool { - assert_eq!(self.keys.is_empty(), self.keys.is_empty()); - self.inner.is_empty() } @@ -91,23 +86,35 @@ where } self.full_waker = Some(cx.waker().clone()); + Poll::Pending } pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll<(ID, Result)> { - match ready!(self.inner.poll_next_unpin(cx)) { + let res = self + .inner + .iter_mut() + .find_map(|(worker_id, (worker, timeout))| { + if timeout.poll_unpin(cx).is_ready() { + return Some((worker_id.clone(), Err(Timeout::new()))); + } + + match worker.poll_unpin(cx) { + Poll::Ready(output) => Some((worker_id.clone(), Ok(output))), + Poll::Pending => None, + } + }); + + match res { None => { self.empty_waker = Some(cx.waker().clone()); + Poll::Pending } - Some(result) => { - if let Some(waker) = self.full_waker.take() { - waker.wake(); - } + Some((worker_id, worker_res)) => { + self.inner.remove(&worker_id); - self.keys.remove(&result.0); - - Poll::Ready(result) + Poll::Ready((worker_id, worker_res)) } } } @@ -126,9 +133,9 @@ mod tests { let mut workers = UniqueWorkers::new(Duration::from_secs(10), 1); assert!(workers.try_push("ID_1", ready(())).is_ok()); - assert_eq!( + matches!( workers.try_push("ID_2", ready(())), - PushResult::BeyondCapacity + Err(PushError::BeyondCapacity(_)) ); } @@ -137,8 +144,10 @@ mod tests { let mut workers = UniqueWorkers::new(Duration::from_secs(10), 5); assert!(workers.try_push("ID", ready(())).is_ok()); - assert_eq!(workers.try_push("ID", ready(())), PushResult::ExistedID); - assert_eq!(workers.try_push("ID", ready(())), PushResult::ExistedID); + matches!( + workers.try_push("ID", ready(())), + Err(PushError::ReplacedWorker(_)) + ); } #[tokio::test] diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 6e1278cafe1..2869a3ee5d7 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -424,7 +424,7 @@ impl Handler { ) .boxed(), ) - .is_failing() + .is_err() { log::warn!("Dropping inbound stream because we are at capacity") } @@ -442,7 +442,7 @@ impl Handler { if self .protocol_futs .try_push(outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed()) - .is_failing() + .is_err() { log::warn!("Dropping outbound stream because we are at capacity") } diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 5ca0e62d594..964f59933ee 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -27,7 +27,7 @@ use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; use futures::sink::SinkExt; use futures::stream::{FuturesUnordered, StreamExt}; -use futures_bounded::{PushResult, Timeout}; +use futures_bounded::{PushError, Timeout}; use futures_timer::Delay; use instant::Instant; use libp2p_core::multiaddr::Protocol; @@ -276,15 +276,15 @@ impl Handler { src_peer_id, circuit.deny(proto::Status::NO_RESERVATION).boxed(), ) { - PushResult::Ok => {} - PushResult::BeyondCapacity => log::warn!( + Err(PushError::BeyondCapacity(_)) => log::warn!( "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", src_peer_id ), - PushResult::ExistedID => log::warn!( - "Dropping inbound circuit request to be denied from {:?} in favor of existing one.", + Err(PushError::ReplacedWorker(_)) => log::warn!( + "Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", src_peer_id ), + Ok(()) => {} } } } @@ -562,7 +562,7 @@ impl ConnectionHandler for Handler { if self .open_circuit_futs .try_push(inbound_stop::handle_open_circuit(stream).boxed()) - .is_failing() + .is_err() { log::warn!("Dropping inbound stream because we are at capacity") } @@ -582,7 +582,7 @@ impl ConnectionHandler for Handler { outbound_hop::handle_reserve_message_response(stream, to_listener) .boxed(), ) - .is_failing() + .is_err() { log::warn!("Dropping outbound stream because we are at capacity") } @@ -602,7 +602,7 @@ impl ConnectionHandler for Handler { ) .boxed(), ) - .is_failing() + .is_err() { log::warn!("Dropping outbound stream because we are at capacity") } From a02fd7ee395aa569247ee61636687aec8e80fa7b Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 6 Sep 2023 17:01:39 +0300 Subject: [PATCH 43/83] fix docs --- misc/futures-bounded/src/unique.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/unique.rs index 3d3df5ccd25..6ab439e581c 100644 --- a/misc/futures-bounded/src/unique.rs +++ b/misc/futures-bounded/src/unique.rs @@ -12,8 +12,7 @@ use crate::Timeout; /// Represents a set of (Worker)-[Future]s. /// -/// This wraps [FuturesUnordered] but bounds it by time and size. Every worker has to have a unique ID. -/// In other words, each worker must finish within the specified time and the set never outgrows its capacity. +/// Each worker must finish within the specified time and the set never outgrows its capacity. pub struct UniqueWorkers { timeout: Duration, capacity: usize, From d36af74b30190db16871736763939b951a4012ea Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 6 Sep 2023 17:07:56 +0300 Subject: [PATCH 44/83] fix clippy --- misc/futures-bounded/src/bounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index 762e74aea37..51ddead681b 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -109,7 +109,7 @@ mod tests { let mut workers = BoundedWorkers::new(Duration::from_secs(10), 1); assert!(workers.try_push(ready(())).is_ok()); - matches!(workers.try_push(ready(())), Err(_)); + assert!(workers.try_push(ready(())).is_err()); } #[tokio::test] From 05959c27d65a6515b65e24081ebf175576b48745 Mon Sep 17 00:00:00 2001 From: dgarus Date: Wed, 6 Sep 2023 17:25:21 +0300 Subject: [PATCH 45/83] rewrite unique workers --- misc/futures-bounded/src/unique.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/unique.rs index 6ab439e581c..ce104a3e718 100644 --- a/misc/futures-bounded/src/unique.rs +++ b/misc/futures-bounded/src/unique.rs @@ -24,7 +24,7 @@ pub struct UniqueWorkers { /// Error of a worker pushing #[derive(PartialEq, Debug)] pub enum PushError { - /// The length of the set is equal the capacity + /// The length of the set is equal to the capacity BeyondCapacity(F), /// The set already contains the given worker's ID ReplacedWorker(F), @@ -113,6 +113,10 @@ where Some((worker_id, worker_res)) => { self.inner.remove(&worker_id); + if let Some(waker) = self.full_waker.take() { + waker.wake(); + } + Poll::Ready((worker_id, worker_res)) } } From 11a36ee900a6d0dd5fa77848c2ff79e02b0ba4a1 Mon Sep 17 00:00:00 2001 From: dgarus Date: Thu, 7 Sep 2023 11:52:20 +0300 Subject: [PATCH 46/83] wrapping UniqueWorkers --- misc/futures-bounded/src/bounded.rs | 72 ++++++++--------------------- 1 file changed, 19 insertions(+), 53 deletions(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index 51ddead681b..d6226da571e 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -1,35 +1,25 @@ use std::future::Future; -use std::task::{Context, Poll, Waker}; +use std::task::{Context, Poll}; use std::time::Duration; -use futures_timer::Delay; -use futures_util::future::{select, BoxFuture, Either}; -use futures_util::stream::FuturesUnordered; -use futures_util::{ready, FutureExt, StreamExt}; +use futures_util::future::BoxFuture; -use crate::Timeout; +use crate::{PushError, Timeout, UniqueWorkers}; /// Represents a set of (Worker)-[Future]s. /// /// This wraps [FuturesUnordered] but bounds it by time and size. /// In other words, each worker must finish within the specified time and the set never outgrows its capacity. pub struct BoundedWorkers { - timeout: Duration, - capacity: usize, - inner: FuturesUnordered>>, - - empty_waker: Option, - full_waker: Option, + id: i32, + inner: UniqueWorkers, } impl BoundedWorkers { pub fn new(timeout: Duration, capacity: usize) -> Self { Self { - timeout, - capacity, - inner: Default::default(), - empty_waker: None, - full_waker: None, + id: i32::MIN, + inner: UniqueWorkers::new(timeout, capacity), } } } @@ -40,30 +30,19 @@ impl BoundedWorkers { /// If the length of the set is equal to the capacity, /// this method returns a error that contains the passed worker. /// In that case, the worker is not added to the set. - pub fn try_push(&mut self, worker: F) -> Result<(), F> + pub fn try_push(&mut self, worker: F) -> Result<(), BoxFuture> where F: Future + Send + 'static + Unpin, { - if self.inner.len() >= self.capacity { - return Err(worker); - } + (self.id, _) = self.id.overflowing_add(1); - let timeout = Delay::new(self.timeout); - self.inner.push( - async move { - match select(worker, timeout).await { - Either::Left((out, _)) => Ok(out), - Either::Right(((), _)) => Err(Timeout::new()), - } + match self.inner.try_push(self.id, worker) { + Ok(()) => Ok(()), + Err(PushError::BeyondCapacity(w)) => Err(w), + Err(PushError::ReplacedWorker(_)) => { + unreachable!() } - .boxed(), - ); - - if let Some(waker) = self.empty_waker.take() { - waker.wake(); } - - Ok(()) } pub fn is_empty(&self) -> bool { @@ -71,27 +50,13 @@ impl BoundedWorkers { } pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { - if self.inner.len() < self.capacity { - return Poll::Ready(()); - } - - self.full_waker = Some(cx.waker().clone()); - Poll::Pending + self.inner.poll_ready_unpin(cx) } pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll> { - match ready!(self.inner.poll_next_unpin(cx)) { - None => { - self.empty_waker = Some(cx.waker().clone()); - Poll::Pending - } - Some(result) => { - if let Some(waker) = self.full_waker.take() { - waker.wake(); - } - - Poll::Ready(result) - } + match self.inner.poll_unpin(cx) { + Poll::Ready((_, res)) => Poll::Ready(res), + Poll::Pending => Poll::Pending, } } } @@ -101,6 +66,7 @@ mod tests { use std::future::{pending, poll_fn, ready}; use std::pin::Pin; use std::time::Instant; + use futures_timer::Delay; use super::*; From eb1a38d2f00fdf6a4cf19d4433ce22a8f3144b3e Mon Sep 17 00:00:00 2001 From: dgarus Date: Thu, 7 Sep 2023 11:56:30 +0300 Subject: [PATCH 47/83] rustfmt --- misc/futures-bounded/src/bounded.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index d6226da571e..1ccb4f3f5c4 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -63,10 +63,10 @@ impl BoundedWorkers { #[cfg(test)] mod tests { + use futures_timer::Delay; use std::future::{pending, poll_fn, ready}; use std::pin::Pin; use std::time::Instant; - use futures_timer::Delay; use super::*; From 5efe055afefa6caf79de07776d5b4af5b7b59a16 Mon Sep 17 00:00:00 2001 From: dgarus Date: Thu, 7 Sep 2023 11:57:39 +0300 Subject: [PATCH 48/83] fix doc --- misc/futures-bounded/src/bounded.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index 1ccb4f3f5c4..18bdcf67047 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -8,8 +8,7 @@ use crate::{PushError, Timeout, UniqueWorkers}; /// Represents a set of (Worker)-[Future]s. /// -/// This wraps [FuturesUnordered] but bounds it by time and size. -/// In other words, each worker must finish within the specified time and the set never outgrows its capacity. +/// Each worker must finish within the specified time and the set never outgrows its capacity. pub struct BoundedWorkers { id: i32, inner: UniqueWorkers, From acb75afec6a3092215b6dd40f6feb29a1386c014 Mon Sep 17 00:00:00 2001 From: dgarus Date: Fri, 8 Sep 2023 14:34:18 +0300 Subject: [PATCH 49/83] fix review comments --- misc/futures-bounded/src/bounded.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index 18bdcf67047..3c48957e6ce 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -1,5 +1,5 @@ use std::future::Future; -use std::task::{Context, Poll}; +use std::task::{ready, Context, Poll}; use std::time::Duration; use futures_util::future::BoxFuture; @@ -53,10 +53,9 @@ impl BoundedWorkers { } pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll> { - match self.inner.poll_unpin(cx) { - Poll::Ready((_, res)) => Poll::Ready(res), - Poll::Pending => Poll::Pending, - } + let (_, res) = ready!(self.inner.poll_unpin(cx)); + + Poll::Ready(res) } } From f4dd44f9b9dd899631df461047481d300d06d2a1 Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Fri, 8 Sep 2023 18:29:57 +0300 Subject: [PATCH 50/83] Update misc/futures-bounded/src/unique.rs Co-authored-by: Thomas Eizinger --- misc/futures-bounded/src/unique.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/unique.rs index ce104a3e718..4103157c314 100644 --- a/misc/futures-bounded/src/unique.rs +++ b/misc/futures-bounded/src/unique.rs @@ -47,6 +47,7 @@ where ID: Clone + Hash + Eq + Send + 'static, { /// Push a worker into the set. + /// /// This method adds the given worker with defined `worker_id` to the set. /// If the length of the set is equal to the capacity, this method returns [PushError::BeyondCapacity], /// that contains the passed worker. In that case, the worker is not added to the set. From adde2e54d236be765ab7669118776087d0dc21af Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Fri, 8 Sep 2023 18:30:25 +0300 Subject: [PATCH 51/83] Update protocols/relay/src/priv_client/handler.rs Co-authored-by: Thomas Eizinger --- protocols/relay/src/priv_client/handler.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 964f59933ee..0ddfd47e3a2 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -367,8 +367,7 @@ impl ConnectionHandler for Handler { )) } Ok(None) => None, - Err(err) => { - let res = match err { + Err(outbound_hop::UpgradeError::CircuitFailed(e)) => { outbound_hop::UpgradeError::CircuitFailed(e) => { ConnectionHandlerEvent::NotifyBehaviour( Event::OutboundCircuitReqFailed { From 407443becb1df7707e025132a5892b4d87f59a32 Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Fri, 8 Sep 2023 18:30:44 +0300 Subject: [PATCH 52/83] Update protocols/relay/src/priv_client/handler.rs Co-authored-by: Thomas Eizinger --- protocols/relay/src/priv_client/handler.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 0ddfd47e3a2..9fc9c9e0b48 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -599,7 +599,6 @@ impl ConnectionHandler for Handler { cmd, tx, ) - .boxed(), ) .is_err() { From 8b5f30758c07ef46f600377f901a4079c09cba1c Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Fri, 8 Sep 2023 18:30:58 +0300 Subject: [PATCH 53/83] Update protocols/relay/src/priv_client/handler.rs Co-authored-by: Thomas Eizinger --- protocols/relay/src/priv_client/handler.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 9fc9c9e0b48..208ee270a70 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -579,7 +579,6 @@ impl ConnectionHandler for Handler { .reserve_futs .try_push( outbound_hop::handle_reserve_message_response(stream, to_listener) - .boxed(), ) .is_err() { From bd67ebf19c5ac0466aa62a4d4debd382601cbb8f Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Fri, 8 Sep 2023 18:31:10 +0300 Subject: [PATCH 54/83] Update protocols/relay/src/priv_client/handler.rs Co-authored-by: Thomas Eizinger --- protocols/relay/src/priv_client/handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 208ee270a70..f6cf4e99889 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -560,7 +560,7 @@ impl ConnectionHandler for Handler { }) => { if self .open_circuit_futs - .try_push(inbound_stop::handle_open_circuit(stream).boxed()) + .try_push(inbound_stop::handle_open_circuit(stream)) .is_err() { log::warn!("Dropping inbound stream because we are at capacity") From 6dfc8c6183067135d528f32843caa541098fcdb8 Mon Sep 17 00:00:00 2001 From: dgarus Date: Fri, 8 Sep 2023 18:43:52 +0300 Subject: [PATCH 55/83] fix review comments --- misc/futures-bounded/src/bounded.rs | 2 +- misc/futures-bounded/src/unique.rs | 2 +- protocols/relay/src/behaviour/handler.rs | 3 +-- protocols/relay/src/priv_client/handler.rs | 31 +++++++++++----------- 4 files changed, 18 insertions(+), 20 deletions(-) diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/bounded.rs index 3c48957e6ce..4ae1e35e14f 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/bounded.rs @@ -31,7 +31,7 @@ impl BoundedWorkers { /// In that case, the worker is not added to the set. pub fn try_push(&mut self, worker: F) -> Result<(), BoxFuture> where - F: Future + Send + 'static + Unpin, + F: Future + Send + 'static, { (self.id, _) = self.id.overflowing_add(1); diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/unique.rs index 4103157c314..cd269f1dcd9 100644 --- a/misc/futures-bounded/src/unique.rs +++ b/misc/futures-bounded/src/unique.rs @@ -55,7 +55,7 @@ where /// In that case, the returned error [PushError::ReplacedWorker] contains the old worker. pub fn try_push(&mut self, worker_id: ID, worker: F) -> Result<(), PushError>> where - F: Future + Send + 'static + Unpin, + F: Future + Send + 'static, { if self.inner.len() >= self.capacity { return Err(PushError::BeyondCapacity(worker.boxed())); diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 2869a3ee5d7..382f4ff3392 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -422,7 +422,6 @@ impl Handler { self.endpoint.clone(), self.active_reservation.is_some(), ) - .boxed(), ) .is_err() { @@ -441,7 +440,7 @@ impl Handler { if self .protocol_futs - .try_push(outbound_stop::handle_stop_message_response(stream, stop_command, tx).boxed()) + .try_push(outbound_stop::handle_stop_message_response(stream, stop_command, tx)) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index f6cf4e99889..f25d5238926 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -274,7 +274,7 @@ impl Handler { match self.circuit_deny_futs.try_push( src_peer_id, - circuit.deny(proto::Status::NO_RESERVATION).boxed(), + circuit.deny(proto::Status::NO_RESERVATION), ) { Err(PushError::BeyondCapacity(_)) => log::warn!( "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", @@ -368,22 +368,21 @@ impl ConnectionHandler for Handler { } Ok(None) => None, Err(outbound_hop::UpgradeError::CircuitFailed(e)) => { - outbound_hop::UpgradeError::CircuitFailed(e) => { - ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(e), - }, - ) - } - outbound_hop::UpgradeError::Fatal(e) => ConnectionHandlerEvent::Close( + Some(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }, + )) + } + Err(outbound_hop::UpgradeError::Fatal(e)) => { + Some( + ConnectionHandlerEvent::Close( StreamUpgradeError::Apply(Either::Right(e)), - ), - outbound_hop::UpgradeError::ReservationFailed(_) => { - unreachable!("do not emit `ReservationFailed` for connection") - } - }; - - Some(res) + ) + ) + } + Err(outbound_hop::UpgradeError::ReservationFailed(_)) => { + unreachable!("do not emit `ReservationFailed` for connection") } _ => unreachable!("do not emit 'Output::Reservation' for connection"), }; From 838595e97501e6cba9495a3184f275a81549acf5 Mon Sep 17 00:00:00 2001 From: dgarus Date: Fri, 8 Sep 2023 19:10:23 +0300 Subject: [PATCH 56/83] Renamed to FuturesList and FuturesMap --- misc/futures-bounded/src/lib.rs | 8 +- .../src/{bounded.rs => list.rs} | 78 ++++++------- .../futures-bounded/src/{unique.rs => map.rs} | 110 +++++++++--------- protocols/relay/src/behaviour/handler.rs | 28 ++--- protocols/relay/src/priv_client/handler.rs | 62 +++++----- 5 files changed, 140 insertions(+), 146 deletions(-) rename misc/futures-bounded/src/{bounded.rs => list.rs} (52%) rename misc/futures-bounded/src/{unique.rs => map.rs} (54%) diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs index 1c6154072bd..0483e98046c 100644 --- a/misc/futures-bounded/src/lib.rs +++ b/misc/futures-bounded/src/lib.rs @@ -1,8 +1,8 @@ -mod bounded; -mod unique; +mod list; +mod map; -pub use bounded::BoundedWorkers; -pub use unique::{PushError, UniqueWorkers}; +pub use list::FuturesList; +pub use map::{FuturesMap, PushError}; #[derive(Debug)] pub struct Timeout { diff --git a/misc/futures-bounded/src/bounded.rs b/misc/futures-bounded/src/list.rs similarity index 52% rename from misc/futures-bounded/src/bounded.rs rename to misc/futures-bounded/src/list.rs index 4ae1e35e14f..e8cc6e2b3bd 100644 --- a/misc/futures-bounded/src/bounded.rs +++ b/misc/futures-bounded/src/list.rs @@ -4,41 +4,41 @@ use std::time::Duration; use futures_util::future::BoxFuture; -use crate::{PushError, Timeout, UniqueWorkers}; +use crate::{FuturesMap, PushError, Timeout}; -/// Represents a set of (Worker)-[Future]s. +/// Represents a list of [Future]s. /// -/// Each worker must finish within the specified time and the set never outgrows its capacity. -pub struct BoundedWorkers { +/// Each future must finish within the specified time and the list never outgrows its capacity. +pub struct FuturesList { id: i32, - inner: UniqueWorkers, + inner: FuturesMap, } -impl BoundedWorkers { +impl FuturesList { pub fn new(timeout: Duration, capacity: usize) -> Self { Self { id: i32::MIN, - inner: UniqueWorkers::new(timeout, capacity), + inner: FuturesMap::new(timeout, capacity), } } } -impl BoundedWorkers { - /// Push a worker into the set. - /// This method adds the given worker to the set. - /// If the length of the set is equal to the capacity, - /// this method returns a error that contains the passed worker. - /// In that case, the worker is not added to the set. - pub fn try_push(&mut self, worker: F) -> Result<(), BoxFuture> +impl FuturesList { + /// Push a future into the list. + /// This method adds the given future to the list. + /// If the length of the list is equal to the capacity, + /// this method returns a error that contains the passed future. + /// In that case, the future is not added to the set. + pub fn try_push(&mut self, future: F) -> Result<(), BoxFuture> where F: Future + Send + 'static, { (self.id, _) = self.id.overflowing_add(1); - match self.inner.try_push(self.id, worker) { + match self.inner.try_push(self.id, future) { Ok(()) => Ok(()), Err(PushError::BeyondCapacity(w)) => Err(w), - Err(PushError::ReplacedWorker(_)) => { + Err(PushError::ReplacedFuture(_)) => { unreachable!() } } @@ -70,51 +70,51 @@ mod tests { #[test] fn cannot_push_more_than_capacity_tasks() { - let mut workers = BoundedWorkers::new(Duration::from_secs(10), 1); + let mut futures = FuturesList::new(Duration::from_secs(10), 1); - assert!(workers.try_push(ready(())).is_ok()); - assert!(workers.try_push(ready(())).is_err()); + assert!(futures.try_push(ready(())).is_ok()); + assert!(futures.try_push(ready(())).is_err()); } #[tokio::test] - async fn workers_timeout() { - let mut workers = BoundedWorkers::new(Duration::from_millis(100), 1); + async fn futures_timeout() { + let mut futures = FuturesList::new(Duration::from_millis(100), 1); - let _ = workers.try_push(pending::<()>()); + let _ = futures.try_push(pending::<()>()); Delay::new(Duration::from_millis(150)).await; - let result = poll_fn(|cx| workers.poll_unpin(cx)).await; + let result = poll_fn(|cx| futures.poll_unpin(cx)).await; assert!(result.is_err()) } - // Each worker causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. - // We stop after NUM_WORKERS tasks, meaning the overall execution must at least take DELAY * NUM_WORKERS. + // Each future causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. + // We stop after NUM_FUTURES tasks, meaning the overall execution must at least take DELAY * NUM_FUTURES. #[tokio::test] async fn backpressure() { const DELAY: Duration = Duration::from_millis(100); - const NUM_WORKERS: u32 = 10; + const NUM_FUTURES: u32 = 10; let start = Instant::now(); - Task::new(DELAY, NUM_WORKERS, 1).await; + Task::new(DELAY, NUM_FUTURES, 1).await; let duration = start.elapsed(); - assert!(duration >= DELAY * NUM_WORKERS); + assert!(duration >= DELAY * NUM_FUTURES); } struct Task { - worker: Duration, - num_workers: usize, + future: Duration, + num_futures: usize, num_processed: usize, - inner: BoundedWorkers<()>, + inner: FuturesList<()>, } impl Task { - fn new(worker: Duration, num_workers: u32, capacity: usize) -> Self { + fn new(future: Duration, num_futures: u32, capacity: usize) -> Self { Self { - worker, - num_workers: num_workers as usize, + future, + num_futures: num_futures as usize, num_processed: 0, - inner: BoundedWorkers::new(Duration::from_secs(60), capacity), + inner: FuturesList::new(Duration::from_secs(60), capacity), } } } @@ -125,10 +125,10 @@ mod tests { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); - while this.num_processed < this.num_workers { + while this.num_processed < this.num_futures { if let Poll::Ready(result) = this.inner.poll_unpin(cx) { if result.is_err() { - panic!("Timeout is great than worker delay") + panic!("Timeout is great than future delay") } this.num_processed += 1; @@ -136,8 +136,8 @@ mod tests { } if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { - let maybe_worker = this.inner.try_push(Delay::new(this.worker)); - assert!(maybe_worker.is_ok(), "we polled for readiness"); + let maybe_future = this.inner.try_push(Delay::new(this.future)); + assert!(maybe_future.is_ok(), "we polled for readiness"); continue; } diff --git a/misc/futures-bounded/src/unique.rs b/misc/futures-bounded/src/map.rs similarity index 54% rename from misc/futures-bounded/src/unique.rs rename to misc/futures-bounded/src/map.rs index cd269f1dcd9..65f2c7271b8 100644 --- a/misc/futures-bounded/src/unique.rs +++ b/misc/futures-bounded/src/map.rs @@ -10,10 +10,10 @@ use futures_util::FutureExt; use crate::Timeout; -/// Represents a set of (Worker)-[Future]s. +/// Represents a map of [Future]s. /// -/// Each worker must finish within the specified time and the set never outgrows its capacity. -pub struct UniqueWorkers { +/// Each future must finish within the specified time and the map never outgrows its capacity. +pub struct FuturesMap { timeout: Duration, capacity: usize, inner: HashMap, Delay)>, @@ -21,16 +21,16 @@ pub struct UniqueWorkers { full_waker: Option, } -/// Error of a worker pushing +/// Error of a future pushing #[derive(PartialEq, Debug)] pub enum PushError { /// The length of the set is equal to the capacity BeyondCapacity(F), - /// The set already contains the given worker's ID - ReplacedWorker(F), + /// The set already contains the given future's ID + ReplacedFuture(F), } -impl UniqueWorkers { +impl FuturesMap { pub fn new(timeout: Duration, capacity: usize) -> Self { Self { timeout, @@ -42,28 +42,28 @@ impl UniqueWorkers { } } -impl UniqueWorkers +impl FuturesMap where ID: Clone + Hash + Eq + Send + 'static, { - /// Push a worker into the set. + /// Push a future into the map. /// - /// This method adds the given worker with defined `worker_id` to the set. - /// If the length of the set is equal to the capacity, this method returns [PushError::BeyondCapacity], - /// that contains the passed worker. In that case, the worker is not added to the set. - /// If a worker with the given `worker_id` already exists, then the old worker will be replaced by a new one. - /// In that case, the returned error [PushError::ReplacedWorker] contains the old worker. - pub fn try_push(&mut self, worker_id: ID, worker: F) -> Result<(), PushError>> + /// This method inserts the given future with defined `future_id` to the set. + /// If the length of the map is equal to the capacity, this method returns [PushError::BeyondCapacity], + /// that contains the passed future. In that case, the future is not inserted to the map. + /// If a future with the given `future_id` already exists, then the old future will be replaced by a new one. + /// In that case, the returned error [PushError::ReplacedFuture] contains the old future. + pub fn try_push(&mut self, future_id: ID, future: F) -> Result<(), PushError>> where F: Future + Send + 'static, { if self.inner.len() >= self.capacity { - return Err(PushError::BeyondCapacity(worker.boxed())); + return Err(PushError::BeyondCapacity(future.boxed())); } let val = self.inner.insert( - worker_id.clone(), - (worker.boxed(), Delay::new(self.timeout)), + future_id.clone(), + (future.boxed(), Delay::new(self.timeout)), ); if let Some(waker) = self.empty_waker.take() { @@ -71,7 +71,7 @@ where } match val { - Some((old_worker, _)) => Err(PushError::ReplacedWorker(old_worker)), + Some((old_future, _)) => Err(PushError::ReplacedFuture(old_future)), None => Ok(()), } } @@ -94,13 +94,13 @@ where let res = self .inner .iter_mut() - .find_map(|(worker_id, (worker, timeout))| { + .find_map(|(future_id, (future, timeout))| { if timeout.poll_unpin(cx).is_ready() { - return Some((worker_id.clone(), Err(Timeout::new()))); + return Some((future_id.clone(), Err(Timeout::new()))); } - match worker.poll_unpin(cx) { - Poll::Ready(output) => Some((worker_id.clone(), Ok(output))), + match future.poll_unpin(cx) { + Poll::Ready(output) => Some((future_id.clone(), Ok(output))), Poll::Pending => None, } }); @@ -111,14 +111,14 @@ where Poll::Pending } - Some((worker_id, worker_res)) => { - self.inner.remove(&worker_id); + Some((future_id, future_res)) => { + self.inner.remove(&future_id); if let Some(waker) = self.full_waker.take() { waker.wake(); } - Poll::Ready((worker_id, worker_res)) + Poll::Ready((future_id, future_res)) } } } @@ -134,65 +134,65 @@ mod tests { #[test] fn cannot_push_more_than_capacity_tasks() { - let mut workers = UniqueWorkers::new(Duration::from_secs(10), 1); + let mut futures = FuturesMap::new(Duration::from_secs(10), 1); - assert!(workers.try_push("ID_1", ready(())).is_ok()); + assert!(futures.try_push("ID_1", ready(())).is_ok()); matches!( - workers.try_push("ID_2", ready(())), + futures.try_push("ID_2", ready(())), Err(PushError::BeyondCapacity(_)) ); } #[test] fn cannot_push_the_same_id_few_times() { - let mut workers = UniqueWorkers::new(Duration::from_secs(10), 5); + let mut futures = FuturesMap::new(Duration::from_secs(10), 5); - assert!(workers.try_push("ID", ready(())).is_ok()); + assert!(futures.try_push("ID", ready(())).is_ok()); matches!( - workers.try_push("ID", ready(())), - Err(PushError::ReplacedWorker(_)) + futures.try_push("ID", ready(())), + Err(PushError::ReplacedFuture(_)) ); } #[tokio::test] - async fn workers_timeout() { - let mut workers = UniqueWorkers::new(Duration::from_millis(100), 1); + async fn futures_timeout() { + let mut futures = FuturesMap::new(Duration::from_millis(100), 1); - let _ = workers.try_push("ID", pending::<()>()); + let _ = futures.try_push("ID", pending::<()>()); Delay::new(Duration::from_millis(150)).await; - let (_, result) = poll_fn(|cx| workers.poll_unpin(cx)).await; + let (_, result) = poll_fn(|cx| futures.poll_unpin(cx)).await; assert!(result.is_err()) } - // Each worker causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. - // We stop after NUM_WORKERS tasks, meaning the overall execution must at least take DELAY * NUM_WORKERS. + // Each future causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. + // We stop after NUM_FUTURES tasks, meaning the overall execution must at least take DELAY * NUM_FUTURES. #[tokio::test] async fn backpressure() { const DELAY: Duration = Duration::from_millis(100); - const NUM_WORKERS: u32 = 10; + const NUM_FUTURES: u32 = 10; let start = Instant::now(); - Task::new(DELAY, NUM_WORKERS, 1).await; + Task::new(DELAY, NUM_FUTURES, 1).await; let duration = start.elapsed(); - assert!(duration >= DELAY * NUM_WORKERS); + assert!(duration >= DELAY * NUM_FUTURES); } struct Task { - worker: Duration, - num_workers: usize, + future: Duration, + num_futures: usize, num_processed: usize, - inner: UniqueWorkers, + inner: FuturesMap, } impl Task { - fn new(worker: Duration, num_workers: u32, capacity: usize) -> Self { + fn new(future: Duration, num_futures: u32, capacity: usize) -> Self { Self { - worker, - num_workers: num_workers as usize, + future, + num_futures: num_futures as usize, num_processed: 0, - inner: UniqueWorkers::new(Duration::from_secs(60), capacity), + inner: FuturesMap::new(Duration::from_secs(60), capacity), } } } @@ -203,10 +203,10 @@ mod tests { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); - while this.num_processed < this.num_workers { + while this.num_processed < this.num_futures { if let Poll::Ready((_, result)) = this.inner.poll_unpin(cx) { if result.is_err() { - panic!("Timeout is great than worker delay") + panic!("Timeout is great than future delay") } this.num_processed += 1; @@ -214,10 +214,10 @@ mod tests { } if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { - // We push the constant worker's ID to prove that user can use the same ID - // if the worker's future was finished - let maybe_worker = this.inner.try_push(1u8, Delay::new(this.worker)); - assert!(maybe_worker.is_ok(), "we polled for readiness"); + // We push the constant future's ID to prove that user can use the same ID + // if the future was finished + let maybe_future = this.inner.try_push(1u8, Delay::new(this.future)); + assert!(maybe_future.is_ok(), "we polled for readiness"); continue; } diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 382f4ff3392..1e326ae1c68 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -385,13 +385,13 @@ pub struct Handler { circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, stop_requested_streams: VecDeque, - protocol_futs: futures_bounded::BoundedWorkers, + protocol_futs: futures_bounded::FuturesList, } impl Handler { pub fn new(config: Config, endpoint: ConnectedPoint) -> Handler { Handler { - protocol_futs: futures_bounded::BoundedWorkers::new( + protocol_futs: futures_bounded::FuturesList::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), @@ -413,16 +413,14 @@ impl Handler { fn on_fully_negotiated_inbound(&mut self, stream: Stream) { if self .protocol_futs - .try_push( - inbound_hop::handle_inbound_request( - stream, - self.config.reservation_duration, - self.config.max_circuit_duration, - self.config.max_circuit_bytes, - self.endpoint.clone(), - self.active_reservation.is_some(), - ) - ) + .try_push(inbound_hop::handle_inbound_request( + stream, + self.config.reservation_duration, + self.config.max_circuit_duration, + self.config.max_circuit_bytes, + self.endpoint.clone(), + self.active_reservation.is_some(), + )) .is_err() { log::warn!("Dropping inbound stream because we are at capacity") @@ -440,7 +438,11 @@ impl Handler { if self .protocol_futs - .try_push(outbound_stop::handle_stop_message_response(stream, stop_command, tx)) + .try_push(outbound_stop::handle_stop_message_response( + stream, + stop_command, + tx, + )) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index f25d5238926..484476508b1 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -137,8 +137,8 @@ pub struct Handler { wait_for_outbound_stream: VecDeque, reserve_futs: - futures_bounded::BoundedWorkers>, - circuit_connection_futs: futures_bounded::BoundedWorkers< + futures_bounded::FuturesList>, + circuit_connection_futs: futures_bounded::FuturesList< Result, outbound_hop::UpgradeError>, >, @@ -154,12 +154,11 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - open_circuit_futs: futures_bounded::BoundedWorkers< + open_circuit_futs: futures_bounded::FuturesList< Result, >, - circuit_deny_futs: - futures_bounded::UniqueWorkers>, + circuit_deny_futs: futures_bounded::FuturesMap>, /// Futures that try to send errors to the transport. /// @@ -177,21 +176,21 @@ impl Handler { queued_events: Default::default(), pending_error: Default::default(), wait_for_outbound_stream: Default::default(), - reserve_futs: futures_bounded::BoundedWorkers::new( + reserve_futs: futures_bounded::FuturesList::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), - circuit_connection_futs: futures_bounded::BoundedWorkers::new( + circuit_connection_futs: futures_bounded::FuturesList::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), - open_circuit_futs: futures_bounded::BoundedWorkers::new( + open_circuit_futs: futures_bounded::FuturesList::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), - circuit_deny_futs: futures_bounded::UniqueWorkers::new( + circuit_deny_futs: futures_bounded::FuturesMap::new( DENYING_CIRCUIT_TIMEOUT, MAX_NUMBER_DENYING_CIRCUIT, ), @@ -280,7 +279,7 @@ impl Handler { "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", src_peer_id ), - Err(PushError::ReplacedWorker(_)) => log::warn!( + Err(PushError::ReplacedFuture(_)) => log::warn!( "Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", src_peer_id ), @@ -367,20 +366,14 @@ impl ConnectionHandler for Handler { )) } Ok(None) => None, - Err(outbound_hop::UpgradeError::CircuitFailed(e)) => { - Some(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(e), - }, - )) - } - Err(outbound_hop::UpgradeError::Fatal(e)) => { - Some( - ConnectionHandlerEvent::Close( - StreamUpgradeError::Apply(Either::Right(e)), - ) - ) - } + Err(outbound_hop::UpgradeError::CircuitFailed(e)) => Some( + ConnectionHandlerEvent::NotifyBehaviour(Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }), + ), + Err(outbound_hop::UpgradeError::Fatal(e)) => Some(ConnectionHandlerEvent::Close( + StreamUpgradeError::Apply(Either::Right(e)), + )), Err(outbound_hop::UpgradeError::ReservationFailed(_)) => { unreachable!("do not emit `ReservationFailed` for connection") } @@ -576,9 +569,10 @@ impl ConnectionHandler for Handler { OutboundStreamInfo::Reserve(to_listener) => { if self .reserve_futs - .try_push( - outbound_hop::handle_reserve_message_response(stream, to_listener) - ) + .try_push(outbound_hop::handle_reserve_message_response( + stream, + to_listener, + )) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") @@ -590,14 +584,12 @@ impl ConnectionHandler for Handler { if self .circuit_connection_futs - .try_push( - outbound_hop::handle_connection_message_response( - stream, - self.remote_peer_id, - cmd, - tx, - ) - ) + .try_push(outbound_hop::handle_connection_message_response( + stream, + self.remote_peer_id, + cmd, + tx, + )) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") From cc0ebe2a5a56d891fadd5019ad244c90e48af650 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sat, 9 Sep 2023 13:03:28 +0300 Subject: [PATCH 57/83] fix review comments --- misc/futures-bounded/src/map.rs | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/misc/futures-bounded/src/map.rs b/misc/futures-bounded/src/map.rs index 65f2c7271b8..ed5d168d366 100644 --- a/misc/futures-bounded/src/map.rs +++ b/misc/futures-bounded/src/map.rs @@ -91,6 +91,12 @@ where } pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll<(ID, Result)> { + if self.inner.is_empty() { + self.empty_waker = Some(cx.waker().clone()); + + return Poll::Pending; + } + let res = self .inner .iter_mut() @@ -106,11 +112,7 @@ where }); match res { - None => { - self.empty_waker = Some(cx.waker().clone()); - - Poll::Pending - } + None => Poll::Pending, Some((future_id, future_res)) => { self.inner.remove(&future_id); From 35d8fd2315e55c3f0645e2cf6cf84887943b1fe8 Mon Sep 17 00:00:00 2001 From: dgarus Date: Sun, 10 Sep 2023 13:49:21 +0300 Subject: [PATCH 58/83] refactoring of outbound_hop::handle_reserve_message_response --- protocols/relay/src/priv_client.rs | 2 +- protocols/relay/src/priv_client/handler.rs | 50 +++------- protocols/relay/src/protocol/outbound_hop.rs | 96 ++++++++++++++------ 3 files changed, 81 insertions(+), 67 deletions(-) diff --git a/protocols/relay/src/priv_client.rs b/protocols/relay/src/priv_client.rs index abc2fe13177..d4f0c07cae3 100644 --- a/protocols/relay/src/priv_client.rs +++ b/protocols/relay/src/priv_client.rs @@ -20,7 +20,7 @@ //! [`NetworkBehaviour`] to act as a circuit relay v2 **client**. -mod handler; +pub(crate) mod handler; pub(crate) mod transport; use crate::multiaddr_ext::MultiaddrExt; diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 484476508b1..c34aec6b652 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -112,6 +112,13 @@ pub enum Event { }, } +pub(crate) type ClientConnectionHandlerEvent = ConnectionHandlerEvent< + ::OutboundProtocol, + ::OutboundOpenInfo, + ::ToBehaviour, + ::Error, +>; + pub struct Handler { local_peer_id: PeerId, remote_peer_id: PeerId, @@ -126,21 +133,12 @@ pub struct Handler { keep_alive: KeepAlive, /// Queue of events to return when polled. - queued_events: VecDeque< - ConnectionHandlerEvent< - ::OutboundProtocol, - ::OutboundOpenInfo, - ::ToBehaviour, - ::Error, - >, - >, + queued_events: VecDeque, wait_for_outbound_stream: VecDeque, reserve_futs: futures_bounded::FuturesList>, - circuit_connection_futs: futures_bounded::FuturesList< - Result, outbound_hop::UpgradeError>, - >, + circuit_connection_futs: futures_bounded::FuturesList>, reservation: Reservation, @@ -352,37 +350,13 @@ impl ConnectionHandler for Handler { // Circuit connections if let Poll::Ready(worker_res) = self.circuit_connection_futs.poll_unpin(cx) { - let res = match worker_res { - Ok(r) => r, + match worker_res { + Ok(None) => {} + Ok(Some(event)) => return Poll::Ready(event), Err(Timeout { .. }) => { return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); } }; - - let opt = match res { - Ok(Some(outbound_hop::Output::Circuit { limit })) => { - Some(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitEstablished { limit }, - )) - } - Ok(None) => None, - Err(outbound_hop::UpgradeError::CircuitFailed(e)) => Some( - ConnectionHandlerEvent::NotifyBehaviour(Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(e), - }), - ), - Err(outbound_hop::UpgradeError::Fatal(e)) => Some(ConnectionHandlerEvent::Close( - StreamUpgradeError::Apply(Either::Right(e)), - )), - Err(outbound_hop::UpgradeError::ReservationFailed(_)) => { - unreachable!("do not emit `ReservationFailed` for connection") - } - _ => unreachable!("do not emit 'Output::Reservation' for connection"), - }; - - if let Some(event) = opt { - return Poll::Ready(event); - } } // Reservations diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 32aacfbfd1f..f7ed2930acf 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -19,6 +19,7 @@ // DEALINGS IN THE SOFTWARE. use asynchronous_codec::{Framed, FramedParts}; +use either::Either; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; use futures_timer::Delay; @@ -29,9 +30,10 @@ use void::Void; use libp2p_core::Multiaddr; use libp2p_identity::PeerId; -use libp2p_swarm::Stream; +use libp2p_swarm::{ConnectionHandlerEvent, Stream, StreamUpgradeError}; -use crate::priv_client::transport; +use crate::priv_client::handler::ClientConnectionHandlerEvent; +use crate::priv_client::{handler, transport}; use crate::protocol::{Limit, MAX_MESSAGE_SIZE}; use crate::{priv_client, proto}; @@ -204,7 +206,7 @@ pub(crate) async fn handle_connection_message_response( remote_peer_id: PeerId, con_command: Command, tx: oneshot::Sender, -) -> Result, UpgradeError> { +) -> Option { let msg = proto::HopMessage { type_pb: proto::HopMessageType::CONNECT, peer: Some(proto::Peer { @@ -218,47 +220,85 @@ pub(crate) async fn handle_connection_message_response( let mut substream = Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); - substream.send(msg).await?; + if substream.send(msg).await.is_err() { + return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(FatalUpgradeError::StreamClosed), + ))); + } + let proto::HopMessage { type_pb, peer: _, reservation: _, limit, status, - } = substream - .next() - .await - .ok_or(FatalUpgradeError::StreamClosed)??; + } = match substream.next().await { + Some(Ok(r)) => r, + _ => { + return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(FatalUpgradeError::StreamClosed), + ))) + } + }; match type_pb { proto::HopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect.into()); + return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(FatalUpgradeError::UnexpectedTypeConnect), + ))); } proto::HopMessageType::RESERVE => { - return Err(FatalUpgradeError::UnexpectedTypeReserve.into()); + return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(FatalUpgradeError::UnexpectedTypeReserve), + ))); } proto::HopMessageType::STATUS => {} } - let limit = limit.map(Into::into); - - match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { - proto::Status::OK => {} - proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(CircuitFailedReason::ResourceLimitExceeded.into()); + match status { + Some(proto::Status::OK) => {} + Some(proto::Status::RESOURCE_LIMIT_EXCEEDED) => { + return Some(ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), + }, + )); } - proto::Status::CONNECTION_FAILED => { - return Err(CircuitFailedReason::ConnectionFailed.into()); + Some(proto::Status::CONNECTION_FAILED) => { + return Some(ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(CircuitFailedReason::ConnectionFailed), + }, + )); } - proto::Status::NO_RESERVATION => { - return Err(CircuitFailedReason::NoReservation.into()); + Some(proto::Status::NO_RESERVATION) => { + return Some(ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(CircuitFailedReason::NoReservation), + }, + )); } - proto::Status::PERMISSION_DENIED => { - return Err(CircuitFailedReason::PermissionDenied.into()); + Some(proto::Status::PERMISSION_DENIED) => { + return Some(ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), + }, + )); + } + Some(s) => { + return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(FatalUpgradeError::UnexpectedStatus(s)), + ))); + } + None => { + return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(FatalUpgradeError::MissingStatusField), + ))); } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), } + let limit = limit.map(Into::into); + let FramedParts { io, read_buffer, @@ -270,10 +310,12 @@ pub(crate) async fn handle_connection_message_response( "Expect a flushed Framed to have empty write buffer." ); - let output = match con_command.send_back.send(Ok(priv_client::Connection { + match con_command.send_back.send(Ok(priv_client::Connection { state: priv_client::ConnectionState::new_outbound(io, read_buffer.freeze(), tx), })) { - Ok(()) => Some(Output::Circuit { limit }), + Ok(()) => Some(ConnectionHandlerEvent::NotifyBehaviour( + handler::Event::OutboundCircuitEstablished { limit }, + )), Err(_) => { debug!( "Oneshot to `client::transport::Dial` future dropped. \ @@ -283,9 +325,7 @@ pub(crate) async fn handle_connection_message_response( None } - }; - - Ok(output) + } } pub(crate) enum OutboundStreamInfo { From 6be712317a8297da94b04a146f856578625e36fc Mon Sep 17 00:00:00 2001 From: dgarus Date: Sun, 10 Sep 2023 16:02:42 +0300 Subject: [PATCH 59/83] refactoring of outbound_hop::handle_reserve_message_response --- protocols/relay/src/priv_client/handler.rs | 5 ++--- protocols/relay/src/protocol/outbound_hop.rs | 23 +++++++------------- 2 files changed, 10 insertions(+), 18 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index c34aec6b652..9942bb2cfb3 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -137,7 +137,7 @@ pub struct Handler { wait_for_outbound_stream: VecDeque, reserve_futs: - futures_bounded::FuturesList>, + futures_bounded::FuturesList>, circuit_connection_futs: futures_bounded::FuturesList>, reservation: Reservation, @@ -369,7 +369,7 @@ impl ConnectionHandler for Handler { }; let event = match res { - Ok(outbound_hop::Output::Reservation { + Ok(outbound_hop::Reservation { renewal_timeout, addrs, limit, @@ -398,7 +398,6 @@ impl ConnectionHandler for Handler { unreachable!("do not emit `CircuitFailed` for reservation") } }, - _ => unreachable!("do not emit 'Output::Circuit' for reservation"), }; return Poll::Ready(event); diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index f7ed2930acf..ead262d04a7 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -101,22 +101,17 @@ pub enum FatalUpgradeError { UnexpectedStatus(proto::Status), } -pub(crate) enum Output { - Reservation { - renewal_timeout: Delay, - addrs: Vec, - limit: Option, - to_listener: mpsc::Sender, - }, - Circuit { - limit: Option, - }, +pub(crate) struct Reservation { + pub(crate) renewal_timeout: Delay, + pub(crate) addrs: Vec, + pub(crate) limit: Option, + pub(crate) to_listener: mpsc::Sender, } pub(crate) async fn handle_reserve_message_response( protocol: Stream, to_listener: mpsc::Sender, -) -> Result { +) -> Result { let msg = proto::HopMessage { type_pb: proto::HopMessageType::RESERVE, peer: None, @@ -191,14 +186,12 @@ pub(crate) async fn handle_reserve_message_response( substream.close().await?; - let output = Output::Reservation { + Ok(Reservation { renewal_timeout, addrs, limit, to_listener, - }; - - Ok(output) + }) } pub(crate) async fn handle_connection_message_response( From 86a0ac75fb444e9e0dc74b2e11b82bb7326bd51c Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:05:43 +1000 Subject: [PATCH 60/83] Refactor `FuturesMap` to use `FuturesUnbounded` for efficiency --- misc/futures-bounded/src/map.rs | 107 +++++++++++++++++++++----------- 1 file changed, 70 insertions(+), 37 deletions(-) diff --git a/misc/futures-bounded/src/map.rs b/misc/futures-bounded/src/map.rs index ed5d168d366..a701f71b7d6 100644 --- a/misc/futures-bounded/src/map.rs +++ b/misc/futures-bounded/src/map.rs @@ -1,12 +1,14 @@ -use std::collections::HashMap; use std::future::Future; use std::hash::Hash; +use std::mem; +use std::pin::Pin; use std::task::{Context, Poll, Waker}; use std::time::Duration; use futures_timer::Delay; use futures_util::future::BoxFuture; -use futures_util::FutureExt; +use futures_util::stream::FuturesUnordered; +use futures_util::{FutureExt, StreamExt}; use crate::Timeout; @@ -16,7 +18,7 @@ use crate::Timeout; pub struct FuturesMap { timeout: Duration, capacity: usize, - inner: HashMap, Delay)>, + inner: FuturesUnordered>>>, empty_waker: Option, full_waker: Option, } @@ -44,7 +46,7 @@ impl FuturesMap { impl FuturesMap where - ID: Clone + Hash + Eq + Send + 'static, + ID: Clone + Hash + Eq + Send + Unpin + 'static, { /// Push a future into the map. /// @@ -61,18 +63,33 @@ where return Err(PushError::BeyondCapacity(future.boxed())); } - let val = self.inner.insert( - future_id.clone(), - (future.boxed(), Delay::new(self.timeout)), - ); - if let Some(waker) = self.empty_waker.take() { waker.wake(); } - match val { - Some((old_future, _)) => Err(PushError::ReplacedFuture(old_future)), - None => Ok(()), + match self.inner.iter_mut().find(|tagged| tagged.tag == future_id) { + None => { + self.inner.push(TaggedFuture { + tag: future_id, + inner: TimeoutFuture { + inner: future.boxed(), + timeout: Delay::new(self.timeout), + }, + }); + + Ok(()) + } + Some(existing) => { + let old_future = mem::replace( + &mut existing.inner, + TimeoutFuture { + inner: future.boxed(), + timeout: Delay::new(self.timeout), + }, + ); + + Err(PushError::ReplacedFuture(old_future.inner)) + } } } @@ -91,38 +108,54 @@ where } pub fn poll_unpin(&mut self, cx: &mut Context<'_>) -> Poll<(ID, Result)> { - if self.inner.is_empty() { - self.empty_waker = Some(cx.waker().clone()); + let maybe_result = futures_util::ready!(self.inner.poll_next_unpin(cx)); - return Poll::Pending; + match maybe_result { + None => { + self.empty_waker = Some(cx.waker().clone()); + Poll::Pending + } + Some(result) => Poll::Ready(result), } + } +} - let res = self - .inner - .iter_mut() - .find_map(|(future_id, (future, timeout))| { - if timeout.poll_unpin(cx).is_ready() { - return Some((future_id.clone(), Err(Timeout::new()))); - } +struct TimeoutFuture { + inner: F, + timeout: Delay, +} - match future.poll_unpin(cx) { - Poll::Ready(output) => Some((future_id.clone(), Ok(output))), - Poll::Pending => None, - } - }); +impl Future for TimeoutFuture +where + F: Future + Unpin, +{ + type Output = Result; - match res { - None => Poll::Pending, - Some((future_id, future_res)) => { - self.inner.remove(&future_id); + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + if self.timeout.poll_unpin(cx).is_ready() { + return Poll::Ready(Err(Timeout::new())); + } - if let Some(waker) = self.full_waker.take() { - waker.wake(); - } + self.inner.poll_unpin(cx).map(Ok) + } +} - Poll::Ready((future_id, future_res)) - } - } +struct TaggedFuture { + tag: T, + inner: F, +} + +impl Future for TaggedFuture +where + T: Clone + Unpin, + F: Future + Unpin, +{ + type Output = (T, F::Output); + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let output = futures_util::ready!(self.inner.poll_unpin(cx)); + + Poll::Ready((self.tag.clone(), output)) } } From 7713aca6ac1617858168eb076bfcc2c1ff23d9a6 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:07:53 +1000 Subject: [PATCH 61/83] Remove tests that are covered by other impl Now that we are implementing `FuturesList` by wrapping `FuturesMap`, we no longer need these tests for `FuturesList`. --- misc/futures-bounded/src/list.rs | 91 -------------------------------- 1 file changed, 91 deletions(-) diff --git a/misc/futures-bounded/src/list.rs b/misc/futures-bounded/src/list.rs index e8cc6e2b3bd..2856ea81ab8 100644 --- a/misc/futures-bounded/src/list.rs +++ b/misc/futures-bounded/src/list.rs @@ -58,94 +58,3 @@ impl FuturesList { Poll::Ready(res) } } - -#[cfg(test)] -mod tests { - use futures_timer::Delay; - use std::future::{pending, poll_fn, ready}; - use std::pin::Pin; - use std::time::Instant; - - use super::*; - - #[test] - fn cannot_push_more_than_capacity_tasks() { - let mut futures = FuturesList::new(Duration::from_secs(10), 1); - - assert!(futures.try_push(ready(())).is_ok()); - assert!(futures.try_push(ready(())).is_err()); - } - - #[tokio::test] - async fn futures_timeout() { - let mut futures = FuturesList::new(Duration::from_millis(100), 1); - - let _ = futures.try_push(pending::<()>()); - Delay::new(Duration::from_millis(150)).await; - let result = poll_fn(|cx| futures.poll_unpin(cx)).await; - - assert!(result.is_err()) - } - - // Each future causes a delay, `Task` only has a capacity of 1, meaning they must be processed in sequence. - // We stop after NUM_FUTURES tasks, meaning the overall execution must at least take DELAY * NUM_FUTURES. - #[tokio::test] - async fn backpressure() { - const DELAY: Duration = Duration::from_millis(100); - const NUM_FUTURES: u32 = 10; - - let start = Instant::now(); - Task::new(DELAY, NUM_FUTURES, 1).await; - let duration = start.elapsed(); - - assert!(duration >= DELAY * NUM_FUTURES); - } - - struct Task { - future: Duration, - num_futures: usize, - num_processed: usize, - inner: FuturesList<()>, - } - - impl Task { - fn new(future: Duration, num_futures: u32, capacity: usize) -> Self { - Self { - future, - num_futures: num_futures as usize, - num_processed: 0, - inner: FuturesList::new(Duration::from_secs(60), capacity), - } - } - } - - impl Future for Task { - type Output = (); - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.get_mut(); - - while this.num_processed < this.num_futures { - if let Poll::Ready(result) = this.inner.poll_unpin(cx) { - if result.is_err() { - panic!("Timeout is great than future delay") - } - - this.num_processed += 1; - continue; - } - - if let Poll::Ready(()) = this.inner.poll_ready_unpin(cx) { - let maybe_future = this.inner.try_push(Delay::new(this.future)); - assert!(maybe_future.is_ok(), "we polled for readiness"); - - continue; - } - - return Poll::Pending; - } - - Poll::Ready(()) - } - } -} From 036164c81637f09bf02bba1e57b22f97905e6365 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:09:04 +1000 Subject: [PATCH 62/83] Reformat docs --- misc/futures-bounded/src/list.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/misc/futures-bounded/src/list.rs b/misc/futures-bounded/src/list.rs index 2856ea81ab8..269bf427b06 100644 --- a/misc/futures-bounded/src/list.rs +++ b/misc/futures-bounded/src/list.rs @@ -25,9 +25,9 @@ impl FuturesList { impl FuturesList { /// Push a future into the list. + /// /// This method adds the given future to the list. - /// If the length of the list is equal to the capacity, - /// this method returns a error that contains the passed future. + /// If the length of the list is equal to the capacity, this method returns a error that contains the passed future. /// In that case, the future is not added to the set. pub fn try_push(&mut self, future: F) -> Result<(), BoxFuture> where From ff3099186a51eefda66bf71e324b542d5e8d422d Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:14:12 +1000 Subject: [PATCH 63/83] Improve panic message --- misc/futures-bounded/src/list.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/misc/futures-bounded/src/list.rs b/misc/futures-bounded/src/list.rs index 269bf427b06..79d7bdc656a 100644 --- a/misc/futures-bounded/src/list.rs +++ b/misc/futures-bounded/src/list.rs @@ -38,9 +38,7 @@ impl FuturesList { match self.inner.try_push(self.id, future) { Ok(()) => Ok(()), Err(PushError::BeyondCapacity(w)) => Err(w), - Err(PushError::ReplacedFuture(_)) => { - unreachable!() - } + Err(PushError::ReplacedFuture(_)) => unreachable!("we never reuse IDs"), } } From 5cb43e00769709da73718ba34ad5254a9830e9ce Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:14:27 +1000 Subject: [PATCH 64/83] Make `Timeout` a proper error --- misc/futures-bounded/src/lib.rs | 18 +++++++++++++++--- misc/futures-bounded/src/map.rs | 7 ++++--- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs index 0483e98046c..355d534e233 100644 --- a/misc/futures-bounded/src/lib.rs +++ b/misc/futures-bounded/src/lib.rs @@ -3,14 +3,26 @@ mod map; pub use list::FuturesList; pub use map::{FuturesMap, PushError}; +use std::fmt; +use std::fmt::Formatter; +use std::time::Duration; +/// A future failed to complete within the given timeout. #[derive(Debug)] pub struct Timeout { - _priv: (), + limit: Duration, } impl Timeout { - fn new() -> Self { - Self { _priv: () } + fn new(duration: Duration) -> Self { + Self { limit: duration } } } + +impl fmt::Display for Timeout { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "future failed to complete within {:?}", self.limit) + } +} + +impl std::error::Error for Timeout {} diff --git a/misc/futures-bounded/src/map.rs b/misc/futures-bounded/src/map.rs index a701f71b7d6..765cdc227f2 100644 --- a/misc/futures-bounded/src/map.rs +++ b/misc/futures-bounded/src/map.rs @@ -115,7 +115,8 @@ where self.empty_waker = Some(cx.waker().clone()); Poll::Pending } - Some(result) => Poll::Ready(result), + Some((id, Ok(output))) => Poll::Ready((id, Ok(output))), + Some((id, Err(_timeout))) => Poll::Ready((id, Err(Timeout::new(self.timeout)))), } } } @@ -129,11 +130,11 @@ impl Future for TimeoutFuture where F: Future + Unpin, { - type Output = Result; + type Output = Result; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { if self.timeout.poll_unpin(cx).is_ready() { - return Poll::Ready(Err(Timeout::new())); + return Poll::Ready(Err(())); } self.inner.poll_unpin(cx).map(Ok) From 0a4aa09149a7144d70cd25142a3cb285ad446d39 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:51:17 +1000 Subject: [PATCH 65/83] Handle outbound circuits using 1 `FuturesList` --- protocols/relay/src/priv_client/handler.rs | 151 ++++++++++--------- protocols/relay/src/protocol/outbound_hop.rs | 117 +++++--------- 2 files changed, 121 insertions(+), 147 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 9942bb2cfb3..62a127dc5c1 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -19,7 +19,6 @@ // DEALINGS IN THE SOFTWARE. use crate::priv_client::transport; -use crate::protocol::outbound_hop::OutboundStreamInfo; use crate::protocol::{self, inbound_stop, outbound_hop}; use crate::{proto, HOP_PROTOCOL_NAME, STOP_PROTOCOL_NAME}; use either::Either; @@ -27,6 +26,7 @@ use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, FutureExt}; use futures::sink::SinkExt; use futures::stream::{FuturesUnordered, StreamExt}; +use futures::TryFutureExt; use futures_bounded::{PushError, Timeout}; use futures_timer::Delay; use instant::Instant; @@ -136,9 +136,15 @@ pub struct Handler { queued_events: VecDeque, wait_for_outbound_stream: VecDeque, - reserve_futs: - futures_bounded::FuturesList>, - circuit_connection_futs: futures_bounded::FuturesList>, + outbound_circuits: futures_bounded::FuturesList< + Result< + Either< + Result, + Result, outbound_hop::CircuitFailedReason>, + >, + outbound_hop::FatalUpgradeError, + >, + >, reservation: Reservation, @@ -174,11 +180,7 @@ impl Handler { queued_events: Default::default(), pending_error: Default::default(), wait_for_outbound_stream: Default::default(), - reserve_futs: futures_bounded::FuturesList::new( - STREAM_TIMEOUT, - MAX_CONCURRENT_STREAMS_PER_CONNECTION, - ), - circuit_connection_futs: futures_bounded::FuturesList::new( + outbound_circuits: futures_bounded::FuturesList::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), @@ -208,7 +210,7 @@ impl Handler { "got a stream error without a pending connection command or a reserve listener", ); match outbound_info { - OutboundStreamInfo::Reserve(mut to_listener) => { + outbound_hop::OutboundStreamInfo::Reserve(mut to_listener) => { let non_fatal_error = match error { StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, @@ -243,7 +245,7 @@ impl Handler { }, )); } - OutboundStreamInfo::CircuitConnection(cmd) => { + outbound_hop::OutboundStreamInfo::CircuitConnection(cmd) => { let non_fatal_error = match error { StreamUpgradeError::Timeout => StreamUpgradeError::Timeout, StreamUpgradeError::NegotiationFailed => StreamUpgradeError::NegotiationFailed, @@ -305,7 +307,7 @@ impl ConnectionHandler for Handler { match event { In::Reserve { to_listener } => { self.wait_for_outbound_stream - .push_back(OutboundStreamInfo::Reserve(to_listener)); + .push_back(outbound_hop::OutboundStreamInfo::Reserve(to_listener)); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -315,10 +317,11 @@ impl ConnectionHandler for Handler { send_back, dst_peer_id, } => { - self.wait_for_outbound_stream - .push_back(OutboundStreamInfo::CircuitConnection( + self.wait_for_outbound_stream.push_back( + outbound_hop::OutboundStreamInfo::CircuitConnection( outbound_hop::Command::new(dst_peer_id, send_back), - )); + ), + ); self.queued_events .push_back(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -349,40 +352,39 @@ impl ConnectionHandler for Handler { } // Circuit connections - if let Poll::Ready(worker_res) = self.circuit_connection_futs.poll_unpin(cx) { - match worker_res { - Ok(None) => {} - Ok(Some(event)) => return Poll::Ready(event), - Err(Timeout { .. }) => { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } - }; - } - - // Reservations - if let Poll::Ready(worker_res) = self.reserve_futs.poll_unpin(cx) { - let res = match worker_res { - Ok(r) => r, - Err(Timeout { .. }) => { - return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - } - }; - - let event = match res { - Ok(outbound_hop::Reservation { - renewal_timeout, - addrs, - limit, - to_listener, - }) => ConnectionHandlerEvent::NotifyBehaviour(self.reservation.accepted( - renewal_timeout, - addrs, - to_listener, - self.local_peer_id, - limit, - )), - Err(err) => match err { - outbound_hop::UpgradeError::ReservationFailed(e) => { + loop { + if let Poll::Ready(worker_res) = self.outbound_circuits.poll_unpin(cx) { + match worker_res { + Ok(Ok(Either::Left(Ok(outbound_hop::Reservation { + renewal_timeout, + addrs, + limit, + to_listener, + })))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + self.reservation.accepted( + renewal_timeout, + addrs, + to_listener, + self.local_peer_id, + limit, + ), + )) + } + Ok(Ok(Either::Right(Ok(Some(outbound_hop::Circuit { limit }))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitEstablished { limit }, + )); + } + Ok(Ok(Either::Right(Ok(None)))) => continue, + Ok(Ok(Either::Right(Err(e)))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }, + )); + } + Ok(Ok(Either::Left(Err(e)))) => { let renewal = self.reservation.failed(); return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( Event::ReservationReqFailed { @@ -391,16 +393,20 @@ impl ConnectionHandler for Handler { }, )); } - outbound_hop::UpgradeError::Fatal(e) => { - ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right(e))) + Ok(Err(e)) => { + return Poll::Ready(ConnectionHandlerEvent::Close( + StreamUpgradeError::Apply(Either::Right(e)), + )) } - outbound_hop::UpgradeError::CircuitFailed(_) => { - unreachable!("do not emit `CircuitFailed` for reservation") + Err(Timeout { .. }) => { + return Poll::Ready(ConnectionHandlerEvent::Close( + StreamUpgradeError::Timeout, + )); } - }, - }; + }; + } - return Poll::Ready(event); + break; } // Return queued events. @@ -453,7 +459,7 @@ impl ConnectionHandler for Handler { if let Poll::Ready(Some(to_listener)) = self.reservation.poll(cx) { self.wait_for_outbound_stream - .push_back(OutboundStreamInfo::Reserve(to_listener)); + .push_back(outbound_hop::OutboundStreamInfo::Reserve(to_listener)); return Poll::Ready(ConnectionHandlerEvent::OutboundSubstreamRequest { protocol: SubstreamProtocol::new(ReadyUpgrade::new(HOP_PROTOCOL_NAME), ()), @@ -539,30 +545,33 @@ impl ConnectionHandler for Handler { "opened a stream without a pending connection command or a reserve listener", ); match outbound_info { - OutboundStreamInfo::Reserve(to_listener) => { + outbound_hop::OutboundStreamInfo::Reserve(to_listener) => { if self - .reserve_futs - .try_push(outbound_hop::handle_reserve_message_response( - stream, - to_listener, - )) + .outbound_circuits + .try_push( + outbound_hop::handle_reserve_message_response(stream, to_listener) + .map_ok(Either::Left), + ) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") } } - OutboundStreamInfo::CircuitConnection(cmd) => { + outbound_hop::OutboundStreamInfo::CircuitConnection(cmd) => { let (tx, rx) = oneshot::channel(); self.alive_lend_out_substreams.push(rx); if self - .circuit_connection_futs - .try_push(outbound_hop::handle_connection_message_response( - stream, - self.remote_peer_id, - cmd, - tx, - )) + .outbound_circuits + .try_push( + outbound_hop::handle_connection_message_response( + stream, + self.remote_peer_id, + cmd, + tx, + ) + .map_ok(Either::Right), + ) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index ead262d04a7..041b3b9775d 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -18,40 +18,33 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use std::time::{Duration, SystemTime}; + use asynchronous_codec::{Framed, FramedParts}; -use either::Either; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; use futures_timer::Delay; use log::debug; -use std::time::{Duration, SystemTime}; use thiserror::Error; use void::Void; use libp2p_core::Multiaddr; use libp2p_identity::PeerId; -use libp2p_swarm::{ConnectionHandlerEvent, Stream, StreamUpgradeError}; +use libp2p_swarm::Stream; -use crate::priv_client::handler::ClientConnectionHandlerEvent; -use crate::priv_client::{handler, transport}; +use crate::priv_client::transport; use crate::protocol::{Limit, MAX_MESSAGE_SIZE}; use crate::{priv_client, proto}; - -#[derive(Debug, Error)] -pub(crate) enum UpgradeError { - #[error("Reservation failed")] - ReservationFailed(#[from] ReservationFailedReason), - #[error("Circuit failed")] - CircuitFailed(#[from] CircuitFailedReason), - #[error("Fatal")] - Fatal(#[from] FatalUpgradeError), -} - -impl From for UpgradeError { - fn from(error: quick_protobuf_codec::Error) -> Self { - Self::Fatal(error.into()) - } -} +// +// #[derive(Debug, Error)] +// pub(crate) enum UpgradeError { +// #[error("Reservation failed")] +// ReservationFailed(#[from] ReservationFailedReason), +// #[error("Circuit failed")] +// CircuitFailed(#[from] CircuitFailedReason), +// #[error("Fatal")] +// Fatal(#[from] FatalUpgradeError), +// } #[derive(Debug, Error)] pub enum CircuitFailedReason { @@ -108,10 +101,14 @@ pub(crate) struct Reservation { pub(crate) to_listener: mpsc::Sender, } +pub(crate) struct Circuit { + pub(crate) limit: Option, +} + pub(crate) async fn handle_reserve_message_response( protocol: Stream, to_listener: mpsc::Sender, -) -> Result { +) -> Result, FatalUpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::RESERVE, peer: None, @@ -136,31 +133,31 @@ pub(crate) async fn handle_reserve_message_response( match type_pb { proto::HopMessageType::CONNECT => { - return Err(FatalUpgradeError::UnexpectedTypeConnect.into()); + return Err(FatalUpgradeError::UnexpectedTypeConnect); } proto::HopMessageType::RESERVE => { - return Err(FatalUpgradeError::UnexpectedTypeReserve.into()); + return Err(FatalUpgradeError::UnexpectedTypeReserve); } proto::HopMessageType::STATUS => {} } let limit = limit.map(Into::into); - match status.ok_or(UpgradeError::Fatal(FatalUpgradeError::MissingStatusField))? { + match status.ok_or(FatalUpgradeError::MissingStatusField)? { proto::Status::OK => {} proto::Status::RESERVATION_REFUSED => { - return Err(ReservationFailedReason::Refused.into()); + return Ok(Err(ReservationFailedReason::Refused)); } proto::Status::RESOURCE_LIMIT_EXCEEDED => { - return Err(ReservationFailedReason::ResourceLimitExceeded.into()); + return Ok(Err(ReservationFailedReason::ResourceLimitExceeded)); } - s => return Err(FatalUpgradeError::UnexpectedStatus(s).into()), + s => return Err(FatalUpgradeError::UnexpectedStatus(s)), } let reservation = reservation.ok_or(FatalUpgradeError::MissingReservationField)?; if reservation.addrs.is_empty() { - return Err(FatalUpgradeError::NoAddressesInReservation.into()); + return Err(FatalUpgradeError::NoAddressesInReservation); } let addrs = reservation @@ -186,12 +183,12 @@ pub(crate) async fn handle_reserve_message_response( substream.close().await?; - Ok(Reservation { + Ok(Ok(Reservation { renewal_timeout, addrs, limit, to_listener, - }) + })) } pub(crate) async fn handle_connection_message_response( @@ -199,7 +196,7 @@ pub(crate) async fn handle_connection_message_response( remote_peer_id: PeerId, con_command: Command, tx: oneshot::Sender, -) -> Option { +) -> Result, CircuitFailedReason>, FatalUpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::CONNECT, peer: Some(proto::Peer { @@ -214,9 +211,7 @@ pub(crate) async fn handle_connection_message_response( let mut substream = Framed::new(protocol, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); if substream.send(msg).await.is_err() { - return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(FatalUpgradeError::StreamClosed), - ))); + return Err(FatalUpgradeError::StreamClosed); } let proto::HopMessage { @@ -227,23 +222,15 @@ pub(crate) async fn handle_connection_message_response( status, } = match substream.next().await { Some(Ok(r)) => r, - _ => { - return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(FatalUpgradeError::StreamClosed), - ))) - } + _ => return Err(FatalUpgradeError::StreamClosed), }; match type_pb { proto::HopMessageType::CONNECT => { - return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(FatalUpgradeError::UnexpectedTypeConnect), - ))); + return Err(FatalUpgradeError::UnexpectedTypeConnect); } proto::HopMessageType::RESERVE => { - return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(FatalUpgradeError::UnexpectedTypeReserve), - ))); + return Err(FatalUpgradeError::UnexpectedTypeReserve); } proto::HopMessageType::STATUS => {} } @@ -251,42 +238,22 @@ pub(crate) async fn handle_connection_message_response( match status { Some(proto::Status::OK) => {} Some(proto::Status::RESOURCE_LIMIT_EXCEEDED) => { - return Some(ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), - }, - )); + return Ok(Err(CircuitFailedReason::ResourceLimitExceeded)); } Some(proto::Status::CONNECTION_FAILED) => { - return Some(ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(CircuitFailedReason::ConnectionFailed), - }, - )); + return Ok(Err(CircuitFailedReason::ConnectionFailed)); } Some(proto::Status::NO_RESERVATION) => { - return Some(ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(CircuitFailedReason::NoReservation), - }, - )); + return Ok(Err(CircuitFailedReason::NoReservation)); } Some(proto::Status::PERMISSION_DENIED) => { - return Some(ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), - }, - )); + return Ok(Err(CircuitFailedReason::PermissionDenied)); } Some(s) => { - return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(FatalUpgradeError::UnexpectedStatus(s)), - ))); + return Err(FatalUpgradeError::UnexpectedStatus(s)); } None => { - return Some(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( - Either::Right(FatalUpgradeError::MissingStatusField), - ))); + return Err(FatalUpgradeError::MissingStatusField); } } @@ -306,9 +273,7 @@ pub(crate) async fn handle_connection_message_response( match con_command.send_back.send(Ok(priv_client::Connection { state: priv_client::ConnectionState::new_outbound(io, read_buffer.freeze(), tx), })) { - Ok(()) => Some(ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundCircuitEstablished { limit }, - )), + Ok(()) => Ok(Ok(Some(Circuit { limit }))), Err(_) => { debug!( "Oneshot to `client::transport::Dial` future dropped. \ @@ -316,7 +281,7 @@ pub(crate) async fn handle_connection_message_response( remote_peer_id, ); - None + Ok(Ok(None)) } } } From 3c4bd9969785a98fd1ead786f6429f1a2afa138c Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 10:53:25 +1000 Subject: [PATCH 66/83] Don't nest `match` --- protocols/relay/src/priv_client/handler.rs | 103 ++++++++++----------- 1 file changed, 49 insertions(+), 54 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 62a127dc5c1..5ee9cb52ca1 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -351,62 +351,57 @@ impl ConnectionHandler for Handler { return Poll::Ready(ConnectionHandlerEvent::Close(err)); } - // Circuit connections + // Inbound circuits loop { - if let Poll::Ready(worker_res) = self.outbound_circuits.poll_unpin(cx) { - match worker_res { - Ok(Ok(Either::Left(Ok(outbound_hop::Reservation { - renewal_timeout, - addrs, - limit, - to_listener, - })))) => { - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( - self.reservation.accepted( - renewal_timeout, - addrs, - to_listener, - self.local_peer_id, - limit, - ), - )) - } - Ok(Ok(Either::Right(Ok(Some(outbound_hop::Circuit { limit }))))) => { - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitEstablished { limit }, - )); - } - Ok(Ok(Either::Right(Ok(None)))) => continue, - Ok(Ok(Either::Right(Err(e)))) => { - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( - Event::OutboundCircuitReqFailed { - error: StreamUpgradeError::Apply(e), - }, - )); - } - Ok(Ok(Either::Left(Err(e)))) => { - let renewal = self.reservation.failed(); - return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( - Event::ReservationReqFailed { - renewal, - error: StreamUpgradeError::Apply(e), - }, - )); - } - Ok(Err(e)) => { - return Poll::Ready(ConnectionHandlerEvent::Close( - StreamUpgradeError::Apply(Either::Right(e)), - )) - } - Err(Timeout { .. }) => { - return Poll::Ready(ConnectionHandlerEvent::Close( - StreamUpgradeError::Timeout, - )); - } - }; + match self.outbound_circuits.poll_unpin(cx) { + Poll::Ready(Ok(Ok(Either::Left(Ok(outbound_hop::Reservation { + renewal_timeout, + addrs, + limit, + to_listener, + }))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + self.reservation.accepted( + renewal_timeout, + addrs, + to_listener, + self.local_peer_id, + limit, + ), + )) + } + Poll::Ready(Ok(Ok(Either::Right(Ok(Some(outbound_hop::Circuit { limit })))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitEstablished { limit }, + )); + } + Poll::Ready(Ok(Ok(Either::Right(Ok(None))))) => continue, + Poll::Ready(Ok(Ok(Either::Right(Err(e))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundCircuitReqFailed { + error: StreamUpgradeError::Apply(e), + }, + )); + } + Poll::Ready(Ok(Ok(Either::Left(Err(e))))) => { + let renewal = self.reservation.failed(); + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::ReservationReqFailed { + renewal, + error: StreamUpgradeError::Apply(e), + }, + )); + } + Poll::Ready(Ok(Err(e))) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(e), + ))) + } + Poll::Ready(Err(Timeout { .. })) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + Poll::Pending => break, } - - break; } // Return queued events. From d73f3e960816ba2663c67de4bf42b105d5ee71cc Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:09:01 +1000 Subject: [PATCH 67/83] Don't publish `futures-bounded` for now --- misc/futures-bounded/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/misc/futures-bounded/Cargo.toml b/misc/futures-bounded/Cargo.toml index 8c4af13be82..b273226239c 100644 --- a/misc/futures-bounded/Cargo.toml +++ b/misc/futures-bounded/Cargo.toml @@ -8,6 +8,7 @@ repository = "https://github.com/libp2p/rust-libp2p" keywords = ["futures", "async", "backpressure"] categories = ["data-structures", "asynchronous"] description = "Utilities for bounding futures in size and time." +publish = false # TEMP FIX until https://github.com/obi1kenobi/cargo-semver-checks-action/issues/53 is fixed. # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html From ced1ae3b09d893cb3ccc987b7ef43824bc4872ad Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:10:22 +1000 Subject: [PATCH 68/83] Inline type-alias that is only used once --- protocols/relay/src/priv_client/handler.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 5ee9cb52ca1..63b7fffb547 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -112,13 +112,6 @@ pub enum Event { }, } -pub(crate) type ClientConnectionHandlerEvent = ConnectionHandlerEvent< - ::OutboundProtocol, - ::OutboundOpenInfo, - ::ToBehaviour, - ::Error, ->; - pub struct Handler { local_peer_id: PeerId, remote_peer_id: PeerId, @@ -133,7 +126,14 @@ pub struct Handler { keep_alive: KeepAlive, /// Queue of events to return when polled. - queued_events: VecDeque, + queued_events: VecDeque< + ConnectionHandlerEvent< + ::OutboundProtocol, + ::OutboundOpenInfo, + ::ToBehaviour, + ::Error, + >, + >, wait_for_outbound_stream: VecDeque, outbound_circuits: futures_bounded::FuturesList< From 68feb389587cc77fbcaf5872203be0e84dd34511 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:11:18 +1000 Subject: [PATCH 69/83] Use `void::unreachable` --- protocols/relay/src/priv_client/handler.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 63b7fffb547..8fe6ae7fc62 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -218,7 +218,7 @@ impl Handler { self.pending_error = Some(StreamUpgradeError::Io(e)); return; } - StreamUpgradeError::Apply(_) => unreachable!("should not update"), + StreamUpgradeError::Apply(v) => void::unreachable(v), }; if self.pending_error.is_none() { @@ -253,7 +253,7 @@ impl Handler { self.pending_error = Some(StreamUpgradeError::Io(e)); return; } - StreamUpgradeError::Apply(_) => unreachable!("should not update"), + StreamUpgradeError::Apply(v) => void::unreachable(v), }; let _ = cmd.send_back.send(Err(())); From c98f2e1f2f091011ecd49d6a999718771e79b510 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:11:43 +1000 Subject: [PATCH 70/83] Log PeerIds using Display --- protocols/relay/src/priv_client/handler.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 8fe6ae7fc62..e727a4875aa 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -276,12 +276,10 @@ impl Handler { circuit.deny(proto::Status::NO_RESERVATION), ) { Err(PushError::BeyondCapacity(_)) => log::warn!( - "Dropping inbound circuit request to be denied from {:?} due to exceeding limit.", - src_peer_id + "Dropping inbound circuit request to be denied from {src_peer_id} due to exceeding limit." ), Err(PushError::ReplacedFuture(_)) => log::warn!( - "Dropping existing inbound circuit request to be denied from {:?} in favor of new one.", - src_peer_id + "Dropping existing inbound circuit request to be denied from {src_peer_id} in favor of new one." ), Ok(()) => {} } From c37c8ee49c18467ebca3b3f2f60f003572a4fe75 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:13:18 +1000 Subject: [PATCH 71/83] Use `void::unreachable` --- protocols/relay/src/priv_client/handler.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index e727a4875aa..45915273b06 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -36,6 +36,7 @@ use libp2p_core::Multiaddr; use libp2p_identity::PeerId; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, + ListenUpgradeError, }; use libp2p_swarm::{ ConnectionHandler, ConnectionHandlerEvent, KeepAlive, StreamProtocol, StreamUpgradeError, @@ -575,8 +576,10 @@ impl ConnectionHandler for Handler { ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) } + ConnectionEvent::ListenUpgradeError(ListenUpgradeError { error, .. }) => { + void::unreachable(error) + } ConnectionEvent::AddressChange(_) - | ConnectionEvent::ListenUpgradeError(_) | ConnectionEvent::LocalProtocolsChange(_) | ConnectionEvent::RemoteProtocolsChange(_) => {} } From 21f4660be39992190cbf7ea311cbee9632e2a1eb Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:13:52 +1000 Subject: [PATCH 72/83] Reduce diff --- protocols/relay/src/priv_client/handler.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 45915273b06..3a65f775851 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -573,12 +573,12 @@ impl ConnectionHandler for Handler { } } } + ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => { + void::unreachable(listen_upgrade_error.error) + } ConnectionEvent::DialUpgradeError(dial_upgrade_error) => { self.on_dial_upgrade_error(dial_upgrade_error) } - ConnectionEvent::ListenUpgradeError(ListenUpgradeError { error, .. }) => { - void::unreachable(error) - } ConnectionEvent::AddressChange(_) | ConnectionEvent::LocalProtocolsChange(_) | ConnectionEvent::RemoteProtocolsChange(_) => {} From 902aed8d0e61717b17cdc65a750e09a03e3bb457 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:14:25 +1000 Subject: [PATCH 73/83] Remove commented out code --- protocols/relay/src/protocol/outbound_hop.rs | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/protocols/relay/src/protocol/outbound_hop.rs b/protocols/relay/src/protocol/outbound_hop.rs index 041b3b9775d..adad0e23711 100644 --- a/protocols/relay/src/protocol/outbound_hop.rs +++ b/protocols/relay/src/protocol/outbound_hop.rs @@ -35,16 +35,6 @@ use libp2p_swarm::Stream; use crate::priv_client::transport; use crate::protocol::{Limit, MAX_MESSAGE_SIZE}; use crate::{priv_client, proto}; -// -// #[derive(Debug, Error)] -// pub(crate) enum UpgradeError { -// #[error("Reservation failed")] -// ReservationFailed(#[from] ReservationFailedReason), -// #[error("Circuit failed")] -// CircuitFailed(#[from] CircuitFailedReason), -// #[error("Fatal")] -// Fatal(#[from] FatalUpgradeError), -// } #[derive(Debug, Error)] pub enum CircuitFailedReason { From d30dcc22815c5422a9412a2e4ce68c656037b836 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:19:04 +1000 Subject: [PATCH 74/83] Use `void::unreachable` --- protocols/relay/src/behaviour/handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 1e326ae1c68..b2f6369793f 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -471,7 +471,7 @@ impl Handler { self.pending_error = Some(StreamUpgradeError::Io(e)); return; } - StreamUpgradeError::Apply(_) => unreachable!("Should not emit handle errors"), + StreamUpgradeError::Apply(v) => void::unreachable(v), }; let stop_command = self From ba8cb444909e75b8fbd61fb2124ef40d4b95b2a5 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:24:18 +1000 Subject: [PATCH 75/83] Rename function to resemble message that is being sent --- protocols/relay/src/behaviour/handler.rs | 6 +----- protocols/relay/src/protocol/outbound_stop.rs | 3 ++- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index b2f6369793f..a920c6a92d2 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -438,11 +438,7 @@ impl Handler { if self .protocol_futs - .try_push(outbound_stop::handle_stop_message_response( - stream, - stop_command, - tx, - )) + .try_push(outbound_stop::connect(stream, stop_command, tx)) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 6f36801eee6..3635557bb91 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -74,7 +74,8 @@ pub enum FatalUpgradeError { UnexpectedStatus(proto::Status), } -pub(crate) async fn handle_stop_message_response( +/// Attempts to _connect_ to a peer via the given stream. +pub(crate) async fn connect( io: Stream, stop_command: StopCommand, tx: oneshot::Sender<()>, From 28832554c0610b4ecce2f292dc7d1c9b2bfef576 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:25:25 +1000 Subject: [PATCH 76/83] Rename `StopCommand` to `PendingConnect` --- protocols/relay/src/behaviour/handler.rs | 12 ++++++------ protocols/relay/src/protocol/outbound_stop.rs | 6 +++--- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index a920c6a92d2..fd04589df04 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -384,7 +384,7 @@ pub struct Handler { /// Futures relaying data for circuit between two peers. circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, - stop_requested_streams: VecDeque, + pending_connect_requests: VecDeque, protocol_futs: futures_bounded::FuturesList, } @@ -406,7 +406,7 @@ impl Handler { circuits: Default::default(), active_reservation: Default::default(), keep_alive: KeepAlive::Yes, - stop_requested_streams: Default::default(), + pending_connect_requests: Default::default(), } } @@ -429,7 +429,7 @@ impl Handler { fn on_fully_negotiated_outbound(&mut self, stream: Stream) { let stop_command = self - .stop_requested_streams + .pending_connect_requests .pop_front() .expect("opened a stream without a pending stop command"); @@ -471,7 +471,7 @@ impl Handler { }; let stop_command = self - .stop_requested_streams + .pending_connect_requests .pop_front() .expect("failed to open a stream without a pending stop command"); @@ -554,8 +554,8 @@ impl ConnectionHandler for Handler { src_peer_id, src_connection_id, } => { - self.stop_requested_streams - .push_back(outbound_stop::StopCommand::new( + self.pending_connect_requests + .push_back(outbound_stop::PendingConnect::new( circuit_id, inbound_circuit_req, src_peer_id, diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index 3635557bb91..a6b35267aea 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -77,7 +77,7 @@ pub enum FatalUpgradeError { /// Attempts to _connect_ to a peer via the given stream. pub(crate) async fn connect( io: Stream, - stop_command: StopCommand, + stop_command: PendingConnect, tx: oneshot::Sender<()>, ) -> handler::RelayConnectionHandlerEvent { let msg = proto::StopMessage { @@ -191,7 +191,7 @@ pub(crate) async fn connect( }) } -pub(crate) struct StopCommand { +pub(crate) struct PendingConnect { pub(crate) circuit_id: CircuitId, pub(crate) inbound_circuit_req: inbound_hop::CircuitReq, pub(crate) src_peer_id: PeerId, @@ -200,7 +200,7 @@ pub(crate) struct StopCommand { max_circuit_bytes: u64, } -impl StopCommand { +impl PendingConnect { pub(crate) fn new( circuit_id: CircuitId, inbound_circuit_req: inbound_hop::CircuitReq, From e2c9df36fbe84494e2a2625e987e981fd8d888ac Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 11 Sep 2023 11:49:00 +1000 Subject: [PATCH 77/83] Remove circular dependencies --- protocols/relay/src/behaviour/handler.rs | 114 +++++++++++++----- protocols/relay/src/priv_client/handler.rs | 1 - protocols/relay/src/protocol/inbound_hop.rs | 72 ++++------- protocols/relay/src/protocol/outbound_stop.rs | 92 +++++++------- 4 files changed, 153 insertions(+), 126 deletions(-) diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index fd04589df04..91b49589874 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -385,13 +385,25 @@ pub struct Handler { circuits: Futures<(CircuitId, PeerId, Result<(), std::io::Error>)>, pending_connect_requests: VecDeque, - protocol_futs: futures_bounded::FuturesList, + + workers: futures_bounded::FuturesList< + Either< + Result< + Either, + inbound_hop::FatalUpgradeError, + >, + Result< + Result, + outbound_stop::FatalUpgradeError, + >, + >, + >, } impl Handler { pub fn new(config: Config, endpoint: ConnectedPoint) -> Handler { Handler { - protocol_futs: futures_bounded::FuturesList::new( + workers: futures_bounded::FuturesList::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), @@ -412,15 +424,16 @@ impl Handler { fn on_fully_negotiated_inbound(&mut self, stream: Stream) { if self - .protocol_futs - .try_push(inbound_hop::handle_inbound_request( - stream, - self.config.reservation_duration, - self.config.max_circuit_duration, - self.config.max_circuit_bytes, - self.endpoint.clone(), - self.active_reservation.is_some(), - )) + .workers + .try_push( + inbound_hop::handle_inbound_request( + stream, + self.config.reservation_duration, + self.config.max_circuit_duration, + self.config.max_circuit_bytes, + ) + .map(Either::Left), + ) .is_err() { log::warn!("Dropping inbound stream because we are at capacity") @@ -437,8 +450,8 @@ impl Handler { self.alive_lend_out_substreams.push(rx); if self - .protocol_futs - .try_push(outbound_stop::connect(stream, stop_command, tx)) + .workers + .try_push(outbound_stop::connect(stream, stop_command, tx).map(Either::Right)) .is_err() { log::warn!("Dropping outbound stream because we are at capacity") @@ -496,13 +509,6 @@ enum ReservationRequestFuture { type Futures = FuturesUnordered>; -pub(crate) type RelayConnectionHandlerEvent = libp2p_swarm::ConnectionHandlerEvent< - ::OutboundProtocol, - ::OutboundOpenInfo, - ::ToBehaviour, - ::Error, ->; - impl ConnectionHandler for Handler { type FromBehaviour = In; type ToBehaviour = Event; @@ -527,7 +533,7 @@ impl ConnectionHandler for Handler { if self .reservation_request_future .replace(ReservationRequestFuture::Accepting( - inbound_reservation_req.accept(addrs).boxed(), + inbound_reservation_req.accept(addrs).err_into().boxed(), )) .is_some() { @@ -541,7 +547,7 @@ impl ConnectionHandler for Handler { if self .reservation_request_future .replace(ReservationRequestFuture::Denying( - inbound_reservation_req.deny(status).boxed(), + inbound_reservation_req.deny(status).err_into().boxed(), )) .is_some() { @@ -576,6 +582,7 @@ impl ConnectionHandler for Handler { self.circuit_deny_futures.push( inbound_circuit_req .deny(status) + .err_into() .map(move |result| (circuit_id, dst_peer_id, result)) .boxed(), ); @@ -591,6 +598,7 @@ impl ConnectionHandler for Handler { self.circuit_accept_futures.push( inbound_circuit_req .accept() + .err_into() .map_ok(move |(src_stream, src_pending_data)| CircuitParts { circuit_id, src_stream, @@ -660,11 +668,63 @@ impl ConnectionHandler for Handler { } // Process protocol requests - if let Poll::Ready(worker_res) = self.protocol_futs.poll_unpin(cx) { - let event = worker_res - .unwrap_or_else(|_| ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); - - return Poll::Ready(event); + match self.workers.poll_unpin(cx) { + Poll::Ready(Ok(Either::Left(Ok(Either::Left(inbound_reservation_req))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::ReservationReqReceived { + inbound_reservation_req, + endpoint: self.endpoint.clone(), + renewed: self.active_reservation.is_some(), + }, + )); + } + Poll::Ready(Ok(Either::Left(Ok(Either::Right(inbound_circuit_req))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::CircuitReqReceived { + inbound_circuit_req, + endpoint: self.endpoint.clone(), + }, + )); + } + Poll::Ready(Ok(Either::Right(Ok(Ok(circuit))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundConnectNegotiated { + circuit_id: circuit.circuit_id, + src_peer_id: circuit.src_peer_id, + src_connection_id: circuit.src_connection_id, + inbound_circuit_req: circuit.inbound_circuit_req, + dst_handler_notifier: circuit.dst_handler_notifier, + dst_stream: circuit.dst_stream, + dst_pending_data: circuit.dst_pending_data, + }, + )); + } + Poll::Ready(Ok(Either::Right(Ok(Err(circuit_failed))))) => { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + Event::OutboundConnectNegotiationFailed { + circuit_id: circuit_failed.circuit_id, + src_peer_id: circuit_failed.src_peer_id, + src_connection_id: circuit_failed.src_connection_id, + inbound_circuit_req: circuit_failed.inbound_circuit_req, + status: circuit_failed.status, + error: circuit_failed.error, + }, + )); + } + Poll::Ready(Err(futures_bounded::Timeout { .. })) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Timeout)); + } + Poll::Ready(Ok(Either::Left(Err(e)))) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Left(e), + ))); + } + Poll::Ready(Ok(Either::Right(Err(e)))) => { + return Poll::Ready(ConnectionHandlerEvent::Close(StreamUpgradeError::Apply( + Either::Right(e), + ))); + } + Poll::Pending => {} } // Deny new circuits. diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 3a65f775851..28427761c2a 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -36,7 +36,6 @@ use libp2p_core::Multiaddr; use libp2p_identity::PeerId; use libp2p_swarm::handler::{ ConnectionEvent, DialUpgradeError, FullyNegotiatedInbound, FullyNegotiatedOutbound, - ListenUpgradeError, }; use libp2p_swarm::{ ConnectionHandler, ConnectionHandlerEvent, KeepAlive, StreamProtocol, StreamUpgradeError, diff --git a/protocols/relay/src/protocol/inbound_hop.rs b/protocols/relay/src/protocol/inbound_hop.rs index a5f2d0b5db9..b44d29e42ce 100644 --- a/protocols/relay/src/protocol/inbound_hop.rs +++ b/protocols/relay/src/protocol/inbound_hop.rs @@ -26,11 +26,10 @@ use either::Either; use futures::prelude::*; use thiserror::Error; -use libp2p_core::{ConnectedPoint, Multiaddr}; +use libp2p_core::Multiaddr; use libp2p_identity::PeerId; -use libp2p_swarm::{ConnectionHandlerEvent, Stream, StreamUpgradeError}; +use libp2p_swarm::Stream; -use crate::behaviour::handler; use crate::proto; use crate::proto::message_v2::pb::mod_HopMessage::Type; use crate::protocol::MAX_MESSAGE_SIZE; @@ -71,7 +70,7 @@ pub struct ReservationReq { } impl ReservationReq { - pub async fn accept(self, addrs: Vec) -> Result<(), UpgradeError> { + pub async fn accept(self, addrs: Vec) -> Result<(), FatalUpgradeError> { if addrs.is_empty() { log::debug!( "Accepting relay reservation without providing external addresses of local node. \ @@ -105,7 +104,7 @@ impl ReservationReq { self.send(msg).await } - pub async fn deny(self, status: proto::Status) -> Result<(), UpgradeError> { + pub async fn deny(self, status: proto::Status) -> Result<(), FatalUpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::STATUS, peer: None, @@ -117,7 +116,7 @@ impl ReservationReq { self.send(msg).await } - async fn send(mut self, msg: proto::HopMessage) -> Result<(), UpgradeError> { + async fn send(mut self, msg: proto::HopMessage) -> Result<(), FatalUpgradeError> { self.substream.send(msg).await?; self.substream.flush().await?; self.substream.close().await?; @@ -136,7 +135,7 @@ impl CircuitReq { self.dst } - pub async fn accept(mut self) -> Result<(Stream, Bytes), UpgradeError> { + pub async fn accept(mut self) -> Result<(Stream, Bytes), FatalUpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::STATUS, peer: None, @@ -161,7 +160,7 @@ impl CircuitReq { Ok((io, read_buffer.freeze())) } - pub async fn deny(mut self, status: proto::Status) -> Result<(), UpgradeError> { + pub async fn deny(mut self, status: proto::Status) -> Result<(), FatalUpgradeError> { let msg = proto::HopMessage { type_pb: proto::HopMessageType::STATUS, peer: None, @@ -186,17 +185,13 @@ pub(crate) async fn handle_inbound_request( reservation_duration: Duration, max_circuit_duration: Duration, max_circuit_bytes: u64, - endpoint: ConnectedPoint, - renewed: bool, -) -> handler::RelayConnectionHandlerEvent { +) -> Result, FatalUpgradeError> { let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); let res = substream.next().await; if let None | Some(Err(_)) = res { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( - FatalUpgradeError::StreamClosed, - ))); + return Err(FatalUpgradeError::StreamClosed); } let proto::HopMessage { @@ -207,53 +202,28 @@ pub(crate) async fn handle_inbound_request( status: _, } = res.unwrap().expect("should be ok"); - let event = match type_pb { - Type::RESERVE => { - let req = ReservationReq { - substream, - reservation_duration, - max_circuit_duration, - max_circuit_bytes, - }; - - handler::Event::ReservationReqReceived { - inbound_reservation_req: req, - endpoint, - renewed, - } - } + let req = match type_pb { + Type::RESERVE => Either::Left(ReservationReq { + substream, + reservation_duration, + max_circuit_duration, + max_circuit_bytes, + }), Type::CONNECT => { let peer_id_res = match peer { Some(r) => PeerId::from_bytes(&r.id), - None => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( - FatalUpgradeError::MissingPeer, - ))) - } + None => return Err(FatalUpgradeError::MissingPeer), }; let dst = match peer_id_res { Ok(res) => res, - Err(_) => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( - FatalUpgradeError::ParsePeerId, - ))) - } + Err(_) => return Err(FatalUpgradeError::ParsePeerId), }; - let req = CircuitReq { dst, substream }; - - handler::Event::CircuitReqReceived { - inbound_circuit_req: req, - endpoint, - } - } - Type::STATUS => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Left( - FatalUpgradeError::UnexpectedTypeStatus, - ))) + Either::Right(CircuitReq { dst, substream }) } + Type::STATUS => return Err(FatalUpgradeError::UnexpectedTypeStatus), }; - ConnectionHandlerEvent::NotifyBehaviour(event) + Ok(req) } diff --git a/protocols/relay/src/protocol/outbound_stop.rs b/protocols/relay/src/protocol/outbound_stop.rs index a6b35267aea..e4502957995 100644 --- a/protocols/relay/src/protocol/outbound_stop.rs +++ b/protocols/relay/src/protocol/outbound_stop.rs @@ -21,15 +21,14 @@ use std::time::Duration; use asynchronous_codec::{Framed, FramedParts}; -use either::Either; +use bytes::Bytes; use futures::channel::oneshot::{self}; use futures::prelude::*; use thiserror::Error; use libp2p_identity::PeerId; -use libp2p_swarm::{ConnectionHandlerEvent, ConnectionId, Stream, StreamUpgradeError}; +use libp2p_swarm::{ConnectionId, Stream, StreamUpgradeError}; -use crate::behaviour::handler; use crate::behaviour::handler::Config; use crate::protocol::{inbound_hop, MAX_MESSAGE_SIZE}; use crate::{proto, CircuitId}; @@ -79,7 +78,7 @@ pub(crate) async fn connect( io: Stream, stop_command: PendingConnect, tx: oneshot::Sender<()>, -) -> handler::RelayConnectionHandlerEvent { +) -> Result, FatalUpgradeError> { let msg = proto::StopMessage { type_pb: proto::StopMessageType::CONNECT, peer: Some(proto::Peer { @@ -102,17 +101,13 @@ pub(crate) async fn connect( let mut substream = Framed::new(io, quick_protobuf_codec::Codec::new(MAX_MESSAGE_SIZE)); if substream.send(msg).await.is_err() { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( - FatalUpgradeError::StreamClosed, - ))); + return Err(FatalUpgradeError::StreamClosed); } let res = substream.next().await; if let None | Some(Err(_)) = res { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( - FatalUpgradeError::StreamClosed, - ))); + return Err(FatalUpgradeError::StreamClosed); } let proto::StopMessage { @@ -123,50 +118,34 @@ pub(crate) async fn connect( } = res.unwrap().expect("should be ok"); match type_pb { - proto::StopMessageType::CONNECT => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( - FatalUpgradeError::UnexpectedTypeConnect, - ))) - } + proto::StopMessageType::CONNECT => return Err(FatalUpgradeError::UnexpectedTypeConnect), proto::StopMessageType::STATUS => {} } match status { Some(proto::Status::OK) => {} Some(proto::Status::RESOURCE_LIMIT_EXCEEDED) => { - return ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto::Status::RESOURCE_LIMIT_EXCEEDED, - error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), - }, - ) + return Ok(Err(CircuitFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto::Status::RESOURCE_LIMIT_EXCEEDED, + error: StreamUpgradeError::Apply(CircuitFailedReason::ResourceLimitExceeded), + })) } Some(proto::Status::PERMISSION_DENIED) => { - return ConnectionHandlerEvent::NotifyBehaviour( - handler::Event::OutboundConnectNegotiationFailed { - circuit_id: stop_command.circuit_id, - src_peer_id: stop_command.src_peer_id, - src_connection_id: stop_command.src_connection_id, - inbound_circuit_req: stop_command.inbound_circuit_req, - status: proto::Status::PERMISSION_DENIED, - error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), - }, - ) - } - Some(s) => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( - FatalUpgradeError::UnexpectedStatus(s), - ))) - } - None => { - return ConnectionHandlerEvent::Close(StreamUpgradeError::Apply(Either::Right( - FatalUpgradeError::MissingStatusField, - ))) + return Ok(Err(CircuitFailed { + circuit_id: stop_command.circuit_id, + src_peer_id: stop_command.src_peer_id, + src_connection_id: stop_command.src_connection_id, + inbound_circuit_req: stop_command.inbound_circuit_req, + status: proto::Status::PERMISSION_DENIED, + error: StreamUpgradeError::Apply(CircuitFailedReason::PermissionDenied), + })) } + Some(s) => return Err(FatalUpgradeError::UnexpectedStatus(s)), + None => return Err(FatalUpgradeError::MissingStatusField), } let FramedParts { @@ -180,7 +159,7 @@ pub(crate) async fn connect( "Expect a flushed Framed to have an empty write buffer." ); - ConnectionHandlerEvent::NotifyBehaviour(handler::Event::OutboundConnectNegotiated { + Ok(Ok(Circuit { circuit_id: stop_command.circuit_id, src_peer_id: stop_command.src_peer_id, src_connection_id: stop_command.src_connection_id, @@ -188,7 +167,26 @@ pub(crate) async fn connect( dst_handler_notifier: tx, dst_stream: io, dst_pending_data: read_buffer.freeze(), - }) + })) +} + +pub(crate) struct Circuit { + pub(crate) circuit_id: CircuitId, + pub(crate) src_peer_id: PeerId, + pub(crate) src_connection_id: ConnectionId, + pub(crate) inbound_circuit_req: inbound_hop::CircuitReq, + pub(crate) dst_handler_notifier: oneshot::Sender<()>, + pub(crate) dst_stream: Stream, + pub(crate) dst_pending_data: Bytes, +} + +pub(crate) struct CircuitFailed { + pub(crate) circuit_id: CircuitId, + pub(crate) src_peer_id: PeerId, + pub(crate) src_connection_id: ConnectionId, + pub(crate) inbound_circuit_req: inbound_hop::CircuitReq, + pub(crate) status: proto::Status, + pub(crate) error: StreamUpgradeError, } pub(crate) struct PendingConnect { From 38d9a286c9c5152ec26d5dfca600f2567a367072 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Wed, 13 Sep 2023 20:13:58 +1000 Subject: [PATCH 78/83] Fix clippy lint --- misc/futures-bounded/src/map.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/misc/futures-bounded/src/map.rs b/misc/futures-bounded/src/map.rs index 765cdc227f2..6b2fa933dab 100644 --- a/misc/futures-bounded/src/map.rs +++ b/misc/futures-bounded/src/map.rs @@ -97,6 +97,7 @@ where self.inner.is_empty() } + #[allow(clippy::needless_pass_by_ref_mut)] // &mut Context is idiomatic. pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { if self.inner.len() < self.capacity { return Poll::Ready(()); From 0cb3841502a6aa79cced23e03e223b22716f80d2 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Wed, 13 Sep 2023 20:22:03 +1000 Subject: [PATCH 79/83] Allow beta lint --- misc/futures-bounded/src/map.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/misc/futures-bounded/src/map.rs b/misc/futures-bounded/src/map.rs index 6b2fa933dab..4d187a0a4ba 100644 --- a/misc/futures-bounded/src/map.rs +++ b/misc/futures-bounded/src/map.rs @@ -97,7 +97,7 @@ where self.inner.is_empty() } - #[allow(clippy::needless_pass_by_ref_mut)] // &mut Context is idiomatic. + #[allow(unknown_lints, clippy::needless_pass_by_ref_mut)] // &mut Context is idiomatic. pub fn poll_ready_unpin(&mut self, cx: &mut Context<'_>) -> Poll<()> { if self.inner.len() < self.capacity { return Poll::Ready(()); From 3b8b07d0bc035ed233d2c31ea8e94eae6dab1142 Mon Sep 17 00:00:00 2001 From: dgarus Date: Tue, 19 Sep 2023 20:05:36 +0300 Subject: [PATCH 80/83] fix review comments --- misc/futures-bounded/src/lib.rs | 4 ++-- misc/futures-bounded/src/{list.rs => set.rs} | 14 +++++++------- protocols/relay/src/behaviour/handler.rs | 4 ++-- protocols/relay/src/priv_client/handler.rs | 8 ++++---- 4 files changed, 15 insertions(+), 15 deletions(-) rename misc/futures-bounded/src/{list.rs => set.rs} (88%) diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs index 355d534e233..091e854981d 100644 --- a/misc/futures-bounded/src/lib.rs +++ b/misc/futures-bounded/src/lib.rs @@ -1,7 +1,7 @@ -mod list; +mod set; mod map; -pub use list::FuturesList; +pub use set::FuturesSet; pub use map::{FuturesMap, PushError}; use std::fmt; use std::fmt::Formatter; diff --git a/misc/futures-bounded/src/list.rs b/misc/futures-bounded/src/set.rs similarity index 88% rename from misc/futures-bounded/src/list.rs rename to misc/futures-bounded/src/set.rs index 79d7bdc656a..96140d82f9a 100644 --- a/misc/futures-bounded/src/list.rs +++ b/misc/futures-bounded/src/set.rs @@ -9,21 +9,21 @@ use crate::{FuturesMap, PushError, Timeout}; /// Represents a list of [Future]s. /// /// Each future must finish within the specified time and the list never outgrows its capacity. -pub struct FuturesList { - id: i32, - inner: FuturesMap, +pub struct FuturesSet { + id: u32, + inner: FuturesMap, } -impl FuturesList { +impl FuturesSet { pub fn new(timeout: Duration, capacity: usize) -> Self { Self { - id: i32::MIN, + id: 0, inner: FuturesMap::new(timeout, capacity), } } } -impl FuturesList { +impl FuturesSet { /// Push a future into the list. /// /// This method adds the given future to the list. @@ -33,7 +33,7 @@ impl FuturesList { where F: Future + Send + 'static, { - (self.id, _) = self.id.overflowing_add(1); + self.id = self.id.wrapping_add(1); match self.inner.try_push(self.id, future) { Ok(()) => Ok(()), diff --git a/protocols/relay/src/behaviour/handler.rs b/protocols/relay/src/behaviour/handler.rs index 91b49589874..895228e807b 100644 --- a/protocols/relay/src/behaviour/handler.rs +++ b/protocols/relay/src/behaviour/handler.rs @@ -386,7 +386,7 @@ pub struct Handler { pending_connect_requests: VecDeque, - workers: futures_bounded::FuturesList< + workers: futures_bounded::FuturesSet< Either< Result< Either, @@ -403,7 +403,7 @@ pub struct Handler { impl Handler { pub fn new(config: Config, endpoint: ConnectedPoint) -> Handler { Handler { - workers: futures_bounded::FuturesList::new( + workers: futures_bounded::FuturesSet::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 28427761c2a..12fc0fb5efa 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -136,7 +136,7 @@ pub struct Handler { >, wait_for_outbound_stream: VecDeque, - outbound_circuits: futures_bounded::FuturesList< + outbound_circuits: futures_bounded::FuturesSet< Result< Either< Result, @@ -158,7 +158,7 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - open_circuit_futs: futures_bounded::FuturesList< + open_circuit_futs: futures_bounded::FuturesSet< Result, >, @@ -180,13 +180,13 @@ impl Handler { queued_events: Default::default(), pending_error: Default::default(), wait_for_outbound_stream: Default::default(), - outbound_circuits: futures_bounded::FuturesList::new( + outbound_circuits: futures_bounded::FuturesSet::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), reservation: Reservation::None, alive_lend_out_substreams: Default::default(), - open_circuit_futs: futures_bounded::FuturesList::new( + open_circuit_futs: futures_bounded::FuturesSet::new( STREAM_TIMEOUT, MAX_CONCURRENT_STREAMS_PER_CONNECTION, ), From d14eb90dc9b0e696c83cdd5964f547451ce4a97d Mon Sep 17 00:00:00 2001 From: Denis Garus Date: Tue, 19 Sep 2023 20:06:35 +0300 Subject: [PATCH 81/83] Update misc/futures-bounded/src/map.rs Co-authored-by: Max Inden --- misc/futures-bounded/src/map.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/misc/futures-bounded/src/map.rs b/misc/futures-bounded/src/map.rs index 4d187a0a4ba..cecf6070efe 100644 --- a/misc/futures-bounded/src/map.rs +++ b/misc/futures-bounded/src/map.rs @@ -12,7 +12,7 @@ use futures_util::{FutureExt, StreamExt}; use crate::Timeout; -/// Represents a map of [Future]s. +/// Represents a map of [`Future`]s. /// /// Each future must finish within the specified time and the map never outgrows its capacity. pub struct FuturesMap { From 2de759c36f045e4f500861860f0a65eb4c546958 Mon Sep 17 00:00:00 2001 From: Sumit <106421807+startup-dreamer@users.noreply.github.com> Date: Wed, 20 Sep 2023 04:02:29 +0530 Subject: [PATCH 82/83] feat(swarm): allow configuration to idle connection timeout Previously, a connection would be shut down immediately as soon as its `ConnectionHandler` reports `KeepAlive::No`. As we have gained experience with libp2p, it turned out that this isn't ideal. For one, tests often need to keep connections alive longer than the configured protocols require. Plus, some usecases require connections to be kept alive in general. Both of these needs are currently served by the `keep_alive::Behaviour`. That one does essentially nothing other than statically returning `KeepAlive::Yes` from its `ConnectionHandler`. It makes much more sense to deprecate `keep_alive::Behaviour` and instead allow users to globally configure an `idle_conncetion_timeout` on the `Swarm`. This timeout comes into effect once a `ConnectionHandler` reports `KeepAlive::No`. To start with, this timeout is 0. Together with https://github.com/libp2p/rust-libp2p/issues/3844, this will allow us to move towards a much more aggressive closing of idle connections, together with a more ergonomic way of opting out of this behaviour. Fixes #4121. Pull-Request: #4161. --- examples/browser-webrtc/src/main.rs | 19 +- examples/metrics/src/main.rs | 9 +- examples/ping-example/src/main.rs | 16 +- examples/rendezvous/src/bin/rzv-discover.rs | 5 +- examples/rendezvous/src/bin/rzv-identify.rs | 5 +- examples/rendezvous/src/bin/rzv-register.rs | 5 +- examples/rendezvous/src/main.rs | 5 +- interop-tests/src/lib.rs | 9 +- libp2p/src/tutorials/ping.rs | 97 +++++------ misc/allow-block-list/src/lib.rs | 98 +++-------- misc/connection-limits/src/lib.rs | 3 - protocols/identify/tests/smoke.rs | 82 +++------ protocols/ping/tests/ping.rs | 38 ++-- swarm-test/src/lib.rs | 4 +- swarm/CHANGELOG.md | 4 + swarm/Cargo.toml | 1 + swarm/src/connection.rs | 182 ++++++++++++++++++-- swarm/src/connection/pool.rs | 10 +- swarm/src/lib.rs | 71 ++++---- transports/pnet/tests/smoke.rs | 13 +- transports/tls/tests/smoke.rs | 14 +- transports/webrtc/examples/listen_ping.rs | 19 +- 22 files changed, 378 insertions(+), 331 deletions(-) diff --git a/examples/browser-webrtc/src/main.rs b/examples/browser-webrtc/src/main.rs index f919f047af5..8a4034a436e 100644 --- a/examples/browser-webrtc/src/main.rs +++ b/examples/browser-webrtc/src/main.rs @@ -13,11 +13,12 @@ use libp2p::{ identity, multiaddr::{Multiaddr, Protocol}, ping, - swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}, + swarm::{SwarmBuilder, SwarmEvent}, }; use libp2p_webrtc as webrtc; use rand::thread_rng; use std::net::{Ipv4Addr, SocketAddr}; +use std::time::Duration; use tower_http::cors::{Any, CorsLayer}; #[tokio::main] @@ -36,12 +37,10 @@ async fn main() -> anyhow::Result<()> { .map(|(peer_id, conn), _| (peer_id, StreamMuxerBox::new(conn))) .boxed(); - let behaviour = Behaviour { - ping: ping::Behaviour::new(ping::Config::new()), - keep_alive: keep_alive::Behaviour, - }; - - let mut swarm = SwarmBuilder::with_tokio_executor(transport, behaviour, local_peer_id).build(); + let mut swarm = + SwarmBuilder::with_tokio_executor(transport, ping::Behaviour::default(), local_peer_id) + .idle_connection_timeout(Duration::from_secs(30)) // Allows us to observe the pings. + .build(); let address_webrtc = Multiaddr::from(Ipv4Addr::UNSPECIFIED) .with(Protocol::Udp(0)) @@ -84,12 +83,6 @@ async fn main() -> anyhow::Result<()> { Ok(()) } -#[derive(NetworkBehaviour)] -struct Behaviour { - ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, -} - #[derive(rust_embed::RustEmbed)] #[folder = "$CARGO_MANIFEST_DIR/static"] struct StaticFiles; diff --git a/examples/metrics/src/main.rs b/examples/metrics/src/main.rs index fa9d5bd37e7..b28abaee941 100644 --- a/examples/metrics/src/main.rs +++ b/examples/metrics/src/main.rs @@ -26,12 +26,13 @@ use futures::stream::StreamExt; use libp2p::core::{upgrade::Version, Multiaddr, Transport}; use libp2p::identity::PeerId; use libp2p::metrics::{Metrics, Recorder}; -use libp2p::swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}; +use libp2p::swarm::{NetworkBehaviour, SwarmBuilder, SwarmEvent}; use libp2p::{identify, identity, noise, ping, tcp, yamux}; use log::info; use prometheus_client::registry::Registry; use std::error::Error; use std::thread; +use std::time::Duration; mod http_service; @@ -51,6 +52,7 @@ fn main() -> Result<(), Box> { Behaviour::new(local_pub_key), local_peer_id, ) + .idle_connection_timeout(Duration::from_secs(60)) .build(); swarm.listen_on("/ip4/0.0.0.0/tcp/0".parse()?)?; @@ -87,13 +89,9 @@ fn main() -> Result<(), Box> { } /// Our network behaviour. -/// -/// For illustrative purposes, this includes the [`keep_alive::Behaviour`]) behaviour so the ping actually happen -/// and can be observed via the metrics. #[derive(NetworkBehaviour)] struct Behaviour { identify: identify::Behaviour, - keep_alive: keep_alive::Behaviour, ping: ping::Behaviour, } @@ -105,7 +103,6 @@ impl Behaviour { "/ipfs/0.1.0".into(), local_pub_key, )), - keep_alive: keep_alive::Behaviour, } } } diff --git a/examples/ping-example/src/main.rs b/examples/ping-example/src/main.rs index 6af079ccfd0..898a25813e0 100644 --- a/examples/ping-example/src/main.rs +++ b/examples/ping-example/src/main.rs @@ -24,10 +24,11 @@ use futures::prelude::*; use libp2p::core::upgrade::Version; use libp2p::{ identity, noise, ping, - swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}, + swarm::{SwarmBuilder, SwarmEvent}, tcp, yamux, Multiaddr, PeerId, Transport, }; use std::error::Error; +use std::time::Duration; #[async_std::main] async fn main() -> Result<(), Box> { @@ -42,7 +43,8 @@ async fn main() -> Result<(), Box> { .boxed(); let mut swarm = - SwarmBuilder::with_async_std_executor(transport, Behaviour::default(), local_peer_id) + SwarmBuilder::with_async_std_executor(transport, ping::Behaviour::default(), local_peer_id) + .idle_connection_timeout(Duration::from_secs(60)) // For illustrative purposes, keep idle connections alive for a minute so we can observe a few pings. .build(); // Tell the swarm to listen on all interfaces and a random, OS-assigned @@ -65,13 +67,3 @@ async fn main() -> Result<(), Box> { } } } - -/// Our network behaviour. -/// -/// For illustrative purposes, this includes the [`KeepAlive`](keep_alive::Behaviour) behaviour so a continuous sequence of -/// pings can be observed. -#[derive(NetworkBehaviour, Default)] -struct Behaviour { - keep_alive: keep_alive::Behaviour, - ping: ping::Behaviour, -} diff --git a/examples/rendezvous/src/bin/rzv-discover.rs b/examples/rendezvous/src/bin/rzv-discover.rs index bd510496ae3..ac45afae840 100644 --- a/examples/rendezvous/src/bin/rzv-discover.rs +++ b/examples/rendezvous/src/bin/rzv-discover.rs @@ -24,7 +24,7 @@ use libp2p::{ identity, multiaddr::Protocol, noise, ping, rendezvous, - swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}, + swarm::{NetworkBehaviour, SwarmBuilder, SwarmEvent}, tcp, yamux, Multiaddr, PeerId, Transport, }; use std::time::Duration; @@ -50,10 +50,10 @@ async fn main() { MyBehaviour { rendezvous: rendezvous::client::Behaviour::new(key_pair.clone()), ping: ping::Behaviour::new(ping::Config::new().with_interval(Duration::from_secs(1))), - keep_alive: keep_alive::Behaviour, }, PeerId::from(key_pair.public()), ) + .idle_connection_timeout(Duration::from_secs(5)) .build(); swarm.dial(rendezvous_point_address.clone()).unwrap(); @@ -127,5 +127,4 @@ async fn main() { struct MyBehaviour { rendezvous: rendezvous::client::Behaviour, ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, } diff --git a/examples/rendezvous/src/bin/rzv-identify.rs b/examples/rendezvous/src/bin/rzv-identify.rs index 1ef8569c612..95ed7a5ccd8 100644 --- a/examples/rendezvous/src/bin/rzv-identify.rs +++ b/examples/rendezvous/src/bin/rzv-identify.rs @@ -22,7 +22,7 @@ use futures::StreamExt; use libp2p::{ core::transport::upgrade::Version, identify, identity, noise, ping, rendezvous, - swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}, + swarm::{NetworkBehaviour, SwarmBuilder, SwarmEvent}, tcp, yamux, Multiaddr, PeerId, Transport, }; use std::time::Duration; @@ -50,10 +50,10 @@ async fn main() { )), rendezvous: rendezvous::client::Behaviour::new(key_pair.clone()), ping: ping::Behaviour::new(ping::Config::new().with_interval(Duration::from_secs(1))), - keep_alive: keep_alive::Behaviour, }, PeerId::from(key_pair.public()), ) + .idle_connection_timeout(Duration::from_secs(5)) .build(); let _ = swarm.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap()); @@ -133,5 +133,4 @@ struct MyBehaviour { identify: identify::Behaviour, rendezvous: rendezvous::client::Behaviour, ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, } diff --git a/examples/rendezvous/src/bin/rzv-register.rs b/examples/rendezvous/src/bin/rzv-register.rs index 95407dd9c7d..51acfee2a71 100644 --- a/examples/rendezvous/src/bin/rzv-register.rs +++ b/examples/rendezvous/src/bin/rzv-register.rs @@ -22,7 +22,7 @@ use futures::StreamExt; use libp2p::{ core::transport::upgrade::Version, identity, noise, ping, rendezvous, - swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}, + swarm::{NetworkBehaviour, SwarmBuilder, SwarmEvent}, tcp, yamux, Multiaddr, PeerId, Transport, }; use std::time::Duration; @@ -46,10 +46,10 @@ async fn main() { MyBehaviour { rendezvous: rendezvous::client::Behaviour::new(key_pair.clone()), ping: ping::Behaviour::new(ping::Config::new().with_interval(Duration::from_secs(1))), - keep_alive: keep_alive::Behaviour, }, PeerId::from(key_pair.public()), ) + .idle_connection_timeout(Duration::from_secs(5)) .build(); // In production the external address should be the publicly facing IP address of the rendezvous point. @@ -130,5 +130,4 @@ async fn main() { struct MyBehaviour { rendezvous: rendezvous::client::Behaviour, ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, } diff --git a/examples/rendezvous/src/main.rs b/examples/rendezvous/src/main.rs index 44b1716e176..a3ed3c0fce5 100644 --- a/examples/rendezvous/src/main.rs +++ b/examples/rendezvous/src/main.rs @@ -24,7 +24,7 @@ use futures::StreamExt; use libp2p::{ core::transport::upgrade::Version, identify, identity, noise, ping, rendezvous, - swarm::{keep_alive, NetworkBehaviour, SwarmBuilder, SwarmEvent}, + swarm::{NetworkBehaviour, SwarmBuilder, SwarmEvent}, tcp, yamux, PeerId, Transport, }; use std::time::Duration; @@ -48,10 +48,10 @@ async fn main() { )), rendezvous: rendezvous::server::Behaviour::new(rendezvous::server::Config::default()), ping: ping::Behaviour::new(ping::Config::new().with_interval(Duration::from_secs(1))), - keep_alive: keep_alive::Behaviour, }, PeerId::from(key_pair.public()), ) + .idle_connection_timeout(Duration::from_secs(5)) .build(); let _ = swarm.listen_on("/ip4/0.0.0.0/tcp/62649".parse().unwrap()); @@ -97,5 +97,4 @@ struct MyBehaviour { identify: identify::Behaviour, rendezvous: rendezvous::server::Behaviour, ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, } diff --git a/interop-tests/src/lib.rs b/interop-tests/src/lib.rs index 54d94430c8e..40c06b57810 100644 --- a/interop-tests/src/lib.rs +++ b/interop-tests/src/lib.rs @@ -3,8 +3,8 @@ use std::time::Duration; use anyhow::{bail, Context, Result}; use futures::{FutureExt, StreamExt}; -use libp2p::swarm::{keep_alive, NetworkBehaviour, SwarmEvent}; -use libp2p::{identify, identity, ping, Multiaddr, PeerId}; +use libp2p::swarm::SwarmEvent; +use libp2p::{identify, identity, ping, swarm::NetworkBehaviour, Multiaddr, PeerId}; #[cfg(target_arch = "wasm32")] use wasm_bindgen::prelude::*; @@ -33,8 +33,7 @@ pub async fn run_test( let mut swarm = swarm_builder( boxed_transport, Behaviour { - ping: ping::Behaviour::new(ping::Config::new().with_interval(Duration::from_secs(1))), - keep_alive: keep_alive::Behaviour, + ping: ping::Behaviour::new(ping::Config::new().with_interval(Duration::from_secs(10))), // Need to include identify until https://github.com/status-im/nim-libp2p/issues/924 is resolved. identify: identify::Behaviour::new(identify::Config::new( "/interop-tests".to_owned(), @@ -43,6 +42,7 @@ pub async fn run_test( }, local_peer_id, ) + .idle_connection_timeout(Duration::from_secs(5)) .build(); log::info!("Running ping test: {}", swarm.local_peer_id()); @@ -242,7 +242,6 @@ impl FromStr for SecProtocol { #[derive(NetworkBehaviour)] struct Behaviour { ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, identify: identify::Behaviour, } diff --git a/libp2p/src/tutorials/ping.rs b/libp2p/src/tutorials/ping.rs index 006c807803c..aedc149228e 100644 --- a/libp2p/src/tutorials/ping.rs +++ b/libp2p/src/tutorials/ping.rs @@ -143,7 +143,7 @@ //! With the above in mind, let's extend our example, creating a [`ping::Behaviour`](crate::ping::Behaviour) at the end: //! //! ```rust -//! use libp2p::swarm::{keep_alive, NetworkBehaviour}; +//! use libp2p::swarm::NetworkBehaviour; //! use libp2p::{identity, ping, PeerId}; //! use std::error::Error; //! @@ -155,20 +155,10 @@ //! //! let transport = libp2p::development_transport(local_key).await?; //! -//! let behaviour = Behaviour::default(); +//! let behaviour = ping::Behaviour::default(); //! //! Ok(()) //! } -//! -//! /// Our network behaviour. -//! /// -//! /// For illustrative purposes, this includes the [`KeepAlive`](behaviour::KeepAlive) behaviour so a continuous sequence of -//! /// pings can be observed. -//! #[derive(NetworkBehaviour, Default)] -//! struct Behaviour { -//! keep_alive: keep_alive::Behaviour, -//! ping: ping::Behaviour, -//! } //! ``` //! //! ## Swarm @@ -185,7 +175,7 @@ //! (In our example, `env_logger` is used) //! //! ```rust -//! use libp2p::swarm::{keep_alive, NetworkBehaviour, SwarmBuilder}; +//! use libp2p::swarm::{NetworkBehaviour, SwarmBuilder}; //! use libp2p::{identity, ping, PeerId}; //! use std::error::Error; //! @@ -197,21 +187,46 @@ //! //! let transport = libp2p::development_transport(local_key).await?; //! -//! let behaviour = Behaviour::default(); +//! let behaviour = ping::Behaviour::default(); //! //! let mut swarm = SwarmBuilder::with_async_std_executor(transport, behaviour, local_peer_id).build(); //! //! Ok(()) //! } +//! ``` +//! +//! ## Idle connection timeout +//! +//! Now, for this example in particular, we need set the idle connection timeout. +//! Otherwise, the connection will be closed immediately. +//! +//! Whether you need to set this in your application too depends on your usecase. +//! Typically, connections are kept alive if they are "in use" by a certain protocol. +//! The ping protocol however is only an "auxiliary" kind of protocol. +//! Thus, without any other behaviour in place, we would not be able to observe the pings. +//! +//! ```rust +//! use libp2p::swarm::{NetworkBehaviour, SwarmBuilder}; +//! use libp2p::{identity, ping, PeerId}; +//! use std::error::Error; +//! use std::time::Duration; +//! +//! #[async_std::main] +//! async fn main() -> Result<(), Box> { +//! use std::time::Duration; +//! let local_key = identity::Keypair::generate_ed25519(); +//! let local_peer_id = PeerId::from(local_key.public()); +//! println!("Local peer id: {local_peer_id:?}"); +//! +//! let transport = libp2p::development_transport(local_key).await?; //! -//! /// Our network behaviour. -//! /// -//! /// For illustrative purposes, this includes the [`KeepAlive`](behaviour:: -//! /// KeepAlive) behaviour so a continuous sequence of pings can be observed. -//! #[derive(NetworkBehaviour, Default)] -//! struct Behaviour { -//! keep_alive: keep_alive::Behaviour, -//! ping: ping::Behaviour, +//! let behaviour = ping::Behaviour::default(); +//! +//! let mut swarm = SwarmBuilder::with_async_std_executor(transport, behaviour, local_peer_id) +//! .idle_connection_timeout(Duration::from_secs(30)) // Allows us to observe pings for 30 seconds. +//! .build(); +//! +//! Ok(()) //! } //! ``` //! @@ -242,9 +257,10 @@ //! remote peer. //! //! ```rust -//! use libp2p::swarm::{keep_alive, NetworkBehaviour, SwarmBuilder}; +//! use libp2p::swarm::{NetworkBehaviour, SwarmBuilder}; //! use libp2p::{identity, ping, Multiaddr, PeerId}; //! use std::error::Error; +//! use std::time::Duration; //! //! #[async_std::main] //! async fn main() -> Result<(), Box> { @@ -254,9 +270,11 @@ //! //! let transport = libp2p::development_transport(local_key).await?; //! -//! let behaviour = Behaviour::default(); +//! let behaviour = ping::Behaviour::default(); //! -//! let mut swarm = SwarmBuilder::with_async_std_executor(transport, behaviour, local_peer_id).build(); +//! let mut swarm = SwarmBuilder::with_async_std_executor(transport, behaviour, local_peer_id) +//! .idle_connection_timeout(Duration::from_secs(30)) // Allows us to observe pings for 30 seconds. +//! .build(); //! //! // Tell the swarm to listen on all interfaces and a random, OS-assigned //! // port. @@ -272,16 +290,6 @@ //! //! Ok(()) //! } -//! -//! /// Our network behaviour. -//! /// -//! /// For illustrative purposes, this includes the [`KeepAlive`](behaviour::KeepAlive) behaviour so a continuous sequence of -//! /// pings can be observed. -//! #[derive(NetworkBehaviour, Default)] -//! struct Behaviour { -//! keep_alive: keep_alive::Behaviour, -//! ping: ping::Behaviour, -//! } //! ``` //! //! ## Continuously polling the Swarm @@ -292,9 +300,10 @@ //! //! ```no_run //! use futures::prelude::*; -//! use libp2p::swarm::{keep_alive, NetworkBehaviour, SwarmEvent, SwarmBuilder}; +//! use libp2p::swarm::{NetworkBehaviour, SwarmEvent, SwarmBuilder}; //! use libp2p::{identity, ping, Multiaddr, PeerId}; //! use std::error::Error; +//! use std::time::Duration; //! //! #[async_std::main] //! async fn main() -> Result<(), Box> { @@ -304,9 +313,11 @@ //! //! let transport = libp2p::development_transport(local_key).await?; //! -//! let behaviour = Behaviour::default(); +//! let behaviour = ping::Behaviour::default(); //! -//! let mut swarm = SwarmBuilder::with_async_std_executor(transport, behaviour, local_peer_id).build(); +//! let mut swarm = SwarmBuilder::with_async_std_executor(transport, behaviour, local_peer_id) +//! .idle_connection_timeout(Duration::from_secs(30)) // Allows us to observe pings for 30 seconds. +//! .build(); //! //! // Tell the swarm to listen on all interfaces and a random, OS-assigned //! // port. @@ -328,16 +339,6 @@ //! } //! } //! } -//! -//! /// Our network behaviour. -//! /// -//! /// For illustrative purposes, this includes the [`KeepAlive`](behaviour::KeepAlive) behaviour so a continuous sequence of -//! /// pings can be observed. -//! #[derive(NetworkBehaviour, Default)] -//! struct Behaviour { -//! keep_alive: keep_alive::Behaviour, -//! ping: ping::Behaviour, -//! } //! ``` //! //! ## Running two nodes diff --git a/misc/allow-block-list/src/lib.rs b/misc/allow-block-list/src/lib.rs index eed79d740a1..1950c47f28b 100644 --- a/misc/allow-block-list/src/lib.rs +++ b/misc/allow-block-list/src/lib.rs @@ -283,14 +283,11 @@ mod tests { #[async_std::test] async fn cannot_dial_blocked_peer() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; - dialer - .behaviour_mut() - .list - .block_peer(*listener.local_peer_id()); + dialer.behaviour_mut().block_peer(*listener.local_peer_id()); let DialError::Denied { cause } = dial(&mut dialer, &listener).unwrap_err() else { panic!("unexpected dial error") @@ -300,17 +297,13 @@ mod tests { #[async_std::test] async fn can_dial_unblocked_peer() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; + dialer.behaviour_mut().block_peer(*listener.local_peer_id()); dialer .behaviour_mut() - .list - .block_peer(*listener.local_peer_id()); - dialer - .behaviour_mut() - .list .unblock_peer(*listener.local_peer_id()); dial(&mut dialer, &listener).unwrap(); @@ -318,14 +311,11 @@ mod tests { #[async_std::test] async fn blocked_peer_cannot_dial_us() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; - listener - .behaviour_mut() - .list - .block_peer(*dialer.local_peer_id()); + listener.behaviour_mut().block_peer(*dialer.local_peer_id()); dial(&mut dialer, &listener).unwrap(); async_std::task::spawn(dialer.loop_on_next()); @@ -343,15 +333,12 @@ mod tests { #[async_std::test] async fn connections_get_closed_upon_blocked() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; dialer.connect(&mut listener).await; - dialer - .behaviour_mut() - .list - .block_peer(*listener.local_peer_id()); + dialer.behaviour_mut().block_peer(*listener.local_peer_id()); let ( [SwarmEvent::ConnectionClosed { @@ -372,8 +359,8 @@ mod tests { #[async_std::test] async fn cannot_dial_peer_unless_allowed() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; let DialError::Denied { cause } = dial(&mut dialer, &listener).unwrap_err() else { @@ -381,26 +368,19 @@ mod tests { }; assert!(cause.downcast::().is_ok()); - dialer - .behaviour_mut() - .list - .allow_peer(*listener.local_peer_id()); + dialer.behaviour_mut().allow_peer(*listener.local_peer_id()); assert!(dial(&mut dialer, &listener).is_ok()); } #[async_std::test] async fn cannot_dial_disallowed_peer() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; + dialer.behaviour_mut().allow_peer(*listener.local_peer_id()); dialer .behaviour_mut() - .list - .allow_peer(*listener.local_peer_id()); - dialer - .behaviour_mut() - .list .disallow_peer(*listener.local_peer_id()); let DialError::Denied { cause } = dial(&mut dialer, &listener).unwrap_err() else { @@ -411,8 +391,8 @@ mod tests { #[async_std::test] async fn not_allowed_peer_cannot_dial_us() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; dialer @@ -448,23 +428,16 @@ mod tests { #[async_std::test] async fn connections_get_closed_upon_disallow() { - let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::new()); - let mut listener = Swarm::new_ephemeral(|_| Behaviour::::new()); + let mut dialer = Swarm::new_ephemeral(|_| Behaviour::::default()); + let mut listener = Swarm::new_ephemeral(|_| Behaviour::::default()); listener.listen().await; - dialer - .behaviour_mut() - .list - .allow_peer(*listener.local_peer_id()); - listener - .behaviour_mut() - .list - .allow_peer(*dialer.local_peer_id()); + dialer.behaviour_mut().allow_peer(*listener.local_peer_id()); + listener.behaviour_mut().allow_peer(*dialer.local_peer_id()); dialer.connect(&mut listener).await; dialer .behaviour_mut() - .list .disallow_peer(*listener.local_peer_id()); let ( [SwarmEvent::ConnectionClosed { @@ -496,27 +469,4 @@ mod tests { .build(), ) } - - #[derive(libp2p_swarm_derive::NetworkBehaviour)] - #[behaviour(prelude = "libp2p_swarm::derive_prelude")] - struct Behaviour { - list: super::Behaviour, - keep_alive: libp2p_swarm::keep_alive::Behaviour, - } - - impl Behaviour - where - S: Default, - { - fn new() -> Self { - Self { - list: super::Behaviour { - waker: None, - close_connections: VecDeque::new(), - state: S::default(), - }, - keep_alive: libp2p_swarm::keep_alive::Behaviour, - } - } - } } diff --git a/misc/connection-limits/src/lib.rs b/misc/connection-limits/src/lib.rs index e4723dd95c6..7de96cc1736 100644 --- a/misc/connection-limits/src/lib.rs +++ b/misc/connection-limits/src/lib.rs @@ -529,7 +529,6 @@ mod tests { #[behaviour(prelude = "libp2p_swarm::derive_prelude")] struct Behaviour { limits: super::Behaviour, - keep_alive: libp2p_swarm::keep_alive::Behaviour, connection_denier: Toggle, } @@ -537,14 +536,12 @@ mod tests { fn new(limits: ConnectionLimits) -> Self { Self { limits: super::Behaviour::new(limits), - keep_alive: libp2p_swarm::keep_alive::Behaviour, connection_denier: None.into(), } } fn new_with_connection_denier(limits: ConnectionLimits) -> Self { Self { limits: super::Behaviour::new(limits), - keep_alive: libp2p_swarm::keep_alive::Behaviour, connection_denier: Some(ConnectionDenier {}).into(), } } diff --git a/protocols/identify/tests/smoke.rs b/protocols/identify/tests/smoke.rs index c70ab3181b4..c1926b4125f 100644 --- a/protocols/identify/tests/smoke.rs +++ b/protocols/identify/tests/smoke.rs @@ -1,6 +1,6 @@ use libp2p_core::multiaddr::Protocol; use libp2p_identify as identify; -use libp2p_swarm::{keep_alive, Swarm, SwarmEvent}; +use libp2p_swarm::{Swarm, SwarmEvent}; use libp2p_swarm_test::SwarmExt; use std::iter; @@ -9,7 +9,7 @@ async fn periodic_identify() { let _ = env_logger::try_init(); let mut swarm1 = Swarm::new_ephemeral(|identity| { - Behaviour::new( + identify::Behaviour::new( identify::Config::new("a".to_string(), identity.public()) .with_agent_version("b".to_string()), ) @@ -17,7 +17,7 @@ async fn periodic_identify() { let swarm1_peer_id = *swarm1.local_peer_id(); let mut swarm2 = Swarm::new_ephemeral(|identity| { - Behaviour::new( + identify::Behaviour::new( identify::Config::new("c".to_string(), identity.public()) .with_agent_version("d".to_string()), ) @@ -33,20 +33,20 @@ async fn periodic_identify() { match libp2p_swarm_test::drive(&mut swarm1, &mut swarm2).await { ( - [BehaviourEvent::Identify(Received { info: s1_info, .. }), BehaviourEvent::Identify(Sent { .. })], - [BehaviourEvent::Identify(Received { info: s2_info, .. }), BehaviourEvent::Identify(Sent { .. })], + [Received { info: s1_info, .. }, Sent { .. }], + [Received { info: s2_info, .. }, Sent { .. }], ) | ( - [BehaviourEvent::Identify(Sent { .. }), BehaviourEvent::Identify(Received { info: s1_info, .. })], - [BehaviourEvent::Identify(Sent { .. }), BehaviourEvent::Identify(Received { info: s2_info, .. })], + [Sent { .. }, Received { info: s1_info, .. }], + [Sent { .. }, Received { info: s2_info, .. }], ) | ( - [BehaviourEvent::Identify(Received { info: s1_info, .. }), BehaviourEvent::Identify(Sent { .. })], - [BehaviourEvent::Identify(Sent { .. }), BehaviourEvent::Identify(Received { info: s2_info, .. })], + [Received { info: s1_info, .. }, Sent { .. }], + [Sent { .. }, Received { info: s2_info, .. }], ) | ( - [BehaviourEvent::Identify(Sent { .. }), BehaviourEvent::Identify(Received { info: s1_info, .. })], - [BehaviourEvent::Identify(Received { info: s2_info, .. }), BehaviourEvent::Identify(Sent { .. })], + [Sent { .. }, Received { info: s1_info, .. }], + [Received { info: s2_info, .. }, Sent { .. }], ) => { assert_eq!(s1_info.public_key.to_peer_id(), swarm2_peer_id); assert_eq!(s1_info.protocol_version, "c"); @@ -83,10 +83,10 @@ async fn identify_push() { let _ = env_logger::try_init(); let mut swarm1 = Swarm::new_ephemeral(|identity| { - Behaviour::new(identify::Config::new("a".to_string(), identity.public())) + identify::Behaviour::new(identify::Config::new("a".to_string(), identity.public())) }); let mut swarm2 = Swarm::new_ephemeral(|identity| { - Behaviour::new( + identify::Behaviour::new( identify::Config::new("a".to_string(), identity.public()) .with_agent_version("b".to_string()), ) @@ -96,33 +96,25 @@ async fn identify_push() { swarm2.connect(&mut swarm1).await; // First, let the periodic identify do its thing. - match libp2p_swarm_test::drive(&mut swarm1, &mut swarm2).await { - ( - [BehaviourEvent::Identify(e1), BehaviourEvent::Identify(e2)], - [BehaviourEvent::Identify(e3), BehaviourEvent::Identify(e4)], - ) => { - use identify::Event::{Received, Sent}; + let ([e1, e2], [e3, e4]) = libp2p_swarm_test::drive(&mut swarm1, &mut swarm2).await; - // These can be received in any order, hence assert them here instead of the pattern above. - assert!(matches!(e1, Received { .. } | Sent { .. })); - assert!(matches!(e2, Received { .. } | Sent { .. })); - assert!(matches!(e3, Received { .. } | Sent { .. })); - assert!(matches!(e4, Received { .. } | Sent { .. })); - } - other => panic!("Unexpected events: {other:?}"), - }; + { + use identify::Event::{Received, Sent}; + + // These can be received in any order, hence assert them here. + assert!(matches!(e1, Received { .. } | Sent { .. })); + assert!(matches!(e2, Received { .. } | Sent { .. })); + assert!(matches!(e3, Received { .. } | Sent { .. })); + assert!(matches!(e4, Received { .. } | Sent { .. })); + } // Second, actively push. swarm2 .behaviour_mut() - .identify .push(iter::once(*swarm1.local_peer_id())); let swarm1_received_info = match libp2p_swarm_test::drive(&mut swarm1, &mut swarm2).await { - ( - [BehaviourEvent::Identify(identify::Event::Received { info, .. })], - [BehaviourEvent::Identify(identify::Event::Pushed { .. })], - ) => info, + ([identify::Event::Received { info, .. }], [identify::Event::Pushed { .. }]) => info, other => panic!("Unexpected events: {other:?}"), }; @@ -141,10 +133,10 @@ async fn discover_peer_after_disconnect() { let _ = env_logger::try_init(); let mut swarm1 = Swarm::new_ephemeral(|identity| { - Behaviour::new(identify::Config::new("a".to_string(), identity.public())) + identify::Behaviour::new(identify::Config::new("a".to_string(), identity.public())) }); let mut swarm2 = Swarm::new_ephemeral(|identity| { - Behaviour::new( + identify::Behaviour::new( identify::Config::new("a".to_string(), identity.public()) .with_agent_version("b".to_string()), ) @@ -161,7 +153,7 @@ async fn discover_peer_after_disconnect() { .wait(|event| { matches!( event, - SwarmEvent::Behaviour(BehaviourEvent::Identify(identify::Event::Received { .. })) + SwarmEvent::Behaviour(identify::Event::Received { .. }) ) .then_some(()) }) @@ -186,23 +178,3 @@ async fn discover_peer_after_disconnect() { assert_eq!(connected_peer, swarm1_peer_id); } - -/// Combined behaviour to keep the connection alive after the periodic identify. -/// -/// The identify implementation sets `keep_alive` to `No` once it has done its thing. -/// This can result in unexpected connection closures if one peer is faster than the other. -#[derive(libp2p_swarm::NetworkBehaviour)] -#[behaviour(prelude = "libp2p_swarm::derive_prelude")] -struct Behaviour { - identify: identify::Behaviour, - keep_alive: keep_alive::Behaviour, -} - -impl Behaviour { - fn new(config: identify::Config) -> Self { - Self { - identify: identify::Behaviour::new(config), - keep_alive: keep_alive::Behaviour, - } - } -} diff --git a/protocols/ping/tests/ping.rs b/protocols/ping/tests/ping.rs index 63836a15a78..946a2daadb6 100644 --- a/protocols/ping/tests/ping.rs +++ b/protocols/ping/tests/ping.rs @@ -21,8 +21,8 @@ //! Integration tests for the `Ping` network behaviour. use libp2p_ping as ping; -use libp2p_swarm::keep_alive; -use libp2p_swarm::{NetworkBehaviour, Swarm, SwarmEvent}; +use libp2p_swarm::dummy; +use libp2p_swarm::{Swarm, SwarmEvent}; use libp2p_swarm_test::SwarmExt; use quickcheck::*; use std::{num::NonZeroU8, time::Duration}; @@ -32,18 +32,16 @@ fn ping_pong() { fn prop(count: NonZeroU8) { let cfg = ping::Config::new().with_interval(Duration::from_millis(10)); - let mut swarm1 = Swarm::new_ephemeral(|_| Behaviour::new(cfg.clone())); - let mut swarm2 = Swarm::new_ephemeral(|_| Behaviour::new(cfg.clone())); + let mut swarm1 = Swarm::new_ephemeral(|_| ping::Behaviour::new(cfg.clone())); + let mut swarm2 = Swarm::new_ephemeral(|_| ping::Behaviour::new(cfg.clone())); async_std::task::block_on(async { swarm1.listen().await; swarm2.connect(&mut swarm1).await; for _ in 0..count.get() { - let (e1, e2) = match libp2p_swarm_test::drive(&mut swarm1, &mut swarm2).await { - ([BehaviourEvent::Ping(e1)], [BehaviourEvent::Ping(e2)]) => (e1, e2), - events => panic!("Unexpected events: {events:?}"), - }; + let ([e1], [e2]): ([ping::Event; 1], [ping::Event; 1]) = + libp2p_swarm_test::drive(&mut swarm1, &mut swarm2).await; assert_eq!(&e1.peer, swarm2.local_peer_id()); assert_eq!(&e2.peer, swarm1.local_peer_id()); @@ -65,8 +63,8 @@ fn assert_ping_rtt_less_than_50ms(e: ping::Event) { #[test] fn unsupported_doesnt_fail() { - let mut swarm1 = Swarm::new_ephemeral(|_| keep_alive::Behaviour); - let mut swarm2 = Swarm::new_ephemeral(|_| Behaviour::new(ping::Config::new())); + let mut swarm1 = Swarm::new_ephemeral(|_| dummy::Behaviour); + let mut swarm2 = Swarm::new_ephemeral(|_| ping::Behaviour::new(ping::Config::new())); let result = async_std::task::block_on(async { swarm1.listen().await; @@ -76,10 +74,10 @@ fn unsupported_doesnt_fail() { loop { match swarm2.next_swarm_event().await { - SwarmEvent::Behaviour(BehaviourEvent::Ping(ping::Event { + SwarmEvent::Behaviour(ping::Event { result: Err(ping::Failure::Unsupported), .. - })) => { + }) => { swarm2.disconnect_peer_id(swarm1_peer_id).unwrap(); } SwarmEvent::ConnectionClosed { cause: Some(e), .. } => { @@ -95,19 +93,3 @@ fn unsupported_doesnt_fail() { result.expect("node with ping should not fail connection due to unsupported protocol"); } - -#[derive(NetworkBehaviour, Default)] -#[behaviour(prelude = "libp2p_swarm::derive_prelude")] -struct Behaviour { - keep_alive: keep_alive::Behaviour, - ping: ping::Behaviour, -} - -impl Behaviour { - fn new(config: ping::Config) -> Self { - Self { - keep_alive: keep_alive::Behaviour, - ping: ping::Behaviour::new(config), - } - } -} diff --git a/swarm-test/src/lib.rs b/swarm-test/src/lib.rs index 0ed8dbce220..819db33ba88 100644 --- a/swarm-test/src/lib.rs +++ b/swarm-test/src/lib.rs @@ -218,7 +218,9 @@ where .timeout(Duration::from_secs(20)) .boxed(); - SwarmBuilder::without_executor(transport, behaviour_fn(identity), peer_id).build() + SwarmBuilder::without_executor(transport, behaviour_fn(identity), peer_id) + .idle_connection_timeout(Duration::from_secs(5)) // Some tests need connections to be kept alive beyond what the individual behaviour configures. + .build() } async fn connect(&mut self, other: &mut Swarm) diff --git a/swarm/CHANGELOG.md b/swarm/CHANGELOG.md index 77ddf3458d1..06ddd740873 100644 --- a/swarm/CHANGELOG.md +++ b/swarm/CHANGELOG.md @@ -6,8 +6,12 @@ - Improve error message when `DialPeerCondition` prevents a dial. See [PR 4409]. +- Introduce `SwarmBuilder::idle_conncetion_timeout` and deprecate `keep_alive::Behaviour` as a result. + See [PR 4161]. + [PR 4426]: https://github.com/libp2p/rust-libp2p/pull/4426 [PR 4409]: https://github.com/libp2p/rust-libp2p/pull/4409 +[PR 4161]: https://github.com/libp2p/rust-libp2p/pull/4161 ## 0.43.3 diff --git a/swarm/Cargo.toml b/swarm/Cargo.toml index d00829ce8b2..86f4c158387 100644 --- a/swarm/Cargo.toml +++ b/swarm/Cargo.toml @@ -55,6 +55,7 @@ quickcheck = { workspace = true } void = "1" once_cell = "1.18.0" trybuild = "1.0.85" +tokio = { version = "1.29.1", features = ["time", "rt", "macros"] } [[test]] name = "swarm_derive" diff --git a/swarm/src/connection.rs b/swarm/src/connection.rs index 3796d9a027d..310cf3a81e3 100644 --- a/swarm/src/connection.rs +++ b/swarm/src/connection.rs @@ -52,6 +52,7 @@ use libp2p_core::upgrade; use libp2p_core::upgrade::{NegotiationError, ProtocolError}; use libp2p_core::Endpoint; use libp2p_identity::PeerId; +use std::cmp::max; use std::collections::HashSet; use std::fmt::{Display, Formatter}; use std::future::Future; @@ -156,6 +157,7 @@ where local_supported_protocols: HashSet, remote_supported_protocols: HashSet, + idle_timeout: Duration, } impl fmt::Debug for Connection @@ -183,9 +185,9 @@ where mut handler: THandler, substream_upgrade_protocol_override: Option, max_negotiating_inbound_streams: usize, + idle_timeout: Duration, ) -> Self { let initial_protocols = gather_supported_protocols(&handler); - if !initial_protocols.is_empty() { handler.on_connection_event(ConnectionEvent::LocalProtocolsChange( ProtocolsChange::Added(ProtocolsAdded::from_set(&initial_protocols)), @@ -203,6 +205,7 @@ where requested_substreams: Default::default(), local_supported_protocols: initial_protocols, remote_supported_protocols: Default::default(), + idle_timeout, } } @@ -234,6 +237,7 @@ where substream_upgrade_protocol_override, local_supported_protocols: supported_protocols, remote_supported_protocols, + idle_timeout, } = self.get_mut(); loop { @@ -348,17 +352,36 @@ where (Shutdown::Later(timer, deadline), KeepAlive::Until(t)) => { if *deadline != t { *deadline = t; - if let Some(dur) = deadline.checked_duration_since(Instant::now()) { - timer.reset(dur) + if let Some(new_duration) = deadline.checked_duration_since(Instant::now()) + { + let effective_keep_alive = max(new_duration, *idle_timeout); + + timer.reset(effective_keep_alive) } } } - (_, KeepAlive::Until(t)) => { - if let Some(dur) = t.checked_duration_since(Instant::now()) { - *shutdown = Shutdown::Later(Delay::new(dur), t) + (_, KeepAlive::Until(earliest_shutdown)) => { + if let Some(requested_keep_alive) = + earliest_shutdown.checked_duration_since(Instant::now()) + { + let effective_keep_alive = max(requested_keep_alive, *idle_timeout); + + // Important: We store the _original_ `Instant` given by the `ConnectionHandler` in the `Later` instance to ensure we can compare it in the above branch. + // This is quite subtle but will hopefully become simpler soon once `KeepAlive::Until` is fully deprecated. See / + *shutdown = + Shutdown::Later(Delay::new(effective_keep_alive), earliest_shutdown) } } - (_, KeepAlive::No) => *shutdown = Shutdown::Asap, + (_, KeepAlive::No) if idle_timeout == &Duration::ZERO => { + *shutdown = Shutdown::Asap; + } + (Shutdown::Later(_, _), KeepAlive::No) => { + // Do nothing, i.e. let the shutdown timer continue to tick. + } + (_, KeepAlive::No) => { + let deadline = Instant::now() + *idle_timeout; + *shutdown = Shutdown::Later(Delay::new(*idle_timeout), deadline); + } (_, KeepAlive::Yes) => *shutdown = Shutdown::None, }; @@ -696,7 +719,7 @@ enum Shutdown { #[cfg(test)] mod tests { use super::*; - use crate::keep_alive; + use crate::dummy; use futures::future; use futures::AsyncRead; use futures::AsyncWrite; @@ -704,6 +727,7 @@ mod tests { use libp2p_core::StreamMuxer; use quickcheck::*; use std::sync::{Arc, Weak}; + use std::time::Instant; use void::Void; #[test] @@ -712,14 +736,14 @@ mod tests { let max_negotiating_inbound_streams: usize = max_negotiating_inbound_streams.into(); let alive_substream_counter = Arc::new(()); - let mut connection = Connection::new( StreamMuxerBox::new(DummyStreamMuxer { counter: alive_substream_counter.clone(), }), - keep_alive::ConnectionHandler, + MockConnectionHandler::new(Duration::ZERO), None, max_negotiating_inbound_streams, + Duration::ZERO, ); let result = connection.poll_noop_waker(); @@ -743,6 +767,7 @@ mod tests { MockConnectionHandler::new(upgrade_timeout), None, 2, + Duration::ZERO, ); connection.handler.open_new_outbound(); @@ -765,6 +790,7 @@ mod tests { ConfigurableProtocolConnectionHandler::default(), None, 0, + Duration::ZERO, ); // First, start listening on a single protocol. @@ -803,6 +829,7 @@ mod tests { ConfigurableProtocolConnectionHandler::default(), None, 0, + Duration::ZERO, ); // First, remote supports a single protocol. @@ -846,6 +873,141 @@ mod tests { assert_eq!(connection.handler.remote_removed, vec![vec!["/bar"]]); } + #[tokio::test] + async fn idle_timeout_with_keep_alive_no() { + let idle_timeout = Duration::from_millis(100); + + let mut connection = Connection::new( + StreamMuxerBox::new(PendingStreamMuxer), + dummy::ConnectionHandler, + None, + 0, + idle_timeout, + ); + + assert!(connection.poll_noop_waker().is_pending()); + + tokio::time::sleep(idle_timeout).await; + + assert!(matches!( + connection.poll_noop_waker(), + Poll::Ready(Err(ConnectionError::KeepAliveTimeout)) + )); + } + + #[tokio::test] + async fn idle_timeout_with_keep_alive_until_greater_than_idle_timeout() { + let idle_timeout = Duration::from_millis(100); + + let mut connection = Connection::new( + StreamMuxerBox::new(PendingStreamMuxer), + KeepAliveUntilConnectionHandler { + until: Instant::now() + idle_timeout * 2, + }, + None, + 0, + idle_timeout, + ); + + assert!(connection.poll_noop_waker().is_pending()); + + tokio::time::sleep(idle_timeout).await; + + assert!( + connection.poll_noop_waker().is_pending(), + "`KeepAlive::Until` is greater than idle-timeout, continue sleeping" + ); + + tokio::time::sleep(idle_timeout).await; + + assert!(matches!( + connection.poll_noop_waker(), + Poll::Ready(Err(ConnectionError::KeepAliveTimeout)) + )); + } + + #[tokio::test] + async fn idle_timeout_with_keep_alive_until_less_than_idle_timeout() { + let idle_timeout = Duration::from_millis(100); + + let mut connection = Connection::new( + StreamMuxerBox::new(PendingStreamMuxer), + KeepAliveUntilConnectionHandler { + until: Instant::now() + idle_timeout / 2, + }, + None, + 0, + idle_timeout, + ); + + assert!(connection.poll_noop_waker().is_pending()); + + tokio::time::sleep(idle_timeout / 2).await; + + assert!( + connection.poll_noop_waker().is_pending(), + "`KeepAlive::Until` is less than idle-timeout, honor idle-timeout" + ); + + tokio::time::sleep(idle_timeout / 2).await; + + assert!(matches!( + connection.poll_noop_waker(), + Poll::Ready(Err(ConnectionError::KeepAliveTimeout)) + )); + } + + struct KeepAliveUntilConnectionHandler { + until: Instant, + } + + impl ConnectionHandler for KeepAliveUntilConnectionHandler { + type FromBehaviour = Void; + type ToBehaviour = Void; + type Error = Void; + type InboundProtocol = DeniedUpgrade; + type OutboundProtocol = DeniedUpgrade; + type InboundOpenInfo = (); + type OutboundOpenInfo = Void; + + fn listen_protocol( + &self, + ) -> SubstreamProtocol { + SubstreamProtocol::new(DeniedUpgrade, ()) + } + + fn connection_keep_alive(&self) -> KeepAlive { + KeepAlive::Until(self.until) + } + + fn poll( + &mut self, + _: &mut Context<'_>, + ) -> Poll< + ConnectionHandlerEvent< + Self::OutboundProtocol, + Self::OutboundOpenInfo, + Self::ToBehaviour, + Self::Error, + >, + > { + Poll::Pending + } + + fn on_behaviour_event(&mut self, _: Self::FromBehaviour) {} + + fn on_connection_event( + &mut self, + _: ConnectionEvent< + Self::InboundProtocol, + Self::OutboundProtocol, + Self::InboundOpenInfo, + Self::OutboundOpenInfo, + >, + ) { + } + } + struct DummyStreamMuxer { counter: Arc<()>, } diff --git a/swarm/src/connection/pool.rs b/swarm/src/connection/pool.rs index e9f7504f529..07fc9075806 100644 --- a/swarm/src/connection/pool.rs +++ b/swarm/src/connection/pool.rs @@ -37,7 +37,7 @@ use futures::{ ready, stream::FuturesUnordered, }; -use instant::Instant; +use instant::{Duration, Instant}; use libp2p_core::connection::Endpoint; use libp2p_core::muxing::{StreamMuxerBox, StreamMuxerExt}; use std::task::Waker; @@ -135,6 +135,9 @@ where /// Receivers for [`NewConnection`] objects that are dropped. new_connection_dropped_listeners: FuturesUnordered>, + + /// How long a connection should be kept alive once it starts idling. + idle_connection_timeout: Duration, } #[derive(Debug)] @@ -322,6 +325,7 @@ where substream_upgrade_protocol_override: config.substream_upgrade_protocol_override, max_negotiating_inbound_streams: config.max_negotiating_inbound_streams, per_connection_event_buffer_size: config.per_connection_event_buffer_size, + idle_connection_timeout: config.idle_connection_timeout, executor, pending_connection_events_tx, pending_connection_events_rx, @@ -518,6 +522,7 @@ where handler, self.substream_upgrade_protocol_override, self.max_negotiating_inbound_streams, + self.idle_connection_timeout, ); self.executor.spawn(task::new_for_established_connection( @@ -947,6 +952,8 @@ pub(crate) struct PoolConfig { pub(crate) per_connection_event_buffer_size: usize, /// Number of addresses concurrently dialed for a single outbound connection attempt. pub(crate) dial_concurrency_factor: NonZeroU8, + /// How long a connection should be kept alive once it is idling. + pub(crate) idle_connection_timeout: Duration, /// The configured override for substream protocol upgrades, if any. substream_upgrade_protocol_override: Option, @@ -963,6 +970,7 @@ impl PoolConfig { task_command_buffer_size: 32, per_connection_event_buffer_size: 7, dial_concurrency_factor: NonZeroU8::new(8).expect("8 > 0"), + idle_connection_timeout: Duration::ZERO, substream_upgrade_protocol_override: None, max_negotiating_inbound_streams: 128, } diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 92bc614777e..93238d04da0 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -67,6 +67,9 @@ pub mod behaviour; pub mod dial_opts; pub mod dummy; pub mod handler; +#[deprecated( + note = "Configure an appropriate idle connection timeout via `SwarmBuilder::idle_connection_timeout` instead. To keep connections alive 'forever', use `Duration::from_secs(u64::MAX)`." +)] pub mod keep_alive; mod listen_opts; @@ -146,6 +149,7 @@ use libp2p_identity::PeerId; use smallvec::SmallVec; use std::collections::{HashMap, HashSet}; use std::num::{NonZeroU32, NonZeroU8, NonZeroUsize}; +use std::time::Duration; use std::{ convert::TryFrom, error, fmt, io, @@ -1518,6 +1522,14 @@ where self } + /// How long to keep a connection alive once it is idling. + /// + /// Defaults to 0. + pub fn idle_connection_timeout(mut self, timeout: Duration) -> Self { + self.pool_config.idle_connection_timeout = timeout; + self + } + /// Builds a `Swarm` with the current configuration. pub fn build(self) -> Swarm { log::info!("Local peer id: {}", self.local_peer_id); @@ -1808,6 +1820,7 @@ fn p2p_addr(peer: Option, addr: Multiaddr) -> Result( - handler_proto: T, - ) -> SwarmBuilder>> - where - T: ConnectionHandler + Clone, - T::ToBehaviour: Clone, - O: Send + 'static, - { + fn new_test_swarm( + ) -> SwarmBuilder>> { let id_keys = identity::Keypair::generate_ed25519(); let local_public_key = id_keys.public(); let transport = transport::MemoryTransport::default() @@ -1846,13 +1853,15 @@ mod tests { }) .multiplex(yamux::Config::default()) .boxed(); - let behaviour = CallTraceBehaviour::new(MockBehaviour::new(handler_proto)); - match ThreadPool::new().ok() { + let behaviour = CallTraceBehaviour::new(MockBehaviour::new(dummy::ConnectionHandler)); + let builder = match ThreadPool::new().ok() { Some(tp) => { SwarmBuilder::with_executor(transport, behaviour, local_public_key.into(), tp) } None => SwarmBuilder::without_executor(transport, behaviour, local_public_key.into()), - } + }; + + builder.idle_connection_timeout(Duration::from_secs(5)) } fn swarms_connected( @@ -1903,12 +1912,8 @@ mod tests { /// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`] #[test] fn test_swarm_disconnect() { - // Since the test does not try to open any substreams, we can - // use the dummy protocols handler. - let handler_proto = keep_alive::ConnectionHandler; - - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); + let mut swarm1 = new_test_swarm().build(); + let mut swarm2 = new_test_swarm().build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -1969,12 +1974,8 @@ mod tests { /// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`] #[test] fn test_behaviour_disconnect_all() { - // Since the test does not try to open any substreams, we can - // use the dummy protocols handler. - let handler_proto = keep_alive::ConnectionHandler; - - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); + let mut swarm1 = new_test_swarm().build(); + let mut swarm2 = new_test_swarm().build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -2039,12 +2040,8 @@ mod tests { /// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`] #[test] fn test_behaviour_disconnect_one() { - // Since the test does not try to open any substreams, we can - // use the dummy protocols handler. - let handler_proto = keep_alive::ConnectionHandler; - - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); + let mut swarm1 = new_test_swarm().build(); + let mut swarm2 = new_test_swarm().build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -2122,7 +2119,7 @@ mod tests { fn prop(concurrency_factor: DialConcurrencyFactor) { block_on(async { - let mut swarm = new_test_swarm::<_, ()>(keep_alive::ConnectionHandler) + let mut swarm = new_test_swarm() .dial_concurrency_factor(concurrency_factor.0) .build(); @@ -2190,8 +2187,8 @@ mod tests { // Checks whether dialing an address containing the wrong peer id raises an error // for the expected peer id instead of the obtained peer id. - let mut swarm1 = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); - let mut swarm2 = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); + let mut swarm1 = new_test_swarm().build(); + let mut swarm2 = new_test_swarm().build(); swarm1.listen_on("/memory/0".parse().unwrap()).unwrap(); @@ -2250,7 +2247,7 @@ mod tests { // // The last two can happen in any order. - let mut swarm = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); + let mut swarm = new_test_swarm().build(); swarm.listen_on("/memory/0".parse().unwrap()).unwrap(); let local_address = @@ -2310,7 +2307,7 @@ mod tests { fn dial_self_by_id() { // Trying to dial self by passing the same `PeerId` shouldn't even be possible in the first // place. - let swarm = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); + let swarm = new_test_swarm().build(); let peer_id = *swarm.local_peer_id(); assert!(!swarm.is_connected(&peer_id)); } @@ -2321,7 +2318,7 @@ mod tests { let target = PeerId::random(); - let mut swarm = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); + let mut swarm = new_test_swarm().build(); let addresses = HashSet::from([ multiaddr![Ip4([0, 0, 0, 0]), Tcp(rand::random::())], @@ -2367,8 +2364,8 @@ mod tests { fn aborting_pending_connection_surfaces_error() { let _ = env_logger::try_init(); - let mut dialer = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); - let mut listener = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build(); + let mut dialer = new_test_swarm().build(); + let mut listener = new_test_swarm().build(); let listener_peer_id = *listener.local_peer_id(); listener.listen_on(multiaddr![Memory(0u64)]).unwrap(); diff --git a/transports/pnet/tests/smoke.rs b/transports/pnet/tests/smoke.rs index a7635c00ca3..5e02ed856c6 100644 --- a/transports/pnet/tests/smoke.rs +++ b/transports/pnet/tests/smoke.rs @@ -6,7 +6,7 @@ use libp2p_core::upgrade::Version; use libp2p_core::Transport; use libp2p_core::{multiaddr::Protocol, Multiaddr}; use libp2p_pnet::{PnetConfig, PreSharedKey}; -use libp2p_swarm::{keep_alive, NetworkBehaviour, Swarm, SwarmBuilder, SwarmEvent}; +use libp2p_swarm::{dummy, NetworkBehaviour, Swarm, SwarmBuilder, SwarmEvent}; const TIMEOUT: Duration = Duration::from_secs(5); @@ -98,7 +98,7 @@ where assert_eq!(&outbound_peer_id, swarm1.local_peer_id()); } -fn make_swarm(transport: T, pnet: PnetConfig) -> Swarm +fn make_swarm(transport: T, pnet: PnetConfig) -> Swarm where T: Transport + Send + Unpin + 'static, ::Error: Send + Sync + 'static, @@ -113,12 +113,9 @@ where .authenticate(libp2p_noise::Config::new(&identity).unwrap()) .multiplex(libp2p_yamux::Config::default()) .boxed(); - SwarmBuilder::with_tokio_executor( - transport, - keep_alive::Behaviour, - identity.public().to_peer_id(), - ) - .build() + SwarmBuilder::with_tokio_executor(transport, dummy::Behaviour, identity.public().to_peer_id()) + .idle_connection_timeout(Duration::from_secs(5)) + .build() } async fn listen_on(swarm: &mut Swarm, addr: Multiaddr) -> Multiaddr { diff --git a/transports/tls/tests/smoke.rs b/transports/tls/tests/smoke.rs index 17aa959c4b2..0db39edf280 100644 --- a/transports/tls/tests/smoke.rs +++ b/transports/tls/tests/smoke.rs @@ -3,7 +3,8 @@ use libp2p_core::multiaddr::Protocol; use libp2p_core::transport::MemoryTransport; use libp2p_core::upgrade::Version; use libp2p_core::Transport; -use libp2p_swarm::{keep_alive, Swarm, SwarmBuilder, SwarmEvent}; +use libp2p_swarm::{dummy, Swarm, SwarmBuilder, SwarmEvent}; +use std::time::Duration; #[tokio::test] async fn can_establish_connection() { @@ -55,7 +56,7 @@ async fn can_establish_connection() { assert_eq!(&outbound_peer_id, swarm1.local_peer_id()); } -fn make_swarm() -> Swarm { +fn make_swarm() -> Swarm { let identity = libp2p_identity::Keypair::generate_ed25519(); let transport = MemoryTransport::default() @@ -64,10 +65,7 @@ fn make_swarm() -> Swarm { .multiplex(libp2p_yamux::Config::default()) .boxed(); - SwarmBuilder::without_executor( - transport, - keep_alive::Behaviour, - identity.public().to_peer_id(), - ) - .build() + SwarmBuilder::without_executor(transport, dummy::Behaviour, identity.public().to_peer_id()) + .idle_connection_timeout(Duration::from_secs(5)) + .build() } diff --git a/transports/webrtc/examples/listen_ping.rs b/transports/webrtc/examples/listen_ping.rs index 8475195a1ce..ad867f26db0 100644 --- a/transports/webrtc/examples/listen_ping.rs +++ b/transports/webrtc/examples/listen_ping.rs @@ -4,8 +4,9 @@ use libp2p_core::muxing::StreamMuxerBox; use libp2p_core::Transport; use libp2p_identity as identity; use libp2p_ping as ping; -use libp2p_swarm::{keep_alive, NetworkBehaviour, Swarm, SwarmBuilder}; +use libp2p_swarm::{Swarm, SwarmBuilder}; use rand::thread_rng; +use std::time::Duration; use void::Void; /// An example WebRTC server that will accept connections and run the ping protocol on them. @@ -21,7 +22,7 @@ async fn main() -> Result<()> { } } -fn create_swarm() -> Result> { +fn create_swarm() -> Result> { let id_keys = identity::Keypair::generate_ed25519(); let peer_id = id_keys.public().to_peer_id(); let transport = libp2p_webrtc::tokio::Transport::new( @@ -29,18 +30,16 @@ fn create_swarm() -> Result> { libp2p_webrtc::tokio::Certificate::generate(&mut thread_rng())?, ); + let cfg = ping::Config::new().with_interval(Duration::from_millis(10)); let transport = transport .map(|(peer_id, conn), _| (peer_id, StreamMuxerBox::new(conn))) .boxed(); - Ok(SwarmBuilder::with_tokio_executor(transport, Behaviour::default(), peer_id).build()) -} - -#[derive(NetworkBehaviour, Default)] -#[behaviour(to_swarm = "Event", prelude = "libp2p_swarm::derive_prelude")] -struct Behaviour { - ping: ping::Behaviour, - keep_alive: keep_alive::Behaviour, + Ok( + SwarmBuilder::with_tokio_executor(transport, ping::Behaviour::new(cfg.clone()), peer_id) + .idle_connection_timeout(Duration::from_secs(5)) + .build(), + ) } #[derive(Debug)] From 9b3d48cf6acde1d511d5060041a9418544128011 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Wed, 20 Sep 2023 11:32:42 +1000 Subject: [PATCH 83/83] Fix formatting --- misc/futures-bounded/src/lib.rs | 4 ++-- protocols/relay/src/priv_client/handler.rs | 5 ++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/misc/futures-bounded/src/lib.rs b/misc/futures-bounded/src/lib.rs index 091e854981d..e7b461dc822 100644 --- a/misc/futures-bounded/src/lib.rs +++ b/misc/futures-bounded/src/lib.rs @@ -1,8 +1,8 @@ -mod set; mod map; +mod set; -pub use set::FuturesSet; pub use map::{FuturesMap, PushError}; +pub use set::FuturesSet; use std::fmt; use std::fmt::Formatter; use std::time::Duration; diff --git a/protocols/relay/src/priv_client/handler.rs b/protocols/relay/src/priv_client/handler.rs index 12fc0fb5efa..25488ac3041 100644 --- a/protocols/relay/src/priv_client/handler.rs +++ b/protocols/relay/src/priv_client/handler.rs @@ -158,9 +158,8 @@ pub struct Handler { /// eventually. alive_lend_out_substreams: FuturesUnordered>, - open_circuit_futs: futures_bounded::FuturesSet< - Result, - >, + open_circuit_futs: + futures_bounded::FuturesSet>, circuit_deny_futs: futures_bounded::FuturesMap>,