libp2p-ping improvements. (#1049)

* libp2p-ping improvements.

  * re #950: Removes use of the `OneShotHandler`, but still sending each
    ping over a new substream, as seems to be intentional since #828.

  * re #842: Adds an integration test that exercises the ping behaviour through
    a Swarm, requiring the RTT to be below a threshold. This requires disabling
    Nagle's algorithm as it can interact badly with delayed ACKs (and has been
    observed to do so in the context of the new ping example and integration test).

  * re #864: Control of the inbound and outbound (sub)stream protocol upgrade
    timeouts has been moved from the `NodeHandlerWrapperBuilder` to the
    `ProtocolsHandler`. That may also alleviate the need for a custom timeout
    on an `OutboundSubstreamRequest` as a `ProtocolsHandler` is now free to
    adjust these timeouts over time.

Other changes:

  * A new ping example.
  * Documentation improvements.

* More documentation improvements.

* Add PingPolicy and ensure no event is dropped.

* Remove inbound_timeout/outbound_timeout.

As per review comment, the inbound timeout is now configured
as part of the `listen_protocol` and the outbound timeout as
part of the `OutboundSubstreamRequest`.

* Simplify and generalise.

Generalise `ListenProtocol` to `SubstreamProtocol`, reusing it in
the context of `ProtocolsHandlerEvent::OutboundSubstreamRequest`.

* Doc comments for SubstreamProtocol.

* Adapt to changes in master.

* Relax upper bound for ping integration test rtt.

For "slow" CI build machines?
This commit is contained in:
Roman Borschel
2019-04-16 15:57:29 +02:00
committed by GitHub
parent 9b6336672b
commit bee5c58b27
22 changed files with 897 additions and 382 deletions

View File

@ -19,7 +19,13 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
protocols_handler::{
KeepAlive,
SubstreamProtocol,
ProtocolsHandler,
ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr
},
upgrade::{
InboundUpgrade,
OutboundUpgrade,
@ -58,8 +64,8 @@ where
type OutboundOpenInfo = Void;
#[inline]
fn listen_protocol(&self) -> Self::InboundProtocol {
DeniedUpgrade
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
SubstreamProtocol::new(DeniedUpgrade)
}
#[inline]

View File

@ -19,7 +19,13 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
protocols_handler::{
KeepAlive,
SubstreamProtocol,
ProtocolsHandler,
ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr
},
upgrade::{
InboundUpgrade,
OutboundUpgrade,
@ -61,7 +67,7 @@ where
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
#[inline]
fn listen_protocol(&self) -> Self::InboundProtocol {
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
self.inner.listen_protocol()
}

View File

@ -19,7 +19,13 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
protocols_handler::{
KeepAlive,
SubstreamProtocol,
ProtocolsHandler,
ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr
},
upgrade::{
InboundUpgrade,
OutboundUpgrade,
@ -58,7 +64,7 @@ where
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
#[inline]
fn listen_protocol(&self) -> Self::InboundProtocol {
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
self.inner.listen_protocol()
}
@ -104,8 +110,8 @@ where
Ok(self.inner.poll()?.map(|ev| {
match ev {
ProtocolsHandlerEvent::Custom(ev) => ProtocolsHandlerEvent::Custom((self.map)(ev)),
ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info }
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info }
}
}
}))

View File

@ -18,18 +18,22 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! Once we are connected to a node, a *protocols handler* handles one or more specific protocols
//! on this connection.
//! Once a connection to a remote peer is established, a `ProtocolsHandler` negotiates
//! and handles one or more specific protocols on the connection.
//!
//! This includes: how to handle incoming substreams, which protocols are supported, when to open
//! a new outbound substream, and so on.
//! Protocols are negotiated and used on individual substreams of the connection.
//! Thus a `ProtocolsHandler` defines the inbound and outbound upgrades to apply
//! when creating a new inbound or outbound substream, respectively, and is notified
//! by a `Swarm` when these upgrades have been successfully applied, including the
//! final output of the upgrade. A `ProtocolsHandler` can then continue communicating
//! with the peer over the substream using the negotiated protocol(s).
//!
//! Each implementation of the `ProtocolsHandler` trait handles one or more specific protocols.
//! Two `ProtocolsHandler`s can be combined together with the `select()` method in order to build
//! a `ProtocolsHandler` that combines both. This can be repeated multiple times in order to create
//! a handler that handles all the protocols that you wish.
//! Two `ProtocolsHandler`s can be composed with [`ProtocolsHandler::select()`]
//! in order to build a new handler supporting the combined set of protocols,
//! with methods being dispatched to the appropriate handler according to the
//! used protocol(s) determined by the associated types of the handlers.
//!
//! > **Note**: A `ProtocolsHandler` handles one or more protocols in relation to a specific
//! > **Note**: A `ProtocolsHandler` handles one or more protocols in the context of a single
//! > connection with a remote. In order to handle a protocol that requires knowledge of
//! > the network as a whole, see the `NetworkBehaviour` trait.
@ -57,73 +61,73 @@ mod node_handler;
mod one_shot;
mod select;
/// Handler for a set of protocols for a specific connection with a remote.
/// A handler for a set of protocols used on a connection with a remote.
///
/// This trait should be implemented on a struct that holds the state for a specific protocol
/// behaviour with a specific remote.
/// This trait should be implemented for a type that maintains the state for
/// the execution of a specific protocol with a remote.
///
/// # Handling a protocol
///
/// Communication with a remote over a set of protocols opened in two different ways:
/// Communication with a remote over a set of protocols is initiated in one of two ways:
///
/// - Dialing, which is a voluntary process. In order to do so, make `poll()` return an
/// `OutboundSubstreamRequest` variant containing the connection upgrade to use to start using a
/// protocol.
/// - Listening, which is used to determine which protocols are supported when the remote wants
/// to open a substream. The `listen_protocol()` method should return the upgrades supported when
/// listening.
/// 1. Dialing by initiating a new outbound substream. In order to do so,
/// [`ProtocolsHandler::poll()`] must return an [`OutboundSubstreamRequest`], providing an
/// instance of [`ProtocolsHandler::OutboundUpgrade`] that is used to negotiate the
/// protocol(s). Upon success, [`ProtocolsHandler::inject_fully_negotiated_outbound`]
/// is called with the final output of the upgrade.
///
/// The upgrade when dialing and the upgrade when listening have to be of the same type, but you
/// are free to return for example an `OrUpgrade` enum, or an enum of your own, containing the
/// upgrade you want depending on the situation.
/// 2. Listening by accepting a new inbound substream. When a new inbound substream
/// is created on a connection, [`ProtocolsHandler::listen_protocol`] is called
/// to obtain an instance of [`ProtocolsHandler::InboundUpgrade`] that is used to
/// negotiate the protocol(s). Upon success,
/// [`ProtocolsHandler::inject_fully_negotiated_inbound`] is called with the final
/// output of the upgrade.
///
/// # Shutting down
/// # Connection Keep-Alive
///
/// A `ProtocolsHandler` can influence the lifetime of the underlying connection
/// through [`ProtocolsHandler::connection_keep_alive`]. That is, the protocol
/// 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 (either by us or by the remote) `shutdown()` is called and the
/// handler continues to be processed until it produces `ProtocolsHandlerEvent::Shutdown`. Only
/// then the handler is destroyed.
///
/// This makes it possible for the handler to finish delivering events even after knowing that it
/// is shutting down.
///
/// Implementors of this trait should keep in mind that when `shutdown()` is called, the connection
/// might already be closed or unresponsive. They should therefore not rely on being able to
/// deliver messages.
///
/// 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.
pub trait ProtocolsHandler {
/// Custom event that can be received from the outside.
type InEvent;
/// Custom event that can be produced by the handler and that will be returned to the outside.
type OutEvent;
/// Error that can happen when polling.
/// The type of errors returned by [`ProtocolsHandler::poll`].
type Error: error::Error;
/// The type of the substream that contains the raw data.
/// The type of substreams on which the protocol(s) are negotiated.
type Substream: AsyncRead + AsyncWrite;
/// The upgrade for the protocol or protocols handled by this handler.
/// The inbound upgrade for the protocol(s) used by the handler.
type InboundProtocol: InboundUpgrade<Self::Substream>;
/// The upgrade for the protocol or protocols handled by this handler.
/// The outbound upgrade for the protocol(s) used by the handler.
type OutboundProtocol: OutboundUpgrade<Self::Substream>;
/// Information about a substream. Can be sent to the handler through a `NodeHandlerEndpoint`,
/// and will be passed back in `inject_substream` or `inject_outbound_closed`.
/// The type of additional information passed to an `OutboundSubstreamRequest`.
type OutboundOpenInfo;
/// Produces a `ConnectionUpgrade` for the protocol or protocols to accept when listening.
/// The [`InboundUpgrade`] to apply on inbound substreams to negotiate the
/// desired protocols.
///
/// > **Note**: You should always accept all the protocols you support, even if in a specific
/// > context you wouldn't accept one in particular (eg. only allow one substream at
/// > a time for a given protocol). The reason is that remotes are allowed to put the
/// > list of supported protocols in a cache in order to avoid spurious queries.
fn listen_protocol(&self) -> Self::InboundProtocol;
/// > **Note**: The returned `InboundUpgrade` should always accept all the generally
/// > supported protocols, even if in a specific context a particular one is
/// > not supported, (eg. when only allowing one substream at a time for a protocol).
/// > This allows a remote to put the list of supported protocols in a cache.
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol>;
/// Injects a fully-negotiated substream in the handler.
///
/// This method is called when a substream has been successfully opened and negotiated.
/// Injects the output of a successful upgrade on a new inbound substream.
fn inject_fully_negotiated_inbound(
&mut self,
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
);
/// Injects the output of a successful upgrade on a new outbound substream.
///
/// The second argument is the information that was previously passed to
/// [`ProtocolsHandlerEvent::OutboundSubstreamRequest`].
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
@ -134,28 +138,43 @@ pub trait ProtocolsHandler {
fn inject_event(&mut self, event: Self::InEvent);
/// Indicates to the handler that upgrading a substream to the given protocol has failed.
fn inject_dial_upgrade_error(&mut self, info: Self::OutboundOpenInfo, error: ProtocolsHandlerUpgrErr<<Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Error>);
fn inject_dial_upgrade_error(
&mut self,
info: Self::OutboundOpenInfo,
error: ProtocolsHandlerUpgrErr<
<Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Error
>
);
/// Returns until when the connection should be kept alive.
///
/// If returns `Until`, that indicates that this connection may be closed and this handler
/// destroyed after the returned `Instant` has elapsed if they think that they will no longer
/// need the connection in the future. Returning `Forever` is equivalent to "infinite".
/// Returning `Now` is equivalent to `Until(Instant::now())`.
/// This method is called by the `Swarm` after each invocation of
/// [`ProtocolsHandler::poll`] to determine if the connection and the associated
/// `ProtocolsHandler`s should be kept alive and if so, for how long.
///
/// On the other hand, the return value is only an indication and doesn't mean that the user
/// will not close the connection.
/// Returning [`KeepAlive::Now`] indicates that the connection should be
/// closed and this handler destroyed immediately.
///
/// When multiple `ProtocolsHandler` are combined together, the largest `KeepAlive` should be
/// used.
/// Returning [`KeepAlive::Until`] indicates that the connection may be closed
/// and this handler destroyed after the specified `Instant`.
///
/// Returning [`KeepAlive::Forever`] indicates that the connection should
/// be kept alive until the next call to this method.
///
/// > **Note**: The connection is always closed and the handler destroyed
/// > when [`ProtocolsHandler::poll`] returns an error. Furthermore, the
/// > connection may be closed for reasons outside of the control
/// > of the handler.
///
/// The result of this method should be checked every time `poll()` is invoked.
fn connection_keep_alive(&self) -> KeepAlive;
/// Should behave like `Stream::poll()`.
///
/// Returning an error will close the connection to the remote.
fn poll(&mut self) -> Poll<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>;
fn poll(&mut self) -> Poll<
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
Self::Error
>;
/// Adds a closure that turns the input event into something else.
#[inline]
@ -177,8 +196,12 @@ pub trait ProtocolsHandler {
MapOutEvent::new(self, map)
}
/// Builds an implementation of `ProtocolsHandler` that handles both this protocol and the
/// other one together.
/// Creates a new `ProtocolsHandler` that selects either this handler or
/// `other` by delegating methods calls appropriately.
///
/// > **Note**: The largest `KeepAlive` returned by the two handlers takes precedence,
/// > i.e. is returned from [`ProtocolsHandler::connection_keep_alive`] by the returned
/// > handler.
#[inline]
fn select<TProto2>(self, other: TProto2) -> ProtocolsHandlerSelect<Self, TProto2>
where
@ -187,8 +210,10 @@ pub trait ProtocolsHandler {
ProtocolsHandlerSelect::new(self, other)
}
/// Creates a builder that will allow creating a `NodeHandler` that handles this protocol
/// Creates a builder that allows creating a `NodeHandler` that handles this protocol
/// exclusively.
///
/// > **Note**: This method should not be redefined in a custom `ProtocolsHandler`.
#[inline]
fn into_node_handler_builder(self) -> NodeHandlerWrapperBuilder<Self>
where
@ -211,13 +236,75 @@ pub trait ProtocolsHandler {
}
}
/// Configuration of inbound or outbound substream protocol(s)
/// for a [`ProtocolsHandler`].
///
/// The inbound substream protocol(s) are defined by [`ProtocolsHandler::listen_protocol`]
/// and the outbound substream protocol(s) by [`ProtocolsHandlerEvent::OutboundSubstreamRequest`].
#[derive(Copy, Clone, Debug, PartialEq, Eq)]
pub struct SubstreamProtocol<TUpgrade> {
upgrade: TUpgrade,
timeout: Duration,
}
impl<TUpgrade> SubstreamProtocol<TUpgrade> {
/// Create a new `ListenProtocol` from the given upgrade.
///
/// The default timeout for applying the given upgrade on a substream is
/// 10 seconds.
pub fn new(upgrade: TUpgrade) -> SubstreamProtocol<TUpgrade> {
SubstreamProtocol {
upgrade,
timeout: Duration::from_secs(10),
}
}
/// Maps a function over the protocol upgrade.
pub fn map_upgrade<U, F>(self, f: F) -> SubstreamProtocol<U>
where
F: FnOnce(TUpgrade) -> U,
{
SubstreamProtocol {
upgrade: f(self.upgrade),
timeout: self.timeout,
}
}
/// Sets a new timeout for the protocol upgrade.
pub fn with_timeout(mut self, timeout: Duration) -> Self {
self.timeout = timeout;
self
}
/// Borrows the contained protocol upgrade.
pub fn upgrade(&self) -> &TUpgrade {
&self.upgrade
}
/// Borrows the timeout for the protocol upgrade.
pub fn timeout(&self) -> &Duration {
&self.timeout
}
/// Converts the substream protocol configuration into the contained upgrade.
pub fn into_upgrade(self) -> TUpgrade {
self.upgrade
}
}
impl<TUpgrade> From<TUpgrade> for SubstreamProtocol<TUpgrade> {
fn from(upgrade: TUpgrade) -> SubstreamProtocol<TUpgrade> {
SubstreamProtocol::new(upgrade)
}
}
/// Event produced by a handler.
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom> {
/// Request a new outbound substream to be opened with the remote.
OutboundSubstreamRequest {
/// The upgrade to apply on the substream.
upgrade: TConnectionUpgrade,
/// The protocol(s) to apply on the substream.
protocol: SubstreamProtocol<TConnectionUpgrade>,
/// User-defined information, passed back when the substream is open.
info: TOutboundOpenInfo,
},
@ -230,7 +317,8 @@ pub enum ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom> {
impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
{
/// If this is an `OutboundSubstreamRequest`, maps the `info` member from a `TOutboundOpenInfo` to something else.
/// If this is an `OutboundSubstreamRequest`, maps the `info` member from a
/// `TOutboundOpenInfo` to something else.
#[inline]
pub fn map_outbound_open_info<F, I>(
self,
@ -240,9 +328,9 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
F: FnOnce(TOutboundOpenInfo) -> I,
{
match self {
ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade,
protocol,
info: map(info),
}
}
@ -250,7 +338,8 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
}
}
/// If this is an `OutboundSubstreamRequest`, maps the protocol (`TConnectionUpgrade`) to something else.
/// If this is an `OutboundSubstreamRequest`, maps the protocol (`TConnectionUpgrade`)
/// to something else.
#[inline]
pub fn map_protocol<F, I>(
self,
@ -260,9 +349,9 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
F: FnOnce(TConnectionUpgrade) -> I,
{
match self {
ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade: map(upgrade),
protocol: protocol.map_upgrade(map),
info,
}
}
@ -280,8 +369,8 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
F: FnOnce(TCustom) -> I,
{
match self {
ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info }
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => {
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info }
}
ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(map(val)),
}
@ -356,7 +445,7 @@ pub trait IntoProtocolsHandler {
where
Self: Sized,
{
NodeHandlerWrapperBuilder::new(self, Duration::from_secs(10), Duration::from_secs(10))
NodeHandlerWrapperBuilder::new(self)
}
}

View File

@ -31,17 +31,13 @@ use crate::{
}
};
use futures::prelude::*;
use std::{error, fmt, time::Duration, time::Instant};
use std::{error, fmt, time::{Duration, Instant}};
use tokio_timer::{Delay, Timeout};
/// Prototype for a `NodeHandlerWrapper`.
pub struct NodeHandlerWrapperBuilder<TIntoProtoHandler> {
/// The underlying handler.
handler: TIntoProtoHandler,
/// Timeout for incoming substreams negotiation.
in_timeout: Duration,
/// Timeout for outgoing substreams negotiation.
out_timeout: Duration,
}
impl<TIntoProtoHandler> NodeHandlerWrapperBuilder<TIntoProtoHandler>
@ -50,28 +46,12 @@ where
{
/// Builds a `NodeHandlerWrapperBuilder`.
#[inline]
pub(crate) fn new(handler: TIntoProtoHandler, in_timeout: Duration, out_timeout: Duration) -> Self {
pub(crate) fn new(handler: TIntoProtoHandler) -> Self {
NodeHandlerWrapperBuilder {
handler,
in_timeout,
out_timeout,
}
}
/// Sets the timeout to use when negotiating a protocol on an ingoing substream.
#[inline]
pub fn with_in_negotiation_timeout(mut self, timeout: Duration) -> Self {
self.in_timeout = timeout;
self
}
/// Sets the timeout to use when negotiating a protocol on an outgoing substream.
#[inline]
pub fn with_out_negotiation_timeout(mut self, timeout: Duration) -> Self {
self.out_timeout = timeout;
self
}
/// Builds the `NodeHandlerWrapper`.
#[deprecated(note = "Pass the NodeHandlerWrapperBuilder directly")]
#[inline]
@ -82,8 +62,6 @@ where
handler: self.handler,
negotiating_in: Vec::new(),
negotiating_out: Vec::new(),
in_timeout: self.in_timeout,
out_timeout: self.out_timeout,
queued_dial_upgrades: Vec::new(),
unique_dial_upgrade_id: 0,
connection_shutdown: None,
@ -105,8 +83,6 @@ where
handler: self.handler.into_handler(remote_peer_id),
negotiating_in: Vec::new(),
negotiating_out: Vec::new(),
in_timeout: self.in_timeout,
out_timeout: self.out_timeout,
queued_dial_upgrades: Vec::new(),
unique_dial_upgrade_id: 0,
connection_shutdown: None,
@ -131,10 +107,6 @@ where
TProtoHandler::OutboundOpenInfo,
Timeout<OutboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::OutboundProtocol>>,
)>,
/// Timeout for incoming substreams negotiation.
in_timeout: Duration,
/// Timeout for outgoing substreams negotiation.
out_timeout: Duration,
/// 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, TProtoHandler::OutboundProtocol)>,
@ -197,7 +169,7 @@ where
type Substream = TProtoHandler::Substream;
// The first element of the tuple is the unique upgrade identifier
// (see `unique_dial_upgrade_id`).
type OutboundOpenInfo = (u64, TProtoHandler::OutboundOpenInfo);
type OutboundOpenInfo = (u64, TProtoHandler::OutboundOpenInfo, Duration);
fn inject_substream(
&mut self,
@ -207,11 +179,12 @@ where
match endpoint {
NodeHandlerEndpoint::Listener => {
let protocol = self.handler.listen_protocol();
let upgrade = upgrade::apply_inbound(substream, protocol);
let with_timeout = Timeout::new(upgrade, self.in_timeout);
let timeout = protocol.timeout().clone();
let upgrade = upgrade::apply_inbound(substream, protocol.into_upgrade());
let with_timeout = Timeout::new(upgrade, timeout);
self.negotiating_in.push(with_timeout);
}
NodeHandlerEndpoint::Dialer((upgrade_id, user_data)) => {
NodeHandlerEndpoint::Dialer((upgrade_id, user_data, timeout)) => {
let pos = match self
.queued_dial_upgrades
.iter()
@ -226,7 +199,7 @@ where
let (_, proto_upgrade) = self.queued_dial_upgrades.remove(pos);
let upgrade = upgrade::apply_outbound(substream, proto_upgrade);
let with_timeout = Timeout::new(upgrade, self.out_timeout);
let with_timeout = Timeout::new(upgrade, timeout);
self.negotiating_out.push((user_data, with_timeout));
}
}
@ -270,7 +243,7 @@ where
} else {
debug_assert!(err.is_inner());
let err = err.into_inner().expect("Timeout error is one of {elapsed, \
timer, inner}; is_inner and is_elapsed are both false; error is \
timer, inner}; is_elapsed and is_timer are both false; error is \
inner; QED");
ProtocolsHandlerUpgrErr::Upgrade(err)
};
@ -280,8 +253,8 @@ where
}
}
// Poll the handler at the end so that we see the consequences of the method calls on
// `self.handler`.
// 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()?;
self.connection_shutdown = match self.handler.connection_keep_alive() {
@ -295,14 +268,15 @@ where
return Ok(Async::Ready(NodeHandlerEvent::Custom(event)));
}
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade,
protocol,
info,
}) => {
let id = self.unique_dial_upgrade_id;
let timeout = protocol.timeout().clone();
self.unique_dial_upgrade_id += 1;
self.queued_dial_upgrades.push((id, upgrade));
self.queued_dial_upgrades.push((id, protocol.into_upgrade()));
return Ok(Async::Ready(
NodeHandlerEvent::OutboundSubstreamRequest((id, info)),
NodeHandlerEvent::OutboundSubstreamRequest((id, info, timeout)),
));
}
Async::NotReady => (),

View File

@ -20,6 +20,7 @@
use crate::protocols_handler::{
KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr,
SubstreamProtocol
};
use crate::upgrade::{InboundUpgrade, OutboundUpgrade};
use futures::prelude::*;
@ -36,7 +37,7 @@ where
TOutProto: OutboundUpgrade<TSubstream>,
{
/// The upgrade for inbound substreams.
listen_protocol: TInProto,
listen_protocol: SubstreamProtocol<TInProto>,
/// If `Some`, something bad happened and we should shut down the handler with an error.
pending_error:
Option<ProtocolsHandlerUpgrErr<<TOutProto as OutboundUpgrade<TSubstream>>::Error>>,
@ -63,7 +64,10 @@ where
{
/// Creates a `OneShotHandler`.
#[inline]
pub fn new(listen_protocol: TInProto, inactive_timeout: Duration) -> Self {
pub fn new(
listen_protocol: SubstreamProtocol<TInProto>,
inactive_timeout: Duration
) -> Self {
OneShotHandler {
listen_protocol,
pending_error: None,
@ -88,7 +92,7 @@ where
/// > **Note**: If you modify the protocol, modifications will only applies to future inbound
/// > substreams, not the ones already being negotiated.
#[inline]
pub fn listen_protocol_ref(&self) -> &TInProto {
pub fn listen_protocol_ref(&self) -> &SubstreamProtocol<TInProto> {
&self.listen_protocol
}
@ -97,7 +101,7 @@ where
/// > **Note**: If you modify the protocol, modifications will only applies to future inbound
/// > substreams, not the ones already being negotiated.
#[inline]
pub fn listen_protocol_mut(&mut self) -> &mut TInProto {
pub fn listen_protocol_mut(&mut self) -> &mut SubstreamProtocol<TInProto> {
&mut self.listen_protocol
}
@ -113,11 +117,11 @@ impl<TSubstream, TInProto, TOutProto, TOutEvent> Default
for OneShotHandler<TSubstream, TInProto, TOutProto, TOutEvent>
where
TOutProto: OutboundUpgrade<TSubstream>,
TInProto: Default,
TInProto: InboundUpgrade<TSubstream> + Default,
{
#[inline]
fn default() -> Self {
OneShotHandler::new(Default::default(), Duration::from_secs(10))
OneShotHandler::new(SubstreamProtocol::new(Default::default()), Duration::from_secs(10))
}
}
@ -125,11 +129,12 @@ impl<TSubstream, TInProto, TOutProto, TOutEvent> ProtocolsHandler
for OneShotHandler<TSubstream, TInProto, TOutProto, TOutEvent>
where
TSubstream: AsyncRead + AsyncWrite,
TInProto: InboundUpgrade<TSubstream> + Clone,
TInProto: InboundUpgrade<TSubstream>,
TOutProto: OutboundUpgrade<TSubstream>,
TInProto::Output: Into<TOutEvent>,
TOutProto::Output: Into<TOutEvent>,
TOutProto::Error: error::Error + 'static,
SubstreamProtocol<TInProto>: Clone,
{
type InEvent = TOutProto;
type OutEvent = TOutEvent;
@ -142,7 +147,7 @@ where
type OutboundOpenInfo = ();
#[inline]
fn listen_protocol(&self) -> Self::InboundProtocol {
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
self.listen_protocol.clone()
}
@ -220,7 +225,7 @@ where
self.dial_negotiated += 1;
return Ok(Async::Ready(
ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade: self.dial_queue.remove(0),
protocol: SubstreamProtocol::new(self.dial_queue.remove(0)),
info: (),
},
));

View File

@ -24,6 +24,7 @@ use crate::{
either::EitherOutput,
protocols_handler::{
KeepAlive,
SubstreamProtocol,
IntoProtocolsHandler,
ProtocolsHandler,
ProtocolsHandlerEvent,
@ -123,10 +124,12 @@ where
type OutboundOpenInfo = EitherOutput<TProto1::OutboundOpenInfo, TProto2::OutboundOpenInfo>;
#[inline]
fn listen_protocol(&self) -> Self::InboundProtocol {
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
let proto1 = self.proto1.listen_protocol();
let proto2 = self.proto2.listen_protocol();
SelectUpgrade::new(proto1, proto2)
let timeout = std::cmp::max(proto1.timeout(), proto2.timeout()).clone();
SubstreamProtocol::new(SelectUpgrade::new(proto1.into_upgrade(), proto2.into_upgrade()))
.with_timeout(timeout)
}
fn inject_fully_negotiated_outbound(&mut self, protocol: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output, endpoint: Self::OutboundOpenInfo) {
@ -206,9 +209,12 @@ where
Async::Ready(ProtocolsHandlerEvent::Custom(event)) => {
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::First(event))));
},
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info}) => {
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
protocol,
info,
}) => {
return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade: EitherUpgrade::A(upgrade),
protocol: protocol.map_upgrade(EitherUpgrade::A),
info: EitherOutput::First(info),
}));
},
@ -219,9 +225,12 @@ where
Async::Ready(ProtocolsHandlerEvent::Custom(event)) => {
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::Second(event))));
},
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info }) => {
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
protocol,
info,
}) => {
return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade: EitherUpgrade::B(upgrade),
protocol: protocol.map_upgrade(EitherUpgrade::B),
info: EitherOutput::Second(info),
}));
},

View File

@ -37,7 +37,7 @@ pub trait NetworkBehaviour {
/// Event generated by the swarm.
type OutEvent;
/// Builds a new `ProtocolsHandler`.
/// Creates a new `ProtocolsHandler` for a connection with a peer.
fn new_handler(&mut self) -> Self::ProtocolsHandler;
/// Addresses that this behaviour is aware of for this specific peer, and that may allow
@ -104,45 +104,48 @@ pub trait NetworkBehaviourEventProcess<TEvent> {
fn inject_event(&mut self, event: TEvent);
}
/// Action to perform.
/// An action that a [`NetworkBehaviour`] can trigger in the [`Swarm`]
/// in whose context it is executing.
#[derive(Debug, Clone)]
pub enum NetworkBehaviourAction<TInEvent, TOutEvent> {
/// Generate an event for the outside.
/// Instructs the `Swarm` to return an event when it is being polled.
GenerateEvent(TOutEvent),
// TODO: report new raw connection for usage after intercepting an address dial
/// Instructs the swarm to dial the given multiaddress without any expectation of a peer id.
/// Instructs the swarm to dial the given multiaddress, without a known `PeerId`.
DialAddress {
/// The address to dial.
address: Multiaddr,
},
/// Instructs the swarm to try reach the given peer.
/// Instructs the swarm to dial a known `PeerId`.
///
/// In the future, a corresponding `inject_dial_failure` or `inject_connected` function call
/// must be performed.
/// On success, [`NetworkBehaviour::inject_connected`] is invoked.
/// On failure, [`NetworkBehaviour::inject_dial_failure`] is invoked.
DialPeer {
/// The peer to try reach.
peer_id: PeerId,
},
/// If we're connected to the given peer, sends a message to the protocol handler.
/// Instructs the `Swarm` to send a message to a connected peer.
///
/// If we're not connected to this peer, does nothing. If necessary, the implementation of
/// `NetworkBehaviour` is supposed to track which peers we are connected to.
/// If the `Swarm` is connected to the peer, the message is delivered to the remote's
/// protocol handler. If there is no connection to the peer, the message is ignored.
/// To ensure delivery, the `NetworkBehaviour` must keep track of connected peers.
SendEvent {
/// The peer which to send the message to.
/// The peer to which to send the message.
peer_id: PeerId,
/// Event to send to the peer.
/// The message to send.
event: TInEvent,
},
/// Reports that a remote observes us as this address.
/// Informs the `Swarm` about a multi-address observed by a remote for
/// the local node.
///
/// The swarm will pass this address through the transport's NAT traversal.
ReportObservedAddr {
/// The address we're being observed as.
/// The observed address of the local node.
address: Multiaddr,
},
}

View File

@ -438,6 +438,7 @@ where TBehaviour: NetworkBehaviour,
.new_handler()
.into_handler(&self.local_peer_id)
.listen_protocol()
.into_upgrade()
.protocol_info()
.into_iter()
.map(|info| info.protocol_name().to_vec())

View File

@ -20,7 +20,14 @@
use crate::{
either::EitherOutput,
protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, IntoProtocolsHandler},
protocols_handler::{
KeepAlive,
SubstreamProtocol,
ProtocolsHandler,
ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr,
IntoProtocolsHandler
},
swarm::{NetworkBehaviour, NetworkBehaviourAction, NetworkBehaviourEventProcess},
upgrade::{InboundUpgrade, OutboundUpgrade, DeniedUpgrade, EitherUpgrade},
PeerId, Multiaddr, nodes::ConnectedPoint, swarm::PollParameters,
@ -168,11 +175,11 @@ where
type OutboundProtocol = TInner::OutboundProtocol;
type OutboundOpenInfo = TInner::OutboundOpenInfo;
fn listen_protocol(&self) -> Self::InboundProtocol {
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
if let Some(inner) = self.inner.as_ref() {
EitherUpgrade::A(inner.listen_protocol())
inner.listen_protocol().map_upgrade(EitherUpgrade::A)
} else {
EitherUpgrade::B(DeniedUpgrade)
SubstreamProtocol::new(EitherUpgrade::B(DeniedUpgrade))
}
}