rust-libp2p/swarm/src/connection/handler_wrapper.rs

394 lines
14 KiB
Rust
Raw Normal View History

// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::connection::{Substream, SubstreamEndpoint};
use crate::protocols_handler::{
KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr,
};
use crate::upgrade::SendWrapper;
use futures::prelude::*;
use futures::stream::FuturesUnordered;
use futures_timer::Delay;
use instant::Instant;
use libp2p_core::{
muxing::StreamMuxerBox,
upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply, UpgradeError},
{core,swarm}: Remove Network abstraction (#2492) This commit removes the `Network` abstraction, thus managing `Listeners` and the connection `Pool` in `Swarm` directly. This is done under the assumption that noone uses the `Network` abstraction directly, but instead everyone always uses it through `Swarm`. Both `Listeners` and `Pool` are moved from `libp2p-core` into `libp2p-swarm`. Given that they are no longer exposed via `Network`, they can be treated as an implementation detail of `libp2p-swarm` and `Swarm`. This change does not include any behavioural changes. This change has the followin benefits: - Removal of `NetworkEvent`, which was mostly an isomorphism of `SwarmEvent`. - Removal of the never-directly-used `Network` abstraction. - Removal of now obsolete verbose `Peer` (`core/src/network/peer.rs`) construct. - Removal of `libp2p-core` `DialOpts`, which is a direct mapping of `libp2p-swarm` `DialOpts`. - Allowing breaking changes to the connection handling and `Swarm` API interface without a breaking change in `libp2p-core` and thus a without a breaking change in `/transport` protocols. This change enables the following potential future changes: - Removal of `NodeHandler` and `ConnectionHandler`. Thus allowing to rename `ProtocolsHandler` into `ConnectionHandler`. - Moving `NetworkBehaviour` and `ProtocolsHandler` into `libp2p-core`, having `libp2p-xxx` protocol crates only depend on `libp2p-core` and thus allowing general breaking changes to `Swarm` without breaking all `libp2p-xxx` crates.
2022-02-13 21:57:38 +01:00
Multiaddr,
};
use std::{error, fmt, pin::Pin, task::Context, task::Poll, time::Duration};
/// A wrapper for an underlying [`ProtocolsHandler`].
///
/// It extends [`ProtocolsHandler`] with:
/// - Enforced substream upgrade timeouts
/// - Driving substream upgrades
/// - Handling connection timeout
// TODO: add a caching system for protocols that are supported or not
pub struct HandlerWrapper<TProtoHandler>
where
TProtoHandler: ProtocolsHandler,
{
/// The underlying handler.
handler: TProtoHandler,
/// Futures that upgrade incoming substreams.
negotiating_in: FuturesUnordered<
SubstreamUpgrade<
TProtoHandler::InboundOpenInfo,
InboundUpgradeApply<
Substream<StreamMuxerBox>,
SendWrapper<TProtoHandler::InboundProtocol>,
>,
>,
>,
/// Futures that upgrade outgoing substreams.
negotiating_out: FuturesUnordered<
SubstreamUpgrade<
TProtoHandler::OutboundOpenInfo,
OutboundUpgradeApply<
Substream<StreamMuxerBox>,
SendWrapper<TProtoHandler::OutboundProtocol>,
>,
>,
>,
/// For each outbound substream request, how to upgrade it. The first element of the tuple
/// is the unique identifier (see `unique_dial_upgrade_id`).
queued_dial_upgrades: Vec<(u64, SendWrapper<TProtoHandler::OutboundProtocol>)>,
/// Unique identifier assigned to each queued dial upgrade.
unique_dial_upgrade_id: u64,
/// The currently planned connection & handler shutdown.
shutdown: Shutdown,
/// The substream upgrade protocol override, if any.
substream_upgrade_protocol_override: Option<upgrade::Version>,
}
impl<TProtoHandler: ProtocolsHandler> std::fmt::Debug for HandlerWrapper<TProtoHandler> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("HandlerWrapper")
.field("negotiating_in", &self.negotiating_in)
.field("negotiating_out", &self.negotiating_out)
.field("unique_dial_upgrade_id", &self.unique_dial_upgrade_id)
.field("shutdown", &self.shutdown)
.field(
"substream_upgrade_protocol_override",
&self.substream_upgrade_protocol_override,
)
.finish()
}
}
impl<TProtoHandler: ProtocolsHandler> HandlerWrapper<TProtoHandler> {
pub(crate) fn new(
handler: TProtoHandler,
substream_upgrade_protocol_override: Option<upgrade::Version>,
) -> Self {
Self {
handler,
negotiating_in: Default::default(),
negotiating_out: Default::default(),
queued_dial_upgrades: Vec::new(),
unique_dial_upgrade_id: 0,
shutdown: Shutdown::None,
substream_upgrade_protocol_override,
}
}
pub(crate) fn into_protocols_handler(self) -> TProtoHandler {
self.handler
}
}
struct SubstreamUpgrade<UserData, Upgrade> {
user_data: Option<UserData>,
timeout: Delay,
upgrade: Upgrade,
}
impl<UserData, Upgrade> Unpin for SubstreamUpgrade<UserData, Upgrade> {}
impl<UserData, Upgrade, UpgradeOutput, TUpgradeError> Future for SubstreamUpgrade<UserData, Upgrade>
where
Upgrade: Future<Output = Result<UpgradeOutput, UpgradeError<TUpgradeError>>> + Unpin,
{
type Output = (
UserData,
Result<UpgradeOutput, ProtocolsHandlerUpgrErr<TUpgradeError>>,
);
fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
match self.timeout.poll_unpin(cx) {
Poll::Ready(()) => {
return Poll::Ready((
self.user_data
.take()
.expect("Future not to be polled again once ready."),
Err(ProtocolsHandlerUpgrErr::Timeout),
))
}
Poll::Pending => {}
}
match self.upgrade.poll_unpin(cx) {
Poll::Ready(Ok(upgrade)) => Poll::Ready((
self.user_data
.take()
.expect("Future not to be polled again once ready."),
Ok(upgrade),
)),
Poll::Ready(Err(err)) => Poll::Ready((
self.user_data
.take()
.expect("Future not to be polled again once ready."),
Err(ProtocolsHandlerUpgrErr::Upgrade(err)),
)),
Poll::Pending => Poll::Pending,
}
}
}
/// The options for a planned connection & handler shutdown.
///
/// A shutdown is planned anew based on the the return value of
/// [`ProtocolsHandler::connection_keep_alive`] of the underlying handler
/// after every invocation of [`ProtocolsHandler::poll`].
///
/// A planned shutdown is always postponed for as long as there are ingoing
/// or outgoing substreams being negotiated, i.e. it is a graceful, "idle"
/// shutdown.
#[derive(Debug)]
enum Shutdown {
/// No shutdown is planned.
None,
/// A shut down is planned as soon as possible.
Asap,
/// A shut down is planned for when a `Delay` has elapsed.
Later(Delay, Instant),
}
/// Error generated by the [`HandlerWrapper`].
#[derive(Debug)]
pub enum Error<TErr> {
/// The connection handler encountered an error.
Handler(TErr),
/// The connection keep-alive timeout expired.
KeepAliveTimeout,
}
impl<TErr> From<TErr> for Error<TErr> {
fn from(err: TErr) -> Error<TErr> {
Error::Handler(err)
}
}
impl<TErr> fmt::Display for Error<TErr>
where
TErr: fmt::Display,
{
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
Error::Handler(err) => write!(f, "{}", err),
Error::KeepAliveTimeout => {
write!(f, "Connection closed due to expired keep-alive timeout.")
}
}
}
}
impl<TErr> error::Error for Error<TErr>
where
TErr: error::Error + 'static,
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
Error::Handler(err) => Some(err),
Error::KeepAliveTimeout => None,
}
}
}
pub type OutboundOpenInfo<TProtoHandler> = (
u64,
<TProtoHandler as ProtocolsHandler>::OutboundOpenInfo,
Duration,
);
impl<TProtoHandler> HandlerWrapper<TProtoHandler>
where
TProtoHandler: ProtocolsHandler,
{
pub fn inject_substream(
&mut self,
substream: Substream<StreamMuxerBox>,
// The first element of the tuple is the unique upgrade identifier
// (see `unique_dial_upgrade_id`).
endpoint: SubstreamEndpoint<OutboundOpenInfo<TProtoHandler>>,
) {
match endpoint {
Multiple connections per peer (#1440) * Allow multiple connections per peer in libp2p-core. Instead of trying to enforce a single connection per peer, which involves quite a bit of additional complexity e.g. to prioritise simultaneously opened connections and can have other undesirable consequences [1], we now make multiple connections per peer a feature. The gist of these changes is as follows: The concept of a "node" with an implicit 1-1 correspondence to a connection has been replaced with the "first-class" concept of a "connection". The code from `src/nodes` has moved (with varying degrees of modification) to `src/connection`. A `HandledNode` has become a `Connection`, a `NodeHandler` a `ConnectionHandler`, the `CollectionStream` was the basis for the new `connection::Pool`, and so forth. Conceptually, a `Network` contains a `connection::Pool` which in turn internally employs the `connection::Manager` for handling the background `connection::manager::Task`s, one per connection, as before. These are all considered implementation details. On the public API, `Peer`s are managed as before through the `Network`, except now the API has changed with the shift of focus to (potentially multiple) connections per peer. The `NetworkEvent`s have accordingly also undergone changes. The Swarm APIs remain largely unchanged, except for the fact that `inject_replaced` is no longer called. It may now practically happen that multiple `ProtocolsHandler`s are associated with a single `NetworkBehaviour`, one per connection. If implementations of `NetworkBehaviour` rely somehow on communicating with exactly one `ProtocolsHandler`, this may cause issues, but it is unlikely. [1]: https://github.com/paritytech/substrate/issues/4272 * Fix intra-rustdoc links. * Update core/src/connection/pool.rs Co-Authored-By: Max Inden <mail@max-inden.de> * Address some review feedback and fix doc links. * Allow responses to be sent on the same connection. * Remove unnecessary remainders of inject_replaced. * Update swarm/src/behaviour.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update swarm/src/lib.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update core/src/connection/manager.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update core/src/connection/manager.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update core/src/connection/pool.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Incorporate more review feedback. * Move module declaration below imports. * Update core/src/connection/manager.rs Co-Authored-By: Toralf Wittner <tw@dtex.org> * Update core/src/connection/manager.rs Co-Authored-By: Toralf Wittner <tw@dtex.org> * Simplify as per review. * Fix rustoc link. * Add try_notify_handler and simplify. * Relocate DialingConnection and DialingAttempt. For better visibility constraints. * Small cleanup. * Small cleanup. More robust EstablishedConnectionIter. * Clarify semantics of `DialingPeer::connect`. * Don't call inject_disconnected on InvalidPeerId. To preserve the previous behavior and ensure calls to `inject_disconnected` are always paired with calls to `inject_connected`. * Provide public ConnectionId constructor. Mainly needed for testing purposes, e.g. in substrate. * Move the established connection limit check to the right place. * Clean up connection error handling. Separate connection errors into those occuring during connection setup or upon rejecting a newly established connection (the `PendingConnectionError`) and those errors occurring on previously established connections, i.e. for which a `ConnectionEstablished` event has been emitted by the connection pool earlier. * Revert change in log level and clarify an invariant. * Remove inject_replaced entirely. * Allow notifying all connection handlers. Thereby simplify by introducing a new enum `NotifyHandler`, used with a single constructor `NetworkBehaviourAction::NotifyHandler`. * Finishing touches. Small API simplifications and code deduplication. Some more useful debug logging. Co-authored-by: Max Inden <mail@max-inden.de> Co-authored-by: Pierre Krieger <pierre.krieger1708@gmail.com> Co-authored-by: Toralf Wittner <tw@dtex.org>
2020-03-04 13:49:25 +01:00
SubstreamEndpoint::Listener => {
let protocol = self.handler.listen_protocol();
let timeout = *protocol.timeout();
let (upgrade, user_data) = protocol.into_upgrade();
let upgrade = upgrade::apply_inbound(substream, SendWrapper(upgrade));
let timeout = Delay::new(timeout);
self.negotiating_in.push(SubstreamUpgrade {
user_data: Some(user_data),
timeout,
upgrade,
});
}
Multiple connections per peer (#1440) * Allow multiple connections per peer in libp2p-core. Instead of trying to enforce a single connection per peer, which involves quite a bit of additional complexity e.g. to prioritise simultaneously opened connections and can have other undesirable consequences [1], we now make multiple connections per peer a feature. The gist of these changes is as follows: The concept of a "node" with an implicit 1-1 correspondence to a connection has been replaced with the "first-class" concept of a "connection". The code from `src/nodes` has moved (with varying degrees of modification) to `src/connection`. A `HandledNode` has become a `Connection`, a `NodeHandler` a `ConnectionHandler`, the `CollectionStream` was the basis for the new `connection::Pool`, and so forth. Conceptually, a `Network` contains a `connection::Pool` which in turn internally employs the `connection::Manager` for handling the background `connection::manager::Task`s, one per connection, as before. These are all considered implementation details. On the public API, `Peer`s are managed as before through the `Network`, except now the API has changed with the shift of focus to (potentially multiple) connections per peer. The `NetworkEvent`s have accordingly also undergone changes. The Swarm APIs remain largely unchanged, except for the fact that `inject_replaced` is no longer called. It may now practically happen that multiple `ProtocolsHandler`s are associated with a single `NetworkBehaviour`, one per connection. If implementations of `NetworkBehaviour` rely somehow on communicating with exactly one `ProtocolsHandler`, this may cause issues, but it is unlikely. [1]: https://github.com/paritytech/substrate/issues/4272 * Fix intra-rustdoc links. * Update core/src/connection/pool.rs Co-Authored-By: Max Inden <mail@max-inden.de> * Address some review feedback and fix doc links. * Allow responses to be sent on the same connection. * Remove unnecessary remainders of inject_replaced. * Update swarm/src/behaviour.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update swarm/src/lib.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update core/src/connection/manager.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update core/src/connection/manager.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Update core/src/connection/pool.rs Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com> * Incorporate more review feedback. * Move module declaration below imports. * Update core/src/connection/manager.rs Co-Authored-By: Toralf Wittner <tw@dtex.org> * Update core/src/connection/manager.rs Co-Authored-By: Toralf Wittner <tw@dtex.org> * Simplify as per review. * Fix rustoc link. * Add try_notify_handler and simplify. * Relocate DialingConnection and DialingAttempt. For better visibility constraints. * Small cleanup. * Small cleanup. More robust EstablishedConnectionIter. * Clarify semantics of `DialingPeer::connect`. * Don't call inject_disconnected on InvalidPeerId. To preserve the previous behavior and ensure calls to `inject_disconnected` are always paired with calls to `inject_connected`. * Provide public ConnectionId constructor. Mainly needed for testing purposes, e.g. in substrate. * Move the established connection limit check to the right place. * Clean up connection error handling. Separate connection errors into those occuring during connection setup or upon rejecting a newly established connection (the `PendingConnectionError`) and those errors occurring on previously established connections, i.e. for which a `ConnectionEstablished` event has been emitted by the connection pool earlier. * Revert change in log level and clarify an invariant. * Remove inject_replaced entirely. * Allow notifying all connection handlers. Thereby simplify by introducing a new enum `NotifyHandler`, used with a single constructor `NetworkBehaviourAction::NotifyHandler`. * Finishing touches. Small API simplifications and code deduplication. Some more useful debug logging. Co-authored-by: Max Inden <mail@max-inden.de> Co-authored-by: Pierre Krieger <pierre.krieger1708@gmail.com> Co-authored-by: Toralf Wittner <tw@dtex.org>
2020-03-04 13:49:25 +01:00
SubstreamEndpoint::Dialer((upgrade_id, user_data, timeout)) => {
let pos = match self
.queued_dial_upgrades
.iter()
.position(|(id, _)| id == &upgrade_id)
{
Some(p) => p,
None => {
debug_assert!(false, "Received an upgrade with an invalid upgrade ID");
return;
}
};
let (_, upgrade) = self.queued_dial_upgrades.remove(pos);
let mut version = upgrade::Version::default();
if let Some(v) = self.substream_upgrade_protocol_override {
if v != version {
log::debug!(
"Substream upgrade protocol override: {:?} -> {:?}",
version,
v
);
version = v;
}
}
let upgrade = upgrade::apply_outbound(substream, upgrade, version);
let timeout = Delay::new(timeout);
self.negotiating_out.push(SubstreamUpgrade {
user_data: Some(user_data),
timeout,
upgrade,
});
}
}
}
pub fn inject_event(&mut self, event: TProtoHandler::InEvent) {
self.handler.inject_event(event);
}
pub fn inject_address_change(&mut self, new_address: &Multiaddr) {
self.handler.inject_address_change(new_address);
}
pub fn poll(
&mut self,
cx: &mut Context<'_>,
) -> Poll<
Result<
Event<OutboundOpenInfo<TProtoHandler>, TProtoHandler::OutEvent>,
Error<TProtoHandler::Error>,
>,
> {
while let Poll::Ready(Some((user_data, res))) = self.negotiating_in.poll_next_unpin(cx) {
match res {
Ok(upgrade) => self
.handler
.inject_fully_negotiated_inbound(upgrade, user_data),
Err(err) => self.handler.inject_listen_upgrade_error(user_data, err),
}
}
while let Poll::Ready(Some((user_data, res))) = self.negotiating_out.poll_next_unpin(cx) {
match res {
Ok(upgrade) => self
.handler
.inject_fully_negotiated_outbound(upgrade, user_data),
Err(err) => self.handler.inject_dial_upgrade_error(user_data, err),
}
}
2019-04-16 15:57:29 +02:00
// Poll the handler at the end so that we see the consequences of the method
// calls on `self.handler`.
let poll_result = self.handler.poll(cx);
// Ask the handler whether it wants the connection (and the handler itself)
// to be kept alive, which determines the planned shutdown, if any.
match (&mut self.shutdown, self.handler.connection_keep_alive()) {
(Shutdown::Later(timer, deadline), KeepAlive::Until(t)) => {
if *deadline != t {
*deadline = t;
if let Some(dur) = deadline.checked_duration_since(Instant::now()) {
timer.reset(dur)
}
}
}
(_, KeepAlive::Until(t)) => {
if let Some(dur) = t.checked_duration_since(Instant::now()) {
self.shutdown = Shutdown::Later(Delay::new(dur), t)
}
}
(_, KeepAlive::No) => self.shutdown = Shutdown::Asap,
(_, KeepAlive::Yes) => self.shutdown = Shutdown::None,
};
match poll_result {
Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => {
return Poll::Ready(Ok(Event::Custom(event)));
}
Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol }) => {
let id = self.unique_dial_upgrade_id;
let timeout = *protocol.timeout();
self.unique_dial_upgrade_id += 1;
let (upgrade, info) = protocol.into_upgrade();
self.queued_dial_upgrades.push((id, SendWrapper(upgrade)));
return Poll::Ready(Ok(Event::OutboundSubstreamRequest((id, info, timeout))));
}
Poll::Ready(ProtocolsHandlerEvent::Close(err)) => return Poll::Ready(Err(err.into())),
Poll::Pending => (),
};
// 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(Error::KeepAliveTimeout)),
Shutdown::Later(ref mut delay, _) => match Future::poll(Pin::new(delay), cx) {
Poll::Ready(_) => return Poll::Ready(Err(Error::KeepAliveTimeout)),
Poll::Pending => {}
},
}
}
Poll::Pending
}
}
/// Event produced by a [`HandlerWrapper`].
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum Event<TOutboundOpenInfo, TCustom> {
/// Require a new outbound substream to be opened with the remote.
OutboundSubstreamRequest(TOutboundOpenInfo),
/// Other event.
Custom(TCustom),
}