From d8860f04699f8d19a015be30d2204f06478ee9a3 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Thu, 27 Jul 2023 17:44:24 +0300 Subject: [PATCH 01/23] Extract block announce validation/polling from `ChainSync` --- client/network/common/src/sync.rs | 62 +----- client/network/sync/src/engine.rs | 293 ++++++++++++++++++++++++++- client/network/sync/src/lib.rs | 325 +++--------------------------- client/network/sync/src/mock.rs | 17 +- 4 files changed, 313 insertions(+), 384 deletions(-) diff --git a/client/network/common/src/sync.rs b/client/network/common/src/sync.rs index b142925aeb10c..173890530185b 100644 --- a/client/network/common/src/sync.rs +++ b/client/network/common/src/sync.rs @@ -27,7 +27,7 @@ use futures::Stream; use libp2p_identity::PeerId; -use message::{BlockAnnounce, BlockData, BlockRequest, BlockResponse}; +use message::{BlockData, BlockRequest, BlockResponse}; use sc_consensus::{import_queue::RuntimeOrigin, IncomingBlock}; use sp_consensus::BlockOrigin; use sp_runtime::{ @@ -157,38 +157,6 @@ pub enum ImportResult { JustificationImport(RuntimeOrigin, B::Hash, NumberFor, Justifications), } -/// Value polled from `ChainSync` -#[derive(Debug)] -pub enum PollResult { - Import(ImportResult), - Announce(PollBlockAnnounceValidation), -} - -/// Result of [`ChainSync::poll_block_announce_validation`]. -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum PollBlockAnnounceValidation { - /// The announcement failed at validation. - /// - /// The peer reputation should be decreased. - Failure { - /// Who sent the processed block announcement? - who: PeerId, - /// Should the peer be disconnected? - disconnect: bool, - }, - /// The announcement does not require further handling. - Nothing { - /// Who sent the processed block announcement? - who: PeerId, - /// Was this their new best block? - is_best: bool, - /// The announcement. - announce: BlockAnnounce, - }, - /// The block announcement should be skipped. - Skip, -} - /// Sync operation mode. #[derive(Copy, Clone, Debug, Eq, PartialEq)] pub enum SyncMode { @@ -408,29 +376,6 @@ pub trait ChainSync: Send { /// Notify about finalization of the given block. fn on_block_finalized(&mut self, hash: &Block::Hash, number: NumberFor); - /// Push a block announce validation. - /// - /// It is required that [`ChainSync::poll_block_announce_validation`] is called - /// to check for finished block announce validations. - fn push_block_announce_validation( - &mut self, - who: PeerId, - hash: Block::Hash, - announce: BlockAnnounce, - is_best: bool, - ); - - /// Poll block announce validation. - /// - /// Block announce validations can be pushed by using - /// [`ChainSync::push_block_announce_validation`]. - /// - /// This should be polled until it returns [`Poll::Pending`]. - fn poll_block_announce_validation( - &mut self, - cx: &mut std::task::Context<'_>, - ) -> Poll>; - /// Call when a peer has disconnected. /// Canceled obsolete block request may result in some blocks being ready for /// import, so this functions checks for such blocks and returns them. @@ -451,10 +396,7 @@ pub trait ChainSync: Send { /// Internally calls [`ChainSync::poll_block_announce_validation()`] and /// this function should be polled until it returns [`Poll::Pending`] to /// consume all pending events. - fn poll( - &mut self, - cx: &mut std::task::Context, - ) -> Poll>; + fn poll(&mut self, cx: &mut std::task::Context) -> Poll<()>; /// Send block request to peer fn send_block_request(&mut self, who: PeerId, request: BlockRequest); diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 47d920771e20e..679ee85c1a209 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -25,9 +25,10 @@ use crate::{ }; use codec::{Decode, Encode}; -use futures::{FutureExt, StreamExt}; +use futures::{stream::FuturesUnordered, Future, FutureExt, StreamExt}; use futures_timer::Delay; use libp2p::PeerId; +use log::{debug, error, warn}; use prometheus_endpoint::{ register, Gauge, GaugeVec, MetricSource, Opts, PrometheusError, Registry, SourcedGauge, U64, }; @@ -45,17 +46,18 @@ use sc_network_common::{ sync::{ message::{BlockAnnounce, BlockAnnouncesHandshake, BlockState}, warp::WarpSyncParams, - BadPeer, ChainSync as ChainSyncT, ExtendedPeerInfo, PollBlockAnnounceValidation, SyncEvent, + BadPeer, ChainSync as ChainSyncT, ExtendedPeerInfo, SyncEvent, }, }; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_blockchain::HeaderMetadata; -use sp_consensus::block_validation::BlockAnnounceValidator; +use sp_consensus::block_validation::{BlockAnnounceValidator, Validation}; use sp_runtime::traits::{Block as BlockT, Header, NumberFor, Zero}; use std::{ - collections::{HashMap, HashSet}, + collections::{hash_map::Entry, HashMap, HashSet}, num::NonZeroUsize, + pin::Pin, sync::{ atomic::{AtomicBool, AtomicUsize, Ordering}, Arc, @@ -70,6 +72,17 @@ const TICK_TIMEOUT: std::time::Duration = std::time::Duration::from_millis(1100) /// Maximum number of known block hashes to keep for a peer. const MAX_KNOWN_BLOCKS: usize = 1024; // ~32kb per peer + LruHashSet overhead +/// Maximum number of concurrent block announce validations. +/// +/// If the queue reaches the maximum, we drop any new block +/// announcements. +const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS: usize = 256; + +/// Maximum number of concurrent block announce validations per peer. +/// +/// See [`MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS`] for more information. +const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER: usize = 4; + /// If the block announces stream to peer has been inactive for 30 seconds meaning local node /// has not sent or received block announcements to/from the peer, report the node for inactivity, /// disconnect it and attempt to establish connection to some other peer. @@ -182,6 +195,75 @@ pub struct Peer { inbound: bool, } +/// Result of [`SyncingEngine::block_announce_validation`]. +#[derive(Debug, Clone, PartialEq, Eq)] +pub(crate) enum PreValidateBlockAnnounce { + /// The announcement failed at validation. + /// + /// The peer reputation should be decreased. + Failure { + /// Who sent the processed block announcement? + who: PeerId, + /// Should the peer be disconnected? + disconnect: bool, + }, + /// The pre-validation was sucessful and the announcement should be + /// further processed. + Process { + /// Is this the new best block of the peer? + is_new_best: bool, + /// The id of the peer that send us the announcement. + who: PeerId, + /// The announcement. + announce: BlockAnnounce, + }, + /// The announcement validation returned an error. + /// + /// An error means that *this* node failed to validate it because some internal error happened. + /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express + /// this. + Error { who: PeerId }, + /// The block announcement should be skipped. + /// + /// This should *only* be returned when there wasn't a slot registered + /// for this block announcement validation. + Skip, +} + +/// Result of [`SyncingEngine::poll_block_announce_validation`]. +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum PollBlockAnnounceValidation { + /// The announcement failed at validation. + /// + /// The peer reputation should be decreased. + Failure { + /// Who sent the processed block announcement? + who: PeerId, + /// Should the peer be disconnected? + disconnect: bool, + }, + /// The announcement does not require further handling. + Nothing { + /// Who sent the processed block announcement? + who: PeerId, + /// Was this their new best block? + is_best: bool, + /// The announcement. + announce: BlockAnnounce, + }, + /// The block announcement should be skipped. + Skip, +} + +/// Result of [`SyncingEngine::has_slot_for_block_announce_validation`]. +enum HasSlotForBlockAnnounceValidation { + /// Yes, there is a slot for the block announce validation. + Yes, + /// We reached the total maximum number of validation slots. + TotalMaximumSlotsReached, + /// We reached the maximum number of validation slots for the given peer. + MaximumPeerSlotsReached, +} pub struct SyncingEngine { /// State machine that handles the list of in-progress requests. Only full node peers are /// registered. @@ -249,6 +331,16 @@ pub struct SyncingEngine { /// A cache for the data that was associated to a block announcement. block_announce_data_cache: LruMap>, + /// A type to check incoming block announcements. + block_announce_validator: Box + Send>, + + /// All block announcement that are currently being validated. + block_announce_validation: + FuturesUnordered> + Send>>>, + + /// Stats per peer about the number of concurrent block announce validations. + block_announce_validation_per_peer_stats: HashMap, + /// The `PeerId`'s of all boot nodes. boot_node_ids: HashSet, @@ -356,7 +448,6 @@ where protocol_id, fork_id, roles, - block_announce_validator, max_parallel_downloads, max_blocks_per_request, warp_sync_params, @@ -393,6 +484,9 @@ where peers: HashMap::new(), evicted: HashSet::new(), block_announce_data_cache: LruMap::new(ByLength::new(cache_capacity)), + block_announce_validator, + block_announce_validation: Default::default(), + block_announce_validation_per_peer_stats: Default::default(), block_announce_protocol_name, num_connected: num_connected.clone(), is_major_syncing: is_major_syncing.clone(), @@ -530,7 +624,188 @@ where BlockState::Normal => false, }; - self.chain_sync.push_block_announce_validation(who, hash, announce, is_best); + self.push_block_announce_validation_inner(who, hash, announce, is_best); + } + } + + fn push_block_announce_validation_inner( + &mut self, + who: PeerId, + hash: B::Hash, + announce: BlockAnnounce, + is_best: bool, + ) { + let header = &announce.header; + let number = *header.number(); + debug!( + target: "sync", + "Pre-validating received block announcement {:?} with number {:?} from {}", + hash, + number, + who, + ); + + if number.is_zero() { + self.block_announce_validation.push( + async move { + warn!( + target: "sync", + "💔 Ignored genesis block (#0) announcement from {}: {}", + who, + hash, + ); + PreValidateBlockAnnounce::Skip + } + .boxed(), + ); + return + } + + // Check if there is a slot for this block announce validation. + match self.has_slot_for_block_announce_validation(&who) { + HasSlotForBlockAnnounceValidation::Yes => {}, + HasSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { + self.block_announce_validation.push( + async move { + warn!( + target: "sync", + "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", + number, + hash, + who, + ); + PreValidateBlockAnnounce::Skip + } + .boxed(), + ); + return + }, + HasSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { + self.block_announce_validation.push(async move { + warn!( + target: "sync", + "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", + number, + hash, + who, + ); + PreValidateBlockAnnounce::Skip + }.boxed()); + return + }, + } + + // Let external validator check the block announcement. + let assoc_data = announce.data.as_ref().map_or(&[][..], |v| v.as_slice()); + let future = self.block_announce_validator.validate(header, assoc_data); + + self.block_announce_validation.push( + async move { + match future.await { + Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { + is_new_best: is_new_best || is_best, + announce, + who, + }, + Ok(Validation::Failure { disconnect }) => { + debug!( + target: "sync", + "Block announcement validation of block {:?} from {} failed", + hash, + who, + ); + PreValidateBlockAnnounce::Failure { who, disconnect } + }, + Err(e) => { + debug!( + target: "sync", + "💔 Block announcement validation of block {:?} errored: {}", + hash, + e, + ); + PreValidateBlockAnnounce::Error { who } + }, + } + } + .boxed(), + ); + } + + fn poll_block_announce_validation( + &mut self, + cx: &mut std::task::Context, + ) -> Poll> { + match self.block_announce_validation.poll_next_unpin(cx) { + Poll::Ready(Some(res)) => { + self.peer_block_announce_validation_finished(&res); + Poll::Ready(self.chain_sync.finish_block_announce_validation(res)) + }, + _ => Poll::Pending, + } + } + + /// Checks if there is a slot for a block announce validation. + /// + /// The total number and the number per peer of concurrent block announce validations + /// is capped. + /// + /// Returns [`HasSlotForBlockAnnounceValidation`] to inform about the result. + /// + /// # Note + /// + /// It is *required* to call [`Self::peer_block_announce_validation_finished`] when the + /// validation is finished to clear the slot. + fn has_slot_for_block_announce_validation( + &mut self, + peer: &PeerId, + ) -> HasSlotForBlockAnnounceValidation { + if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { + return HasSlotForBlockAnnounceValidation::TotalMaximumSlotsReached + } + + match self.block_announce_validation_per_peer_stats.entry(*peer) { + Entry::Vacant(entry) => { + entry.insert(1); + HasSlotForBlockAnnounceValidation::Yes + }, + Entry::Occupied(mut entry) => { + if *entry.get() < MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { + *entry.get_mut() += 1; + HasSlotForBlockAnnounceValidation::Yes + } else { + HasSlotForBlockAnnounceValidation::MaximumPeerSlotsReached + } + }, + } + } + + /// Should be called when a block announce validation is finished, to update the slots + /// of the peer that send the block announce. + fn peer_block_announce_validation_finished( + &mut self, + res: &PreValidateBlockAnnounce, + ) { + let peer = match res { + PreValidateBlockAnnounce::Failure { who, .. } | + PreValidateBlockAnnounce::Process { who, .. } | + PreValidateBlockAnnounce::Error { who } => who, + PreValidateBlockAnnounce::Skip => return, + }; + + match self.block_announce_validation_per_peer_stats.entry(*peer) { + Entry::Vacant(_) => { + error!( + target: "sync", + "💔 Block announcement validation from peer {} finished for that no slot was allocated!", + peer, + ); + }, + Entry::Occupied(mut entry) => { + *entry.get_mut() = entry.get().saturating_sub(1); + if *entry.get() == 0 { + entry.remove(); + } + }, } } @@ -766,9 +1041,7 @@ where // Make sure that the newly added block announce validation future // was polled once to be registered in the task. - if let Poll::Ready(res) = - self.chain_sync.poll_block_announce_validation(cx) - { + if let Poll::Ready(res) = self.poll_block_announce_validation(cx) { self.process_block_announce_validation_result(res) } } else { @@ -794,7 +1067,7 @@ where // poll `ChainSync` last because of a block announcement was received through the // event stream between `SyncingEngine` and `Protocol` and the validation finished // right after it as queued, the resulting block request (if any) can be sent right away. - while let Poll::Ready(result) = self.chain_sync.poll(cx) { + while let Poll::Ready(result) = self.poll_block_announce_validation(cx) { self.process_block_announce_validation_result(result); } diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index 03a22995bb924..ddc4bb90fd1cd 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -30,6 +30,7 @@ use crate::{ blocks::BlockCollection, + engine::{PollBlockAnnounceValidation, PreValidateBlockAnnounce}, schema::v1::{StateRequest, StateResponse}, state::StateSync, warp::{WarpProofImportResult, WarpSync}, @@ -37,9 +38,7 @@ use crate::{ use codec::{Decode, DecodeAll, Encode}; use extra_requests::ExtraRequests; -use futures::{ - channel::oneshot, stream::FuturesUnordered, task::Poll, Future, FutureExt, StreamExt, -}; +use futures::{channel::oneshot, task::Poll, Future, FutureExt}; use libp2p::{request_response::OutboundFailure, PeerId}; use log::{debug, error, info, trace, warn}; use prost::Message; @@ -60,22 +59,18 @@ use sc_network_common::{ role::Roles, sync::{ message::{ - BlockAnnounce, BlockAnnouncesHandshake, BlockAttributes, BlockData, BlockRequest, - BlockResponse, Direction, FromBlock, + BlockAnnouncesHandshake, BlockAttributes, BlockData, BlockRequest, BlockResponse, + Direction, FromBlock, }, warp::{EncodedProof, WarpProofRequest, WarpSyncParams, WarpSyncPhase, WarpSyncProgress}, BadPeer, ChainSync as ChainSyncT, ImportResult, Metrics, OnBlockData, OnBlockJustification, OnStateData, OpaqueBlockRequest, OpaqueBlockResponse, OpaqueStateRequest, - OpaqueStateResponse, PeerInfo, PeerRequest, PollBlockAnnounceValidation, SyncMode, - SyncState, SyncStatus, + OpaqueStateResponse, PeerInfo, PeerRequest, SyncMode, SyncState, SyncStatus, }, }; use sp_arithmetic::traits::Saturating; use sp_blockchain::{Error as ClientError, HeaderBackend, HeaderMetadata}; -use sp_consensus::{ - block_validation::{BlockAnnounceValidator, Validation}, - BlockOrigin, BlockStatus, -}; +use sp_consensus::{BlockOrigin, BlockStatus}; use sp_runtime::{ traits::{ Block as BlockT, CheckedSub, Hash, HashFor, Header as HeaderT, NumberFor, One, @@ -85,7 +80,7 @@ use sp_runtime::{ }; use std::{ - collections::{hash_map::Entry, HashMap, HashSet}, + collections::{HashMap, HashSet}, iter, ops::Range, pin::Pin, @@ -117,17 +112,6 @@ const MAX_DOWNLOAD_AHEAD: u32 = 2048; /// common block of a node. const MAX_BLOCKS_TO_LOOK_BACKWARDS: u32 = MAX_DOWNLOAD_AHEAD / 2; -/// Maximum number of concurrent block announce validations. -/// -/// If the queue reaches the maximum, we drop any new block -/// announcements. -const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS: usize = 256; - -/// Maximum number of concurrent block announce validations per peer. -/// -/// See [`MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS`] for more information. -const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER: usize = 4; - /// Pick the state to sync as the latest finalized number minus this. const STATE_SYNC_FINALITY_THRESHOLD: u32 = 8; @@ -307,19 +291,12 @@ pub struct ChainSync { fork_targets: HashMap>, /// A set of peers for which there might be potential block requests allowed_requests: AllowedRequests, - /// A type to check incoming block announcements. - block_announce_validator: Box + Send>, /// Maximum number of peers to ask the same blocks in parallel. max_parallel_downloads: u32, /// Maximum blocks per request. max_blocks_per_request: u32, /// Total number of downloaded blocks. downloaded_blocks: usize, - /// All block announcement that are currently being validated. - block_announce_validation: - FuturesUnordered> + Send>>>, - /// Stats per peer about the number of concurrent block announce validations. - block_announce_validation_per_peer_stats: HashMap, /// State sync in progress, if any. state_sync: Option>, /// Warp sync in progress, if any. @@ -424,51 +401,6 @@ impl PeerSyncState { } } -/// Result of [`ChainSync::block_announce_validation`]. -#[derive(Debug, Clone, PartialEq, Eq)] -enum PreValidateBlockAnnounce { - /// The announcement failed at validation. - /// - /// The peer reputation should be decreased. - Failure { - /// Who sent the processed block announcement? - who: PeerId, - /// Should the peer be disconnected? - disconnect: bool, - }, - /// The pre-validation was sucessful and the announcement should be - /// further processed. - Process { - /// Is this the new best block of the peer? - is_new_best: bool, - /// The id of the peer that send us the announcement. - who: PeerId, - /// The announcement. - announce: BlockAnnounce, - }, - /// The announcement validation returned an error. - /// - /// An error means that *this* node failed to validate it because some internal error happened. - /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express - /// this. - Error { who: PeerId }, - /// The block announcement should be skipped. - /// - /// This should *only* be returned when there wasn't a slot registered - /// for this block announcement validation. - Skip, -} - -/// Result of [`ChainSync::has_slot_for_block_announce_validation`]. -enum HasSlotForBlockAnnounceValidation { - /// Yes, there is a slot for the block announce validation. - Yes, - /// We reached the total maximum number of validation slots. - TotalMaximumSlotsReached, - /// We reached the maximum number of validation slots for the given peer. - MaximumPeerSlotsReached, -} - impl ChainSyncT for ChainSync where B: BlockT, @@ -1107,122 +1039,6 @@ where } } - fn push_block_announce_validation( - &mut self, - who: PeerId, - hash: B::Hash, - announce: BlockAnnounce, - is_best: bool, - ) { - let header = &announce.header; - let number = *header.number(); - debug!( - target: "sync", - "Pre-validating received block announcement {:?} with number {:?} from {}", - hash, - number, - who, - ); - - if number.is_zero() { - self.block_announce_validation.push( - async move { - warn!( - target: "sync", - "💔 Ignored genesis block (#0) announcement from {}: {}", - who, - hash, - ); - PreValidateBlockAnnounce::Skip - } - .boxed(), - ); - return - } - - // Check if there is a slot for this block announce validation. - match self.has_slot_for_block_announce_validation(&who) { - HasSlotForBlockAnnounceValidation::Yes => {}, - HasSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { - self.block_announce_validation.push( - async move { - warn!( - target: "sync", - "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", - number, - hash, - who, - ); - PreValidateBlockAnnounce::Skip - } - .boxed(), - ); - return - }, - HasSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { - self.block_announce_validation.push(async move { - warn!( - target: "sync", - "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", - number, - hash, - who, - ); - PreValidateBlockAnnounce::Skip - }.boxed()); - return - }, - } - - // Let external validator check the block announcement. - let assoc_data = announce.data.as_ref().map_or(&[][..], |v| v.as_slice()); - let future = self.block_announce_validator.validate(header, assoc_data); - - self.block_announce_validation.push( - async move { - match future.await { - Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { - is_new_best: is_new_best || is_best, - announce, - who, - }, - Ok(Validation::Failure { disconnect }) => { - debug!( - target: "sync", - "Block announcement validation of block {:?} from {} failed", - hash, - who, - ); - PreValidateBlockAnnounce::Failure { who, disconnect } - }, - Err(e) => { - debug!( - target: "sync", - "💔 Block announcement validation of block {:?} errored: {}", - hash, - e, - ); - PreValidateBlockAnnounce::Error { who } - }, - } - } - .boxed(), - ); - } - - fn poll_block_announce_validation( - &mut self, - cx: &mut std::task::Context, - ) -> Poll> { - match self.block_announce_validation.poll_next_unpin(cx) { - Poll::Ready(Some(res)) => { - self.peer_block_announce_validation_finished(&res); - Poll::Ready(self.finish_block_announce_validation(res)) - }, - _ => Poll::Pending, - } - } - fn peer_disconnected(&mut self, who: &PeerId) { self.blocks.clear_peer_download(who); if let Some(gap_sync) = &mut self.gap_sync { @@ -1319,10 +1135,7 @@ where .map_err(|error: codec::Error| error.to_string()) } - fn poll( - &mut self, - cx: &mut std::task::Context, - ) -> Poll> { + fn poll(&mut self, cx: &mut std::task::Context) -> Poll<()> { // Should be called before `process_outbound_requests` to ensure // that a potential target block is directly leading to requests. if let Some(warp_sync) = &mut self.warp_sync { @@ -1339,10 +1152,6 @@ where } } - if let Poll::Ready(announce) = self.poll_block_announce_validation(cx) { - return Poll::Ready(announce) - } - Poll::Pending } @@ -1395,7 +1204,6 @@ where protocol_id: ProtocolId, fork_id: &Option, roles: Roles, - block_announce_validator: Box + Send>, max_parallel_downloads: u32, max_blocks_per_request: u32, warp_sync_params: Option>, @@ -1430,12 +1238,9 @@ where queue_blocks: Default::default(), fork_targets: Default::default(), allowed_requests: Default::default(), - block_announce_validator, max_parallel_downloads, max_blocks_per_request, downloaded_blocks: 0, - block_announce_validation: Default::default(), - block_announce_validation_per_peer_stats: Default::default(), state_sync: None, warp_sync: None, import_existing: false, @@ -1586,71 +1391,6 @@ where self.allowed_requests.set_all(); } - /// Checks if there is a slot for a block announce validation. - /// - /// The total number and the number per peer of concurrent block announce validations - /// is capped. - /// - /// Returns [`HasSlotForBlockAnnounceValidation`] to inform about the result. - /// - /// # Note - /// - /// It is *required* to call [`Self::peer_block_announce_validation_finished`] when the - /// validation is finished to clear the slot. - fn has_slot_for_block_announce_validation( - &mut self, - peer: &PeerId, - ) -> HasSlotForBlockAnnounceValidation { - if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { - return HasSlotForBlockAnnounceValidation::TotalMaximumSlotsReached - } - - match self.block_announce_validation_per_peer_stats.entry(*peer) { - Entry::Vacant(entry) => { - entry.insert(1); - HasSlotForBlockAnnounceValidation::Yes - }, - Entry::Occupied(mut entry) => { - if *entry.get() < MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { - *entry.get_mut() += 1; - HasSlotForBlockAnnounceValidation::Yes - } else { - HasSlotForBlockAnnounceValidation::MaximumPeerSlotsReached - } - }, - } - } - - /// Should be called when a block announce validation is finished, to update the slots - /// of the peer that send the block announce. - fn peer_block_announce_validation_finished( - &mut self, - res: &PreValidateBlockAnnounce, - ) { - let peer = match res { - PreValidateBlockAnnounce::Failure { who, .. } | - PreValidateBlockAnnounce::Process { who, .. } | - PreValidateBlockAnnounce::Error { who } => who, - PreValidateBlockAnnounce::Skip => return, - }; - - match self.block_announce_validation_per_peer_stats.entry(*peer) { - Entry::Vacant(_) => { - error!( - target: "sync", - "💔 Block announcement validation from peer {} finished for that no slot was allocated!", - peer, - ); - }, - Entry::Occupied(mut entry) => { - *entry.get_mut() = entry.get().saturating_sub(1); - if *entry.get() == 0 { - entry.remove(); - } - }, - } - } - /// This will finish processing of the block announcement. fn finish_block_announce_validation( &mut self, @@ -3161,15 +2901,14 @@ fn validate_blocks( #[cfg(test)] mod test { use super::*; - use crate::service::network::NetworkServiceProvider; - use futures::{executor::block_on, future::poll_fn}; + use crate::{engine::PreValidateBlockAnnounce, service::network::NetworkServiceProvider}; + use futures::executor::block_on; use sc_block_builder::BlockBuilderProvider; use sc_network_common::{ role::Role, - sync::message::{BlockData, BlockState, FromBlock}, + sync::message::{BlockAnnounce, BlockData, BlockState, FromBlock}, }; use sp_blockchain::HeaderBackend; - use sp_consensus::block_validation::DefaultBlockAnnounceValidator; use substrate_test_runtime_client::{ runtime::{Block, Hash, Header}, BlockBuilderExt, ClientBlockImportExt, ClientExt, DefaultTestClientBuilderExt, TestClient, @@ -3183,7 +2922,6 @@ mod test { // internally we should process the response as the justification not being available. let client = Arc::new(TestClientBuilder::new().build()); - let block_announce_validator = Box::new(DefaultBlockAnnounceValidator); let peer_id = PeerId::random(); let import_queue = Box::new(sc_consensus::import_queue::mock::MockImportQueueHandle::new()); @@ -3195,7 +2933,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - block_announce_validator, 1, 64, None, @@ -3262,7 +2999,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 1, 64, None, @@ -3344,23 +3080,20 @@ mod test { /// Send a block annoucnement for the given `header`. fn send_block_announce( header: Header, - peer_id: &PeerId, + peer_id: PeerId, sync: &mut ChainSync, ) { - let block_annnounce = BlockAnnounce { + let announce = BlockAnnounce { header: header.clone(), state: Some(BlockState::Best), data: Some(Vec::new()), }; - sync.push_block_announce_validation(*peer_id, header.hash(), block_annnounce, true); - - // Poll until we have procssed the block announcement - block_on(poll_fn(|cx| loop { - if sync.poll_block_announce_validation(cx).is_pending() { - break Poll::Ready(()) - } - })) + sync.finish_block_announce_validation(PreValidateBlockAnnounce::Process { + announce, + is_new_best: true, + who: peer_id, + }); } /// Create a block response from the given `blocks`. @@ -3444,7 +3177,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 5, 64, None, @@ -3491,7 +3223,7 @@ mod test { assert!(sync.block_requests().is_empty()); // Let peer2 announce a fork of block 3 - send_block_announce(block3_fork.header().clone(), &peer_id2, &mut sync); + send_block_announce(block3_fork.header().clone(), peer_id2, &mut sync); // Import and tell sync that we now have the fork. block_on(client.import(BlockOrigin::Own, block3_fork.clone())).unwrap(); @@ -3500,13 +3232,13 @@ mod test { let block4 = build_block_at(block3_fork.hash(), false); // Let peer2 announce block 4 and check that sync wants to get the block. - send_block_announce(block4.header().clone(), &peer_id2, &mut sync); + send_block_announce(block4.header().clone(), peer_id2, &mut sync); let request = get_block_request(&mut sync, FromBlock::Hash(block4.hash()), 2, &peer_id2); // Peer1 announces the same block, but as the common block is still `1`, sync will request // block 2 again. - send_block_announce(block4.header().clone(), &peer_id1, &mut sync); + send_block_announce(block4.header().clone(), peer_id1, &mut sync); let request2 = get_block_request(&mut sync, FromBlock::Number(2), 1, &peer_id1); @@ -3571,7 +3303,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 5, 64, None, @@ -3647,7 +3378,7 @@ mod test { sync.queue_blocks.clear(); // Let peer2 announce that it finished syncing - send_block_announce(best_block.header().clone(), &peer_id2, &mut sync); + send_block_announce(best_block.header().clone(), peer_id2, &mut sync); let (peer1_req, peer2_req) = sync.block_requests().into_iter().fold((None, None), |res, req| { @@ -3729,7 +3460,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 5, 64, None, @@ -3754,7 +3484,7 @@ mod test { sync.new_peer(peer_id1, common_block.hash(), *common_block.header().number()) .unwrap(); - send_block_announce(fork_blocks.last().unwrap().header().clone(), &peer_id1, &mut sync); + send_block_announce(fork_blocks.last().unwrap().header().clone(), peer_id1, &mut sync); let mut request = get_block_request(&mut sync, FromBlock::Number(info.best_number), 1, &peer_id1); @@ -3872,7 +3602,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 5, 64, None, @@ -3897,7 +3626,7 @@ mod test { sync.new_peer(peer_id1, common_block.hash(), *common_block.header().number()) .unwrap(); - send_block_announce(fork_blocks.last().unwrap().header().clone(), &peer_id1, &mut sync); + send_block_announce(fork_blocks.last().unwrap().header().clone(), peer_id1, &mut sync); let mut request = get_block_request(&mut sync, FromBlock::Number(info.best_number), 1, &peer_id1); @@ -4017,7 +3746,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 1, 64, None, @@ -4039,7 +3767,7 @@ mod test { // Create a "new" header and announce it let mut header = blocks[0].header().clone(); header.number = 4; - send_block_announce(header, &peer_id1, &mut sync); + send_block_announce(header, peer_id1, &mut sync); assert!(sync.fork_targets.len() == 1); sync.peer_disconnected(&peer_id1); @@ -4063,7 +3791,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - Box::new(DefaultBlockAnnounceValidator), 1, 64, None, @@ -4107,7 +3834,6 @@ mod test { #[test] fn sync_restart_removes_block_but_not_justification_requests() { let mut client = Arc::new(TestClientBuilder::new().build()); - let block_announce_validator = Box::new(DefaultBlockAnnounceValidator); let import_queue = Box::new(sc_consensus::import_queue::mock::MockImportQueueHandle::new()); let (_chain_sync_network_provider, chain_sync_network_handle) = NetworkServiceProvider::new(); @@ -4117,7 +3843,6 @@ mod test { ProtocolId::from("test-protocol-name"), &Some(String::from("test-fork-id")), Roles::from(&Role::Full), - block_announce_validator, 1, 64, None, diff --git a/client/network/sync/src/mock.rs b/client/network/sync/src/mock.rs index 838c6cf7667a2..dfee0ee09aca5 100644 --- a/client/network/sync/src/mock.rs +++ b/client/network/sync/src/mock.rs @@ -22,9 +22,9 @@ use futures::task::Poll; use libp2p::PeerId; use sc_network_common::sync::{ - message::{BlockAnnounce, BlockData, BlockRequest, BlockResponse}, + message::{BlockData, BlockRequest, BlockResponse}, BadPeer, ChainSync as ChainSyncT, Metrics, OnBlockData, OnBlockJustification, - OpaqueBlockResponse, PeerInfo, PollBlockAnnounceValidation, SyncStatus, + OpaqueBlockResponse, PeerInfo, SyncStatus, }; use sp_runtime::traits::{Block as BlockT, NumberFor}; @@ -71,17 +71,6 @@ mockall::mock! { success: bool, ); fn on_block_finalized(&mut self, hash: &Block::Hash, number: NumberFor); - fn push_block_announce_validation( - &mut self, - who: PeerId, - hash: Block::Hash, - announce: BlockAnnounce, - is_best: bool, - ); - fn poll_block_announce_validation<'a>( - &mut self, - cx: &mut std::task::Context<'a>, - ) -> Poll>; fn peer_disconnected(&mut self, who: &PeerId); fn metrics(&self) -> Metrics; fn block_response_into_blocks( @@ -92,7 +81,7 @@ mockall::mock! { fn poll<'a>( &mut self, cx: &mut std::task::Context<'a>, - ) -> Poll>; + ) -> Poll<()>; fn send_block_request( &mut self, who: PeerId, From 26be4809dccad134433d4bea93a1ce3a2d625fe7 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Thu, 27 Jul 2023 17:52:56 +0300 Subject: [PATCH 02/23] Get rid of `push_block_announce_validation_inner` --- client/network/sync/src/engine.rs | 23 ++++++++--------------- 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 679ee85c1a209..c758458c49160 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -618,23 +618,15 @@ where peer.known_blocks.insert(hash); peer.last_notification_received = Instant::now(); - if peer.info.roles.is_full() { - let is_best = match announce.state.unwrap_or(BlockState::Best) { - BlockState::Best => true, - BlockState::Normal => false, - }; - - self.push_block_announce_validation_inner(who, hash, announce, is_best); + if !peer.info.roles.is_full() { + return } - } - fn push_block_announce_validation_inner( - &mut self, - who: PeerId, - hash: B::Hash, - announce: BlockAnnounce, - is_best: bool, - ) { + let is_best = match announce.state.unwrap_or(BlockState::Best) { + BlockState::Best => true, + BlockState::Normal => false, + }; + let header = &announce.header; let number = *header.number(); debug!( @@ -731,6 +723,7 @@ where ); } + /// Poll for finished block announce validations and notify `ChainSync`. fn poll_block_announce_validation( &mut self, cx: &mut std::task::Context, From e48c931ec276d2ff5b620d15a291419ee343c3ce Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Fri, 28 Jul 2023 14:33:01 +0300 Subject: [PATCH 03/23] Extend `ChainSync` trait with `on_validated_block_announce` --- client/network/common/src/sync.rs | 14 +- client/network/sync/src/engine.rs | 160 +++++++++++----------- client/network/sync/src/lib.rs | 213 +++++++++++++----------------- client/network/sync/src/mock.rs | 8 +- 4 files changed, 191 insertions(+), 204 deletions(-) diff --git a/client/network/common/src/sync.rs b/client/network/common/src/sync.rs index 173890530185b..461c4ae411d6a 100644 --- a/client/network/common/src/sync.rs +++ b/client/network/common/src/sync.rs @@ -22,7 +22,7 @@ pub mod message; pub mod metrics; pub mod warp; -use crate::{role::Roles, types::ReputationChange}; +use crate::{role::Roles, sync::message::BlockAnnounce, types::ReputationChange}; use futures::Stream; use libp2p_identity::PeerId; @@ -376,6 +376,14 @@ pub trait ChainSync: Send { /// Notify about finalization of the given block. fn on_block_finalized(&mut self, hash: &Block::Hash, number: NumberFor); + /// Notify about pre-validated block announcement. + fn on_validated_block_announce( + &mut self, + is_best: bool, + who: PeerId, + announce: &BlockAnnounce, + ); + /// Call when a peer has disconnected. /// Canceled obsolete block request may result in some blocks being ready for /// import, so this functions checks for such blocks and returns them. @@ -392,10 +400,6 @@ pub trait ChainSync: Send { ) -> Result>, String>; /// Advance the state of `ChainSync` - /// - /// Internally calls [`ChainSync::poll_block_announce_validation()`] and - /// this function should be polled until it returns [`Poll::Pending`] to - /// consume all pending events. fn poll(&mut self, cx: &mut std::task::Context) -> Poll<()>; /// Send block request to peer diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index c758458c49160..ad180026c5f6d 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -28,7 +28,7 @@ use codec::{Decode, Encode}; use futures::{stream::FuturesUnordered, Future, FutureExt, StreamExt}; use futures_timer::Delay; use libp2p::PeerId; -use log::{debug, error, warn}; +use log::{debug, error, trace, warn}; use prometheus_endpoint::{ register, Gauge, GaugeVec, MetricSource, Opts, PrometheusError, Registry, SourcedGauge, U64, }; @@ -223,11 +223,16 @@ pub(crate) enum PreValidateBlockAnnounce { /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express /// this. Error { who: PeerId }, - /// The block announcement should be skipped. - /// - /// This should *only* be returned when there wasn't a slot registered - /// for this block announcement validation. - Skip, +} + +impl PreValidateBlockAnnounce { + fn who(&self) -> &PeerId { + match self { + PreValidateBlockAnnounce::Failure { who, .. } | + PreValidateBlockAnnounce::Process { who, .. } | + PreValidateBlockAnnounce::Error { who } => who, + } + } } /// Result of [`SyncingEngine::poll_block_announce_validation`]. @@ -247,7 +252,7 @@ pub enum PollBlockAnnounceValidation { /// Who sent the processed block announcement? who: PeerId, /// Was this their new best block? - is_best: bool, + is_new_best: bool, /// The announcement. announce: BlockAnnounce, }, @@ -255,10 +260,10 @@ pub enum PollBlockAnnounceValidation { Skip, } -/// Result of [`SyncingEngine::has_slot_for_block_announce_validation`]. -enum HasSlotForBlockAnnounceValidation { - /// Yes, there is a slot for the block announce validation. - Yes, +/// Result of [`SyncingEngine::allocate_slot_for_block_announce_validation`]. +enum AllocateSlotForBlockAnnounceValidation { + /// Success, there is a slot for the block announce validation. + Allocated, /// We reached the total maximum number of validation slots. TotalMaximumSlotsReached, /// We reached the maximum number of validation slots for the given peer. @@ -567,7 +572,7 @@ where ) { match validation_result { PollBlockAnnounceValidation::Skip => {}, - PollBlockAnnounceValidation::Nothing { is_best: _, who, announce } => { + PollBlockAnnounceValidation::Nothing { is_new_best: _, who, announce } => { self.update_peer_info(&who); if let Some(data) = announce.data { @@ -589,7 +594,7 @@ where /// Push a block announce validation. /// - /// It is required that [`ChainSync::poll_block_announce_validation`] is + /// It is required that [`SyncingEngine::poll_block_announce_validation`] is /// called later to check for finished validations. The result of the validation /// needs to be passed to [`SyncingEngine::process_block_announce_validation_result`] /// to finish the processing. @@ -598,7 +603,7 @@ where /// /// This will internally create a future, but this future will not be registered /// in the task before being polled once. So, it is required to call - /// [`ChainSync::poll_block_announce_validation`] to ensure that the future is + /// [`SyncingEngine::poll_block_announce_validation`] to ensure that the future is /// registered properly and will wake up the task when being ready. pub fn push_block_announce_validation( &mut self, @@ -638,51 +643,36 @@ where ); if number.is_zero() { - self.block_announce_validation.push( - async move { - warn!( - target: "sync", - "💔 Ignored genesis block (#0) announcement from {}: {}", - who, - hash, - ); - PreValidateBlockAnnounce::Skip - } - .boxed(), + warn!( + target: "sync", + "💔 Ignored genesis block (#0) announcement from {}: {}", + who, + hash, ); return } - // Check if there is a slot for this block announce validation. - match self.has_slot_for_block_announce_validation(&who) { - HasSlotForBlockAnnounceValidation::Yes => {}, - HasSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { - self.block_announce_validation.push( - async move { - warn!( - target: "sync", - "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", - number, - hash, - who, - ); - PreValidateBlockAnnounce::Skip - } - .boxed(), + // Try to allocate a slot for this block announce validation. + match self.allocate_slot_for_block_announce_validation(&who) { + AllocateSlotForBlockAnnounceValidation::Allocated => {}, + AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { + warn!( + target: "sync", + "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", + number, + hash, + who, ); return }, - HasSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { - self.block_announce_validation.push(async move { - warn!( - target: "sync", - "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", - number, - hash, - who, - ); - PreValidateBlockAnnounce::Skip - }.boxed()); + AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { + warn!( + target: "sync", + "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", + number, + hash, + who, + ); return }, } @@ -729,9 +719,41 @@ where cx: &mut std::task::Context, ) -> Poll> { match self.block_announce_validation.poll_next_unpin(cx) { - Poll::Ready(Some(res)) => { - self.peer_block_announce_validation_finished(&res); - Poll::Ready(self.chain_sync.finish_block_announce_validation(res)) + Poll::Ready(Some(pre_validation)) => { + self.deallocate_slot_for_block_announce_validation(pre_validation.who()); + + let res = match pre_validation { + PreValidateBlockAnnounce::Process { is_new_best, who, announce } => { + trace!( + target: "sync", + "Finished block announce validation: from {:?}: {:?}. local_best={}", + who, + announce.summary(), + is_new_best, + ); + self.chain_sync.on_validated_block_announce(is_new_best, who, &announce); + PollBlockAnnounceValidation::Nothing { is_new_best, who, announce } + }, + PreValidateBlockAnnounce::Failure { who, disconnect } => { + debug!( + target: "sync", + "Failed announce validation: {:?}, disconnect: {}", + who, + disconnect, + ); + PollBlockAnnounceValidation::Failure { who, disconnect } + }, + PreValidateBlockAnnounce::Error { who } => { + debug!( + target: "sync", + "Ignored announce validation from {:?} due to internal validation error", + who, + ); + PollBlockAnnounceValidation::Skip + }, + }; + + Poll::Ready(res) }, _ => Poll::Pending, } @@ -742,31 +764,31 @@ where /// The total number and the number per peer of concurrent block announce validations /// is capped. /// - /// Returns [`HasSlotForBlockAnnounceValidation`] to inform about the result. + /// Returns [`AllocateSlotForBlockAnnounceValidation`] to inform about the result. /// /// # Note /// - /// It is *required* to call [`Self::peer_block_announce_validation_finished`] when the + /// It is *required* to call [`Self::deallocate_slot_for_block_announce_validation`] when the /// validation is finished to clear the slot. - fn has_slot_for_block_announce_validation( + fn allocate_slot_for_block_announce_validation( &mut self, peer: &PeerId, - ) -> HasSlotForBlockAnnounceValidation { + ) -> AllocateSlotForBlockAnnounceValidation { if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { - return HasSlotForBlockAnnounceValidation::TotalMaximumSlotsReached + return AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached } match self.block_announce_validation_per_peer_stats.entry(*peer) { Entry::Vacant(entry) => { entry.insert(1); - HasSlotForBlockAnnounceValidation::Yes + AllocateSlotForBlockAnnounceValidation::Allocated }, Entry::Occupied(mut entry) => { if *entry.get() < MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { *entry.get_mut() += 1; - HasSlotForBlockAnnounceValidation::Yes + AllocateSlotForBlockAnnounceValidation::Allocated } else { - HasSlotForBlockAnnounceValidation::MaximumPeerSlotsReached + AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached } }, } @@ -774,17 +796,7 @@ where /// Should be called when a block announce validation is finished, to update the slots /// of the peer that send the block announce. - fn peer_block_announce_validation_finished( - &mut self, - res: &PreValidateBlockAnnounce, - ) { - let peer = match res { - PreValidateBlockAnnounce::Failure { who, .. } | - PreValidateBlockAnnounce::Process { who, .. } | - PreValidateBlockAnnounce::Error { who } => who, - PreValidateBlockAnnounce::Skip => return, - }; - + fn deallocate_slot_for_block_announce_validation(&mut self, peer: &PeerId) { match self.block_announce_validation_per_peer_stats.entry(*peer) { Entry::Vacant(_) => { error!( diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index ddc4bb90fd1cd..b7e4ff4c93501 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -30,7 +30,6 @@ use crate::{ blocks::BlockCollection, - engine::{PollBlockAnnounceValidation, PreValidateBlockAnnounce}, schema::v1::{StateRequest, StateResponse}, state::StateSync, warp::{WarpProofImportResult, WarpSync}, @@ -59,8 +58,8 @@ use sc_network_common::{ role::Roles, sync::{ message::{ - BlockAnnouncesHandshake, BlockAttributes, BlockData, BlockRequest, BlockResponse, - Direction, FromBlock, + BlockAnnounce, BlockAnnouncesHandshake, BlockAttributes, BlockData, BlockRequest, + BlockResponse, Direction, FromBlock, }, warp::{EncodedProof, WarpProofRequest, WarpSyncParams, WarpSyncPhase, WarpSyncProgress}, BadPeer, ChainSync as ChainSyncT, ImportResult, Metrics, OnBlockData, OnBlockJustification, @@ -1039,6 +1038,91 @@ where } } + fn on_validated_block_announce( + &mut self, + is_best: bool, + who: PeerId, + announce: &BlockAnnounce, + ) { + let number = *announce.header.number(); + let hash = announce.header.hash(); + let parent_status = + self.block_status(announce.header.parent_hash()).unwrap_or(BlockStatus::Unknown); + let known_parent = parent_status != BlockStatus::Unknown; + let ancient_parent = parent_status == BlockStatus::InChainPruned; + + let known = self.is_known(&hash); + let peer = if let Some(peer) = self.peers.get_mut(&who) { + peer + } else { + error!(target: "sync", "💔 Called `on_validated_block_announce` with a bad peer ID"); + return + }; + + if let PeerSyncState::AncestorSearch { .. } = peer.state { + trace!(target: "sync", "Peer {} state is ancestor search.", who); + return + } + + if is_best { + // update their best block + peer.best_number = number; + peer.best_hash = hash; + } + + // If the announced block is the best they have and is not ahead of us, our common number + // is either one further ahead or it's the one they just announced, if we know about it. + if is_best { + if known && self.best_queued_number >= number { + self.update_peer_common_number(&who, number); + } else if announce.header.parent_hash() == &self.best_queued_hash || + known_parent && self.best_queued_number >= number + { + self.update_peer_common_number(&who, number - One::one()); + } + } + self.allowed_requests.add(&who); + + // known block case + if known || self.is_already_downloading(&hash) { + trace!(target: "sync", "Known block announce from {}: {}", who, hash); + if let Some(target) = self.fork_targets.get_mut(&hash) { + target.peers.insert(who); + } + return + } + + if ancient_parent { + trace!( + target: "sync", + "Ignored ancient block announced from {}: {} {:?}", + who, + hash, + announce.header, + ); + return + } + + if self.status().state == SyncState::Idle { + trace!( + target: "sync", + "Added sync target for block announced from {}: {} {:?}", + who, + hash, + announce.summary(), + ); + self.fork_targets + .entry(hash) + .or_insert_with(|| ForkTarget { + number, + parent_hash: Some(*announce.header.parent_hash()), + peers: Default::default(), + }) + .peers + .insert(who); + } + } + fn peer_disconnected(&mut self, who: &PeerId) { self.blocks.clear_peer_download(who); if let Some(gap_sync) = &mut self.gap_sync { @@ -1391,121 +1475,6 @@ where self.allowed_requests.set_all(); } - /// This will finish processing of the block announcement. - fn finish_block_announce_validation( - &mut self, - pre_validation_result: PreValidateBlockAnnounce, - ) -> PollBlockAnnounceValidation { - let (announce, is_best, who) = match pre_validation_result { - PreValidateBlockAnnounce::Failure { who, disconnect } => { - debug!( - target: "sync", - "Failed announce validation: {:?}, disconnect: {}", - who, - disconnect, - ); - return PollBlockAnnounceValidation::Failure { who, disconnect } - }, - PreValidateBlockAnnounce::Process { announce, is_new_best, who } => - (announce, is_new_best, who), - PreValidateBlockAnnounce::Error { .. } | PreValidateBlockAnnounce::Skip => { - debug!( - target: "sync", - "Ignored announce validation", - ); - return PollBlockAnnounceValidation::Skip - }, - }; - - trace!( - target: "sync", - "Finished block announce validation: from {:?}: {:?}. local_best={}", - who, - announce.summary(), - is_best, - ); - - let number = *announce.header.number(); - let hash = announce.header.hash(); - let parent_status = - self.block_status(announce.header.parent_hash()).unwrap_or(BlockStatus::Unknown); - let known_parent = parent_status != BlockStatus::Unknown; - let ancient_parent = parent_status == BlockStatus::InChainPruned; - - let known = self.is_known(&hash); - let peer = if let Some(peer) = self.peers.get_mut(&who) { - peer - } else { - error!(target: "sync", "💔 Called on_block_announce with a bad peer ID"); - return PollBlockAnnounceValidation::Nothing { is_best, who, announce } - }; - - if let PeerSyncState::AncestorSearch { .. } = peer.state { - trace!(target: "sync", "Peer state is ancestor search."); - return PollBlockAnnounceValidation::Nothing { is_best, who, announce } - } - - if is_best { - // update their best block - peer.best_number = number; - peer.best_hash = hash; - } - - // If the announced block is the best they have and is not ahead of us, our common number - // is either one further ahead or it's the one they just announced, if we know about it. - if is_best { - if known && self.best_queued_number >= number { - self.update_peer_common_number(&who, number); - } else if announce.header.parent_hash() == &self.best_queued_hash || - known_parent && self.best_queued_number >= number - { - self.update_peer_common_number(&who, number - One::one()); - } - } - self.allowed_requests.add(&who); - - // known block case - if known || self.is_already_downloading(&hash) { - trace!(target: "sync", "Known block announce from {}: {}", who, hash); - if let Some(target) = self.fork_targets.get_mut(&hash) { - target.peers.insert(who); - } - return PollBlockAnnounceValidation::Nothing { is_best, who, announce } - } - - if ancient_parent { - trace!( - target: "sync", - "Ignored ancient block announced from {}: {} {:?}", - who, - hash, - announce.header, - ); - return PollBlockAnnounceValidation::Nothing { is_best, who, announce } - } - - if self.status().state == SyncState::Idle { - trace!( - target: "sync", - "Added sync target for block announced from {}: {} {:?}", - who, - hash, - announce.summary(), - ); - self.fork_targets - .entry(hash) - .or_insert_with(|| ForkTarget { - number, - parent_hash: Some(*announce.header.parent_hash()), - peers: Default::default(), - }) - .peers - .insert(who); - } - - PollBlockAnnounceValidation::Nothing { is_best, who, announce } - } - /// Restart the sync process. This will reset all pending block requests and return an iterator /// of new block requests to make to peers. Peers that were downloading finality data (i.e. /// their state was `DownloadingJustification`) are unaffected and will stay in the same state. @@ -2901,7 +2870,7 @@ fn validate_blocks( #[cfg(test)] mod test { use super::*; - use crate::{engine::PreValidateBlockAnnounce, service::network::NetworkServiceProvider}; + use crate::service::network::NetworkServiceProvider; use futures::executor::block_on; use sc_block_builder::BlockBuilderProvider; use sc_network_common::{ @@ -3089,11 +3058,7 @@ mod test { data: Some(Vec::new()), }; - sync.finish_block_announce_validation(PreValidateBlockAnnounce::Process { - announce, - is_new_best: true, - who: peer_id, - }); + sync.on_validated_block_announce(true, peer_id, &announce); } /// Create a block response from the given `blocks`. diff --git a/client/network/sync/src/mock.rs b/client/network/sync/src/mock.rs index dfee0ee09aca5..d37095c17d2c5 100644 --- a/client/network/sync/src/mock.rs +++ b/client/network/sync/src/mock.rs @@ -22,7 +22,7 @@ use futures::task::Poll; use libp2p::PeerId; use sc_network_common::sync::{ - message::{BlockData, BlockRequest, BlockResponse}, + message::{BlockAnnounce, BlockData, BlockRequest, BlockResponse}, BadPeer, ChainSync as ChainSyncT, Metrics, OnBlockData, OnBlockJustification, OpaqueBlockResponse, PeerInfo, SyncStatus, }; @@ -71,6 +71,12 @@ mockall::mock! { success: bool, ); fn on_block_finalized(&mut self, hash: &Block::Hash, number: NumberFor); + fn on_validated_block_announce( + &mut self, + is_best: bool, + who: PeerId, + announce: &BlockAnnounce, + ); fn peer_disconnected(&mut self, who: &PeerId); fn metrics(&self) -> Metrics; fn block_response_into_blocks( From 66b205f9e942a784c30056865c476f2ca4f9bea3 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Fri, 28 Jul 2023 14:54:49 +0300 Subject: [PATCH 04/23] minor: fix visibilities --- client/network/sync/src/engine.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index ad180026c5f6d..1cae780fe3314 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -197,7 +197,7 @@ pub struct Peer { /// Result of [`SyncingEngine::block_announce_validation`]. #[derive(Debug, Clone, PartialEq, Eq)] -pub(crate) enum PreValidateBlockAnnounce { +enum PreValidateBlockAnnounce { /// The announcement failed at validation. /// /// The peer reputation should be decreased. @@ -237,7 +237,7 @@ impl PreValidateBlockAnnounce { /// Result of [`SyncingEngine::poll_block_announce_validation`]. #[derive(Debug, Clone, PartialEq, Eq)] -pub enum PollBlockAnnounceValidation { +enum PollBlockAnnounceValidation { /// The announcement failed at validation. /// /// The peer reputation should be decreased. @@ -566,7 +566,7 @@ where } /// Process the result of the block announce validation. - pub fn process_block_announce_validation_result( + fn process_block_announce_validation_result( &mut self, validation_result: PollBlockAnnounceValidation, ) { @@ -605,7 +605,7 @@ where /// in the task before being polled once. So, it is required to call /// [`SyncingEngine::poll_block_announce_validation`] to ensure that the future is /// registered properly and will wake up the task when being ready. - pub fn push_block_announce_validation( + fn push_block_announce_validation( &mut self, who: PeerId, announce: BlockAnnounce, From e48af2646f7168e0d6637946c4aa458ca8085f61 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Fri, 28 Jul 2023 15:33:01 +0300 Subject: [PATCH 05/23] Fix: poll `ChainSync` from `SyncingEngine` --- client/network/sync/src/engine.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 1cae780fe3314..2481cd54bfee3 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -605,11 +605,7 @@ where /// in the task before being polled once. So, it is required to call /// [`SyncingEngine::poll_block_announce_validation`] to ensure that the future is /// registered properly and will wake up the task when being ready. - fn push_block_announce_validation( - &mut self, - who: PeerId, - announce: BlockAnnounce, - ) { + fn push_block_announce_validation(&mut self, who: PeerId, announce: BlockAnnounce) { let hash = announce.header.hash(); let peer = match self.peers.get_mut(&who) { @@ -1069,9 +1065,13 @@ where } } - // poll `ChainSync` last because of a block announcement was received through the - // event stream between `SyncingEngine` and `Protocol` and the validation finished - // right after it as queued, the resulting block request (if any) can be sent right away. + // Drive `ChainSync`. + while let Poll::Ready(()) = self.chain_sync.poll(cx) {} + + // Poll block announce validations last, because if a block announcement was received + // through the event stream between `SyncingEngine` and `Protocol` and the validation + // finished right after it as queued, the resulting block request (if any) can be sent + // right away. while let Poll::Ready(result) = self.poll_block_announce_validation(cx) { self.process_block_announce_validation_result(result); } From 210aac4e8f5ddd51ad1bcf5e0d3026f4cf9783ca Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Fri, 28 Jul 2023 15:35:29 +0300 Subject: [PATCH 06/23] minor: spelling --- client/network/sync/src/engine.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 2481cd54bfee3..bc61882c83e3c 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -1070,7 +1070,7 @@ where // Poll block announce validations last, because if a block announcement was received // through the event stream between `SyncingEngine` and `Protocol` and the validation - // finished right after it as queued, the resulting block request (if any) can be sent + // finished right after it is queued, the resulting block request (if any) can be sent // right away. while let Poll::Ready(result) = self.poll_block_announce_validation(cx) { self.process_block_announce_validation_result(result); From 56cc4d9ffc0b186a38bda3733575c7228a7047f1 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 15:31:36 +0300 Subject: [PATCH 07/23] Unify `PeerId` variable names as `peer_id` --- client/network/sync/src/engine.rs | 182 ++++++++++++++++-------------- 1 file changed, 99 insertions(+), 83 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index bc61882c83e3c..997f899f3eb0a 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -202,8 +202,8 @@ enum PreValidateBlockAnnounce { /// /// The peer reputation should be decreased. Failure { - /// Who sent the processed block announcement? - who: PeerId, + /// The id of the peer that send us the announcement. + peer_id: PeerId, /// Should the peer be disconnected? disconnect: bool, }, @@ -213,7 +213,7 @@ enum PreValidateBlockAnnounce { /// Is this the new best block of the peer? is_new_best: bool, /// The id of the peer that send us the announcement. - who: PeerId, + peer_id: PeerId, /// The announcement. announce: BlockAnnounce, }, @@ -222,15 +222,15 @@ enum PreValidateBlockAnnounce { /// An error means that *this* node failed to validate it because some internal error happened. /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express /// this. - Error { who: PeerId }, + Error { peer_id: PeerId }, } impl PreValidateBlockAnnounce { - fn who(&self) -> &PeerId { + fn peer_id(&self) -> &PeerId { match self { - PreValidateBlockAnnounce::Failure { who, .. } | - PreValidateBlockAnnounce::Process { who, .. } | - PreValidateBlockAnnounce::Error { who } => who, + PreValidateBlockAnnounce::Failure { peer_id, .. } | + PreValidateBlockAnnounce::Process { peer_id, .. } | + PreValidateBlockAnnounce::Error { peer_id } => peer_id, } } } @@ -242,15 +242,15 @@ enum PollBlockAnnounceValidation { /// /// The peer reputation should be decreased. Failure { - /// Who sent the processed block announcement? - who: PeerId, + /// The id of the peer that send us the announcement. + peer_id: PeerId, /// Should the peer be disconnected? disconnect: bool, }, /// The announcement does not require further handling. Nothing { - /// Who sent the processed block announcement? - who: PeerId, + /// The id of the peer that send us the announcement. + peer_id: PeerId, /// Was this their new best block? is_new_best: bool, /// The announcement. @@ -556,9 +556,9 @@ where } } - fn update_peer_info(&mut self, who: &PeerId) { - if let Some(info) = self.chain_sync.peer_info(who) { - if let Some(ref mut peer) = self.peers.get_mut(who) { + fn update_peer_info(&mut self, peer_id: &PeerId) { + if let Some(info) = self.chain_sync.peer_info(peer_id) { + if let Some(ref mut peer) = self.peers.get_mut(peer_id) { peer.info.best_hash = info.best_hash; peer.info.best_number = info.best_number; } @@ -572,8 +572,8 @@ where ) { match validation_result { PollBlockAnnounceValidation::Skip => {}, - PollBlockAnnounceValidation::Nothing { is_new_best: _, who, announce } => { - self.update_peer_info(&who); + PollBlockAnnounceValidation::Nothing { is_new_best: _, peer_id, announce } => { + self.update_peer_info(&peer_id); if let Some(data) = announce.data { if !data.is_empty() { @@ -581,13 +581,13 @@ where } } }, - PollBlockAnnounceValidation::Failure { who, disconnect } => { + PollBlockAnnounceValidation::Failure { peer_id, disconnect } => { if disconnect { self.network_service - .disconnect_peer(who, self.block_announce_protocol_name.clone()); + .disconnect_peer(peer_id, self.block_announce_protocol_name.clone()); } - self.network_service.report_peer(who, rep::BAD_BLOCK_ANNOUNCEMENT); + self.network_service.report_peer(peer_id, rep::BAD_BLOCK_ANNOUNCEMENT); }, } } @@ -605,13 +605,17 @@ where /// in the task before being polled once. So, it is required to call /// [`SyncingEngine::poll_block_announce_validation`] to ensure that the future is /// registered properly and will wake up the task when being ready. - fn push_block_announce_validation(&mut self, who: PeerId, announce: BlockAnnounce) { + fn push_block_announce_validation( + &mut self, + peer_id: PeerId, + announce: BlockAnnounce, + ) { let hash = announce.header.hash(); - let peer = match self.peers.get_mut(&who) { + let peer = match self.peers.get_mut(&peer_id) { Some(p) => p, None => { - log::error!(target: "sync", "Received block announce from disconnected peer {}", who); + log::error!(target: "sync", "Received block announce from disconnected peer {}", peer_id); debug_assert!(false); return }, @@ -635,21 +639,21 @@ where "Pre-validating received block announcement {:?} with number {:?} from {}", hash, number, - who, + peer_id, ); if number.is_zero() { warn!( target: "sync", "💔 Ignored genesis block (#0) announcement from {}: {}", - who, + peer_id, hash, ); return } // Try to allocate a slot for this block announce validation. - match self.allocate_slot_for_block_announce_validation(&who) { + match self.allocate_slot_for_block_announce_validation(&peer_id) { AllocateSlotForBlockAnnounceValidation::Allocated => {}, AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { warn!( @@ -657,7 +661,7 @@ where "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", number, hash, - who, + peer_id, ); return }, @@ -667,7 +671,7 @@ where "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", number, hash, - who, + peer_id, ); return }, @@ -683,16 +687,16 @@ where Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { is_new_best: is_new_best || is_best, announce, - who, + peer_id, }, Ok(Validation::Failure { disconnect }) => { debug!( target: "sync", "Block announcement validation of block {:?} from {} failed", hash, - who, + peer_id, ); - PreValidateBlockAnnounce::Failure { who, disconnect } + PreValidateBlockAnnounce::Failure { peer_id, disconnect } }, Err(e) => { debug!( @@ -701,7 +705,7 @@ where hash, e, ); - PreValidateBlockAnnounce::Error { who } + PreValidateBlockAnnounce::Error { peer_id } }, } } @@ -716,34 +720,42 @@ where ) -> Poll> { match self.block_announce_validation.poll_next_unpin(cx) { Poll::Ready(Some(pre_validation)) => { - self.deallocate_slot_for_block_announce_validation(pre_validation.who()); + self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); let res = match pre_validation { - PreValidateBlockAnnounce::Process { is_new_best, who, announce } => { + PreValidateBlockAnnounce::Process { + is_new_best, + peer_id, + announce, + } => { trace!( target: "sync", "Finished block announce validation: from {:?}: {:?}. local_best={}", - who, + peer_id, announce.summary(), is_new_best, ); - self.chain_sync.on_validated_block_announce(is_new_best, who, &announce); - PollBlockAnnounceValidation::Nothing { is_new_best, who, announce } + self.chain_sync.on_validated_block_announce( + is_new_best, + peer_id, + &announce, + ); + PollBlockAnnounceValidation::Nothing { is_new_best, peer_id, announce } }, - PreValidateBlockAnnounce::Failure { who, disconnect } => { + PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { debug!( target: "sync", "Failed announce validation: {:?}, disconnect: {}", - who, + peer_id, disconnect, ); - PollBlockAnnounceValidation::Failure { who, disconnect } + PollBlockAnnounceValidation::Failure { peer_id, disconnect } }, - PreValidateBlockAnnounce::Error { who } => { + PreValidateBlockAnnounce::Error { peer_id } => { debug!( target: "sync", "Ignored announce validation from {:?} due to internal validation error", - who, + peer_id, ); PollBlockAnnounceValidation::Skip }, @@ -768,13 +780,13 @@ where /// validation is finished to clear the slot. fn allocate_slot_for_block_announce_validation( &mut self, - peer: &PeerId, + peer_id: &PeerId, ) -> AllocateSlotForBlockAnnounceValidation { if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { return AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached } - match self.block_announce_validation_per_peer_stats.entry(*peer) { + match self.block_announce_validation_per_peer_stats.entry(*peer_id) { Entry::Vacant(entry) => { entry.insert(1); AllocateSlotForBlockAnnounceValidation::Allocated @@ -792,13 +804,13 @@ where /// Should be called when a block announce validation is finished, to update the slots /// of the peer that send the block announce. - fn deallocate_slot_for_block_announce_validation(&mut self, peer: &PeerId) { - match self.block_announce_validation_per_peer_stats.entry(*peer) { + fn deallocate_slot_for_block_announce_validation(&mut self, peer_id: &PeerId) { + match self.block_announce_validation_per_peer_stats.entry(*peer_id) { Entry::Vacant(_) => { error!( target: "sync", "💔 Block announcement validation from peer {} finished for that no slot was allocated!", - peer, + peer_id, ); }, Entry::Occupied(mut entry) => { @@ -839,10 +851,10 @@ where .or_else(|| self.block_announce_data_cache.get(&hash).cloned()) .unwrap_or_default(); - for (who, ref mut peer) in self.peers.iter_mut() { + for (peer_id, ref mut peer) in self.peers.iter_mut() { let inserted = peer.known_blocks.insert(hash); if inserted { - log::trace!(target: "sync", "Announcing block {:?} to {}", hash, who); + log::trace!(target: "sync", "Announcing block {:?} to {}", hash, peer_id); let message = BlockAnnounce { header: header.clone(), state: if is_best { Some(BlockState::Best) } else { Some(BlockState::Normal) }, @@ -953,14 +965,14 @@ where } } }, - ToServiceCommand::JustificationImported(peer, hash, number, success) => { + ToServiceCommand::JustificationImported(peer_id, hash, number, success) => { self.chain_sync.on_justification_import(hash, number, success); if !success { - log::info!(target: "sync", "💔 Invalid justification provided by {} for #{}", peer, hash); + log::info!(target: "sync", "💔 Invalid justification provided by {} for #{}", peer_id, hash); self.network_service - .disconnect_peer(peer, self.block_announce_protocol_name.clone()); + .disconnect_peer(peer_id, self.block_announce_protocol_name.clone()); self.network_service.report_peer( - peer, + peer_id, ReputationChange::new_fatal("Invalid justification"), ); } @@ -995,8 +1007,11 @@ where let _ = tx.send(self.chain_sync.num_sync_requests()); }, ToServiceCommand::PeersInfo(tx) => { - let peers_info = - self.peers.iter().map(|(id, peer)| (*id, peer.info.clone())).collect(); + let peers_info = self + .peers + .iter() + .map(|(peer_id, peer)| (*peer_id, peer.info.clone())) + .collect(); let _ = tx.send(peers_info); }, ToServiceCommand::OnBlockFinalized(hash, header) => @@ -1082,15 +1097,15 @@ where /// Called by peer when it is disconnecting. /// /// Returns a result if the handshake of this peer was indeed accepted. - pub fn on_sync_peer_disconnected(&mut self, peer: PeerId) -> Result<(), ()> { - if let Some(info) = self.peers.remove(&peer) { - if self.important_peers.contains(&peer) { - log::warn!(target: "sync", "Reserved peer {} disconnected", peer); + pub fn on_sync_peer_disconnected(&mut self, peer_id: PeerId) -> Result<(), ()> { + if let Some(info) = self.peers.remove(&peer_id) { + if self.important_peers.contains(&peer_id) { + log::warn!(target: "sync", "Reserved peer {} disconnected", peer_id); } else { - log::debug!(target: "sync", "{} disconnected", peer); + log::debug!(target: "sync", "{} disconnected", peer_id); } - if !self.default_peers_set_no_slot_connected_peers.remove(&peer) && + if !self.default_peers_set_no_slot_connected_peers.remove(&peer_id) && info.inbound && info.info.roles.is_full() { match self.num_in_peers.checked_sub(1) { @@ -1107,9 +1122,10 @@ where } } - self.chain_sync.peer_disconnected(&peer); - self.event_streams - .retain(|stream| stream.unbounded_send(SyncEvent::PeerDisconnected(peer)).is_ok()); + self.chain_sync.peer_disconnected(&peer_id); + self.event_streams.retain(|stream| { + stream.unbounded_send(SyncEvent::PeerDisconnected(peer_id)).is_ok() + }); Ok(()) } else { Err(()) @@ -1123,35 +1139,35 @@ where /// from. pub fn on_sync_peer_connected( &mut self, - who: PeerId, + peer_id: PeerId, status: &BlockAnnouncesHandshake, sink: NotificationsSink, inbound: bool, ) -> Result<(), ()> { - log::trace!(target: "sync", "New peer {} {:?}", who, status); + log::trace!(target: "sync", "New peer {} {:?}", peer_id, status); - if self.peers.contains_key(&who) { - log::error!(target: "sync", "Called on_sync_peer_connected with already connected peer {}", who); + if self.peers.contains_key(&peer_id) { + log::error!(target: "sync", "Called on_sync_peer_connected with already connected peer {}", peer_id); debug_assert!(false); return Err(()) } if status.genesis_hash != self.genesis_hash { - self.network_service.report_peer(who, rep::GENESIS_MISMATCH); + self.network_service.report_peer(peer_id, rep::GENESIS_MISMATCH); - if self.important_peers.contains(&who) { + if self.important_peers.contains(&peer_id) { log::error!( target: "sync", "Reserved peer id `{}` is on a different chain (our genesis: {} theirs: {})", - who, + peer_id, self.genesis_hash, status.genesis_hash, ); - } else if self.boot_node_ids.contains(&who) { + } else if self.boot_node_ids.contains(&peer_id) { log::error!( target: "sync", "Bootnode with peer id `{}` is on a different chain (our genesis: {} theirs: {})", - who, + peer_id, self.genesis_hash, status.genesis_hash, ); @@ -1166,7 +1182,7 @@ where return Err(()) } - let no_slot_peer = self.default_peers_set_no_slot_peers.contains(&who); + let no_slot_peer = self.default_peers_set_no_slot_peers.contains(&peer_id); let this_peer_reserved_slot: usize = if no_slot_peer { 1 } else { 0 }; // make sure to accept no more than `--in-peers` many full nodes @@ -1174,7 +1190,7 @@ where status.roles.is_full() && inbound && self.num_in_peers == self.max_in_peers { - log::debug!(target: "sync", "All inbound slots have been consumed, rejecting {who}"); + log::debug!(target: "sync", "All inbound slots have been consumed, rejecting {peer_id}"); return Err(()) } @@ -1184,7 +1200,7 @@ where self.default_peers_set_no_slot_connected_peers.len() + this_peer_reserved_slot { - log::debug!(target: "sync", "Too many full nodes, rejecting {}", who); + log::debug!(target: "sync", "Too many full nodes, rejecting {}", peer_id); return Err(()) } @@ -1192,7 +1208,7 @@ where (self.peers.len() - self.chain_sync.num_peers()) >= self.default_peers_set_num_light { // Make sure that not all slots are occupied by light clients. - log::debug!(target: "sync", "Too many light nodes, rejecting {}", who); + log::debug!(target: "sync", "Too many light nodes, rejecting {}", peer_id); return Err(()) } @@ -1212,7 +1228,7 @@ where }; let req = if peer.info.roles.is_full() { - match self.chain_sync.new_peer(who, peer.info.best_hash, peer.info.best_number) { + match self.chain_sync.new_peer(peer_id, peer.info.best_hash, peer.info.best_number) { Ok(req) => req, Err(BadPeer(id, repu)) => { self.network_service.report_peer(id, repu); @@ -1223,22 +1239,22 @@ where None }; - log::debug!(target: "sync", "Connected {}", who); + log::debug!(target: "sync", "Connected {}", peer_id); - self.peers.insert(who, peer); + self.peers.insert(peer_id, peer); if no_slot_peer { - self.default_peers_set_no_slot_connected_peers.insert(who); + self.default_peers_set_no_slot_connected_peers.insert(peer_id); } else if inbound && status.roles.is_full() { self.num_in_peers += 1; } if let Some(req) = req { - self.chain_sync.send_block_request(who, req); + self.chain_sync.send_block_request(peer_id, req); } self.event_streams - .retain(|stream| stream.unbounded_send(SyncEvent::PeerConnected(who)).is_ok()); + .retain(|stream| stream.unbounded_send(SyncEvent::PeerConnected(peer_id)).is_ok()); Ok(()) } From 919e84c52439ea61f3eda0760d9313d54a3360fb Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 15:49:20 +0300 Subject: [PATCH 08/23] Assert when deallocating block announce validation slot counter --- client/network/sync/src/engine.rs | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 997f899f3eb0a..642a8115f0382 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -814,9 +814,26 @@ where ); }, Entry::Occupied(mut entry) => { - *entry.get_mut() = entry.get().saturating_sub(1); - if *entry.get() == 0 { - entry.remove(); + match entry.get().checked_sub(1) { + Some(value) => { + if value == 0 { + entry.remove(); + } else { + *entry.get_mut() = value; + } + } + None => { + entry.remove(); + + error!( + target: "sync", + "Invalid (zero) block announce validation slot counter for peer {peer_id}.", + ); + debug_assert!( + false, + "Invalid (zero) block announce validation slot counter for peer {peer_id}.", + ); + } } }, } From 8c26a6a1e390d9a71b2ed577f2e162c508fc12b0 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 17:12:29 +0300 Subject: [PATCH 09/23] Extract block announce validation from `SyncingEngine` --- .../sync/src/block_announce_validator.rs | 346 +++++++++++++++++ client/network/sync/src/engine.rs | 354 ++---------------- client/network/sync/src/lib.rs | 1 + 3 files changed, 379 insertions(+), 322 deletions(-) create mode 100644 client/network/sync/src/block_announce_validator.rs diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs new file mode 100644 index 0000000000000..1394c540b6181 --- /dev/null +++ b/client/network/sync/src/block_announce_validator.rs @@ -0,0 +1,346 @@ +// This file is part of Substrate. + +// Copyright (C) 2017-2023 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 . + +//! `BlockAnnounceValidator` is responsible for async validation of block announcements. + +use futures::{stream::FuturesUnordered, Future, FutureExt, StreamExt}; +use libp2p::PeerId; +use log::{debug, error, trace, warn}; +use sc_network_common::sync::message::BlockAnnounce; +use sp_consensus::block_validation::Validation; +use sp_runtime::traits::{Block as BlockT, Header, Zero}; +use std::{ + collections::{hash_map::Entry, HashMap}, + pin::Pin, + task::Poll, +}; + +/// Maximum number of concurrent block announce validations. +/// +/// If the queue reaches the maximum, we drop any new block +/// announcements. +const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS: usize = 256; + +/// Maximum number of concurrent block announce validations per peer. +/// +/// See [`MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS`] for more information. +const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER: usize = 4; + +/// Result of [`BlockAnnounceValidator::block_announce_validation`]. +#[derive(Debug, Clone, PartialEq, Eq)] +enum PreValidateBlockAnnounce { + /// The announcement failed at validation. + /// + /// The peer reputation should be decreased. + Failure { + /// The id of the peer that send us the announcement. + peer_id: PeerId, + /// Should the peer be disconnected? + disconnect: bool, + }, + /// The pre-validation was sucessful and the announcement should be + /// further processed. + Process { + /// Is this the new best block of the peer? + is_new_best: bool, + /// The id of the peer that send us the announcement. + peer_id: PeerId, + /// The announcement. + announce: BlockAnnounce, + }, + /// The announcement validation returned an error. + /// + /// An error means that *this* node failed to validate it because some internal error happened. + /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express + /// this. + Error { peer_id: PeerId }, +} + +impl PreValidateBlockAnnounce { + fn peer_id(&self) -> &PeerId { + match self { + PreValidateBlockAnnounce::Failure { peer_id, .. } | + PreValidateBlockAnnounce::Process { peer_id, .. } | + PreValidateBlockAnnounce::Error { peer_id } => peer_id, + } + } +} + +/// Result of [`BlockAnnounceValidator::poll_block_announce_validation`]. +#[derive(Debug, Clone, PartialEq, Eq)] +pub(crate) enum PollBlockAnnounceValidation { + /// The announcement failed at validation. + /// + /// The peer reputation should be decreased. + Failure { + /// The id of the peer that send us the announcement. + peer_id: PeerId, + /// Should the peer be disconnected? + disconnect: bool, + }, + /// The announcement was validated successfully and should be passed to [`crate::ChainSync`]. + Process { + /// The id of the peer that send us the announcement. + peer_id: PeerId, + /// Was this their new best block? + is_new_best: bool, + /// The announcement. + announce: BlockAnnounce, + }, + /// The block announcement should be skipped. + Skip, +} + +/// Result of [`BlockAnnounceValidator::allocate_slot_for_block_announce_validation`]. +enum AllocateSlotForBlockAnnounceValidation { + /// Success, there is a slot for the block announce validation. + Allocated, + /// We reached the total maximum number of validation slots. + TotalMaximumSlotsReached, + /// We reached the maximum number of validation slots for the given peer. + MaximumPeerSlotsReached, +} + +pub(crate) struct BlockAnnounceValidator { + /// A type to check incoming block announcements. + block_announce_validator: + Box + Send>, + + /// All block announcements that are currently being validated. + block_announce_validation: + FuturesUnordered> + Send>>>, + + /// Stats per peer about the number of concurrent block announce validations. + block_announce_validation_per_peer_stats: HashMap, +} + +impl BlockAnnounceValidator { + pub(crate) fn new( + validator: Box + Send>, + ) -> Self { + Self { + block_announce_validator: validator, + block_announce_validation: Default::default(), + block_announce_validation_per_peer_stats: Default::default(), + } + } + + /// Push a block announce validation. + pub(crate) fn push_block_announce_validation( + &mut self, + peer_id: PeerId, + hash: B::Hash, + announce: BlockAnnounce, + is_best: bool, + ) { + let header = &announce.header; + let number = *header.number(); + debug!( + target: "sync", + "Pre-validating received block announcement {:?} with number {:?} from {}", + hash, + number, + peer_id, + ); + + if number.is_zero() { + warn!( + target: "sync", + "💔 Ignored genesis block (#0) announcement from {}: {}", + peer_id, + hash, + ); + return + } + + // Try to allocate a slot for this block announce validation. + match self.allocate_slot_for_block_announce_validation(&peer_id) { + AllocateSlotForBlockAnnounceValidation::Allocated => {}, + AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { + warn!( + target: "sync", + "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", + number, + hash, + peer_id, + ); + return + }, + AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { + warn!( + target: "sync", + "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", + number, + hash, + peer_id, + ); + return + }, + } + + // Let external validator check the block announcement. + let assoc_data = announce.data.as_ref().map_or(&[][..], |v| v.as_slice()); + let future = self.block_announce_validator.validate(header, assoc_data); + + self.block_announce_validation.push( + async move { + match future.await { + Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { + is_new_best: is_new_best || is_best, + announce, + peer_id, + }, + Ok(Validation::Failure { disconnect }) => { + debug!( + target: "sync", + "Block announcement validation of block {:?} from {} failed", + hash, + peer_id, + ); + PreValidateBlockAnnounce::Failure { peer_id, disconnect } + }, + Err(e) => { + debug!( + target: "sync", + "💔 Block announcement validation of block {:?} errored: {}", + hash, + e, + ); + PreValidateBlockAnnounce::Error { peer_id } + }, + } + } + .boxed(), + ); + } + + /// Poll for finished block announce validations and notify `ChainSync`. + pub(crate) fn poll_block_announce_validation( + &mut self, + cx: &mut std::task::Context, + ) -> Poll> { + match self.block_announce_validation.poll_next_unpin(cx) { + Poll::Ready(Some(pre_validation)) => { + self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); + + let res = match pre_validation { + PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { + trace!( + target: "sync", + "Finished block announce validation: from {:?}: {:?}. local_best={}", + peer_id, + announce.summary(), + is_new_best, + ); + + PollBlockAnnounceValidation::Process { is_new_best, peer_id, announce } + }, + PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { + debug!( + target: "sync", + "Failed announce validation: {:?}, disconnect: {}", + peer_id, + disconnect, + ); + PollBlockAnnounceValidation::Failure { peer_id, disconnect } + }, + PreValidateBlockAnnounce::Error { peer_id } => { + debug!( + target: "sync", + "Ignored announce validation from {:?} due to internal validation error", + peer_id, + ); + PollBlockAnnounceValidation::Skip + }, + }; + + Poll::Ready(res) + }, + _ => Poll::Pending, + } + } + + /// Checks if there is a slot for a block announce validation. + /// + /// The total number and the number per peer of concurrent block announce validations + /// is capped. + /// + /// Returns [`AllocateSlotForBlockAnnounceValidation`] to inform about the result. + /// + /// # Note + /// + /// It is *required* to call [`Self::deallocate_slot_for_block_announce_validation`] when the + /// validation is finished to clear the slot. + fn allocate_slot_for_block_announce_validation( + &mut self, + peer_id: &PeerId, + ) -> AllocateSlotForBlockAnnounceValidation { + if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { + return AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached + } + + match self.block_announce_validation_per_peer_stats.entry(*peer_id) { + Entry::Vacant(entry) => { + entry.insert(1); + AllocateSlotForBlockAnnounceValidation::Allocated + }, + Entry::Occupied(mut entry) => { + if *entry.get() < MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { + *entry.get_mut() += 1; + AllocateSlotForBlockAnnounceValidation::Allocated + } else { + AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached + } + }, + } + } + + /// Should be called when a block announce validation is finished, to update the slots + /// of the peer that send the block announce. + fn deallocate_slot_for_block_announce_validation(&mut self, peer_id: &PeerId) { + match self.block_announce_validation_per_peer_stats.entry(*peer_id) { + Entry::Vacant(_) => { + error!( + target: "sync", + "💔 Block announcement validation from peer {} finished for that no slot was allocated!", + peer_id, + ); + }, + Entry::Occupied(mut entry) => match entry.get().checked_sub(1) { + Some(value) => + if value == 0 { + entry.remove(); + } else { + *entry.get_mut() = value; + }, + None => { + entry.remove(); + + error!( + target: "sync", + "Invalid (zero) block announce validation slot counter for peer {peer_id}.", + ); + debug_assert!( + false, + "Invalid (zero) block announce validation slot counter for peer {peer_id}.", + ); + }, + }, + } + } +} diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 642a8115f0382..38aabf982679f 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -20,15 +20,17 @@ //! to tip and keep the blockchain up to date with network updates. use crate::{ + block_announce_validator::{ + BlockAnnounceValidator as BlockAnnounceValidatorStream, PollBlockAnnounceValidation, + }, service::{self, chain_sync::ToServiceCommand}, ChainSync, ClientError, SyncingService, }; use codec::{Decode, Encode}; -use futures::{stream::FuturesUnordered, Future, FutureExt, StreamExt}; +use futures::{FutureExt, StreamExt}; use futures_timer::Delay; use libp2p::PeerId; -use log::{debug, error, trace, warn}; use prometheus_endpoint::{ register, Gauge, GaugeVec, MetricSource, Opts, PrometheusError, Registry, SourcedGauge, U64, }; @@ -51,13 +53,12 @@ use sc_network_common::{ }; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_blockchain::HeaderMetadata; -use sp_consensus::block_validation::{BlockAnnounceValidator, Validation}; +use sp_consensus::block_validation::BlockAnnounceValidator; use sp_runtime::traits::{Block as BlockT, Header, NumberFor, Zero}; use std::{ - collections::{hash_map::Entry, HashMap, HashSet}, + collections::{HashMap, HashSet}, num::NonZeroUsize, - pin::Pin, sync::{ atomic::{AtomicBool, AtomicUsize, Ordering}, Arc, @@ -72,17 +73,6 @@ const TICK_TIMEOUT: std::time::Duration = std::time::Duration::from_millis(1100) /// Maximum number of known block hashes to keep for a peer. const MAX_KNOWN_BLOCKS: usize = 1024; // ~32kb per peer + LruHashSet overhead -/// Maximum number of concurrent block announce validations. -/// -/// If the queue reaches the maximum, we drop any new block -/// announcements. -const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS: usize = 256; - -/// Maximum number of concurrent block announce validations per peer. -/// -/// See [`MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS`] for more information. -const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER: usize = 4; - /// If the block announces stream to peer has been inactive for 30 seconds meaning local node /// has not sent or received block announcements to/from the peer, report the node for inactivity, /// disconnect it and attempt to establish connection to some other peer. @@ -195,80 +185,6 @@ pub struct Peer { inbound: bool, } -/// Result of [`SyncingEngine::block_announce_validation`]. -#[derive(Debug, Clone, PartialEq, Eq)] -enum PreValidateBlockAnnounce { - /// The announcement failed at validation. - /// - /// The peer reputation should be decreased. - Failure { - /// The id of the peer that send us the announcement. - peer_id: PeerId, - /// Should the peer be disconnected? - disconnect: bool, - }, - /// The pre-validation was sucessful and the announcement should be - /// further processed. - Process { - /// Is this the new best block of the peer? - is_new_best: bool, - /// The id of the peer that send us the announcement. - peer_id: PeerId, - /// The announcement. - announce: BlockAnnounce, - }, - /// The announcement validation returned an error. - /// - /// An error means that *this* node failed to validate it because some internal error happened. - /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express - /// this. - Error { peer_id: PeerId }, -} - -impl PreValidateBlockAnnounce { - fn peer_id(&self) -> &PeerId { - match self { - PreValidateBlockAnnounce::Failure { peer_id, .. } | - PreValidateBlockAnnounce::Process { peer_id, .. } | - PreValidateBlockAnnounce::Error { peer_id } => peer_id, - } - } -} - -/// Result of [`SyncingEngine::poll_block_announce_validation`]. -#[derive(Debug, Clone, PartialEq, Eq)] -enum PollBlockAnnounceValidation { - /// The announcement failed at validation. - /// - /// The peer reputation should be decreased. - Failure { - /// The id of the peer that send us the announcement. - peer_id: PeerId, - /// Should the peer be disconnected? - disconnect: bool, - }, - /// The announcement does not require further handling. - Nothing { - /// The id of the peer that send us the announcement. - peer_id: PeerId, - /// Was this their new best block? - is_new_best: bool, - /// The announcement. - announce: BlockAnnounce, - }, - /// The block announcement should be skipped. - Skip, -} - -/// Result of [`SyncingEngine::allocate_slot_for_block_announce_validation`]. -enum AllocateSlotForBlockAnnounceValidation { - /// Success, there is a slot for the block announce validation. - Allocated, - /// We reached the total maximum number of validation slots. - TotalMaximumSlotsReached, - /// We reached the maximum number of validation slots for the given peer. - MaximumPeerSlotsReached, -} pub struct SyncingEngine { /// State machine that handles the list of in-progress requests. Only full node peers are /// registered. @@ -333,19 +249,12 @@ pub struct SyncingEngine { /// Number of inbound peers accepted so far. num_in_peers: usize, + /// Async processor of block announce validations. + block_announce_validator: BlockAnnounceValidatorStream, + /// A cache for the data that was associated to a block announcement. block_announce_data_cache: LruMap>, - /// A type to check incoming block announcements. - block_announce_validator: Box + Send>, - - /// All block announcement that are currently being validated. - block_announce_validation: - FuturesUnordered> + Send>>>, - - /// Stats per peer about the number of concurrent block announce validations. - block_announce_validation_per_peer_stats: HashMap, - /// The `PeerId`'s of all boot nodes. boot_node_ids: HashSet, @@ -489,10 +398,10 @@ where peers: HashMap::new(), evicted: HashSet::new(), block_announce_data_cache: LruMap::new(ByLength::new(cache_capacity)), - block_announce_validator, - block_announce_validation: Default::default(), - block_announce_validation_per_peer_stats: Default::default(), block_announce_protocol_name, + block_announce_validator: BlockAnnounceValidatorStream::new( + block_announce_validator, + ), num_connected: num_connected.clone(), is_major_syncing: is_major_syncing.clone(), service_rx, @@ -572,7 +481,9 @@ where ) { match validation_result { PollBlockAnnounceValidation::Skip => {}, - PollBlockAnnounceValidation::Nothing { is_new_best: _, peer_id, announce } => { + PollBlockAnnounceValidation::Process { is_new_best, peer_id, announce } => { + self.chain_sync.on_validated_block_announce(is_new_best, peer_id, &announce); + self.update_peer_info(&peer_id); if let Some(data) = announce.data { @@ -594,7 +505,7 @@ where /// Push a block announce validation. /// - /// It is required that [`SyncingEngine::poll_block_announce_validation`] is + /// It is required that [`BlockAnnounceValidatorStream::poll_block_announce_validation`] is /// called later to check for finished validations. The result of the validation /// needs to be passed to [`SyncingEngine::process_block_announce_validation_result`] /// to finish the processing. @@ -603,9 +514,9 @@ where /// /// This will internally create a future, but this future will not be registered /// in the task before being polled once. So, it is required to call - /// [`SyncingEngine::poll_block_announce_validation`] to ensure that the future is - /// registered properly and will wake up the task when being ready. - fn push_block_announce_validation( + /// [`BlockAnnounceValidatorStream::poll_block_announce_validation`] to ensure that the future + /// is registered properly and will wake up the task when being ready. + pub fn push_block_announce_validation( &mut self, peer_id: PeerId, announce: BlockAnnounce, @@ -623,219 +534,14 @@ where peer.known_blocks.insert(hash); peer.last_notification_received = Instant::now(); - if !peer.info.roles.is_full() { - return - } - - let is_best = match announce.state.unwrap_or(BlockState::Best) { - BlockState::Best => true, - BlockState::Normal => false, - }; + if peer.info.roles.is_full() { + let is_best = match announce.state.unwrap_or(BlockState::Best) { + BlockState::Best => true, + BlockState::Normal => false, + }; - let header = &announce.header; - let number = *header.number(); - debug!( - target: "sync", - "Pre-validating received block announcement {:?} with number {:?} from {}", - hash, - number, - peer_id, - ); - - if number.is_zero() { - warn!( - target: "sync", - "💔 Ignored genesis block (#0) announcement from {}: {}", - peer_id, - hash, - ); - return - } - - // Try to allocate a slot for this block announce validation. - match self.allocate_slot_for_block_announce_validation(&peer_id) { - AllocateSlotForBlockAnnounceValidation::Allocated => {}, - AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { - warn!( - target: "sync", - "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", - number, - hash, - peer_id, - ); - return - }, - AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { - warn!( - target: "sync", - "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", - number, - hash, - peer_id, - ); - return - }, - } - - // Let external validator check the block announcement. - let assoc_data = announce.data.as_ref().map_or(&[][..], |v| v.as_slice()); - let future = self.block_announce_validator.validate(header, assoc_data); - - self.block_announce_validation.push( - async move { - match future.await { - Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { - is_new_best: is_new_best || is_best, - announce, - peer_id, - }, - Ok(Validation::Failure { disconnect }) => { - debug!( - target: "sync", - "Block announcement validation of block {:?} from {} failed", - hash, - peer_id, - ); - PreValidateBlockAnnounce::Failure { peer_id, disconnect } - }, - Err(e) => { - debug!( - target: "sync", - "💔 Block announcement validation of block {:?} errored: {}", - hash, - e, - ); - PreValidateBlockAnnounce::Error { peer_id } - }, - } - } - .boxed(), - ); - } - - /// Poll for finished block announce validations and notify `ChainSync`. - fn poll_block_announce_validation( - &mut self, - cx: &mut std::task::Context, - ) -> Poll> { - match self.block_announce_validation.poll_next_unpin(cx) { - Poll::Ready(Some(pre_validation)) => { - self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); - - let res = match pre_validation { - PreValidateBlockAnnounce::Process { - is_new_best, - peer_id, - announce, - } => { - trace!( - target: "sync", - "Finished block announce validation: from {:?}: {:?}. local_best={}", - peer_id, - announce.summary(), - is_new_best, - ); - self.chain_sync.on_validated_block_announce( - is_new_best, - peer_id, - &announce, - ); - PollBlockAnnounceValidation::Nothing { is_new_best, peer_id, announce } - }, - PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { - debug!( - target: "sync", - "Failed announce validation: {:?}, disconnect: {}", - peer_id, - disconnect, - ); - PollBlockAnnounceValidation::Failure { peer_id, disconnect } - }, - PreValidateBlockAnnounce::Error { peer_id } => { - debug!( - target: "sync", - "Ignored announce validation from {:?} due to internal validation error", - peer_id, - ); - PollBlockAnnounceValidation::Skip - }, - }; - - Poll::Ready(res) - }, - _ => Poll::Pending, - } - } - - /// Checks if there is a slot for a block announce validation. - /// - /// The total number and the number per peer of concurrent block announce validations - /// is capped. - /// - /// Returns [`AllocateSlotForBlockAnnounceValidation`] to inform about the result. - /// - /// # Note - /// - /// It is *required* to call [`Self::deallocate_slot_for_block_announce_validation`] when the - /// validation is finished to clear the slot. - fn allocate_slot_for_block_announce_validation( - &mut self, - peer_id: &PeerId, - ) -> AllocateSlotForBlockAnnounceValidation { - if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { - return AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached - } - - match self.block_announce_validation_per_peer_stats.entry(*peer_id) { - Entry::Vacant(entry) => { - entry.insert(1); - AllocateSlotForBlockAnnounceValidation::Allocated - }, - Entry::Occupied(mut entry) => { - if *entry.get() < MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { - *entry.get_mut() += 1; - AllocateSlotForBlockAnnounceValidation::Allocated - } else { - AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached - } - }, - } - } - - /// Should be called when a block announce validation is finished, to update the slots - /// of the peer that send the block announce. - fn deallocate_slot_for_block_announce_validation(&mut self, peer_id: &PeerId) { - match self.block_announce_validation_per_peer_stats.entry(*peer_id) { - Entry::Vacant(_) => { - error!( - target: "sync", - "💔 Block announcement validation from peer {} finished for that no slot was allocated!", - peer_id, - ); - }, - Entry::Occupied(mut entry) => { - match entry.get().checked_sub(1) { - Some(value) => { - if value == 0 { - entry.remove(); - } else { - *entry.get_mut() = value; - } - } - None => { - entry.remove(); - - error!( - target: "sync", - "Invalid (zero) block announce validation slot counter for peer {peer_id}.", - ); - debug_assert!( - false, - "Invalid (zero) block announce validation slot counter for peer {peer_id}.", - ); - } - } - }, + self.block_announce_validator + .push_block_announce_validation(peer_id, hash, announce, is_best); } } @@ -1074,7 +780,9 @@ where // Make sure that the newly added block announce validation future // was polled once to be registered in the task. - if let Poll::Ready(res) = self.poll_block_announce_validation(cx) { + if let Poll::Ready(res) = + self.block_announce_validator.poll_block_announce_validation(cx) + { self.process_block_announce_validation_result(res) } } else { @@ -1104,7 +812,9 @@ where // through the event stream between `SyncingEngine` and `Protocol` and the validation // finished right after it is queued, the resulting block request (if any) can be sent // right away. - while let Poll::Ready(result) = self.poll_block_announce_validation(cx) { + while let Poll::Ready(result) = + self.block_announce_validator.poll_block_announce_validation(cx) + { self.process_block_announce_validation_result(result); } diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index b7e4ff4c93501..7bfe150257a85 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -88,6 +88,7 @@ use std::{ pub use service::chain_sync::SyncingService; +mod block_announce_validator; mod extra_requests; mod schema; From 49bac21bef663914b2de5c093f9c1404b5326922 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 17:32:30 +0300 Subject: [PATCH 10/23] Simplify naming in `BlockAnnounceValidator` --- .../sync/src/block_announce_validator.rs | 37 +++++++++---------- client/network/sync/src/engine.rs | 10 ++--- 2 files changed, 23 insertions(+), 24 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index 1394c540b6181..70490bb21669e 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -83,7 +83,7 @@ impl PreValidateBlockAnnounce { /// Result of [`BlockAnnounceValidator::poll_block_announce_validation`]. #[derive(Debug, Clone, PartialEq, Eq)] -pub(crate) enum PollBlockAnnounceValidation { +pub(crate) enum BlockAnnounceValidationResult { /// The announcement failed at validation. /// /// The peer reputation should be decreased. @@ -118,15 +118,14 @@ enum AllocateSlotForBlockAnnounceValidation { pub(crate) struct BlockAnnounceValidator { /// A type to check incoming block announcements. - block_announce_validator: - Box + Send>, + validator: Box + Send>, /// All block announcements that are currently being validated. - block_announce_validation: + validations: FuturesUnordered> + Send>>>, - /// Stats per peer about the number of concurrent block announce validations. - block_announce_validation_per_peer_stats: HashMap, + /// Number of concurrent block announce validations per peer. + validations_per_peer: HashMap, } impl BlockAnnounceValidator { @@ -134,9 +133,9 @@ impl BlockAnnounceValidator { validator: Box + Send>, ) -> Self { Self { - block_announce_validator: validator, - block_announce_validation: Default::default(), - block_announce_validation_per_peer_stats: Default::default(), + validator, + validations: Default::default(), + validations_per_peer: Default::default(), } } @@ -195,9 +194,9 @@ impl BlockAnnounceValidator { // Let external validator check the block announcement. let assoc_data = announce.data.as_ref().map_or(&[][..], |v| v.as_slice()); - let future = self.block_announce_validator.validate(header, assoc_data); + let future = self.validator.validate(header, assoc_data); - self.block_announce_validation.push( + self.validations.push( async move { match future.await { Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { @@ -233,8 +232,8 @@ impl BlockAnnounceValidator { pub(crate) fn poll_block_announce_validation( &mut self, cx: &mut std::task::Context, - ) -> Poll> { - match self.block_announce_validation.poll_next_unpin(cx) { + ) -> Poll> { + match self.validations.poll_next_unpin(cx) { Poll::Ready(Some(pre_validation)) => { self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); @@ -248,7 +247,7 @@ impl BlockAnnounceValidator { is_new_best, ); - PollBlockAnnounceValidation::Process { is_new_best, peer_id, announce } + BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } }, PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { debug!( @@ -257,7 +256,7 @@ impl BlockAnnounceValidator { peer_id, disconnect, ); - PollBlockAnnounceValidation::Failure { peer_id, disconnect } + BlockAnnounceValidationResult::Failure { peer_id, disconnect } }, PreValidateBlockAnnounce::Error { peer_id } => { debug!( @@ -265,7 +264,7 @@ impl BlockAnnounceValidator { "Ignored announce validation from {:?} due to internal validation error", peer_id, ); - PollBlockAnnounceValidation::Skip + BlockAnnounceValidationResult::Skip }, }; @@ -290,11 +289,11 @@ impl BlockAnnounceValidator { &mut self, peer_id: &PeerId, ) -> AllocateSlotForBlockAnnounceValidation { - if self.block_announce_validation.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { + if self.validations.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { return AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached } - match self.block_announce_validation_per_peer_stats.entry(*peer_id) { + match self.validations_per_peer.entry(*peer_id) { Entry::Vacant(entry) => { entry.insert(1); AllocateSlotForBlockAnnounceValidation::Allocated @@ -313,7 +312,7 @@ impl BlockAnnounceValidator { /// Should be called when a block announce validation is finished, to update the slots /// of the peer that send the block announce. fn deallocate_slot_for_block_announce_validation(&mut self, peer_id: &PeerId) { - match self.block_announce_validation_per_peer_stats.entry(*peer_id) { + match self.validations_per_peer.entry(*peer_id) { Entry::Vacant(_) => { error!( target: "sync", diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 38aabf982679f..f3a1893776fe1 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -21,7 +21,7 @@ use crate::{ block_announce_validator::{ - BlockAnnounceValidator as BlockAnnounceValidatorStream, PollBlockAnnounceValidation, + BlockAnnounceValidationResult, BlockAnnounceValidator as BlockAnnounceValidatorStream, }, service::{self, chain_sync::ToServiceCommand}, ChainSync, ClientError, SyncingService, @@ -477,11 +477,11 @@ where /// Process the result of the block announce validation. fn process_block_announce_validation_result( &mut self, - validation_result: PollBlockAnnounceValidation, + validation_result: BlockAnnounceValidationResult, ) { match validation_result { - PollBlockAnnounceValidation::Skip => {}, - PollBlockAnnounceValidation::Process { is_new_best, peer_id, announce } => { + BlockAnnounceValidationResult::Skip => {}, + BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } => { self.chain_sync.on_validated_block_announce(is_new_best, peer_id, &announce); self.update_peer_info(&peer_id); @@ -492,7 +492,7 @@ where } } }, - PollBlockAnnounceValidation::Failure { peer_id, disconnect } => { + BlockAnnounceValidationResult::Failure { peer_id, disconnect } => { if disconnect { self.network_service .disconnect_peer(peer_id, self.block_announce_protocol_name.clone()); From 76af1c00a6566fffa028c90b490ea275b1ec8174 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 18:19:28 +0300 Subject: [PATCH 11/23] Implement `Stream` trait for `BlockAnnounceValidator` --- .../sync/src/block_announce_validator.rs | 97 ++++++++++--------- client/network/sync/src/engine.rs | 8 +- 2 files changed, 52 insertions(+), 53 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index 70490bb21669e..396307e323656 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -18,7 +18,7 @@ //! `BlockAnnounceValidator` is responsible for async validation of block announcements. -use futures::{stream::FuturesUnordered, Future, FutureExt, StreamExt}; +use futures::{stream::FuturesUnordered, Future, FutureExt, Stream, StreamExt}; use libp2p::PeerId; use log::{debug, error, trace, warn}; use sc_network_common::sync::message::BlockAnnounce; @@ -27,7 +27,7 @@ use sp_runtime::traits::{Block as BlockT, Header, Zero}; use std::{ collections::{hash_map::Entry, HashMap}, pin::Pin, - task::Poll, + task::{Context, Poll}, }; /// Maximum number of concurrent block announce validations. @@ -228,52 +228,6 @@ impl BlockAnnounceValidator { ); } - /// Poll for finished block announce validations and notify `ChainSync`. - pub(crate) fn poll_block_announce_validation( - &mut self, - cx: &mut std::task::Context, - ) -> Poll> { - match self.validations.poll_next_unpin(cx) { - Poll::Ready(Some(pre_validation)) => { - self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); - - let res = match pre_validation { - PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { - trace!( - target: "sync", - "Finished block announce validation: from {:?}: {:?}. local_best={}", - peer_id, - announce.summary(), - is_new_best, - ); - - BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } - }, - PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { - debug!( - target: "sync", - "Failed announce validation: {:?}, disconnect: {}", - peer_id, - disconnect, - ); - BlockAnnounceValidationResult::Failure { peer_id, disconnect } - }, - PreValidateBlockAnnounce::Error { peer_id } => { - debug!( - target: "sync", - "Ignored announce validation from {:?} due to internal validation error", - peer_id, - ); - BlockAnnounceValidationResult::Skip - }, - }; - - Poll::Ready(res) - }, - _ => Poll::Pending, - } - } - /// Checks if there is a slot for a block announce validation. /// /// The total number and the number per peer of concurrent block announce validations @@ -343,3 +297,50 @@ impl BlockAnnounceValidator { } } } + +impl Stream for BlockAnnounceValidator { + type Item = BlockAnnounceValidationResult; + + /// Poll for finished block announce validations. The stream never terminates. + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + match self.validations.poll_next_unpin(cx) { + Poll::Ready(Some(pre_validation)) => { + self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); + + let res = match pre_validation { + PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { + trace!( + target: "sync", + "Finished block announce validation: from {:?}: {:?}. local_best={}", + peer_id, + announce.summary(), + is_new_best, + ); + + BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } + }, + PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { + debug!( + target: "sync", + "Failed announce validation: {:?}, disconnect: {}", + peer_id, + disconnect, + ); + BlockAnnounceValidationResult::Failure { peer_id, disconnect } + }, + PreValidateBlockAnnounce::Error { peer_id } => { + debug!( + target: "sync", + "Ignored announce validation from {:?} due to internal validation error", + peer_id, + ); + BlockAnnounceValidationResult::Skip + }, + }; + + Poll::Ready(Some(res)) + }, + _ => Poll::Pending, + } + } +} diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index f3a1893776fe1..bc46a381466f1 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -780,8 +780,8 @@ where // Make sure that the newly added block announce validation future // was polled once to be registered in the task. - if let Poll::Ready(res) = - self.block_announce_validator.poll_block_announce_validation(cx) + if let Poll::Ready(Some(res)) = + self.block_announce_validator.poll_next_unpin(cx) { self.process_block_announce_validation_result(res) } @@ -812,9 +812,7 @@ where // through the event stream between `SyncingEngine` and `Protocol` and the validation // finished right after it is queued, the resulting block request (if any) can be sent // right away. - while let Poll::Ready(result) = - self.block_announce_validator.poll_block_announce_validation(cx) - { + while let Poll::Ready(Some(result)) = self.block_announce_validator.poll_next_unpin(cx) { self.process_block_announce_validation_result(result); } From dbdb22dfd0e761ee16155a7afd7b5ecff54da23a Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 20:31:48 +0300 Subject: [PATCH 12/23] minor: simplify code a little --- .../sync/src/block_announce_validator.rs | 73 ++++++++++--------- 1 file changed, 39 insertions(+), 34 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index 396307e323656..3aedc79751119 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -81,7 +81,7 @@ impl PreValidateBlockAnnounce { } } -/// Result of [`BlockAnnounceValidator::poll_block_announce_validation`]. +/// Item that yields [`Stream`] implementation of [`BlockAnnounceValidator`]. #[derive(Debug, Clone, PartialEq, Eq)] pub(crate) enum BlockAnnounceValidationResult { /// The announcement failed at validation. @@ -119,11 +119,9 @@ enum AllocateSlotForBlockAnnounceValidation { pub(crate) struct BlockAnnounceValidator { /// A type to check incoming block announcements. validator: Box + Send>, - /// All block announcements that are currently being validated. validations: FuturesUnordered> + Send>>>, - /// Number of concurrent block announce validations per peer. validations_per_peer: HashMap, } @@ -228,6 +226,42 @@ impl BlockAnnounceValidator { ); } + // TODO: merge this code into future above. + fn handle_pre_validation( + pre_validation: PreValidateBlockAnnounce, + ) -> BlockAnnounceValidationResult { + match pre_validation { + PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { + trace!( + target: "sync", + "Finished block announce validation: from {:?}: {:?}. local_best={}", + peer_id, + announce.summary(), + is_new_best, + ); + + BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } + }, + PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { + debug!( + target: "sync", + "Failed announce validation: {:?}, disconnect: {}", + peer_id, + disconnect, + ); + BlockAnnounceValidationResult::Failure { peer_id, disconnect } + }, + PreValidateBlockAnnounce::Error { peer_id } => { + debug!( + target: "sync", + "Ignored announce validation from {:?} due to internal validation error", + peer_id, + ); + BlockAnnounceValidationResult::Skip + }, + } + } + /// Checks if there is a slot for a block announce validation. /// /// The total number and the number per peer of concurrent block announce validations @@ -307,38 +341,9 @@ impl Stream for BlockAnnounceValidator { Poll::Ready(Some(pre_validation)) => { self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); - let res = match pre_validation { - PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { - trace!( - target: "sync", - "Finished block announce validation: from {:?}: {:?}. local_best={}", - peer_id, - announce.summary(), - is_new_best, - ); - - BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } - }, - PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { - debug!( - target: "sync", - "Failed announce validation: {:?}, disconnect: {}", - peer_id, - disconnect, - ); - BlockAnnounceValidationResult::Failure { peer_id, disconnect } - }, - PreValidateBlockAnnounce::Error { peer_id } => { - debug!( - target: "sync", - "Ignored announce validation from {:?} due to internal validation error", - peer_id, - ); - BlockAnnounceValidationResult::Skip - }, - }; + let validation = Self::handle_pre_validation(pre_validation); - Poll::Ready(Some(res)) + Poll::Ready(Some(validation)) }, _ => Poll::Pending, } From 33c04f85db21ded26b03c41c9ba85257f03be7e3 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Mon, 31 Jul 2023 20:36:40 +0300 Subject: [PATCH 13/23] WIP: wake-up task if new validation is added to `BlockAnnounceValidator` --- Cargo.lock | 1 + client/network/sync/Cargo.toml | 1 + .../sync/src/block_announce_validator.rs | 39 ++++++++++++++----- 3 files changed, 31 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2e78549ad9b20..95d950d900a1a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9887,6 +9887,7 @@ dependencies = [ "array-bytes", "async-channel", "async-trait", + "event-listener", "fork-tree", "futures", "futures-timer", diff --git a/client/network/sync/Cargo.toml b/client/network/sync/Cargo.toml index 1feb1316dbcde..9e06eed543f3f 100644 --- a/client/network/sync/Cargo.toml +++ b/client/network/sync/Cargo.toml @@ -20,6 +20,7 @@ array-bytes = "6.1" async-channel = "1.8.0" async-trait = "0.1.58" codec = { package = "parity-scale-codec", version = "3.6.1", features = ["derive"] } +event-listener = "2.5.3" futures = "0.3.21" futures-timer = "3.0.2" libp2p = "0.51.3" diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index 3aedc79751119..d460a600d7a83 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -18,6 +18,7 @@ //! `BlockAnnounceValidator` is responsible for async validation of block announcements. +use event_listener::{Event, EventListener}; use futures::{stream::FuturesUnordered, Future, FutureExt, Stream, StreamExt}; use libp2p::PeerId; use log::{debug, error, trace, warn}; @@ -124,6 +125,10 @@ pub(crate) struct BlockAnnounceValidator { FuturesUnordered> + Send>>>, /// Number of concurrent block announce validations per peer. validations_per_peer: HashMap, + /// Wake-up event when new validations are pushed. + event: Event, + /// Listener for wake-up events in [`Stream::poll_next`] implementation. + event_listener: Option, } impl BlockAnnounceValidator { @@ -134,6 +139,8 @@ impl BlockAnnounceValidator { validator, validations: Default::default(), validations_per_peer: Default::default(), + event: Event::new(), + event_listener: None, } } @@ -224,12 +231,13 @@ impl BlockAnnounceValidator { } .boxed(), ); + + // Make sure [`Stream::poll_next`] is woken up. + self.event.notify(1); } // TODO: merge this code into future above. - fn handle_pre_validation( - pre_validation: PreValidateBlockAnnounce, - ) -> BlockAnnounceValidationResult { + fn handle_pre_validation(pre_validation: PreValidateBlockAnnounce) -> BlockAnnounceValidationResult { match pre_validation { PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { trace!( @@ -337,15 +345,26 @@ impl Stream for BlockAnnounceValidator { /// Poll for finished block announce validations. The stream never terminates. fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - match self.validations.poll_next_unpin(cx) { - Poll::Ready(Some(pre_validation)) => { - self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); + loop { + match self.validations.poll_next_unpin(cx) { + Poll::Ready(Some(pre_validation)) => { + self.event_listener = None; - let validation = Self::handle_pre_validation(pre_validation); + self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); - Poll::Ready(Some(validation)) - }, - _ => Poll::Pending, + let res = Self::handle_pre_validation(pre_validation); + + return Poll::Ready(Some(res)) + }, + Poll::Ready(None) => { + if self.event_listener.is_none() { + self.event_listener = Some(self.event.listen()); + } + + // TODO: poll `event_listener`. + }, + _ => return Poll::Pending, + } } } } From 1c4b584cdc84ff1e3eabc2223c1443598d312059 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Tue, 1 Aug 2023 10:24:43 +0300 Subject: [PATCH 14/23] Wake up task when new validation is added to `BlockAnnounceValidator` --- .../sync/src/block_announce_validator.rs | 96 ++++++++++++++++--- 1 file changed, 81 insertions(+), 15 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index d460a600d7a83..b26dfca0c37f2 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -237,7 +237,9 @@ impl BlockAnnounceValidator { } // TODO: merge this code into future above. - fn handle_pre_validation(pre_validation: PreValidateBlockAnnounce) -> BlockAnnounceValidationResult { + fn handle_pre_validation( + pre_validation: PreValidateBlockAnnounce, + ) -> BlockAnnounceValidationResult { match pre_validation { PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { trace!( @@ -345,26 +347,90 @@ impl Stream for BlockAnnounceValidator { /// Poll for finished block announce validations. The stream never terminates. fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + // Note: the wake-up code below is modeled after `async-channel`. + // See https://github.com/smol-rs/async-channel/blob/4cae9cb0cbbd7c3c0518438e03a01e312b303e59/src/lib.rs#L787-L825 loop { - match self.validations.poll_next_unpin(cx) { - Poll::Ready(Some(pre_validation)) => { - self.event_listener = None; + // Wait for wake-up event if we are in a waiting state after `self.validations` + // was deplenished. + if let Some(listener) = self.event_listener.as_mut() { + match listener.poll_unpin(cx) { + Poll::Ready(()) => self.event_listener = None, + Poll::Pending => return Poll::Pending, + } + } - self.deallocate_slot_for_block_announce_validation(pre_validation.peer_id()); + loop { + match self.validations.poll_next_unpin(cx) { + Poll::Ready(Some(pre_validation)) => { + self.event_listener = None; - let res = Self::handle_pre_validation(pre_validation); + self.deallocate_slot_for_block_announce_validation( + pre_validation.peer_id(), + ); - return Poll::Ready(Some(res)) - }, - Poll::Ready(None) => { - if self.event_listener.is_none() { - self.event_listener = Some(self.event.listen()); - } + let res = Self::handle_pre_validation(pre_validation); - // TODO: poll `event_listener`. - }, - _ => return Poll::Pending, + return Poll::Ready(Some(res)) + }, + Poll::Ready(None) => {}, + Poll::Pending => return Poll::Pending, + } + + // `self.validations` was deplenished, start/continue waiting for a wake-up event. + match self.event_listener { + Some(_) => { + // Go back to the outer loop to wait for a wake-up event. + break + }, + None => { + // Create listener and go polling `self.validations` again in case it was + // populated just before the listener was created. + self.event_listener = Some(self.event.listen()); + }, + } } } } } + +#[cfg(test)] +mod tests { + use std::task::Poll; + + use futures::{future::BoxFuture, stream::FuturesUnordered, FutureExt, StreamExt}; + + /// `Stream` implementation for `BlockAnnounceValidator` relies on the undocumented + /// feature that `FuturesUnordered` can be polled and repeatedly yield + /// `Poll::Ready(None)` before any futures were added into it. + #[tokio::test] + async fn empty_futures_unordered_can_be_polled() { + let mut unordered = FuturesUnordered::>::default(); + + futures::future::poll_fn(|cx| { + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + + Poll::Ready(()) + }) + .await; + } + + /// `Stream` implementation for `BlockAnnounceValidator` relies on the undocumented + /// feature that `FuturesUnordered` can be polled and repeatedly yield + /// `Poll::Ready(None)` after all the futures in it have resolved. + #[tokio::test] + async fn deplenished_futures_unordered_can_be_polled() { + let mut unordered = FuturesUnordered::>::default(); + + unordered.push(futures::future::ready(()).boxed()); + assert_eq!(unordered.next().await, Some(())); + + futures::future::poll_fn(|cx| { + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + + Poll::Ready(()) + }) + .await; + } +} From 734294bf5fdbd2f3cc40413afcbb6eccd2a4f5c5 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Tue, 1 Aug 2023 10:52:02 +0300 Subject: [PATCH 15/23] Get rid of `PreValidateBlockAnnounce` --- .../sync/src/block_announce_validator.rs | 155 ++++++------------ client/network/sync/src/engine.rs | 2 +- 2 files changed, 53 insertions(+), 104 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index b26dfca0c37f2..40afee20fcc9d 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -31,6 +31,9 @@ use std::{ task::{Context, Poll}, }; +/// Log target for this file. +const LOG_TARGET: &str = "sync"; + /// Maximum number of concurrent block announce validations. /// /// If the queue reaches the maximum, we drop any new block @@ -42,46 +45,6 @@ const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS: usize = 256; /// See [`MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS`] for more information. const MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER: usize = 4; -/// Result of [`BlockAnnounceValidator::block_announce_validation`]. -#[derive(Debug, Clone, PartialEq, Eq)] -enum PreValidateBlockAnnounce { - /// The announcement failed at validation. - /// - /// The peer reputation should be decreased. - Failure { - /// The id of the peer that send us the announcement. - peer_id: PeerId, - /// Should the peer be disconnected? - disconnect: bool, - }, - /// The pre-validation was sucessful and the announcement should be - /// further processed. - Process { - /// Is this the new best block of the peer? - is_new_best: bool, - /// The id of the peer that send us the announcement. - peer_id: PeerId, - /// The announcement. - announce: BlockAnnounce, - }, - /// The announcement validation returned an error. - /// - /// An error means that *this* node failed to validate it because some internal error happened. - /// If the block announcement was invalid, [`Self::Failure`] is the correct variant to express - /// this. - Error { peer_id: PeerId }, -} - -impl PreValidateBlockAnnounce { - fn peer_id(&self) -> &PeerId { - match self { - PreValidateBlockAnnounce::Failure { peer_id, .. } | - PreValidateBlockAnnounce::Process { peer_id, .. } | - PreValidateBlockAnnounce::Error { peer_id } => peer_id, - } - } -} - /// Item that yields [`Stream`] implementation of [`BlockAnnounceValidator`]. #[derive(Debug, Clone, PartialEq, Eq)] pub(crate) enum BlockAnnounceValidationResult { @@ -104,7 +67,20 @@ pub(crate) enum BlockAnnounceValidationResult { announce: BlockAnnounce, }, /// The block announcement should be skipped. - Skip, + Skip { + /// The id of the peer that send us the announcement. + peer_id: PeerId, + }, +} + +impl BlockAnnounceValidationResult { + fn peer_id(&self) -> &PeerId { + match self { + BlockAnnounceValidationResult::Failure { peer_id, .. } | + BlockAnnounceValidationResult::Process { peer_id, .. } | + BlockAnnounceValidationResult::Skip { peer_id } => peer_id, + } + } } /// Result of [`BlockAnnounceValidator::allocate_slot_for_block_announce_validation`]. @@ -121,8 +97,9 @@ pub(crate) struct BlockAnnounceValidator { /// A type to check incoming block announcements. validator: Box + Send>, /// All block announcements that are currently being validated. - validations: - FuturesUnordered> + Send>>>, + validations: FuturesUnordered< + Pin> + Send>>, + >, /// Number of concurrent block announce validations per peer. validations_per_peer: HashMap, /// Wake-up event when new validations are pushed. @@ -155,7 +132,7 @@ impl BlockAnnounceValidator { let header = &announce.header; let number = *header.number(); debug!( - target: "sync", + target: LOG_TARGET, "Pre-validating received block announcement {:?} with number {:?} from {}", hash, number, @@ -164,7 +141,7 @@ impl BlockAnnounceValidator { if number.is_zero() { warn!( - target: "sync", + target: LOG_TARGET, "💔 Ignored genesis block (#0) announcement from {}: {}", peer_id, hash, @@ -177,7 +154,7 @@ impl BlockAnnounceValidator { AllocateSlotForBlockAnnounceValidation::Allocated => {}, AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached => { warn!( - target: "sync", + target: LOG_TARGET, "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots are occupied.", number, hash, @@ -187,7 +164,7 @@ impl BlockAnnounceValidator { }, AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached => { warn!( - target: "sync", + target: LOG_TARGET, "💔 Ignored block (#{} -- {}) announcement from {} because all validation slots for this peer are occupied.", number, hash, @@ -204,28 +181,40 @@ impl BlockAnnounceValidator { self.validations.push( async move { match future.await { - Ok(Validation::Success { is_new_best }) => PreValidateBlockAnnounce::Process { - is_new_best: is_new_best || is_best, - announce, - peer_id, + Ok(Validation::Success { is_new_best }) => { + let is_new_best = is_new_best || is_best; + + trace!( + target: LOG_TARGET, + "Block announcement validated successfully: from {}: {:?}. Local best: {}.", + peer_id, + announce.summary(), + is_new_best, + ); + + BlockAnnounceValidationResult::Process { is_new_best, announce, peer_id } }, Ok(Validation::Failure { disconnect }) => { debug!( - target: "sync", - "Block announcement validation of block {:?} from {} failed", - hash, + target: LOG_TARGET, + "Block announcement validation failed: from {}, block {:?}. Disconnect: {}.", peer_id, + hash, + disconnect, ); - PreValidateBlockAnnounce::Failure { peer_id, disconnect } + + BlockAnnounceValidationResult::Failure { peer_id, disconnect } }, Err(e) => { debug!( - target: "sync", - "💔 Block announcement validation of block {:?} errored: {}", + target: LOG_TARGET, + "💔 Ignoring block announcement validation from {} of block {:?} due to internal error: {}.", + peer_id, hash, e, ); - PreValidateBlockAnnounce::Error { peer_id } + + BlockAnnounceValidationResult::Skip { peer_id } }, } } @@ -236,42 +225,6 @@ impl BlockAnnounceValidator { self.event.notify(1); } - // TODO: merge this code into future above. - fn handle_pre_validation( - pre_validation: PreValidateBlockAnnounce, - ) -> BlockAnnounceValidationResult { - match pre_validation { - PreValidateBlockAnnounce::Process { is_new_best, peer_id, announce } => { - trace!( - target: "sync", - "Finished block announce validation: from {:?}: {:?}. local_best={}", - peer_id, - announce.summary(), - is_new_best, - ); - - BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } - }, - PreValidateBlockAnnounce::Failure { peer_id, disconnect } => { - debug!( - target: "sync", - "Failed announce validation: {:?}, disconnect: {}", - peer_id, - disconnect, - ); - BlockAnnounceValidationResult::Failure { peer_id, disconnect } - }, - PreValidateBlockAnnounce::Error { peer_id } => { - debug!( - target: "sync", - "Ignored announce validation from {:?} due to internal validation error", - peer_id, - ); - BlockAnnounceValidationResult::Skip - }, - } - } - /// Checks if there is a slot for a block announce validation. /// /// The total number and the number per peer of concurrent block announce validations @@ -313,7 +266,7 @@ impl BlockAnnounceValidator { match self.validations_per_peer.entry(*peer_id) { Entry::Vacant(_) => { error!( - target: "sync", + target: LOG_TARGET, "💔 Block announcement validation from peer {} finished for that no slot was allocated!", peer_id, ); @@ -329,7 +282,7 @@ impl BlockAnnounceValidator { entry.remove(); error!( - target: "sync", + target: LOG_TARGET, "Invalid (zero) block announce validation slot counter for peer {peer_id}.", ); debug_assert!( @@ -361,16 +314,12 @@ impl Stream for BlockAnnounceValidator { loop { match self.validations.poll_next_unpin(cx) { - Poll::Ready(Some(pre_validation)) => { + Poll::Ready(Some(validation)) => { self.event_listener = None; - self.deallocate_slot_for_block_announce_validation( - pre_validation.peer_id(), - ); - - let res = Self::handle_pre_validation(pre_validation); + self.deallocate_slot_for_block_announce_validation(validation.peer_id()); - return Poll::Ready(Some(res)) + return Poll::Ready(Some(validation)) }, Poll::Ready(None) => {}, Poll::Pending => return Poll::Pending, diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index bc46a381466f1..15f04f3cbec49 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -480,7 +480,7 @@ where validation_result: BlockAnnounceValidationResult, ) { match validation_result { - BlockAnnounceValidationResult::Skip => {}, + BlockAnnounceValidationResult::Skip { peer_id: _ } => {}, BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } => { self.chain_sync.on_validated_block_announce(is_new_best, peer_id, &announce); From f77f8e8a0affd3194ee442ff8c6d85ad0f06d81e Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Tue, 1 Aug 2023 11:23:41 +0300 Subject: [PATCH 16/23] Remove now unneeded polling after pushing block announce validation --- client/network/sync/src/engine.rs | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 15f04f3cbec49..f45168dcd1538 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -509,13 +509,6 @@ where /// called later to check for finished validations. The result of the validation /// needs to be passed to [`SyncingEngine::process_block_announce_validation_result`] /// to finish the processing. - /// - /// # Note - /// - /// This will internally create a future, but this future will not be registered - /// in the task before being polled once. So, it is required to call - /// [`BlockAnnounceValidatorStream::poll_block_announce_validation`] to ensure that the future - /// is registered properly and will wake up the task when being ready. pub fn push_block_announce_validation( &mut self, peer_id: PeerId, @@ -777,14 +770,6 @@ where if self.peers.contains_key(&remote) { if let Ok(announce) = BlockAnnounce::decode(&mut message.as_ref()) { self.push_block_announce_validation(remote, announce); - - // Make sure that the newly added block announce validation future - // was polled once to be registered in the task. - if let Poll::Ready(Some(res)) = - self.block_announce_validator.poll_next_unpin(cx) - { - self.process_block_announce_validation_result(res) - } } else { log::warn!(target: "sub-libp2p", "Failed to decode block announce"); } From 3630b312fa9fff77cb98cae788990e7be6f74af2 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Tue, 1 Aug 2023 12:02:46 +0300 Subject: [PATCH 17/23] Add tests for validation slot allocation --- .../sync/src/block_announce_validator.rs | 103 +++++++++++++++++- 1 file changed, 101 insertions(+), 2 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index 40afee20fcc9d..da29641a9b272 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -344,9 +344,13 @@ impl Stream for BlockAnnounceValidator { #[cfg(test)] mod tests { - use std::task::Poll; - + use super::*; + use crate::block_announce_validator::AllocateSlotForBlockAnnounceValidation; use futures::{future::BoxFuture, stream::FuturesUnordered, FutureExt, StreamExt}; + use libp2p::PeerId; + use sp_consensus::block_validation::DefaultBlockAnnounceValidator; + use std::task::Poll; + use substrate_test_runtime_client::runtime::Block; /// `Stream` implementation for `BlockAnnounceValidator` relies on the undocumented /// feature that `FuturesUnordered` can be polled and repeatedly yield @@ -382,4 +386,99 @@ mod tests { }) .await; } + + #[test] + fn allocate_one_validation_slot() { + let mut validator = + BlockAnnounceValidator::::new(Box::new(DefaultBlockAnnounceValidator {})); + let peer_id = PeerId::random(); + + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::Allocated, + )); + } + + #[test] + fn allocate_validation_slots_for_two_peers() { + let mut validator = + BlockAnnounceValidator::::new(Box::new(DefaultBlockAnnounceValidator {})); + let peer_id_1 = PeerId::random(); + let peer_id_2 = PeerId::random(); + + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id_1), + AllocateSlotForBlockAnnounceValidation::Allocated, + )); + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id_2), + AllocateSlotForBlockAnnounceValidation::Allocated, + )); + } + + #[test] + fn maximum_validation_slots_per_peer() { + let mut validator = + BlockAnnounceValidator::::new(Box::new(DefaultBlockAnnounceValidator {})); + let peer_id = PeerId::random(); + + for _ in 0..MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::Allocated, + )); + } + + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached, + )); + } + + #[test] + fn validation_slots_per_peer_deallocated() { + let mut validator = + BlockAnnounceValidator::::new(Box::new(DefaultBlockAnnounceValidator {})); + let peer_id = PeerId::random(); + + for _ in 0..MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS_PER_PEER { + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::Allocated, + )); + } + + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::MaximumPeerSlotsReached, + )); + + validator.deallocate_slot_for_block_announce_validation(&peer_id); + + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::Allocated, + )); + } + + #[test] + fn maximum_validation_slots_for_all_peers() { + let mut validator = + BlockAnnounceValidator::::new(Box::new(DefaultBlockAnnounceValidator {})); + + for _ in 0..MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { + validator.validations.push( + futures::future::ready(BlockAnnounceValidationResult::Skip { + peer_id: PeerId::random(), + }) + .boxed(), + ) + } + + let peer_id = PeerId::random(); + assert!(matches!( + validator.allocate_slot_for_block_announce_validation(&peer_id), + AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached, + )); + } } From 8d62a27430887e2b8aec8e7b169d6b878532ce77 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Tue, 1 Aug 2023 16:00:18 +0300 Subject: [PATCH 18/23] minor: make rutdoc happy --- client/network/sync/src/engine.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index f45168dcd1538..4b3cfc378eaca 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -504,11 +504,6 @@ where } /// Push a block announce validation. - /// - /// It is required that [`BlockAnnounceValidatorStream::poll_block_announce_validation`] is - /// called later to check for finished validations. The result of the validation - /// needs to be passed to [`SyncingEngine::process_block_announce_validation_result`] - /// to finish the processing. pub fn push_block_announce_validation( &mut self, peer_id: PeerId, From 927c9f792c733f9963a0b9bd0292c5114b6ad859 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Wed, 2 Aug 2023 11:04:29 +0300 Subject: [PATCH 19/23] Apply suggestions from code review Co-authored-by: Aaro Altonen <48052676+altonen@users.noreply.github.com> Co-authored-by: Anton --- client/network/sync/src/block_announce_validator.rs | 4 ++-- client/network/sync/src/lib.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index da29641a9b272..a87e7fb44774f 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -1,6 +1,6 @@ // This file is part of Substrate. -// Copyright (C) 2017-2023 Parity Technologies (UK) Ltd. +// 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 @@ -267,7 +267,7 @@ impl BlockAnnounceValidator { Entry::Vacant(_) => { error!( target: LOG_TARGET, - "💔 Block announcement validation from peer {} finished for that no slot was allocated!", + "💔 Block announcement validation from peer {} finished for a slot that was not allocated!", peer_id, ); }, diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index b059e66acffb8..5a4691e975171 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -1061,7 +1061,7 @@ where }; if let PeerSyncState::AncestorSearch { .. } = peer.state { - trace!(target: "sync", "Peer {} state is ancestor search.", who); + trace!(target: "sync", "Peer {} is in the ancestor search state.", who); return } From f137c55c15ff6949b6a459e5a23d0b136b2f9bdc Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Wed, 2 Aug 2023 11:29:50 +0300 Subject: [PATCH 20/23] Use `futures::ready!` to simplify polling --- .../sync/src/block_announce_validator.rs | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index a87e7fb44774f..7e0896f3198bf 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -306,23 +306,16 @@ impl Stream for BlockAnnounceValidator { // Wait for wake-up event if we are in a waiting state after `self.validations` // was deplenished. if let Some(listener) = self.event_listener.as_mut() { - match listener.poll_unpin(cx) { - Poll::Ready(()) => self.event_listener = None, - Poll::Pending => return Poll::Pending, - } + let () = futures::ready!(listener.poll_unpin(cx)); + self.event_listener = None; } loop { - match self.validations.poll_next_unpin(cx) { - Poll::Ready(Some(validation)) => { - self.event_listener = None; - - self.deallocate_slot_for_block_announce_validation(validation.peer_id()); + if let Some(validation) = futures::ready!(self.validations.poll_next_unpin(cx)) { + self.event_listener = None; + self.deallocate_slot_for_block_announce_validation(validation.peer_id()); - return Poll::Ready(Some(validation)) - }, - Poll::Ready(None) => {}, - Poll::Pending => return Poll::Pending, + return Poll::Ready(Some(validation)) } // `self.validations` was deplenished, start/continue waiting for a wake-up event. From c717dbc9e4e41967ecdc9b205347a24ce5841904 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Thu, 24 Aug 2023 23:21:44 +0300 Subject: [PATCH 21/23] Prepare `BloackAnnounceValidator` for pushing block announcements from somewhere else --- Cargo.lock | 1 - client/network/sync/Cargo.toml | 1 - .../sync/src/block_announce_validator.rs | 116 ++------ client/network/sync/src/futures_stream.rs | 251 ++++++++++++++++++ client/network/sync/src/lib.rs | 1 + client/utils/src/mpsc.rs | 10 + 6 files changed, 289 insertions(+), 91 deletions(-) create mode 100644 client/network/sync/src/futures_stream.rs diff --git a/Cargo.lock b/Cargo.lock index 68fba6646c1f2..93a8e5e988305 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9323,7 +9323,6 @@ dependencies = [ "array-bytes", "async-channel", "async-trait", - "event-listener", "fork-tree", "futures", "futures-timer", diff --git a/client/network/sync/Cargo.toml b/client/network/sync/Cargo.toml index b263e4da28680..53bb987a6f486 100644 --- a/client/network/sync/Cargo.toml +++ b/client/network/sync/Cargo.toml @@ -20,7 +20,6 @@ array-bytes = "6.1" async-channel = "1.8.0" async-trait = "0.1.58" codec = { package = "parity-scale-codec", version = "3.6.1", features = ["derive"] } -event-listener = "2.5.3" futures = "0.3.21" futures-timer = "3.0.2" libp2p = "0.52.1" diff --git a/client/network/sync/src/block_announce_validator.rs b/client/network/sync/src/block_announce_validator.rs index 7e0896f3198bf..973ee1f95aa65 100644 --- a/client/network/sync/src/block_announce_validator.rs +++ b/client/network/sync/src/block_announce_validator.rs @@ -18,8 +18,8 @@ //! `BlockAnnounceValidator` is responsible for async validation of block announcements. -use event_listener::{Event, EventListener}; -use futures::{stream::FuturesUnordered, Future, FutureExt, Stream, StreamExt}; +use crate::futures_stream::{futures_stream, FuturesStreamReceiver, FuturesStreamSender}; +use futures::{Future, FutureExt, Stream, StreamExt}; use libp2p::PeerId; use log::{debug, error, trace, warn}; use sc_network_common::sync::message::BlockAnnounce; @@ -27,6 +27,7 @@ use sp_consensus::block_validation::Validation; use sp_runtime::traits::{Block as BlockT, Header, Zero}; use std::{ collections::{hash_map::Entry, HashMap}, + default::Default, pin::Pin, task::{Context, Poll}, }; @@ -96,29 +97,27 @@ enum AllocateSlotForBlockAnnounceValidation { pub(crate) struct BlockAnnounceValidator { /// A type to check incoming block announcements. validator: Box + Send>, - /// All block announcements that are currently being validated. - validations: FuturesUnordered< + /// All block announcements that are currently being validated, sending side of the stream. + tx_validations: FuturesStreamSender< + Pin> + Send>>, + >, + /// All block announcements that are currently being validated, receiving side of the stream. + rx_validations: FuturesStreamReceiver< Pin> + Send>>, >, /// Number of concurrent block announce validations per peer. validations_per_peer: HashMap, - /// Wake-up event when new validations are pushed. - event: Event, - /// Listener for wake-up events in [`Stream::poll_next`] implementation. - event_listener: Option, } impl BlockAnnounceValidator { pub(crate) fn new( validator: Box + Send>, ) -> Self { - Self { - validator, - validations: Default::default(), - validations_per_peer: Default::default(), - event: Event::new(), - event_listener: None, - } + let (tx_validations, rx_validations) = futures_stream( + "mpsc_block_announce_validation_stream", + MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS + 1, + ); + Self { validator, tx_validations, rx_validations, validations_per_peer: Default::default() } } /// Push a block announce validation. @@ -178,7 +177,7 @@ impl BlockAnnounceValidator { let assoc_data = announce.data.as_ref().map_or(&[][..], |v| v.as_slice()); let future = self.validator.validate(header, assoc_data); - self.validations.push( + let _ = self.tx_validations.push( async move { match future.await { Ok(Validation::Success { is_new_best }) => { @@ -220,9 +219,6 @@ impl BlockAnnounceValidator { } .boxed(), ); - - // Make sure [`Stream::poll_next`] is woken up. - self.event.notify(1); } /// Checks if there is a slot for a block announce validation. @@ -240,7 +236,7 @@ impl BlockAnnounceValidator { &mut self, peer_id: &PeerId, ) -> AllocateSlotForBlockAnnounceValidation { - if self.validations.len() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { + if self.rx_validations.len_lower_bound() >= MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { return AllocateSlotForBlockAnnounceValidation::TotalMaximumSlotsReached } @@ -300,37 +296,16 @@ impl Stream for BlockAnnounceValidator { /// Poll for finished block announce validations. The stream never terminates. fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - // Note: the wake-up code below is modeled after `async-channel`. - // See https://github.com/smol-rs/async-channel/blob/4cae9cb0cbbd7c3c0518438e03a01e312b303e59/src/lib.rs#L787-L825 - loop { - // Wait for wake-up event if we are in a waiting state after `self.validations` - // was deplenished. - if let Some(listener) = self.event_listener.as_mut() { - let () = futures::ready!(listener.poll_unpin(cx)); - self.event_listener = None; - } - - loop { - if let Some(validation) = futures::ready!(self.validations.poll_next_unpin(cx)) { - self.event_listener = None; - self.deallocate_slot_for_block_announce_validation(validation.peer_id()); + if let Some(validation) = futures::ready!(self.rx_validations.poll_next_unpin(cx)) { + self.deallocate_slot_for_block_announce_validation(validation.peer_id()); - return Poll::Ready(Some(validation)) - } - - // `self.validations` was deplenished, start/continue waiting for a wake-up event. - match self.event_listener { - Some(_) => { - // Go back to the outer loop to wait for a wake-up event. - break - }, - None => { - // Create listener and go polling `self.validations` again in case it was - // populated just before the listener was created. - self.event_listener = Some(self.event.listen()); - }, - } - } + Poll::Ready(Some(validation)) + } else { + trace!( + target: LOG_TARGET, + "Block announce validations stream terminated, terminating `BlockAnnounceValidator`", + ); + Poll::Ready(None) } } } @@ -339,47 +314,10 @@ impl Stream for BlockAnnounceValidator { mod tests { use super::*; use crate::block_announce_validator::AllocateSlotForBlockAnnounceValidation; - use futures::{future::BoxFuture, stream::FuturesUnordered, FutureExt, StreamExt}; use libp2p::PeerId; use sp_consensus::block_validation::DefaultBlockAnnounceValidator; - use std::task::Poll; use substrate_test_runtime_client::runtime::Block; - /// `Stream` implementation for `BlockAnnounceValidator` relies on the undocumented - /// feature that `FuturesUnordered` can be polled and repeatedly yield - /// `Poll::Ready(None)` before any futures were added into it. - #[tokio::test] - async fn empty_futures_unordered_can_be_polled() { - let mut unordered = FuturesUnordered::>::default(); - - futures::future::poll_fn(|cx| { - assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); - assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); - - Poll::Ready(()) - }) - .await; - } - - /// `Stream` implementation for `BlockAnnounceValidator` relies on the undocumented - /// feature that `FuturesUnordered` can be polled and repeatedly yield - /// `Poll::Ready(None)` after all the futures in it have resolved. - #[tokio::test] - async fn deplenished_futures_unordered_can_be_polled() { - let mut unordered = FuturesUnordered::>::default(); - - unordered.push(futures::future::ready(()).boxed()); - assert_eq!(unordered.next().await, Some(())); - - futures::future::poll_fn(|cx| { - assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); - assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); - - Poll::Ready(()) - }) - .await; - } - #[test] fn allocate_one_validation_slot() { let mut validator = @@ -460,12 +398,12 @@ mod tests { BlockAnnounceValidator::::new(Box::new(DefaultBlockAnnounceValidator {})); for _ in 0..MAX_CONCURRENT_BLOCK_ANNOUNCE_VALIDATIONS { - validator.validations.push( + let _ = validator.tx_validations.push( futures::future::ready(BlockAnnounceValidationResult::Skip { peer_id: PeerId::random(), }) .boxed(), - ) + ); } let peer_id = PeerId::random(); diff --git a/client/network/sync/src/futures_stream.rs b/client/network/sync/src/futures_stream.rs new file mode 100644 index 0000000000000..a46e9d9558819 --- /dev/null +++ b/client/network/sync/src/futures_stream.rs @@ -0,0 +1,251 @@ +// 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 . + +//! Async channel that receives futures as input, processes them in unordered manner, and +//! yield the results once they are resolved. The receiver implements `Stream`. + +use futures::{stream::FuturesUnordered, Future, Stream, StreamExt}; +use sc_utils::mpsc::{ + tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender, TrySendError, +}; +use std::{ + pin::Pin, + task::{Context, Poll}, +}; + +/// Sending side of the futures stream. +pub struct FuturesStreamSender { + tx: TracingUnboundedSender, +} + +/// Receiving side of the futures stream. +pub struct FuturesStreamReceiver { + rx: TracingUnboundedReceiver, + rx_terminated: bool, + futures: FuturesUnordered, +} + +pub fn futures_stream( + name: &'static str, + queue_size_warning: usize, +) -> (FuturesStreamSender, FuturesStreamReceiver) { + let (tx, rx) = tracing_unbounded(name, queue_size_warning); + + ( + FuturesStreamSender { tx }, + FuturesStreamReceiver { rx, rx_terminated: false, futures: Default::default() }, + ) +} + +impl FuturesStreamSender { + /// Push a futures for processing. + pub fn push(&self, future: F) -> Result<(), TrySendError> { + self.tx.unbounded_send(future) + } +} + +impl FuturesStreamReceiver { + /// The lower bound of the number of futures in the stream. Note that this estimate might be + /// less than actual number by 1 in case the future was taken from the channel and not yet put + /// into [`FuturesUnordered`]. + pub fn len_lower_bound(&self) -> usize { + self.rx.len() + self.futures.len() + } +} + +impl Stream for FuturesStreamReceiver { + type Item = ::Output; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + // Get futures out of `TracingUnboundedReceiver` and push them into `FuturesUnordered`. + if !self.rx_terminated { + while let Poll::Ready(item) = self.rx.poll_next_unpin(cx) { + if let Some(future) = item { + self.futures.push(future); + } else { + self.rx_terminated = true; + break + } + } + } + + // Poll `FuturesUnordered`. + match self.futures.poll_next_unpin(cx) { + Poll::Ready(Some(result)) => Poll::Ready(Some(result)), + Poll::Ready(None) => + if self.rx_terminated { + Poll::Ready(None) + } else { + Poll::Pending + }, + Poll::Pending => Poll::Pending, + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use futures::future::{BoxFuture, FutureExt}; + + /// [`Stream`] implementation for [`FuturesStreamReceiver`] relies on the undocumented + /// feature that [`FuturesUnordered`] can be polled and repeatedly yield + /// `Poll::Ready(None)` before any futures are added into it. + #[tokio::test] + async fn empty_futures_unordered_can_be_polled() { + let mut unordered = FuturesUnordered::>::default(); + + futures::future::poll_fn(|cx| { + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + + Poll::Ready(()) + }) + .await; + } + + /// [`Stream`] implementation for [`FuturesStreamReceiver`] relies on the undocumented + /// feature that [`FuturesUnordered`] can be polled and repeatedly yield + /// `Poll::Ready(None)` after all the futures in it have resolved. + #[tokio::test] + async fn deplenished_futures_unordered_can_be_polled() { + let mut unordered = FuturesUnordered::>::default(); + + unordered.push(futures::future::ready(()).boxed()); + assert_eq!(unordered.next().await, Some(())); + + futures::future::poll_fn(|cx| { + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + assert_eq!(unordered.poll_next_unpin(cx), Poll::Ready(None)); + + Poll::Ready(()) + }) + .await; + } + + #[tokio::test] + async fn empty_futures_stream_yields_pending() { + let (_tx, mut stream) = futures_stream::>("test", 100); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); + Poll::Ready(()) + }) + .await; + } + + #[tokio::test] + async fn futures_stream_resolves_futures_and_yields_pending() { + let (tx, mut stream) = futures_stream("test", 100); + tx.push(futures::future::ready(17)).unwrap(); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(17))); + assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); + Poll::Ready(()) + }) + .await; + } + + #[tokio::test] + async fn futures_stream_terminates_if_sender_is_dropped() { + let (tx, mut stream) = futures_stream::>("test", 100); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); + Poll::Ready(()) + }) + .await; + + drop(tx); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(None)); + Poll::Ready(()) + }) + .await; + } + + #[tokio::test] + async fn futures_stream_terminates_after_resolving_all_futures_if_sender_is_dropped() { + let (tx, mut stream) = futures_stream("test", 100); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); + Poll::Ready(()) + }) + .await; + + tx.push(futures::future::ready(17)).unwrap(); + drop(tx); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(17))); + assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(None)); + Poll::Ready(()) + }) + .await; + } + + #[test] + fn futures_stream_len_is_zsro_for_empty_stream() { + let (_tx, stream) = futures_stream::>("test", 100); + assert_eq!(stream.len_lower_bound(), 0); + } + + #[tokio::test] + async fn futures_stream_len_counts_and_discounts_resolved_futures() { + let (tx, mut stream) = futures_stream("test", 100); + assert_eq!(stream.len_lower_bound(), 0); + + tx.push(futures::future::ready(17)).unwrap(); + assert_eq!(stream.len_lower_bound(), 1); + + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(17))); + assert_eq!(stream.len_lower_bound(), 0); + + assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); + assert_eq!(stream.len_lower_bound(), 0); + + Poll::Ready(()) + }) + .await; + } + + #[tokio::test] + async fn futures_stream_len_counts_taken_pending_futures() { + let (tx, mut stream) = futures_stream("test", 100); + assert_eq!(stream.len_lower_bound(), 0); + + tx.push(futures::future::pending::<()>()).unwrap(); + + // The future in the unbounded stream is counted. + assert_eq!(stream.len_lower_bound(), 1); + + // Poll once to move the future from unbounded stream into [`FuturesUnordered`]. + futures::future::poll_fn(|cx| { + assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); + Poll::Ready(()) + }) + .await; + + // The future is still counted in [`FuturesUnordered`]. + assert_eq!(stream.len_lower_bound(), 1); + } +} diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index 5a4691e975171..f485e4b85c57b 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -90,6 +90,7 @@ pub use service::chain_sync::SyncingService; mod block_announce_validator; mod extra_requests; +mod futures_stream; mod schema; pub mod block_request_handler; diff --git a/client/utils/src/mpsc.rs b/client/utils/src/mpsc.rs index 36e44be5e2950..039e03f9e6188 100644 --- a/client/utils/src/mpsc.rs +++ b/client/utils/src/mpsc.rs @@ -123,6 +123,11 @@ impl TracingUnboundedSender { s }) } + + /// The number of elements in the channel (proxy function to [`async_channel::Sender`]). + pub fn len(&self) -> usize { + self.inner.len() + } } impl TracingUnboundedReceiver { @@ -139,6 +144,11 @@ impl TracingUnboundedReceiver { s }) } + + /// The number of elements in the channel (proxy function to [`async_channel::Receiver`]). + pub fn len(&self) -> usize { + self.inner.len() + } } impl Drop for TracingUnboundedReceiver { From cfb1912d972fb2be9e4801c93996019252e6cda9 Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Thu, 24 Aug 2023 23:26:22 +0300 Subject: [PATCH 22/23] minor: comment --- client/network/sync/src/futures_stream.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/network/sync/src/futures_stream.rs b/client/network/sync/src/futures_stream.rs index a46e9d9558819..7c698417927cc 100644 --- a/client/network/sync/src/futures_stream.rs +++ b/client/network/sync/src/futures_stream.rs @@ -17,7 +17,7 @@ // along with this program. If not, see . //! Async channel that receives futures as input, processes them in unordered manner, and -//! yield the results once they are resolved. The receiver implements `Stream`. +//! yields the results once they are resolved via [`Stream`] interface. use futures::{stream::FuturesUnordered, Future, Stream, StreamExt}; use sc_utils::mpsc::{ From 5879f617aa79494ca0b1edccc42eeabdb65a870e Mon Sep 17 00:00:00 2001 From: Dmitry Markin Date: Thu, 24 Aug 2023 23:30:10 +0300 Subject: [PATCH 23/23] minor: rustfmt --- client/network/sync/src/futures_stream.rs | 38 +++++++++++------------ 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/client/network/sync/src/futures_stream.rs b/client/network/sync/src/futures_stream.rs index 7c698417927cc..aac9e146a9b73 100644 --- a/client/network/sync/src/futures_stream.rs +++ b/client/network/sync/src/futures_stream.rs @@ -202,50 +202,50 @@ mod tests { .await; } - #[test] - fn futures_stream_len_is_zsro_for_empty_stream() { - let (_tx, stream) = futures_stream::>("test", 100); - assert_eq!(stream.len_lower_bound(), 0); - } + #[test] + fn futures_stream_len_is_zero_for_empty_stream() { + let (_tx, stream) = futures_stream::>("test", 100); + assert_eq!(stream.len_lower_bound(), 0); + } - #[tokio::test] + #[tokio::test] async fn futures_stream_len_counts_and_discounts_resolved_futures() { let (tx, mut stream) = futures_stream("test", 100); - assert_eq!(stream.len_lower_bound(), 0); + assert_eq!(stream.len_lower_bound(), 0); - tx.push(futures::future::ready(17)).unwrap(); - assert_eq!(stream.len_lower_bound(), 1); + tx.push(futures::future::ready(17)).unwrap(); + assert_eq!(stream.len_lower_bound(), 1); futures::future::poll_fn(|cx| { assert_eq!(stream.poll_next_unpin(cx), Poll::Ready(Some(17))); - assert_eq!(stream.len_lower_bound(), 0); + assert_eq!(stream.len_lower_bound(), 0); assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); - assert_eq!(stream.len_lower_bound(), 0); + assert_eq!(stream.len_lower_bound(), 0); Poll::Ready(()) }) .await; } - #[tokio::test] + #[tokio::test] async fn futures_stream_len_counts_taken_pending_futures() { let (tx, mut stream) = futures_stream("test", 100); - assert_eq!(stream.len_lower_bound(), 0); + assert_eq!(stream.len_lower_bound(), 0); - tx.push(futures::future::pending::<()>()).unwrap(); + tx.push(futures::future::pending::<()>()).unwrap(); - // The future in the unbounded stream is counted. - assert_eq!(stream.len_lower_bound(), 1); + // The future in the unbounded stream is counted. + assert_eq!(stream.len_lower_bound(), 1); - // Poll once to move the future from unbounded stream into [`FuturesUnordered`]. + // Poll once to move the future from unbounded stream into [`FuturesUnordered`]. futures::future::poll_fn(|cx| { assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); Poll::Ready(()) }) .await; - // The future is still counted in [`FuturesUnordered`]. - assert_eq!(stream.len_lower_bound(), 1); + // The future is still counted in [`FuturesUnordered`]. + assert_eq!(stream.len_lower_bound(), 1); } }