mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-24 23:31:33 +00:00
feat: report changes in supported protocols to ConnectionHandler
With this patch, implementations of `ConnectionHandler` (which are typically composed in a tree) can exchange information about the supported protocols of a remote with each other via `ConnectionHandlerEvent::ReportRemoteProtocols`. The provided `ProtocolSupport` enum can describe either additions or removals of the remote peer's protocols. This information is aggregated in the connection and passed down to the `ConnectionHandler` via `ConnectionEvent::RemoteProtocolsChange`. Similarly, if the listen protocols of a connection change, all `ConnectionHandler`s on the connection will be notified via `ConnectionEvent::LocalProtocolsChange`. This will allow us to eventually remove `PollParameters` from `NetworkBehaviour`. This pattern allows protocols on a connection to communicate with each other. For example, protocols like identify can share the list of (supposedly) supported protocols by the remote with all other handlers. A protocol like kademlia can accurately add and remove a remote from its routing table as a result. Resolves: #2680. Related: #3124. Pull-Request: #3651.
This commit is contained in:
@ -230,6 +230,9 @@ pub trait PollParameters {
|
||||
/// The iterator's elements are the ASCII names as reported on the wire.
|
||||
///
|
||||
/// Note that the list is computed once at initialization and never refreshed.
|
||||
#[deprecated(
|
||||
note = "Use `libp2p_swarm::SupportedProtocols` in your `ConnectionHandler` instead."
|
||||
)]
|
||||
fn supported_protocols(&self) -> Self::SupportedProtocolsIter;
|
||||
|
||||
/// Returns the list of the addresses we're listening on.
|
||||
|
@ -362,6 +362,16 @@ where
|
||||
ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => {
|
||||
self.on_listen_upgrade_error(listen_upgrade_error)
|
||||
}
|
||||
ConnectionEvent::LocalProtocolsChange(change) => {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.on_connection_event(ConnectionEvent::LocalProtocolsChange(change));
|
||||
}
|
||||
}
|
||||
ConnectionEvent::RemoteProtocolsChange(change) => {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.on_connection_event(ConnectionEvent::RemoteProtocolsChange(change));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -21,18 +21,23 @@
|
||||
mod error;
|
||||
|
||||
pub(crate) mod pool;
|
||||
mod supported_protocols;
|
||||
|
||||
pub use error::ConnectionError;
|
||||
pub(crate) use error::{
|
||||
PendingConnectionError, PendingInboundConnectionError, PendingOutboundConnectionError,
|
||||
};
|
||||
pub use supported_protocols::SupportedProtocols;
|
||||
|
||||
use crate::handler::{
|
||||
AddressChange, ConnectionEvent, ConnectionHandler, DialUpgradeError, FullyNegotiatedInbound,
|
||||
FullyNegotiatedOutbound, ListenUpgradeError,
|
||||
FullyNegotiatedOutbound, ListenUpgradeError, ProtocolSupport, ProtocolsAdded, ProtocolsChange,
|
||||
UpgradeInfoSend,
|
||||
};
|
||||
use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, SendWrapper};
|
||||
use crate::{ConnectionHandlerEvent, KeepAlive, StreamUpgradeError, SubstreamProtocol};
|
||||
use crate::{
|
||||
ConnectionHandlerEvent, KeepAlive, StreamProtocol, StreamUpgradeError, SubstreamProtocol,
|
||||
};
|
||||
use futures::stream::FuturesUnordered;
|
||||
use futures::FutureExt;
|
||||
use futures::StreamExt;
|
||||
@ -47,6 +52,7 @@ use libp2p_core::upgrade::{
|
||||
};
|
||||
use libp2p_core::Endpoint;
|
||||
use libp2p_identity::PeerId;
|
||||
use std::collections::HashSet;
|
||||
use std::future::Future;
|
||||
use std::sync::atomic::{AtomicUsize, Ordering};
|
||||
use std::task::Waker;
|
||||
@ -147,6 +153,9 @@ where
|
||||
requested_substreams: FuturesUnordered<
|
||||
SubstreamRequested<THandler::OutboundOpenInfo, THandler::OutboundProtocol>,
|
||||
>,
|
||||
|
||||
local_supported_protocols: HashSet<StreamProtocol>,
|
||||
remote_supported_protocols: HashSet<StreamProtocol>,
|
||||
}
|
||||
|
||||
impl<THandler> fmt::Debug for Connection<THandler>
|
||||
@ -171,10 +180,18 @@ where
|
||||
/// and connection handler.
|
||||
pub(crate) fn new(
|
||||
muxer: StreamMuxerBox,
|
||||
handler: THandler,
|
||||
mut handler: THandler,
|
||||
substream_upgrade_protocol_override: Option<upgrade::Version>,
|
||||
max_negotiating_inbound_streams: usize,
|
||||
) -> Self {
|
||||
let initial_protocols = gather_supported_protocols(&handler);
|
||||
|
||||
if !initial_protocols.is_empty() {
|
||||
handler.on_connection_event(ConnectionEvent::LocalProtocolsChange(
|
||||
ProtocolsChange::Added(ProtocolsAdded::from_set(&initial_protocols)),
|
||||
));
|
||||
}
|
||||
|
||||
Connection {
|
||||
muxing: muxer,
|
||||
handler,
|
||||
@ -184,6 +201,8 @@ where
|
||||
substream_upgrade_protocol_override,
|
||||
max_negotiating_inbound_streams,
|
||||
requested_substreams: Default::default(),
|
||||
local_supported_protocols: initial_protocols,
|
||||
remote_supported_protocols: Default::default(),
|
||||
}
|
||||
}
|
||||
|
||||
@ -213,6 +232,8 @@ where
|
||||
shutdown,
|
||||
max_negotiating_inbound_streams,
|
||||
substream_upgrade_protocol_override,
|
||||
local_supported_protocols: supported_protocols,
|
||||
remote_supported_protocols,
|
||||
} = self.get_mut();
|
||||
|
||||
loop {
|
||||
@ -246,6 +267,31 @@ where
|
||||
Poll::Ready(ConnectionHandlerEvent::Close(err)) => {
|
||||
return Poll::Ready(Err(ConnectionError::Handler(err)));
|
||||
}
|
||||
Poll::Ready(ConnectionHandlerEvent::ReportRemoteProtocols(
|
||||
ProtocolSupport::Added(protocols),
|
||||
)) => {
|
||||
if let Some(added) =
|
||||
ProtocolsChange::add(remote_supported_protocols, &protocols)
|
||||
{
|
||||
handler.on_connection_event(ConnectionEvent::RemoteProtocolsChange(added));
|
||||
remote_supported_protocols.extend(protocols);
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(ConnectionHandlerEvent::ReportRemoteProtocols(
|
||||
ProtocolSupport::Removed(protocols),
|
||||
)) => {
|
||||
if let Some(removed) =
|
||||
ProtocolsChange::remove(remote_supported_protocols, &protocols)
|
||||
{
|
||||
handler
|
||||
.on_connection_event(ConnectionEvent::RemoteProtocolsChange(removed));
|
||||
remote_supported_protocols.retain(|p| !protocols.contains(p));
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
// In case the [`ConnectionHandler`] can not make any more progress, poll the negotiating outbound streams.
|
||||
@ -376,9 +422,33 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
let new_protocols = gather_supported_protocols(handler);
|
||||
|
||||
for change in ProtocolsChange::from_full_sets(supported_protocols, &new_protocols) {
|
||||
handler.on_connection_event(ConnectionEvent::LocalProtocolsChange(change));
|
||||
}
|
||||
|
||||
*supported_protocols = new_protocols;
|
||||
|
||||
return Poll::Pending; // Nothing can make progress, return `Pending`.
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn poll_noop_waker(
|
||||
&mut self,
|
||||
) -> Poll<Result<Event<THandler::OutEvent>, ConnectionError<THandler::Error>>> {
|
||||
Pin::new(self).poll(&mut Context::from_waker(futures::task::noop_waker_ref()))
|
||||
}
|
||||
}
|
||||
|
||||
fn gather_supported_protocols(handler: &impl ConnectionHandler) -> HashSet<StreamProtocol> {
|
||||
handler
|
||||
.listen_protocol()
|
||||
.upgrade()
|
||||
.protocol_info()
|
||||
.filter_map(|i| StreamProtocol::try_from_owned(i.as_ref().to_owned()).ok())
|
||||
.collect()
|
||||
}
|
||||
|
||||
/// Borrowed information about an incoming connection currently being negotiated.
|
||||
@ -605,9 +675,10 @@ enum Shutdown {
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::keep_alive;
|
||||
use futures::future;
|
||||
use futures::AsyncRead;
|
||||
use futures::AsyncWrite;
|
||||
use libp2p_core::upgrade::DeniedUpgrade;
|
||||
use libp2p_core::upgrade::{DeniedUpgrade, InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||
use libp2p_core::StreamMuxer;
|
||||
use quickcheck::*;
|
||||
use std::sync::{Arc, Weak};
|
||||
@ -629,8 +700,7 @@ mod tests {
|
||||
max_negotiating_inbound_streams,
|
||||
);
|
||||
|
||||
let result = Pin::new(&mut connection)
|
||||
.poll(&mut Context::from_waker(futures::task::noop_waker_ref()));
|
||||
let result = connection.poll_noop_waker();
|
||||
|
||||
assert!(result.is_pending());
|
||||
assert_eq!(
|
||||
@ -654,13 +724,11 @@ mod tests {
|
||||
);
|
||||
|
||||
connection.handler.open_new_outbound();
|
||||
let _ = Pin::new(&mut connection)
|
||||
.poll(&mut Context::from_waker(futures::task::noop_waker_ref()));
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
std::thread::sleep(upgrade_timeout + Duration::from_secs(1));
|
||||
|
||||
let _ = Pin::new(&mut connection)
|
||||
.poll(&mut Context::from_waker(futures::task::noop_waker_ref()));
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert!(matches!(
|
||||
connection.handler.error.unwrap(),
|
||||
@ -668,6 +736,94 @@ mod tests {
|
||||
))
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn propagates_changes_to_supported_inbound_protocols() {
|
||||
let mut connection = Connection::new(
|
||||
StreamMuxerBox::new(PendingStreamMuxer),
|
||||
ConfigurableProtocolConnectionHandler::default(),
|
||||
None,
|
||||
0,
|
||||
);
|
||||
|
||||
// First, start listening on a single protocol.
|
||||
connection.handler.listen_on(&["/foo"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(connection.handler.local_added, vec![vec!["/foo"]]);
|
||||
assert!(connection.handler.local_removed.is_empty());
|
||||
|
||||
// Second, listen on two protocols.
|
||||
connection.handler.listen_on(&["/foo", "/bar"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(
|
||||
connection.handler.local_added,
|
||||
vec![vec!["/foo"], vec!["/bar"]],
|
||||
"expect to only receive an event for the newly added protocols"
|
||||
);
|
||||
assert!(connection.handler.local_removed.is_empty());
|
||||
|
||||
// Third, stop listening on the first protocol.
|
||||
connection.handler.listen_on(&["/bar"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(
|
||||
connection.handler.local_added,
|
||||
vec![vec!["/foo"], vec!["/bar"]]
|
||||
);
|
||||
assert_eq!(connection.handler.local_removed, vec![vec!["/foo"]]);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn only_propagtes_actual_changes_to_remote_protocols_to_handler() {
|
||||
let mut connection = Connection::new(
|
||||
StreamMuxerBox::new(PendingStreamMuxer),
|
||||
ConfigurableProtocolConnectionHandler::default(),
|
||||
None,
|
||||
0,
|
||||
);
|
||||
|
||||
// First, remote supports a single protocol.
|
||||
connection.handler.remote_adds_support_for(&["/foo"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(connection.handler.remote_added, vec![vec!["/foo"]]);
|
||||
assert!(connection.handler.remote_removed.is_empty());
|
||||
|
||||
// Second, it adds a protocol but also still includes the first one.
|
||||
connection
|
||||
.handler
|
||||
.remote_adds_support_for(&["/foo", "/bar"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(
|
||||
connection.handler.remote_added,
|
||||
vec![vec!["/foo"], vec!["/bar"]],
|
||||
"expect to only receive an event for the newly added protocol"
|
||||
);
|
||||
assert!(connection.handler.remote_removed.is_empty());
|
||||
|
||||
// Third, stop listening on a protocol it never advertised (we can't control what handlers do so this needs to be handled gracefully).
|
||||
connection.handler.remote_removes_support_for(&["/baz"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(
|
||||
connection.handler.remote_added,
|
||||
vec![vec!["/foo"], vec!["/bar"]]
|
||||
);
|
||||
assert!(&connection.handler.remote_removed.is_empty());
|
||||
|
||||
// Fourth, stop listening on a protocol that was previously supported
|
||||
connection.handler.remote_removes_support_for(&["/bar"]);
|
||||
let _ = connection.poll_noop_waker();
|
||||
|
||||
assert_eq!(
|
||||
connection.handler.remote_added,
|
||||
vec![vec!["/foo"], vec!["/bar"]]
|
||||
);
|
||||
assert_eq!(connection.handler.remote_removed, vec![vec!["/bar"]]);
|
||||
}
|
||||
|
||||
struct DummyStreamMuxer {
|
||||
counter: Arc<()>,
|
||||
}
|
||||
@ -785,6 +941,40 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
struct ConfigurableProtocolConnectionHandler {
|
||||
events: Vec<ConnectionHandlerEvent<DeniedUpgrade, (), Void, Void>>,
|
||||
active_protocols: HashSet<StreamProtocol>,
|
||||
local_added: Vec<Vec<StreamProtocol>>,
|
||||
local_removed: Vec<Vec<StreamProtocol>>,
|
||||
remote_added: Vec<Vec<StreamProtocol>>,
|
||||
remote_removed: Vec<Vec<StreamProtocol>>,
|
||||
}
|
||||
|
||||
impl ConfigurableProtocolConnectionHandler {
|
||||
fn listen_on(&mut self, protocols: &[&'static str]) {
|
||||
self.active_protocols = protocols.iter().copied().map(StreamProtocol::new).collect();
|
||||
}
|
||||
|
||||
fn remote_adds_support_for(&mut self, protocols: &[&'static str]) {
|
||||
self.events
|
||||
.push(ConnectionHandlerEvent::ReportRemoteProtocols(
|
||||
ProtocolSupport::Added(
|
||||
protocols.iter().copied().map(StreamProtocol::new).collect(),
|
||||
),
|
||||
));
|
||||
}
|
||||
|
||||
fn remote_removes_support_for(&mut self, protocols: &[&'static str]) {
|
||||
self.events
|
||||
.push(ConnectionHandlerEvent::ReportRemoteProtocols(
|
||||
ProtocolSupport::Removed(
|
||||
protocols.iter().copied().map(StreamProtocol::new).collect(),
|
||||
),
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
impl ConnectionHandler for MockConnectionHandler {
|
||||
type InEvent = Void;
|
||||
type OutEvent = Void;
|
||||
@ -821,7 +1011,10 @@ mod tests {
|
||||
ConnectionEvent::DialUpgradeError(DialUpgradeError { error, .. }) => {
|
||||
self.error = Some(error)
|
||||
}
|
||||
ConnectionEvent::AddressChange(_) | ConnectionEvent::ListenUpgradeError(_) => {}
|
||||
ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::ListenUpgradeError(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_) => {}
|
||||
}
|
||||
}
|
||||
|
||||
@ -855,6 +1048,112 @@ mod tests {
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
||||
impl ConnectionHandler for ConfigurableProtocolConnectionHandler {
|
||||
type InEvent = Void;
|
||||
type OutEvent = Void;
|
||||
type Error = Void;
|
||||
type InboundProtocol = ManyProtocolsUpgrade;
|
||||
type OutboundProtocol = DeniedUpgrade;
|
||||
type InboundOpenInfo = ();
|
||||
type OutboundOpenInfo = ();
|
||||
|
||||
fn listen_protocol(
|
||||
&self,
|
||||
) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
|
||||
SubstreamProtocol::new(
|
||||
ManyProtocolsUpgrade {
|
||||
protocols: Vec::from_iter(self.active_protocols.clone()),
|
||||
},
|
||||
(),
|
||||
)
|
||||
}
|
||||
|
||||
fn on_connection_event(
|
||||
&mut self,
|
||||
event: ConnectionEvent<
|
||||
Self::InboundProtocol,
|
||||
Self::OutboundProtocol,
|
||||
Self::InboundOpenInfo,
|
||||
Self::OutboundOpenInfo,
|
||||
>,
|
||||
) {
|
||||
match event {
|
||||
ConnectionEvent::LocalProtocolsChange(ProtocolsChange::Added(added)) => {
|
||||
self.local_added.push(added.cloned().collect())
|
||||
}
|
||||
ConnectionEvent::LocalProtocolsChange(ProtocolsChange::Removed(removed)) => {
|
||||
self.local_removed.push(removed.cloned().collect())
|
||||
}
|
||||
ConnectionEvent::RemoteProtocolsChange(ProtocolsChange::Added(added)) => {
|
||||
self.remote_added.push(added.cloned().collect())
|
||||
}
|
||||
ConnectionEvent::RemoteProtocolsChange(ProtocolsChange::Removed(removed)) => {
|
||||
self.remote_removed.push(removed.cloned().collect())
|
||||
}
|
||||
_ => {}
|
||||
}
|
||||
}
|
||||
|
||||
fn on_behaviour_event(&mut self, event: Self::InEvent) {
|
||||
void::unreachable(event)
|
||||
}
|
||||
|
||||
fn connection_keep_alive(&self) -> KeepAlive {
|
||||
KeepAlive::Yes
|
||||
}
|
||||
|
||||
fn poll(
|
||||
&mut self,
|
||||
_: &mut Context<'_>,
|
||||
) -> Poll<
|
||||
ConnectionHandlerEvent<
|
||||
Self::OutboundProtocol,
|
||||
Self::OutboundOpenInfo,
|
||||
Self::OutEvent,
|
||||
Self::Error,
|
||||
>,
|
||||
> {
|
||||
if let Some(event) = self.events.pop() {
|
||||
return Poll::Ready(event);
|
||||
}
|
||||
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
||||
struct ManyProtocolsUpgrade {
|
||||
protocols: Vec<StreamProtocol>,
|
||||
}
|
||||
|
||||
impl UpgradeInfo for ManyProtocolsUpgrade {
|
||||
type Info = StreamProtocol;
|
||||
type InfoIter = std::vec::IntoIter<Self::Info>;
|
||||
|
||||
fn protocol_info(&self) -> Self::InfoIter {
|
||||
self.protocols.clone().into_iter()
|
||||
}
|
||||
}
|
||||
|
||||
impl<C> InboundUpgrade<C> for ManyProtocolsUpgrade {
|
||||
type Output = C;
|
||||
type Error = Void;
|
||||
type Future = future::Ready<Result<Self::Output, Self::Error>>;
|
||||
|
||||
fn upgrade_inbound(self, stream: C, _: Self::Info) -> Self::Future {
|
||||
future::ready(Ok(stream))
|
||||
}
|
||||
}
|
||||
|
||||
impl<C> OutboundUpgrade<C> for ManyProtocolsUpgrade {
|
||||
type Output = C;
|
||||
type Error = Void;
|
||||
type Future = future::Ready<Result<Self::Output, Self::Error>>;
|
||||
|
||||
fn upgrade_outbound(self, stream: C, _: Self::Info) -> Self::Future {
|
||||
future::ready(Ok(stream))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// The endpoint roles associated with a pending peer-to-peer connection.
|
||||
|
88
swarm/src/connection/supported_protocols.rs
Normal file
88
swarm/src/connection/supported_protocols.rs
Normal file
@ -0,0 +1,88 @@
|
||||
use crate::handler::ProtocolsChange;
|
||||
use crate::StreamProtocol;
|
||||
use std::collections::HashSet;
|
||||
|
||||
#[derive(Default, Clone, Debug)]
|
||||
pub struct SupportedProtocols {
|
||||
protocols: HashSet<StreamProtocol>,
|
||||
}
|
||||
|
||||
impl SupportedProtocols {
|
||||
pub fn on_protocols_change(&mut self, change: ProtocolsChange) -> bool {
|
||||
match change {
|
||||
ProtocolsChange::Added(added) => {
|
||||
let mut changed = false;
|
||||
|
||||
for p in added {
|
||||
changed |= self.protocols.insert(p.clone());
|
||||
}
|
||||
|
||||
changed
|
||||
}
|
||||
ProtocolsChange::Removed(removed) => {
|
||||
let mut changed = false;
|
||||
|
||||
for p in removed {
|
||||
changed |= self.protocols.remove(p);
|
||||
}
|
||||
|
||||
changed
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn iter(&self) -> impl Iterator<Item = &StreamProtocol> {
|
||||
self.protocols.iter()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::handler::{ProtocolsAdded, ProtocolsRemoved};
|
||||
use once_cell::sync::Lazy;
|
||||
|
||||
#[test]
|
||||
fn protocols_change_added_returns_correct_changed_value() {
|
||||
let mut protocols = SupportedProtocols::default();
|
||||
|
||||
let changed = protocols.on_protocols_change(add_foo());
|
||||
assert!(changed);
|
||||
|
||||
let changed = protocols.on_protocols_change(add_foo());
|
||||
assert!(!changed);
|
||||
|
||||
let changed = protocols.on_protocols_change(add_foo_bar());
|
||||
assert!(changed);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn protocols_change_removed_returns_correct_changed_value() {
|
||||
let mut protocols = SupportedProtocols::default();
|
||||
|
||||
let changed = protocols.on_protocols_change(remove_foo());
|
||||
assert!(!changed);
|
||||
|
||||
protocols.on_protocols_change(add_foo());
|
||||
|
||||
let changed = protocols.on_protocols_change(remove_foo());
|
||||
assert!(changed);
|
||||
}
|
||||
|
||||
fn add_foo() -> ProtocolsChange<'static> {
|
||||
ProtocolsChange::Added(ProtocolsAdded::from_set(&FOO_PROTOCOLS))
|
||||
}
|
||||
|
||||
fn add_foo_bar() -> ProtocolsChange<'static> {
|
||||
ProtocolsChange::Added(ProtocolsAdded::from_set(&FOO_BAR_PROTOCOLS))
|
||||
}
|
||||
|
||||
fn remove_foo() -> ProtocolsChange<'static> {
|
||||
ProtocolsChange::Removed(ProtocolsRemoved::from_set(&FOO_PROTOCOLS))
|
||||
}
|
||||
|
||||
static FOO_PROTOCOLS: Lazy<HashSet<StreamProtocol>> =
|
||||
Lazy::new(|| HashSet::from([StreamProtocol::new("/foo")]));
|
||||
static FOO_BAR_PROTOCOLS: Lazy<HashSet<StreamProtocol>> =
|
||||
Lazy::new(|| HashSet::from([StreamProtocol::new("/foo"), StreamProtocol::new("/bar")]));
|
||||
}
|
@ -138,7 +138,10 @@ impl crate::handler::ConnectionHandler for ConnectionHandler {
|
||||
unreachable!("Denied upgrade does not support any protocols")
|
||||
}
|
||||
},
|
||||
ConnectionEvent::AddressChange(_) | ConnectionEvent::ListenUpgradeError(_) => {}
|
||||
ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::ListenUpgradeError(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_) => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -47,17 +47,24 @@ mod pending;
|
||||
mod select;
|
||||
|
||||
pub use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, SendWrapper, UpgradeInfoSend};
|
||||
|
||||
use instant::Instant;
|
||||
use libp2p_core::Multiaddr;
|
||||
use std::{cmp::Ordering, error, fmt, io, task::Context, task::Poll, time::Duration};
|
||||
|
||||
pub use map_in::MapInEvent;
|
||||
pub use map_out::MapOutEvent;
|
||||
pub use one_shot::{OneShotHandler, OneShotHandlerConfig};
|
||||
pub use pending::PendingConnectionHandler;
|
||||
pub use select::ConnectionHandlerSelect;
|
||||
|
||||
use crate::StreamProtocol;
|
||||
use ::either::Either;
|
||||
use instant::Instant;
|
||||
use libp2p_core::Multiaddr;
|
||||
use once_cell::sync::Lazy;
|
||||
use smallvec::SmallVec;
|
||||
use std::collections::hash_map::RandomState;
|
||||
use std::collections::hash_set::{Difference, Intersection};
|
||||
use std::collections::HashSet;
|
||||
use std::iter::Peekable;
|
||||
use std::{cmp::Ordering, error, fmt, io, task::Context, task::Poll, time::Duration};
|
||||
|
||||
/// A handler for a set of protocols used on a connection with a remote.
|
||||
///
|
||||
/// This trait should be implemented for a type that maintains the state for
|
||||
@ -209,6 +216,10 @@ pub enum ConnectionEvent<'a, IP: InboundUpgradeSend, OP: OutboundUpgradeSend, IO
|
||||
DialUpgradeError(DialUpgradeError<OOI, OP>),
|
||||
/// Informs the handler that upgrading an inbound substream to the given protocol has failed.
|
||||
ListenUpgradeError(ListenUpgradeError<IOI, IP>),
|
||||
/// The local [`ConnectionHandler`] added or removed support for one or more protocols.
|
||||
LocalProtocolsChange(ProtocolsChange<'a>),
|
||||
/// The remote [`ConnectionHandler`] now supports a different set of protocols.
|
||||
RemoteProtocolsChange(ProtocolsChange<'a>),
|
||||
}
|
||||
|
||||
impl<'a, IP: InboundUpgradeSend, OP: OutboundUpgradeSend, IOI, OOI>
|
||||
@ -222,6 +233,8 @@ impl<'a, IP: InboundUpgradeSend, OP: OutboundUpgradeSend, IOI, OOI>
|
||||
}
|
||||
ConnectionEvent::FullyNegotiatedInbound(_)
|
||||
| ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_)
|
||||
| ConnectionEvent::ListenUpgradeError(_) => false,
|
||||
}
|
||||
}
|
||||
@ -234,6 +247,8 @@ impl<'a, IP: InboundUpgradeSend, OP: OutboundUpgradeSend, IOI, OOI>
|
||||
}
|
||||
ConnectionEvent::FullyNegotiatedOutbound(_)
|
||||
| ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_)
|
||||
| ConnectionEvent::DialUpgradeError(_) => false,
|
||||
}
|
||||
}
|
||||
@ -266,6 +281,122 @@ pub struct AddressChange<'a> {
|
||||
pub new_address: &'a Multiaddr,
|
||||
}
|
||||
|
||||
/// [`ConnectionEvent`] variant that informs the handler about a change in the protocols supported on the connection.
|
||||
#[derive(Clone)]
|
||||
pub enum ProtocolsChange<'a> {
|
||||
Added(ProtocolsAdded<'a>),
|
||||
Removed(ProtocolsRemoved<'a>),
|
||||
}
|
||||
|
||||
impl<'a> ProtocolsChange<'a> {
|
||||
/// Compute the [`ProtocolsChange`] that results from adding `to_add` to `existing_protocols`.
|
||||
///
|
||||
/// Returns `None` if the change is a no-op, i.e. `to_add` is a subset of `existing_protocols`.
|
||||
pub(crate) fn add(
|
||||
existing_protocols: &'a HashSet<StreamProtocol>,
|
||||
to_add: &'a HashSet<StreamProtocol>,
|
||||
) -> Option<Self> {
|
||||
let mut actually_added_protocols = to_add.difference(existing_protocols).peekable();
|
||||
|
||||
actually_added_protocols.peek()?;
|
||||
|
||||
Some(ProtocolsChange::Added(ProtocolsAdded {
|
||||
protocols: actually_added_protocols,
|
||||
}))
|
||||
}
|
||||
|
||||
/// Compute the [`ProtocolsChange`] that results from removing `to_remove` from `existing_protocols`.
|
||||
///
|
||||
/// Returns `None` if the change is a no-op, i.e. none of the protocols in `to_remove` are in `existing_protocols`.
|
||||
pub(crate) fn remove(
|
||||
existing_protocols: &'a HashSet<StreamProtocol>,
|
||||
to_remove: &'a HashSet<StreamProtocol>,
|
||||
) -> Option<Self> {
|
||||
let mut actually_removed_protocols = existing_protocols.intersection(to_remove).peekable();
|
||||
|
||||
actually_removed_protocols.peek()?;
|
||||
|
||||
Some(ProtocolsChange::Removed(ProtocolsRemoved {
|
||||
protocols: Either::Right(actually_removed_protocols),
|
||||
}))
|
||||
}
|
||||
|
||||
/// Compute the [`ProtocolsChange`]s required to go from `existing_protocols` to `new_protocols`.
|
||||
pub(crate) fn from_full_sets(
|
||||
existing_protocols: &'a HashSet<StreamProtocol>,
|
||||
new_protocols: &'a HashSet<StreamProtocol>,
|
||||
) -> SmallVec<[Self; 2]> {
|
||||
if existing_protocols == new_protocols {
|
||||
return SmallVec::new();
|
||||
}
|
||||
|
||||
let mut changes = SmallVec::new();
|
||||
|
||||
let mut added_protocols = new_protocols.difference(existing_protocols).peekable();
|
||||
let mut removed_protocols = existing_protocols.difference(new_protocols).peekable();
|
||||
|
||||
if added_protocols.peek().is_some() {
|
||||
changes.push(ProtocolsChange::Added(ProtocolsAdded {
|
||||
protocols: added_protocols,
|
||||
}));
|
||||
}
|
||||
|
||||
if removed_protocols.peek().is_some() {
|
||||
changes.push(ProtocolsChange::Removed(ProtocolsRemoved {
|
||||
protocols: Either::Left(removed_protocols),
|
||||
}));
|
||||
}
|
||||
|
||||
changes
|
||||
}
|
||||
}
|
||||
|
||||
/// An [`Iterator`] over all protocols that have been added.
|
||||
#[derive(Clone)]
|
||||
pub struct ProtocolsAdded<'a> {
|
||||
protocols: Peekable<Difference<'a, StreamProtocol, RandomState>>,
|
||||
}
|
||||
|
||||
impl<'a> ProtocolsAdded<'a> {
|
||||
pub(crate) fn from_set(protocols: &'a HashSet<StreamProtocol, RandomState>) -> Self {
|
||||
ProtocolsAdded {
|
||||
protocols: protocols.difference(&EMPTY_HASHSET).peekable(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// An [`Iterator`] over all protocols that have been removed.
|
||||
#[derive(Clone)]
|
||||
pub struct ProtocolsRemoved<'a> {
|
||||
protocols: Either<
|
||||
Peekable<Difference<'a, StreamProtocol, RandomState>>,
|
||||
Peekable<Intersection<'a, StreamProtocol, RandomState>>,
|
||||
>,
|
||||
}
|
||||
|
||||
impl<'a> ProtocolsRemoved<'a> {
|
||||
#[cfg(test)]
|
||||
pub(crate) fn from_set(protocols: &'a HashSet<StreamProtocol, RandomState>) -> Self {
|
||||
ProtocolsRemoved {
|
||||
protocols: Either::Left(protocols.difference(&EMPTY_HASHSET).peekable()),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Iterator for ProtocolsAdded<'a> {
|
||||
type Item = &'a StreamProtocol;
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
self.protocols.next()
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Iterator for ProtocolsRemoved<'a> {
|
||||
type Item = &'a StreamProtocol;
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
self.protocols.next()
|
||||
}
|
||||
}
|
||||
|
||||
/// [`ConnectionEvent`] variant that informs the handler
|
||||
/// that upgrading an outbound substream to the given protocol has failed.
|
||||
pub struct DialUpgradeError<OOI, OP: OutboundUpgradeSend> {
|
||||
@ -357,7 +488,7 @@ impl<TUpgrade, TInfo> SubstreamProtocol<TUpgrade, TInfo> {
|
||||
}
|
||||
|
||||
/// Event produced by a handler.
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
pub enum ConnectionHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr> {
|
||||
/// Request a new outbound substream to be opened with the remote.
|
||||
OutboundSubstreamRequest {
|
||||
@ -374,11 +505,21 @@ pub enum ConnectionHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom,
|
||||
/// the connection, return [`KeepAlive::No`] in
|
||||
/// [`ConnectionHandler::connection_keep_alive`].
|
||||
Close(TErr),
|
||||
/// We learned something about the protocols supported by the remote.
|
||||
ReportRemoteProtocols(ProtocolSupport),
|
||||
|
||||
/// Other event.
|
||||
Custom(TCustom),
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
pub enum ProtocolSupport {
|
||||
/// The remote now supports these additional protocols.
|
||||
Added(HashSet<StreamProtocol>),
|
||||
/// The remote no longer supports these protocols.
|
||||
Removed(HashSet<StreamProtocol>),
|
||||
}
|
||||
|
||||
/// Event produced by a handler.
|
||||
impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
ConnectionHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
@ -400,6 +541,9 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
}
|
||||
ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(val),
|
||||
ConnectionHandlerEvent::Close(val) => ConnectionHandlerEvent::Close(val),
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support) => {
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -420,6 +564,9 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
}
|
||||
ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(val),
|
||||
ConnectionHandlerEvent::Close(val) => ConnectionHandlerEvent::Close(val),
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support) => {
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -437,6 +584,9 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
}
|
||||
ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(map(val)),
|
||||
ConnectionHandlerEvent::Close(val) => ConnectionHandlerEvent::Close(val),
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support) => {
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -454,6 +604,9 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
}
|
||||
ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(val),
|
||||
ConnectionHandlerEvent::Close(val) => ConnectionHandlerEvent::Close(map(val)),
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support) => {
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -558,3 +711,7 @@ impl Ord for KeepAlive {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A statically declared, empty [`HashSet`] allows us to work around borrow-checker rules for
|
||||
/// [`ProtocolsAdded::from_set`]. The lifetimes don't work unless we have a [`HashSet`] with a `'static' lifetime.
|
||||
static EMPTY_HASHSET: Lazy<HashSet<StreamProtocol>> = Lazy::new(HashSet::new);
|
||||
|
@ -208,6 +208,22 @@ where
|
||||
handler.on_connection_event(ConnectionEvent::AddressChange(address_change))
|
||||
}
|
||||
},
|
||||
ConnectionEvent::LocalProtocolsChange(supported_protocols) => match self {
|
||||
Either::Left(handler) => handler.on_connection_event(
|
||||
ConnectionEvent::LocalProtocolsChange(supported_protocols),
|
||||
),
|
||||
Either::Right(handler) => handler.on_connection_event(
|
||||
ConnectionEvent::LocalProtocolsChange(supported_protocols),
|
||||
),
|
||||
},
|
||||
ConnectionEvent::RemoteProtocolsChange(supported_protocols) => match self {
|
||||
Either::Left(handler) => handler.on_connection_event(
|
||||
ConnectionEvent::RemoteProtocolsChange(supported_protocols),
|
||||
),
|
||||
Either::Right(handler) => handler.on_connection_event(
|
||||
ConnectionEvent::RemoteProtocolsChange(supported_protocols),
|
||||
),
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -81,6 +81,9 @@ where
|
||||
ConnectionHandlerEvent::OutboundSubstreamRequest { protocol } => {
|
||||
ConnectionHandlerEvent::OutboundSubstreamRequest { protocol }
|
||||
}
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support) => {
|
||||
ConnectionHandlerEvent::ReportRemoteProtocols(support)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -205,6 +205,20 @@ where
|
||||
ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => {
|
||||
self.on_listen_upgrade_error(listen_upgrade_error)
|
||||
}
|
||||
ConnectionEvent::LocalProtocolsChange(supported_protocols) => {
|
||||
for h in self.handlers.values_mut() {
|
||||
h.on_connection_event(ConnectionEvent::LocalProtocolsChange(
|
||||
supported_protocols.clone(),
|
||||
));
|
||||
}
|
||||
}
|
||||
ConnectionEvent::RemoteProtocolsChange(supported_protocols) => {
|
||||
for h in self.handlers.values_mut() {
|
||||
h.on_connection_event(ConnectionEvent::RemoteProtocolsChange(
|
||||
supported_protocols.clone(),
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -217,7 +217,10 @@ where
|
||||
self.keep_alive = KeepAlive::No;
|
||||
}
|
||||
}
|
||||
ConnectionEvent::AddressChange(_) | ConnectionEvent::ListenUpgradeError(_) => {}
|
||||
ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::ListenUpgradeError(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_) => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -99,7 +99,9 @@ impl ConnectionHandler for PendingConnectionHandler {
|
||||
}
|
||||
ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::DialUpgradeError(_)
|
||||
| ConnectionEvent::ListenUpgradeError(_) => {}
|
||||
| ConnectionEvent::ListenUpgradeError(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_) => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -240,6 +240,9 @@ where
|
||||
.map_info(Either::Left),
|
||||
});
|
||||
}
|
||||
Poll::Ready(ConnectionHandlerEvent::ReportRemoteProtocols(support)) => {
|
||||
return Poll::Ready(ConnectionHandlerEvent::ReportRemoteProtocols(support));
|
||||
}
|
||||
Poll::Pending => (),
|
||||
};
|
||||
|
||||
@ -257,6 +260,9 @@ where
|
||||
.map_info(Either::Right),
|
||||
});
|
||||
}
|
||||
Poll::Ready(ConnectionHandlerEvent::ReportRemoteProtocols(support)) => {
|
||||
return Poll::Ready(ConnectionHandlerEvent::ReportRemoteProtocols(support));
|
||||
}
|
||||
Poll::Pending => (),
|
||||
};
|
||||
|
||||
@ -317,6 +323,26 @@ where
|
||||
ConnectionEvent::ListenUpgradeError(listen_upgrade_error) => {
|
||||
self.on_listen_upgrade_error(listen_upgrade_error)
|
||||
}
|
||||
ConnectionEvent::LocalProtocolsChange(supported_protocols) => {
|
||||
self.proto1
|
||||
.on_connection_event(ConnectionEvent::LocalProtocolsChange(
|
||||
supported_protocols.clone(),
|
||||
));
|
||||
self.proto2
|
||||
.on_connection_event(ConnectionEvent::LocalProtocolsChange(
|
||||
supported_protocols,
|
||||
));
|
||||
}
|
||||
ConnectionEvent::RemoteProtocolsChange(supported_protocols) => {
|
||||
self.proto1
|
||||
.on_connection_event(ConnectionEvent::RemoteProtocolsChange(
|
||||
supported_protocols.clone(),
|
||||
));
|
||||
self.proto2
|
||||
.on_connection_event(ConnectionEvent::RemoteProtocolsChange(
|
||||
supported_protocols,
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -136,7 +136,9 @@ impl crate::handler::ConnectionHandler for ConnectionHandler {
|
||||
}) => void::unreachable(protocol),
|
||||
ConnectionEvent::DialUpgradeError(_)
|
||||
| ConnectionEvent::ListenUpgradeError(_)
|
||||
| ConnectionEvent::AddressChange(_) => {}
|
||||
| ConnectionEvent::AddressChange(_)
|
||||
| ConnectionEvent::LocalProtocolsChange(_)
|
||||
| ConnectionEvent::RemoteProtocolsChange(_) => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -116,7 +116,7 @@ pub use behaviour::{
|
||||
PollParameters, ToSwarm,
|
||||
};
|
||||
pub use connection::pool::ConnectionCounters;
|
||||
pub use connection::{ConnectionError, ConnectionId};
|
||||
pub use connection::{ConnectionError, ConnectionId, SupportedProtocols};
|
||||
pub use executor::Executor;
|
||||
pub use handler::{
|
||||
ConnectionHandler, ConnectionHandlerEvent, ConnectionHandlerSelect, KeepAlive, OneShotHandler,
|
||||
|
Reference in New Issue
Block a user