From 8b72049890dc526695600376ee1f4a6352e5bb1a Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 17 Jun 2020 18:06:25 +0200 Subject: [PATCH 01/19] Emit events for active connection close and fix `disconnect()`. The `Network` does currently not emit events for actively closed connections, e.g. via `EstablishedConnection::close` or `ConnectedPeer::disconnect()`. As a result, when actively closing connections, there will be `ConnectionEstablished` events emitted without eventually a matching `ConnectionClosed` event. This seems undesirable and has the consequence that the `Swarm::ban_peer_id` feature in `libp2p-swarm` does not result in appropriate calls to `NetworkBehaviour::inject_connection_closed` and `NetworkBehaviour::inject_disconnected`. Furthermore, the `disconnect()` functionality in `libp2p-core` is currently broken as it leaves the `Pool` in an inconsistent state. This commit does the following: 1. When connection background tasks are dropped (i.e. removed from the `Manager`), they always terminate immediately, without attempting an orderly close of the connection. 2. An orderly close is sent to the background task of a connection as a regular command. The background task emits a `Closed` event before terminating. 3. `Pool::disconnect()` removes all connection tasks for the affected peer from the `Manager`, i.e. without an orderly close, thereby also fixing the discovered state inconsistency due to not removing the corresponding entries in the `Pool` itself after removing them from the `Manager`. 4. A new test is added to `libp2p-swarm` that exercises the ban/unban functionality and places assertions on the number and order of calls to the `NetworkBehaviour`. In that context some new testing utilities have been added to `libp2p-swarm`. This addresses https://github.com/libp2p/rust-libp2p/issues/1584. --- core/src/connection.rs | 2 +- core/src/connection/manager.rs | 62 ++++--- core/src/connection/manager/task.rs | 205 +++++++++++---------- core/src/connection/pool.rs | 177 ++++++++++++------- core/src/network.rs | 6 +- core/src/network/event.rs | 32 ++-- core/src/network/peer.rs | 4 +- swarm/Cargo.toml | 1 + swarm/src/lib.rs | 196 +++++++++++++++++---- swarm/src/protocols_handler/dummy.rs | 14 +- swarm/src/test.rs | 254 +++++++++++++++++++++++++++ 11 files changed, 722 insertions(+), 231 deletions(-) create mode 100644 swarm/src/test.rs diff --git a/core/src/connection.rs b/core/src/connection.rs index deaa272a7e0..34c8921e604 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -31,7 +31,7 @@ pub use handler::{ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandl pub use listeners::{ListenerId, ListenersStream, ListenersEvent}; pub use manager::ConnectionId; pub use substream::{Substream, SubstreamEndpoint, Close}; -pub use pool::{EstablishedConnection, EstablishedConnectionIter, PendingConnection}; +pub use pool::{EstablishedConnection, EstablishedConnectionIter, PendingConnection, StartClose}; use crate::muxing::StreamMuxer; use crate::{Multiaddr, PeerId}; diff --git a/core/src/connection/manager.rs b/core/src/connection/manager.rs index 9dbb644a443..63ac13779e2 100644 --- a/core/src/connection/manager.rs +++ b/core/src/connection/manager.rs @@ -194,18 +194,19 @@ pub enum Event<'a, I, O, H, TE, HE, C> { handler: H }, - /// An established connection has encountered an error. - ConnectionError { + /// An established connection has been closed. + ConnectionClosed { /// The connection ID. /// - /// As a result of the error, the connection has been removed - /// from the `Manager` and is being closed. Hence this ID will - /// no longer resolve to a valid entry in the manager. + /// > **Note**: Closed connections are removed from the `Manager`. + /// > Hence this ID will no longer resolve to a valid entry in + /// > the manager. id: ConnectionId, - /// Information about the connection that encountered the error. + /// Information about the closed connection. connected: Connected, - /// The error that occurred. - error: ConnectionError, + /// The error that occurred, if any. If `None`, the connection + /// has been actively closed. + error: Option>, }, /// A connection has been established. @@ -336,11 +337,11 @@ impl Manager { /// Polls the manager for events relating to the managed connections. pub fn poll<'a>(&'a mut self, cx: &mut Context) -> Poll> { // Advance the content of `local_spawns`. - while let Poll::Ready(Some(_)) = Stream::poll_next(Pin::new(&mut self.local_spawns), cx) {} + while let Poll::Ready(Some(_)) = self.local_spawns.poll_next_unpin(cx) {} // Poll for the first event for which the manager still has a registered task, if any. let event = loop { - match Stream::poll_next(Pin::new(&mut self.events_rx), cx) { + match self.events_rx.poll_next_unpin(cx) { Poll::Ready(Some(event)) => { if self.tasks.contains_key(event.id()) { // (1) break event @@ -369,18 +370,17 @@ impl Manager { let _ = task.remove(); Event::PendingConnectionError { id, error, handler } } - task::Event::Error { id, error } => { + task::Event::Closed { id, error } => { let id = ConnectionId(id); let task = task.remove(); match task.state { TaskState::Established(connected) => - Event::ConnectionError { id, connected, error }, + Event::ConnectionClosed { id, connected, error }, TaskState::Pending => unreachable!( - "`Event::Error` implies (2) occurred on that task and thus (3)." + "`Event::Closed` implies (2) occurred on that task and thus (3)." ), } } - }) } else { unreachable!("By (1)") @@ -426,10 +426,11 @@ impl<'a, I, C> EstablishedEntry<'a, I, C> { /// > task _may not be notified_ if sending the event fails due to /// > the connection handler not being ready at this time. pub fn notify_handler(&mut self, event: I) -> Result<(), I> { - let cmd = task::Command::NotifyHandler(event); + let cmd = task::Command::NotifyHandler(event); // (*) self.task.get_mut().sender.try_send(cmd) .map_err(|e| match e.into_inner() { - task::Command::NotifyHandler(event) => event + task::Command::NotifyHandler(event) => event, + _ => unreachable!("by (*)") }) } @@ -443,6 +444,24 @@ impl<'a, I, C> EstablishedEntry<'a, I, C> { self.task.get_mut().sender.poll_ready(cx).map_err(|_| ()) } + /// Tries to send a close command to the associated background task, + /// thus initiating a graceful active close of the connection. + /// + /// When the connection is ultimately closed, [`Event::ConnectionClosed`] + /// is emitted by [`Manager::poll`]. + pub fn poll_start_close(&mut self, cx: &mut Context) -> Poll<()> { + match self.task.get_mut().sender.poll_ready(cx) { + Poll::Ready(result) => { + if result.is_ok() { + // If it fails now then the task is already gone. + let _ = self.task.get_mut().sender.try_send(task::Command::Close); + } + Poll::Ready(()) + } + Poll::Pending => Poll::Pending + } + } + /// Obtains information about the established connection. pub fn connected(&self) -> &Connected { match &self.task.get().state { @@ -451,16 +470,18 @@ impl<'a, I, C> EstablishedEntry<'a, I, C> { } } - /// Closes the connection represented by this entry, - /// returning the connection information. - pub fn close(self) -> Connected { + /// Instantly removes the entry from the manager, dropping + /// the command channel to the background task of the connection, + /// which will thus drop the connection asap without an orderly + /// close or emitting another event. + pub fn remove(self) -> Connected { match self.task.remove().state { TaskState::Established(c) => c, TaskState::Pending => unreachable!("By Entry::new()") } } - /// Returns the connection id. + /// Returns the connection ID. pub fn id(&self) -> ConnectionId { ConnectionId(*self.task.key()) } @@ -484,3 +505,4 @@ impl<'a, I, C> PendingEntry<'a, I, C> { self.task.remove(); } } + diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index 4272722d5a6..4b088d3d0c3 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -44,6 +44,9 @@ pub struct TaskId(pub(super) usize); pub enum Command { /// Notify the connection handler of an event. NotifyHandler(T), + /// Gracefully close the connection (active close) before + /// terminating the task. + Close, } /// Events that a task can emit to its manager. @@ -51,21 +54,24 @@ pub enum Command { pub enum Event { /// A connection to a node has succeeded. Established { id: TaskId, info: Connected }, - /// An established connection produced an error. - Error { id: TaskId, error: ConnectionError }, /// A pending connection failed. Failed { id: TaskId, error: PendingConnectionError, handler: H }, /// Notify the manager of an event from the connection. - Notify { id: TaskId, event: T } + Notify { id: TaskId, event: T }, + /// A connection closed, possibly due to an error. + /// + /// If `error` is `None`, the connection has completed + /// an active orderly close. + Closed { id: TaskId, error: Option> } } impl Event { pub fn id(&self) -> &TaskId { match self { Event::Established { id, .. } => id, - Event::Error { id, .. } => id, Event::Notify { id, .. } => id, Event::Failed { id, .. } => id, + Event::Closed { id, .. } => id, } } } @@ -126,7 +132,7 @@ where id, events, commands: commands.fuse(), - state: State::EstablishedPending(connection), + state: State::Established { connection, event: None }, } } } @@ -138,7 +144,7 @@ where H: IntoConnectionHandler, H::Handler: ConnectionHandler> { - /// The task is waiting for the connection to be established. + /// The connection is being negotiated. Pending { /// The future that will attempt to reach the node. // TODO: don't pin this Future; this requires deeper changes though @@ -147,20 +153,22 @@ where handler: H, }, - /// The connection is established and a new event is ready to be emitted. - EstablishedReady { - /// The node, if available. - connection: Option>, - /// The actual event message to send. - event: Event::Error, C> + /// The connection is established. + Established { + connection: Connection, + /// An event to send to the `Manager`. If `None`, the `connection` + /// is polled for new events in this state, otherwise the event + /// must be sent to the `Manager` before the connection can be + /// polled again. + event: Option::Error, C>> }, - /// The connection is established and pending a new event to occur. - EstablishedPending(Connection), - - /// The task is closing the connection. + /// The connection is closing (active close). Closing(Close), + /// The task is terminating with a final event for the `Manager`. + Terminating(Event::Error, C>), + /// The task has finished. Done } @@ -192,24 +200,27 @@ where 'poll: loop { match std::mem::replace(&mut this.state, State::Done) { State::Pending { mut future, handler } => { - // Check if the manager aborted this task by dropping the `commands` - // channel sender side. - match Stream::poll_next(Pin::new(&mut this.commands), cx) { + // Check whether the task is still registered with a `Manager` + // by polling the commands channel. + match this.commands.poll_next_unpin(cx) { Poll::Pending => {}, - Poll::Ready(None) => return Poll::Ready(()), - Poll::Ready(Some(Command::NotifyHandler(_))) => unreachable!( - "Manager does not allow sending commands to pending tasks.", + Poll::Ready(None) => { + // The manager has dropped the task; abort. + return Poll::Ready(()) + } + Poll::Ready(Some(_)) => panic!( + "Task received command while the connection is pending." ) } // Check if the connection succeeded. - match Future::poll(Pin::new(&mut future), cx) { + match future.poll_unpin(cx) { Poll::Ready(Ok((info, muxer))) => { - this.state = State::EstablishedReady { - connection: Some(Connection::new( + this.state = State::Established { + connection: Connection::new( muxer, handler.into_handler(&info), - )), - event: Event::Established { id, info } + ), + event: Some(Event::Established { id, info }) } } Poll::Pending => { @@ -218,112 +229,114 @@ where } Poll::Ready(Err(error)) => { let event = Event::Failed { id, handler, error }; - this.state = State::EstablishedReady { connection: None, event } + this.state = State::Terminating(event) } } } - State::EstablishedPending(mut connection) => { - // Start by handling commands received from the manager, if any. + State::Established { mut connection, event } => { + // Check for commands from the `Manager`. loop { - match Stream::poll_next(Pin::new(&mut this.commands), cx) { + match this.commands.poll_next_unpin(cx) { Poll::Pending => break, Poll::Ready(Some(Command::NotifyHandler(event))) => connection.inject_event(event), - Poll::Ready(None) => { - // The manager has dropped the task, thus initiate a - // graceful shutdown of the connection. + Poll::Ready(Some(Command::Close)) => { + // Discard the event, if any, and start a graceful close. this.state = State::Closing(connection.close()); continue 'poll } + Poll::Ready(None) => { + // The manager has dropped the task; abort. + return Poll::Ready(()) + } } } - // Poll the connection for new events. - loop { + + if let Some(event) = event { + // Send the event to the manager. + match this.events.poll_ready(cx) { + Poll::Pending => { + this.state = State::Established { connection, event: Some(event) }; + return Poll::Pending + } + Poll::Ready(result) => { + if result.is_ok() { + if let Ok(()) = this.events.start_send(event) { + this.state = State::Established { connection, event: None }; + continue 'poll + } + } + // The manager is no longer reachable, maybe due to + // application shutdown. Try a graceful shutdown of the + // connection before terminating the task. + this.state = State::Closing(connection.close()); + } + } + } else { + // Poll the connection for new events. match Connection::poll(Pin::new(&mut connection), cx) { Poll::Pending => { - this.state = State::EstablishedPending(connection); + this.state = State::Established { connection, event: None }; return Poll::Pending } Poll::Ready(Ok(event)) => { - this.state = State::EstablishedReady { - connection: Some(connection), - event: Event::Notify { id, event } + this.state = State::Established { + connection: connection, + event: Some(Event::Notify { id, event }) }; - continue 'poll } Poll::Ready(Err(error)) => { - // Notify the manager of the error via an event, - // dropping the connection. - let event = Event::Error { id, error }; - this.state = State::EstablishedReady { connection: None, event }; - continue 'poll + // Terminate the task with the error, dropping + // the connection. + let event = Event::Closed { id, error: Some(error) }; + this.state = State::Terminating(event); } } } } - // Deliver an event to the manager. - State::EstablishedReady { mut connection, event } => { - // Process commands received from the manager, if any. - loop { - match Stream::poll_next(Pin::new(&mut this.commands), cx) { - Poll::Pending => break, - Poll::Ready(Some(Command::NotifyHandler(event))) => - if let Some(ref mut c) = connection { - c.inject_event(event) - } - Poll::Ready(None) => - // The manager has dropped the task, thus initiate a - // graceful shutdown of the connection, if given. - if let Some(c) = connection { - this.state = State::Closing(c.close()); - continue 'poll - } else { - return Poll::Ready(()) - } + State::Closing(mut closing) => { + // Ignore further commands received from the manager, if any. + while let Poll::Ready(Some(_)) = this.commands.poll_next_unpin(cx) {} + // Try to gracefully close the connection. + match closing.poll_unpin(cx) { + Poll::Ready(Ok(())) => { + let event = Event::Closed { id: this.id, error: None }; + this.state = State::Terminating(event); + } + Poll::Ready(Err(e)) => { + let event = Event::Closed { + id: this.id, + error: Some(ConnectionError::IO(e)) + }; + this.state = State::Terminating(event); } - } - // Send the event to the manager. - match this.events.poll_ready(cx) { Poll::Pending => { - self.state = State::EstablishedReady { connection, event }; + this.state = State::Closing(closing); return Poll::Pending } - Poll::Ready(Ok(())) => { - // We assume that if `poll_ready` has succeeded, then sending the event - // will succeed as well. If it turns out that it didn't, we will detect - // the closing at the next loop iteration. - let _ = this.events.start_send(event); - if let Some(c) = connection { - this.state = State::EstablishedPending(c) - } else { - // The connection has been dropped, thus this was the last event - // to send to the manager and the task is done. - return Poll::Ready(()) - } - }, - Poll::Ready(Err(_)) => { - // The manager is no longer reachable, maybe due to - // application shutdown. Try a graceful shutdown of the - // connection, if available, and end the task. - if let Some(c) = connection { - this.state = State::Closing(c.close()); - continue 'poll - } - return Poll::Ready(()) - } } } - State::Closing(mut closing) => - match Future::poll(Pin::new(&mut closing), cx) { - Poll::Ready(_) => return Poll::Ready(()), // end task + State::Terminating(event) => { + // Ignore further commands received from the manager, if any. + while let Poll::Ready(Some(_)) = this.commands.poll_next_unpin(cx) + {} + // Try to deliver the final event. + match this.events.poll_ready(cx) { Poll::Pending => { - this.state = State::Closing(closing); + self.state = State::Terminating(event); return Poll::Pending } + Poll::Ready(result) => { + if result.is_ok() { + let _ = this.events.start_send(event); + } + return Poll::Ready(()) + } } + } State::Done => panic!("`Task::poll()` called after completion.") } diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index b319ca7647a..ca4b0f9c210 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -64,6 +64,12 @@ pub struct Pool)>, + + /// Established connections that have been closed in the context of + /// a [`Pool::disconnect`] in order to emit a `ConnectionClosed` + /// event for each. Every `ConnectionEstablished` event must be + /// paired with (eventually) a `ConnectionClosed`. + disconnected: Vec>, } impl fmt::Debug @@ -84,17 +90,28 @@ for Pool { /// A new connection has been established. ConnectionEstablished { - connection: EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, + connection: EstablishedConnection<'a, TInEvent, TConnInfo>, num_established: NonZeroU32, }, - /// An established connection has encountered an error. - ConnectionError { + /// An established connection was closed. + /// + /// A connection may close if + /// + /// * it encounters an error, which includes the connection being + /// closed by the remote. In this case `error` is `Some`. + /// * it was actively closed by [`EstablishedConnection::close`], + /// i.e. a successful, orderly close. + /// * it was actively closed by [`Pool::disconnect`], i.e. + /// dropped without an orderly close. + /// + ConnectionClosed { id: ConnectionId, /// Information about the connection that errored. connected: Connected, - /// The error that occurred. - error: ConnectionError, + /// The error that occurred, if any. If `None`, the connection + /// was closed by the local peer. + error: Option>, /// A reference to the pool that used to manage the connection. pool: &'a mut Pool, /// The remaining number of established connections to the same peer. @@ -121,7 +138,7 @@ pub enum PoolEvent<'a, TInEvent, TOutEvent, THandler, TTransErr, THandlerErr, TC /// A node has produced an event. ConnectionEvent { /// The connection that has generated the event. - connection: EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, + connection: EstablishedConnection<'a, TInEvent, TConnInfo>, /// The produced event. event: TOutEvent, }, @@ -143,8 +160,8 @@ where .field(connection) .finish() }, - PoolEvent::ConnectionError { ref id, ref connected, ref error, .. } => { - f.debug_struct("PoolEvent::ConnectionError") + PoolEvent::ConnectionClosed { ref id, ref connected, ref error, .. } => { + f.debug_struct("PoolEvent::ConnectionClosed") .field("id", id) .field("connected", connected) .field("error", error) @@ -183,6 +200,7 @@ where manager: Manager::new(manager_config), established: Default::default(), pending: Default::default(), + disconnected: Vec::new(), } } @@ -375,8 +393,7 @@ where match self.manager.entry(id) { Some(manager::Entry::Established(entry)) => Some(PoolConnection::Established(EstablishedConnection { - entry, - established: &mut self.established, + entry })), Some(manager::Entry::Pending(entry)) => Some(PoolConnection::Pending(PendingConnection { @@ -389,7 +406,7 @@ where /// Gets an established connection from the pool by ID. pub fn get_established(&mut self, id: ConnectionId) - -> Option> + -> Option> { match self.get(id) { Some(PoolConnection::Established(c)) => Some(c), @@ -428,25 +445,49 @@ where self.established.len() } - /// Close all connections to the given peer. + /// (Forcefully) close all connections to the given peer. + /// + /// All connections to the peer, whether pending or established are + /// dropped asap and no more events from these connections are emitted + /// by the pool effective immediately. + /// + /// > **Note**: Established connections are dropped without performing + /// > an orderly close. See [`EstablishedConnection::close`] for + /// > performing such an orderly close. pub fn disconnect(&mut self, peer: &TPeerId) { if let Some(conns) = self.established.get(peer) { - for id in conns.keys() { - match self.manager.entry(*id) { - Some(manager::Entry::Established(e)) => { e.close(); }, + // Count upwards because we push to / pop from the end. See also `Pool::poll`. + let mut num_established = 0; + for &id in conns.keys() { + match self.manager.entry(id) { + Some(manager::Entry::Established(e)) => { + let connected = e.remove(); + self.disconnected.push(Disconnected { + id, connected, num_established + }); + num_established += 1; + }, _ => {} } } } + self.established.remove(peer); - for (id, (_endpoint, peer2)) in &self.pending { + let mut aborted = Vec::new(); + for (&id, (_endpoint, peer2)) in &self.pending { if Some(peer) == peer2.as_ref() { - match self.manager.entry(*id) { - Some(manager::Entry::Pending(e)) => { e.abort(); }, + match self.manager.entry(id) { + Some(manager::Entry::Pending(e)) => { + e.abort(); + aborted.push(id); + }, _ => {} } } } + for id in aborted { + self.pending.remove(&id); + } } /// Counts the number of established connections in the pool. @@ -551,6 +592,20 @@ where TConnInfo: ConnectionInfo + Clone, TPeerId: Clone { + // Drain events resulting from forced disconnections. + while let Some(Disconnected { + id, connected, num_established + }) = self.disconnected.pop() { + return Poll::Ready(PoolEvent::ConnectionClosed { + id, + connected, + num_established, + error: None, + pool: self, + }) + } + + // Poll the connection `Manager`. loop { let item = match self.manager.poll(cx) { Poll::Ready(item) => item, @@ -570,7 +625,7 @@ where }) } }, - manager::Event::ConnectionError { id, connected, error } => { + manager::Event::ConnectionClosed { id, connected, error } => { let num_established = if let Some(conns) = self.established.get_mut(connected.peer_id()) { conns.remove(&id); @@ -581,10 +636,10 @@ where if num_established == 0 { self.established.remove(connected.peer_id()); } - return Poll::Ready(PoolEvent::ConnectionError { + return Poll::Ready(PoolEvent::ConnectionClosed { id, connected, error, num_established, pool: self }) - }, + } manager::Event::ConnectionEstablished { entry } => { let id = entry.id(); if let Some((endpoint, peer)) = self.pending.remove(&id) { @@ -593,7 +648,7 @@ where let current = || established.get(entry.connected().peer_id()) .map_or(0, |conns| conns.len()); if let Err(e) = self.limits.check_established(current) { - let connected = entry.close(); + let connected = entry.remove(); return Poll::Ready(PoolEvent::PendingConnectionError { id, endpoint: connected.endpoint, @@ -649,7 +704,7 @@ where /// A connection in a [`Pool`]. pub enum PoolConnection<'a, TInEvent, TConnInfo, TPeerId> { Pending(PendingConnection<'a, TInEvent, TConnInfo, TPeerId>), - Established(EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>), + Established(EstablishedConnection<'a, TInEvent, TConnInfo>), } /// A pending connection in a [`Pool`]. @@ -684,13 +739,12 @@ impl } /// An established connection in a [`Pool`]. -pub struct EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId> { +pub struct EstablishedConnection<'a, TInEvent, TConnInfo> { entry: manager::EstablishedEntry<'a, TInEvent, TConnInfo>, - established: &'a mut FnvHashMap>, } -impl fmt::Debug -for EstablishedConnection<'_, TInEvent, TConnInfo, TPeerId> +impl fmt::Debug +for EstablishedConnection<'_, TInEvent, TConnInfo> where TInEvent: fmt::Debug, TConnInfo: fmt::Debug, @@ -702,9 +756,7 @@ where } } -impl - EstablishedConnection<'_, TInEvent, TConnInfo, TPeerId> -{ +impl EstablishedConnection<'_, TInEvent, TConnInfo> { pub fn connected(&self) -> &Connected { self.entry.connected() } @@ -720,11 +772,9 @@ impl } } -impl - EstablishedConnection<'_, TInEvent, TConnInfo, TPeerId> +impl<'a, TInEvent, TConnInfo> EstablishedConnection<'a, TInEvent, TConnInfo> where - TConnInfo: ConnectionInfo, - TPeerId: Eq + Hash + Clone, + TConnInfo: ConnectionInfo, { /// Returns the local connection ID. pub fn id(&self) -> ConnectionId { @@ -732,7 +782,7 @@ where } /// Returns the identity of the connected peer. - pub fn peer_id(&self) -> &TPeerId { + pub fn peer_id(&self) -> &TConnInfo::PeerId { self.info().peer_id() } @@ -760,24 +810,9 @@ where self.entry.poll_ready_notify_handler(cx) } - /// Closes the connection, returning the connection information. - pub fn close(self) -> Connected { - let id = self.entry.id(); - let info = self.entry.close(); - - let empty = - if let Some(conns) = self.established.get_mut(info.peer_id()) { - conns.remove(&id); - conns.is_empty() - } else { - false - }; - - if empty { - self.established.remove(info.peer_id()); - } - - info + /// Initiates a graceful close of the connection. + pub fn close(self) -> StartClose<'a, TInEvent, TConnInfo> { + StartClose(self.entry) } } @@ -796,14 +831,13 @@ where I: Iterator { /// Obtains the next connection, if any. - pub fn next<'b>(&'b mut self) -> Option> + pub fn next<'b>(&'b mut self) -> Option> { while let Some(id) = self.ids.next() { if self.pool.manager.is_established(&id) { // (*) match self.pool.manager.entry(id) { Some(manager::Entry::Established(entry)) => { - let established = &mut self.pool.established; - return Some(EstablishedConnection { entry, established }) + return Some(EstablishedConnection { entry }) } _ => unreachable!("by (*)") } @@ -819,15 +853,14 @@ where /// Returns the first connection, if any, consuming the iterator. pub fn into_first<'b>(mut self) - -> Option> + -> Option> where 'a: 'b { while let Some(id) = self.ids.next() { if self.pool.manager.is_established(&id) { // (*) match self.pool.manager.entry(id) { Some(manager::Entry::Established(entry)) => { - let established = &mut self.pool.established; - return Some(EstablishedConnection { entry, established }) + return Some(EstablishedConnection { entry }) } _ => unreachable!("by (*)") } @@ -888,3 +921,29 @@ impl PoolLimits { Ok(()) } } + +/// A `StartClose` future resolves when the command to +/// close has been enqueued for the background task associated +/// with a connection. +/// +/// When the connection is ultimately closed, +/// [`crate::network::NetworkEvent::ConnectionClosed`] +/// is emitted with no `error` on success. +#[derive(Debug)] +pub struct StartClose<'a, TInEvent, TConnInfo>( + manager::EstablishedEntry<'a, TInEvent, TConnInfo>, +); + +impl<'a, TInEvent, TConnInfo> Future for StartClose<'a, TInEvent, TConnInfo> { + type Output = (); + + fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context) -> Poll<()> { + self.0.poll_start_close(cx) + } +} + +struct Disconnected { + id: ConnectionId, + connected: Connected, + num_established: u32, +} diff --git a/core/src/network.rs b/core/src/network.rs index 1e89dcd7cc3..93d2250af47 100644 --- a/core/src/network.rs +++ b/core/src/network.rs @@ -405,12 +405,12 @@ where } event } - Poll::Ready(PoolEvent::ConnectionError { id, connected, error, num_established, .. }) => { - NetworkEvent::ConnectionError { + Poll::Ready(PoolEvent::ConnectionClosed { id, connected, error, num_established, .. }) => { + NetworkEvent::ConnectionClosed { id, connected, - error, num_established, + error, } } Poll::Ready(PoolEvent::ConnectionEvent { connection, event }) => { diff --git a/core/src/network/event.rs b/core/src/network/event.rs index a63dc47946f..84b3f49ce05 100644 --- a/core/src/network/event.rs +++ b/core/src/network/event.rs @@ -101,25 +101,34 @@ where error: PendingConnectionError, }, - /// A new connection to a peer has been opened. + /// A new connection to a peer has been established. ConnectionEstablished { /// The newly established connection. - connection: EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, - /// The total number of established connections to the same peer, including the one that - /// has just been opened. + connection: EstablishedConnection<'a, TInEvent, TConnInfo>, + /// The total number of established connections to the same peer, + /// including the one that has just been opened. num_established: NonZeroU32, }, - /// An established connection to a peer has encountered an error. + /// An established connection to a peer has been closed. /// - /// The connection is closed as a result of the error. - ConnectionError { + /// A connection may close if + /// + /// * it encounters an error, which includes the connection being + /// closed by the remote. In this case `error` is `Some`. + /// * it was actively closed by [`EstablishedConnection::close`], + /// i.e. a successful, orderly close. In this case `error` is `None`. + /// * it was actively closed by [`super::peer::ConnectedPeer::disconnect`] or + /// [`super::peer::DialingPeer::disconnect`], i.e. dropped without an + /// orderly close. In this case `error` is `None`. + /// + ConnectionClosed { /// The ID of the connection that encountered an error. id: ConnectionId, /// Information about the connection that encountered the error. connected: Connected, /// The error that occurred. - error: ConnectionError<::Error>, + error: Option::Error>>, /// The remaining number of established connections to the same peer. num_established: u32, }, @@ -151,7 +160,7 @@ where /// An established connection produced an event. ConnectionEvent { /// The connection on which the event occurred. - connection: EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, + connection: EstablishedConnection<'a, TInEvent, TConnInfo>, /// Event that was produced by the node. event: TOutEvent, }, @@ -214,8 +223,9 @@ where .field("connection", connection) .finish() } - NetworkEvent::ConnectionError { connected, error, .. } => { - f.debug_struct("ConnectionError") + NetworkEvent::ConnectionClosed { id, connected, error, .. } => { + f.debug_struct("ConnectionClosed") + .field("id", id) .field("connected", connected) .field("error", error) .finish() diff --git a/core/src/network/peer.rs b/core/src/network/peer.rs index 8f9dd0991cc..2966404759a 100644 --- a/core/src/network/peer.rs +++ b/core/src/network/peer.rs @@ -313,7 +313,7 @@ where /// Obtains an established connection to the peer by ID. pub fn connection<'b>(&'b mut self, id: ConnectionId) - -> Option> + -> Option> { self.network.pool.get_established(id) } @@ -359,7 +359,7 @@ where /// Obtains some established connection to the peer. pub fn some_connection<'b>(&'b mut self) - -> EstablishedConnection<'b, TInEvent, TConnInfo, TPeerId> + -> EstablishedConnection<'b, TInEvent, TConnInfo> { self.connections() .into_first() diff --git a/swarm/Cargo.toml b/swarm/Cargo.toml index 535a620911f..02f3fdaba6b 100644 --- a/swarm/Cargo.toml +++ b/swarm/Cargo.toml @@ -20,5 +20,6 @@ void = "1" [dev-dependencies] libp2p-mplex = { path = "../muxers/mplex" } +libp2p-secio = { path = "../protocols/secio" } quickcheck = "0.9.0" rand = "0.7.2" diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index f848f64f9d3..8dbe6dc20a3 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -55,6 +55,8 @@ mod behaviour; mod registry; +#[cfg(test)] +mod test; mod upgrade; pub mod protocols_handler; @@ -156,7 +158,8 @@ pub enum SwarmEvent { /// opened. num_established: NonZeroU32, }, - /// A connection with the given peer has been closed. + /// A connection with the given peer has been closed, + /// possibly as a result of an error. ConnectionClosed { /// Identity of the peer that we have connected to. peer_id: PeerId, @@ -164,8 +167,9 @@ pub enum SwarmEvent { endpoint: ConnectedPoint, /// Number of other remaining connections to this same peer. num_established: u32, - /// Reason for the disconnection. - cause: ConnectionError>, + /// Reason for the disconnection, if it was not a successful + /// active close. + cause: Option>>, }, /// A new connection arrived on a listener and is in the process of protocol negotiation. /// @@ -366,22 +370,19 @@ where TBehaviour: NetworkBehaviour, me.network.remove_listener(id) } - /// Tries to dial the given address. - /// - /// Returns an error if the address is not supported. + /// Initiates a new dialing attempt to the given address. pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), ConnectionLimit> { let handler = me.behaviour.new_handler(); me.network.dial(&addr, handler.into_node_handler_builder()).map(|_id| ()) } - /// Tries to initiate a dialing attempt to the given peer. - /// - /// If a new dialing attempt has been initiated, `Ok(true)` is returned. - /// - /// If no new dialing attempt has been initiated, meaning there is an ongoing - /// dialing attempt or `addresses_of_peer` reports no addresses, `Ok(false)` - /// is returned. + /// Initiates a new dialing attempt to the given peer. pub fn dial(me: &mut Self, peer_id: &PeerId) -> Result<(), DialError> { + if me.banned_peers.contains(peer_id) { + me.behaviour.inject_dial_failure(peer_id); + return Err(DialError::Banned) + } + let mut addrs = me.behaviour.addresses_of_peer(peer_id).into_iter(); let peer = me.network.peer(peer_id.clone()); @@ -443,11 +444,12 @@ where TBehaviour: NetworkBehaviour, /// Bans a peer by its peer ID. /// /// Any incoming connection and any dialing attempt will immediately be rejected. - /// This function has no effect is the peer is already banned. + /// This function has no effect if the peer is already banned. pub fn ban_peer_id(me: &mut Self, peer_id: PeerId) { - me.banned_peers.insert(peer_id.clone()); - if let Some(c) = me.network.peer(peer_id).into_connected() { - c.disconnect(); + if me.banned_peers.insert(peer_id.clone()) { + if let Some(peer) = me.network.peer(peer_id).into_connected() { + peer.disconnect(); + } } } @@ -521,8 +523,12 @@ where TBehaviour: NetworkBehaviour, }); } }, - Poll::Ready(NetworkEvent::ConnectionError { id, connected, error, num_established }) => { - log::debug!("Connection {:?} closed: {:?}", connected, error); + Poll::Ready(NetworkEvent::ConnectionClosed { id, connected, error, num_established }) => { + if error.is_some() { + log::debug!("Connection {:?} closed: {:?}", connected, error); + } else { + log::debug!("Connection {:?} closed (active close).", connected); + } let info = connected.info; let endpoint = connected.endpoint; this.behaviour.inject_connection_closed(info.peer_id(), &id, &endpoint); @@ -765,14 +771,13 @@ enum PendingNotifyHandler { /// /// Returns `None` if the connection is closing or the event has been /// successfully sent, in either case the event is consumed. -fn notify_one<'a, TInEvent, TConnInfo, TPeerId>( - conn: &mut EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, +fn notify_one<'a, TInEvent, TConnInfo>( + conn: &mut EstablishedConnection<'a, TInEvent, TConnInfo>, event: TInEvent, cx: &mut Context, ) -> Option where - TPeerId: Eq + std::hash::Hash + Clone, - TConnInfo: ConnectionInfo + TConnInfo: ConnectionInfo { match conn.poll_ready_notify_handler(cx) { Poll::Pending => Some(event), @@ -1113,6 +1118,8 @@ where TBehaviour: NetworkBehaviour, /// The possible failures of [`ExpandedSwarm::dial`]. #[derive(Debug)] pub enum DialError { + /// The peer is currently banned. + Banned, /// The configured limit for simultaneous outgoing connections /// has been reached. ConnectionLimit(ConnectionLimit), @@ -1125,7 +1132,8 @@ impl fmt::Display for DialError { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { DialError::ConnectionLimit(err) => write!(f, "Dial error: {}", err), - DialError::NoAddresses => write!(f, "Dial error: no addresses for peer.") + DialError::NoAddresses => write!(f, "Dial error: no addresses for peer."), + DialError::Banned => write!(f, "Dial error: peer is banned.") } } } @@ -1134,7 +1142,8 @@ impl error::Error for DialError { fn source(&self) -> Option<&(dyn error::Error + 'static)> { match self { DialError::ConnectionLimit(err) => Some(err), - DialError::NoAddresses => None + DialError::NoAddresses => None, + DialError::Banned => None } } } @@ -1173,24 +1182,45 @@ impl NetworkBehaviour for DummyBehaviour { { Poll::Pending } - } #[cfg(test)] mod tests { - use crate::{DummyBehaviour, SwarmBuilder}; + use crate::protocols_handler::DummyProtocolsHandler; + use crate::test::{MockBehaviour, CallTraceBehaviour}; + use futures::{future, executor}; use libp2p_core::{ - PeerId, - PublicKey, identity, - transport::dummy::{DummyStream, DummyTransport} + upgrade, + multiaddr, + transport::{self, dummy::*} }; use libp2p_mplex::Multiplex; + use super::*; - fn get_random_id() -> PublicKey { + fn get_random_id() -> identity::PublicKey { identity::Keypair::generate_ed25519().public() } + fn new_test_swarm(handler_proto: T) -> Swarm>> + where + T: ProtocolsHandler + Clone, + T::OutEvent: Clone, + O: Send + 'static + { + let keypair1 = identity::Keypair::generate_ed25519(); + let pubkey1 = keypair1.public(); + let transport1 = transport::MemoryTransport::default() + .upgrade(upgrade::Version::V1) + .authenticate(libp2p_secio::SecioConfig::new(keypair1)) + .multiplex(libp2p_mplex::MplexConfig::new()) + .map(|(p, m), _| (p, StreamMuxerBox::new(m))) + .map_err(|e| -> io::Error { panic!("Failed to create transport: {:?}", e); }) + .boxed(); + let behaviour1 = CallTraceBehaviour::new(MockBehaviour::new(handler_proto)); + SwarmBuilder::new(transport1, behaviour1, pubkey1.into()).build() + } + #[test] fn test_build_swarm() { let id = get_random_id(); @@ -1209,4 +1239,108 @@ mod tests { let swarm = SwarmBuilder::new(transport, DummyBehaviour {}, id.into()).build(); assert!(swarm.network.incoming_limit().is_none()) } + + /// Establishes a number of connections between two peers, + /// after which one peer bans the other. + /// + /// The test expects both behaviours to be notified via pairs of + /// inject_connected / inject_disconnected as well as + /// inject_connection_established / inject_connection_closed calls. + #[test] + fn test_connect_disconnect_ban() { + // Since the test does not try to open any substreams, we can + // use the dummy protocols handler. + let mut handler_proto = DummyProtocolsHandler::default(); + handler_proto.keep_alive = KeepAlive::Yes; + + let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()); + let mut swarm2 = new_test_swarm::<_, ()>(handler_proto); + + let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); + let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); + + Swarm::listen_on(&mut swarm1, addr1.clone().into()).unwrap(); + Swarm::listen_on(&mut swarm2, addr2.clone().into()).unwrap(); + + // Test execution state. Connection => Disconnecting => Connecting. + enum State { + Connecting, + Disconnecting, + } + + let swarm1_id = Swarm::local_peer_id(&swarm1).clone(); + + let mut banned = false; + let mut unbanned = false; + + let num_connections = 10; + + for _ in 0 .. num_connections { + Swarm::dial_addr(&mut swarm1, addr2.clone()).unwrap(); + } + let mut state = State::Connecting; + + executor::block_on(future::poll_fn(move |cx| { + loop { + let poll1 = Swarm::poll_next_event(Pin::new(&mut swarm1), cx); + let poll2 = Swarm::poll_next_event(Pin::new(&mut swarm2), cx); + match state { + State::Connecting => { + for s in &[&swarm1, &swarm2] { + if s.behaviour.inject_connection_established.len() > 0 { + assert_eq!(s.behaviour.inject_connected.len(), 1); + } else { + assert_eq!(s.behaviour.inject_connected.len(), 0); + } + assert!(s.behaviour.inject_connection_closed.len() == 0); + assert!(s.behaviour.inject_disconnected.len() == 0); + } + if [&swarm1, &swarm2].iter().all(|s| { + s.behaviour.inject_connection_established.len() == num_connections + }) { + if banned { + return Poll::Ready(()) + } + Swarm::ban_peer_id(&mut swarm2, swarm1_id.clone()); + swarm1.behaviour.reset(); + swarm2.behaviour.reset(); + banned = true; + state = State::Disconnecting; + } + } + State::Disconnecting => { + for s in &[&swarm1, &swarm2] { + if s.behaviour.inject_connection_closed.len() < num_connections { + assert_eq!(s.behaviour.inject_disconnected.len(), 0); + } else { + assert_eq!(s.behaviour.inject_disconnected.len(), 1); + } + assert_eq!(s.behaviour.inject_connection_established.len(), 0); + assert_eq!(s.behaviour.inject_connected.len(), 0); + } + if [&swarm1, &swarm2].iter().all(|s| { + s.behaviour.inject_connection_closed.len() == num_connections + }) { + if unbanned { + return Poll::Ready(()) + } + // Unban the first peer and reconnect. + Swarm::unban_peer_id(&mut swarm2, swarm1_id.clone()); + swarm1.behaviour.reset(); + swarm2.behaviour.reset(); + unbanned = true; + for _ in 0 .. num_connections { + Swarm::dial_addr(&mut swarm2, addr1.clone()).unwrap(); + } + state = State::Connecting; + } + } + } + + if poll1.is_pending() && poll2.is_pending() { + return Poll::Pending + } + } + })) + } } diff --git a/swarm/src/protocols_handler/dummy.rs b/swarm/src/protocols_handler/dummy.rs index fb2b781f83c..cb758794afc 100644 --- a/swarm/src/protocols_handler/dummy.rs +++ b/swarm/src/protocols_handler/dummy.rs @@ -31,12 +31,15 @@ use std::task::{Context, Poll}; use void::Void; /// Implementation of `ProtocolsHandler` that doesn't handle anything. +#[derive(Clone, Debug)] pub struct DummyProtocolsHandler { + pub keep_alive: KeepAlive, } impl Default for DummyProtocolsHandler { fn default() -> Self { DummyProtocolsHandler { + keep_alive: KeepAlive::No } } } @@ -49,19 +52,16 @@ impl ProtocolsHandler for DummyProtocolsHandler { type OutboundProtocol = DeniedUpgrade; type OutboundOpenInfo = Void; - #[inline] fn listen_protocol(&self) -> SubstreamProtocol { SubstreamProtocol::new(DeniedUpgrade) } - #[inline] fn inject_fully_negotiated_inbound( &mut self, _: >::Output ) { } - #[inline] fn inject_fully_negotiated_outbound( &mut self, _: >::Output, @@ -69,16 +69,14 @@ impl ProtocolsHandler for DummyProtocolsHandler { ) { } - #[inline] fn inject_event(&mut self, _: Self::InEvent) {} - #[inline] fn inject_dial_upgrade_error(&mut self, _: Self::OutboundOpenInfo, _: ProtocolsHandlerUpgrErr<>::Error>) {} - #[inline] - fn connection_keep_alive(&self) -> KeepAlive { KeepAlive::No } + fn connection_keep_alive(&self) -> KeepAlive { + self.keep_alive + } - #[inline] fn poll( &mut self, _: &mut Context, diff --git a/swarm/src/test.rs b/swarm/src/test.rs new file mode 100644 index 00000000000..8249c358bae --- /dev/null +++ b/swarm/src/test.rs @@ -0,0 +1,254 @@ +// Copyright 2020 Parity Technologies (UK) Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +use crate::{ + NetworkBehaviour, + NetworkBehaviourAction, + ProtocolsHandler, + IntoProtocolsHandler, + PollParameters +}; +use libp2p_core::{ + ConnectedPoint, + PeerId, + connection::{ConnectionId, ListenerId}, + multiaddr::Multiaddr, +}; +use std::collections::HashMap; +use std::task::{Context, Poll}; + +/// A `MockBehaviour` is a `NetworkBehaviour` that allows for +/// the instrumentation of return values, without keeping +/// any further state. +pub struct MockBehaviour +where + THandler: ProtocolsHandler, +{ + /// The prototype protocols handler that is cloned for every + /// invocation of `new_handler`. + pub handler_proto: THandler, + /// The addresses to return from `addresses_of_peer`. + pub addresses: HashMap>, + /// The next action to return from `poll`. + /// + /// An action is only returned once. + pub next_action: Option>, +} + +impl MockBehaviour +where + THandler: ProtocolsHandler +{ + pub fn new(handler_proto: THandler) -> Self { + MockBehaviour { + handler_proto, + addresses: HashMap::new(), + next_action: None, + } + } +} + +impl NetworkBehaviour for MockBehaviour +where + THandler: ProtocolsHandler + Clone, + THandler::OutEvent: Clone, + TOutEvent: Send + 'static, +{ + type ProtocolsHandler = THandler; + type OutEvent = TOutEvent; + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + self.handler_proto.clone() + } + + fn addresses_of_peer(&mut self, p: &PeerId) -> Vec { + self.addresses.get(p).map_or(Vec::new(), |v| v.clone()) + } + + fn inject_connected(&mut self, _: &PeerId) { + } + + fn inject_disconnected(&mut self, _: &PeerId) { + } + + fn inject_event(&mut self, _: PeerId, _: ConnectionId, _: THandler::OutEvent) { + } + + fn poll(&mut self, _: &mut Context, _: &mut impl PollParameters) -> + Poll> + { + self.next_action.take().map_or(Poll::Pending, Poll::Ready) + } +} + +/// A `CallTraceBehaviour` is a `NetworkBehaviour` that tracks +/// invocations of callback methods and their arguments, wrapping +/// around an inner behaviour. +pub struct CallTraceBehaviour +where + TInner: NetworkBehaviour, +{ + inner: TInner, + + pub addresses_of_peer: Vec, + pub inject_connected: Vec, + pub inject_disconnected: Vec, + pub inject_connection_established: Vec<(PeerId, ConnectionId, ConnectedPoint)>, + pub inject_connection_closed: Vec<(PeerId, ConnectionId, ConnectedPoint)>, + pub inject_event: Vec<(PeerId, ConnectionId, <::Handler as ProtocolsHandler>::OutEvent)>, + pub inject_addr_reach_failure: Vec<(Option, Multiaddr)>, + pub inject_dial_failure: Vec, + pub inject_new_listen_addr: Vec, + pub inject_new_external_addr: Vec, + pub inject_expired_listen_addr: Vec, + pub inject_listener_error: Vec, + pub inject_listener_closed: Vec<(ListenerId, bool)>, + pub poll: usize, +} + +impl CallTraceBehaviour +where + TInner: NetworkBehaviour +{ + pub fn new(inner: TInner) -> Self { + Self { + inner, + addresses_of_peer: Vec::new(), + inject_connected: Vec::new(), + inject_disconnected: Vec::new(), + inject_connection_established: Vec::new(), + inject_connection_closed: Vec::new(), + inject_event: Vec::new(), + inject_addr_reach_failure: Vec::new(), + inject_dial_failure: Vec::new(), + inject_new_listen_addr: Vec::new(), + inject_new_external_addr: Vec::new(), + inject_expired_listen_addr: Vec::new(), + inject_listener_error: Vec::new(), + inject_listener_closed: Vec::new(), + poll: 0, + } + } + + pub fn reset(&mut self) { + self.addresses_of_peer = Vec::new(); + self.inject_connected = Vec::new(); + self.inject_disconnected = Vec::new(); + self.inject_connection_established = Vec::new(); + self.inject_connection_closed = Vec::new(); + self.inject_event = Vec::new(); + self.inject_addr_reach_failure = Vec::new(); + self.inject_dial_failure = Vec::new(); + self.inject_new_listen_addr = Vec::new(); + self.inject_new_external_addr = Vec::new(); + self.inject_expired_listen_addr = Vec::new(); + self.inject_listener_error = Vec::new(); + self.inject_listener_closed = Vec::new(); + self.poll = 0; + } +} + +impl NetworkBehaviour for CallTraceBehaviour +where + TInner: NetworkBehaviour, + <::Handler as ProtocolsHandler>::OutEvent: Clone, +{ + type ProtocolsHandler = TInner::ProtocolsHandler; + type OutEvent = TInner::OutEvent; + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + self.inner.new_handler() + } + + fn addresses_of_peer(&mut self, p: &PeerId) -> Vec { + self.addresses_of_peer.push(p.clone()); + self.inner.addresses_of_peer(p) + } + + fn inject_connected(&mut self, peer: &PeerId) { + self.inject_connected.push(peer.clone()); + self.inner.inject_connected(peer); + } + + fn inject_connection_established(&mut self, p: &PeerId, c: &ConnectionId, e: &ConnectedPoint) { + self.inject_connection_established.push((p.clone(), c.clone(), e.clone())); + self.inner.inject_connection_established(p, c, e); + } + + fn inject_disconnected(&mut self, peer: &PeerId) { + self.inject_disconnected.push(peer.clone()); + self.inner.inject_disconnected(peer); + } + + fn inject_connection_closed(&mut self, p: &PeerId, c: &ConnectionId, e: &ConnectedPoint) { + self.inject_connection_closed.push((p.clone(), c.clone(), e.clone())); + self.inner.inject_connection_closed(p, c, e); + } + + fn inject_event(&mut self, p: PeerId, c: ConnectionId, e: <::Handler as ProtocolsHandler>::OutEvent) { + self.inject_event.push((p.clone(), c.clone(), e.clone())); + self.inner.inject_event(p, c, e); + } + + fn inject_addr_reach_failure(&mut self, p: Option<&PeerId>, a: &Multiaddr, e: &dyn std::error::Error) { + self.inject_addr_reach_failure.push((p.cloned(), a.clone())); + self.inner.inject_addr_reach_failure(p, a, e); + } + + fn inject_dial_failure(&mut self, p: &PeerId) { + self.inject_dial_failure.push(p.clone()); + self.inner.inject_dial_failure(p); + } + + fn inject_new_listen_addr(&mut self, a: &Multiaddr) { + self.inject_new_listen_addr.push(a.clone()); + self.inner.inject_new_listen_addr(a); + } + + fn inject_expired_listen_addr(&mut self, a: &Multiaddr) { + self.inject_expired_listen_addr.push(a.clone()); + self.inner.inject_expired_listen_addr(a); + } + + fn inject_new_external_addr(&mut self, a: &Multiaddr) { + self.inject_new_external_addr.push(a.clone()); + self.inner.inject_new_external_addr(a); + } + + fn inject_listener_error(&mut self, l: ListenerId, e: &(dyn std::error::Error + 'static)) { + self.inject_listener_error.push(l.clone()); + self.inner.inject_listener_error(l, e); + } + + fn inject_listener_closed(&mut self, l: ListenerId, r: Result<(), &std::io::Error>) { + self.inject_listener_closed.push((l, r.is_ok())); + self.inner.inject_listener_closed(l, r); + } + + fn poll(&mut self, cx: &mut Context, args: &mut impl PollParameters) -> + Poll::Handler as ProtocolsHandler>::InEvent, + Self::OutEvent + >> + { + self.poll += 1; + self.inner.poll(cx, args) + } +} From 69a61938383054504c7bb5701258c5d8685044f9 Mon Sep 17 00:00:00 2001 From: Roman Borschel Date: Wed, 8 Jul 2020 11:17:30 +0200 Subject: [PATCH 02/19] Update swarm/src/lib.rs Co-authored-by: Toralf Wittner --- swarm/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 8dbe6dc20a3..2f3f36c35f6 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -524,7 +524,7 @@ where TBehaviour: NetworkBehaviour, } }, Poll::Ready(NetworkEvent::ConnectionClosed { id, connected, error, num_established }) => { - if error.is_some() { + if let Some(error) = error { log::debug!("Connection {:?} closed: {:?}", connected, error); } else { log::debug!("Connection {:?} closed (active close).", connected); From b6394590c88fc0b5364e75aa20d25553c5875b1c Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 8 Jul 2020 11:31:59 +0200 Subject: [PATCH 03/19] Incorporate some review feedback. --- core/src/connection/manager/task.rs | 23 +++++++++++------------ core/src/connection/pool.rs | 10 ++++++++++ swarm/src/lib.rs | 2 +- 3 files changed, 22 insertions(+), 13 deletions(-) diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index 4b088d3d0c3..e36683e9ee7 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -228,6 +228,9 @@ where return Poll::Pending } Poll::Ready(Err(error)) => { + // Don't accept any further commands and terminate the + // task with a final event. + this.commands.get_mut().close(); let event = Event::Failed { id, handler, error }; this.state = State::Terminating(event) } @@ -242,12 +245,14 @@ where Poll::Ready(Some(Command::NotifyHandler(event))) => connection.inject_event(event), Poll::Ready(Some(Command::Close)) => { + // Don't accept any further commands. + this.commands.get_mut().close(); // Discard the event, if any, and start a graceful close. this.state = State::Closing(connection.close()); continue 'poll } Poll::Ready(None) => { - // The manager has dropped the task; abort. + // The manager has dropped the task or disappeared; abort. return Poll::Ready(()) } } @@ -267,10 +272,8 @@ where continue 'poll } } - // The manager is no longer reachable, maybe due to - // application shutdown. Try a graceful shutdown of the - // connection before terminating the task. - this.state = State::Closing(connection.close()); + // The manager is no longer reachable; abort. + return Poll::Ready(()) } } } else { @@ -287,8 +290,9 @@ where }; } Poll::Ready(Err(error)) => { - // Terminate the task with the error, dropping - // the connection. + // Don't accept any further commands. + this.commands.get_mut().close(); + // Terminate the task with the error, dropping the connection. let event = Event::Closed { id, error: Some(error) }; this.state = State::Terminating(event); } @@ -297,8 +301,6 @@ where } State::Closing(mut closing) => { - // Ignore further commands received from the manager, if any. - while let Poll::Ready(Some(_)) = this.commands.poll_next_unpin(cx) {} // Try to gracefully close the connection. match closing.poll_unpin(cx) { Poll::Ready(Ok(())) => { @@ -320,9 +322,6 @@ where } State::Terminating(event) => { - // Ignore further commands received from the manager, if any. - while let Poll::Ready(Some(_)) = this.commands.poll_next_unpin(cx) - {} // Try to deliver the final event. match this.events.poll_ready(cx) { Poll::Pending => { diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index ca4b0f9c210..6379cabf225 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -593,6 +593,12 @@ where TPeerId: Clone { // Drain events resulting from forced disconnections. + // + // Note: The `Disconnected` entries in `self.disconnected` + // are inserted in ascending order of the remaining `num_established` + // connections. Thus we `pop()` them off from the end to emit the + // events in an order that properly counts down `num_established`. + // See also `Pool::disconnect`. while let Some(Disconnected { id, connected, num_established }) = self.disconnected.pop() { @@ -942,8 +948,12 @@ impl<'a, TInEvent, TConnInfo> Future for StartClose<'a, TInEvent, TConnInfo> { } } +/// Information about a former established connection to a peer +/// that was dropped via [`Pool::disconnect`]. struct Disconnected { id: ConnectionId, connected: Connected, + /// The remaining number of established connections + /// to the same peer. num_established: u32, } diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 2f3f36c35f6..f53418b2ce9 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -524,7 +524,7 @@ where TBehaviour: NetworkBehaviour, } }, Poll::Ready(NetworkEvent::ConnectionClosed { id, connected, error, num_established }) => { - if let Some(error) = error { + if let Some(error) = error.as_ref() { log::debug!("Connection {:?} closed: {:?}", connected, error); } else { log::debug!("Connection {:?} closed (active close).", connected); From 16362e130eb0b72d1a39d45523fc93a3943c825b Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Fri, 17 Jul 2020 12:49:25 +0200 Subject: [PATCH 04/19] Adapt to changes in master. --- core/src/connection/manager/task.rs | 1 - core/src/connection/pool.rs | 2 +- core/src/network/event.rs | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index 4cf39785512..bea84513dbe 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -73,7 +73,6 @@ impl Event { pub fn id(&self) -> &TaskId { match self { Event::Established { id, .. } => id, - Event::Error { id, .. } => id, Event::Failed { id, .. } => id, Event::AddressChange { id, .. } => id, Event::Notify { id, .. } => id, diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 81699a183f5..8038c228509 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -146,7 +146,7 @@ pub enum PoolEvent<'a, TInEvent, TOutEvent, THandler, TTransErr, THandlerErr, TC /// The connection to a node has changed its address. AddressChange { /// The connection that has changed address. - connection: EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, + connection: EstablishedConnection<'a, TInEvent, TConnInfo>, /// The new endpoint. new_endpoint: ConnectedPoint, /// The old endpoint. diff --git a/core/src/network/event.rs b/core/src/network/event.rs index 9e44a477d8b..5b0fbbc84bd 100644 --- a/core/src/network/event.rs +++ b/core/src/network/event.rs @@ -168,7 +168,7 @@ where /// An established connection has changed its address. AddressChange { /// The connection whose address has changed. - connection: EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>, + connection: EstablishedConnection<'a, TInEvent, TConnInfo>, /// New endpoint of this connection. new_endpoint: ConnectedPoint, /// Old endpoint of this connection. From 603cb2b29958170ba42375b6c5bd439274eb6a71 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Tue, 21 Jul 2020 10:35:55 +0200 Subject: [PATCH 05/19] More verbose panic messages. --- core/src/connection/manager.rs | 2 +- core/src/connection/pool.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/connection/manager.rs b/core/src/connection/manager.rs index d8e13b26747..0fbfa7ac291 100644 --- a/core/src/connection/manager.rs +++ b/core/src/connection/manager.rs @@ -459,7 +459,7 @@ impl<'a, I, C> EstablishedEntry<'a, I, C> { self.task.get_mut().sender.try_send(cmd) .map_err(|e| match e.into_inner() { task::Command::NotifyHandler(event) => event, - _ => unreachable!("by (*)") + _ => panic!("Unexpected command. Expected `NotifyHandler`") // see (*) }) } diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 8038c228509..6f01e62c465 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -882,7 +882,7 @@ where Some(manager::Entry::Established(entry)) => { return Some(EstablishedConnection { entry }) } - _ => unreachable!("by (*)") + _ => panic!("Established entry not found in manager.") // see (*) } } } @@ -905,7 +905,7 @@ where Some(manager::Entry::Established(entry)) => { return Some(EstablishedConnection { entry }) } - _ => unreachable!("by (*)") + _ => panic!("Established entry not found in manager.") // see (*) } } } From d1f8fe8fd1718bb91854e8dd9a289b809be38492 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Mon, 27 Jul 2020 11:45:34 +0200 Subject: [PATCH 06/19] Simplify There is no need for a `StartClose` future. --- core/src/connection.rs | 2 +- core/src/connection/manager.rs | 20 +++++++++----------- core/src/connection/pool.rs | 24 ++---------------------- 3 files changed, 12 insertions(+), 34 deletions(-) diff --git a/core/src/connection.rs b/core/src/connection.rs index 0349f1181fa..d160aeb23b3 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -31,7 +31,7 @@ pub use handler::{ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandl pub use listeners::{ListenerId, ListenersStream, ListenersEvent}; pub use manager::ConnectionId; pub use substream::{Substream, SubstreamEndpoint, Close}; -pub use pool::{EstablishedConnection, EstablishedConnectionIter, PendingConnection, StartClose}; +pub use pool::{EstablishedConnection, EstablishedConnectionIter, PendingConnection}; use crate::muxing::StreamMuxer; use crate::{Multiaddr, PeerId}; diff --git a/core/src/connection/manager.rs b/core/src/connection/manager.rs index 0fbfa7ac291..ae1ca4097f1 100644 --- a/core/src/connection/manager.rs +++ b/core/src/connection/manager.rs @@ -473,21 +473,19 @@ impl<'a, I, C> EstablishedEntry<'a, I, C> { self.task.get_mut().sender.poll_ready(cx).map_err(|_| ()) } - /// Tries to send a close command to the associated background task, + /// Sends a close command to the associated background task, /// thus initiating a graceful active close of the connection. /// + /// Has no effect if the connection is already closing. + /// /// When the connection is ultimately closed, [`Event::ConnectionClosed`] /// is emitted by [`Manager::poll`]. - pub fn poll_start_close(&mut self, cx: &mut Context) -> Poll<()> { - match self.task.get_mut().sender.poll_ready(cx) { - Poll::Ready(result) => { - if result.is_ok() { - // If it fails now then the task is already gone. - let _ = self.task.get_mut().sender.try_send(task::Command::Close); - } - Poll::Ready(()) - } - Poll::Pending => Poll::Pending + pub fn start_close(&mut self) { + // Clone the sender so that we are guaranteed to have + // capacity for the close command (every sender gets a slot). + match self.task.get_mut().sender.clone().try_send(task::Command::Close) { + Ok(()) => {}, + Err(e) => assert!(e.is_disconnected(), "No capacity for close command.") } } diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 6f01e62c465..77da1311500 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -854,8 +854,8 @@ where } /// Initiates a graceful close of the connection. - pub fn close(self) -> StartClose<'a, TInEvent, TConnInfo> { - StartClose(self.entry) + pub fn start_close(mut self) { + self.entry.start_close() } } @@ -965,26 +965,6 @@ impl PoolLimits { } } -/// A `StartClose` future resolves when the command to -/// close has been enqueued for the background task associated -/// with a connection. -/// -/// When the connection is ultimately closed, -/// [`crate::network::NetworkEvent::ConnectionClosed`] -/// is emitted with no `error` on success. -#[derive(Debug)] -pub struct StartClose<'a, TInEvent, TConnInfo>( - manager::EstablishedEntry<'a, TInEvent, TConnInfo>, -); - -impl<'a, TInEvent, TConnInfo> Future for StartClose<'a, TInEvent, TConnInfo> { - type Output = (); - - fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context) -> Poll<()> { - self.0.poll_start_close(cx) - } -} - /// Information about a former established connection to a peer /// that was dropped via [`Pool::disconnect`]. struct Disconnected { From 9322433d780d54e25c2d2ddaf865d54397757a8c Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Mon, 27 Jul 2020 12:42:45 +0200 Subject: [PATCH 07/19] Fix doc links. --- core/src/connection/pool.rs | 4 ++-- core/src/network/event.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 77da1311500..003fd50ae12 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -100,7 +100,7 @@ pub enum PoolEvent<'a, TInEvent, TOutEvent, THandler, TTransErr, THandlerErr, TC /// /// * it encounters an error, which includes the connection being /// closed by the remote. In this case `error` is `Some`. - /// * it was actively closed by [`EstablishedConnection::close`], + /// * it was actively closed by [`EstablishedConnection::start_close`], /// i.e. a successful, orderly close. /// * it was actively closed by [`Pool::disconnect`], i.e. /// dropped without an orderly close. @@ -469,7 +469,7 @@ where /// by the pool effective immediately. /// /// > **Note**: Established connections are dropped without performing - /// > an orderly close. See [`EstablishedConnection::close`] for + /// > an orderly close. See [`EstablishedConnection::start_close`] for /// > performing such an orderly close. pub fn disconnect(&mut self, peer: &TPeerId) { if let Some(conns) = self.established.get(peer) { diff --git a/core/src/network/event.rs b/core/src/network/event.rs index 5b0fbbc84bd..c740cf0f86c 100644 --- a/core/src/network/event.rs +++ b/core/src/network/event.rs @@ -116,7 +116,7 @@ where /// /// * it encounters an error, which includes the connection being /// closed by the remote. In this case `error` is `Some`. - /// * it was actively closed by [`EstablishedConnection::close`], + /// * it was actively closed by [`EstablishedConnection::start_close`], /// i.e. a successful, orderly close. In this case `error` is `None`. /// * it was actively closed by [`super::peer::ConnectedPeer::disconnect`] or /// [`super::peer::DialingPeer::disconnect`], i.e. dropped without an From 487dc69107cff18aa747305bc98c8b90a96870d5 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 29 Jul 2020 17:05:32 +0200 Subject: [PATCH 08/19] Further small cleanup. --- core/src/connection/manager.rs | 2 +- core/src/connection/pool.rs | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/connection/manager.rs b/core/src/connection/manager.rs index ae1ca4097f1..0451ccf4df4 100644 --- a/core/src/connection/manager.rs +++ b/core/src/connection/manager.rs @@ -480,7 +480,7 @@ impl<'a, I, C> EstablishedEntry<'a, I, C> { /// /// When the connection is ultimately closed, [`Event::ConnectionClosed`] /// is emitted by [`Manager::poll`]. - pub fn start_close(&mut self) { + pub fn start_close(mut self) { // Clone the sender so that we are guaranteed to have // capacity for the close command (every sender gets a slot). match self.task.get_mut().sender.clone().try_send(task::Command::Close) { diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 003fd50ae12..ee2b7d054bc 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -854,7 +854,9 @@ where } /// Initiates a graceful close of the connection. - pub fn start_close(mut self) { + /// + /// Has no effect if the connection is already closing. + pub fn start_close(self) { self.entry.start_close() } } From 4df478db24ec3807dc49f7d94c8b99087e652d39 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Tue, 21 Jul 2020 10:27:04 +0200 Subject: [PATCH 09/19] Graceful shutdown for connections, networks and swarms. Building on the ability to wait for connection shutdown to complete introduced in https://github.com/libp2p/rust-libp2p/pull/1619, this commit extends the ability for performing graceful shutdowns in the following ways: 1. The `ConnectionHandler` (and thus also `ProtocolsHandler`) can participate in the shutdown, via new `poll_close` methods. The muxer and underlying transport connection only starts closing once the connection handler signals readiness to do so. 2. A `Network` can be gracefully shut down, which involves a graceful shutdown of the underlying connection `Pool`. The `Pool` in turn proceeds with a shutdown by rejecting new connections while draining established connections. 3. A `Swarm` can be gracefully shut down, which involves a graceful shutdown of the underlying `Network` followed by polling the `NetworkBehaviour` until it returns `Poll::Pending`, i.e. it has no more output. In particular, the following are important details: * Analogous to new inbound and outbound connections during shutdown, while a single connection is shutting down, it rejects new inbound substreams and, by the return type of `ConnectionHandler::poll_close`, no new outbound substreams can be requested. * The `NodeHandlerWrapper` managing the `ProtocolsHandler` always waits for already ongoing inbound and outbound substream upgrades to complete. Since the `NodeHandlerWrapper` is a `ConnectionHandler`, the previous point applies w.r.t. new inbound and outbound substreams. * When the `connection_keep_alive` expires, a graceful shutdown is initiated. --- core/src/connection.rs | 122 ++++- core/src/connection/handler.rs | 38 +- core/src/connection/manager/task.rs | 64 +-- core/src/connection/pool.rs | 126 ++++- core/src/connection/substream.rs | 40 +- core/src/network.rs | 207 ++++--- core/src/network/peer.rs | 3 +- core/tests/network_dial_error.rs | 29 +- examples/ipfs-kad.rs | 4 +- misc/core-derive/src/lib.rs | 6 + misc/core-derive/tests/test.rs | 5 +- protocols/gossipsub/src/behaviour.rs | 6 + protocols/gossipsub/tests/smoke.rs | 16 +- protocols/identify/src/identify.rs | 6 +- protocols/kad/src/behaviour/test.rs | 577 +++++++++++--------- protocols/ping/tests/ping.rs | 4 +- protocols/request-response/src/handler.rs | 83 ++- protocols/request-response/src/lib.rs | 28 +- protocols/request-response/tests/ping.rs | 8 +- swarm/src/behaviour.rs | 11 + swarm/src/lib.rs | 197 +++++-- swarm/src/protocols_handler.rs | 48 +- swarm/src/protocols_handler/node_handler.rs | 165 +++--- 23 files changed, 1204 insertions(+), 589 deletions(-) diff --git a/core/src/connection.rs b/core/src/connection.rs index cc1bd0a3ad4..50a0ff618b7 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -30,7 +30,7 @@ pub use error::{ConnectionError, PendingConnectionError}; pub use handler::{ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandler}; pub use listeners::{ListenerId, ListenersStream, ListenersEvent}; pub use manager::ConnectionId; -pub use substream::{Substream, SubstreamEndpoint, Close}; +pub use substream::{Substream, SubstreamEndpoint}; pub use pool::{EstablishedConnection, EstablishedConnectionIter, PendingConnection}; use crate::muxing::StreamMuxer; @@ -194,10 +194,12 @@ where TMuxer: StreamMuxer, THandler: ConnectionHandler>, { - /// Node that handles the muxing. + /// The substream multiplexer over the connection I/O stream. muxing: substream::Muxing, - /// Handler that processes substreams. + /// The connection handler for the substreams. handler: THandler, + /// The operating state of the connection. + state: ConnectionState, } impl fmt::Debug for Connection @@ -231,44 +233,76 @@ where Connection { muxing: Muxing::new(muxer), handler, + state: ConnectionState::Open, } } - /// Returns a reference to the `ConnectionHandler` - pub fn handler(&self) -> &THandler { - &self.handler - } - - /// Returns a mutable reference to the `ConnectionHandler` - pub fn handler_mut(&mut self) -> &mut THandler { - &mut self.handler - } - /// Notifies the connection handler of an event. + /// + /// Has no effect if the connection handler is already closed. pub fn inject_event(&mut self, event: THandler::InEvent) { - self.handler.inject_event(event); + match self.state { + ConnectionState::Open | ConnectionState::CloseHandler + => self.handler.inject_event(event), + _ => { + log::trace!("Ignoring handler event. Handler is closed.") + } + } } - /// Begins an orderly shutdown of the connection, returning a - /// `Future` that resolves when connection shutdown is complete. - pub fn close(self) -> Close { - self.muxing.close().0 + /// Begins a graceful shutdown of the connection. + /// + /// The connection must continue to be `poll()`ed to drive the + /// shutdown process to completion. Once connection shutdown is + /// complete, `poll()` returns `Ok(None)`. + pub fn start_close(&mut self) { + if self.state == ConnectionState::Open { + self.state = ConnectionState::CloseHandler; + } } /// Polls the connection for events produced by the associated handler /// as a result of I/O activity on the substream multiplexer. + /// + /// > **Note**: A return value of `Ok(None)` signals successful + /// > connection shutdown, whereas an `Err` signals termination + /// > of the connection due to an error. In either case, the + /// > connection must be dropped; any further method calls + /// > result in unspecified behaviour. pub fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll, ConnectionError>> + -> Poll>, ConnectionError>> { loop { + if let ConnectionState::Closed = self.state { // (1) + return Poll::Ready(Ok(None)) + } + + if let ConnectionState::CloseMuxer = self.state { // (2) + match futures::ready!(self.muxing.poll_close(cx)) { + Ok(()) => { + self.state = ConnectionState::Closed; + return Poll::Ready(Ok(None)) + } + Err(e) => return Poll::Ready(Err(ConnectionError::IO(e))) + } + } + + // At this point the connection is either open or in the process + // of a graceful shutdown by the connection handler. let mut io_pending = false; // Perform I/O on the connection through the muxer, informing the handler - // of new substreams. + // of new substreams or other muxer events. match self.muxing.poll(cx) { Poll::Pending => io_pending = true, Poll::Ready(Ok(SubstreamEvent::InboundSubstream { substream })) => { - self.handler.inject_substream(substream, SubstreamEndpoint::Listener) + // Drop new inbound substreams when closing. This is analogous + // to rejecting new connections. + if self.state == ConnectionState::Open { + self.handler.inject_substream(substream, SubstreamEndpoint::Listener) + } else { + log::trace!("Inbound substream dropped. Connection is closing.") + } } Poll::Ready(Ok(SubstreamEvent::OutboundSubstream { user_data, substream })) => { let endpoint = SubstreamEndpoint::Dialer(user_data); @@ -276,23 +310,37 @@ where } Poll::Ready(Ok(SubstreamEvent::AddressChange(address))) => { self.handler.inject_address_change(&address); - return Poll::Ready(Ok(Event::AddressChange(address))); + return Poll::Ready(Ok(Some(Event::AddressChange(address)))); } Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::IO(err))), } // Poll the handler for new events. - match self.handler.poll(cx) { + let poll = match &self.state { + ConnectionState::Open => self.handler.poll(cx).map_ok(Some), + ConnectionState::CloseHandler => self.handler.poll_close(cx).map_ok( + |event| event.map(ConnectionHandlerEvent::Custom)), + s => panic!("Unexpected closing state: {:?}", s) // s.a. (1),(2) + }; + + match poll { Poll::Pending => { if io_pending { return Poll::Pending // Nothing to do } } - Poll::Ready(Ok(ConnectionHandlerEvent::OutboundSubstreamRequest(user_data))) => { + Poll::Ready(Ok(Some(ConnectionHandlerEvent::OutboundSubstreamRequest(user_data)))) => { self.muxing.open_substream(user_data); } - Poll::Ready(Ok(ConnectionHandlerEvent::Custom(event))) => { - return Poll::Ready(Ok(Event::Handler(event))); + Poll::Ready(Ok(Some(ConnectionHandlerEvent::Custom(event)))) => { + return Poll::Ready(Ok(Some(Event::Handler(event)))); + } + Poll::Ready(Ok(Some(ConnectionHandlerEvent::Close))) => { + self.start_close() + } + Poll::Ready(Ok(None)) => { + // The handler is done, we can now close the muxer (i.e. connection). + self.state = ConnectionState::CloseMuxer; } Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::Handler(err))), } @@ -352,3 +400,25 @@ impl fmt::Display for ConnectionLimit { /// A `ConnectionLimit` can represent an error if it has been exceeded. impl Error for ConnectionLimit {} + +/// The state of a [`Connection`] w.r.t. an active graceful close. +#[derive(Debug, PartialEq, Eq)] +enum ConnectionState { + /// The connection is open, accepting new inbound and outbound + /// substreams. + Open, + /// The connection is closing, rejecting new inbound substreams + /// and not permitting new outbound substreams while the + /// connection handler closes. [`ConnectionHandler::poll_close`] + /// is called until completion which results in transitioning to + /// `CloseMuxer`. + CloseHandler, + /// The connection is closing, rejecting new inbound substreams + /// and not permitting new outbound substreams while the + /// muxer is closing the transport connection. [`Muxer::poll_close`] + /// is called until completion, which results in transitioning + /// to `Closed`. + CloseMuxer, + /// The connection is closed. + Closed +} diff --git a/core/src/connection/handler.rs b/core/src/connection/handler.rs index 625269a3045..b2bdee6859f 100644 --- a/core/src/connection/handler.rs +++ b/core/src/connection/handler.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. use crate::{Multiaddr, PeerId}; -use std::{task::Context, task::Poll}; +use std::task::{Context, Poll}; use super::{Connected, SubstreamEndpoint}; /// The interface of a connection handler. @@ -66,6 +66,37 @@ pub trait ConnectionHandler { /// Returning an error will close the connection to the remote. fn poll(&mut self, cx: &mut Context<'_>) -> Poll, Self::Error>>; + + /// Polls the handler to make progress towards closing the connection. + /// + /// When a connection is actively closed, the handler can perform + /// a graceful shutdown of the connection by draining the I/O + /// activity, e.g. allowing in-flight requests to complete without + /// accepting new ones, possibly signaling the remote that it + /// should direct further requests elsewhere. + /// + /// The handler can also use the opportunity to flush any buffers + /// or clean up any other (asynchronous) resources before the + /// connection is ultimately dropped and closed on the transport + /// layer. + /// + /// While closing, new inbound substreams are rejected and the + /// handler is unable to request new outbound substreams as + /// per the return type of `poll_close`. + /// + /// The handler signals its readiness for the connection + /// to be closed by returning `Ready(Ok(None))`, which is the + /// default implementation. Hence, by default, connection + /// shutdown is not delayed and may result in ungraceful + /// interruption of ongoing I/O. + /// + /// > **Note**: Once `poll_close()` is invoked, the handler is no + /// > longer `poll()`ed. + fn poll_close(&mut self, _: &mut Context) + -> Poll, Self::Error>> + { + Poll::Ready(Ok(None)) + } } /// Prototype for a `ConnectionHandler`. @@ -99,6 +130,9 @@ pub enum ConnectionHandlerEvent { /// Other event. Custom(TCustom), + + /// Initiate connection shutdown. + Close, } /// Event produced by a handler. @@ -112,6 +146,7 @@ impl ConnectionHandlerEvent ConnectionHandlerEvent::Custom(val), + ConnectionHandlerEvent::Close => ConnectionHandlerEvent::Close, } } @@ -124,6 +159,7 @@ impl ConnectionHandlerEvent ConnectionHandlerEvent::Custom(map(val)), + ConnectionHandlerEvent::Close => ConnectionHandlerEvent::Close, } } } diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index 4b7ca0dd6f5..4e37ad671bf 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -23,7 +23,6 @@ use crate::{ muxing::StreamMuxer, connection::{ self, - Close, Connected, Connection, ConnectionError, @@ -168,9 +167,6 @@ where event: Option::Error, C>> }, - /// The connection is closing (active close). - Closing(Close), - /// The task is terminating with a final event for the `Manager`. Terminating(Event::Error, C>), @@ -250,11 +246,8 @@ where Poll::Ready(Some(Command::NotifyHandler(event))) => connection.inject_event(event), Poll::Ready(Some(Command::Close)) => { - // Don't accept any further commands. - this.commands.get_mut().close(); - // Discard the event, if any, and start a graceful close. - this.state = State::Closing(connection.close()); - continue 'poll + // Start closing the connection, if not already. + connection.start_close(); } Poll::Ready(None) => { // The manager has dropped the task or disappeared; abort. @@ -267,13 +260,19 @@ where // Send the event to the manager. match this.events.poll_ready(cx) { Poll::Pending => { - this.state = State::Established { connection, event: Some(event) }; + this.state = State::Established { + connection, + event: Some(event), + }; return Poll::Pending } Poll::Ready(result) => { if result.is_ok() { if let Ok(()) = this.events.start_send(event) { - this.state = State::Established { connection, event: None }; + this.state = State::Established { + connection, + event: None, + }; continue 'poll } } @@ -282,24 +281,34 @@ where } } } else { - // Poll the connection for new events. match Connection::poll(Pin::new(&mut connection), cx) { Poll::Pending => { - this.state = State::Established { connection, event: None }; + this.state = State::Established { + connection, + event: None, + }; return Poll::Pending } - Poll::Ready(Ok(connection::Event::Handler(event))) => { + Poll::Ready(Ok(Some(connection::Event::Handler(event)))) => { this.state = State::Established { connection, - event: Some(Event::Notify { id, event }) + event: Some(Event::Notify { id, event }), }; } - Poll::Ready(Ok(connection::Event::AddressChange(new_address))) => { + Poll::Ready(Ok(Some(connection::Event::AddressChange(new_address)))) => { this.state = State::Established { connection, - event: Some(Event::AddressChange { id, new_address }) + event: Some(Event::AddressChange { id, new_address }), }; } + Poll::Ready(Ok(None)) => { + // The connection is closed, don't accept any further commands + // and terminate the task with a final event. + this.commands.get_mut().close(); + let event = Event::Closed { id: this.id, error: None }; + this.state = State::Terminating(event); + continue 'poll + } Poll::Ready(Err(error)) => { // Don't accept any further commands. this.commands.get_mut().close(); @@ -311,27 +320,6 @@ where } } - State::Closing(mut closing) => { - // Try to gracefully close the connection. - match closing.poll_unpin(cx) { - Poll::Ready(Ok(())) => { - let event = Event::Closed { id: this.id, error: None }; - this.state = State::Terminating(event); - } - Poll::Ready(Err(e)) => { - let event = Event::Closed { - id: this.id, - error: Some(ConnectionError::IO(e)) - }; - this.state = State::Terminating(event); - } - Poll::Pending => { - this.state = State::Closing(closing); - return Poll::Pending - } - } - } - State::Terminating(event) => { // Try to deliver the final event. match this.events.poll_ready(cx) { diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index e27fde27ec6..032e564aa02 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -70,15 +70,20 @@ pub struct Pool>, + + /// The current operating state of the pool. + state: PoolState, } impl fmt::Debug for Pool { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - // TODO: More useful debug impl? f.debug_struct("Pool") + .field("state", &self.state) .field("limits", &self.limits) + .field("peers", &self.established.len()) + .field("pending", &self.pending.len()) .finish() } } @@ -218,6 +223,7 @@ where established: Default::default(), pending: Default::default(), disconnected: Vec::new(), + state: PoolState::Open, } } @@ -226,6 +232,22 @@ where &self.limits } + /// Whether the `Pool` is open, i.e. accepting new connections. + pub fn is_open(&self) -> bool { + self.state == PoolState::Open + } + + /// Whether the `Pool` is closing, i.e. rejecting new connections. + pub fn is_closing(&self) -> bool { + self.state == PoolState::Draining + } + + /// Whether the `Pool` is closed, i.e. all connections are closed + /// and no new connections are accepted. + pub fn is_closed(&self) -> bool { + self.state == PoolState::Closed + } + /// Adds a pending incoming connection to the pool in the form of a /// `Future` that establishes and negotiates the connection. /// @@ -599,16 +621,57 @@ where self.established.keys() } + /// Initiates a graceful shutdown of the `Pool`. + /// + /// All pending connections are immediately aborted and the `Pool` + /// refuses to accept any new connections. Established connections + /// will be drained by continued `poll()`ing of the `Pool`. + pub fn start_close(&mut self) { + if self.state != PoolState::Open { + return // Already closing + } + + // Set the state and limits for the shutdown. + self.state = PoolState::Draining; + self.limits = SHUTDOWN_LIMITS; + + // Immediately abort all pending connections. + for (id, _) in self.pending.drain() { + match self.manager.entry(id) { + Some(manager::Entry::Pending(e)) => { e.abort(); }, + _ => {} + } + } + + // Start a clean shutdown for all established connections. + for id in self.established.values().flat_map(|conns| conns.keys()) { + match self.manager.entry(*id) { + Some(manager::Entry::Established(mut e)) => { + e.start_close(); + }, + _ => { + panic!("Entry for established connection not found: {:?}", id) + } + } + } + + // Shutdown progress for established connections is driven by `Pool::poll`. + } + /// Polls the connection pool for events. /// /// > **Note**: We use a regular `poll` method instead of implementing `Stream`, /// > because we want the `Pool` to stay borrowed if necessary. pub fn poll<'a>(&'a mut self, cx: &mut Context<'_>) -> Poll< - PoolEvent<'a, TInEvent, TOutEvent, THandler, TTransErr, THandlerErr, TConnInfo, TPeerId> + Option> > where TConnInfo: ConnectionInfo + Clone, TPeerId: Clone { + if self.state == PoolState::Closed { + return Poll::Ready(None) + } + // Drain events resulting from forced disconnections. // // Note: The `Disconnected` entries in `self.disconnected` @@ -619,33 +682,39 @@ where while let Some(Disconnected { id, connected, num_established }) = self.disconnected.pop() { - return Poll::Ready(PoolEvent::ConnectionClosed { + return Poll::Ready(Some(PoolEvent::ConnectionClosed { id, connected, num_established, error: None, pool: self, - }) + })) } // Poll the connection `Manager`. loop { + // If there are no more established connections, shutdown is complete. + if self.state == PoolState::Draining && self.established.is_empty() { + self.state = PoolState::Closed; + return Poll::Ready(None) + } + let item = match self.manager.poll(cx) { Poll::Ready(item) => item, - Poll::Pending => return Poll::Pending, + Poll::Pending => return Poll::Pending }; match item { manager::Event::PendingConnectionError { id, error, handler } => { if let Some((endpoint, peer)) = self.pending.remove(&id) { - return Poll::Ready(PoolEvent::PendingConnectionError { + return Poll::Ready(Some(PoolEvent::PendingConnectionError { id, endpoint, error, handler: Some(handler), peer, pool: self - }) + })) } }, manager::Event::ConnectionClosed { id, connected, error } => { @@ -659,9 +728,9 @@ where if num_established == 0 { self.established.remove(connected.peer_id()); } - return Poll::Ready(PoolEvent::ConnectionClosed { + return Poll::Ready(Some(PoolEvent::ConnectionClosed { id, connected, error, num_established, pool: self - }) + })) } manager::Event::ConnectionEstablished { entry } => { let id = entry.id(); @@ -672,14 +741,14 @@ where .map_or(0, |conns| conns.len()); if let Err(e) = self.limits.check_established(current) { let connected = entry.remove(); - return Poll::Ready(PoolEvent::PendingConnectionError { + return Poll::Ready(Some(PoolEvent::PendingConnectionError { id, endpoint: connected.endpoint, error: PendingConnectionError::ConnectionLimit(e), handler: None, peer, pool: self - }) + })) } // Peer ID checks must already have happened. See `add_pending`. if cfg!(debug_assertions) { @@ -700,9 +769,9 @@ where conns.insert(id, endpoint); match self.get(id) { Some(PoolConnection::Established(connection)) => - return Poll::Ready(PoolEvent::ConnectionEstablished { + return Poll::Ready(Some(PoolEvent::ConnectionEstablished { connection, num_established - }), + })), _ => unreachable!("since `entry` is an `EstablishedEntry`.") } } @@ -711,10 +780,10 @@ where let id = entry.id(); match self.get(id) { Some(PoolConnection::Established(connection)) => - return Poll::Ready(PoolEvent::ConnectionEvent { + return Poll::Ready(Some(PoolEvent::ConnectionEvent { connection, event, - }), + })), _ => unreachable!("since `entry` is an `EstablishedEntry`.") } }, @@ -730,11 +799,11 @@ where match self.get(id) { Some(PoolConnection::Established(connection)) => - return Poll::Ready(PoolEvent::AddressChange { + return Poll::Ready(Some(PoolEvent::AddressChange { connection, new_endpoint, old_endpoint, - }), + })), _ => unreachable!("since `entry` is an `EstablishedEntry`.") } }, @@ -916,7 +985,7 @@ where } /// The configurable limits of a connection [`Pool`]. -#[derive(Debug, Clone, Default)] +#[derive(Debug, Clone, Default, PartialEq, Eq)] pub struct PoolLimits { pub max_outgoing: Option, pub max_incoming: Option, @@ -924,6 +993,14 @@ pub struct PoolLimits { pub max_outgoing_per_peer: Option, } +const SHUTDOWN_LIMITS: PoolLimits = + PoolLimits { + max_outgoing: Some(0), + max_incoming: Some(0), + max_established_per_peer: Some(0), + max_outgoing_per_peer: Some(0), + }; + impl PoolLimits { fn check_established(&self, current: F) -> Result<(), ConnectionLimit> where @@ -976,3 +1053,16 @@ struct Disconnected { /// to the same peer. num_established: u32, } + +/// The operating state of a [`Pool`]. +#[derive(Debug, PartialEq, Eq, Copy, Clone)] +enum PoolState { + /// The pool is open for new connections, subject to the + /// configured limits. + Open, + /// The pool is waiting for established connections to close + /// while rejecting new connections. + Draining, + /// The pool has shut down and is closed. + Closed +} diff --git a/core/src/connection/substream.rs b/core/src/connection/substream.rs index ac537b488e9..1c96d091730 100644 --- a/core/src/connection/substream.rs +++ b/core/src/connection/substream.rs @@ -19,11 +19,10 @@ // DEALINGS IN THE SOFTWARE. use crate::muxing::{StreamMuxer, StreamMuxerEvent, SubstreamRef, substream_from_ref}; -use futures::prelude::*; use multiaddr::Multiaddr; use smallvec::SmallVec; use std::sync::Arc; -use std::{fmt, io::Error as IoError, pin::Pin, task::Context, task::Poll}; +use std::{fmt, io, task::Context, task::Poll}; /// Endpoint for a received substream. #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -67,12 +66,6 @@ where outbound_substreams: SmallVec<[(TUserData, TMuxer::OutboundSubstream); 8]>, } -/// Future that signals the remote that we have closed the connection. -pub struct Close { - /// Muxer to close. - muxer: Arc, -} - /// A successfully opened substream. pub type Substream = SubstreamRef>; @@ -130,27 +123,27 @@ where self.outbound_substreams.push((user_data, raw)); } - /// Destroys the node stream and returns all the pending outbound substreams, plus an object - /// that signals the remote that we shut down the connection. - #[must_use] - pub fn close(mut self) -> (Close, Vec) { - let substreams = self.cancel_outgoing(); - let close = Close { muxer: self.inner.clone() }; - (close, substreams) + /// Closes the underlying connection, canceling any pending outbound substreams. + pub fn poll_close(&mut self, cx: &mut Context<'_>) -> Poll> { + self.cancel_outgoing(); + match self.inner.close(cx) { + Poll::Pending => Poll::Pending, + Poll::Ready(Ok(())) => Poll::Ready(Ok(())), + Poll::Ready(Err(err)) => Poll::Ready(Err(err.into())), + } } - /// Destroys all outbound streams and returns the corresponding user data. - pub fn cancel_outgoing(&mut self) -> Vec { - let mut out = Vec::with_capacity(self.outbound_substreams.len()); - for (user_data, outbound) in self.outbound_substreams.drain(..) { - out.push(user_data); + /// Destroys all outbound streams. + fn cancel_outgoing(&mut self) { + for (_, outbound) in self.outbound_substreams.drain(..) { self.inner.destroy_outbound(outbound); } - out } /// Provides an API similar to `Future`. - pub fn poll(&mut self, cx: &mut Context<'_>) -> Poll, IoError>> { + pub fn poll(&mut self, cx: &mut Context<'_>) + -> Poll, io::Error>> + { // Polling inbound substream. match self.inner.poll_event(cx) { Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) => { @@ -218,6 +211,7 @@ where } } +<<<<<<< HEAD impl Future for Close where TMuxer: StreamMuxer, @@ -243,6 +237,8 @@ where } } +======= +>>>>>>> Graceful shutdown for connections, networks and swarms. impl fmt::Debug for SubstreamEvent where TMuxer: StreamMuxer, diff --git a/core/src/network.rs b/core/src/network.rs index 36360dad189..6a798db2782 100644 --- a/core/src/network.rs +++ b/core/src/network.rs @@ -137,14 +137,20 @@ where impl Network where - TTrans: Transport + Clone, - TMuxer: StreamMuxer, + TTrans: Transport + Clone, + TTrans::Error: Send + 'static, + TTrans::Dial: Send + 'static, + TTrans::ListenerUpgrade: Send + 'static, + TMuxer: StreamMuxer + Send + Sync + 'static, + TMuxer::OutboundSubstream: Send, THandler: IntoConnectionHandler + Send + 'static, THandler::Handler: ConnectionHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, ::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary ::Error: error::Error + Send + 'static, - TConnInfo: fmt::Debug + ConnectionInfo + Send + 'static, - TPeerId: Eq + Hash + Clone, + TConnInfo: ConnectionInfo + fmt::Debug + Clone + Send + 'static, + TPeerId: Eq + Hash + Clone + Send + 'static, + TInEvent: Send + 'static, + TOutEvent: Send + 'static, { /// Creates a new node events stream. pub fn new( @@ -220,16 +226,6 @@ where /// connection ID is returned. pub fn dial(&mut self, address: &Multiaddr, handler: THandler) -> Result - where - TTrans: Transport, - TTrans::Error: Send + 'static, - TTrans::Dial: Send + 'static, - TMuxer: Send + Sync + 'static, - TMuxer::OutboundSubstream: Send, - TInEvent: Send + 'static, - TOutEvent: Send + 'static, - TConnInfo: Send + 'static, - TPeerId: Send + 'static, { let info = OutgoingInfo { address, peer_id: None }; match self.transport().clone().dial(address.clone()) { @@ -326,59 +322,125 @@ where Peer::new(self, peer_id) } + /// Initiates a graceful shutdown of the `Network`. + /// + /// A graceful shutdown proceeds by not accepting any new + /// connections while waiting for all currently established + /// connections to close. + /// + /// After calling this method, [`Network::poll`] makes progress + /// towards shutdown, eventually returning `Poll::Ready(None)` when + /// shutdown is complete. + /// + /// A graceful shutdown involves gracefully closing all established + /// connections, as defined by [`ConnectionHandler::poll_close`]. + pub fn start_close(&mut self) { + self.pool.start_close(); + } + + /// Performs a graceful shutdown of the `Network`, ignoring + /// any further events. + /// + /// See [`Network::start_close`] for further details. + pub async fn close(&mut self) { + self.start_close(); + future::poll_fn(move |cx| { + loop { + match self.poll(cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(None) => return Poll::Ready(()), + Poll::Ready(Some(_)) => {}, + } + } + }).await + } + + /// Whether the `Network` is closed. + /// + /// The `Network` is closed after successful completion + /// of a graceful shutdown. See [`Network::start_close`] + /// and [`Network::closed`]. + /// + /// A closed `Network` no longer performs any I/O and + /// shoudl eventually be discarded. + pub fn is_closed(&self) -> bool { + self.pool.is_closed() + } + + /// Whether the `Network` is closing. + /// + /// When the `Network` is closing, no new inbound connections + /// are accepted and no new outbound connections can be requested + /// while already established connections are drained. + /// + /// Returns `false` if the network is already closed + /// or is not currently closing. + pub fn is_closing(&self) -> bool { + self.pool.is_closing() + } + /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll<'a>(&'a mut self, cx: &mut Context<'_>) -> Poll> - where - TTrans: Transport, - TTrans::Error: Send + 'static, - TTrans::Dial: Send + 'static, - TTrans::ListenerUpgrade: Send + 'static, - TMuxer: Send + Sync + 'static, - TMuxer::OutboundSubstream: Send, - TInEvent: Send + 'static, - TOutEvent: Send + 'static, - THandler: IntoConnectionHandler + Send + 'static, - THandler::Handler: ConnectionHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, - ::Error: error::Error + Send + 'static, - TConnInfo: Clone, - TPeerId: Send + 'static, + pub fn poll<'a>(&'a mut self, cx: &mut Context<'_>) + -> Poll>> { - // Poll the listener(s) for new connections. - match ListenersStream::poll(Pin::new(&mut self.listeners), cx) { - Poll::Pending => (), - Poll::Ready(ListenersEvent::Incoming { - listener_id, - upgrade, - local_addr, - send_back_addr - }) => { - return Poll::Ready(NetworkEvent::IncomingConnection( - IncomingConnectionEvent { - listener_id, - upgrade, - local_addr, - send_back_addr, - pool: &mut self.pool, - })) - } - Poll::Ready(ListenersEvent::NewAddress { listener_id, listen_addr }) => { - return Poll::Ready(NetworkEvent::NewListenerAddress { listener_id, listen_addr }) - } - Poll::Ready(ListenersEvent::AddressExpired { listener_id, listen_addr }) => { - return Poll::Ready(NetworkEvent::ExpiredListenerAddress { listener_id, listen_addr }) - } - Poll::Ready(ListenersEvent::Closed { listener_id, addresses, reason }) => { - return Poll::Ready(NetworkEvent::ListenerClosed { listener_id, addresses, reason }) - } - Poll::Ready(ListenersEvent::Error { listener_id, error }) => { - return Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) + // If the connection pool is closed, the network is considered closed as well. + if self.pool.is_closed() { + return Poll::Ready(None) + } + + // If the pool accepts new connections, poll the listeners stream. + if self.pool.is_open() { + match ListenersStream::poll(Pin::new(&mut self.listeners), cx) { + Poll::Pending => (), + Poll::Ready(ListenersEvent::Incoming { + listener_id, + upgrade, + local_addr, + send_back_addr + }) => { + return Poll::Ready(Some( + NetworkEvent::IncomingConnection( + IncomingConnectionEvent { + listener_id, + upgrade, + local_addr, + send_back_addr, + pool: &mut self.pool, + }))) + } + Poll::Ready(ListenersEvent::NewAddress { listener_id, listen_addr }) => { + return Poll::Ready(Some( + NetworkEvent::NewListenerAddress { + listener_id, listen_addr + })) + } + Poll::Ready(ListenersEvent::AddressExpired { listener_id, listen_addr }) => { + return Poll::Ready(Some( + NetworkEvent::ExpiredListenerAddress { + listener_id, listen_addr + })) + } + Poll::Ready(ListenersEvent::Closed { listener_id, addresses, reason }) => { + return Poll::Ready(Some( + NetworkEvent::ListenerClosed { + listener_id, addresses, reason + })) + } + Poll::Ready(ListenersEvent::Error { listener_id, error }) => { + return Poll::Ready(Some( + NetworkEvent::ListenerError { + listener_id, error + })) + } } } - // Poll the known peers. + // Poll the connection pool. let event = match self.pool.poll(cx) { Poll::Pending => return Poll::Pending, - Poll::Ready(PoolEvent::ConnectionEstablished { connection, num_established }) => { + Poll::Ready(Some( + PoolEvent::ConnectionEstablished { connection, num_established } + )) => { match self.dialing.entry(connection.peer_id().clone()) { hash_map::Entry::Occupied(mut e) => { e.get_mut().retain(|s| s.current.0 != connection.id()); @@ -394,7 +456,9 @@ where num_established, } } - Poll::Ready(PoolEvent::PendingConnectionError { id, endpoint, error, handler, pool, .. }) => { + Poll::Ready(Some( + PoolEvent::PendingConnectionError { id, endpoint, error, handler, pool, .. } + )) => { let dialing = &mut self.dialing; let (next, event) = on_connection_failed(dialing, id, endpoint, error, handler); if let Some(dial) = next { @@ -405,7 +469,9 @@ where } event } - Poll::Ready(PoolEvent::ConnectionClosed { id, connected, error, num_established, .. }) => { + Poll::Ready(Some( + PoolEvent::ConnectionClosed { id, connected, error, num_established, .. } + )) => { NetworkEvent::ConnectionClosed { id, connected, @@ -413,31 +479,36 @@ where error, } } - Poll::Ready(PoolEvent::ConnectionEvent { connection, event }) => { + Poll::Ready(Some( + PoolEvent::ConnectionEvent { connection, event } + )) => { NetworkEvent::ConnectionEvent { connection, event, } } - Poll::Ready(PoolEvent::AddressChange { connection, new_endpoint, old_endpoint }) => { + Poll::Ready(Some( + PoolEvent::AddressChange { connection, new_endpoint, old_endpoint } + )) => { NetworkEvent::AddressChange { connection, new_endpoint, old_endpoint, } } + Poll::Ready(None) => { + // If the connection pool closed, so does the network. + return Poll::Ready(None) + } }; - Poll::Ready(event) + Poll::Ready(Some(event)) } /// Initiates a connection attempt to a known peer. fn dial_peer(&mut self, opts: DialingOpts) -> Result where - TTrans: Transport, - TTrans::Dial: Send + 'static, - TTrans::Error: Send + 'static, TMuxer: Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TInEvent: Send + 'static, diff --git a/core/src/network/peer.rs b/core/src/network/peer.rs index 2966404759a..b07d6feda13 100644 --- a/core/src/network/peer.rs +++ b/core/src/network/peer.rs @@ -165,6 +165,7 @@ where TTrans: Transport + Clone, TTrans::Error: Send + 'static, TTrans::Dial: Send + 'static, + TTrans::ListenerUpgrade: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TInEvent: Send + 'static, @@ -173,7 +174,7 @@ where THandler::Handler: ConnectionHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send, ::OutboundOpenInfo: Send, ::Error: error::Error + Send + 'static, - TConnInfo: fmt::Debug + ConnectionInfo + Send + 'static, + TConnInfo: fmt::Debug + ConnectionInfo + Clone + Send + 'static, TPeerId: Eq + Hash + Clone + Send + 'static, { /// Checks whether the peer is currently connected. diff --git a/core/tests/network_dial_error.rs b/core/tests/network_dial_error.rs index 630eccc01e1..a83ea4cbc23 100644 --- a/core/tests/network_dial_error.rs +++ b/core/tests/network_dial_error.rs @@ -80,7 +80,7 @@ fn deny_incoming_connec() { swarm1.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()).unwrap(); let address = async_std::task::block_on(future::poll_fn(|cx| { - if let Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm1.poll(cx) { + if let Poll::Ready(Some(NetworkEvent::NewListenerAddress { listen_addr, .. })) = swarm1.poll(cx) { Poll::Ready(listen_addr) } else { panic!("Was expecting the listen address to be reported") @@ -94,18 +94,18 @@ fn deny_incoming_connec() { async_std::task::block_on(future::poll_fn(|cx| -> Poll> { match swarm1.poll(cx) { - Poll::Ready(NetworkEvent::IncomingConnection(inc)) => drop(inc), + Poll::Ready(Some(NetworkEvent::IncomingConnection(inc))) => drop(inc), Poll::Ready(_) => unreachable!(), Poll::Pending => (), } match swarm2.poll(cx) { - Poll::Ready(NetworkEvent::DialError { + Poll::Ready(Some(NetworkEvent::DialError { attempts_remaining: 0, peer_id, multiaddr, error: PendingConnectionError::Transport(_) - }) => { + })) => { assert_eq!(peer_id, *swarm1.local_peer_id()); assert_eq!(multiaddr, address); return Poll::Ready(Ok(())); @@ -136,7 +136,7 @@ fn dial_self() { let (local_address, mut swarm) = async_std::task::block_on( future::lazy(move |cx| { - if let Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) = swarm.poll(cx) { + if let Poll::Ready(Some(NetworkEvent::NewListenerAddress { listen_addr, .. })) = swarm.poll(cx) { Ok::<_, void::Void>((listen_addr, swarm)) } else { panic!("Was expecting the listen address to be reported") @@ -151,11 +151,11 @@ fn dial_self() { async_std::task::block_on(future::poll_fn(|cx| -> Poll> { loop { match swarm.poll(cx) { - Poll::Ready(NetworkEvent::UnknownPeerDialError { + Poll::Ready(Some(NetworkEvent::UnknownPeerDialError { multiaddr, error: PendingConnectionError::InvalidPeerId { .. }, .. - }) => { + })) => { assert!(!got_dial_err); assert_eq!(multiaddr, local_address); got_dial_err = true; @@ -163,9 +163,9 @@ fn dial_self() { return Poll::Ready(Ok(())) } }, - Poll::Ready(NetworkEvent::IncomingConnectionError { + Poll::Ready(Some(NetworkEvent::IncomingConnectionError { local_addr, .. - }) => { + })) => { assert!(!got_inc_err); assert_eq!(local_addr, local_address); got_inc_err = true; @@ -173,13 +173,16 @@ fn dial_self() { return Poll::Ready(Ok(())) } }, - Poll::Ready(NetworkEvent::IncomingConnection(inc)) => { + Poll::Ready(Some(NetworkEvent::IncomingConnection(inc))) => { assert_eq!(*inc.local_addr(), local_address); inc.accept(TestHandler()).unwrap(); }, - Poll::Ready(ev) => { + Poll::Ready(Some(ev)) => { panic!("Unexpected event: {:?}", ev) } + Poll::Ready(None) => { + panic!("Unexpected shutdown") + } Poll::Pending => break Poll::Pending, } } @@ -221,12 +224,12 @@ fn multiple_addresses_err() { async_std::task::block_on(future::poll_fn(|cx| -> Poll> { loop { match swarm.poll(cx) { - Poll::Ready(NetworkEvent::DialError { + Poll::Ready(Some(NetworkEvent::DialError { attempts_remaining, peer_id, multiaddr, error: PendingConnectionError::Transport(_) - }) => { + })) => { assert_eq!(peer_id, target); let expected = addresses.remove(0); assert_eq!(multiaddr, expected); diff --git a/examples/ipfs-kad.rs b/examples/ipfs-kad.rs index ec48435db00..3d29ad3afd6 100644 --- a/examples/ipfs-kad.rs +++ b/examples/ipfs-kad.rs @@ -97,10 +97,10 @@ fn main() -> Result<(), Box> { task::block_on(async move { loop { let event = swarm.next().await; - if let KademliaEvent::QueryResult { + if let Some(KademliaEvent::QueryResult { result: QueryResult::GetClosestPeers(result), .. - } = event { + }) = event { match result { Ok(ok) => if !ok.peers.is_empty() { diff --git a/misc/core-derive/src/lib.rs b/misc/core-derive/src/lib.rs index 762a8bc7482..7fc145f5609 100644 --- a/misc/core-derive/src/lib.rs +++ b/misc/core-derive/src/lib.rs @@ -453,6 +453,12 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { std::task::Poll::Ready(#network_behaviour_action::ReportObservedAddr { address }) => { return std::task::Poll::Ready(#network_behaviour_action::ReportObservedAddr { address }); } + std::task::Poll::Ready(#network_behaviour_action::CloseConnection { peer_id, connection_id }) => { + return std::task::Poll::Ready(#network_behaviour_action::CloseConnection { peer_id, connection_id }); + } + std::task::Poll::Ready(#network_behaviour_action::DisconnectPeer { peer_id }) => { + return std::task::Poll::Ready(#network_behaviour_action::DisconnectPeer { peer_id }); + } std::task::Poll::Pending => break, } } diff --git a/misc/core-derive/tests/test.rs b/misc/core-derive/tests/test.rs index 1dfcd84723e..2aadd2d1c32 100644 --- a/misc/core-derive/tests/test.rs +++ b/misc/core-derive/tests/test.rs @@ -300,8 +300,9 @@ fn event_process_false() { // check that the event is bubbled up all the way to swarm let _ = async { match swarm.next().await { - BehaviourOutEvent::Ping(_) => {}, - BehaviourOutEvent::Identify(_) => {}, + Some(BehaviourOutEvent::Ping(_)) => {}, + Some(BehaviourOutEvent::Identify(_)) => {}, + None => panic!("swarm terminated unexpectedly") } }; } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 4f1ae52a8d2..9870c58a2d6 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -1171,6 +1171,12 @@ impl NetworkBehaviour for Gossipsub { NetworkBehaviourAction::ReportObservedAddr { address } => { return Poll::Ready(NetworkBehaviourAction::ReportObservedAddr { address }); } + NetworkBehaviourAction::CloseConnection { peer_id, connection_id } => { + return Poll::Ready(NetworkBehaviourAction::CloseConnection { peer_id, connection_id }) + } + NetworkBehaviourAction::DisconnectPeer { peer_id } => { + return Poll::Ready(NetworkBehaviourAction::DisconnectPeer { peer_id }) + } } } diff --git a/protocols/gossipsub/tests/smoke.rs b/protocols/gossipsub/tests/smoke.rs index 136481f39dc..6df632d8afe 100644 --- a/protocols/gossipsub/tests/smoke.rs +++ b/protocols/gossipsub/tests/smoke.rs @@ -48,17 +48,20 @@ struct Graph { } impl Future for Graph { - type Output = (Multiaddr, GossipsubEvent); + type Output = Option<(Multiaddr, GossipsubEvent)>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { for (addr, node) in &mut self.nodes { match node.poll_next_unpin(cx) { - Poll::Ready(Some(event)) => return Poll::Ready((addr.clone(), event)), - Poll::Ready(None) => panic!("unexpected None when polling nodes"), - Poll::Pending => {} + Poll::Ready(Some(event)) => return Poll::Ready(Some((addr.clone(), event))), + Poll::Ready(None) | Poll::Pending => {}, } } + if self.nodes.iter().all(|(_, s)| Swarm::is_closed(s)) { + return Poll::Ready(None) + } + Poll::Pending } } @@ -118,11 +121,12 @@ impl Graph { let fut = futures::future::poll_fn(move |cx| match &mut this { Some(graph) => loop { match graph.poll_unpin(cx) { - Poll::Ready((_addr, ev)) => { + Poll::Ready(Some((_addr, ev))) => { if f(ev) { - return Poll::Ready(this.take().unwrap()); + graph.nodes.iter_mut().for_each(|(_, s)| Swarm::start_close(s)); } } + Poll::Ready(None) => return Poll::Ready(this.take().unwrap()), Poll::Pending => return Poll::Pending, } }, diff --git a/protocols/identify/src/identify.rs b/protocols/identify/src/identify.rs index dbaf2c07f05..3195b0a3747 100644 --- a/protocols/identify/src/identify.rs +++ b/protocols/identify/src/identify.rs @@ -323,7 +323,7 @@ mod tests { let swarm1_fut = swarm1.next_event(); pin_mut!(swarm1_fut); match swarm1_fut.await { - SwarmEvent::NewListenAddr(addr) => return addr, + Some(SwarmEvent::NewListenAddr(addr)) => return addr, _ => {} } } @@ -342,7 +342,7 @@ mod tests { pin_mut!(swarm2_fut); match future::select(swarm1_fut, swarm2_fut).await.factor_second().0 { - future::Either::Left(IdentifyEvent::Received { info, .. }) => { + future::Either::Left(Some(IdentifyEvent::Received { info, .. })) => { assert_eq!(info.public_key, pubkey2); assert_eq!(info.protocol_version, "c"); assert_eq!(info.agent_version, "d"); @@ -350,7 +350,7 @@ mod tests { assert!(info.listen_addrs.is_empty()); return; } - future::Either::Right(IdentifyEvent::Received { info, .. }) => { + future::Either::Right(Some(IdentifyEvent::Received { info, .. })) => { assert_eq!(info.public_key, pubkey1); assert_eq!(info.protocol_version, "a"); assert_eq!(info.agent_version, "b"); diff --git a/protocols/kad/src/behaviour/test.rs b/protocols/kad/src/behaviour/test.rs index f7c5f97818a..6ca005afd1e 100644 --- a/protocols/kad/src/behaviour/test.rs +++ b/protocols/kad/src/behaviour/test.rs @@ -62,7 +62,7 @@ fn build_node_with_config(cfg: KademliaConfig) -> (Multiaddr, TestSwarm) { .authenticate(SecioConfig::new(local_key)) .multiplex(yamux::Config::default()) .map(|(p, m), _| (p, StreamMuxerBox::new(m))) - .map_err(|e| -> io::Error { panic!("Failed to create transport: {:?}", e); }) + .map_err(|e| -> io::Error { io::Error::new(io::ErrorKind::Other, e.to_string()) }) .boxed(); let local_id = local_public_key.clone().into_peer_id(); @@ -179,43 +179,55 @@ fn bootstrap() { let expected_known = swarm_ids.iter().skip(1).cloned().collect::>(); let mut first = true; + let mut success = false; + // Run test - block_on( - poll_fn(move |ctx| { - for (i, swarm) in swarms.iter_mut().enumerate() { - loop { - match swarm.poll_next_unpin(ctx) { - Poll::Ready(Some(KademliaEvent::QueryResult { - id, result: QueryResult::Bootstrap(Ok(ok)), .. - })) => { - assert_eq!(id, qid); - assert_eq!(i, 0); - if first { - // Bootstrapping must start with a self-lookup. - assert_eq!(ok.peer, swarm_ids[0]); - } - first = false; - if ok.num_remaining == 0 { - let mut known = HashSet::new(); - for b in swarm.kbuckets.iter() { - for e in b.iter() { - known.insert(e.node.key.preimage().clone()); - } + block_on(poll_fn(move |ctx| { + for (i, swarm) in swarms.iter_mut().enumerate() { + loop { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::QueryResult { + id, result: QueryResult::Bootstrap(Ok(ok)), .. + })) => { + assert_eq!(id, qid); + assert_eq!(i, 0); + if first { + // Bootstrapping must start with a self-lookup. + assert_eq!(ok.peer, swarm_ids[0]); + } + first = false; + if ok.num_remaining == 0 { + let mut known = HashSet::new(); + for b in swarm.kbuckets.iter() { + for e in b.iter() { + known.insert(e.node.key.preimage().clone()); } - assert_eq!(expected_known, known); - return Poll::Ready(()) } + assert_eq!(expected_known, known); + success = true; + Swarm::start_close(swarm); } - // Ignore any other event. - Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, } + // Ignore any other event. + Poll::Ready(Some(_)) => (), + Poll::Ready(None) => break, + Poll::Pending => + if success && !Swarm::is_closing(swarm) { + Swarm::start_close(swarm) + } else { + break + } } } - Poll::Pending - }) - ) + } + + if swarms.iter().all(Swarm::is_closed) { + assert!(success); + return Poll::Ready(()) + } + + Poll::Pending + })) } QuickCheck::new().tests(10).quickcheck(prop as fn(_) -> _) @@ -259,34 +271,46 @@ fn query_iter() { let mut expected_distances = distances(&search_target_key, expected_peer_ids.clone()); expected_distances.sort(); + let mut success = false; + // Run test - block_on( - poll_fn(move |ctx| { - for (i, swarm) in swarms.iter_mut().enumerate() { - loop { - match swarm.poll_next_unpin(ctx) { - Poll::Ready(Some(KademliaEvent::QueryResult { - id, result: QueryResult::GetClosestPeers(Ok(ok)), .. - })) => { - assert_eq!(id, qid); - assert_eq!(&ok.key[..], search_target.as_bytes()); - assert_eq!(swarm_ids[i], expected_swarm_id); - assert_eq!(swarm.queries.size(), 0); - assert!(expected_peer_ids.iter().all(|p| ok.peers.contains(p))); - let key = kbucket::Key::new(ok.key); - assert_eq!(expected_distances, distances(&key, ok.peers)); - return Poll::Ready(()); - } - // Ignore any other event. - Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, + block_on(poll_fn(move |ctx| { + for (i, swarm) in swarms.iter_mut().enumerate() { + loop { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::QueryResult { + id, result: QueryResult::GetClosestPeers(Ok(ok)), .. + })) => { + assert_eq!(id, qid); + assert_eq!(&ok.key[..], search_target.as_bytes()); + assert_eq!(swarm_ids[i], expected_swarm_id); + assert_eq!(swarm.queries.size(), 0); + assert!(expected_peer_ids.iter().all(|p| ok.peers.contains(p))); + let key = kbucket::Key::new(ok.key); + assert_eq!(expected_distances, distances(&key, ok.peers)); + success = true; + Swarm::start_close(swarm); } + // Ignore any other event. + Poll::Ready(Some(_)) => (), + Poll::Ready(None) => break, + Poll::Pending => + if success && !Swarm::is_closing(swarm) { + Swarm::start_close(swarm); + } else { + break + } } } - Poll::Pending - }) - ) + } + + if swarms.iter().all(Swarm::is_closed) { + assert!(success); + return Poll::Ready(()) + } + + Poll::Pending + })) } let mut rng = thread_rng(); @@ -323,11 +347,11 @@ fn unresponsive_not_returned_direct() { })) => { assert_eq!(&ok.key[..], search_target.as_bytes()); assert_eq!(ok.peers.len(), 0); - return Poll::Ready(()); + Swarm::start_close(swarm); } // Ignore any other event. Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Ready(None) => return Poll::Ready(()), Poll::Pending => break, } } @@ -374,11 +398,11 @@ fn unresponsive_not_returned_indirect() { assert_eq!(&ok.key[..], search_target.as_bytes()); assert_eq!(ok.peers.len(), 1); assert_eq!(ok.peers[0], first_peer_id); - return Poll::Ready(()); + Swarm::start_close(swarm); } // Ignore any other event. Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), + Poll::Ready(None) => return Poll::Ready(()), Poll::Pending => break, } } @@ -408,36 +432,40 @@ fn get_record_not_found() { let target_key = record::Key::from(random_multihash()); let qid = swarms[0].get_record(&target_key, Quorum::One); - block_on( - poll_fn(move |ctx| { - for swarm in &mut swarms { - loop { - match swarm.poll_next_unpin(ctx) { - Poll::Ready(Some(KademliaEvent::QueryResult { - id, result: QueryResult::GetRecord(Err(e)), .. - })) => { - assert_eq!(id, qid); - if let GetRecordError::NotFound { key, closest_peers, } = e { - assert_eq!(key, target_key); - assert_eq!(closest_peers.len(), 2); - assert!(closest_peers.contains(&swarm_ids[1])); - assert!(closest_peers.contains(&swarm_ids[2])); - return Poll::Ready(()); - } else { - panic!("Unexpected error result: {:?}", e); - } + let mut success = false; + + block_on(poll_fn(move |ctx| { + for swarm in &mut swarms { + loop { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::QueryResult { + id, result: QueryResult::GetRecord(Err(e)), .. + })) => { + assert_eq!(id, qid); + if let GetRecordError::NotFound { key, closest_peers, } = e { + assert_eq!(key, target_key); + assert_eq!(closest_peers.len(), 2); + assert!(closest_peers.contains(&swarm_ids[1])); + assert!(closest_peers.contains(&swarm_ids[2])); + success = true; + Swarm::start_close(swarm); + } else { + panic!("Unexpected error result: {:?}", e); } - // Ignore any other event. - Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, + } + Poll::Pending => break, + // Ignore any other event. + Poll::Ready(Some(_)) => (), + Poll::Ready(None) => { + assert!(success); + return Poll::Ready(()) } } } + } - Poll::Pending - }) - ) + Poll::Pending + })) } /// A node joining a fully connected network via three (ALPHA_VALUE) bootnodes @@ -514,6 +542,8 @@ fn put_record() { // The accumulated results for one round of publishing. let mut results = Vec::new(); + let mut success = false; + block_on( poll_fn(move |ctx| loop { // Poll all swarms until they are "Pending". @@ -542,15 +572,14 @@ fn put_record() { } // Ignore any other event. Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, + Poll::Ready(None) | Poll::Pending => break, } } } // All swarms are Pending and not enough results have been collected // so far, thus wait to be polled again for further progress. - if results.len() != records.len() { + if !success && results.len() != records.len() { return Poll::Pending } @@ -608,14 +637,21 @@ fn put_record() { } if republished { - assert_eq!(swarms[0].store.records().count(), records.len()); - assert_eq!(swarms[0].queries.size(), 0); - for k in records.keys() { - swarms[0].store.remove(&k); + if !success { + assert_eq!(swarms[0].store.records().count(), records.len()); + assert_eq!(swarms[0].queries.size(), 0); + for k in records.keys() { + swarms[0].store.remove(&k); + } + assert_eq!(swarms[0].store.records().count(), 0); + // All records have been republished, thus the test is complete. + swarms.iter_mut().for_each(Swarm::start_close); + success = true; + } + if swarms.iter().all(Swarm::is_closed) { + assert!(success); + return Poll::Ready(()) } - assert_eq!(swarms[0].store.records().count(), 0); - // All records have been republished, thus the test is complete. - return Poll::Ready(()); } // Tell the replication job to republish asap. @@ -646,32 +682,44 @@ fn get_record() { swarms[1].store.put(record.clone()).unwrap(); let qid = swarms[0].get_record(&record.key, Quorum::One); - block_on( - poll_fn(move |ctx| { - for swarm in &mut swarms { - loop { - match swarm.poll_next_unpin(ctx) { - Poll::Ready(Some(KademliaEvent::QueryResult { - id, - result: QueryResult::GetRecord(Ok(GetRecordOk { records })), - .. - })) => { - assert_eq!(id, qid); - assert_eq!(records.len(), 1); - assert_eq!(records.first().unwrap().record, record); - return Poll::Ready(()); - } - // Ignore any other event. - Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, + let mut success = false; + + block_on(poll_fn(move |ctx| { + for swarm in &mut swarms { + loop { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::QueryResult { + id, + result: QueryResult::GetRecord(Ok(GetRecordOk { records })), + .. + })) => { + assert!(!success); + assert_eq!(id, qid); + assert_eq!(records.len(), 1); + assert_eq!(records.first().unwrap().record, record); + success = true; + Swarm::start_close(swarm); } + // Ignore any other event. + Poll::Ready(Some(..)) => {}, + Poll::Pending => + if success && !Swarm::is_closing(swarm) { + Swarm::start_close(swarm); + } else { + break + }, + Poll::Ready(None) => break } } + } - Poll::Pending - }) - ) + if swarms.iter().all(Swarm::is_closed) { + assert!(success); + return Poll::Ready(()) + } + + Poll::Pending + })); } #[test] @@ -692,31 +740,43 @@ fn get_record_many() { let quorum = Quorum::N(NonZeroUsize::new(num_results).unwrap()); let qid = swarms[0].get_record(&record.key, quorum); - block_on( - poll_fn(move |ctx| { - for swarm in &mut swarms { - loop { - match swarm.poll_next_unpin(ctx) { - Poll::Ready(Some(KademliaEvent::QueryResult { - id, - result: QueryResult::GetRecord(Ok(GetRecordOk { records })), - .. - })) => { - assert_eq!(id, qid); - assert_eq!(records.len(), num_results); - assert_eq!(records.first().unwrap().record, record); - return Poll::Ready(()); - } - // Ignore any other event. - Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, + let mut success = false; + + block_on(poll_fn(move |ctx| { + for swarm in &mut swarms { + loop { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::QueryResult { + id, + result: QueryResult::GetRecord(Ok(GetRecordOk { records })), + .. + })) => { + assert_eq!(id, qid); + assert_eq!(records.len(), num_results); + assert_eq!(records.first().unwrap().record, record); + success = true; + Swarm::start_close(swarm); } + // Ignore any other event. + Poll::Ready(Some(_)) => (), + Poll::Ready(None) => break, + Poll::Pending => + if success && !Swarm::is_closing(swarm) { + Swarm::start_close(swarm); + } else { + break + } } } - Poll::Pending - }) - ) + } + + if swarms.iter().all(Swarm::is_closed) { + assert!(success); + return Poll::Ready(()) + } + + Poll::Pending + })); } /// A node joining a fully connected network via three (ALPHA_VALUE) bootnodes @@ -773,106 +833,112 @@ fn add_provider() { qids.insert(qid); } - block_on( - poll_fn(move |ctx| loop { - // Poll all swarms until they are "Pending". - for swarm in &mut swarms { - loop { - match swarm.poll_next_unpin(ctx) { - Poll::Ready(Some(KademliaEvent::QueryResult { - id, result: QueryResult::StartProviding(res), .. - })) | - Poll::Ready(Some(KademliaEvent::QueryResult { - id, result: QueryResult::RepublishProvider(res), .. - })) => { - assert!(qids.is_empty() || qids.remove(&id)); - match res { - Err(e) => panic!(e), - Ok(ok) => { - assert!(keys.contains(&ok.key)); - results.push(ok.key); - } + let mut success = false; + + block_on(poll_fn(move |ctx| loop { + // Poll all swarms until they are "Pending". + for swarm in &mut swarms { + loop { + match swarm.poll_next_unpin(ctx) { + Poll::Ready(Some(KademliaEvent::QueryResult { + id, result: QueryResult::StartProviding(res), .. + })) | + Poll::Ready(Some(KademliaEvent::QueryResult { + id, result: QueryResult::RepublishProvider(res), .. + })) => { + assert!(qids.is_empty() || qids.remove(&id)); + match res { + Err(e) => panic!(e), + Ok(ok) => { + assert!(keys.contains(&ok.key)); + results.push(ok.key); } } - // Ignore any other event. - Poll::Ready(Some(_)) => (), - e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), - Poll::Pending => break, } + // Ignore any other event. + Poll::Ready(Some(_)) => (), + Poll::Ready(None) | Poll::Pending => break, } } + } - if results.len() == keys.len() { - // All requests have been sent for one round of publishing. - published = true - } + if results.len() == keys.len() { + // All requests have been sent for one round of publishing. + published = true + } + + if !success && !published { + // Still waiting for all requests to be sent for one round + // of publishing. + return Poll::Pending + } - if !published { - // Still waiting for all requests to be sent for one round - // of publishing. + // A round of publishing is complete. Consume the results, checking that + // each key was published to the `replication_factor` closest peers. + while let Some(key) = results.pop() { + // Collect the nodes that have a provider record for `key`. + let actual = swarms.iter().skip(1) + .filter_map(|swarm| + if swarm.store.providers(&key).len() == 1 { + Some(Swarm::local_peer_id(&swarm).clone()) + } else { + None + }) + .collect::>(); + + if actual.len() != replication_factor.get() { + // Still waiting for some nodes to process the request. + results.push(key); return Poll::Pending } - // A round of publishing is complete. Consume the results, checking that - // each key was published to the `replication_factor` closest peers. - while let Some(key) = results.pop() { - // Collect the nodes that have a provider record for `key`. - let actual = swarms.iter().skip(1) - .filter_map(|swarm| - if swarm.store.providers(&key).len() == 1 { - Some(Swarm::local_peer_id(&swarm).clone()) - } else { - None - }) - .collect::>(); - - if actual.len() != replication_factor.get() { - // Still waiting for some nodes to process the request. - results.push(key); - return Poll::Pending - } - - let mut expected = swarms.iter() - .skip(1) - .map(Swarm::local_peer_id) - .cloned() - .collect::>(); - let kbucket_key = kbucket::Key::new(key); - expected.sort_by(|id1, id2| - kbucket::Key::new(id1.clone()).distance(&kbucket_key).cmp( - &kbucket::Key::new(id2.clone()).distance(&kbucket_key))); - - let expected = expected - .into_iter() - .take(replication_factor.get()) - .collect::>(); - - assert_eq!(actual, expected); - } + let mut expected = swarms.iter() + .skip(1) + .map(Swarm::local_peer_id) + .cloned() + .collect::>(); + let kbucket_key = kbucket::Key::new(key); + expected.sort_by(|id1, id2| + kbucket::Key::new(id1.clone()).distance(&kbucket_key).cmp( + &kbucket::Key::new(id2.clone()).distance(&kbucket_key))); + + let expected = expected + .into_iter() + .take(replication_factor.get()) + .collect::>(); + + assert_eq!(actual, expected); + } - // One round of publishing is complete. - assert!(results.is_empty()); - for swarm in &swarms { - assert_eq!(swarm.queries.size(), 0); - } + // One round of publishing is complete. + assert!(results.is_empty()); + for swarm in &swarms { + assert_eq!(swarm.queries.size(), 0); + } - if republished { + if republished { + // All records have been republished, thus the test is complete. + if !success { assert_eq!(swarms[0].store.provided().count(), keys.len()); for k in &keys { swarms[0].stop_providing(&k); } assert_eq!(swarms[0].store.provided().count(), 0); - // All records have been republished, thus the test is complete. + success = true; + swarms.iter_mut().for_each(Swarm::start_close); + } + if swarms.iter().all(Swarm::is_closed) { + assert!(success); return Poll::Ready(()); } + } - // Initiate the second round of publishing by telling the - // periodic provider job to run asap. - swarms[0].add_provider_job.as_mut().unwrap().asap(); - published = false; - republished = true; - }) - ) + // Initiate the second round of publishing by telling the + // periodic provider job to run asap. + swarms[0].add_provider_job.as_mut().unwrap().asap(); + published = false; + republished = true; + })) } QuickCheck::new().tests(3).quickcheck(prop as fn(_,_)) @@ -891,25 +957,23 @@ fn exceed_jobs_max_queries() { assert_eq!(swarm.queries.size(), num); - block_on( - poll_fn(move |ctx| { - for _ in 0 .. num { - // There are no other nodes, so the queries finish instantly. - if let Poll::Ready(Some(e)) = swarm.poll_next_unpin(ctx) { - if let KademliaEvent::QueryResult { - result: QueryResult::GetClosestPeers(Ok(r)), .. - } = e { - assert!(r.peers.is_empty()) - } else { - panic!("Unexpected event: {:?}", e) - } + block_on(poll_fn(move |ctx| { + for _ in 0 .. num { + // There are no other nodes, so the queries finish instantly. + if let Poll::Ready(Some(e)) = swarm.poll_next_unpin(ctx) { + if let KademliaEvent::QueryResult { + result: QueryResult::GetClosestPeers(Ok(r)), .. + } = e { + assert!(r.peers.is_empty()) } else { - panic!("Expected event") + panic!("Unexpected event: {:?}", e) } + } else { + panic!("Expected event") } - Poll::Ready(()) - }) - ) + } + Poll::Ready(()) + })) } #[test] @@ -1011,9 +1075,11 @@ fn disjoint_query_does_not_finish_before_all_paths_did() { } }); + let mut records = Vec::new(); + // Poll `alice` and `bob` expecting `alice` to return a successful query // result as it is now able to explore the second disjoint path. - let records = block_on( + block_on( poll_fn(|ctx| { for (i, swarm) in [&mut alice, &mut bob].iter_mut().enumerate() { loop { @@ -1027,20 +1093,31 @@ fn disjoint_query_does_not_finish_before_all_paths_did() { } match result { - Ok(ok) => return Poll::Ready(ok.records), + Ok(ok) => { + records = ok.records; + Swarm::start_close(swarm); + }, Err(e) => unreachable!("{:?}", e), } } // Ignore any other event. Poll::Ready(Some(_)) => (), - Poll::Ready(None) => panic!( - "Expected Kademlia behaviour not to finish.", - ), - Poll::Pending => break, + Poll::Ready(None) => break, + Poll::Pending => { + if !records.is_empty() && !Swarm::is_closing(swarm) { + Swarm::start_close(swarm) + } else { + break + } + } } } } + if [&alice, &bob].iter().all(|s| Swarm::is_closed(*s)) { + return Poll::Ready(()) + } + Poll::Pending }) ); @@ -1074,6 +1151,7 @@ fn manual_bucket_inserts() { let mut routable = Vec::new(); // Start an iterative query from the first peer. swarms[0].1.get_closest_peers(PeerId::random()); + let mut success = false; block_on(poll_fn(move |ctx| { for (_, swarm) in swarms.iter_mut() { loop { @@ -1088,14 +1166,27 @@ fn manual_bucket_inserts() { let bucket = swarm.kbucket(peer.clone()).unwrap(); assert!(bucket.iter().all(|e| e.node.key.preimage() != peer)); } - return Poll::Ready(()) + success = true; + Swarm::start_close(swarm); } } - Poll::Ready(..) => {}, - Poll::Pending => break + Poll::Ready(Some(_)) => {}, + Poll::Ready(None) => break, + Poll::Pending => + if success && !Swarm::is_closing(swarm) { + Swarm::start_close(swarm) + } else { + break + } } } } + + if swarms.iter().all(|(_, s)| Swarm::is_closed(s)) { + assert!(success); + return Poll::Ready(()) + } + Poll::Pending })); } diff --git a/protocols/ping/tests/ping.rs b/protocols/ping/tests/ping.rs index 30e8de601ee..4f36673e3e3 100644 --- a/protocols/ping/tests/ping.rs +++ b/protocols/ping/tests/ping.rs @@ -60,7 +60,7 @@ fn ping() { loop { match swarm1.next().await { - PingEvent { peer, result: Ok(PingSuccess::Ping { rtt }) } => { + Some(PingEvent { peer, result: Ok(PingSuccess::Ping { rtt }) }) => { return (pid1.clone(), peer, rtt) }, _ => {} @@ -74,7 +74,7 @@ fn ping() { loop { match swarm2.next().await { - PingEvent { peer, result: Ok(PingSuccess::Ping { rtt }) } => { + Some(PingEvent { peer, result: Ok(PingSuccess::Ping { rtt }) }) => { return (pid2.clone(), peer, rtt) }, _ => {} diff --git a/protocols/request-response/src/handler.rs b/protocols/request-response/src/handler.rs index 17eff4db913..c34441e4764 100644 --- a/protocols/request-response/src/handler.rs +++ b/protocols/request-response/src/handler.rs @@ -86,7 +86,7 @@ where impl RequestResponseHandler where - TCodec: RequestResponseCodec, + TCodec: RequestResponseCodec + Clone + Send + 'static, { pub(super) fn new( inbound_protocols: SmallVec<[TCodec::Protocol; 2]>, @@ -106,6 +106,30 @@ where pending_error: None, } } + + fn poll_inbound(&mut self, cx: &mut Context<'_>) + -> Poll> + { + while let Poll::Ready(Some(result)) = self.inbound.poll_next_unpin(cx) { + match result { + Ok((rq, rs_sender)) => { + // We received an inbound request. + self.keep_alive = KeepAlive::Yes; + return Poll::Ready( + RequestResponseHandlerEvent::Request { + 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 `inject_listen_upgrade_error`. + } + } + } + + Poll::Pending + } } /// The events emitted by the [`RequestResponseHandler`]. @@ -132,6 +156,8 @@ where InboundTimeout, /// An inbound request failed to negotiate a mutually supported protocol. InboundUnsupportedProtocols, + /// An outbound request could not be sent because the connection is closing. + Closing(RequestProtocol), } impl ProtocolsHandler for RequestResponseHandler @@ -256,10 +282,7 @@ where self.keep_alive } - fn poll( - &mut self, - cx: &mut Context<'_>, - ) -> Poll< + fn poll(&mut self, cx: &mut Context<'_>) -> Poll< ProtocolsHandlerEvent, RequestId, Self::OutEvent, Self::Error>, > { // Check for a pending (fatal) error. @@ -276,22 +299,8 @@ where } // Check for inbound requests. - while let Poll::Ready(Some(result)) = self.inbound.poll_next_unpin(cx) { - match result { - Ok((rq, rs_sender)) => { - // We received an inbound request. - self.keep_alive = KeepAlive::Yes; - return Poll::Ready(ProtocolsHandlerEvent::Custom( - RequestResponseHandlerEvent::Request { - 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 `inject_listen_upgrade_error`. - } - } + if let Poll::Ready(event) = self.poll_inbound(cx) { + return Poll::Ready(ProtocolsHandlerEvent::Custom(event)) } // Emit outbound requests. @@ -322,5 +331,37 @@ where Poll::Pending } + + fn poll_close(&mut self, cx: &mut Context<'_>) -> Poll< + Result, Self::Error> + > { + // Check for a pending (fatal) error. + if let Some(err) = self.pending_error.take() { + // The handler will not be polled again by the `Swarm`. + return Poll::Ready(Err(err)) + } + + // Drain pending events. + if let Some(event) = self.pending_events.pop_front() { + return Poll::Ready(Ok(Some(event))) + } + + // Drain remaining inbound requests. New inbound requests + // will not be received. + if let Poll::Ready(event) = self.poll_inbound(cx) { + return Poll::Ready(Ok(Some(event))) + } + + // Deny new outbound requests so they can be rescheduled on + // a different connection. + if let Some(request) = self.outbound.pop_front() { + return Poll::Ready(Ok(Some(RequestResponseHandlerEvent::Closing(request)))) + } + + // Ongoing inbound and outbound upgrades are always allowed to + // complete, thus there is nothing else that needs to delay + // connection shutdown. + return Poll::Ready(Ok(None)) + } } diff --git a/protocols/request-response/src/lib.rs b/protocols/request-response/src/lib.rs index 25b69d7ad08..7d016b9f235 100644 --- a/protocols/request-response/src/lib.rs +++ b/protocols/request-response/src/lib.rs @@ -398,7 +398,7 @@ where /// Tries to send a request by queueing an appropriate event to be /// emitted to the `Swarm`. If the peer is not currently connected, - /// the given request is return unchanged. + /// the given request is returned unchanged. fn try_send_request(&mut self, peer: &PeerId, request: RequestProtocol) -> Option> { @@ -520,7 +520,7 @@ where fn inject_event( &mut self, peer: PeerId, - _: ConnectionId, + conn: ConnectionId, event: RequestResponseHandlerEvent, ) { match event { @@ -574,6 +574,30 @@ where error: InboundFailure::UnsupportedProtocols, })); } + RequestResponseHandlerEvent::Closing(request) => { + if let Some((req_peer, req_conn)) = self.pending_responses.remove(&request.request_id) { + debug_assert_eq!(req_peer, peer); + debug_assert_eq!(req_conn, conn); + // Try to send the request on a different connection. + if let Some(conn) = self.connected.get(&peer).and_then(|conns| + conns.iter().find(|c| c.id != conn) + ) { + self.pending_responses.insert(request.request_id, (peer, conn.id)); + self.pending_events.push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: req_peer, + handler: NotifyHandler::One(conn.id), + event: request + }); + } else { + // There is no other existing connection to use, so request a new one. + self.pending_events.push_back(NetworkBehaviourAction::DialPeer { + peer_id: peer.clone(), + condition: DialPeerCondition::Disconnected, + }); + self.pending_requests.entry(peer).or_default().push(request); + } + } + } } } diff --git a/protocols/request-response/tests/ping.rs b/protocols/request-response/tests/ping.rs index 107a37edf04..6e8cfde22b1 100644 --- a/protocols/request-response/tests/ping.rs +++ b/protocols/request-response/tests/ping.rs @@ -72,10 +72,10 @@ fn ping_protocol() { loop { match swarm1.next().await { - RequestResponseEvent::Message { + Some(RequestResponseEvent::Message { peer, message: RequestResponseMessage::Request { request, channel } - } => { + }) => { assert_eq!(&request, &expected_ping); assert_eq!(&peer, &peer2_id); swarm1.send_response(channel, pong.clone()); @@ -93,10 +93,10 @@ fn ping_protocol() { loop { match swarm2.next().await { - RequestResponseEvent::Message { + Some(RequestResponseEvent::Message { peer, message: RequestResponseMessage::Response { request_id, response } - } => { + }) => { count += 1; assert_eq!(&response, &expected_pong); assert_eq!(&peer, &peer1_id); diff --git a/swarm/src/behaviour.rs b/swarm/src/behaviour.rs index 93d2df0960f..2b42c885aa8 100644 --- a/swarm/src/behaviour.rs +++ b/swarm/src/behaviour.rs @@ -279,6 +279,17 @@ pub enum NetworkBehaviourAction { /// The observed address of the local node. address: Multiaddr, }, + + /// Instructs the `Swarm` to initiate a shutdown of a connection. + CloseConnection { + peer_id: PeerId, + connection_id: ConnectionId + }, + + /// Instructs the `Swarm to immediately disconnect a peer. + DisconnectPeer { + peer_id: PeerId, + } } /// The options w.r.t. which connection handlers to notify of an event. diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index f2ad226664c..bf0369d31cc 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -289,7 +289,10 @@ where /// Pending event to be delivered to connection handlers /// (or dropped if the peer disconnected) before the `behaviour` /// can be polled again. - pending_event: Option<(PeerId, PendingNotifyHandler, TInEvent)> + pending_event: Option<(PeerId, PendingNotifyHandler, TInEvent)>, + + /// Whether the `Swarm` has been closed, i.e. completed a clean shutdown. + closed: bool, } impl Deref for @@ -461,20 +464,66 @@ where TBehaviour: NetworkBehaviour, me.banned_peers.remove(&peer_id); } + /// Starts closing down the `Swarm` and underlying `Network`. + /// + /// Closing down a `Swarm` proceeds as follows: + /// + /// 1. The underlying `Network` drains all established connections + /// while rejecting new connections. Established connections are + /// drained by asking the associated [`ProtocolsHandler`] to close, + /// after which the underlying connection is closed. + /// + /// 2. The [`NetworkBehaviour`] associated with the `Swarm` continues + /// to be polled once the `Network` closed until it returns + /// `Poll::Pending`, indicating that further network I/O would be + /// needed for the behaviour to make progress. In this way, the + /// behaviour can finish any work and emit resulting events. + /// + /// After calling this method, the `Swarm` must be `poll()`ed to + /// drive the shutdown to completion. + pub fn start_close(me: &mut Self) { + me.network.start_close() + } + + /// Closes the `Swarm`, ignoring any further events. + /// + /// See [`Self::start_close`] for details. + pub async fn close(&mut self) { + self.network.start_close(); + while let Some(_) = self.next_event().await {} + } + + /// Whether the `Swarm` is closed. + /// + /// When the `Swarm` is closed, neither the underlying `Network` + /// nor the associated `NetworkBehaviour` are polled again. + pub fn is_closed(me: &Self) -> bool { + me.closed + } + + /// Whether the `Swarm` is closing. + /// + /// While the `Swarm` is closing, new connections are rejected + /// while established connections and the `NetworkBehaviour` + /// are drained until no more work is to be done. + pub fn is_closing(me: &Self) -> bool { + me.network.is_closing() || (me.network.is_closed() && !me.closed) + } + /// Returns the next event that happens in the `Swarm`. /// /// Includes events from the `NetworkBehaviour` but also events about the connections status. - pub async fn next_event(&mut self) -> SwarmEvent { + pub async fn next_event(&mut self) -> Option> { future::poll_fn(move |cx| ExpandedSwarm::poll_next_event(Pin::new(self), cx)).await } /// Returns the next event produced by the [`NetworkBehaviour`]. - pub async fn next(&mut self) -> TBehaviour::OutEvent { + pub async fn next(&mut self) -> Option { future::poll_fn(move |cx| { loop { let event = futures::ready!(ExpandedSwarm::poll_next_event(Pin::new(self), cx)); - if let SwarmEvent::Behaviour(event) = event { - return Poll::Ready(event); + if let Some(SwarmEvent::Behaviour(event)) = event { + return Poll::Ready(Some(event)); } } }).await @@ -484,29 +533,44 @@ where TBehaviour: NetworkBehaviour, /// /// Polls the `Swarm` for the next event. fn poll_next_event(mut self: Pin<&mut Self>, cx: &mut Context<'_>) - -> Poll> + -> Poll>> { // We use a `this` variable because the compiler can't mutably borrow multiple times // across a `Deref`. let this = &mut *self; loop { + if this.closed { + return Poll::Ready(None) + } + let mut network_not_ready = false; // First let the network make progress. match this.network.poll(cx) { Poll::Pending => network_not_ready = true, - Poll::Ready(NetworkEvent::ConnectionEvent { connection, event }) => { + Poll::Ready(None) => { + // The network closed, but the behaviour may still be + // doing work and produce events. Thus we wait until it + // returns `Pending`. + } + Poll::Ready(Some( + NetworkEvent::ConnectionEvent { connection, event } + )) => { let peer = connection.peer_id().clone(); let connection = connection.id(); this.behaviour.inject_event(peer, connection, event); }, - Poll::Ready(NetworkEvent::AddressChange { connection, new_endpoint, old_endpoint }) => { + Poll::Ready(Some( + NetworkEvent::AddressChange { connection, new_endpoint, old_endpoint } + )) => { let peer = connection.peer_id(); let connection = connection.id(); this.behaviour.inject_address_change(&peer, &connection, &old_endpoint, &new_endpoint); }, - Poll::Ready(NetworkEvent::ConnectionEstablished { connection, num_established }) => { + Poll::Ready(Some( + NetworkEvent::ConnectionEstablished { connection, num_established } + )) => { let peer_id = connection.peer_id().clone(); let endpoint = connection.endpoint().clone(); if this.banned_peers.contains(&peer_id) { @@ -514,10 +578,10 @@ where TBehaviour: NetworkBehaviour, .into_connected() .expect("the Network just notified us that we were connected; QED") .disconnect(); - return Poll::Ready(SwarmEvent::BannedPeer { + return Poll::Ready(Some(SwarmEvent::BannedPeer { peer_id, endpoint, - }); + })); } else { log::debug!("Connection established: {:?}; Total (peer): {}.", connection.connected(), num_established); @@ -526,12 +590,14 @@ where TBehaviour: NetworkBehaviour, if num_established.get() == 1 { this.behaviour.inject_connected(&peer_id); } - return Poll::Ready(SwarmEvent::ConnectionEstablished { + return Poll::Ready(Some(SwarmEvent::ConnectionEstablished { peer_id, num_established, endpoint - }); + })); } }, - Poll::Ready(NetworkEvent::ConnectionClosed { id, connected, error, num_established }) => { + Poll::Ready(Some( + NetworkEvent::ConnectionClosed { id, connected, error, num_established } + )) => { if let Some(error) = error.as_ref() { log::debug!("Connection {:?} closed: {:?}", connected, error); } else { @@ -543,40 +609,46 @@ where TBehaviour: NetworkBehaviour, if num_established == 0 { this.behaviour.inject_disconnected(info.peer_id()); } - return Poll::Ready(SwarmEvent::ConnectionClosed { + return Poll::Ready(Some(SwarmEvent::ConnectionClosed { peer_id: info.peer_id().clone(), endpoint, cause: error, num_established, - }); + })); }, - Poll::Ready(NetworkEvent::IncomingConnection(incoming)) => { + Poll::Ready(Some(NetworkEvent::IncomingConnection(incoming))) => { let handler = this.behaviour.new_handler(); let local_addr = incoming.local_addr().clone(); let send_back_addr = incoming.send_back_addr().clone(); if let Err(e) = incoming.accept(handler.into_node_handler_builder()) { log::warn!("Incoming connection rejected: {:?}", e); } - return Poll::Ready(SwarmEvent::IncomingConnection { + return Poll::Ready(Some(SwarmEvent::IncomingConnection { local_addr, send_back_addr, - }); + })); }, - Poll::Ready(NetworkEvent::NewListenerAddress { listener_id, listen_addr }) => { + Poll::Ready(Some( + NetworkEvent::NewListenerAddress { listener_id, listen_addr } + )) => { log::debug!("Listener {:?}; New address: {:?}", listener_id, listen_addr); if !this.listened_addrs.contains(&listen_addr) { this.listened_addrs.push(listen_addr.clone()) } this.behaviour.inject_new_listen_addr(&listen_addr); - return Poll::Ready(SwarmEvent::NewListenAddr(listen_addr)); + return Poll::Ready(Some(SwarmEvent::NewListenAddr(listen_addr))); } - Poll::Ready(NetworkEvent::ExpiredListenerAddress { listener_id, listen_addr }) => { + Poll::Ready(Some( + NetworkEvent::ExpiredListenerAddress { listener_id, listen_addr } + )) => { log::debug!("Listener {:?}; Expired address {:?}.", listener_id, listen_addr); this.listened_addrs.retain(|a| a != &listen_addr); this.behaviour.inject_expired_listen_addr(&listen_addr); - return Poll::Ready(SwarmEvent::ExpiredListenAddr(listen_addr)); + return Poll::Ready(Some(SwarmEvent::ExpiredListenAddr(listen_addr))); } - Poll::Ready(NetworkEvent::ListenerClosed { listener_id, addresses, reason }) => { + Poll::Ready(Some( + NetworkEvent::ListenerClosed { listener_id, addresses, reason } + )) => { log::debug!("Listener {:?}; Closed by {:?}.", listener_id, reason); for addr in addresses.iter() { this.behaviour.inject_expired_listen_addr(addr); @@ -585,26 +657,32 @@ where TBehaviour: NetworkBehaviour, Ok(()) => Ok(()), Err(err) => Err(err), }); - return Poll::Ready(SwarmEvent::ListenerClosed { + return Poll::Ready(Some(SwarmEvent::ListenerClosed { addresses, reason, - }); + })); } - Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) => { + Poll::Ready(Some( + NetworkEvent::ListenerError { listener_id, error } + )) => { this.behaviour.inject_listener_error(listener_id, &error); - return Poll::Ready(SwarmEvent::ListenerError { + return Poll::Ready(Some(SwarmEvent::ListenerError { error, - }); + })); }, - Poll::Ready(NetworkEvent::IncomingConnectionError { local_addr, send_back_addr, error }) => { + Poll::Ready(Some( + NetworkEvent::IncomingConnectionError { local_addr, send_back_addr, error } + )) => { log::debug!("Incoming connection failed: {:?}", error); - return Poll::Ready(SwarmEvent::IncomingConnectionError { + return Poll::Ready(Some(SwarmEvent::IncomingConnectionError { local_addr, send_back_addr, error, - }); + })); }, - Poll::Ready(NetworkEvent::DialError { peer_id, multiaddr, error, attempts_remaining }) => { + Poll::Ready(Some( + NetworkEvent::DialError { peer_id, multiaddr, error, attempts_remaining } + )) => { log::debug!( "Connection attempt to {:?} via {:?} failed with {:?}. Attempts remaining: {}.", peer_id, multiaddr, error, attempts_remaining); @@ -612,21 +690,23 @@ where TBehaviour: NetworkBehaviour, if attempts_remaining == 0 { this.behaviour.inject_dial_failure(&peer_id); } - return Poll::Ready(SwarmEvent::UnreachableAddr { + return Poll::Ready(Some(SwarmEvent::UnreachableAddr { peer_id, address: multiaddr, error, attempts_remaining, - }); + })); }, - Poll::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error, .. }) => { + Poll::Ready(Some( + NetworkEvent::UnknownPeerDialError { multiaddr, error, .. } + )) => { log::debug!("Connection attempt to address {:?} of unknown peer failed with {:?}", multiaddr, error); this.behaviour.inject_addr_reach_failure(None, &multiaddr, &error); - return Poll::Ready(SwarmEvent::UnknownPeerUnreachableAddr { + return Poll::Ready(Some(SwarmEvent::UnknownPeerUnreachableAddr { address: multiaddr, error, - }); + })); }, } @@ -676,10 +756,19 @@ where TBehaviour: NetworkBehaviour, }; match behaviour_poll { - Poll::Pending if network_not_ready => return Poll::Pending, - Poll::Pending => (), + Poll::Pending if network_not_ready => { + return Poll::Pending + } + Poll::Pending => { + if this.network.is_closed() { + // The network is closed and the behaviour is + // waiting for network I/O. Hence the swarm terminates. + this.closed = true; + return Poll::Ready(None) + } + }, Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => { - return Poll::Ready(SwarmEvent::Behaviour(event)) + return Poll::Ready(Some(SwarmEvent::Behaviour(event))) }, Poll::Ready(NetworkBehaviourAction::DialAddress { address }) => { let _ = ExpandedSwarm::dial_addr(&mut *this, address); @@ -697,7 +786,7 @@ where TBehaviour: NetworkBehaviour, }; if condition_matched { if ExpandedSwarm::dial(this, &peer_id).is_ok() { - return Poll::Ready(SwarmEvent::Dialing(peer_id)) + return Poll::Ready(Some(SwarmEvent::Dialing(peer_id))) } } else { // Even if the condition for a _new_ dialing attempt is not met, @@ -718,6 +807,18 @@ where TBehaviour: NetworkBehaviour, } } }, + Poll::Ready(NetworkBehaviourAction::DisconnectPeer { peer_id }) => { + if let Some(peer) = this.network.peer(peer_id).into_connected() { + peer.disconnect(); + } + } + Poll::Ready(NetworkBehaviourAction::CloseConnection { peer_id, connection_id }) => { + if let Some(mut peer) = this.network.peer(peer_id).into_connected() { + if let Some(conn) = peer.connection(connection_id) { + conn.start_close(); + } + } + } Poll::Ready(NetworkBehaviourAction::NotifyHandler { peer_id, handler, event }) => { if let Some(mut peer) = this.network.peer(peer_id.clone()).into_connected() { match handler { @@ -914,9 +1015,10 @@ where TBehaviour: NetworkBehaviour, fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { - let event = futures::ready!(ExpandedSwarm::poll_next_event(self.as_mut(), cx)); - if let SwarmEvent::Behaviour(event) = event { - return Poll::Ready(Some(event)); + match futures::ready!(ExpandedSwarm::poll_next_event(self.as_mut(), cx)) { + Some(SwarmEvent::Behaviour(event)) => return Poll::Ready(Some(event)), + Some(_) => {} + None => return Poll::Ready(None), } } } @@ -933,7 +1035,7 @@ where TBehaviour: NetworkBehaviour, TConnInfo: ConnectionInfo + fmt::Debug + Clone + Send + 'static, { fn is_terminated(&self) -> bool { - false + ExpandedSwarm::is_closed(self) } } @@ -1121,7 +1223,8 @@ where TBehaviour: NetworkBehaviour, listened_addrs: SmallVec::new(), external_addrs: Addresses::default(), banned_peers: HashSet::new(), - pending_event: None + pending_event: None, + closed: false, } } } diff --git a/swarm/src/protocols_handler.rs b/swarm/src/protocols_handler.rs index a994cadfd9e..8f3ce70b941 100644 --- a/swarm/src/protocols_handler.rs +++ b/swarm/src/protocols_handler.rs @@ -96,9 +96,13 @@ pub use select::{IntoProtocolsHandlerSelect, ProtocolsHandlerSelect}; /// implemented by the handler can include conditions for terminating the connection. /// The lifetime of successfully negotiated substreams is fully controlled by the handler. /// -/// Implementors of this trait should keep in mind that the connection can be closed at any time. -/// When a connection is closed gracefully, the substreams used by the handler may still -/// continue reading data until the remote closes its side of the connection. +/// When the keep-alive expires, the connection is gracefully closed. However, +/// implementors of this trait should keep in mind that the connection can close +/// as a result of encountering an error (including reading EOF as a result of +/// the connection being closed by the remote) at any time. +/// +/// When a connection is gracefully closed, the substreams used by the handler may be +/// shut down in an orderly fashion by implementing [`ProtocolsHandler::poll_close`]. pub trait ProtocolsHandler: Send + 'static { /// Custom event that can be received from the outside. type InEvent: Send + 'static; @@ -164,15 +168,14 @@ pub trait ProtocolsHandler: Send + 'static { /// Returns until when the connection should be kept alive. /// /// This method is called by the `Swarm` after each invocation of - /// [`ProtocolsHandler::poll`] to determine if the connection and the associated + /// [`Self::poll`] to determine if the connection and the associated /// `ProtocolsHandler`s should be kept alive as far as this handler is concerned /// and if so, for how long. /// - /// Returning [`KeepAlive::No`] indicates that the connection should be - /// closed and this handler destroyed immediately. + /// Returning [`KeepAlive::No`] indicates that the connection should be closed. /// /// Returning [`KeepAlive::Until`] indicates that the connection may be closed - /// and this handler destroyed after the specified `Instant`. + /// after the specified `Instant`. /// /// Returning [`KeepAlive::Yes`] indicates that the connection should /// be kept alive until the next call to this method. @@ -188,6 +191,37 @@ pub trait ProtocolsHandler: Send + 'static { ProtocolsHandlerEvent >; + /// Polls the handler to make progress towards closing the connection. + /// + /// When a connection is actively closed, the handler can perform + /// a graceful shutdown of the connection by draining the I/O + /// activity, e.g. allowing in-flight requests to complete without + /// accepting new ones, possibly signaling the remote that it + /// should direct further requests elsewhere. + /// + /// The handler can also use the opportunity to flush any buffers + /// or clean up any other (asynchronous) resources before the + /// connection is ultimately dropped and closed on the transport + /// layer. + /// + /// While closing, new inbound substreams are rejected and the + /// handler is unable to request new outbound substreams as + /// per the return type of `poll_close`. + /// + /// The handler signals its readiness for the connection + /// to be closed by returning `Ready(Ok(None))`, which is the + /// default implementation. Hence, by default, connection + /// shutdown is not delayed and may result in sudden + /// interruption of ongoing I/O. + /// + /// > **Note**: Once `poll_close()` is invoked, the handler is no + /// > longer `poll()`ed. + fn poll_close(&mut self, _: &mut Context) + -> Poll, Self::Error>> + { + Poll::Ready(Ok(None)) + } + /// Adds a closure that turns the input event into something else. #[inline] fn map_in_event(self, map: TMap) -> MapInEvent diff --git a/swarm/src/protocols_handler/node_handler.rs b/swarm/src/protocols_handler/node_handler.rs index c11aa83a493..80fb9807ac5 100644 --- a/swarm/src/protocols_handler/node_handler.rs +++ b/swarm/src/protocols_handler/node_handler.rs @@ -113,6 +113,75 @@ where shutdown: Shutdown, } +impl NodeHandlerWrapper +where + TProtoHandler: ProtocolsHandler, +{ + fn poll_inbound(&mut self, cx: &mut Context<'_>) { + // Continue negotiation of newly-opened substreams on the listening side. + // We remove each element from `negotiating_in` one by one and add them back if not ready. + for n in (0..self.negotiating_in.len()).rev() { + let (mut in_progress, mut timeout) = self.negotiating_in.swap_remove(n); + match Future::poll(Pin::new(&mut timeout), cx) { + Poll::Ready(Ok(_)) => { + let err = ProtocolsHandlerUpgrErr::Timeout; + self.handler.inject_listen_upgrade_error(err); + continue + } + Poll::Ready(Err(_)) => { + let err = ProtocolsHandlerUpgrErr::Timer; + self.handler.inject_listen_upgrade_error(err); + continue; + } + Poll::Pending => {}, + } + match Future::poll(Pin::new(&mut in_progress), cx) { + Poll::Ready(Ok(upgrade)) => + self.handler.inject_fully_negotiated_inbound(upgrade), + Poll::Pending => self.negotiating_in.push((in_progress, timeout)), + Poll::Ready(Err(err)) => { + let err = ProtocolsHandlerUpgrErr::Upgrade(err); + self.handler.inject_listen_upgrade_error(err); + } + } + } + + } + + fn poll_outbound(&mut self, cx: &mut Context<'_>) { + // Continue negotiation of newly-opened substreams. + // We remove each element from `negotiating_out` one by one and add them back if not ready. + for n in (0..self.negotiating_out.len()).rev() { + let (upgr_info, mut in_progress, mut timeout) = self.negotiating_out.swap_remove(n); + match Future::poll(Pin::new(&mut timeout), cx) { + Poll::Ready(Ok(_)) => { + let err = ProtocolsHandlerUpgrErr::Timeout; + self.handler.inject_dial_upgrade_error(upgr_info, err); + continue; + }, + Poll::Ready(Err(_)) => { + let err = ProtocolsHandlerUpgrErr::Timer; + self.handler.inject_dial_upgrade_error(upgr_info, err); + continue; + }, + Poll::Pending => {}, + } + match Future::poll(Pin::new(&mut in_progress), cx) { + Poll::Ready(Ok(upgrade)) => { + self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info); + } + Poll::Pending => { + self.negotiating_out.push((upgr_info, in_progress, timeout)); + } + Poll::Ready(Err(err)) => { + let err = ProtocolsHandlerUpgrErr::Upgrade(err); + self.handler.inject_dial_upgrade_error(upgr_info, err); + } + } + } + } +} + /// The options for a planned connection & handler shutdown. /// /// A shutdown is planned anew based on the the return value of @@ -228,64 +297,8 @@ where fn poll(&mut self, cx: &mut Context<'_>) -> Poll< Result, Self::Error> > { - // Continue negotiation of newly-opened substreams on the listening side. - // We remove each element from `negotiating_in` one by one and add them back if not ready. - for n in (0..self.negotiating_in.len()).rev() { - let (mut in_progress, mut timeout) = self.negotiating_in.swap_remove(n); - match Future::poll(Pin::new(&mut timeout), cx) { - Poll::Ready(Ok(_)) => { - let err = ProtocolsHandlerUpgrErr::Timeout; - self.handler.inject_listen_upgrade_error(err); - continue - } - Poll::Ready(Err(_)) => { - let err = ProtocolsHandlerUpgrErr::Timer; - self.handler.inject_listen_upgrade_error(err); - continue; - } - Poll::Pending => {}, - } - match Future::poll(Pin::new(&mut in_progress), cx) { - Poll::Ready(Ok(upgrade)) => - self.handler.inject_fully_negotiated_inbound(upgrade), - Poll::Pending => self.negotiating_in.push((in_progress, timeout)), - Poll::Ready(Err(err)) => { - let err = ProtocolsHandlerUpgrErr::Upgrade(err); - self.handler.inject_listen_upgrade_error(err); - } - } - } - - // Continue negotiation of newly-opened substreams. - // We remove each element from `negotiating_out` one by one and add them back if not ready. - for n in (0..self.negotiating_out.len()).rev() { - let (upgr_info, mut in_progress, mut timeout) = self.negotiating_out.swap_remove(n); - match Future::poll(Pin::new(&mut timeout), cx) { - Poll::Ready(Ok(_)) => { - let err = ProtocolsHandlerUpgrErr::Timeout; - self.handler.inject_dial_upgrade_error(upgr_info, err); - continue; - }, - Poll::Ready(Err(_)) => { - let err = ProtocolsHandlerUpgrErr::Timer; - self.handler.inject_dial_upgrade_error(upgr_info, err); - continue; - }, - Poll::Pending => {}, - } - match Future::poll(Pin::new(&mut in_progress), cx) { - Poll::Ready(Ok(upgrade)) => { - self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info); - } - Poll::Pending => { - self.negotiating_out.push((upgr_info, in_progress, timeout)); - } - Poll::Ready(Err(err)) => { - let err = ProtocolsHandlerUpgrErr::Upgrade(err); - self.handler.inject_dial_upgrade_error(upgr_info, err); - } - } - } + self.poll_inbound(cx); + self.poll_outbound(cx); // Poll the handler at the end so that we see the consequences of the method // calls on `self.handler`. @@ -328,16 +341,42 @@ where // Check if the connection (and handler) should be shut down. // As long as we're still negotiating substreams, shutdown is always postponed. if self.negotiating_in.is_empty() && self.negotiating_out.is_empty() { - match self.shutdown { - Shutdown::None => {}, - Shutdown::Asap => return Poll::Ready(Err(NodeHandlerWrapperError::KeepAliveTimeout)), + let close = match self.shutdown { + Shutdown::None => false, + Shutdown::Asap => true, Shutdown::Later(ref mut delay, _) => match Future::poll(Pin::new(delay), cx) { - Poll::Ready(_) => return Poll::Ready(Err(NodeHandlerWrapperError::KeepAliveTimeout)), - Poll::Pending => {} + Poll::Ready(_) => true, + Poll::Pending => false } + }; + if close { + log::debug!("Closing connection due to keep-alive timeout."); + return Poll::Ready(Ok(ConnectionHandlerEvent::Close)) } } Poll::Pending } + + fn poll_close(&mut self, cx: &mut Context) + -> Poll, Self::Error>> + { + // Allow ongoing inbound and outbound substream upgrades to complete. + // New inbound substreams are dropped / rejected and new outbound + // substreams are not permitted as per the return type of `poll_close`. + self.poll_inbound(cx); + self.poll_outbound(cx); + + // If the handler is ready to close and there are no more + // substreams being negotiated, the connection can close. + match self.handler.poll_close(cx).map_err(NodeHandlerWrapperError::Handler) { + Poll::Ready(Ok(None)) => + if self.negotiating_in.is_empty() && self.negotiating_out.is_empty() { + return Poll::Ready(Ok(None)) + } else { + return Poll::Pending + } + poll => poll + } + } } From 672bf825f98c40e6b4b59f5b31f187d37625a279 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 29 Jul 2020 17:09:28 +0200 Subject: [PATCH 10/19] Cleanup --- core/src/connection/pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 032e564aa02..c16105abfb2 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -646,7 +646,7 @@ where // Start a clean shutdown for all established connections. for id in self.established.values().flat_map(|conns| conns.keys()) { match self.manager.entry(*id) { - Some(manager::Entry::Established(mut e)) => { + Some(manager::Entry::Established(e)) => { e.start_close(); }, _ => { From 5727d6d08dab439bf4e7ea3a4a96774ba7859363 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 29 Jul 2020 17:50:40 +0200 Subject: [PATCH 11/19] Resolve missed conflict. --- core/src/connection/substream.rs | 28 ---------------------------- 1 file changed, 28 deletions(-) diff --git a/core/src/connection/substream.rs b/core/src/connection/substream.rs index 1c96d091730..806895acbd2 100644 --- a/core/src/connection/substream.rs +++ b/core/src/connection/substream.rs @@ -211,34 +211,6 @@ where } } -<<<<<<< HEAD -impl Future for Close -where - TMuxer: StreamMuxer, -{ - type Output = Result<(), IoError>; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - match self.muxer.close(cx) { - Poll::Pending => Poll::Pending, - Poll::Ready(Ok(())) => Poll::Ready(Ok(())), - Poll::Ready(Err(err)) => Poll::Ready(Err(err.into())), - } - } -} - -impl fmt::Debug for Close -where - TMuxer: StreamMuxer, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - f.debug_struct("Close") - .finish() - } -} - -======= ->>>>>>> Graceful shutdown for connections, networks and swarms. impl fmt::Debug for SubstreamEvent where TMuxer: StreamMuxer, From 3c5efe321104640d557f0673ecb8121e0d2b94ac Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 29 Jul 2020 17:55:51 +0200 Subject: [PATCH 12/19] Fix rustdoc link. --- core/src/network.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/network.rs b/core/src/network.rs index 6a798db2782..31282ae0427 100644 --- a/core/src/network.rs +++ b/core/src/network.rs @@ -359,7 +359,7 @@ where /// /// The `Network` is closed after successful completion /// of a graceful shutdown. See [`Network::start_close`] - /// and [`Network::closed`]. + /// and [`Network::close`]. /// /// A closed `Network` no longer performs any I/O and /// shoudl eventually be discarded. From 069af14e9d0063d3f7b48f95cd9079b87877cb91 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Wed, 29 Jul 2020 18:05:16 +0200 Subject: [PATCH 13/19] Fix rustdoc link. --- core/src/connection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/connection.rs b/core/src/connection.rs index 50a0ff618b7..2933742ee70 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -415,7 +415,7 @@ enum ConnectionState { CloseHandler, /// The connection is closing, rejecting new inbound substreams /// and not permitting new outbound substreams while the - /// muxer is closing the transport connection. [`Muxer::poll_close`] + /// muxer is closing the transport connection. [`substream::Muxing::poll_close`] /// is called until completion, which results in transitioning /// to `Closed`. CloseMuxer, From bd4cf05c16944953df01ba005b15f5e258a60d59 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Thu, 30 Jul 2020 10:59:21 +0200 Subject: [PATCH 14/19] Fix rustdoc link. --- swarm/src/lib.rs | 2 +- swarm/src/protocols_handler.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index bf0369d31cc..6179dcfd4bf 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -487,7 +487,7 @@ where TBehaviour: NetworkBehaviour, /// Closes the `Swarm`, ignoring any further events. /// - /// See [`Self::start_close`] for details. + /// See [`ExpandedSwarm::start_close`] for details. pub async fn close(&mut self) { self.network.start_close(); while let Some(_) = self.next_event().await {} diff --git a/swarm/src/protocols_handler.rs b/swarm/src/protocols_handler.rs index 8f3ce70b941..40c1aa08392 100644 --- a/swarm/src/protocols_handler.rs +++ b/swarm/src/protocols_handler.rs @@ -168,7 +168,7 @@ pub trait ProtocolsHandler: Send + 'static { /// Returns until when the connection should be kept alive. /// /// This method is called by the `Swarm` after each invocation of - /// [`Self::poll`] to determine if the connection and the associated + /// [`ProtocolsHandler::poll`] to determine if the connection and the associated /// `ProtocolsHandler`s should be kept alive as far as this handler is concerned /// and if so, for how long. /// From 7d437e466c9e8ef4148259eaf05686d84fe068ec Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Thu, 30 Jul 2020 15:54:19 +0200 Subject: [PATCH 15/19] Cleaner shutdown for Swarms. Shut down the `NetworkBehaviour` in tandem with the `Network` via a dedicated `poll_close` API. --- core/src/connection.rs | 22 +++++--- core/src/connection/manager/task.rs | 25 ++++++--- core/src/network.rs | 6 +++ misc/core-derive/src/lib.rs | 79 ++++++++++++++++++++++++++++- protocols/kad/src/behaviour/test.rs | 4 +- swarm/src/behaviour.rs | 65 +++++++++++++++++++++--- swarm/src/lib.rs | 40 ++++++++++++--- 7 files changed, 210 insertions(+), 31 deletions(-) diff --git a/core/src/connection.rs b/core/src/connection.rs index 2933742ee70..4773208b9bb 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -239,14 +239,15 @@ where /// Notifies the connection handler of an event. /// - /// Has no effect if the connection handler is already closed. - pub fn inject_event(&mut self, event: THandler::InEvent) { + /// Returns `Ok` if the event was delivered to the handler, `Err` + /// if the connection is closing and the handler is already closed. + pub fn inject_event(&mut self, event: THandler::InEvent) -> Result<(), THandler::InEvent> { match self.state { - ConnectionState::Open | ConnectionState::CloseHandler - => self.handler.inject_event(event), - _ => { - log::trace!("Ignoring handler event. Handler is closed.") + ConnectionState::Open | ConnectionState::CloseHandler => { + self.handler.inject_event(event); + Ok(()) } + _ => Err(event) } } @@ -261,6 +262,11 @@ where } } + /// Whether the connection is open, i.e. neither closing nor already closed. + pub fn is_open(&self) -> bool { + self.state == ConnectionState::Open + } + /// Polls the connection for events produced by the associated handler /// as a result of I/O activity on the substream multiplexer. /// @@ -283,7 +289,9 @@ where self.state = ConnectionState::Closed; return Poll::Ready(Ok(None)) } - Err(e) => return Poll::Ready(Err(ConnectionError::IO(e))) + Err(e) => { + return Poll::Ready(Err(ConnectionError::IO(e))) + } } } diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index 4e37ad671bf..0a082a94533 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -198,7 +198,7 @@ where let this = &mut *self; let id = this.id; - 'poll: loop { + loop { match std::mem::replace(&mut this.state, State::Done) { State::Pending { mut future, handler } => { // Check whether the task is still registered with a `Manager` @@ -244,14 +244,23 @@ where match this.commands.poll_next_unpin(cx) { Poll::Pending => break, Poll::Ready(Some(Command::NotifyHandler(event))) => - connection.inject_event(event), + if connection.inject_event(event).is_err() { + // The handler is already closed and the connection + // about to close; stop accepting commands. + this.commands.get_mut().close(); + break + }, Poll::Ready(Some(Command::Close)) => { // Start closing the connection, if not already. connection.start_close(); } Poll::Ready(None) => { - // The manager has dropped the task or disappeared; abort. - return Poll::Ready(()) + if connection.is_open() { + // The manager has dropped the task or disappeared + // while the connection was open; abort. + return Poll::Ready(()) + } + break } } } @@ -273,7 +282,7 @@ where connection, event: None, }; - continue 'poll + continue } } // The manager is no longer reachable; abort. @@ -281,6 +290,7 @@ where } } } else { + // Poll the connection for new events. match Connection::poll(Pin::new(&mut connection), cx) { Poll::Pending => { this.state = State::Established { @@ -294,12 +304,14 @@ where connection, event: Some(Event::Notify { id, event }), }; + continue } Poll::Ready(Ok(Some(connection::Event::AddressChange(new_address)))) => { this.state = State::Established { connection, event: Some(Event::AddressChange { id, new_address }), }; + continue } Poll::Ready(Ok(None)) => { // The connection is closed, don't accept any further commands @@ -307,7 +319,7 @@ where this.commands.get_mut().close(); let event = Event::Closed { id: this.id, error: None }; this.state = State::Terminating(event); - continue 'poll + continue } Poll::Ready(Err(error)) => { // Don't accept any further commands. @@ -315,6 +327,7 @@ where // Terminate the task with the error, dropping the connection. let event = Event::Closed { id, error: Some(error) }; this.state = State::Terminating(event); + continue } } } diff --git a/core/src/network.rs b/core/src/network.rs index 31282ae0427..deebf8d7de5 100644 --- a/core/src/network.rs +++ b/core/src/network.rs @@ -379,6 +379,12 @@ where self.pool.is_closing() } + /// Whether the `Network` is open for new connections, i.e. closing + /// or already closed. + pub fn is_open(&self) -> bool { + self.pool.is_open() + } + /// Provides an API similar to `Stream`, except that it cannot error. pub fn poll<'a>(&'a mut self, cx: &mut Context<'_>) -> Poll>> diff --git a/misc/core-derive/src/lib.rs b/misc/core-derive/src/lib.rs index 7fc145f5609..26597faea34 100644 --- a/misc/core-derive/src/lib.rs +++ b/misc/core-derive/src/lib.rs @@ -52,6 +52,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { let net_behv_event_proc = quote!{::libp2p::swarm::NetworkBehaviourEventProcess}; let either_ident = quote!{::libp2p::core::either::EitherOutput}; let network_behaviour_action = quote!{::libp2p::swarm::NetworkBehaviourAction}; + let network_behaviour_close_action = quote!{::libp2p::swarm::NetworkBehaviourCloseAction}; let into_protocols_handler = quote!{::libp2p::swarm::IntoProtocolsHandler}; let protocols_handler = quote!{::libp2p::swarm::ProtocolsHandler}; let into_proto_select_ident = quote!{::libp2p::swarm::IntoProtocolsHandlerSelect}; @@ -404,7 +405,12 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // List of statements to put in `poll()`. // // We poll each child one by one and wrap around the output. - let poll_stmts = data_struct.fields.iter().enumerate().filter(|f| !is_ignored(&f.1)).enumerate().map(|(enum_n, (field_n, field))| { + let poll_stmts = data_struct.fields.iter() + .enumerate() + .filter(|f| !is_ignored(&f.1)) + .enumerate() + .map(|(enum_n, (field_n, field))| + { let field_name = match field.ident { Some(ref i) => quote!{ self.#i }, None => quote!{ self.#field_n }, @@ -465,6 +471,67 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { }) }); + + // List of statements to put in `poll_close()`. + // + // We poll each child one by one and wrap around the output. + let poll_close_stmts = data_struct.fields.iter() + .enumerate() + .filter(|f| !is_ignored(&f.1)) + .enumerate() + .map(|(enum_n, (field_n, field))| + { + let field_name = match field.ident { + Some(ref i) => quote!{ self.#i }, + None => quote!{ self.#field_n }, + }; + + let mut wrapped_event = if enum_n != 0 { + quote!{ #either_ident::Second(event) } + } else { + quote!{ event } + }; + for _ in 0 .. data_struct.fields.iter().filter(|f| !is_ignored(f)).count() - 1 - enum_n { + wrapped_event = quote!{ #either_ident::First(#wrapped_event) }; + } + + let generate_event_match_arm = if event_process { + quote! { + std::task::Poll::Ready(Some(#network_behaviour_close_action::GenerateEvent(event))) => { + #net_behv_event_proc::inject_event(self, event) + } + } + } else { + quote! { + std::task::Poll::Ready(Some(#network_behaviour_close_action::GenerateEvent(event))) => { + return std::task::Poll::Ready(Some(#network_behaviour_close_action::GenerateEvent(event.into()))) + } + } + }; + + Some(quote!{ + loop { + match #trait_to_impl::poll_close(&mut #field_name, cx, poll_params) { + #generate_event_match_arm + std::task::Poll::Ready(Some(#network_behaviour_close_action::NotifyHandler { peer_id, handler, event })) => { + return std::task::Poll::Ready(Some(#network_behaviour_close_action::NotifyHandler { + peer_id, + handler, + event: #wrapped_event, + })); + }, + std::task::Poll::Ready(None) => { + break + } + std::task::Poll::Pending => { + pending = true; + break + } + } + } + }) + }); + // Now the magic happens. let final_quote = quote!{ impl #impl_generics #trait_to_impl for #name #ty_generics @@ -544,11 +611,19 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { } fn poll(&mut self, cx: &mut std::task::Context, poll_params: &mut impl #poll_parameters) -> std::task::Poll<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> { - use libp2p::futures::prelude::*; #(#poll_stmts)* let f: std::task::Poll<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> = #poll_method; f } + + fn poll_close(&mut self, cx: &mut std::task::Context, poll_params: &mut impl #poll_parameters) -> std::task::Poll::Handler as #protocols_handler>::InEvent, Self::OutEvent>>> { + let mut pending = false; + #(#poll_close_stmts)* + if pending { + return std::task::Poll::Pending + } + return std::task::Poll::Ready(None) + } } }; diff --git a/protocols/kad/src/behaviour/test.rs b/protocols/kad/src/behaviour/test.rs index 6ca005afd1e..01f2175f466 100644 --- a/protocols/kad/src/behaviour/test.rs +++ b/protocols/kad/src/behaviour/test.rs @@ -941,7 +941,7 @@ fn add_provider() { })) } - QuickCheck::new().tests(3).quickcheck(prop as fn(_,_)) + QuickCheck::new().tests(1).quickcheck(prop as fn(_,_)) } /// User code should be able to start queries beyond the internal @@ -1173,7 +1173,7 @@ fn manual_bucket_inserts() { Poll::Ready(Some(_)) => {}, Poll::Ready(None) => break, Poll::Pending => - if success && !Swarm::is_closing(swarm) { + if success && !Swarm::is_closed(swarm) { Swarm::start_close(swarm) } else { break diff --git a/swarm/src/behaviour.rs b/swarm/src/behaviour.rs index 2b42c885aa8..328e5aaef36 100644 --- a/swarm/src/behaviour.rs +++ b/swarm/src/behaviour.rs @@ -167,12 +167,41 @@ pub trait NetworkBehaviour: Send + 'static { fn inject_listener_closed(&mut self, _id: ListenerId, _reason: Result<(), &std::io::Error>) { } - /// Polls for things that swarm should do. + /// Polls the behaviour for the next action to perform. /// - /// This API mimics the API of the `Stream` trait. The method may register the current task in - /// order to wake it up at a later point in time. - fn poll(&mut self, cx: &mut Context<'_>, params: &mut impl PollParameters) - -> Poll::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>; + /// The method may register the current task in order to wake it up at a later point in time. + fn poll(&mut self, cx: &mut Context<'_>, params: &mut impl PollParameters) -> Poll< + NetworkBehaviourAction< + <::Handler as ProtocolsHandler>::InEvent, + Self::OutEvent + > + >; + + /// Polls the behaviour for the next action to perform during shutdown. + /// + /// The underlying `Network` and the `NetworkBehaviour` perform a + /// graceful shutdown in tandem. Shutdown is complete when `Network` + /// is closed (i.e. all established connections closed) and the + /// `NetworkBehaviour` returns `Ready(None)`. Since the `NetworkBehaviour` + /// does not know when the `Network` ultimately closed, it is perfectly + /// legitimate and expected for the `NetworkBehaviour` to return `Poll::Ready` + /// with `Some` event after having previously returned `None` if more + /// input from the `Network` was received that results in further work + /// to do for a clean shutdown. + /// + /// The default implementation is to always return `Poll::Ready(None)`, + /// i.e. shutdown completes as soon as the underlying `Network` is closed. + /// + /// Like `poll`, this method may register the current task in order to wake + /// it up at a later point in time. + fn poll_close(&mut self, _: &mut Context<'_>, _: &mut impl PollParameters) -> Poll< + Option::Handler as ProtocolsHandler>::InEvent, + Self::OutEvent + >> + > { + Poll::Ready(None) + } } /// Parameters passed to `poll()`, that the `NetworkBehaviour` has access to. @@ -263,7 +292,7 @@ pub enum NetworkBehaviourAction { NotifyHandler { /// The peer for whom a `ProtocolsHandler` should be notified. peer_id: PeerId, - /// The ID of the connection whose `ProtocolsHandler` to notify. + /// The handlers to notify. handler: NotifyHandler, /// The event to send. event: TInEvent, @@ -303,6 +332,30 @@ pub enum NotifyHandler { All } +/// An action that a [`NetworkBehaviour`] can trigger in the [`Swarm`] +/// while the `Swarm` is shutting down. +/// +/// [`Swarm`]: super::Swarm +#[derive(Debug, Clone)] +pub enum NetworkBehaviourCloseAction { + /// Instructs the `Swarm` to return an event when it is being polled. + GenerateEvent(TOutEvent), + + /// Instructs the `Swarm` to send an event to the handler dedicated to a + /// connection with a peer. + /// + /// See [`NetworkBehaviourAction::NotifyHandler`]. + NotifyHandler { + /// The peer for whom a `ProtocolsHandler` should be notified. + peer_id: PeerId, + /// The handlers to notify. + handler: NotifyHandler, + /// The event to send. + event: TInEvent, + }, +} + + /// The available conditions under which a new dialing attempt to /// a peer is initiated when requested by [`NetworkBehaviourAction::DialPeer`]. #[derive(Debug, Copy, Clone)] diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 6179dcfd4bf..9c9feda5d8c 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -65,6 +65,7 @@ pub mod toggle; pub use behaviour::{ NetworkBehaviour, NetworkBehaviourAction, + NetworkBehaviourCloseAction, NetworkBehaviourEventProcess, PollParameters, NotifyHandler, @@ -752,7 +753,23 @@ where TBehaviour: NetworkBehaviour, listened_addrs: &this.listened_addrs, external_addrs: &this.external_addrs }; - this.behaviour.poll(cx, &mut parameters) + if this.network.is_open() { + this.behaviour.poll(cx, &mut parameters).map(Some) + } else { + // The `Network` and `NetworkBehaviourehaviour` are + // shutting down together. + this.behaviour.poll_close(cx, &mut parameters) + .map(|next| next.map(|action| + match action { + NetworkBehaviourCloseAction::GenerateEvent(e) => + NetworkBehaviourAction::GenerateEvent(e), + NetworkBehaviourCloseAction::NotifyHandler { + peer_id, handler, event + } => NetworkBehaviourAction::NotifyHandler { + peer_id, handler, event + }, + })) + } }; match behaviour_poll { @@ -767,13 +784,13 @@ where TBehaviour: NetworkBehaviour, return Poll::Ready(None) } }, - Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => { + Poll::Ready(Some(NetworkBehaviourAction::GenerateEvent(event))) => { return Poll::Ready(Some(SwarmEvent::Behaviour(event))) }, - Poll::Ready(NetworkBehaviourAction::DialAddress { address }) => { + Poll::Ready(Some(NetworkBehaviourAction::DialAddress { address })) => { let _ = ExpandedSwarm::dial_addr(&mut *this, address); }, - Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id, condition }) => { + Poll::Ready(Some(NetworkBehaviourAction::DialPeer { peer_id, condition })) => { if this.banned_peers.contains(&peer_id) { this.behaviour.inject_dial_failure(&peer_id); } else { @@ -807,19 +824,19 @@ where TBehaviour: NetworkBehaviour, } } }, - Poll::Ready(NetworkBehaviourAction::DisconnectPeer { peer_id }) => { + Poll::Ready(Some(NetworkBehaviourAction::DisconnectPeer { peer_id })) => { if let Some(peer) = this.network.peer(peer_id).into_connected() { peer.disconnect(); } } - Poll::Ready(NetworkBehaviourAction::CloseConnection { peer_id, connection_id }) => { + Poll::Ready(Some(NetworkBehaviourAction::CloseConnection { peer_id, connection_id })) => { if let Some(mut peer) = this.network.peer(peer_id).into_connected() { if let Some(conn) = peer.connection(connection_id) { conn.start_close(); } } } - Poll::Ready(NetworkBehaviourAction::NotifyHandler { peer_id, handler, event }) => { + Poll::Ready(Some(NetworkBehaviourAction::NotifyHandler { peer_id, handler, event })) => { if let Some(mut peer) = this.network.peer(peer_id.clone()).into_connected() { match handler { NotifyHandler::One(connection) => { @@ -850,7 +867,7 @@ where TBehaviour: NetworkBehaviour, } } }, - Poll::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => { + Poll::Ready(Some(NetworkBehaviourAction::ReportObservedAddr { address })) => { for addr in this.network.address_translation(&address) { if this.external_addrs.iter().all(|a| *a != addr) { this.behaviour.inject_new_external_addr(&addr); @@ -858,6 +875,13 @@ where TBehaviour: NetworkBehaviour, this.external_addrs.add(addr); } }, + Poll::Ready(None) => { + if this.network.is_closed() { + this.closed = true; + return Poll::Ready(None) + } + debug_assert!(this.network.is_closing()); + } } } } From f2f17f73365c2cf31f1e91e5287168d16cce0a8b Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Thu, 30 Jul 2020 18:01:42 +0200 Subject: [PATCH 16/19] Documentation update. --- swarm/src/lib.rs | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 9c9feda5d8c..726f549e3d9 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -474,14 +474,21 @@ where TBehaviour: NetworkBehaviour, /// drained by asking the associated [`ProtocolsHandler`] to close, /// after which the underlying connection is closed. /// - /// 2. The [`NetworkBehaviour`] associated with the `Swarm` continues - /// to be polled once the `Network` closed until it returns - /// `Poll::Pending`, indicating that further network I/O would be - /// needed for the behaviour to make progress. In this way, the - /// behaviour can finish any work and emit resulting events. + /// 2. While the `Network` is shutting down, the [`NetworkBehaviour`] + /// is polled to close as well, via [`NetworkBehaviour::poll_close`], + /// i.e. `Network` and `NetworkBehaviour` perform the shutdown in tandem. + /// + /// 3. Once the `Network` is closed and at the same time + /// [`NetworkBehaviour::poll_close`] returns `Poll::Ready(None)`, + /// shutdown is complete. In this way, the behaviour can finish + /// any work and emit resulting events even after the `Network` + /// already closed. /// /// After calling this method, the `Swarm` must be `poll()`ed to /// drive the shutdown to completion. + /// + /// See [`ExpandedSwarm::close`] to perform a shutdown that + /// ignores any further events that are emitted. pub fn start_close(me: &mut Self) { me.network.start_close() } From 1dbee7684673753fbfb9bec17deb8306b3f0161f Mon Sep 17 00:00:00 2001 From: Roman Borschel Date: Mon, 3 Aug 2020 13:27:53 +0200 Subject: [PATCH 17/19] Update protocols/request-response/src/handler.rs Co-authored-by: Toralf Wittner --- protocols/request-response/src/handler.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/protocols/request-response/src/handler.rs b/protocols/request-response/src/handler.rs index c34441e4764..c49fbe1a00d 100644 --- a/protocols/request-response/src/handler.rs +++ b/protocols/request-response/src/handler.rs @@ -86,7 +86,7 @@ where impl RequestResponseHandler where - TCodec: RequestResponseCodec + Clone + Send + 'static, + TCodec: RequestResponseCodec { pub(super) fn new( inbound_protocols: SmallVec<[TCodec::Protocol; 2]>, @@ -364,4 +364,3 @@ where return Poll::Ready(Ok(None)) } } - From a16f4f553d95e8d161e8a6a9b20e0cd8b4003e50 Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Mon, 3 Aug 2020 13:30:42 +0200 Subject: [PATCH 18/19] Incorporate some review feedback. --- core/src/connection.rs | 135 +++++++++++++++++++++-------------------- 1 file changed, 69 insertions(+), 66 deletions(-) diff --git a/core/src/connection.rs b/core/src/connection.rs index 4773208b9bb..464659aaaef 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -279,78 +279,81 @@ where -> Poll>, ConnectionError>> { loop { - if let ConnectionState::Closed = self.state { // (1) - return Poll::Ready(Ok(None)) - } - - if let ConnectionState::CloseMuxer = self.state { // (2) - match futures::ready!(self.muxing.poll_close(cx)) { - Ok(()) => { - self.state = ConnectionState::Closed; - return Poll::Ready(Ok(None)) - } - Err(e) => { - return Poll::Ready(Err(ConnectionError::IO(e))) - } + match self.state { + ConnectionState::Closed => { + return Poll::Ready(Ok(None)) } - } - - // At this point the connection is either open or in the process - // of a graceful shutdown by the connection handler. - let mut io_pending = false; - - // Perform I/O on the connection through the muxer, informing the handler - // of new substreams or other muxer events. - match self.muxing.poll(cx) { - Poll::Pending => io_pending = true, - Poll::Ready(Ok(SubstreamEvent::InboundSubstream { substream })) => { - // Drop new inbound substreams when closing. This is analogous - // to rejecting new connections. - if self.state == ConnectionState::Open { - self.handler.inject_substream(substream, SubstreamEndpoint::Listener) - } else { - log::trace!("Inbound substream dropped. Connection is closing.") + ConnectionState::CloseMuxer => { + match futures::ready!(self.muxing.poll_close(cx)) { + Ok(()) => { + self.state = ConnectionState::Closed; + return Poll::Ready(Ok(None)) + } + Err(e) => { + return Poll::Ready(Err(ConnectionError::IO(e))) + } } } - Poll::Ready(Ok(SubstreamEvent::OutboundSubstream { user_data, substream })) => { - let endpoint = SubstreamEndpoint::Dialer(user_data); - self.handler.inject_substream(substream, endpoint) - } - Poll::Ready(Ok(SubstreamEvent::AddressChange(address))) => { - self.handler.inject_address_change(&address); - return Poll::Ready(Ok(Some(Event::AddressChange(address)))); - } - Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::IO(err))), - } + ConnectionState::Open | ConnectionState::CloseHandler => { + // At this point the connection is either open or in the process + // of a graceful shutdown by the connection handler. + let mut io_pending = false; + + // Perform I/O on the connection through the muxer, informing the handler + // of new substreams or other muxer events. + match self.muxing.poll(cx) { + Poll::Pending => io_pending = true, + Poll::Ready(Ok(SubstreamEvent::InboundSubstream { substream })) => { + // Drop new inbound substreams when closing. This is analogous + // to rejecting new connections. + if self.state == ConnectionState::Open { + self.handler.inject_substream(substream, SubstreamEndpoint::Listener) + } else { + log::trace!("Inbound substream dropped. Connection is closing.") + } + } + Poll::Ready(Ok(SubstreamEvent::OutboundSubstream { user_data, substream })) => { + let endpoint = SubstreamEndpoint::Dialer(user_data); + self.handler.inject_substream(substream, endpoint) + } + Poll::Ready(Ok(SubstreamEvent::AddressChange(address))) => { + self.handler.inject_address_change(&address); + return Poll::Ready(Ok(Some(Event::AddressChange(address)))); + } + Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::IO(err))), + } - // Poll the handler for new events. - let poll = match &self.state { - ConnectionState::Open => self.handler.poll(cx).map_ok(Some), - ConnectionState::CloseHandler => self.handler.poll_close(cx).map_ok( - |event| event.map(ConnectionHandlerEvent::Custom)), - s => panic!("Unexpected closing state: {:?}", s) // s.a. (1),(2) - }; - - match poll { - Poll::Pending => { - if io_pending { - return Poll::Pending // Nothing to do + // Poll the handler for new events. + let poll = + if self.state == ConnectionState::Open { + self.handler.poll(cx).map_ok(Some) + } else { + self.handler.poll_close(cx).map_ok(|event| + event.map(ConnectionHandlerEvent::Custom)) + }; + + match poll { + Poll::Pending => { + if io_pending { + return Poll::Pending // Nothing to do + } + } + Poll::Ready(Ok(Some(ConnectionHandlerEvent::OutboundSubstreamRequest(user_data)))) => { + self.muxing.open_substream(user_data); + } + Poll::Ready(Ok(Some(ConnectionHandlerEvent::Custom(event)))) => { + return Poll::Ready(Ok(Some(Event::Handler(event)))); + } + Poll::Ready(Ok(Some(ConnectionHandlerEvent::Close))) => { + self.start_close() + } + Poll::Ready(Ok(None)) => { + // The handler is done, we can now close the muxer (i.e. connection). + self.state = ConnectionState::CloseMuxer; + } + Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::Handler(err))), } } - Poll::Ready(Ok(Some(ConnectionHandlerEvent::OutboundSubstreamRequest(user_data)))) => { - self.muxing.open_substream(user_data); - } - Poll::Ready(Ok(Some(ConnectionHandlerEvent::Custom(event)))) => { - return Poll::Ready(Ok(Some(Event::Handler(event)))); - } - Poll::Ready(Ok(Some(ConnectionHandlerEvent::Close))) => { - self.start_close() - } - Poll::Ready(Ok(None)) => { - // The handler is done, we can now close the muxer (i.e. connection). - self.state = ConnectionState::CloseMuxer; - } - Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::Handler(err))), } } } From 6e91e543339c8ecd8c84e3becc50db206105641d Mon Sep 17 00:00:00 2001 From: "Roman S. Borschel" Date: Mon, 3 Aug 2020 15:52:03 +0200 Subject: [PATCH 19/19] Remove unnecessary 'continue's. --- core/src/connection/manager/task.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index 0a082a94533..887886b9762 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -304,14 +304,12 @@ where connection, event: Some(Event::Notify { id, event }), }; - continue } Poll::Ready(Ok(Some(connection::Event::AddressChange(new_address)))) => { this.state = State::Established { connection, event: Some(Event::AddressChange { id, new_address }), }; - continue } Poll::Ready(Ok(None)) => { // The connection is closed, don't accept any further commands @@ -319,7 +317,6 @@ where this.commands.get_mut().close(); let event = Event::Closed { id: this.id, error: None }; this.state = State::Terminating(event); - continue } Poll::Ready(Err(error)) => { // Don't accept any further commands. @@ -327,7 +324,6 @@ where // Terminate the task with the error, dropping the connection. let event = Event::Closed { id, error: Some(error) }; this.state = State::Terminating(event); - continue } } }