From 2153a9a724a3ea975a41a23ea3f21bd781e28a0c Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Tue, 23 May 2023 14:49:02 +0300 Subject: [PATCH] Split `Peerset` into `PeerStore` & `ProtocolController`s (#13611) --- Cargo.lock | 11 + .../grandpa/src/communication/tests.rs | 12 - client/network-gossip/src/bridge.rs | 12 - client/network-gossip/src/state_machine.rs | 12 - client/network/src/protocol.rs | 42 +- .../src/protocol/notifications/behaviour.rs | 212 +- .../src/protocol/notifications/tests.rs | 14 +- client/network/src/service.rs | 66 +- client/network/src/service/traits.rs | 40 +- client/network/sync/src/service/mock.rs | 6 - client/offchain/src/api.rs | 12 - client/offchain/src/lib.rs | 12 - client/peerset/Cargo.toml | 5 + client/peerset/src/lib.rs | 837 ++----- client/peerset/src/peer_store.rs | 403 ++++ client/peerset/src/peersstate.rs | 737 ------- client/peerset/src/protocol_controller.rs | 1939 +++++++++++++++++ client/peerset/tests/fuzz.rs | 266 ++- client/utils/src/mpsc.rs | 4 +- 19 files changed, 2914 insertions(+), 1728 deletions(-) create mode 100644 client/peerset/src/peer_store.rs delete mode 100644 client/peerset/src/peersstate.rs create mode 100644 client/peerset/src/protocol_controller.rs diff --git a/Cargo.lock b/Cargo.lock index f387b5ef95227..59ccf359f10de 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7565,6 +7565,12 @@ dependencies = [ "windows-sys 0.45.0", ] +[[package]] +name = "partial_sort" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7924d1d0ad836f665c9065e26d016c673ece3993f30d340068b16f282afc1156" + [[package]] name = "paste" version = "1.0.12" @@ -9719,9 +9725,14 @@ dependencies = [ "futures", "libp2p-identity", "log", + "mockall", + "parking_lot 0.12.1", + "partial_sort", "rand 0.8.5", "sc-utils", "serde_json", + "sp-arithmetic", + "sp-tracing", "wasm-timer", ] diff --git a/client/consensus/grandpa/src/communication/tests.rs b/client/consensus/grandpa/src/communication/tests.rs index f97b1f1e88181..eb88382989175 100644 --- a/client/consensus/grandpa/src/communication/tests.rs +++ b/client/consensus/grandpa/src/communication/tests.rs @@ -116,18 +116,6 @@ impl NetworkPeers for TestNetwork { fn remove_peers_from_reserved_set(&self, _protocol: ProtocolName, _peers: Vec) {} - fn add_to_peers_set( - &self, - _protocol: ProtocolName, - _peers: HashSet, - ) -> Result<(), String> { - unimplemented!(); - } - - fn remove_from_peers_set(&self, _protocol: ProtocolName, _peers: Vec) { - unimplemented!(); - } - fn sync_num_connected(&self) -> usize { unimplemented!(); } diff --git a/client/network-gossip/src/bridge.rs b/client/network-gossip/src/bridge.rs index 4793d7822ddbe..baf11c9e8649b 100644 --- a/client/network-gossip/src/bridge.rs +++ b/client/network-gossip/src/bridge.rs @@ -416,18 +416,6 @@ mod tests { fn remove_peers_from_reserved_set(&self, _protocol: ProtocolName, _peers: Vec) {} - fn add_to_peers_set( - &self, - _protocol: ProtocolName, - _peers: HashSet, - ) -> Result<(), String> { - unimplemented!(); - } - - fn remove_from_peers_set(&self, _protocol: ProtocolName, _peers: Vec) { - unimplemented!(); - } - fn sync_num_connected(&self) -> usize { unimplemented!(); } diff --git a/client/network-gossip/src/state_machine.rs b/client/network-gossip/src/state_machine.rs index e6d2b0e2ae4c8..ee65bd890e15b 100644 --- a/client/network-gossip/src/state_machine.rs +++ b/client/network-gossip/src/state_machine.rs @@ -646,18 +646,6 @@ mod tests { fn remove_peers_from_reserved_set(&self, _protocol: ProtocolName, _peers: Vec) {} - fn add_to_peers_set( - &self, - _protocol: ProtocolName, - _peers: HashSet, - ) -> Result<(), String> { - unimplemented!(); - } - - fn remove_from_peers_set(&self, _protocol: ProtocolName, _peers: Vec) { - unimplemented!(); - } - fn sync_num_connected(&self) -> usize { unimplemented!(); } diff --git a/client/network/src/protocol.rs b/client/network/src/protocol.rs index e7214d814dda8..5fbb518ce02ea 100644 --- a/client/network/src/protocol.rs +++ b/client/network/src/protocol.rs @@ -257,8 +257,8 @@ impl Protocol { } /// Returns the list of reserved peers. - pub fn reserved_peers(&self) -> impl Iterator { - self.behaviour.reserved_peers(HARDCODED_PEERSETS_SYNC) + pub fn reserved_peers(&self, pending_response: oneshot::Sender>) { + self.behaviour.reserved_peers(HARDCODED_PEERSETS_SYNC, pending_response); } /// Adds a `PeerId` to the list of reserved peers for syncing purposes. @@ -310,39 +310,13 @@ impl Protocol { } } - /// Notify the protocol that we have learned about the existence of nodes on the default set. + /// Notify the protocol that we have learned about the existence of some peer. /// - /// Can be called multiple times with the same `PeerId`s. - pub fn add_default_set_discovered_nodes(&mut self, peer_ids: impl Iterator) { - for peer_id in peer_ids { - self.peerset_handle.add_to_peers_set(HARDCODED_PEERSETS_SYNC, peer_id); - } - } - - /// Add a peer to a peers set. - pub fn add_to_peers_set(&self, protocol: ProtocolName, peer: PeerId) { - if let Some(index) = self.notification_protocols.iter().position(|p| *p == protocol) { - self.peerset_handle.add_to_peers_set(sc_peerset::SetId::from(index), peer); - } else { - error!( - target: "sub-libp2p", - "add_to_peers_set with unknown protocol: {}", - protocol - ); - } - } - - /// Remove a peer from a peers set. - pub fn remove_from_peers_set(&self, protocol: ProtocolName, peer: PeerId) { - if let Some(index) = self.notification_protocols.iter().position(|p| *p == protocol) { - self.peerset_handle.remove_from_peers_set(sc_peerset::SetId::from(index), peer); - } else { - error!( - target: "sub-libp2p", - "remove_from_peers_set with unknown protocol: {}", - protocol - ); - } + /// Can be called multiple times with the same `PeerId`. + pub fn add_known_peer(&mut self, peer_id: PeerId) { + // TODO: get rid of this function and call `Peerset`/`PeerStore` directly + // from `NetworkWorker`. + self.peerset_handle.add_known_peer(peer_id); } } diff --git a/client/network/src/protocol/notifications/behaviour.rs b/client/network/src/protocol/notifications/behaviour.rs index 7e56793939b55..8f1324dfa8c06 100644 --- a/client/network/src/protocol/notifications/behaviour.rs +++ b/client/network/src/protocol/notifications/behaviour.rs @@ -25,7 +25,7 @@ use crate::{ use bytes::BytesMut; use fnv::FnvHashMap; -use futures::prelude::*; +use futures::{channel::oneshot, prelude::*}; use libp2p::{ core::{ConnectedPoint, Endpoint, Multiaddr}, swarm::{ @@ -35,7 +35,7 @@ use libp2p::{ }, PeerId, }; -use log::{error, trace, warn}; +use log::{debug, error, info, trace, warn}; use parking_lot::RwLock; use rand::distributions::{Distribution as _, Uniform}; use sc_peerset::DropReason; @@ -231,6 +231,9 @@ enum PeerState { /// If `Some`, any dial attempts to this peer are delayed until the given `Instant`. backoff_until: Option, + /// Incoming index tracking this connection. + incoming_index: sc_peerset::IncomingIndex, + /// List of connections with this peer, and their state. connections: SmallVec<[(ConnectionId, ConnectionState); crate::MAX_CONNECTIONS_PER_PEER]>, }, @@ -493,7 +496,7 @@ impl Notifications { // Incoming => Disabled. // Ongoing opening requests from the remote are rejected. - PeerState::Incoming { mut connections, backoff_until } => { + PeerState::Incoming { mut connections, backoff_until, .. } => { let inc = if let Some(inc) = self .incoming .iter_mut() @@ -536,8 +539,12 @@ impl Notifications { } /// Returns the list of reserved peers. - pub fn reserved_peers(&self, set_id: sc_peerset::SetId) -> impl Iterator { - self.peerset.reserved_peers(set_id) + pub fn reserved_peers( + &self, + set_id: sc_peerset::SetId, + pending_response: oneshot::Sender>, + ) { + self.peerset.reserved_peers(set_id, pending_response); } /// Returns the state of the peerset manager, for debugging purposes. @@ -686,65 +693,34 @@ impl Notifications { }; } }, - - // Incoming => Enabled - PeerState::Incoming { mut connections, .. } => { - trace!(target: "sub-libp2p", "PSM => Connect({}, {:?}): Enabling connections.", - occ_entry.key().0, set_id); - if let Some(inc) = self - .incoming - .iter_mut() - .find(|i| i.peer_id == occ_entry.key().0 && i.set_id == set_id && i.alive) - { - inc.alive = false; - } else { - error!( - target: "sub-libp2p", - "State mismatch in libp2p: no entry in incoming for incoming peer", - ) - } - - debug_assert!(connections - .iter() - .any(|(_, s)| matches!(s, ConnectionState::OpenDesiredByRemote))); - for (connec_id, connec_state) in connections - .iter_mut() - .filter(|(_, s)| matches!(s, ConnectionState::OpenDesiredByRemote)) - { - trace!(target: "sub-libp2p", "Handler({:?}, {:?}) <= Open({:?})", - occ_entry.key(), *connec_id, set_id); - self.events.push_back(ToSwarm::NotifyHandler { - peer_id: occ_entry.key().0, - handler: NotifyHandler::One(*connec_id), - event: NotifsHandlerIn::Open { protocol_index: set_id.into() }, - }); - *connec_state = ConnectionState::Opening; - } - - *occ_entry.into_mut() = PeerState::Enabled { connections }; + // Incoming => Incoming + st @ PeerState::Incoming { .. } => { + debug!( + target: "sub-libp2p", + "PSM => Connect({}, {:?}): Ignoring obsolete connect, we are awaiting accept/reject.", + occ_entry.key().0, set_id + ); + *occ_entry.into_mut() = st; }, // Other states are kept as-is. st @ PeerState::Enabled { .. } => { - warn!(target: "sub-libp2p", + debug!(target: "sub-libp2p", "PSM => Connect({}, {:?}): Already connected.", occ_entry.key().0, set_id); *occ_entry.into_mut() = st; - debug_assert!(false); }, st @ PeerState::DisabledPendingEnable { .. } => { - warn!(target: "sub-libp2p", + debug!(target: "sub-libp2p", "PSM => Connect({}, {:?}): Already pending enabling.", occ_entry.key().0, set_id); *occ_entry.into_mut() = st; - debug_assert!(false); }, st @ PeerState::Requested { .. } | st @ PeerState::PendingRequest { .. } => { - warn!(target: "sub-libp2p", + debug!(target: "sub-libp2p", "PSM => Connect({}, {:?}): Duplicate request.", occ_entry.key().0, set_id); *occ_entry.into_mut() = st; - debug_assert!(false); }, PeerState::Poisoned => { @@ -847,10 +823,12 @@ impl Notifications { // Invalid state transitions. st @ PeerState::Incoming { .. } => { - error!(target: "sub-libp2p", "PSM => Drop({}, {:?}): Not enabled (Incoming).", - entry.key().0, set_id); + info!( + target: "sub-libp2p", + "PSM => Drop({}, {:?}): Ignoring obsolete disconnect, we are awaiting accept/reject.", + entry.key().0, set_id, + ); *entry.into_mut() = st; - debug_assert!(false); }, PeerState::Poisoned => { error!(target: "sub-libp2p", "State of {:?} is poisoned", entry.key()); @@ -895,7 +873,24 @@ impl Notifications { match mem::replace(state, PeerState::Poisoned) { // Incoming => Enabled - PeerState::Incoming { mut connections, .. } => { + PeerState::Incoming { mut connections, incoming_index, .. } => { + if index < incoming_index { + warn!( + target: "sub-libp2p", + "PSM => Accept({:?}, {}, {:?}): Ignoring obsolete incoming index, we are already awaiting {:?}.", + index, incoming.peer_id, incoming.set_id, incoming_index + ); + return + } else if index > incoming_index { + error!( + target: "sub-libp2p", + "PSM => Accept({:?}, {}, {:?}): Ignoring incoming index from the future, we are awaiting {:?}.", + index, incoming.peer_id, incoming.set_id, incoming_index + ); + debug_assert!(false); + return + } + trace!(target: "sub-libp2p", "PSM => Accept({:?}, {}, {:?}): Enabling connections.", index, incoming.peer_id, incoming.set_id); @@ -955,7 +950,24 @@ impl Notifications { match mem::replace(state, PeerState::Poisoned) { // Incoming => Disabled - PeerState::Incoming { mut connections, backoff_until } => { + PeerState::Incoming { mut connections, backoff_until, incoming_index } => { + if index < incoming_index { + warn!( + target: "sub-libp2p", + "PSM => Reject({:?}, {}, {:?}): Ignoring obsolete incoming index, we are already awaiting {:?}.", + index, incoming.peer_id, incoming.set_id, incoming_index + ); + return + } else if index > incoming_index { + error!( + target: "sub-libp2p", + "PSM => Reject({:?}, {}, {:?}): Ignoring incoming index from the future, we are awaiting {:?}.", + index, incoming.peer_id, incoming.set_id, incoming_index + ); + debug_assert!(false); + return + } + trace!(target: "sub-libp2p", "PSM => Reject({:?}, {}, {:?}): Rejecting connections.", index, incoming.peer_id, incoming.set_id); @@ -1195,7 +1207,7 @@ impl NetworkBehaviour for Notifications { }, // Incoming => Incoming | Disabled | Backoff | Ø - PeerState::Incoming { mut connections, backoff_until } => { + PeerState::Incoming { mut connections, backoff_until, incoming_index } => { trace!( target: "sub-libp2p", "Libp2p => Disconnected({}, {:?}, {:?}): OpenDesiredByRemote.", @@ -1269,8 +1281,11 @@ impl NetworkBehaviour for Notifications { *entry.get_mut() = PeerState::Disabled { connections, backoff_until }; } else { - *entry.get_mut() = - PeerState::Incoming { connections, backoff_until }; + *entry.get_mut() = PeerState::Incoming { + connections, + backoff_until, + incoming_index, + }; } }, @@ -1489,7 +1504,7 @@ impl NetworkBehaviour for Notifications { match mem::replace(entry.get_mut(), PeerState::Poisoned) { // Incoming => Incoming - PeerState::Incoming { mut connections, backoff_until } => { + PeerState::Incoming { mut connections, backoff_until, incoming_index } => { debug_assert!(connections .iter() .any(|(_, s)| matches!(s, ConnectionState::OpenDesiredByRemote))); @@ -1517,7 +1532,8 @@ impl NetworkBehaviour for Notifications { debug_assert!(false); } - *entry.into_mut() = PeerState::Incoming { connections, backoff_until }; + *entry.into_mut() = + PeerState::Incoming { connections, backoff_until, incoming_index }; }, PeerState::Enabled { mut connections } => { @@ -1582,8 +1598,11 @@ impl NetworkBehaviour for Notifications { incoming_id, }); - *entry.into_mut() = - PeerState::Incoming { connections, backoff_until }; + *entry.into_mut() = PeerState::Incoming { + connections, + backoff_until, + incoming_index: incoming_id, + }; } else { // Connections in `OpeningThenClosing` and `Closing` state can be // in a Closed phase, and as such can emit `OpenDesiredByRemote` @@ -2087,6 +2106,7 @@ mod tests { use super::*; use crate::protocol::notifications::handler::tests::*; use libp2p::swarm::AddressRecord; + use sc_peerset::IncomingIndex; use std::{collections::HashSet, iter}; impl PartialEq for ConnectionState { @@ -2279,7 +2299,7 @@ mod tests { NotifsHandlerOut::OpenDesiredByRemote { protocol_index: 0 }, ); - if let Some(&PeerState::Incoming { ref connections, backoff_until: None }) = + if let Some(&PeerState::Incoming { ref connections, backoff_until: None, .. }) = notif.peers.get(&(peer, 0.into())) { assert_eq!(connections.len(), 1); @@ -2424,8 +2444,10 @@ mod tests { NotifsHandlerOut::OpenDesiredByRemote { protocol_index: 0 }, ); - // attempt to connect to the peer and verify that the peer state is `Enabled` - notif.peerset_report_connect(peer, set_id); + // attempt to connect to the peer and verify that the peer state is `Enabled`; + // we rely on implementation detail that incoming indices are counted from 0 + // to not mock the `Peerset` + notif.peerset_report_accept(IncomingIndex(0)); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Enabled { .. }))); } @@ -2502,7 +2524,9 @@ mod tests { conn, NotifsHandlerOut::OpenDesiredByRemote { protocol_index: 0 }, ); - notif.peerset_report_connect(peer, set_id); + // we rely on the implementation detail that incoming indices are counted from 0 + // to not mock the `Peerset` + notif.peerset_report_accept(IncomingIndex(0)); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Enabled { .. }))); // disconnect peer and verify that the state is `Disabled` @@ -2859,7 +2883,9 @@ mod tests { ); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Incoming { .. }))); - notif.peerset_report_connect(peer, set_id); + // We rely on the implementation detail that incoming indices are counted + // from 0 to not mock the `Peerset`. + notif.peerset_report_accept(sc_peerset::IncomingIndex(0)); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Enabled { .. }))); // open new substream @@ -2948,7 +2974,6 @@ mod tests { // check peer information assert_eq!(notif.open_peers().collect::>(), vec![&peer],); - assert_eq!(notif.reserved_peers(set_id).collect::>(), Vec::<&PeerId>::new(),); assert_eq!(notif.num_discovered_peers(), 0usize); // close the other connection and verify that notification replacement event is emitted @@ -3703,7 +3728,9 @@ mod tests { ); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Incoming { .. }))); - notif.peerset_report_connect(peer, set_id); + // we rely on the implementation detail that incoming indices are counted from 0 + // to not mock the `Peerset` + notif.peerset_report_accept(IncomingIndex(0)); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Enabled { .. }))); let event = conn_yielder.open_substream(peer, 0, connected, vec![1, 2, 3, 4]); @@ -3834,7 +3861,6 @@ mod tests { } #[test] - #[should_panic] #[cfg(debug_assertions)] fn peerset_report_connect_with_disabled_pending_enable_peer() { let (mut notif, _peerset) = development_notifs(); @@ -3872,11 +3898,15 @@ mod tests { Some(&PeerState::DisabledPendingEnable { .. }) )); + // duplicate "connect" must not change the state notif.peerset_report_connect(peer, set_id); + assert!(std::matches!( + notif.peers.get(&(peer, set_id)), + Some(&PeerState::DisabledPendingEnable { .. }) + )); } #[test] - #[should_panic] #[cfg(debug_assertions)] fn peerset_report_connect_with_requested_peer() { let (mut notif, _peerset) = development_notifs(); @@ -3887,11 +3917,12 @@ mod tests { notif.peerset_report_connect(peer, set_id); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Requested))); + // Duplicate "connect" must not change the state. notif.peerset_report_connect(peer, set_id); + assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Requested))); } #[test] - #[should_panic] #[cfg(debug_assertions)] fn peerset_report_connect_with_pending_requested() { let (mut notif, _peerset) = development_notifs(); @@ -3940,13 +3971,17 @@ mod tests { Some(&PeerState::PendingRequest { .. }) )); + // duplicate "connect" must not change the state notif.peerset_report_connect(peer, set_id); + assert!(std::matches!( + notif.peers.get(&(peer, set_id)), + Some(&PeerState::PendingRequest { .. }) + )); } #[test] - #[should_panic] #[cfg(debug_assertions)] - fn peerset_report_disconnect_with_incoming_peer() { + fn peerset_report_connect_with_incoming_peer() { let (mut notif, _peerset) = development_notifs(); let peer = PeerId::random(); let set_id = sc_peerset::SetId::from(0); @@ -3973,10 +4008,45 @@ mod tests { conn, NotifsHandlerOut::OpenDesiredByRemote { protocol_index: 0 }, ); + assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Incoming { .. }))); + + notif.peerset_report_connect(peer, set_id); + assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Incoming { .. }))); + } + #[test] + #[cfg(debug_assertions)] + fn peerset_report_disconnect_with_incoming_peer() { + let (mut notif, _peerset) = development_notifs(); + let peer = PeerId::random(); + let set_id = sc_peerset::SetId::from(0); + let conn = ConnectionId::new_unchecked(0); + let connected = ConnectedPoint::Listener { + local_addr: Multiaddr::empty(), + send_back_addr: Multiaddr::empty(), + }; + + notif.on_swarm_event(FromSwarm::ConnectionEstablished( + libp2p::swarm::behaviour::ConnectionEstablished { + peer_id: peer, + connection_id: conn, + endpoint: &connected, + failed_addresses: &[], + other_established: 0usize, + }, + )); + assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Disabled { .. }))); + + // remote opens a substream, verify that peer state is updated to `Incoming` + notif.on_connection_handler_event( + peer, + conn, + NotifsHandlerOut::OpenDesiredByRemote { protocol_index: 0 }, + ); assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Incoming { .. }))); notif.peerset_report_disconnect(peer, set_id); + assert!(std::matches!(notif.peers.get(&(peer, set_id)), Some(&PeerState::Incoming { .. }))); } #[test] diff --git a/client/network/src/protocol/notifications/tests.rs b/client/network/src/protocol/notifications/tests.rs index d13a4fcfa3809..998ec79f81f88 100644 --- a/client/network/src/protocol/notifications/tests.rs +++ b/client/network/src/protocol/notifications/tests.rs @@ -307,8 +307,20 @@ fn reconnect_after_disconnect() { _ => {}, } + // Due to the bug in `Notifications`, the disconnected node does not always detect that + // it was disconnected. The closed inbound substream is tolerated by design, and the + // closed outbound substream is not detected until something is sent into it. + // See [PR #13396](https://github.com/paritytech/substrate/pull/13396). + // This happens if the disconnecting node reconnects to it fast enough. + // In this case the disconnected node does not transit via `ServiceState::NotConnected` + // and stays in `ServiceState::FirstConnec`. + // TODO: update this once the fix is finally merged. if service1_state == ServiceState::ConnectedAgain && - service2_state == ServiceState::ConnectedAgain + service2_state == ServiceState::ConnectedAgain || + service1_state == ServiceState::ConnectedAgain && + service2_state == ServiceState::FirstConnec || + service1_state == ServiceState::FirstConnec && + service2_state == ServiceState::ConnectedAgain { break } diff --git a/client/network/src/service.rs b/client/network/src/service.rs index cd8e18a2e7d9f..34d97ca07cf03 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -620,8 +620,11 @@ where } /// Returns the list of reserved peers. - pub fn reserved_peers(&self) -> impl Iterator { - self.network_service.behaviour().user_protocol().reserved_peers() + fn reserved_peers(&self, pending_response: oneshot::Sender>) { + self.network_service + .behaviour() + .user_protocol() + .reserved_peers(pending_response); } } @@ -882,40 +885,6 @@ where } } - fn add_to_peers_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String> { - let peers = self.split_multiaddr_and_peer_id(peers)?; - - for (peer_id, addr) in peers.into_iter() { - // Make sure the local peer ID is never added to the PSM. - if peer_id == self.local_peer_id { - return Err("Local peer ID cannot be added as a reserved peer.".to_string()) - } - - if !addr.is_empty() { - let _ = self - .to_worker - .unbounded_send(ServiceToWorkerMsg::AddKnownAddress(peer_id, addr)); - } - let _ = self - .to_worker - .unbounded_send(ServiceToWorkerMsg::AddToPeersSet(protocol.clone(), peer_id)); - } - - Ok(()) - } - - fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec) { - for peer_id in peers.into_iter() { - let _ = self - .to_worker - .unbounded_send(ServiceToWorkerMsg::RemoveFromPeersSet(protocol.clone(), peer_id)); - } - } - fn sync_num_connected(&self) -> usize { self.num_connected.load(Ordering::Relaxed) } @@ -1128,8 +1097,6 @@ enum ServiceToWorkerMsg { SetPeersetReserved(ProtocolName, HashSet), AddSetReserved(ProtocolName, PeerId), RemoveSetReserved(ProtocolName, PeerId), - AddToPeersSet(ProtocolName, PeerId), - RemoveFromPeersSet(ProtocolName, PeerId), EventStream(out_events::Sender), Request { target: PeerId, @@ -1306,16 +1273,6 @@ where .remove_set_reserved_peer(protocol, peer_id), ServiceToWorkerMsg::AddKnownAddress(peer_id, addr) => self.network_service.behaviour_mut().add_known_address(peer_id, addr), - ServiceToWorkerMsg::AddToPeersSet(protocol, peer_id) => self - .network_service - .behaviour_mut() - .user_protocol_mut() - .add_to_peers_set(protocol, peer_id), - ServiceToWorkerMsg::RemoveFromPeersSet(protocol, peer_id) => self - .network_service - .behaviour_mut() - .user_protocol_mut() - .remove_from_peers_set(protocol, peer_id), ServiceToWorkerMsg::EventStream(sender) => self.event_streams.push(sender), ServiceToWorkerMsg::Request { target, @@ -1349,8 +1306,7 @@ where .user_protocol_mut() .set_notification_handshake(protocol, handshake), ServiceToWorkerMsg::ReservedPeers { pending_response } => { - let _ = - pending_response.send(self.reserved_peers().map(ToOwned::to_owned).collect()); + self.reserved_peers(pending_response); }, } } @@ -1454,16 +1410,10 @@ where .behaviour_mut() .add_self_reported_address_to_dht(&peer_id, &protocols, addr); } - self.network_service - .behaviour_mut() - .user_protocol_mut() - .add_default_set_discovered_nodes(iter::once(peer_id)); + self.network_service.behaviour_mut().user_protocol_mut().add_known_peer(peer_id); }, SwarmEvent::Behaviour(BehaviourOut::Discovered(peer_id)) => { - self.network_service - .behaviour_mut() - .user_protocol_mut() - .add_default_set_discovered_nodes(iter::once(peer_id)); + self.network_service.behaviour_mut().user_protocol_mut().add_known_peer(peer_id); }, SwarmEvent::Behaviour(BehaviourOut::RandomKademliaStarted) => { if let Some(metrics) = self.metrics.as_ref() { diff --git a/client/network/src/service/traits.rs b/client/network/src/service/traits.rs index 787ef4b5ae445..22af3816bd15d 100644 --- a/client/network/src/service/traits.rs +++ b/client/network/src/service/traits.rs @@ -156,10 +156,6 @@ pub trait NetworkPeers { /// Disconnect from a node as soon as possible. /// /// This triggers the same effects as if the connection had closed itself spontaneously. - /// - /// See also [`NetworkPeers::remove_from_peers_set`], which has the same effect but also - /// prevents the local node from re-establishing an outgoing substream to this peer until it - /// is added again. fn disconnect_peer(&self, who: PeerId, protocol: ProtocolName); /// Connect to unreserved peers and allow unreserved peers to connect for syncing purposes. @@ -216,26 +212,6 @@ pub trait NetworkPeers { /// Remove peers from a peer set. fn remove_peers_from_reserved_set(&self, protocol: ProtocolName, peers: Vec); - /// Add a peer to a set of peers. - /// - /// If the set has slots available, it will try to open a substream with this peer. - /// - /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also - /// consist of only `/p2p/`. - /// - /// Returns an `Err` if one of the given addresses is invalid or contains an - /// invalid peer ID (which includes the local peer ID). - fn add_to_peers_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String>; - - /// Remove peers from a peer set. - /// - /// If we currently have an open substream with this peer, it will soon be closed. - fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec); - /// Returns the number of peers in the sync peer set we're connected to. fn sync_num_connected(&self) -> usize; } @@ -259,6 +235,10 @@ where } fn report_peer(&self, who: PeerId, cost_benefit: ReputationChange) { + // TODO: when we get rid of `Peerset`, we'll likely need to add some kind of async + // interface to `PeerStore`, otherwise we'll have trouble calling functions accepting + // `&mut self` via `Arc`. + // See https://github.com/paritytech/substrate/issues/14170. T::report_peer(self, who, cost_benefit) } @@ -302,18 +282,6 @@ where T::remove_peers_from_reserved_set(self, protocol, peers) } - fn add_to_peers_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String> { - T::add_to_peers_set(self, protocol, peers) - } - - fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec) { - T::remove_from_peers_set(self, protocol, peers) - } - fn sync_num_connected(&self) -> usize { T::sync_num_connected(self) } diff --git a/client/network/sync/src/service/mock.rs b/client/network/sync/src/service/mock.rs index c882633993c8b..9bce9f91b1d24 100644 --- a/client/network/sync/src/service/mock.rs +++ b/client/network/sync/src/service/mock.rs @@ -101,12 +101,6 @@ mockall::mock! { peers: HashSet, ) -> Result<(), String>; fn remove_peers_from_reserved_set(&self, protocol: ProtocolName, peers: Vec); - fn add_to_peers_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String>; - fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec); fn sync_num_connected(&self) -> usize; } diff --git a/client/offchain/src/api.rs b/client/offchain/src/api.rs index a15f03bab6f84..f6cfdcd536bb5 100644 --- a/client/offchain/src/api.rs +++ b/client/offchain/src/api.rs @@ -392,18 +392,6 @@ mod tests { unimplemented!(); } - fn add_to_peers_set( - &self, - _protocol: ProtocolName, - _peers: HashSet, - ) -> Result<(), String> { - unimplemented!(); - } - - fn remove_from_peers_set(&self, _protocol: ProtocolName, _peers: Vec) { - unimplemented!(); - } - fn sync_num_connected(&self) -> usize { unimplemented!(); } diff --git a/client/offchain/src/lib.rs b/client/offchain/src/lib.rs index f46fb637a92d3..013bb27980e29 100644 --- a/client/offchain/src/lib.rs +++ b/client/offchain/src/lib.rs @@ -333,18 +333,6 @@ mod tests { unimplemented!(); } - fn add_to_peers_set( - &self, - _protocol: ProtocolName, - _peers: HashSet, - ) -> Result<(), String> { - unimplemented!(); - } - - fn remove_from_peers_set(&self, _protocol: ProtocolName, _peers: Vec) { - unimplemented!(); - } - fn sync_num_connected(&self) -> usize { unimplemented!(); } diff --git a/client/peerset/Cargo.toml b/client/peerset/Cargo.toml index 043f8a8352caa..b07a6ffa09c7c 100644 --- a/client/peerset/Cargo.toml +++ b/client/peerset/Cargo.toml @@ -17,9 +17,14 @@ targets = ["x86_64-unknown-linux-gnu"] futures = "0.3.21" libp2p-identity = { version = "0.1.2", features = ["peerid", "ed25519"] } log = "0.4.17" +parking_lot = "0.12.1" +partial_sort = "0.2.0" serde_json = "1.0.85" wasm-timer = "0.2" sc-utils = { version = "4.0.0-dev", path = "../utils" } +sp-arithmetic = { version = "6.0.0", path = "../../primitives/arithmetic" } [dev-dependencies] +sp-tracing = { version = "6.0.0", path = "../../primitives/tracing" } +mockall = "0.11.3" rand = "0.8.5" diff --git a/client/peerset/src/lib.rs b/client/peerset/src/lib.rs index e169be8e8ed5b..8a5ed7ddcd134 100644 --- a/client/peerset/src/lib.rs +++ b/client/peerset/src/lib.rs @@ -32,29 +32,32 @@ //! In addition, for each, set, the peerset also holds a list of reserved nodes towards which it //! will at all time try to maintain a connection with. -mod peersstate; +mod peer_store; +mod protocol_controller; -use futures::{channel::oneshot, prelude::*}; -use log::{debug, error, trace}; +use peer_store::{PeerStore, PeerStoreHandle, PeerStoreProvider}; +use protocol_controller::{ProtocolController, ProtocolHandle}; + +use futures::{ + channel::oneshot, + future::{join_all, BoxFuture, JoinAll}, + prelude::*, + stream::Stream, +}; +use log::debug; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use serde_json::json; use std::{ - collections::{HashMap, HashSet, VecDeque}, + collections::HashSet, pin::Pin, task::{Context, Poll}, - time::{Duration, Instant}, }; -use wasm_timer::Delay; pub use libp2p_identity::PeerId; -/// We don't accept nodes whose reputation is under this value. -pub const BANNED_THRESHOLD: i32 = 82 * (i32::MIN / 100); -/// Reputation change for a node when we get disconnected from it. -const DISCONNECT_REPUTATION_CHANGE: i32 = -256; -/// Amount of time between the moment we disconnect from a node and the moment we remove it from -/// the list. -const FORGET_AFTER: Duration = Duration::from_secs(3600); +pub use peer_store::BANNED_THRESHOLD; + +pub const LOG_TARGET: &str = "peerset"; #[derive(Debug)] enum Action { @@ -63,8 +66,7 @@ enum Action { SetReservedPeers(SetId, HashSet), SetReservedOnly(SetId, bool), ReportPeer(PeerId, ReputationChange), - AddToPeersSet(SetId, PeerId), - RemoveFromPeersSet(SetId, PeerId), + AddKnownPeer(PeerId), PeerReputation(PeerId, oneshot::Sender), } @@ -157,14 +159,9 @@ impl PeersetHandle { let _ = self.tx.unbounded_send(Action::ReportPeer(peer_id, score_diff)); } - /// Add a peer to a set. - pub fn add_to_peers_set(&self, set_id: SetId, peer_id: PeerId) { - let _ = self.tx.unbounded_send(Action::AddToPeersSet(set_id, peer_id)); - } - - /// Remove a peer from a set. - pub fn remove_from_peers_set(&self, set_id: SetId, peer_id: PeerId) { - let _ = self.tx.unbounded_send(Action::RemoveFromPeersSet(set_id, peer_id)); + /// Add a peer to the list of known peers. + pub fn add_known_peer(&self, peer_id: PeerId) { + let _ = self.tx.unbounded_send(Action::AddKnownPeer(peer_id)); } /// Returns the reputation value of the peer. @@ -184,6 +181,7 @@ pub enum Message { /// Request to open a connection to the given peer. From the point of view of the PSM, we are /// immediately connected. Connect { + /// Set id to connect on. set_id: SetId, /// Peer to connect to. peer_id: PeerId, @@ -191,6 +189,7 @@ pub enum Message { /// Drop the connection to the given peer, or cancel the connection attempt after a `Connect`. Drop { + /// Set id to disconnect on. set_id: SetId, /// Peer to disconnect from. peer_id: PeerId, @@ -204,7 +203,7 @@ pub enum Message { } /// Opaque identifier for an incoming connection. Allocated by the network. -#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] pub struct IncomingIndex(pub u64); impl From for IncomingIndex { @@ -249,360 +248,73 @@ pub struct SetConfig { /// /// Implements the `Stream` trait and can be polled for messages. The `Stream` never ends and never /// errors. -#[derive(Debug)] pub struct Peerset { - /// Underlying data structure for the nodes's states. - data: peersstate::PeersState, - /// For each set, lists of nodes that don't occupy slots and that we should try to always be - /// connected to, and whether only reserved nodes are accepted. Is kept in sync with the list - /// of non-slot-occupying nodes in [`Peerset::data`]. - reserved_nodes: Vec<(HashSet, bool)>, - /// Receiver for messages from the `PeersetHandle` and from `tx`. - rx: TracingUnboundedReceiver, - /// Sending side of `rx`. - tx: TracingUnboundedSender, - /// Queue of messages to be emitted when the `Peerset` is polled. - message_queue: VecDeque, - /// When the `Peerset` was created. - created: Instant, - /// Last time when we updated the reputations of connected nodes. - latest_time_update: Instant, - /// Next time to do a periodic call to `alloc_slots` with all sets. This is done once per - /// second, to match the period of the reputation updates. - next_periodic_alloc_slots: Delay, + /// Peer reputation store handle. + peer_store_handle: PeerStoreHandle, + /// Peer reputation store. + peer_store_future: BoxFuture<'static, ()>, + /// Protocol handles. + protocol_handles: Vec, + /// Protocol controllers responsible for connections, per `SetId`. + protocol_controller_futures: JoinAll>, + /// Commands sent from protocol controllers to `Notifications`. The size of this vector never + /// changes. + from_controllers: TracingUnboundedReceiver, + /// Receiver for messages from the `PeersetHandle` and from `to_self`. + from_handle: TracingUnboundedReceiver, + /// Sending side of `from_handle`. + to_self: TracingUnboundedSender, } impl Peerset { /// Builds a new peerset from the given configuration. - pub fn from_config(config: PeersetConfig) -> (Self, PeersetHandle) { - let (tx, rx) = tracing_unbounded("mpsc_peerset_messages", 10_000); - - let handle = PeersetHandle { tx: tx.clone() }; - - let mut peerset = { - let now = Instant::now(); - - Self { - data: peersstate::PeersState::new(config.sets.iter().map(|set| { - peersstate::SetConfig { in_peers: set.in_peers, out_peers: set.out_peers } - })), - tx, - rx, - reserved_nodes: config - .sets - .iter() - .map(|set| (set.reserved_nodes.clone(), set.reserved_only)) - .collect(), - message_queue: VecDeque::new(), - created: now, - latest_time_update: now, - next_periodic_alloc_slots: Delay::new(Duration::new(0, 0)), - } + pub fn from_config(config: PeersetConfig) -> (Peerset, PeersetHandle) { + let default_set_config = &config.sets[0]; + let peer_store = PeerStore::new(default_set_config.bootnodes.clone()); + + let (to_notifications, from_controllers) = + tracing_unbounded("mpsc_protocol_controllers_to_notifications", 10_000); + + let controllers = config + .sets + .into_iter() + .enumerate() + .map(|(set, set_config)| { + ProtocolController::new( + SetId::from(set), + set_config, + to_notifications.clone(), + Box::new(peer_store.handle()), + ) + }) + .collect::>(); + + let (protocol_handles, protocol_controllers): (Vec, Vec<_>) = + controllers.into_iter().unzip(); + + let (to_self, from_handle) = tracing_unbounded("mpsc_peerset_messages", 10_000); + + let handle = PeersetHandle { tx: to_self.clone() }; + + let protocol_controller_futures = + join_all(protocol_controllers.into_iter().map(|c| c.run().boxed())); + + let peerset = Peerset { + peer_store_handle: peer_store.handle(), + peer_store_future: peer_store.run().boxed(), + protocol_handles, + protocol_controller_futures, + from_controllers, + from_handle, + to_self, }; - for (set, set_config) in config.sets.into_iter().enumerate() { - for node in set_config.reserved_nodes { - peerset.data.add_no_slot_node(set, node); - } - - for peer_id in set_config.bootnodes { - if let peersstate::Peer::Unknown(entry) = peerset.data.peer(set, &peer_id) { - entry.discover(); - } else { - debug!(target: "peerset", "Duplicate bootnode in config: {:?}", peer_id); - } - } - } - - for set_index in 0..peerset.data.num_sets() { - peerset.alloc_slots(SetId(set_index)); - } - (peerset, handle) } - fn on_add_reserved_peer(&mut self, set_id: SetId, peer_id: PeerId) { - let newly_inserted = self.reserved_nodes[set_id.0].0.insert(peer_id); - if !newly_inserted { - return - } - - self.data.add_no_slot_node(set_id.0, peer_id); - self.alloc_slots(set_id); - } - - fn on_remove_reserved_peer(&mut self, set_id: SetId, peer_id: PeerId) { - if !self.reserved_nodes[set_id.0].0.remove(&peer_id) { - return - } - - self.data.remove_no_slot_node(set_id.0, &peer_id); - - // Nothing more to do if not in reserved-only mode. - if !self.reserved_nodes[set_id.0].1 { - return - } - - // If, however, the peerset is in reserved-only mode, then the removed node needs to be - // disconnected. - if let peersstate::Peer::Connected(peer) = self.data.peer(set_id.0, &peer_id) { - peer.disconnect(); - self.message_queue.push_back(Message::Drop { set_id, peer_id }); - } - } - - fn on_set_reserved_peers(&mut self, set_id: SetId, peer_ids: HashSet) { - // Determine the difference between the current group and the new list. - let (to_insert, to_remove) = { - let to_insert = peer_ids - .difference(&self.reserved_nodes[set_id.0].0) - .cloned() - .collect::>(); - let to_remove = self.reserved_nodes[set_id.0] - .0 - .difference(&peer_ids) - .cloned() - .collect::>(); - (to_insert, to_remove) - }; - - for node in to_insert { - self.on_add_reserved_peer(set_id, node); - } - - for node in to_remove { - self.on_remove_reserved_peer(set_id, node); - } - } - - fn on_set_reserved_only(&mut self, set_id: SetId, reserved_only: bool) { - self.reserved_nodes[set_id.0].1 = reserved_only; - - if reserved_only { - // Disconnect all the nodes that aren't reserved. - for peer_id in - self.data.connected_peers(set_id.0).cloned().collect::>().into_iter() - { - if self.reserved_nodes[set_id.0].0.contains(&peer_id) { - continue - } - - let peer = self.data.peer(set_id.0, &peer_id).into_connected().expect( - "We are enumerating connected peers, therefore the peer is connected; qed", - ); - peer.disconnect(); - self.message_queue.push_back(Message::Drop { set_id, peer_id }); - } - } else { - self.alloc_slots(set_id); - } - } - /// Returns the list of reserved peers. - pub fn reserved_peers(&self, set_id: SetId) -> impl Iterator { - self.reserved_nodes[set_id.0].0.iter() - } - - /// Adds a node to the given set. The peerset will, if possible and not already the case, - /// try to connect to it. - /// - /// > **Note**: This has the same effect as [`PeersetHandle::add_to_peers_set`]. - pub fn add_to_peers_set(&mut self, set_id: SetId, peer_id: PeerId) { - if let peersstate::Peer::Unknown(entry) = self.data.peer(set_id.0, &peer_id) { - entry.discover(); - self.alloc_slots(set_id); - } - } - - fn on_remove_from_peers_set(&mut self, set_id: SetId, peer_id: PeerId) { - // Don't do anything if node is reserved. - if self.reserved_nodes[set_id.0].0.contains(&peer_id) { - return - } - - match self.data.peer(set_id.0, &peer_id) { - peersstate::Peer::Connected(peer) => { - self.message_queue.push_back(Message::Drop { set_id, peer_id: *peer.peer_id() }); - peer.disconnect().forget_peer(); - }, - peersstate::Peer::NotConnected(peer) => { - peer.forget_peer(); - }, - peersstate::Peer::Unknown(_) => {}, - } - } - - fn on_report_peer(&mut self, peer_id: PeerId, change: ReputationChange) { - // We want reputations to be up-to-date before adjusting them. - self.update_time(); - - let mut reputation = self.data.peer_reputation(peer_id); - reputation.add_reputation(change.value); - if reputation.reputation() >= BANNED_THRESHOLD { - trace!(target: "peerset", "Report {}: {:+} to {}. Reason: {}", - peer_id, change.value, reputation.reputation(), change.reason - ); - return - } - - debug!(target: "peerset", "Report {}: {:+} to {}. Reason: {}, Disconnecting", - peer_id, change.value, reputation.reputation(), change.reason - ); - - drop(reputation); - - for set_index in 0..self.data.num_sets() { - if let peersstate::Peer::Connected(peer) = self.data.peer(set_index, &peer_id) { - let peer = peer.disconnect(); - self.message_queue.push_back(Message::Drop { - set_id: SetId(set_index), - peer_id: peer.into_peer_id(), - }); - - self.alloc_slots(SetId(set_index)); - } - } - } - - fn on_peer_reputation(&mut self, peer_id: PeerId, pending_response: oneshot::Sender) { - let reputation = self.data.peer_reputation(peer_id); - let _ = pending_response.send(reputation.reputation()); - } - - /// Updates the value of `self.latest_time_update` and performs all the updates that happen - /// over time, such as reputation increases for staying connected. - fn update_time(&mut self) { - let now = Instant::now(); - - // We basically do `(now - self.latest_update).as_secs()`, except that by the way we do it - // we know that we're not going to miss seconds because of rounding to integers. - let secs_diff = { - let elapsed_latest = self.latest_time_update - self.created; - let elapsed_now = now - self.created; - self.latest_time_update = now; - elapsed_now.as_secs() - elapsed_latest.as_secs() - }; - - // For each elapsed second, move the node reputation towards zero. - // If we multiply each second the reputation by `k` (where `k` is between 0 and 1), it - // takes `ln(0.5) / ln(k)` seconds to reduce the reputation by half. Use this formula to - // empirically determine a value of `k` that looks correct. - for _ in 0..secs_diff { - for peer_id in self.data.peers().cloned().collect::>() { - // We use `k = 0.98`, so we divide by `50`. With that value, it takes 34.3 seconds - // to reduce the reputation by half. - fn reput_tick(reput: i32) -> i32 { - let mut diff = reput / 50; - if diff == 0 && reput < 0 { - diff = -1; - } else if diff == 0 && reput > 0 { - diff = 1; - } - reput.saturating_sub(diff) - } - - let mut peer_reputation = self.data.peer_reputation(peer_id); - - let before = peer_reputation.reputation(); - let after = reput_tick(before); - trace!(target: "peerset", "Fleeting {}: {} -> {}", peer_id, before, after); - peer_reputation.set_reputation(after); - - if after != 0 { - continue - } - - drop(peer_reputation); - - // If the peer reaches a reputation of 0, and there is no connection to it, - // forget it. - for set_index in 0..self.data.num_sets() { - match self.data.peer(set_index, &peer_id) { - peersstate::Peer::Connected(_) => {}, - peersstate::Peer::NotConnected(peer) => { - if peer.last_connected_or_discovered() + FORGET_AFTER < now { - peer.forget_peer(); - } - }, - peersstate::Peer::Unknown(_) => { - // Happens if this peer does not belong to this set. - }, - } - } - } - } - } - - /// Try to fill available out slots with nodes for the given set. - fn alloc_slots(&mut self, set_id: SetId) { - self.update_time(); - - // Try to connect to all the reserved nodes that we are not connected to. - for reserved_node in &self.reserved_nodes[set_id.0].0 { - let entry = match self.data.peer(set_id.0, reserved_node) { - peersstate::Peer::Unknown(n) => n.discover(), - peersstate::Peer::NotConnected(n) => n, - peersstate::Peer::Connected(_) => continue, - }; - - // Don't connect to nodes with an abysmal reputation, even if they're reserved. - // This is a rather opinionated behaviour, and it wouldn't be fundamentally wrong to - // remove that check. If necessary, the peerset should be refactored to give more - // control over what happens in that situation. - if entry.reputation() < BANNED_THRESHOLD { - break - } - - match entry.try_outgoing() { - Ok(conn) => self - .message_queue - .push_back(Message::Connect { set_id, peer_id: conn.into_peer_id() }), - Err(_) => { - // An error is returned only if no slot is available. Reserved nodes are - // marked in the state machine with a flag saying "doesn't occupy a slot", - // and as such this should never happen. - debug_assert!(false); - log::error!( - target: "peerset", - "Not enough slots to connect to reserved node" - ); - }, - } - } - - // Now, we try to connect to other nodes. - - // Nothing more to do if we're in reserved mode. - if self.reserved_nodes[set_id.0].1 { - return - } - - // Try to grab the next node to attempt to connect to. - // Since `highest_not_connected_peer` is rather expensive to call, check beforehand - // whether we have an available slot. - while self.data.has_free_outgoing_slot(set_id.0) { - let next = match self.data.highest_not_connected_peer(set_id.0) { - Some(n) => n, - None => break, - }; - - // Don't connect to nodes with an abysmal reputation. - if next.reputation() < BANNED_THRESHOLD { - break - } - - match next.try_outgoing() { - Ok(conn) => self - .message_queue - .push_back(Message::Connect { set_id, peer_id: conn.into_peer_id() }), - Err(_) => { - // This branch can only be entered if there is no free slot, which is - // checked above. - debug_assert!(false); - break - }, - } - } + pub fn reserved_peers(&self, set_id: SetId, pending_response: oneshot::Sender>) { + self.protocol_handles[set_id.0].reserved_peers(pending_response); } /// Indicate that we received an incoming connection. Must be answered either with @@ -615,62 +327,15 @@ impl Peerset { // message to the output channel with a `PeerId`, and that `incoming` gets called with the same // `PeerId` before that message has been read by the user. In this situation we must not answer. pub fn incoming(&mut self, set_id: SetId, peer_id: PeerId, index: IncomingIndex) { - trace!(target: "peerset", "Incoming {:?}", peer_id); - - self.update_time(); - - if self.reserved_nodes[set_id.0].1 && !self.reserved_nodes[set_id.0].0.contains(&peer_id) { - self.message_queue.push_back(Message::Reject(index)); - return - } - - let not_connected = match self.data.peer(set_id.0, &peer_id) { - // If we're already connected, don't answer, as the docs mention. - peersstate::Peer::Connected(_) => return, - peersstate::Peer::NotConnected(mut entry) => { - entry.bump_last_connected_or_discovered(); - entry - }, - peersstate::Peer::Unknown(entry) => entry.discover(), - }; - - if not_connected.reputation() < BANNED_THRESHOLD { - self.message_queue.push_back(Message::Reject(index)); - return - } - - match not_connected.try_accept_incoming() { - Ok(_) => self.message_queue.push_back(Message::Accept(index)), - Err(_) => self.message_queue.push_back(Message::Reject(index)), - } + self.protocol_handles[set_id.0].incoming_connection(peer_id, index); } /// Indicate that we dropped an active connection with a peer, or that we failed to connect. /// /// Must only be called after the PSM has either generated a `Connect` message with this /// `PeerId`, or accepted an incoming connection with this `PeerId`. - pub fn dropped(&mut self, set_id: SetId, peer_id: PeerId, reason: DropReason) { - // We want reputations to be up-to-date before adjusting them. - self.update_time(); - - match self.data.peer(set_id.0, &peer_id) { - peersstate::Peer::Connected(mut entry) => { - // Decrease the node's reputation so that we don't try it again and again and again. - entry.add_reputation(DISCONNECT_REPUTATION_CHANGE); - trace!(target: "peerset", "Dropping {}: {:+} to {}", - peer_id, DISCONNECT_REPUTATION_CHANGE, entry.reputation()); - entry.disconnect(); - }, - peersstate::Peer::NotConnected(_) | peersstate::Peer::Unknown(_) => { - error!(target: "peerset", "Received dropped() for non-connected node") - }, - } - - if let DropReason::Refused = reason { - self.on_remove_from_peers_set(set_id, peer_id); - } - - self.alloc_slots(set_id); + pub fn dropped(&mut self, set_id: SetId, peer_id: PeerId, _reason: DropReason) { + self.protocol_handles[set_id.0].dropped(peer_id); } /// Reports an adjustment to the reputation of the given peer. @@ -678,44 +343,19 @@ impl Peerset { // We don't immediately perform the adjustments in order to have state consistency. We // don't want the reporting here to take priority over messages sent using the // `PeersetHandle`. - let _ = self.tx.unbounded_send(Action::ReportPeer(peer_id, score_diff)); + let _ = self.to_self.unbounded_send(Action::ReportPeer(peer_id, score_diff)); } /// Produces a JSON object containing the state of the peerset manager, for debugging purposes. pub fn debug_info(&mut self) -> serde_json::Value { - self.update_time(); - - json!({ - "sets": (0..self.data.num_sets()).map(|set_index| { - json!({ - "nodes": self.data.peers().cloned().collect::>().into_iter().filter_map(|peer_id| { - let state = match self.data.peer(set_index, &peer_id) { - peersstate::Peer::Connected(entry) => json!({ - "connected": true, - "reputation": entry.reputation() - }), - peersstate::Peer::NotConnected(entry) => json!({ - "connected": false, - "reputation": entry.reputation() - }), - peersstate::Peer::Unknown(_) => return None, - }; - - Some((peer_id.to_base58(), state)) - }).collect::>(), - "reserved_nodes": self.reserved_nodes[set_index].0.iter().map(|peer_id| { - peer_id.to_base58() - }).collect::>(), - "reserved_only": self.reserved_nodes[set_index].1, - }) - }).collect::>(), - "message_queue": self.message_queue.len(), - }) + // TODO: Check what info we can include here. + // Issue reference: https://github.com/paritytech/substrate/issues/14160. + json!("unimplemented") } /// Returns the number of peers that we have discovered. pub fn num_discovered_peers(&self) -> usize { - self.data.peers().len() + self.peer_store_handle.num_known_peers() } } @@ -723,277 +363,66 @@ impl Stream for Peerset { type Item = Message; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { - loop { - if let Some(message) = self.message_queue.pop_front() { - return Poll::Ready(Some(message)) + if let Poll::Ready(msg) = self.from_controllers.poll_next_unpin(cx) { + if let Some(msg) = msg { + return Poll::Ready(Some(msg)) + } else { + debug!( + target: LOG_TARGET, + "All `ProtocolController`s have terminated, terminating `Peerset`." + ); + return Poll::Ready(None) } + } - if Future::poll(Pin::new(&mut self.next_periodic_alloc_slots), cx).is_ready() { - self.next_periodic_alloc_slots = Delay::new(Duration::new(1, 0)); - - for set_index in 0..self.data.num_sets() { - self.alloc_slots(SetId(set_index)); + while let Poll::Ready(action) = self.from_handle.poll_next_unpin(cx) { + if let Some(action) = action { + match action { + Action::AddReservedPeer(set_id, peer_id) => + self.protocol_handles[set_id.0].add_reserved_peer(peer_id), + Action::RemoveReservedPeer(set_id, peer_id) => + self.protocol_handles[set_id.0].remove_reserved_peer(peer_id), + Action::SetReservedPeers(set_id, peer_ids) => + self.protocol_handles[set_id.0].set_reserved_peers(peer_ids), + Action::SetReservedOnly(set_id, reserved_only) => + self.protocol_handles[set_id.0].set_reserved_only(reserved_only), + Action::ReportPeer(peer_id, score_diff) => + self.peer_store_handle.report_peer(peer_id, score_diff), + Action::AddKnownPeer(peer_id) => self.peer_store_handle.add_known_peer(peer_id), + Action::PeerReputation(peer_id, pending_response) => { + let _ = + pending_response.send(self.peer_store_handle.peer_reputation(&peer_id)); + }, } + } else { + debug!(target: LOG_TARGET, "`PeersetHandle` was dropped, terminating `Peerset`."); + return Poll::Ready(None) } + } - let action = match Stream::poll_next(Pin::new(&mut self.rx), cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(Some(event)) => event, - Poll::Ready(None) => return Poll::Pending, - }; - - match action { - Action::AddReservedPeer(set_id, peer_id) => - self.on_add_reserved_peer(set_id, peer_id), - Action::RemoveReservedPeer(set_id, peer_id) => - self.on_remove_reserved_peer(set_id, peer_id), - Action::SetReservedPeers(set_id, peer_ids) => - self.on_set_reserved_peers(set_id, peer_ids), - Action::SetReservedOnly(set_id, reserved) => - self.on_set_reserved_only(set_id, reserved), - Action::ReportPeer(peer_id, score_diff) => self.on_report_peer(peer_id, score_diff), - Action::AddToPeersSet(sets_name, peer_id) => - self.add_to_peers_set(sets_name, peer_id), - Action::RemoveFromPeersSet(sets_name, peer_id) => - self.on_remove_from_peers_set(sets_name, peer_id), - Action::PeerReputation(peer_id, pending_response) => - self.on_peer_reputation(peer_id, pending_response), - } + if let Poll::Ready(()) = self.peer_store_future.poll_unpin(cx) { + debug!(target: LOG_TARGET, "`PeerStore` has terminated, terminating `PeerSet`."); + return Poll::Ready(None) + } + + if let Poll::Ready(_) = self.protocol_controller_futures.poll_unpin(cx) { + debug!( + target: LOG_TARGET, + "All `ProtocolHandle`s have terminated, terminating `PeerSet`." + ); + return Poll::Ready(None) } + + Poll::Pending } } /// Reason for calling [`Peerset::dropped`]. +#[derive(Debug)] pub enum DropReason { /// Substream or connection has been closed for an unknown reason. Unknown, /// Substream or connection has been explicitly refused by the target. In other words, the /// peer doesn't actually belong to this set. - /// - /// This has the side effect of calling [`PeersetHandle::remove_from_peers_set`]. Refused, } - -#[cfg(test)] -mod tests { - use super::{ - IncomingIndex, Message, Peerset, PeersetConfig, ReputationChange, SetConfig, SetId, - BANNED_THRESHOLD, - }; - use futures::prelude::*; - use libp2p_identity::PeerId; - use std::{pin::Pin, task::Poll, thread, time::Duration}; - - fn assert_messages(mut peerset: Peerset, messages: Vec) -> Peerset { - for expected_message in messages { - let (message, p) = next_message(peerset).expect("expected message"); - assert_eq!(message, expected_message); - peerset = p; - } - peerset - } - - fn next_message(mut peerset: Peerset) -> Result<(Message, Peerset), ()> { - let next = futures::executor::block_on_stream(&mut peerset).next(); - let message = next.ok_or(())?; - Ok((message, peerset)) - } - - #[test] - fn test_peerset_add_reserved_peer() { - let bootnode = PeerId::random(); - let reserved_peer = PeerId::random(); - let reserved_peer2 = PeerId::random(); - let config = PeersetConfig { - sets: vec![SetConfig { - in_peers: 0, - out_peers: 2, - bootnodes: vec![bootnode], - reserved_nodes: Default::default(), - reserved_only: true, - }], - }; - - let (peerset, handle) = Peerset::from_config(config); - handle.add_reserved_peer(SetId::from(0), reserved_peer); - handle.add_reserved_peer(SetId::from(0), reserved_peer2); - - assert_messages( - peerset, - vec![ - Message::Connect { set_id: SetId::from(0), peer_id: reserved_peer }, - Message::Connect { set_id: SetId::from(0), peer_id: reserved_peer2 }, - ], - ); - } - - #[test] - fn test_peerset_incoming() { - let bootnode = PeerId::random(); - let incoming = PeerId::random(); - let incoming2 = PeerId::random(); - let incoming3 = PeerId::random(); - let ii = IncomingIndex(1); - let ii2 = IncomingIndex(2); - let ii3 = IncomingIndex(3); - let ii4 = IncomingIndex(3); - let config = PeersetConfig { - sets: vec![SetConfig { - in_peers: 2, - out_peers: 1, - bootnodes: vec![bootnode], - reserved_nodes: Default::default(), - reserved_only: false, - }], - }; - - let (mut peerset, _handle) = Peerset::from_config(config); - peerset.incoming(SetId::from(0), incoming, ii); - peerset.incoming(SetId::from(0), incoming, ii4); - peerset.incoming(SetId::from(0), incoming2, ii2); - peerset.incoming(SetId::from(0), incoming3, ii3); - - assert_messages( - peerset, - vec![ - Message::Connect { set_id: SetId::from(0), peer_id: bootnode }, - Message::Accept(ii), - Message::Accept(ii2), - Message::Reject(ii3), - ], - ); - } - - #[test] - fn test_peerset_reject_incoming_in_reserved_only() { - let incoming = PeerId::random(); - let ii = IncomingIndex(1); - let config = PeersetConfig { - sets: vec![SetConfig { - in_peers: 50, - out_peers: 50, - bootnodes: vec![], - reserved_nodes: Default::default(), - reserved_only: true, - }], - }; - - let (mut peerset, _) = Peerset::from_config(config); - peerset.incoming(SetId::from(0), incoming, ii); - - assert_messages(peerset, vec![Message::Reject(ii)]); - } - - #[test] - fn test_peerset_discovered() { - let bootnode = PeerId::random(); - let discovered = PeerId::random(); - let discovered2 = PeerId::random(); - let config = PeersetConfig { - sets: vec![SetConfig { - in_peers: 0, - out_peers: 2, - bootnodes: vec![bootnode], - reserved_nodes: Default::default(), - reserved_only: false, - }], - }; - - let (mut peerset, _handle) = Peerset::from_config(config); - peerset.add_to_peers_set(SetId::from(0), discovered); - peerset.add_to_peers_set(SetId::from(0), discovered); - peerset.add_to_peers_set(SetId::from(0), discovered2); - - assert_messages( - peerset, - vec![ - Message::Connect { set_id: SetId::from(0), peer_id: bootnode }, - Message::Connect { set_id: SetId::from(0), peer_id: discovered }, - ], - ); - } - - #[test] - fn test_peerset_banned() { - let (mut peerset, handle) = Peerset::from_config(PeersetConfig { - sets: vec![SetConfig { - in_peers: 25, - out_peers: 25, - bootnodes: vec![], - reserved_nodes: Default::default(), - reserved_only: false, - }], - }); - - // We ban a node by setting its reputation under the threshold. - let peer_id = PeerId::random(); - handle.report_peer(peer_id, ReputationChange::new(BANNED_THRESHOLD - 1, "")); - - let fut = futures::future::poll_fn(move |cx| { - // We need one polling for the message to be processed. - assert_eq!(Stream::poll_next(Pin::new(&mut peerset), cx), Poll::Pending); - - // Check that an incoming connection from that node gets refused. - peerset.incoming(SetId::from(0), peer_id, IncomingIndex(1)); - if let Poll::Ready(msg) = Stream::poll_next(Pin::new(&mut peerset), cx) { - assert_eq!(msg.unwrap(), Message::Reject(IncomingIndex(1))); - } else { - panic!() - } - - // Wait a bit for the node's reputation to go above the threshold. - thread::sleep(Duration::from_millis(1500)); - - // Try again. This time the node should be accepted. - peerset.incoming(SetId::from(0), peer_id, IncomingIndex(2)); - while let Poll::Ready(msg) = Stream::poll_next(Pin::new(&mut peerset), cx) { - assert_eq!(msg.unwrap(), Message::Accept(IncomingIndex(2))); - } - - Poll::Ready(()) - }); - - futures::executor::block_on(fut); - } - - #[test] - fn test_relloc_after_banned() { - let (mut peerset, handle) = Peerset::from_config(PeersetConfig { - sets: vec![SetConfig { - in_peers: 25, - out_peers: 25, - bootnodes: vec![], - reserved_nodes: Default::default(), - reserved_only: false, - }], - }); - - // We ban a node by setting its reputation under the threshold. - let peer_id = PeerId::random(); - handle.report_peer(peer_id, ReputationChange::new(BANNED_THRESHOLD - 1, "")); - - let fut = futures::future::poll_fn(move |cx| { - // We need one polling for the message to be processed. - assert_eq!(Stream::poll_next(Pin::new(&mut peerset), cx), Poll::Pending); - - // Check that an incoming connection from that node gets refused. - // This is already tested in other tests, but it is done again here because it doesn't - // hurt. - peerset.incoming(SetId::from(0), peer_id, IncomingIndex(1)); - if let Poll::Ready(msg) = Stream::poll_next(Pin::new(&mut peerset), cx) { - assert_eq!(msg.unwrap(), Message::Reject(IncomingIndex(1))); - } else { - panic!() - } - - // Wait for the peerset to change its mind and actually connect to it. - while let Poll::Ready(msg) = Stream::poll_next(Pin::new(&mut peerset), cx) { - assert_eq!(msg.unwrap(), Message::Connect { set_id: SetId::from(0), peer_id }); - } - - Poll::Ready(()) - }); - - futures::executor::block_on(fut); - } -} diff --git a/client/peerset/src/peer_store.rs b/client/peerset/src/peer_store.rs new file mode 100644 index 0000000000000..4255118213209 --- /dev/null +++ b/client/peerset/src/peer_store.rs @@ -0,0 +1,403 @@ +// This file is part of Substrate. + +// Copyright (C) Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +use libp2p_identity::PeerId; +use log::trace; +use parking_lot::Mutex; +use partial_sort::PartialSort; +use std::{ + cmp::{Ord, Ordering, PartialOrd}, + collections::{hash_map::Entry, HashMap, HashSet}, + fmt::Debug, + sync::Arc, + time::{Duration, Instant}, +}; +use wasm_timer::Delay; + +use crate::{protocol_controller::ProtocolHandle, ReputationChange, LOG_TARGET}; + +/// We don't accept nodes whose reputation is under this value. +pub const BANNED_THRESHOLD: i32 = 82 * (i32::MIN / 100); +/// Reputation change for a node when we get disconnected from it. +const DISCONNECT_REPUTATION_CHANGE: i32 = -256; +/// Relative decrement of a reputation value that is applied every second. I.e., for inverse +/// decrement of 50 we decrease absolute value of the reputation by 1/50. This corresponds to a +/// factor of `k = 0.98`. It takes ~ `ln(0.5) / ln(k)` seconds to reduce the reputation by half, +/// or 34.3 seconds for the values above. In this setup the maximum allowed absolute value of +/// `i32::MAX` becomes 0 in ~1100 seconds (actually less due to integer arithmetic). +const INVERSE_DECREMENT: i32 = 50; +/// Amount of time between the moment we last updated the [`PeerStore`] entry and the moment we +/// remove it, once the reputation value reaches 0. +const FORGET_AFTER: Duration = Duration::from_secs(3600); + +pub trait PeerStoreProvider: Debug + Send { + /// Check whether the peer is banned. + fn is_banned(&self, peer_id: &PeerId) -> bool; + + /// Register a protocol handle to disconnect peers whose reputation drops below the threshold. + fn register_protocol(&self, protocol_handle: ProtocolHandle); + + /// Report peer disconnection for reputation adjustment. + fn report_disconnect(&mut self, peer_id: PeerId); + + /// Adjust peer reputation. + fn report_peer(&mut self, peer_id: PeerId, change: ReputationChange); + + /// Get peer reputation. + fn peer_reputation(&self, peer_id: &PeerId) -> i32; + + /// Get candidates with highest reputations for initiating outgoing connections. + fn outgoing_candidates(&self, count: usize, ignored: HashSet<&PeerId>) -> Vec; +} + +#[derive(Debug, Clone)] +pub struct PeerStoreHandle { + inner: Arc>, +} + +impl PeerStoreProvider for PeerStoreHandle { + fn is_banned(&self, peer_id: &PeerId) -> bool { + self.inner.lock().is_banned(peer_id) + } + + fn register_protocol(&self, protocol_handle: ProtocolHandle) { + self.inner.lock().register_protocol(protocol_handle); + } + + fn report_disconnect(&mut self, peer_id: PeerId) { + self.inner.lock().report_disconnect(peer_id) + } + + fn report_peer(&mut self, peer_id: PeerId, change: ReputationChange) { + self.inner.lock().report_peer(peer_id, change) + } + + fn peer_reputation(&self, peer_id: &PeerId) -> i32 { + self.inner.lock().peer_reputation(peer_id) + } + + fn outgoing_candidates(&self, count: usize, ignored: HashSet<&PeerId>) -> Vec { + self.inner.lock().outgoing_candidates(count, ignored) + } +} + +impl PeerStoreHandle { + /// Get the number of known peers. + /// + /// This number might not include some connected peers in rare cases when their reputation + /// was not updated for one hour, because their entries in [`PeerStore`] were dropped. + pub fn num_known_peers(&self) -> usize { + self.inner.lock().peers.len() + } + + /// Add known peer. + pub fn add_known_peer(&mut self, peer_id: PeerId) { + self.inner.lock().add_known_peer(peer_id); + } +} + +#[derive(Debug, Clone, Copy)] +struct PeerInfo { + reputation: i32, + last_updated: Instant, +} + +impl Default for PeerInfo { + fn default() -> Self { + Self { reputation: 0, last_updated: Instant::now() } + } +} + +impl PartialEq for PeerInfo { + fn eq(&self, other: &Self) -> bool { + self.reputation == other.reputation + } +} + +impl Eq for PeerInfo {} + +impl Ord for PeerInfo { + // We define reverse order by reputation values. + fn cmp(&self, other: &Self) -> Ordering { + self.reputation.cmp(&other.reputation).reverse() + } +} + +impl PartialOrd for PeerInfo { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl PeerInfo { + fn is_banned(&self) -> bool { + self.reputation < BANNED_THRESHOLD + } + + fn add_reputation(&mut self, increment: i32) { + self.reputation = self.reputation.saturating_add(increment); + self.bump_last_updated(); + } + + fn decay_reputation(&mut self, seconds_passed: u64) { + // Note that decaying the reputation value happens "on its own", + // so we don't do `bump_last_updated()`. + for _ in 0..seconds_passed { + let mut diff = self.reputation / INVERSE_DECREMENT; + if diff == 0 && self.reputation < 0 { + diff = -1; + } else if diff == 0 && self.reputation > 0 { + diff = 1; + } + + self.reputation = self.reputation.saturating_sub(diff); + + if self.reputation == 0 { + break + } + } + } + + fn bump_last_updated(&mut self) { + self.last_updated = Instant::now(); + } +} + +#[derive(Debug)] +struct PeerStoreInner { + peers: HashMap, + protocols: Vec, +} + +impl PeerStoreInner { + fn is_banned(&self, peer_id: &PeerId) -> bool { + self.peers.get(peer_id).map_or(false, |info| info.is_banned()) + } + + fn register_protocol(&mut self, protocol_handle: ProtocolHandle) { + self.protocols.push(protocol_handle); + } + + fn report_disconnect(&mut self, peer_id: PeerId) { + let peer_info = self.peers.entry(peer_id).or_default(); + peer_info.add_reputation(DISCONNECT_REPUTATION_CHANGE); + + log::trace!( + target: LOG_TARGET, + "Peer {} disconnected, reputation: {:+} to {}", + peer_id, + DISCONNECT_REPUTATION_CHANGE, + peer_info.reputation, + ); + } + + fn report_peer(&mut self, peer_id: PeerId, change: ReputationChange) { + let peer_info = self.peers.entry(peer_id).or_default(); + peer_info.add_reputation(change.value); + + if peer_info.reputation < BANNED_THRESHOLD { + self.protocols.iter().for_each(|handle| handle.disconnect_peer(peer_id)); + + log::trace!( + target: LOG_TARGET, + "Report {}: {:+} to {}. Reason: {}. Banned, disconnecting.", + peer_id, + change.value, + peer_info.reputation, + change.reason, + ); + } else { + log::trace!( + target: LOG_TARGET, + "Report {}: {:+} to {}. Reason: {}.", + peer_id, + change.value, + peer_info.reputation, + change.reason, + ); + } + } + + fn peer_reputation(&self, peer_id: &PeerId) -> i32 { + self.peers.get(peer_id).map_or(0, |info| info.reputation) + } + + fn outgoing_candidates(&self, count: usize, ignored: HashSet<&PeerId>) -> Vec { + let mut candidates = self + .peers + .iter() + .filter_map(|(peer_id, info)| { + (!info.is_banned() && !ignored.contains(peer_id)).then_some((*peer_id, *info)) + }) + .collect::>(); + let count = std::cmp::min(count, candidates.len()); + candidates.partial_sort(count, |(_, info1), (_, info2)| info1.cmp(info2)); + candidates.iter().take(count).map(|(peer_id, _)| *peer_id).collect() + + // TODO: keep the peers sorted (in a "bi-multi-map"?) to not repeat sorting every time. + } + + fn progress_time(&mut self, seconds_passed: u64) { + if seconds_passed == 0 { + return + } + + // Drive reputation values towards 0. + self.peers + .iter_mut() + .for_each(|(_, info)| info.decay_reputation(seconds_passed)); + + // Retain only entries with non-zero reputation values or not expired ones. + let now = Instant::now(); + self.peers + .retain(|_, info| info.reputation != 0 || info.last_updated + FORGET_AFTER > now); + } + + fn add_known_peer(&mut self, peer_id: PeerId) { + match self.peers.entry(peer_id) { + Entry::Occupied(mut e) => { + trace!( + target: LOG_TARGET, + "Trying to add an already known peer {peer_id}, bumping `last_updated`.", + ); + e.get_mut().bump_last_updated(); + }, + Entry::Vacant(e) => { + trace!(target: LOG_TARGET, "Adding a new known peer {peer_id}."); + e.insert(PeerInfo::default()); + }, + } + } +} + +#[derive(Debug)] +pub struct PeerStore { + inner: Arc>, +} + +impl PeerStore { + /// Create a new peer store from the list of bootnodes. + pub fn new(bootnodes: Vec) -> Self { + PeerStore { + inner: Arc::new(Mutex::new(PeerStoreInner { + peers: bootnodes + .into_iter() + .map(|peer_id| (peer_id, PeerInfo::default())) + .collect(), + protocols: Vec::new(), + })), + } + } + + /// Get `PeerStoreHandle`. + pub fn handle(&self) -> PeerStoreHandle { + PeerStoreHandle { inner: self.inner.clone() } + } + + /// Drive the `PeerStore`, decaying reputation values over time and removing expired entries. + pub async fn run(self) { + let started = Instant::now(); + let mut latest_time_update = started; + + loop { + let now = Instant::now(); + // We basically do `(now - self.latest_update).as_secs()`, except that by the way we do + // it we know that we're not going to miss seconds because of rounding to integers. + let seconds_passed = { + let elapsed_latest = latest_time_update - started; + let elapsed_now = now - started; + latest_time_update = now; + elapsed_now.as_secs() - elapsed_latest.as_secs() + }; + + self.inner.lock().progress_time(seconds_passed); + let _ = Delay::new(Duration::from_secs(1)).await; + } + } +} + +#[cfg(test)] +mod tests { + use super::PeerInfo; + + #[test] + fn decaying_zero_reputation_yields_zero() { + let mut peer_info = PeerInfo::default(); + assert_eq!(peer_info.reputation, 0); + + peer_info.decay_reputation(1); + assert_eq!(peer_info.reputation, 0); + + peer_info.decay_reputation(100_000); + assert_eq!(peer_info.reputation, 0); + } + + #[test] + fn decaying_positive_reputation_decreases_it() { + const INITIAL_REPUTATION: i32 = 100; + + let mut peer_info = PeerInfo::default(); + peer_info.reputation = INITIAL_REPUTATION; + + peer_info.decay_reputation(1); + assert!(peer_info.reputation >= 0); + assert!(peer_info.reputation < INITIAL_REPUTATION); + } + + #[test] + fn decaying_negative_reputation_increases_it() { + const INITIAL_REPUTATION: i32 = -100; + + let mut peer_info = PeerInfo::default(); + peer_info.reputation = INITIAL_REPUTATION; + + peer_info.decay_reputation(1); + assert!(peer_info.reputation <= 0); + assert!(peer_info.reputation > INITIAL_REPUTATION); + } + + #[test] + fn decaying_max_reputation_finally_yields_zero() { + const INITIAL_REPUTATION: i32 = i32::MAX; + const SECONDS: u64 = 1000; + + let mut peer_info = PeerInfo::default(); + peer_info.reputation = INITIAL_REPUTATION; + + peer_info.decay_reputation(SECONDS / 2); + assert!(peer_info.reputation > 0); + + peer_info.decay_reputation(SECONDS / 2); + assert_eq!(peer_info.reputation, 0); + } + + #[test] + fn decaying_min_reputation_finally_yields_zero() { + const INITIAL_REPUTATION: i32 = i32::MIN; + const SECONDS: u64 = 1000; + + let mut peer_info = PeerInfo::default(); + peer_info.reputation = INITIAL_REPUTATION; + + peer_info.decay_reputation(SECONDS / 2); + assert!(peer_info.reputation < 0); + + peer_info.decay_reputation(SECONDS / 2); + assert_eq!(peer_info.reputation, 0); + } +} diff --git a/client/peerset/src/peersstate.rs b/client/peerset/src/peersstate.rs deleted file mode 100644 index 2d4a9295c24c9..0000000000000 --- a/client/peerset/src/peersstate.rs +++ /dev/null @@ -1,737 +0,0 @@ -// This file is part of Substrate. - -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 - -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . - -//! Reputation and slots allocation system behind the peerset. -//! -//! The [`PeersState`] state machine is responsible for managing the reputation and allocating -//! slots. It holds a list of nodes, each associated with a reputation value, a list of sets the -//! node belongs to, and for each set whether we are connected or not to this node. Thanks to this -//! list, it knows how many slots are occupied. It also holds a list of nodes which don't occupy -//! slots. -//! -//! > Note: This module is purely dedicated to managing slots and reputations. Features such as -//! > for example connecting to some nodes in priority should be added outside of this -//! > module, rather than inside. - -use libp2p_identity::PeerId; -use log::error; -use std::{ - borrow::Cow, - collections::{ - hash_map::{Entry, OccupiedEntry}, - HashMap, HashSet, - }, - time::Instant, -}; - -/// State storage behind the peerset. -/// -/// # Usage -/// -/// This struct is nothing more but a data structure containing a list of nodes, where each node -/// has a reputation and is either connected to us or not. -#[derive(Debug, Clone)] -pub struct PeersState { - /// List of nodes that we know about. - /// - /// > **Note**: This list should really be ordered by decreasing reputation, so that we can - /// > easily select the best node to connect to. As a first draft, however, we don't sort, to - /// > make the logic easier. - nodes: HashMap, - - /// Configuration of each set. The size of this `Vec` is never modified. - sets: Vec, -} - -/// Configuration of a single set. -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -pub struct SetConfig { - /// Maximum allowed number of slot-occupying nodes for ingoing connections. - pub in_peers: u32, - - /// Maximum allowed number of slot-occupying nodes for outgoing connections. - pub out_peers: u32, -} - -/// State of a single set. -#[derive(Debug, Clone, PartialEq, Eq)] -struct SetInfo { - /// Number of slot-occupying nodes for which the `MembershipState` is `In`. - num_in: u32, - - /// Number of slot-occupying nodes for which the `MembershipState` is `In`. - num_out: u32, - - /// Maximum allowed number of slot-occupying nodes for which the `MembershipState` is `In`. - max_in: u32, - - /// Maximum allowed number of slot-occupying nodes for which the `MembershipState` is `Out`. - max_out: u32, - - /// List of node identities (discovered or not) that don't occupy slots. - /// - /// Note for future readers: this module is purely dedicated to managing slots. If you are - /// considering adding more features, please consider doing so outside of this module rather - /// than inside. - no_slot_nodes: HashSet, -} - -/// State of a single node that we know about. -#[derive(Debug, Clone, PartialEq, Eq)] -struct Node { - /// List of sets the node belongs to. - /// Always has a fixed size equal to the one of [`PeersState::set`]. The various possible sets - /// are indices into this `Vec`. - sets: Vec, - - /// Reputation value of the node, between `i32::MIN` (we hate that node) and - /// `i32::MAX` (we love that node). - reputation: i32, -} - -impl Node { - fn new(num_sets: usize) -> Self { - Self { sets: (0..num_sets).map(|_| MembershipState::NotMember).collect(), reputation: 0 } - } -} - -/// Whether we are connected to a node in the context of a specific set. -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -enum MembershipState { - /// Node isn't part of that set. - NotMember, - /// We are connected through an ingoing connection. - In, - /// We are connected through an outgoing connection. - Out, - /// Node is part of that set, but we are not connected to it. - NotConnected { - /// When we were last connected to the node, or if we were never connected when we - /// discovered it. - last_connected: Instant, - }, -} - -impl MembershipState { - /// Returns `true` for [`MembershipState::In`] and [`MembershipState::Out`]. - fn is_connected(self) -> bool { - match self { - Self::In | Self::Out => true, - Self::NotMember | Self::NotConnected { .. } => false, - } - } - - /// Returns `true` for [`MembershipState::NotConnected`]. - fn is_not_connected(self) -> bool { - matches!(self, Self::NotConnected { .. }) - } -} - -impl PeersState { - /// Builds a new empty [`PeersState`]. - pub fn new(sets: impl IntoIterator) -> Self { - Self { - nodes: HashMap::new(), - sets: sets - .into_iter() - .map(|config| SetInfo { - num_in: 0, - num_out: 0, - max_in: config.in_peers, - max_out: config.out_peers, - no_slot_nodes: HashSet::new(), - }) - .collect(), - } - } - - /// Returns the number of sets. - /// - /// Corresponds to the number of elements passed to [`PeersState::new`]. - pub fn num_sets(&self) -> usize { - self.sets.len() - } - - /// Returns an object that grants access to the reputation value of a peer. - pub fn peer_reputation(&mut self, peer_id: PeerId) -> Reputation { - self.nodes.entry(peer_id).or_insert_with(|| Node::new(self.sets.len())); - - let entry = match self.nodes.entry(peer_id) { - Entry::Vacant(_) => unreachable!("guaranteed to be inserted above; qed"), - Entry::Occupied(e) => e, - }; - - Reputation { node: Some(entry) } - } - - /// Returns an object that grants access to the state of a peer in the context of a specific - /// set. - /// - /// # Panic - /// - /// `set` must be within range of the sets passed to [`PeersState::new`]. - pub fn peer<'a>(&'a mut self, set: usize, peer_id: &'a PeerId) -> Peer<'a> { - // The code below will panic anyway if this happens to be false, but this earlier assert - // makes it explicit what is wrong. - assert!(set < self.sets.len()); - - match self.nodes.get_mut(peer_id).map(|p| &p.sets[set]) { - None | Some(MembershipState::NotMember) => - Peer::Unknown(UnknownPeer { parent: self, set, peer_id: Cow::Borrowed(peer_id) }), - Some(MembershipState::In) | Some(MembershipState::Out) => - Peer::Connected(ConnectedPeer { state: self, set, peer_id: Cow::Borrowed(peer_id) }), - Some(MembershipState::NotConnected { .. }) => Peer::NotConnected(NotConnectedPeer { - state: self, - set, - peer_id: Cow::Borrowed(peer_id), - }), - } - } - - /// Returns the list of all the peers we know of. - // Note: this method could theoretically return a `Peer`, but implementing that - // isn't simple. - pub fn peers(&self) -> impl ExactSizeIterator { - self.nodes.keys() - } - - /// Returns the list of peers we are connected to in the context of a specific set. - /// - /// # Panic - /// - /// `set` must be within range of the sets passed to [`PeersState::new`]. - // Note: this method could theoretically return a `ConnectedPeer`, but implementing that - // isn't simple. - pub fn connected_peers(&self, set: usize) -> impl Iterator { - // The code below will panic anyway if this happens to be false, but this earlier assert - // makes it explicit what is wrong. - assert!(set < self.sets.len()); - - self.nodes - .iter() - .filter(move |(_, p)| p.sets[set].is_connected()) - .map(|(p, _)| p) - } - - /// Returns the peer with the highest reputation and that we are not connected to. - /// - /// If multiple nodes have the same reputation, which one is returned is unspecified. - /// - /// # Panic - /// - /// `set` must be within range of the sets passed to [`PeersState::new`]. - pub fn highest_not_connected_peer(&mut self, set: usize) -> Option { - // The code below will panic anyway if this happens to be false, but this earlier assert - // makes it explicit what is wrong. - assert!(set < self.sets.len()); - - let outcome = self - .nodes - .iter_mut() - .filter(|(_, Node { sets, .. })| sets[set].is_not_connected()) - .fold(None::<(&PeerId, &mut Node)>, |mut cur_node, to_try| { - if let Some(cur_node) = cur_node.take() { - if cur_node.1.reputation >= to_try.1.reputation { - return Some(cur_node) - } - } - Some(to_try) - }) - .map(|(peer_id, _)| *peer_id); - - outcome.map(move |peer_id| NotConnectedPeer { - state: self, - set, - peer_id: Cow::Owned(peer_id), - }) - } - - /// Returns `true` if there is a free outgoing slot available related to this set. - pub fn has_free_outgoing_slot(&self, set: usize) -> bool { - self.sets[set].num_out < self.sets[set].max_out - } - - /// Add a node to the list of nodes that don't occupy slots. - /// - /// Has no effect if the node was already in the group. - pub fn add_no_slot_node(&mut self, set: usize, peer_id: PeerId) { - // Reminder: `HashSet::insert` returns false if the node was already in the set - if !self.sets[set].no_slot_nodes.insert(peer_id) { - return - } - - if let Some(peer) = self.nodes.get_mut(&peer_id) { - match peer.sets[set] { - MembershipState::In => self.sets[set].num_in -= 1, - MembershipState::Out => self.sets[set].num_out -= 1, - MembershipState::NotConnected { .. } | MembershipState::NotMember => {}, - } - } - } - - /// Removes a node from the list of nodes that don't occupy slots. - /// - /// Has no effect if the node was not in the group. - pub fn remove_no_slot_node(&mut self, set: usize, peer_id: &PeerId) { - // Reminder: `HashSet::remove` returns false if the node was already not in the set - if !self.sets[set].no_slot_nodes.remove(peer_id) { - return - } - - if let Some(peer) = self.nodes.get_mut(peer_id) { - match peer.sets[set] { - MembershipState::In => self.sets[set].num_in += 1, - MembershipState::Out => self.sets[set].num_out += 1, - MembershipState::NotConnected { .. } | MembershipState::NotMember => {}, - } - } - } -} - -/// Grants access to the state of a peer in the [`PeersState`] in the context of a specific set. -pub enum Peer<'a> { - /// We are connected to this node. - Connected(ConnectedPeer<'a>), - /// We are not connected to this node. - NotConnected(NotConnectedPeer<'a>), - /// We have never heard of this node, or it is not part of the set. - Unknown(UnknownPeer<'a>), -} - -impl<'a> Peer<'a> { - /// If we are the `Connected` variant, returns the inner [`ConnectedPeer`]. Returns `None` - /// otherwise. - pub fn into_connected(self) -> Option> { - match self { - Self::Connected(peer) => Some(peer), - Self::NotConnected(..) | Self::Unknown(..) => None, - } - } - - /// If we are the `NotConnected` variant, returns the inner [`NotConnectedPeer`]. Returns `None` - /// otherwise. - #[cfg(test)] // Feel free to remove this if this function is needed outside of tests - pub fn into_not_connected(self) -> Option> { - match self { - Self::NotConnected(peer) => Some(peer), - Self::Connected(..) | Self::Unknown(..) => None, - } - } - - /// If we are the `Unknown` variant, returns the inner [`UnknownPeer`]. Returns `None` - /// otherwise. - #[cfg(test)] // Feel free to remove this if this function is needed outside of tests - pub fn into_unknown(self) -> Option> { - match self { - Self::Unknown(peer) => Some(peer), - Self::Connected(..) | Self::NotConnected(..) => None, - } - } -} - -/// A peer that is connected to us. -pub struct ConnectedPeer<'a> { - state: &'a mut PeersState, - set: usize, - peer_id: Cow<'a, PeerId>, -} - -impl<'a> ConnectedPeer<'a> { - /// Get the `PeerId` associated to this `ConnectedPeer`. - pub fn peer_id(&self) -> &PeerId { - &self.peer_id - } - - /// Destroys this `ConnectedPeer` and returns the `PeerId` inside of it. - pub fn into_peer_id(self) -> PeerId { - self.peer_id.into_owned() - } - - /// Switches the peer to "not connected". - pub fn disconnect(self) -> NotConnectedPeer<'a> { - let is_no_slot_occupy = self.state.sets[self.set].no_slot_nodes.contains(&*self.peer_id); - if let Some(node) = self.state.nodes.get_mut(&*self.peer_id) { - if !is_no_slot_occupy { - match node.sets[self.set] { - MembershipState::In => self.state.sets[self.set].num_in -= 1, - MembershipState::Out => self.state.sets[self.set].num_out -= 1, - MembershipState::NotMember | MembershipState::NotConnected { .. } => { - debug_assert!( - false, - "State inconsistency: disconnecting a disconnected node" - ) - }, - } - } - node.sets[self.set] = MembershipState::NotConnected { last_connected: Instant::now() }; - } else { - debug_assert!(false, "State inconsistency: disconnecting a disconnected node"); - } - - NotConnectedPeer { state: self.state, set: self.set, peer_id: self.peer_id } - } - - /// Performs an arithmetic addition on the reputation score of that peer. - /// - /// In case of overflow, the value will be capped. - /// - /// > **Note**: Reputation values aren't specific to a set but are global per peer. - pub fn add_reputation(&mut self, modifier: i32) { - if let Some(node) = self.state.nodes.get_mut(&*self.peer_id) { - node.reputation = node.reputation.saturating_add(modifier); - } else { - debug_assert!(false, "State inconsistency: add_reputation on an unknown node"); - } - } - - /// Returns the reputation value of the node. - /// - /// > **Note**: Reputation values aren't specific to a set but are global per peer. - pub fn reputation(&self) -> i32 { - self.state.nodes.get(&*self.peer_id).map_or(0, |p| p.reputation) - } -} - -/// A peer that is not connected to us. -#[derive(Debug)] -pub struct NotConnectedPeer<'a> { - state: &'a mut PeersState, - set: usize, - peer_id: Cow<'a, PeerId>, -} - -impl<'a> NotConnectedPeer<'a> { - /// Destroys this `NotConnectedPeer` and returns the `PeerId` inside of it. - pub fn into_peer_id(self) -> PeerId { - self.peer_id.into_owned() - } - - /// Bumps the value that `last_connected_or_discovered` would return to now, even if we - /// didn't connect or disconnect. - pub fn bump_last_connected_or_discovered(&mut self) { - let state = match self.state.nodes.get_mut(&*self.peer_id) { - Some(s) => s, - None => return, - }; - - if let MembershipState::NotConnected { last_connected } = &mut state.sets[self.set] { - *last_connected = Instant::now(); - } - } - - /// Returns when we were last connected to this peer, or when we discovered it if we were - /// never connected. - /// - /// Guaranteed to be earlier than calling `Instant::now()` after the function returns. - pub fn last_connected_or_discovered(&self) -> Instant { - let state = match self.state.nodes.get(&*self.peer_id) { - Some(s) => s, - None => { - error!( - target: "peerset", - "State inconsistency with {}; not connected after borrow", - self.peer_id - ); - return Instant::now() - }, - }; - - match state.sets[self.set] { - MembershipState::NotConnected { last_connected } => last_connected, - _ => { - error!(target: "peerset", "State inconsistency with {}", self.peer_id); - Instant::now() - }, - } - } - - /// Tries to set the peer as connected as an outgoing connection. - /// - /// If there are enough slots available, switches the node to "connected" and returns `Ok`. If - /// the slots are full, the node stays "not connected" and we return `Err`. - /// - /// Non-slot-occupying nodes don't count towards the number of slots. - pub fn try_outgoing(self) -> Result, Self> { - let is_no_slot_occupy = self.state.sets[self.set].no_slot_nodes.contains(&*self.peer_id); - - // Note that it is possible for num_out to be strictly superior to the max, in case we were - // connected to reserved node then marked them as not reserved. - if !self.state.has_free_outgoing_slot(self.set) && !is_no_slot_occupy { - return Err(self) - } - - if let Some(peer) = self.state.nodes.get_mut(&*self.peer_id) { - peer.sets[self.set] = MembershipState::Out; - if !is_no_slot_occupy { - self.state.sets[self.set].num_out += 1; - } - } else { - debug_assert!(false, "State inconsistency: try_outgoing on an unknown node"); - } - - Ok(ConnectedPeer { state: self.state, set: self.set, peer_id: self.peer_id }) - } - - /// Tries to accept the peer as an incoming connection. - /// - /// If there are enough slots available, switches the node to "connected" and returns `Ok`. If - /// the slots are full, the node stays "not connected" and we return `Err`. - /// - /// Non-slot-occupying nodes don't count towards the number of slots. - pub fn try_accept_incoming(self) -> Result, Self> { - let is_no_slot_occupy = self.state.sets[self.set].no_slot_nodes.contains(&*self.peer_id); - - // Note that it is possible for num_in to be strictly superior to the max, in case we were - // connected to reserved node then marked them as not reserved. - if self.state.sets[self.set].num_in >= self.state.sets[self.set].max_in && - !is_no_slot_occupy - { - return Err(self) - } - - if let Some(peer) = self.state.nodes.get_mut(&*self.peer_id) { - peer.sets[self.set] = MembershipState::In; - if !is_no_slot_occupy { - self.state.sets[self.set].num_in += 1; - } - } else { - debug_assert!(false, "State inconsistency: try_accept_incoming on an unknown node"); - } - - Ok(ConnectedPeer { state: self.state, set: self.set, peer_id: self.peer_id }) - } - - /// Returns the reputation value of the node. - /// - /// > **Note**: Reputation values aren't specific to a set but are global per peer. - pub fn reputation(&self) -> i32 { - self.state.nodes.get(&*self.peer_id).map_or(0, |p| p.reputation) - } - - /// Sets the reputation of the peer. - /// - /// > **Note**: Reputation values aren't specific to a set but are global per peer. - #[cfg(test)] // Feel free to remove this if this function is needed outside of tests - pub fn set_reputation(&mut self, value: i32) { - if let Some(node) = self.state.nodes.get_mut(&*self.peer_id) { - node.reputation = value; - } else { - debug_assert!(false, "State inconsistency: set_reputation on an unknown node"); - } - } - - /// Removes the peer from the list of members of the set. - pub fn forget_peer(self) -> UnknownPeer<'a> { - if let Some(peer) = self.state.nodes.get_mut(&*self.peer_id) { - debug_assert!(!matches!(peer.sets[self.set], MembershipState::NotMember)); - peer.sets[self.set] = MembershipState::NotMember; - - // Remove the peer from `self.state.nodes` entirely if it isn't a member of any set. - if peer.reputation == 0 && - peer.sets.iter().all(|set| matches!(set, MembershipState::NotMember)) - { - self.state.nodes.remove(&*self.peer_id); - } - } else { - debug_assert!(false, "State inconsistency: forget_peer on an unknown node"); - error!( - target: "peerset", - "State inconsistency with {} when forgetting peer", - self.peer_id - ); - }; - - UnknownPeer { parent: self.state, set: self.set, peer_id: self.peer_id } - } -} - -/// A peer that we have never heard of or that isn't part of the set. -pub struct UnknownPeer<'a> { - parent: &'a mut PeersState, - set: usize, - peer_id: Cow<'a, PeerId>, -} - -impl<'a> UnknownPeer<'a> { - /// Inserts the peer identity in our list. - /// - /// The node starts with a reputation of 0. You can adjust these default - /// values using the `NotConnectedPeer` that this method returns. - pub fn discover(self) -> NotConnectedPeer<'a> { - let num_sets = self.parent.sets.len(); - - self.parent - .nodes - .entry(self.peer_id.clone().into_owned()) - .or_insert_with(|| Node::new(num_sets)) - .sets[self.set] = MembershipState::NotConnected { last_connected: Instant::now() }; - - NotConnectedPeer { state: self.parent, set: self.set, peer_id: self.peer_id } - } -} - -/// Access to the reputation of a peer. -pub struct Reputation<'a> { - /// Node entry in [`PeersState::nodes`]. Always `Some` except right before dropping. - node: Option>, -} - -impl<'a> Reputation<'a> { - /// Returns the reputation value of the node. - pub fn reputation(&self) -> i32 { - self.node.as_ref().unwrap().get().reputation - } - - /// Sets the reputation of the peer. - pub fn set_reputation(&mut self, value: i32) { - self.node.as_mut().unwrap().get_mut().reputation = value; - } - - /// Performs an arithmetic addition on the reputation score of that peer. - /// - /// In case of overflow, the value will be capped. - pub fn add_reputation(&mut self, modifier: i32) { - let reputation = &mut self.node.as_mut().unwrap().get_mut().reputation; - *reputation = reputation.saturating_add(modifier); - } -} - -impl<'a> Drop for Reputation<'a> { - fn drop(&mut self) { - if let Some(node) = self.node.take() { - if node.get().reputation == 0 && - node.get().sets.iter().all(|set| matches!(set, MembershipState::NotMember)) - { - node.remove(); - } - } - } -} - -#[cfg(test)] -mod tests { - use super::{Peer, PeersState, SetConfig}; - use libp2p_identity::PeerId; - use std::iter; - - #[test] - fn full_slots_in() { - let mut peers_state = PeersState::new(iter::once(SetConfig { in_peers: 1, out_peers: 1 })); - let id1 = PeerId::random(); - let id2 = PeerId::random(); - - if let Peer::Unknown(e) = peers_state.peer(0, &id1) { - assert!(e.discover().try_accept_incoming().is_ok()); - } - - if let Peer::Unknown(e) = peers_state.peer(0, &id2) { - assert!(e.discover().try_accept_incoming().is_err()); - } - } - - #[test] - fn no_slot_node_doesnt_use_slot() { - let mut peers_state = PeersState::new(iter::once(SetConfig { in_peers: 1, out_peers: 1 })); - let id1 = PeerId::random(); - let id2 = PeerId::random(); - - peers_state.add_no_slot_node(0, id1); - if let Peer::Unknown(p) = peers_state.peer(0, &id1) { - assert!(p.discover().try_accept_incoming().is_ok()); - } else { - panic!() - } - - if let Peer::Unknown(e) = peers_state.peer(0, &id2) { - assert!(e.discover().try_accept_incoming().is_ok()); - } else { - panic!() - } - } - - #[test] - fn disconnecting_frees_slot() { - let mut peers_state = PeersState::new(iter::once(SetConfig { in_peers: 1, out_peers: 1 })); - let id1 = PeerId::random(); - let id2 = PeerId::random(); - - assert!(peers_state - .peer(0, &id1) - .into_unknown() - .unwrap() - .discover() - .try_accept_incoming() - .is_ok()); - assert!(peers_state - .peer(0, &id2) - .into_unknown() - .unwrap() - .discover() - .try_accept_incoming() - .is_err()); - peers_state.peer(0, &id1).into_connected().unwrap().disconnect(); - assert!(peers_state - .peer(0, &id2) - .into_not_connected() - .unwrap() - .try_accept_incoming() - .is_ok()); - } - - #[test] - fn highest_not_connected_peer() { - let mut peers_state = - PeersState::new(iter::once(SetConfig { in_peers: 25, out_peers: 25 })); - let id1 = PeerId::random(); - let id2 = PeerId::random(); - - assert!(peers_state.highest_not_connected_peer(0).is_none()); - peers_state.peer(0, &id1).into_unknown().unwrap().discover().set_reputation(50); - peers_state.peer(0, &id2).into_unknown().unwrap().discover().set_reputation(25); - assert_eq!(peers_state.highest_not_connected_peer(0).map(|p| p.into_peer_id()), Some(id1)); - peers_state.peer(0, &id2).into_not_connected().unwrap().set_reputation(75); - assert_eq!(peers_state.highest_not_connected_peer(0).map(|p| p.into_peer_id()), Some(id2)); - peers_state - .peer(0, &id2) - .into_not_connected() - .unwrap() - .try_accept_incoming() - .unwrap(); - assert_eq!(peers_state.highest_not_connected_peer(0).map(|p| p.into_peer_id()), Some(id1)); - peers_state.peer(0, &id1).into_not_connected().unwrap().set_reputation(100); - peers_state.peer(0, &id2).into_connected().unwrap().disconnect(); - assert_eq!(peers_state.highest_not_connected_peer(0).map(|p| p.into_peer_id()), Some(id1)); - peers_state.peer(0, &id1).into_not_connected().unwrap().set_reputation(-100); - assert_eq!(peers_state.highest_not_connected_peer(0).map(|p| p.into_peer_id()), Some(id2)); - } - - #[test] - fn disconnect_no_slot_doesnt_panic() { - let mut peers_state = PeersState::new(iter::once(SetConfig { in_peers: 1, out_peers: 1 })); - let id = PeerId::random(); - peers_state.add_no_slot_node(0, id); - let peer = peers_state - .peer(0, &id) - .into_unknown() - .unwrap() - .discover() - .try_outgoing() - .unwrap(); - peer.disconnect(); - } -} diff --git a/client/peerset/src/protocol_controller.rs b/client/peerset/src/protocol_controller.rs new file mode 100644 index 0000000000000..ce3961f130af0 --- /dev/null +++ b/client/peerset/src/protocol_controller.rs @@ -0,0 +1,1939 @@ +// This file is part of Substrate. + +// Copyright (C) Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +//! Protocol Controller. Generic implementation of peer management for protocols. +//! Responsible for accepting/rejecting incoming connections and initiating outgoing connections, +//! respecting the inbound and outbound peer slot counts. Communicates with `PeerStore` to get and +//! update peer reputation values and sends commands to `Notifications`. +//! +//! Due to asynchronous nature of communication between `ProtocolController` and `Notifications`, +//! `ProtocolController` has an imperfect view of the states of the peers. To reduce this +//! desynchronization, the following measures are taken: +//! +//! 1. Network peer events from `Notifications` are prioritized over actions from external API and +//! internal actions by `ProtocolController` (like slot allocation). +//! 2. `Notifications` ignores all commands from `ProtocolController` after sending "incoming" +//! request until receiving the answer to this "incoming" request. +//! 3. After sending a "connect" message, `ProtocolController` switches the state of the peer from +//! `Outbound` to `Inbound` if it receives an "incoming" request from `Notifications` for this +//! peer. +//! +//! These measures do not eliminate confusing commands from `ProtocolController` completely, +//! so `Notifications` must correctly handle seemingly inconsistent commands, like a "connect" +//! command for the peer it thinks is already connected, and a "drop" command for a peer that +//! was previously dropped. +//! +//! Even though this does not guarantee that `ProtocolController` and `Notifications` have the same +//! view of the peers' states at any given moment, the eventual consistency is maintained. + +use futures::{channel::oneshot, future::Either, FutureExt, StreamExt}; +use libp2p_identity::PeerId; +use log::{error, trace, warn}; +use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; +use sp_arithmetic::traits::SaturatedConversion; +use std::{ + collections::{HashMap, HashSet}, + time::{Duration, Instant}, +}; +use wasm_timer::Delay; + +use crate::{peer_store::PeerStoreProvider, IncomingIndex, Message, SetConfig, SetId, LOG_TARGET}; + +/// External API actions. +#[derive(Debug)] +enum Action { + /// Add a reserved peer or mark already connected peer as reserved. + AddReservedPeer(PeerId), + /// Remove a reserved peer. + RemoveReservedPeer(PeerId), + /// Update reserved peers to match the provided set. + SetReservedPeers(HashSet), + /// Set/unset reserved-only mode. + SetReservedOnly(bool), + /// Disconnect a peer. + DisconnectPeer(PeerId), + /// Get the list of reserved peers. + GetReservedPeers(oneshot::Sender>), +} + +/// Network events from `Notifications`. +#[derive(Debug)] +enum Event { + /// Incoming connection from the peer. + IncomingConnection(PeerId, IncomingIndex), + /// Connection with the peer dropped. + Dropped(PeerId), +} + +/// Shared handle to [`ProtocolController`]. Distributed around the code outside of the +/// protocol implementation. +#[derive(Debug, Clone)] +pub struct ProtocolHandle { + /// Actions from outer API. + actions_tx: TracingUnboundedSender, + /// Connection events from `Notifications`. We prioritize them over actions. + events_tx: TracingUnboundedSender, +} + +impl ProtocolHandle { + /// Adds a new reserved peer. [`ProtocolController`] will make an effort + /// to always remain connected to this peer. + /// + /// Has no effect if the node was already a reserved peer. + /// + /// > **Note**: Keep in mind that the networking has to know an address for this node, + /// > otherwise it will not be able to connect to it. + pub fn add_reserved_peer(&self, peer_id: PeerId) { + let _ = self.actions_tx.unbounded_send(Action::AddReservedPeer(peer_id)); + } + + /// Demotes reserved peer to non-reserved. Does not disconnect the peer. + /// + /// Has no effect if the node was not a reserved peer. + pub fn remove_reserved_peer(&self, peer_id: PeerId) { + let _ = self.actions_tx.unbounded_send(Action::RemoveReservedPeer(peer_id)); + } + + /// Set reserved peers to the new set. + pub fn set_reserved_peers(&self, peer_ids: HashSet) { + let _ = self.actions_tx.unbounded_send(Action::SetReservedPeers(peer_ids)); + } + + /// Sets whether or not [`ProtocolController`] only has connections with nodes marked + /// as reserved for the given set. + pub fn set_reserved_only(&self, reserved: bool) { + let _ = self.actions_tx.unbounded_send(Action::SetReservedOnly(reserved)); + } + + /// Disconnect peer. You should remove the `PeerId` from the `PeerStore` first + /// to not connect to the peer again during the next slot allocation. + pub fn disconnect_peer(&self, peer_id: PeerId) { + let _ = self.actions_tx.unbounded_send(Action::DisconnectPeer(peer_id)); + } + + /// Get the list of reserved peers. + pub fn reserved_peers(&self, pending_response: oneshot::Sender>) { + let _ = self.actions_tx.unbounded_send(Action::GetReservedPeers(pending_response)); + } + + /// Notify about incoming connection. [`ProtocolController`] will either accept or reject it. + pub fn incoming_connection(&self, peer_id: PeerId, incoming_index: IncomingIndex) { + let _ = self + .events_tx + .unbounded_send(Event::IncomingConnection(peer_id, incoming_index)); + } + + /// Notify that connection was dropped (either refused or disconnected). + pub fn dropped(&self, peer_id: PeerId) { + let _ = self.events_tx.unbounded_send(Event::Dropped(peer_id)); + } +} + +/// Direction of a connection +#[derive(Clone, Copy, Debug)] +enum Direction { + Inbound, + Outbound, +} + +/// Status of a connection with a peer. +#[derive(Clone, Debug)] +enum PeerState { + /// We are connected to the peer. + Connected(Direction), + /// We are not connected. + NotConnected, +} + +impl PeerState { + /// Returns true if we are connected with the node. + fn is_connected(&self) -> bool { + matches!(self, PeerState::Connected(_)) + } +} + +impl Default for PeerState { + fn default() -> PeerState { + PeerState::NotConnected + } +} + +/// Side of [`ProtocolHandle`] responsible for all the logic. Currently all instances are +/// owned by [`crate::Peerset`], but they should eventually be moved to corresponding protocols. +#[derive(Debug)] +pub struct ProtocolController { + /// Set id to use when sending connect/drop requests to `Notifications`. + // Will likely be replaced by `ProtocolName` in the future. + set_id: SetId, + /// Receiver for outer API messages from [`ProtocolHandle`]. + actions_rx: TracingUnboundedReceiver, + /// Receiver for connection events from `Notifications` sent via [`ProtocolHandle`]. + events_rx: TracingUnboundedReceiver, + /// Number of occupied slots for incoming connections (not counting reserved nodes). + num_in: u32, + /// Number of occupied slots for outgoing connections (not counting reserved nodes). + num_out: u32, + /// Maximum number of slots for incoming connections (not counting reserved nodes). + max_in: u32, + /// Maximum number of slots for outgoing connections (not counting reserved nodes). + max_out: u32, + /// Connected regular nodes. + nodes: HashMap, + /// Reserved nodes. Should be always connected and do not occupy peer slots. + reserved_nodes: HashMap, + /// Connect only to reserved nodes. + reserved_only: bool, + /// Next time to allocate slots. This is done once per second. + next_periodic_alloc_slots: Instant, + /// Outgoing channel for messages to `Notifications`. + to_notifications: TracingUnboundedSender, + /// `PeerStore` handle for checking peer reputation values and getting connection candidates + /// with highest reputation. + peer_store: Box, +} + +impl ProtocolController { + /// Construct new [`ProtocolController`]. + pub fn new( + set_id: SetId, + config: SetConfig, + to_notifications: TracingUnboundedSender, + peer_store: Box, + ) -> (ProtocolHandle, ProtocolController) { + let (actions_tx, actions_rx) = tracing_unbounded("mpsc_api_protocol", 10_000); + let (events_tx, events_rx) = tracing_unbounded("mpsc_notifications_protocol", 10_000); + let handle = ProtocolHandle { actions_tx, events_tx }; + peer_store.register_protocol(handle.clone()); + let reserved_nodes = + config.reserved_nodes.iter().map(|p| (*p, PeerState::NotConnected)).collect(); + let controller = ProtocolController { + set_id, + actions_rx, + events_rx, + num_in: 0, + num_out: 0, + max_in: config.in_peers, + max_out: config.out_peers, + nodes: HashMap::new(), + reserved_nodes, + reserved_only: config.reserved_only, + next_periodic_alloc_slots: Instant::now(), + to_notifications, + peer_store, + }; + (handle, controller) + } + + /// Drive [`ProtocolController`]. This function returns when all instances of + /// [`ProtocolHandle`] are dropped. + pub async fn run(mut self) { + while self.next_action().await {} + } + + /// Perform one action. Returns `true` if it should be called again. + /// + /// Intended for tests only. Use `run` for driving [`ProtocolController`]. + pub async fn next_action(&mut self) -> bool { + let either = loop { + let mut next_alloc_slots = Delay::new_at(self.next_periodic_alloc_slots).fuse(); + + // See the module doc for why we use `select_biased!`. + futures::select_biased! { + event = self.events_rx.next() => match event { + Some(event) => break Either::Left(event), + None => return false, + }, + action = self.actions_rx.next() => match action { + Some(action) => break Either::Right(action), + None => return false, + }, + _ = next_alloc_slots => { + self.alloc_slots(); + self.next_periodic_alloc_slots = Instant::now() + Duration::new(1, 0); + }, + } + }; + + match either { + Either::Left(event) => self.process_event(event), + Either::Right(action) => self.process_action(action), + } + + true + } + + /// Process connection event. + fn process_event(&mut self, event: Event) { + match event { + Event::IncomingConnection(peer_id, index) => + self.on_incoming_connection(peer_id, index), + Event::Dropped(peer_id) => self.on_peer_dropped(peer_id), + } + } + + /// Process action command. + fn process_action(&mut self, action: Action) { + match action { + Action::AddReservedPeer(peer_id) => self.on_add_reserved_peer(peer_id), + Action::RemoveReservedPeer(peer_id) => self.on_remove_reserved_peer(peer_id), + Action::SetReservedPeers(peer_ids) => self.on_set_reserved_peers(peer_ids), + Action::SetReservedOnly(reserved_only) => self.on_set_reserved_only(reserved_only), + Action::DisconnectPeer(peer_id) => self.on_disconnect_peer(peer_id), + Action::GetReservedPeers(pending_response) => + self.on_get_reserved_peers(pending_response), + } + } + + /// Send "accept" message to `Notifications`. + fn accept_connection(&mut self, incoming_index: IncomingIndex) { + trace!( + target: LOG_TARGET, + "Accepting {:?} on {:?} ({}/{} num_in/max_in).", + incoming_index, + self.set_id, + self.num_in, + self.max_in, + ); + + let _ = self.to_notifications.unbounded_send(Message::Accept(incoming_index)); + } + + /// Send "reject" message to `Notifications`. + fn reject_connection(&mut self, incoming_index: IncomingIndex) { + trace!( + target: LOG_TARGET, + "Rejecting {:?} on {:?} ({}/{} num_in/max_in).", + incoming_index, + self.set_id, + self.num_in, + self.max_in, + ); + + let _ = self.to_notifications.unbounded_send(Message::Reject(incoming_index)); + } + + /// Send "connect" message to `Notifications`. + fn start_connection(&mut self, peer_id: PeerId) { + trace!( + target: LOG_TARGET, + "Connecting to {} on {:?} ({}/{} num_out/max_out).", + peer_id, + self.set_id, + self.num_out, + self.max_out, + ); + + let _ = self + .to_notifications + .unbounded_send(Message::Connect { set_id: self.set_id, peer_id }); + } + + /// Send "drop" message to `Notifications`. + fn drop_connection(&mut self, peer_id: PeerId) { + trace!( + target: LOG_TARGET, + "Dropping {} on {:?} ({}/{} num_in/max_in, {}/{} num_out/max_out).", + peer_id, + self.set_id, + self.num_in, + self.max_in, + self.num_out, + self.max_out, + ); + + let _ = self + .to_notifications + .unbounded_send(Message::Drop { set_id: self.set_id, peer_id }); + } + + /// Report peer disconnect event to `PeerStore` for it to update peer's reputation accordingly. + /// Should only be called if the remote node disconnected us, not the other way around. + fn report_disconnect(&mut self, peer_id: PeerId) { + self.peer_store.report_disconnect(peer_id); + } + + /// Ask `Peerset` if the peer has a reputation value not sufficent for connection with it. + fn is_banned(&self, peer_id: &PeerId) -> bool { + self.peer_store.is_banned(peer_id) + } + + /// Add the peer to the set of reserved peers. [`ProtocolController`] will try to always + /// maintain connections with such peers. + fn on_add_reserved_peer(&mut self, peer_id: PeerId) { + if self.reserved_nodes.contains_key(&peer_id) { + warn!( + target: LOG_TARGET, + "Trying to add an already reserved node as reserved: {peer_id}.", + ); + return + } + + // Get the peer out of non-reserved peers if it's there. + let state = match self.nodes.remove(&peer_id) { + Some(direction) => { + trace!( + target: LOG_TARGET, + "Marking previously connected node {peer_id} ({direction:?}) as reserved.", + ); + PeerState::Connected(direction) + }, + None => { + trace!(target: LOG_TARGET, "Adding reserved node {peer_id}."); + PeerState::NotConnected + }, + }; + + self.reserved_nodes.insert(peer_id, state.clone()); + + // Discount occupied slots or connect to the node. + match state { + PeerState::Connected(Direction::Inbound) => self.num_in -= 1, + PeerState::Connected(Direction::Outbound) => self.num_out -= 1, + PeerState::NotConnected => self.alloc_slots(), + } + } + + /// Remove the peer from the set of reserved peers. The peer is moved to the set of regular + /// nodes. + fn on_remove_reserved_peer(&mut self, peer_id: PeerId) { + let state = match self.reserved_nodes.remove(&peer_id) { + Some(state) => state, + None => { + warn!(target: LOG_TARGET, "Trying to remove unknown reserved node: {peer_id}."); + return + }, + }; + + if let PeerState::Connected(direction) = state { + if self.reserved_only { + // Disconnect the node. + trace!( + target: LOG_TARGET, + "Disconnecting previously reserved node {} ({:?}) on {:?}.", + peer_id, + direction, + self.set_id, + ); + self.drop_connection(peer_id); + } else { + // Count connections as of regular node. + trace!( + target: LOG_TARGET, + "Making a connected reserved node {} ({:?}) on {:?} a regular one.", + peer_id, + direction, + self.set_id, + ); + + match direction { + Direction::Inbound => self.num_in += 1, + Direction::Outbound => self.num_out += 1, + } + + // Put the node into the list of regular nodes. + let prev = self.nodes.insert(peer_id, direction); + assert!(prev.is_none(), "Corrupted state: reserved node was also non-reserved."); + } + } else { + trace!(target: LOG_TARGET, "Removed disconnected reserved node {peer_id}."); + } + } + + /// Replace the set of reserved peers. + fn on_set_reserved_peers(&mut self, peer_ids: HashSet) { + // Determine the difference between the current group and the new list. + let current = self.reserved_nodes.keys().cloned().collect(); + let to_insert = peer_ids.difference(¤t).cloned().collect::>(); + let to_remove = current.difference(&peer_ids).cloned().collect::>(); + + for node in to_insert { + self.on_add_reserved_peer(node); + } + + for node in to_remove { + self.on_remove_reserved_peer(node); + } + } + + /// Change "reserved only" flag. In "reserved only" mode we connect and accept connections to + /// reserved nodes only. + fn on_set_reserved_only(&mut self, reserved_only: bool) { + trace!(target: LOG_TARGET, "Set reserved only: {reserved_only}"); + + self.reserved_only = reserved_only; + + if !reserved_only { + return self.alloc_slots() + } + + // Disconnect all non-reserved peers. + self.nodes + .iter() + .map(|(k, v)| (*k, *v)) + .collect::>() + .iter() + .for_each(|(peer_id, direction)| { + // Update counters in the loop for `drop_connection` to report the correct number. + match direction { + Direction::Inbound => self.num_in -= 1, + Direction::Outbound => self.num_out -= 1, + } + self.drop_connection(*peer_id) + }); + self.nodes.clear(); + } + + /// Get the list of reserved peers. + fn on_get_reserved_peers(&self, pending_response: oneshot::Sender>) { + let _ = pending_response.send(self.reserved_nodes.keys().cloned().collect()); + } + + /// Disconnect the peer. + fn on_disconnect_peer(&mut self, peer_id: PeerId) { + // Don't do anything if the node is reserved. + if self.reserved_nodes.contains_key(&peer_id) { + warn!( + target: LOG_TARGET, + "Ignoring request to disconnect reserved peer {} from {:?}.", peer_id, self.set_id, + ); + return + } + + match self.nodes.remove(&peer_id) { + Some(direction) => { + trace!(target: LOG_TARGET, "Disconnecting peer {peer_id} ({direction:?})."); + match direction { + Direction::Inbound => self.num_in -= 1, + Direction::Outbound => self.num_out -= 1, + } + self.drop_connection(peer_id); + }, + None => { + warn!( + target: LOG_TARGET, + "Trying to disconnect unknown peer {} from {:?}.", peer_id, self.set_id, + ); + }, + } + } + + /// Indicate that we received an incoming connection. Must be answered either with + /// a corresponding `Accept` or `Reject`, except if we were already connected to this peer. + /// + /// Note that this mechanism is orthogonal to `Connect`/`Drop`. Accepting an incoming + /// connection implicitly means `Connect`, but incoming connections aren't cancelled by + /// `dropped`. + // Implementation note: because of concurrency issues, `ProtocolController` has an imperfect + // view of the peers' states, and may issue commands for a peer after `Notifications` received + // an incoming request for that peer. In this case, `Notifications` ignores all the commands + // until it receives a response for the incoming request to `ProtocolController`, so we must + // ensure we handle this incoming request correctly. + fn on_incoming_connection(&mut self, peer_id: PeerId, incoming_index: IncomingIndex) { + trace!(target: LOG_TARGET, "Incoming connection from peer {peer_id} ({incoming_index:?}).",); + + if self.reserved_only && !self.reserved_nodes.contains_key(&peer_id) { + self.reject_connection(incoming_index); + return + } + + // Check if the node is reserved first. + if let Some(state) = self.reserved_nodes.get_mut(&peer_id) { + match state { + PeerState::Connected(ref mut direction) => { + // We are accepting an incoming connection, so ensure the direction is inbound. + // (See the implementation note above.) + *direction = Direction::Inbound; + self.accept_connection(incoming_index); + }, + PeerState::NotConnected => + if self.peer_store.is_banned(&peer_id) { + self.reject_connection(incoming_index); + } else { + *state = PeerState::Connected(Direction::Inbound); + self.accept_connection(incoming_index); + }, + } + return + } + + // If we're already connected, pretend we are not connected and decide on the node again. + // (See the note above.) + if let Some(direction) = self.nodes.remove(&peer_id) { + trace!( + target: LOG_TARGET, + "Handling incoming connection from peer {} we think we already connected as {:?}.", + peer_id, + direction, + ); + match direction { + Direction::Inbound => self.num_in -= 1, + Direction::Outbound => self.num_out -= 1, + } + } + + if self.num_in >= self.max_in { + self.reject_connection(incoming_index); + return + } + + if self.is_banned(&peer_id) { + self.reject_connection(incoming_index); + return + } + + self.num_in += 1; + self.nodes.insert(peer_id, Direction::Inbound); + self.accept_connection(incoming_index); + } + + /// Indicate that a connection with the peer was dropped. + fn on_peer_dropped(&mut self, peer_id: PeerId) { + self.on_peer_dropped_inner(peer_id).unwrap_or_else(|peer_id| { + // We do not assert here, because due to asynchronous nature of communication + // between `ProtocolController` and `Notifications` we can receive `Action::Dropped` + // for a peer we already disconnected ourself. + trace!( + target: LOG_TARGET, + "Received `Action::Dropped` for not connected peer {} on {:?}.", + peer_id, + self.set_id, + ) + }); + } + + /// Indicate that a connection with the peer was dropped. + /// Returns `Err(PeerId)` if the peer wasn't connected or is not known to us. + fn on_peer_dropped_inner(&mut self, peer_id: PeerId) -> Result<(), PeerId> { + if self.drop_reserved_peer(&peer_id)? || self.drop_regular_peer(&peer_id) { + // The peer found and disconnected. + self.report_disconnect(peer_id); + Ok(()) + } else { + // The peer was not found in neither regular or reserved lists. + Err(peer_id) + } + } + + /// Try dropping the peer as a reserved peer. Return `Ok(true)` if the peer was found and + /// disconnected, `Ok(false)` if it wasn't found, `Err(PeerId)`, if the peer found, but not in + /// connected state. + fn drop_reserved_peer(&mut self, peer_id: &PeerId) -> Result { + let Some(state) = self.reserved_nodes.get_mut(peer_id) else { + return Ok(false) + }; + + if let PeerState::Connected(direction) = state { + trace!(target: LOG_TARGET, "Reserved peer {peer_id} ({direction:?}) dropped."); + *state = PeerState::NotConnected; + Ok(true) + } else { + Err(*peer_id) + } + } + + /// Try dropping the peer as a regular peer. Return `true` if the peer was found and + /// disconnected, `false` if it wasn't found. + fn drop_regular_peer(&mut self, peer_id: &PeerId) -> bool { + let Some(direction) = self.nodes.remove(peer_id) else { + return false + }; + + trace!(target: LOG_TARGET, "Peer {peer_id} ({direction:?}) dropped."); + + match direction { + Direction::Inbound => self.num_in -= 1, + Direction::Outbound => self.num_out -= 1, + } + + true + } + + /// Initiate outgoing connections trying to connect all reserved nodes and fill in all outgoing + /// slots. + fn alloc_slots(&mut self) { + // Try connecting to reserved nodes first, ignoring nodes with outstanding events/actions. + self.reserved_nodes + .iter_mut() + .filter_map(|(peer_id, state)| { + (!state.is_connected() && !self.peer_store.is_banned(peer_id)).then(|| { + *state = PeerState::Connected(Direction::Outbound); + peer_id + }) + }) + .cloned() + .collect::>() + .into_iter() + .for_each(|peer_id| { + self.start_connection(peer_id); + }); + + // Nothing more to do if we're in reserved-only mode or don't have slots available. + if self.reserved_only || self.num_out >= self.max_out { + return + } + + // Fill available slots. + let available_slots = (self.max_out - self.num_out).saturated_into(); + + // Ignore reserved nodes (connected above), already connected nodes, and nodes with + // outstanding events/actions. + let ignored = self + .reserved_nodes + .keys() + .collect::>() + .union(&self.nodes.keys().collect::>()) + .cloned() + .collect(); + + let candidates = self + .peer_store + .outgoing_candidates(available_slots, ignored) + .into_iter() + .filter_map(|peer_id| { + (!self.reserved_nodes.contains_key(&peer_id) && !self.nodes.contains_key(&peer_id)) + .then_some(peer_id) + .or_else(|| { + error!( + target: LOG_TARGET, + "`PeerStore` returned a node we asked to ignore: {peer_id}.", + ); + debug_assert!(false, "`PeerStore` returned a node we asked to ignore."); + None + }) + }) + .collect::>(); + + if candidates.len() > available_slots { + error!( + target: LOG_TARGET, + "`PeerStore` returned more nodes than there are slots available.", + ); + debug_assert!(false, "`PeerStore` returned more nodes than there are slots available."); + } + + candidates.into_iter().take(available_slots).for_each(|peer_id| { + self.num_out += 1; + self.nodes.insert(peer_id, Direction::Outbound); + self.start_connection(peer_id); + }) + } +} + +#[cfg(test)] +mod tests { + use super::{Direction, PeerState, ProtocolController, ProtocolHandle}; + use crate::{ + peer_store::PeerStoreProvider, IncomingIndex, Message, ReputationChange, SetConfig, SetId, + }; + use libp2p_identity::PeerId; + use sc_utils::mpsc::{tracing_unbounded, TryRecvError}; + use std::collections::HashSet; + + mockall::mock! { + #[derive(Debug)] + pub PeerStoreHandle {} + + impl PeerStoreProvider for PeerStoreHandle { + fn is_banned(&self, peer_id: &PeerId) -> bool; + fn register_protocol(&self, protocol_handle: ProtocolHandle); + fn report_disconnect(&mut self, peer_id: PeerId); + fn report_peer(&mut self, peer_id: PeerId, change: ReputationChange); + fn peer_reputation(&self, peer_id: &PeerId) -> i32; + fn outgoing_candidates<'a>(&self, count: usize, ignored: HashSet<&'a PeerId>) -> Vec; + } + } + + #[test] + fn reserved_nodes_are_connected_dropped_and_accepted() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + // Add first reserved node via config. + let config = SetConfig { + in_peers: 0, + out_peers: 0, + bootnodes: Vec::new(), + reserved_nodes: std::iter::once(reserved1).collect(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(4).return_const(false); + peer_store.expect_report_disconnect().times(2).return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Add second reserved node at runtime (this currently calls `alloc_slots` internally). + controller.on_add_reserved_peer(reserved2); + + // Initiate connections (currently, `alloc_slots` is also called internally in + // `on_add_reserved_peer` above). + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + + // Reserved peers do not occupy slots. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Drop connections to be able to accept reserved nodes. + controller.on_peer_dropped(reserved1); + controller.on_peer_dropped(reserved2); + + // Incoming connection from `reserved1`. + let incoming1 = IncomingIndex(1); + controller.on_incoming_connection(reserved1, incoming1); + assert_eq!(rx.try_recv().unwrap(), Message::Accept(incoming1)); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // Incoming connection from `reserved2`. + let incoming2 = IncomingIndex(2); + controller.on_incoming_connection(reserved2, incoming2); + assert_eq!(rx.try_recv().unwrap(), Message::Accept(incoming2)); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // Reserved peers do not occupy slots. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn banned_reserved_nodes_are_not_connected_and_not_accepted() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + // Add first reserved node via config. + let config = SetConfig { + in_peers: 0, + out_peers: 0, + bootnodes: Vec::new(), + reserved_nodes: std::iter::once(reserved1).collect(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(6).return_const(true); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Add second reserved node at runtime (this currently calls `alloc_slots` internally). + controller.on_add_reserved_peer(reserved2); + + // Initiate connections. + controller.alloc_slots(); + + // No slots occupied. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // No commands are generated. + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // Incoming connection from `reserved1`. + let incoming1 = IncomingIndex(1); + controller.on_incoming_connection(reserved1, incoming1); + assert_eq!(rx.try_recv().unwrap(), Message::Reject(incoming1)); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // Incoming connection from `reserved2`. + let incoming2 = IncomingIndex(2); + controller.on_incoming_connection(reserved2, incoming2); + assert_eq!(rx.try_recv().unwrap(), Message::Reject(incoming2)); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // No slots occupied. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn we_try_to_reconnect_to_dropped_reserved_nodes() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + // Add first reserved node via config. + let config = SetConfig { + in_peers: 0, + out_peers: 0, + bootnodes: Vec::new(), + reserved_nodes: std::iter::once(reserved1).collect(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(4).return_const(false); + peer_store.expect_report_disconnect().times(2).return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Add second reserved node at runtime (this calls `alloc_slots` internally). + controller.on_add_reserved_peer(reserved2); + + // Initiate connections (actually redundant, see previous comment). + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + + // Drop both reserved nodes. + controller.on_peer_dropped(reserved1); + controller.on_peer_dropped(reserved2); + + // Initiate connections. + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + + // No slots occupied. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn nodes_supplied_by_peer_store_are_connected() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + let candidates = vec![peer1, peer2]; + + let config = SetConfig { + in_peers: 0, + // Less slots than candidates. + out_peers: 2, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_outgoing_candidates().once().return_const(candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Initiate connections. + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + // Only first two peers are connected (we only have 2 slots). + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer2 })); + + // Outgoing slots occupied. + assert_eq!(controller.num_out, 2); + assert_eq!(controller.num_in, 0); + + // No more nodes are connected. + controller.alloc_slots(); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // No more slots occupied. + assert_eq!(controller.num_out, 2); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn both_reserved_nodes_and_nodes_supplied_by_peer_store_are_connected() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + let regular1 = PeerId::random(); + let regular2 = PeerId::random(); + let outgoing_candidates = vec![regular1, regular2]; + let reserved_nodes = [reserved1, reserved2].iter().cloned().collect(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes, + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(2).return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Initiate connections. + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 4); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: regular1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: regular2 })); + assert_eq!(controller.num_out, 2); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn if_slots_are_freed_we_try_to_allocate_them_again() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + let peer3 = PeerId::random(); + let candidates1 = vec![peer1, peer2]; + let candidates2 = vec![peer3]; + + let config = SetConfig { + in_peers: 0, + // Less slots than candidates. + out_peers: 2, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_outgoing_candidates().once().return_const(candidates1); + peer_store.expect_outgoing_candidates().once().return_const(candidates2); + peer_store.expect_report_disconnect().times(2).return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Initiate connections. + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + // Only first two peers are connected (we only have 2 slots). + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer2 })); + + // Outgoing slots occupied. + assert_eq!(controller.num_out, 2); + assert_eq!(controller.num_in, 0); + + // No more nodes are connected. + controller.alloc_slots(); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // No more slots occupied. + assert_eq!(controller.num_out, 2); + assert_eq!(controller.num_in, 0); + + // Drop peers. + controller.on_peer_dropped(peer1); + controller.on_peer_dropped(peer2); + + // Slots are freed. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Initiate connections. + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + // Peers are connected. + assert_eq!(messages.len(), 1); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer3 })); + + // Outgoing slots occupied. + assert_eq!(controller.num_out, 1); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn in_reserved_only_mode_no_peers_are_requested_from_peer_store_and_connected() { + let config = SetConfig { + in_peers: 0, + // Make sure we have slots available. + out_peers: 2, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Initiate connections. + controller.alloc_slots(); + + // No nodes are connected. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + } + + #[test] + fn in_reserved_only_mode_no_regular_peers_are_accepted() { + let config = SetConfig { + // Make sure we have slots available. + in_peers: 2, + out_peers: 0, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + let peer = PeerId::random(); + let incoming_index = IncomingIndex(1); + controller.on_incoming_connection(peer, incoming_index); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + // Peer is rejected. + assert_eq!(messages.len(), 1); + assert!(messages.contains(&Message::Reject(incoming_index))); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn disabling_reserved_only_mode_allows_to_connect_to_peers() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + let candidates = vec![peer1, peer2]; + + let config = SetConfig { + in_peers: 0, + // Make sure we have slots available. + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_outgoing_candidates().once().return_const(candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Initiate connections. + controller.alloc_slots(); + + // No nodes are connected. + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // Disable reserved-only mode (this also connects to peers). + controller.on_set_reserved_only(false); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer2 })); + assert_eq!(controller.num_out, 2); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn enabling_reserved_only_mode_disconnects_regular_peers() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + let regular1 = PeerId::random(); + let regular2 = PeerId::random(); + let outgoing_candidates = vec![regular1]; + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: [reserved1, reserved2].iter().cloned().collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(3).return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Connect `regular1` as outbound. + controller.alloc_slots(); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 3); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: regular1 })); + assert_eq!(controller.num_out, 1); + assert_eq!(controller.num_in, 0); + + // Connect `regular2` as inbound. + let incoming_index = IncomingIndex(1); + controller.on_incoming_connection(regular2, incoming_index); + assert_eq!(rx.try_recv().unwrap(), Message::Accept(incoming_index)); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.num_out, 1); + assert_eq!(controller.num_in, 1); + + // Switch to reserved-only mode. + controller.on_set_reserved_only(true); + + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Drop { set_id: SetId(0), peer_id: regular1 })); + assert!(messages.contains(&Message::Drop { set_id: SetId(0), peer_id: regular2 })); + assert_eq!(controller.nodes.len(), 0); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn removed_disconnected_reserved_node_is_forgotten() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: [reserved1, reserved2].iter().cloned().collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert_eq!(controller.reserved_nodes.len(), 2); + assert_eq!(controller.nodes.len(), 0); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + controller.on_remove_reserved_peer(reserved1); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.reserved_nodes.len(), 1); + assert!(!controller.reserved_nodes.contains_key(&reserved1)); + assert_eq!(controller.nodes.len(), 0); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + } + + #[test] + fn removed_connected_reserved_node_is_disconnected_in_reserved_only_mode() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: [reserved1, reserved2].iter().cloned().collect(), + reserved_only: true, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(2).return_const(false); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Initiate connections. + controller.alloc_slots(); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved1 })); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + assert_eq!(controller.reserved_nodes.len(), 2); + assert!(controller.reserved_nodes.contains_key(&reserved1)); + assert!(controller.reserved_nodes.contains_key(&reserved2)); + assert!(controller.nodes.is_empty()); + + // Remove reserved node + controller.on_remove_reserved_peer(reserved1); + assert_eq!(rx.try_recv().unwrap(), Message::Drop { set_id: SetId(0), peer_id: reserved1 }); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.reserved_nodes.len(), 1); + assert!(controller.reserved_nodes.contains_key(&reserved2)); + assert!(controller.nodes.is_empty()); + } + + #[test] + fn removed_connected_reserved_nodes_become_regular_in_non_reserved_mode() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: [peer1, peer2].iter().cloned().collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(2).return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(Vec::new()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `peer1` as inbound, `peer2` as outbound. + controller.on_incoming_connection(peer1, IncomingIndex(1)); + controller.alloc_slots(); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Accept(IncomingIndex(1)))); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer2 })); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Remove reserved nodes (and make them regular) + controller.on_remove_reserved_peer(peer1); + controller.on_remove_reserved_peer(peer2); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.nodes.len(), 2); + assert!(matches!(controller.nodes.get(&peer1), Some(Direction::Inbound))); + assert!(matches!(controller.nodes.get(&peer2), Some(Direction::Outbound))); + assert_eq!(controller.num_out, 1); + assert_eq!(controller.num_in, 1); + } + + #[test] + fn regular_nodes_stop_occupying_slots_when_become_reserved() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + let outgoing_candidates = vec![peer1]; + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `peer1` as outbound & `peer2` as inbound. + controller.alloc_slots(); + controller.on_incoming_connection(peer2, IncomingIndex(1)); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer1 })); + assert!(messages.contains(&Message::Accept(IncomingIndex(1)))); + assert_eq!(controller.num_in, 1); + assert_eq!(controller.num_out, 1); + + controller.on_add_reserved_peer(peer1); + controller.on_add_reserved_peer(peer2); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.num_in, 0); + assert_eq!(controller.num_out, 0); + } + + #[test] + fn disconnecting_regular_peers_work() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + let outgoing_candidates = vec![peer1]; + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `peer1` as outbound & `peer2` as inbound. + controller.alloc_slots(); + controller.on_incoming_connection(peer2, IncomingIndex(1)); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer1 })); + assert!(messages.contains(&Message::Accept(IncomingIndex(1)))); + assert_eq!(controller.nodes.len(), 2); + assert!(matches!(controller.nodes.get(&peer1), Some(Direction::Outbound))); + assert!(matches!(controller.nodes.get(&peer2), Some(Direction::Inbound))); + assert_eq!(controller.num_in, 1); + assert_eq!(controller.num_out, 1); + + controller.on_disconnect_peer(peer1); + assert_eq!(rx.try_recv().unwrap(), Message::Drop { set_id: SetId(0), peer_id: peer1 }); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.nodes.len(), 1); + assert!(!controller.nodes.contains_key(&peer1)); + assert_eq!(controller.num_in, 1); + assert_eq!(controller.num_out, 0); + + controller.on_disconnect_peer(peer2); + assert_eq!(rx.try_recv().unwrap(), Message::Drop { set_id: SetId(0), peer_id: peer2 }); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.nodes.len(), 0); + assert_eq!(controller.num_in, 0); + assert_eq!(controller.num_out, 0); + } + + #[test] + fn disconnecting_reserved_peers_is_a_noop() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: [reserved1, reserved2].iter().cloned().collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(2).return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(Vec::new()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `reserved1` as inbound & `reserved2` as outbound. + controller.on_incoming_connection(reserved1, IncomingIndex(1)); + controller.alloc_slots(); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Accept(IncomingIndex(1)))); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + assert!(matches!( + controller.reserved_nodes.get(&reserved1), + Some(PeerState::Connected(Direction::Inbound)) + )); + assert!(matches!( + controller.reserved_nodes.get(&reserved2), + Some(PeerState::Connected(Direction::Outbound)) + )); + + controller.on_disconnect_peer(reserved1); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!( + controller.reserved_nodes.get(&reserved1), + Some(PeerState::Connected(Direction::Inbound)) + )); + + controller.on_disconnect_peer(reserved2); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!( + controller.reserved_nodes.get(&reserved2), + Some(PeerState::Connected(Direction::Outbound)) + )); + } + + #[test] + fn dropping_regular_peers_work() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + let outgoing_candidates = vec![peer1]; + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + peer_store.expect_report_disconnect().times(2).return_const(()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `peer1` as outbound & `peer2` as inbound. + controller.alloc_slots(); + controller.on_incoming_connection(peer2, IncomingIndex(1)); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: peer1 })); + assert!(messages.contains(&Message::Accept(IncomingIndex(1)))); + assert_eq!(controller.nodes.len(), 2); + assert!(matches!(controller.nodes.get(&peer1), Some(Direction::Outbound))); + assert!(matches!(controller.nodes.get(&peer2), Some(Direction::Inbound))); + assert_eq!(controller.num_in, 1); + assert_eq!(controller.num_out, 1); + + controller.on_peer_dropped(peer1); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.nodes.len(), 1); + assert!(!controller.nodes.contains_key(&peer1)); + assert_eq!(controller.num_in, 1); + assert_eq!(controller.num_out, 0); + + controller.on_peer_dropped(peer2); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert_eq!(controller.nodes.len(), 0); + assert_eq!(controller.num_in, 0); + assert_eq!(controller.num_out, 0); + } + + #[test] + fn incoming_request_for_connected_reserved_node_switches_it_to_inbound() { + let reserved1 = PeerId::random(); + let reserved2 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: [reserved1, reserved2].iter().cloned().collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(2).return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(Vec::new()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `reserved1` as inbound & `reserved2` as outbound. + controller.on_incoming_connection(reserved1, IncomingIndex(1)); + controller.alloc_slots(); + let mut messages = Vec::new(); + while let Some(message) = rx.try_recv().ok() { + messages.push(message); + } + assert_eq!(messages.len(), 2); + assert!(messages.contains(&Message::Accept(IncomingIndex(1)))); + assert!(messages.contains(&Message::Connect { set_id: SetId(0), peer_id: reserved2 })); + assert!(matches!( + controller.reserved_nodes.get(&reserved1), + Some(PeerState::Connected(Direction::Inbound)) + )); + assert!(matches!( + controller.reserved_nodes.get(&reserved2), + Some(PeerState::Connected(Direction::Outbound)) + )); + + // Incoming request for `reserved1`. + controller.on_incoming_connection(reserved1, IncomingIndex(2)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(2))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!( + controller.reserved_nodes.get(&reserved1), + Some(PeerState::Connected(Direction::Inbound)) + )); + + // Incoming request for `reserved2`. + controller.on_incoming_connection(reserved2, IncomingIndex(3)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(3))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!( + controller.reserved_nodes.get(&reserved2), + Some(PeerState::Connected(Direction::Inbound)) + )); + } + + #[test] + fn incoming_request_for_connected_regular_node_switches_it_to_inbound() { + let regular1 = PeerId::random(); + let regular2 = PeerId::random(); + let outgoing_candidates = vec![regular1]; + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().times(3).return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Connect `regular1` as outbound. + controller.alloc_slots(); + assert_eq!( + rx.try_recv().ok().unwrap(), + Message::Connect { set_id: SetId(0), peer_id: regular1 } + ); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular1).unwrap(), Direction::Outbound,)); + + // Connect `regular2` as inbound. + controller.on_incoming_connection(regular2, IncomingIndex(0)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(0))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular2).unwrap(), Direction::Inbound,)); + + // Incoming request for `regular1`. + controller.on_incoming_connection(regular1, IncomingIndex(1)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(1))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular1).unwrap(), Direction::Inbound,)); + + // Incoming request for `regular2`. + controller.on_incoming_connection(regular2, IncomingIndex(2)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(2))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular2).unwrap(), Direction::Inbound,)); + } + + #[test] + fn incoming_request_for_connected_node_is_rejected_if_its_banned() { + let regular1 = PeerId::random(); + let regular2 = PeerId::random(); + let outgoing_candidates = vec![regular1]; + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(false); + peer_store.expect_is_banned().times(2).return_const(true); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Connect `regular1` as outbound. + controller.alloc_slots(); + assert_eq!( + rx.try_recv().ok().unwrap(), + Message::Connect { set_id: SetId(0), peer_id: regular1 } + ); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular1).unwrap(), Direction::Outbound,)); + + // Connect `regular2` as inbound. + controller.on_incoming_connection(regular2, IncomingIndex(0)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(0))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular2).unwrap(), Direction::Inbound,)); + + // Incoming request for `regular1`. + controller.on_incoming_connection(regular1, IncomingIndex(1)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Reject(IncomingIndex(1))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(!controller.nodes.contains_key(®ular1)); + + // Incoming request for `regular2`. + controller.on_incoming_connection(regular2, IncomingIndex(2)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Reject(IncomingIndex(2))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(!controller.nodes.contains_key(®ular2)); + } + + #[test] + fn incoming_request_for_connected_node_is_rejected_if_no_slots_available() { + let regular1 = PeerId::random(); + let regular2 = PeerId::random(); + let outgoing_candidates = vec![regular1]; + + let config = SetConfig { + in_peers: 1, + out_peers: 1, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(false); + peer_store.expect_outgoing_candidates().once().return_const(outgoing_candidates); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert_eq!(controller.num_out, 0); + assert_eq!(controller.num_in, 0); + + // Connect `regular1` as outbound. + controller.alloc_slots(); + assert_eq!( + rx.try_recv().ok().unwrap(), + Message::Connect { set_id: SetId(0), peer_id: regular1 } + ); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular1).unwrap(), Direction::Outbound,)); + + // Connect `regular2` as inbound. + controller.on_incoming_connection(regular2, IncomingIndex(0)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Accept(IncomingIndex(0))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(matches!(controller.nodes.get(®ular2).unwrap(), Direction::Inbound,)); + + controller.max_in = 0; + + // Incoming request for `regular1`. + controller.on_incoming_connection(regular1, IncomingIndex(1)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Reject(IncomingIndex(1))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(!controller.nodes.contains_key(®ular1)); + + // Incoming request for `regular2`. + controller.on_incoming_connection(regular2, IncomingIndex(2)); + assert_eq!(rx.try_recv().ok().unwrap(), Message::Reject(IncomingIndex(2))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + assert!(!controller.nodes.contains_key(®ular2)); + } + + #[test] + fn incoming_peers_that_exceed_slots_are_rejected() { + let peer1 = PeerId::random(); + let peer2 = PeerId::random(); + + let config = SetConfig { + in_peers: 1, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(false); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Connect `peer1` as inbound. + controller.on_incoming_connection(peer1, IncomingIndex(1)); + assert_eq!(rx.try_recv().unwrap(), Message::Accept(IncomingIndex(1))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + + // Incoming requests for `peer2`. + controller.on_incoming_connection(peer2, IncomingIndex(2)); + assert_eq!(rx.try_recv().unwrap(), Message::Reject(IncomingIndex(2))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + } + + #[test] + fn banned_regular_incoming_node_is_rejected() { + let peer1 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: HashSet::new(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(true); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + + // Incoming request. + controller.on_incoming_connection(peer1, IncomingIndex(1)); + assert_eq!(rx.try_recv().unwrap(), Message::Reject(IncomingIndex(1))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + } + + #[test] + fn banned_reserved_incoming_node_is_rejected() { + let reserved1 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: std::iter::once(reserved1).collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(true); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert!(controller.reserved_nodes.contains_key(&reserved1)); + + // Incoming request. + controller.on_incoming_connection(reserved1, IncomingIndex(1)); + assert_eq!(rx.try_recv().unwrap(), Message::Reject(IncomingIndex(1))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + } + + #[test] + fn we_dont_connect_to_banned_reserved_node() { + let reserved1 = PeerId::random(); + + let config = SetConfig { + in_peers: 10, + out_peers: 10, + bootnodes: Vec::new(), + reserved_nodes: std::iter::once(reserved1).collect(), + reserved_only: false, + }; + let (tx, mut rx) = tracing_unbounded("mpsc_test_to_notifications", 100); + + let mut peer_store = MockPeerStoreHandle::new(); + peer_store.expect_register_protocol().once().return_const(()); + peer_store.expect_is_banned().once().return_const(true); + peer_store.expect_outgoing_candidates().once().return_const(Vec::new()); + + let (_handle, mut controller) = + ProtocolController::new(SetId(0), config, tx, Box::new(peer_store)); + assert!(matches!(controller.reserved_nodes.get(&reserved1), Some(PeerState::NotConnected))); + + // Initiate connectios + controller.alloc_slots(); + assert!(matches!(controller.reserved_nodes.get(&reserved1), Some(PeerState::NotConnected))); + assert_eq!(rx.try_recv().unwrap_err(), TryRecvError::Empty); + } +} diff --git a/client/peerset/tests/fuzz.rs b/client/peerset/tests/fuzz.rs index 122f17062577d..855d2339eda12 100644 --- a/client/peerset/tests/fuzz.rs +++ b/client/peerset/tests/fuzz.rs @@ -31,19 +31,101 @@ use std::{ task::Poll, }; +/// Peer events as observed by `Notifications` / fuzz test. +#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] +enum Event { + /// Either API requested to disconnect from the peer, or the peer dropped. + Disconnected, + /// Incoming request. + Incoming, + /// Answer from PSM: accept. + PsmAccept, + /// Answer from PSM: reject. + PsmReject, + /// Command from PSM: connect. + PsmConnect, + /// Command from PSM: drop connection. + PsmDrop, +} + +/// Simplified peer state as thought by `Notifications` / fuzz test. +#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] +enum State { + /// Peer is not connected. + Disconnected, + /// We have an inbound connection, but have not decided yet whether to accept it. + Incoming(IncomingIndex), + /// Peer is connected via an inbound connection. + Inbound, + /// Peer is connected via an outbound connection. + Outbound, +} + +/// Bare simplified state without incoming index. +#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] +enum BareState { + /// Peer is not connected. + Disconnected, + /// We have an inbound connection, but have not decided yet whether to accept it. + Incoming, + /// Peer is connected via an inbound connection. + Inbound, + /// Peer is connected via an outbound connection. + Outbound, +} + +fn discard_incoming_index(state: State) -> BareState { + match state { + State::Disconnected => BareState::Disconnected, + State::Incoming(_) => BareState::Incoming, + State::Inbound => BareState::Inbound, + State::Outbound => BareState::Outbound, + } +} + #[test] fn run() { + sp_tracing::try_init_simple(); + for _ in 0..50 { test_once(); } } fn test_once() { + // Allowed events that can be received in a specific state. + let allowed_events: HashMap> = [ + ( + BareState::Disconnected, + [Event::Incoming, Event::PsmConnect, Event::PsmDrop /* must be ignored */] + .into_iter() + .collect::>(), + ), + ( + BareState::Incoming, + [Event::PsmAccept, Event::PsmReject].into_iter().collect::>(), + ), + ( + BareState::Inbound, + [Event::Disconnected, Event::PsmDrop, Event::PsmConnect /* must be ignored */] + .into_iter() + .collect::>(), + ), + ( + BareState::Outbound, + [Event::Disconnected, Event::PsmDrop, Event::PsmConnect /* must be ignored */] + .into_iter() + .collect::>(), + ), + ] + .into_iter() + .collect(); + // PRNG to use. let mut rng = rand::thread_rng(); // Nodes that the peerset knows about. - let mut known_nodes = HashSet::::new(); + let mut known_nodes = HashMap::::new(); // Nodes that we have reserved. Always a subset of `known_nodes`. let mut reserved_nodes = HashSet::::new(); @@ -52,7 +134,7 @@ fn test_once() { bootnodes: (0..Uniform::new_inclusive(0, 4).sample(&mut rng)) .map(|_| { let id = PeerId::random(); - known_nodes.insert(id); + known_nodes.insert(id, State::Disconnected); id }) .collect(), @@ -60,7 +142,7 @@ fn test_once() { (0..Uniform::new_inclusive(0, 2).sample(&mut rng)) .map(|_| { let id = PeerId::random(); - known_nodes.insert(id); + known_nodes.insert(id, State::Disconnected); reserved_nodes.insert(id); id }) @@ -72,6 +154,10 @@ fn test_once() { }], }); + let new_id = PeerId::random(); + known_nodes.insert(new_id, State::Disconnected); + peerset_handle.add_known_peer(new_id); + futures::executor::block_on(futures::future::poll_fn(move |cx| { // List of nodes the user of `peerset` assumes it's connected to. Always a subset of // `known_nodes`. @@ -84,28 +170,129 @@ fn test_once() { // Perform a certain number of actions while checking that the state is consistent. If we // reach the end of the loop, the run has succeeded. + // Note that with the ACKing and event postponing mechanism in `ProtocolController` + // the test time grows quadratically with the number of iterations below. for _ in 0..2500 { + // Peer we are working with. + let mut current_peer = None; + // Current event for event bigrams validation. + let mut current_event = None; + // Last peer state for allowed event validation. + let mut last_state = None; + // Each of these weights corresponds to an action that we may perform. let action_weights = [150, 90, 90, 30, 30, 1, 1, 4, 4]; match WeightedIndex::new(&action_weights).unwrap().sample(&mut rng) { // If we generate 0, poll the peerset. 0 => match Stream::poll_next(Pin::new(&mut peerset), cx) { Poll::Ready(Some(Message::Connect { peer_id, .. })) => { - if let Some(id) = - incoming_nodes.iter().find(|(_, v)| **v == peer_id).map(|(&id, _)| id) - { - incoming_nodes.remove(&id); + log::info!("PSM: connecting to peer {}", peer_id); + + let state = known_nodes.get_mut(&peer_id).unwrap(); + if matches!(*state, State::Incoming(_)) { + log::info!( + "Awaiting incoming response, ignoring obsolete Connect from PSM for peer {}", + peer_id, + ); + continue } - assert!(connected_nodes.insert(peer_id)); + + last_state = Some(*state); + + if *state != State::Inbound { + *state = State::Outbound; + } + + if !connected_nodes.insert(peer_id) { + log::info!("Request to connect to an already connected node {peer_id}"); + } + + current_peer = Some(peer_id); + current_event = Some(Event::PsmConnect); }, Poll::Ready(Some(Message::Drop { peer_id, .. })) => { - connected_nodes.remove(&peer_id); + log::info!("PSM: dropping peer {}", peer_id); + + let state = known_nodes.get_mut(&peer_id).unwrap(); + if matches!(*state, State::Incoming(_)) { + log::info!( + "Awaiting incoming response, ignoring obsolete Drop from PSM for peer {}", + peer_id, + ); + continue + } + + last_state = Some(*state); + *state = State::Disconnected; + + if !connected_nodes.remove(&peer_id) { + log::info!("Ignoring attempt to drop a disconnected peer {}", peer_id); + } + + current_peer = Some(peer_id); + current_event = Some(Event::PsmDrop); }, Poll::Ready(Some(Message::Accept(n))) => { - assert!(connected_nodes.insert(incoming_nodes.remove(&n).unwrap())) + log::info!("PSM: accepting index {}", n.0); + + let peer_id = incoming_nodes.remove(&n).unwrap(); + + let state = known_nodes.get_mut(&peer_id).unwrap(); + match *state { + State::Incoming(incoming_index) => + if n.0 < incoming_index.0 { + log::info!( + "Ignoring obsolete Accept for {:?} while awaiting {:?} for peer {}", + n, incoming_index, peer_id, + ); + continue + } else if n.0 > incoming_index.0 { + panic!( + "Received {:?} while awaiting {:?} for peer {}", + n, incoming_index, peer_id, + ); + }, + _ => {}, + } + + last_state = Some(*state); + *state = State::Inbound; + + assert!(connected_nodes.insert(peer_id)); + + current_peer = Some(peer_id); + current_event = Some(Event::PsmAccept); }, Poll::Ready(Some(Message::Reject(n))) => { - assert!(!connected_nodes.contains(&incoming_nodes.remove(&n).unwrap())) + log::info!("PSM: rejecting index {}", n.0); + + let peer_id = incoming_nodes.remove(&n).unwrap(); + + let state = known_nodes.get_mut(&peer_id).unwrap(); + match *state { + State::Incoming(incoming_index) => + if n.0 < incoming_index.0 { + log::info!( + "Ignoring obsolete Reject for {:?} while awaiting {:?} for peer {}", + n, incoming_index, peer_id, + ); + continue + } else if n.0 > incoming_index.0 { + panic!( + "Received {:?} while awaiting {:?} for peer {}", + n, incoming_index, peer_id, + ); + }, + _ => {}, + } + + last_state = Some(*state); + *state = State::Disconnected; + + assert!(!connected_nodes.contains(&peer_id)); + + current_peer = Some(peer_id); + current_event = Some(Event::PsmReject); }, Poll::Ready(None) => panic!(), Poll::Pending => {}, @@ -114,13 +301,13 @@ fn test_once() { // If we generate 1, discover a new node. 1 => { let new_id = PeerId::random(); - known_nodes.insert(new_id); - peerset.add_to_peers_set(SetId::from(0), new_id); + known_nodes.insert(new_id, State::Disconnected); + peerset_handle.add_known_peer(new_id); }, // If we generate 2, adjust a random reputation. 2 => - if let Some(id) = known_nodes.iter().choose(&mut rng) { + if let Some(id) = known_nodes.keys().choose(&mut rng) { let val = Uniform::new_inclusive(i32::MIN, i32::MAX).sample(&mut rng); peerset_handle.report_peer(*id, ReputationChange::new(val, "")); }, @@ -128,47 +315,86 @@ fn test_once() { // If we generate 3, disconnect from a random node. 3 => if let Some(id) = connected_nodes.iter().choose(&mut rng).cloned() { + log::info!("Disconnected from {}", id); connected_nodes.remove(&id); + + let state = known_nodes.get_mut(&id).unwrap(); + last_state = Some(*state); + *state = State::Disconnected; + peerset.dropped(SetId::from(0), id, DropReason::Unknown); + + current_peer = Some(id); + current_event = Some(Event::Disconnected); }, // If we generate 4, connect to a random node. 4 => { if let Some(id) = known_nodes - .iter() + .keys() .filter(|n| { incoming_nodes.values().all(|m| m != *n) && !connected_nodes.contains(*n) }) .choose(&mut rng) + .cloned() { - peerset.incoming(SetId::from(0), *id, next_incoming_id); - incoming_nodes.insert(next_incoming_id, *id); + log::info!("Incoming connection from {}, index {}", id, next_incoming_id.0); + peerset.incoming(SetId::from(0), id, next_incoming_id); + incoming_nodes.insert(next_incoming_id, id); + + let state = known_nodes.get_mut(&id).unwrap(); + last_state = Some(*state); + *state = State::Incoming(next_incoming_id); + next_incoming_id.0 += 1; + + current_peer = Some(id); + current_event = Some(Event::Incoming); } }, // 5 and 6 are the reserved-only mode. - 5 => peerset_handle.set_reserved_only(SetId::from(0), true), - 6 => peerset_handle.set_reserved_only(SetId::from(0), false), + 5 => { + log::info!("Set reserved only"); + peerset_handle.set_reserved_only(SetId::from(0), true); + }, + 6 => { + log::info!("Unset reserved only"); + peerset_handle.set_reserved_only(SetId::from(0), false); + }, // 7 and 8 are about switching a random node in or out of reserved mode. 7 => { if let Some(id) = - known_nodes.iter().filter(|n| !reserved_nodes.contains(*n)).choose(&mut rng) + known_nodes.keys().filter(|n| !reserved_nodes.contains(*n)).choose(&mut rng) { + log::info!("Add reserved: {}", id); peerset_handle.add_reserved_peer(SetId::from(0), *id); reserved_nodes.insert(*id); } }, 8 => if let Some(id) = reserved_nodes.iter().choose(&mut rng).cloned() { + log::info!("Remove reserved: {}", id); reserved_nodes.remove(&id); peerset_handle.remove_reserved_peer(SetId::from(0), id); }, _ => unreachable!(), } + + // Validate event bigrams and state transitions. + if let Some(peer_id) = current_peer { + let event = current_event.unwrap(); + let last_state = discard_incoming_index(last_state.unwrap()); + if !allowed_events.get(&last_state).unwrap().contains(&event) { + panic!( + "Invalid state transition: {:?} x {:?} for peer {}", + last_state, event, peer_id, + ); + } + } } Poll::Ready(()) diff --git a/client/utils/src/mpsc.rs b/client/utils/src/mpsc.rs index 7e06bd203b010..36e44be5e2950 100644 --- a/client/utils/src/mpsc.rs +++ b/client/utils/src/mpsc.rs @@ -18,8 +18,10 @@ //! Code to meter unbounded channels. +pub use async_channel::{TryRecvError, TrySendError}; + use crate::metrics::UNBOUNDED_CHANNELS_COUNTER; -use async_channel::{Receiver, Sender, TryRecvError, TrySendError}; +use async_channel::{Receiver, Sender}; use futures::{ stream::{FusedStream, Stream}, task::{Context, Poll},