sc_network_sync/
engine.rs

1// This file is part of Substrate.
2
3// Copyright (C) Parity Technologies (UK) Ltd.
4// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0
5
6// This program is free software: you can redistribute it and/or modify
7// it under the terms of the GNU General Public License as published by
8// the Free Software Foundation, either version 3 of the License, or
9// (at your option) any later version.
10
11// This program is distributed in the hope that it will be useful,
12// but WITHOUT ANY WARRANTY; without even the implied warranty of
13// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
14// GNU General Public License for more details.
15
16// You should have received a copy of the GNU General Public License
17// along with this program. If not, see <https://www.gnu.org/licenses/>.
18
19//! `SyncingEngine` is the actor responsible for syncing Substrate chain
20//! to tip and keep the blockchain up to date with network updates.
21
22use crate::{
23	block_announce_validator::{
24		BlockAnnounceValidationResult, BlockAnnounceValidator as BlockAnnounceValidatorStream,
25	},
26	pending_responses::{PendingResponses, ResponseEvent},
27	service::{
28		self,
29		syncing_service::{SyncingService, ToServiceCommand},
30	},
31	strategy::{SyncingAction, SyncingStrategy},
32	types::{BadPeer, ExtendedPeerInfo, SyncEvent},
33	LOG_TARGET,
34};
35
36use codec::{Decode, DecodeAll, Encode};
37use futures::{channel::oneshot, StreamExt};
38use log::{debug, error, trace, warn};
39use prometheus_endpoint::{
40	register, Counter, Gauge, MetricSource, Opts, PrometheusError, Registry, SourcedGauge, U64,
41};
42use schnellru::{ByLength, LruMap};
43use tokio::time::{Interval, MissedTickBehavior};
44
45use sc_client_api::{BlockBackend, HeaderBackend, ProofProvider};
46use sc_consensus::{import_queue::ImportQueueService, IncomingBlock};
47use sc_network::{
48	config::{FullNetworkConfiguration, NotificationHandshake, ProtocolId, SetConfig},
49	peer_store::PeerStoreProvider,
50	request_responses::{OutboundFailure, RequestFailure},
51	service::{
52		traits::{Direction, NotificationConfig, NotificationEvent, ValidationResult},
53		NotificationMetrics,
54	},
55	types::ProtocolName,
56	utils::LruHashSet,
57	NetworkBackend, NotificationService, ReputationChange,
58};
59use sc_network_common::{
60	role::Roles,
61	sync::message::{BlockAnnounce, BlockAnnouncesHandshake, BlockState},
62};
63use sc_network_types::PeerId;
64use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender};
65use sp_blockchain::{Error as ClientError, HeaderMetadata};
66use sp_consensus::{block_validation::BlockAnnounceValidator, BlockOrigin};
67use sp_runtime::{
68	traits::{Block as BlockT, Header, NumberFor, Zero},
69	Justifications,
70};
71
72use std::{
73	collections::{HashMap, HashSet},
74	iter,
75	num::NonZeroUsize,
76	sync::{
77		atomic::{AtomicBool, AtomicUsize, Ordering},
78		Arc,
79	},
80};
81
82/// Interval at which we perform time based maintenance
83const TICK_TIMEOUT: std::time::Duration = std::time::Duration::from_millis(1100);
84
85/// Maximum number of known block hashes to keep for a peer.
86const MAX_KNOWN_BLOCKS: usize = 1024; // ~32kb per peer + LruHashSet overhead
87
88/// Maximum allowed size for a block announce.
89const MAX_BLOCK_ANNOUNCE_SIZE: u64 = 1024 * 1024;
90
91mod rep {
92	use sc_network::ReputationChange as Rep;
93	/// Peer has different genesis.
94	pub const GENESIS_MISMATCH: Rep = Rep::new_fatal("Genesis mismatch");
95	/// Peer send us a block announcement that failed at validation.
96	pub const BAD_BLOCK_ANNOUNCEMENT: Rep = Rep::new(-(1 << 12), "Bad block announcement");
97	/// Peer is on unsupported protocol version.
98	pub const BAD_PROTOCOL: Rep = Rep::new_fatal("Unsupported protocol");
99	/// Reputation change when a peer refuses a request.
100	pub const REFUSED: Rep = Rep::new(-(1 << 10), "Request refused");
101	/// Reputation change when a peer doesn't respond in time to our messages.
102	pub const TIMEOUT: Rep = Rep::new(-(1 << 10), "Request timeout");
103	/// Reputation change when a peer connection failed with IO error.
104	pub const IO: Rep = Rep::new(-(1 << 10), "IO error during request");
105}
106
107struct Metrics {
108	peers: Gauge<U64>,
109	import_queue_blocks_submitted: Counter<U64>,
110	import_queue_justifications_submitted: Counter<U64>,
111}
112
113impl Metrics {
114	fn register(r: &Registry, major_syncing: Arc<AtomicBool>) -> Result<Self, PrometheusError> {
115		let _ = MajorSyncingGauge::register(r, major_syncing)?;
116		Ok(Self {
117			peers: {
118				let g = Gauge::new("substrate_sync_peers", "Number of peers we sync with")?;
119				register(g, r)?
120			},
121			import_queue_blocks_submitted: {
122				let c = Counter::new(
123					"substrate_sync_import_queue_blocks_submitted",
124					"Number of blocks submitted to the import queue.",
125				)?;
126				register(c, r)?
127			},
128			import_queue_justifications_submitted: {
129				let c = Counter::new(
130					"substrate_sync_import_queue_justifications_submitted",
131					"Number of justifications submitted to the import queue.",
132				)?;
133				register(c, r)?
134			},
135		})
136	}
137}
138
139/// The "major syncing" metric.
140#[derive(Clone)]
141pub struct MajorSyncingGauge(Arc<AtomicBool>);
142
143impl MajorSyncingGauge {
144	/// Registers the [`MajorSyncGauge`] metric whose value is
145	/// obtained from the given `AtomicBool`.
146	fn register(registry: &Registry, value: Arc<AtomicBool>) -> Result<(), PrometheusError> {
147		prometheus_endpoint::register(
148			SourcedGauge::new(
149				&Opts::new(
150					"substrate_sub_libp2p_is_major_syncing",
151					"Whether the node is performing a major sync or not.",
152				),
153				MajorSyncingGauge(value),
154			)?,
155			registry,
156		)?;
157
158		Ok(())
159	}
160}
161
162impl MetricSource for MajorSyncingGauge {
163	type N = u64;
164
165	fn collect(&self, mut set: impl FnMut(&[&str], Self::N)) {
166		set(&[], self.0.load(Ordering::Relaxed) as u64);
167	}
168}
169
170/// Peer information
171#[derive(Debug)]
172pub struct Peer<B: BlockT> {
173	pub info: ExtendedPeerInfo<B>,
174	/// Holds a set of blocks known to this peer.
175	pub known_blocks: LruHashSet<B::Hash>,
176	/// Is the peer inbound.
177	inbound: bool,
178}
179
180pub struct SyncingEngine<B: BlockT, Client> {
181	/// Syncing strategy.
182	strategy: Box<dyn SyncingStrategy<B>>,
183
184	/// Blockchain client.
185	client: Arc<Client>,
186
187	/// Number of peers we're connected to.
188	num_connected: Arc<AtomicUsize>,
189
190	/// Are we actively catching up with the chain?
191	is_major_syncing: Arc<AtomicBool>,
192
193	/// Network service.
194	network_service: service::network::NetworkServiceHandle,
195
196	/// Channel for receiving service commands
197	service_rx: TracingUnboundedReceiver<ToServiceCommand<B>>,
198
199	/// Assigned roles.
200	roles: Roles,
201
202	/// Genesis hash.
203	genesis_hash: B::Hash,
204
205	/// Set of channels for other protocols that have subscribed to syncing events.
206	event_streams: Vec<TracingUnboundedSender<SyncEvent>>,
207
208	/// Interval at which we call `tick`.
209	tick_timeout: Interval,
210
211	/// All connected peers. Contains both full and light node peers.
212	peers: HashMap<PeerId, Peer<B>>,
213
214	/// List of nodes for which we perform additional logging because they are important for the
215	/// user.
216	important_peers: HashSet<PeerId>,
217
218	/// Actual list of connected no-slot nodes.
219	default_peers_set_no_slot_connected_peers: HashSet<PeerId>,
220
221	/// List of nodes that should never occupy peer slots.
222	default_peers_set_no_slot_peers: HashSet<PeerId>,
223
224	/// Value that was passed as part of the configuration. Used to cap the number of full
225	/// nodes.
226	default_peers_set_num_full: usize,
227
228	/// Number of slots to allocate to light nodes.
229	default_peers_set_num_light: usize,
230
231	/// Maximum number of inbound peers.
232	max_in_peers: usize,
233
234	/// Number of inbound peers accepted so far.
235	num_in_peers: usize,
236
237	/// Async processor of block announce validations.
238	block_announce_validator: BlockAnnounceValidatorStream<B>,
239
240	/// A cache for the data that was associated to a block announcement.
241	block_announce_data_cache: LruMap<B::Hash, Vec<u8>>,
242
243	/// The `PeerId`'s of all boot nodes.
244	boot_node_ids: HashSet<PeerId>,
245
246	/// Protocol name used for block announcements
247	block_announce_protocol_name: ProtocolName,
248
249	/// Prometheus metrics.
250	metrics: Option<Metrics>,
251
252	/// Handle that is used to communicate with `sc_network::Notifications`.
253	notification_service: Box<dyn NotificationService>,
254
255	/// Handle to `PeerStore`.
256	peer_store_handle: Arc<dyn PeerStoreProvider>,
257
258	/// Pending responses
259	pending_responses: PendingResponses,
260
261	/// Handle to import queue.
262	import_queue: Box<dyn ImportQueueService<B>>,
263}
264
265impl<B: BlockT, Client> SyncingEngine<B, Client>
266where
267	B: BlockT,
268	Client: HeaderBackend<B>
269		+ BlockBackend<B>
270		+ HeaderMetadata<B, Error = sp_blockchain::Error>
271		+ ProofProvider<B>
272		+ Send
273		+ Sync
274		+ 'static,
275{
276	pub fn new<N>(
277		roles: Roles,
278		client: Arc<Client>,
279		metrics_registry: Option<&Registry>,
280		network_metrics: NotificationMetrics,
281		net_config: &FullNetworkConfiguration<B, <B as BlockT>::Hash, N>,
282		protocol_id: ProtocolId,
283		fork_id: Option<&str>,
284		block_announce_validator: Box<dyn BlockAnnounceValidator<B> + Send>,
285		syncing_strategy: Box<dyn SyncingStrategy<B>>,
286		network_service: service::network::NetworkServiceHandle,
287		import_queue: Box<dyn ImportQueueService<B>>,
288		peer_store_handle: Arc<dyn PeerStoreProvider>,
289	) -> Result<(Self, SyncingService<B>, N::NotificationProtocolConfig), ClientError>
290	where
291		N: NetworkBackend<B, <B as BlockT>::Hash>,
292	{
293		let cache_capacity = (net_config.network_config.default_peers_set.in_peers +
294			net_config.network_config.default_peers_set.out_peers)
295			.max(1);
296		let important_peers = {
297			let mut imp_p = HashSet::new();
298			for reserved in &net_config.network_config.default_peers_set.reserved_nodes {
299				imp_p.insert(reserved.peer_id);
300			}
301			for config in net_config.notification_protocols() {
302				let peer_ids = config.set_config().reserved_nodes.iter().map(|info| info.peer_id);
303				imp_p.extend(peer_ids);
304			}
305
306			imp_p.shrink_to_fit();
307			imp_p
308		};
309		let boot_node_ids = {
310			let mut list = HashSet::new();
311			for node in &net_config.network_config.boot_nodes {
312				list.insert(node.peer_id);
313			}
314			list.shrink_to_fit();
315			list
316		};
317		let default_peers_set_no_slot_peers = {
318			let mut no_slot_p: HashSet<PeerId> = net_config
319				.network_config
320				.default_peers_set
321				.reserved_nodes
322				.iter()
323				.map(|reserved| reserved.peer_id)
324				.collect();
325			no_slot_p.shrink_to_fit();
326			no_slot_p
327		};
328		let default_peers_set_num_full =
329			net_config.network_config.default_peers_set_num_full as usize;
330		let default_peers_set_num_light = {
331			let total = net_config.network_config.default_peers_set.out_peers +
332				net_config.network_config.default_peers_set.in_peers;
333			total.saturating_sub(net_config.network_config.default_peers_set_num_full) as usize
334		};
335
336		let info = client.info();
337
338		let (block_announce_config, notification_service) =
339			Self::get_block_announce_proto_config::<N>(
340				protocol_id,
341				fork_id,
342				roles,
343				info.best_number,
344				info.best_hash,
345				info.genesis_hash,
346				&net_config.network_config.default_peers_set,
347				network_metrics,
348				Arc::clone(&peer_store_handle),
349			);
350
351		let block_announce_protocol_name = block_announce_config.protocol_name().clone();
352		let (tx, service_rx) = tracing_unbounded("mpsc_chain_sync", 100_000);
353		let num_connected = Arc::new(AtomicUsize::new(0));
354		let is_major_syncing = Arc::new(AtomicBool::new(false));
355
356		// `default_peers_set.in_peers` contains an unspecified amount of light peers so the number
357		// of full inbound peers must be calculated from the total full peer count
358		let max_full_peers = net_config.network_config.default_peers_set_num_full;
359		let max_out_peers = net_config.network_config.default_peers_set.out_peers;
360		let max_in_peers = (max_full_peers - max_out_peers) as usize;
361
362		let tick_timeout = {
363			let mut interval = tokio::time::interval(TICK_TIMEOUT);
364			interval.set_missed_tick_behavior(MissedTickBehavior::Delay);
365			interval
366		};
367
368		Ok((
369			Self {
370				roles,
371				client,
372				strategy: syncing_strategy,
373				network_service,
374				peers: HashMap::new(),
375				block_announce_data_cache: LruMap::new(ByLength::new(cache_capacity)),
376				block_announce_protocol_name,
377				block_announce_validator: BlockAnnounceValidatorStream::new(
378					block_announce_validator,
379				),
380				num_connected: num_connected.clone(),
381				is_major_syncing: is_major_syncing.clone(),
382				service_rx,
383				genesis_hash: info.genesis_hash,
384				important_peers,
385				default_peers_set_no_slot_connected_peers: HashSet::new(),
386				boot_node_ids,
387				default_peers_set_no_slot_peers,
388				default_peers_set_num_full,
389				default_peers_set_num_light,
390				num_in_peers: 0usize,
391				max_in_peers,
392				event_streams: Vec::new(),
393				notification_service,
394				tick_timeout,
395				peer_store_handle,
396				metrics: if let Some(r) = metrics_registry {
397					match Metrics::register(r, is_major_syncing.clone()) {
398						Ok(metrics) => Some(metrics),
399						Err(err) => {
400							log::error!(target: LOG_TARGET, "Failed to register metrics {err:?}");
401							None
402						},
403					}
404				} else {
405					None
406				},
407				pending_responses: PendingResponses::new(),
408				import_queue,
409			},
410			SyncingService::new(tx, num_connected, is_major_syncing),
411			block_announce_config,
412		))
413	}
414
415	fn update_peer_info(
416		&mut self,
417		peer_id: &PeerId,
418		best_hash: B::Hash,
419		best_number: NumberFor<B>,
420	) {
421		if let Some(ref mut peer) = self.peers.get_mut(peer_id) {
422			peer.info.best_hash = best_hash;
423			peer.info.best_number = best_number;
424		}
425	}
426
427	/// Process the result of the block announce validation.
428	fn process_block_announce_validation_result(
429		&mut self,
430		validation_result: BlockAnnounceValidationResult<B::Header>,
431	) {
432		match validation_result {
433			BlockAnnounceValidationResult::Skip { peer_id: _ } => {},
434			BlockAnnounceValidationResult::Process { is_new_best, peer_id, announce } => {
435				if let Some((best_hash, best_number)) =
436					self.strategy.on_validated_block_announce(is_new_best, peer_id, &announce)
437				{
438					self.update_peer_info(&peer_id, best_hash, best_number);
439				}
440
441				if let Some(data) = announce.data {
442					if !data.is_empty() {
443						self.block_announce_data_cache.insert(announce.header.hash(), data);
444					}
445				}
446			},
447			BlockAnnounceValidationResult::Failure { peer_id, disconnect } => {
448				if disconnect {
449					log::debug!(
450						target: LOG_TARGET,
451						"Disconnecting peer {peer_id} due to block announce validation failure",
452					);
453					self.network_service
454						.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
455				}
456
457				self.network_service.report_peer(peer_id, rep::BAD_BLOCK_ANNOUNCEMENT);
458			},
459		}
460	}
461
462	/// Push a block announce validation.
463	pub fn push_block_announce_validation(
464		&mut self,
465		peer_id: PeerId,
466		announce: BlockAnnounce<B::Header>,
467	) {
468		let hash = announce.header.hash();
469
470		let peer = match self.peers.get_mut(&peer_id) {
471			Some(p) => p,
472			None => {
473				log::error!(
474					target: LOG_TARGET,
475					"Received block announce from disconnected peer {peer_id}",
476				);
477				debug_assert!(false);
478				return;
479			},
480		};
481		peer.known_blocks.insert(hash);
482
483		if peer.info.roles.is_full() {
484			let is_best = match announce.state.unwrap_or(BlockState::Best) {
485				BlockState::Best => true,
486				BlockState::Normal => false,
487			};
488
489			self.block_announce_validator
490				.push_block_announce_validation(peer_id, hash, announce, is_best);
491		}
492	}
493
494	/// Make sure an important block is propagated to peers.
495	///
496	/// In chain-based consensus, we often need to make sure non-best forks are
497	/// at least temporarily synced.
498	pub fn announce_block(&mut self, hash: B::Hash, data: Option<Vec<u8>>) {
499		let header = match self.client.header(hash) {
500			Ok(Some(header)) => header,
501			Ok(None) => {
502				log::warn!(target: LOG_TARGET, "Trying to announce unknown block: {hash}");
503				return;
504			},
505			Err(e) => {
506				log::warn!(target: LOG_TARGET, "Error reading block header {hash}: {e}");
507				return;
508			},
509		};
510
511		// don't announce genesis block since it will be ignored
512		if header.number().is_zero() {
513			return;
514		}
515
516		let is_best = self.client.info().best_hash == hash;
517		log::debug!(target: LOG_TARGET, "Reannouncing block {hash:?} is_best: {is_best}");
518
519		let data = data
520			.or_else(|| self.block_announce_data_cache.get(&hash).cloned())
521			.unwrap_or_default();
522
523		for (peer_id, ref mut peer) in self.peers.iter_mut() {
524			let inserted = peer.known_blocks.insert(hash);
525			if inserted {
526				log::trace!(target: LOG_TARGET, "Announcing block {hash:?} to {peer_id}");
527				let message = BlockAnnounce {
528					header: header.clone(),
529					state: if is_best { Some(BlockState::Best) } else { Some(BlockState::Normal) },
530					data: Some(data.clone()),
531				};
532
533				let _ = self.notification_service.send_sync_notification(peer_id, message.encode());
534			}
535		}
536	}
537
538	pub async fn run(mut self) {
539		loop {
540			tokio::select! {
541				_ = self.tick_timeout.tick() => {
542					// TODO: This tick should not be necessary, but
543					//  `self.process_strategy_actions()` is not called in some cases otherwise and
544					//  some tests fail because of this
545				},
546				command = self.service_rx.select_next_some() =>
547					self.process_service_command(command),
548				notification_event = self.notification_service.next_event() => match notification_event {
549					Some(event) => self.process_notification_event(event),
550					None => {
551						error!(
552							target: LOG_TARGET,
553							"Terminating `SyncingEngine` because `NotificationService` has terminated.",
554						);
555
556						return;
557					}
558				},
559				response_event = self.pending_responses.select_next_some() =>
560					self.process_response_event(response_event),
561				validation_result = self.block_announce_validator.select_next_some() =>
562					self.process_block_announce_validation_result(validation_result),
563			}
564
565			// Update atomic variables
566			self.is_major_syncing.store(self.strategy.is_major_syncing(), Ordering::Relaxed);
567
568			// Process actions requested by a syncing strategy.
569			if let Err(e) = self.process_strategy_actions() {
570				error!(
571					target: LOG_TARGET,
572					"Terminating `SyncingEngine` due to fatal error: {e:?}.",
573				);
574				return;
575			}
576		}
577	}
578
579	fn process_strategy_actions(&mut self) -> Result<(), ClientError> {
580		for action in self.strategy.actions(&self.network_service)? {
581			match action {
582				SyncingAction::StartRequest { peer_id, key, request, remove_obsolete } => {
583					if !self.peers.contains_key(&peer_id) {
584						trace!(
585							target: LOG_TARGET,
586							"Cannot start request with strategy key {key:?} to unknown peer \
587							{peer_id}",
588						);
589						debug_assert!(false);
590						continue;
591					}
592					if remove_obsolete {
593						if self.pending_responses.remove(peer_id, key) {
594							warn!(
595								target: LOG_TARGET,
596								"Processed `SyncingAction::StartRequest` to {peer_id} with \
597								strategy key {key:?}. Stale response removed!",
598							)
599						} else {
600							trace!(
601								target: LOG_TARGET,
602								"Processed `SyncingAction::StartRequest` to {peer_id} with \
603								strategy key {key:?}.",
604							)
605						}
606					}
607
608					self.pending_responses.insert(peer_id, key, request);
609				},
610				SyncingAction::CancelRequest { peer_id, key } => {
611					let removed = self.pending_responses.remove(peer_id, key);
612
613					trace!(
614						target: LOG_TARGET,
615						"Processed `SyncingAction::CancelRequest`, response removed: {removed}.",
616					);
617				},
618				SyncingAction::DropPeer(BadPeer(peer_id, rep)) => {
619					self.pending_responses.remove_all(&peer_id);
620					self.network_service
621						.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
622					self.network_service.report_peer(peer_id, rep);
623
624					trace!(target: LOG_TARGET, "{peer_id:?} dropped: {rep:?}.");
625				},
626				SyncingAction::ImportBlocks { origin, blocks } => {
627					let count = blocks.len();
628					self.import_blocks(origin, blocks);
629
630					trace!(
631						target: LOG_TARGET,
632						"Processed `ChainSyncAction::ImportBlocks` with {count} blocks.",
633					);
634				},
635				SyncingAction::ImportJustifications { peer_id, hash, number, justifications } => {
636					self.import_justifications(peer_id, hash, number, justifications);
637
638					trace!(
639						target: LOG_TARGET,
640						"Processed `ChainSyncAction::ImportJustifications` from peer {} for block {} ({}).",
641						peer_id,
642						hash,
643						number,
644					)
645				},
646				// Nothing to do, this is handled internally by `PolkadotSyncingStrategy`.
647				SyncingAction::Finished => {},
648			}
649		}
650
651		Ok(())
652	}
653
654	fn process_service_command(&mut self, command: ToServiceCommand<B>) {
655		match command {
656			ToServiceCommand::SetSyncForkRequest(peers, hash, number) => {
657				self.strategy.set_sync_fork_request(peers, &hash, number);
658			},
659			ToServiceCommand::EventStream(tx) => {
660				// Let a new subscriber know about already connected peers.
661				for peer_id in self.peers.keys() {
662					let _ = tx.unbounded_send(SyncEvent::PeerConnected(*peer_id));
663				}
664				self.event_streams.push(tx);
665			},
666			ToServiceCommand::RequestJustification(hash, number) =>
667				self.strategy.request_justification(&hash, number),
668			ToServiceCommand::ClearJustificationRequests =>
669				self.strategy.clear_justification_requests(),
670			ToServiceCommand::BlocksProcessed(imported, count, results) => {
671				self.strategy.on_blocks_processed(imported, count, results);
672			},
673			ToServiceCommand::JustificationImported(peer_id, hash, number, success) => {
674				self.strategy.on_justification_import(hash, number, success);
675				if !success {
676					log::info!(
677						target: LOG_TARGET,
678						"💔 Invalid justification provided by {peer_id} for #{hash}",
679					);
680					self.network_service
681						.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
682					self.network_service
683						.report_peer(peer_id, ReputationChange::new_fatal("Invalid justification"));
684				}
685			},
686			ToServiceCommand::AnnounceBlock(hash, data) => self.announce_block(hash, data),
687			ToServiceCommand::NewBestBlockImported(hash, number) => {
688				log::debug!(target: LOG_TARGET, "New best block imported {:?}/#{}", hash, number);
689
690				self.strategy.update_chain_info(&hash, number);
691				let _ = self.notification_service.try_set_handshake(
692					BlockAnnouncesHandshake::<B>::build(
693						self.roles,
694						number,
695						hash,
696						self.genesis_hash,
697					)
698					.encode(),
699				);
700			},
701			ToServiceCommand::Status(tx) => {
702				let _ = tx.send(self.strategy.status());
703			},
704			ToServiceCommand::NumActivePeers(tx) => {
705				let _ = tx.send(self.num_active_peers());
706			},
707			ToServiceCommand::NumDownloadedBlocks(tx) => {
708				let _ = tx.send(self.strategy.num_downloaded_blocks());
709			},
710			ToServiceCommand::NumSyncRequests(tx) => {
711				let _ = tx.send(self.strategy.num_sync_requests());
712			},
713			ToServiceCommand::PeersInfo(tx) => {
714				let peers_info =
715					self.peers.iter().map(|(peer_id, peer)| (*peer_id, peer.info)).collect();
716				let _ = tx.send(peers_info);
717			},
718			ToServiceCommand::OnBlockFinalized(hash, header) =>
719				self.strategy.on_block_finalized(&hash, *header.number()),
720		}
721	}
722
723	fn process_notification_event(&mut self, event: NotificationEvent) {
724		match event {
725			NotificationEvent::ValidateInboundSubstream { peer, handshake, result_tx } => {
726				let validation_result = self
727					.validate_connection(&peer, handshake, Direction::Inbound)
728					.map_or(ValidationResult::Reject, |_| ValidationResult::Accept);
729
730				let _ = result_tx.send(validation_result);
731			},
732			NotificationEvent::NotificationStreamOpened { peer, handshake, direction, .. } => {
733				log::debug!(
734					target: LOG_TARGET,
735					"Substream opened for {peer}, handshake {handshake:?}"
736				);
737
738				match self.validate_connection(&peer, handshake, direction) {
739					Ok(handshake) => {
740						if self.on_sync_peer_connected(peer, &handshake, direction).is_err() {
741							log::debug!(target: LOG_TARGET, "Failed to register peer {peer}");
742							self.network_service
743								.disconnect_peer(peer, self.block_announce_protocol_name.clone());
744						}
745					},
746					Err(wrong_genesis) => {
747						log::debug!(target: LOG_TARGET, "`SyncingEngine` rejected {peer}");
748
749						if wrong_genesis {
750							self.peer_store_handle.report_peer(peer, rep::GENESIS_MISMATCH);
751						}
752
753						self.network_service
754							.disconnect_peer(peer, self.block_announce_protocol_name.clone());
755					},
756				}
757			},
758			NotificationEvent::NotificationStreamClosed { peer } => {
759				self.on_sync_peer_disconnected(peer);
760			},
761			NotificationEvent::NotificationReceived { peer, notification } => {
762				if !self.peers.contains_key(&peer) {
763					log::error!(
764						target: LOG_TARGET,
765						"received notification from {peer} who had been earlier refused by `SyncingEngine`",
766					);
767					return;
768				}
769
770				let Ok(announce) = BlockAnnounce::decode(&mut notification.as_ref()) else {
771					log::warn!(target: LOG_TARGET, "failed to decode block announce");
772					return;
773				};
774
775				self.push_block_announce_validation(peer, announce);
776			},
777		}
778	}
779
780	/// Called by peer when it is disconnecting.
781	///
782	/// Returns a result if the handshake of this peer was indeed accepted.
783	fn on_sync_peer_disconnected(&mut self, peer_id: PeerId) {
784		let Some(info) = self.peers.remove(&peer_id) else {
785			log::debug!(target: LOG_TARGET, "{peer_id} does not exist in `SyncingEngine`");
786			return;
787		};
788		if let Some(metrics) = &self.metrics {
789			metrics.peers.dec();
790		}
791		self.num_connected.fetch_sub(1, Ordering::AcqRel);
792
793		if self.important_peers.contains(&peer_id) {
794			log::warn!(target: LOG_TARGET, "Reserved peer {peer_id} disconnected");
795		} else {
796			log::debug!(target: LOG_TARGET, "{peer_id} disconnected");
797		}
798
799		if !self.default_peers_set_no_slot_connected_peers.remove(&peer_id) &&
800			info.inbound &&
801			info.info.roles.is_full()
802		{
803			match self.num_in_peers.checked_sub(1) {
804				Some(value) => {
805					self.num_in_peers = value;
806				},
807				None => {
808					log::error!(
809						target: LOG_TARGET,
810						"trying to disconnect an inbound node which is not counted as inbound"
811					);
812					debug_assert!(false);
813				},
814			}
815		}
816
817		self.strategy.remove_peer(&peer_id);
818		self.pending_responses.remove_all(&peer_id);
819		self.event_streams
820			.retain(|stream| stream.unbounded_send(SyncEvent::PeerDisconnected(peer_id)).is_ok());
821	}
822
823	/// Validate received handshake.
824	fn validate_handshake(
825		&mut self,
826		peer_id: &PeerId,
827		handshake: Vec<u8>,
828	) -> Result<BlockAnnouncesHandshake<B>, bool> {
829		log::trace!(target: LOG_TARGET, "Validate handshake for {peer_id}");
830
831		let handshake = <BlockAnnouncesHandshake<B> as DecodeAll>::decode_all(&mut &handshake[..])
832			.map_err(|error| {
833				log::debug!(target: LOG_TARGET, "Failed to decode handshake for {peer_id}: {error:?}");
834				false
835			})?;
836
837		if handshake.genesis_hash != self.genesis_hash {
838			if self.important_peers.contains(&peer_id) {
839				log::error!(
840					target: LOG_TARGET,
841					"Reserved peer id `{peer_id}` is on a different chain (our genesis: {} theirs: {})",
842					self.genesis_hash,
843					handshake.genesis_hash,
844				);
845			} else if self.boot_node_ids.contains(&peer_id) {
846				log::error!(
847					target: LOG_TARGET,
848					"Bootnode with peer id `{peer_id}` is on a different chain (our genesis: {} theirs: {})",
849					self.genesis_hash,
850					handshake.genesis_hash,
851				);
852			} else {
853				log::debug!(
854					target: LOG_TARGET,
855					"Peer is on different chain (our genesis: {} theirs: {})",
856					self.genesis_hash,
857					handshake.genesis_hash
858				);
859			}
860
861			return Err(true);
862		}
863
864		Ok(handshake)
865	}
866
867	/// Validate connection.
868	// NOTE Returning `Err(bool)` is a really ugly hack to work around the issue
869	// that `ProtocolController` thinks the peer is connected when in fact it can
870	// still be under validation. If the peer has different genesis than the
871	// local node the validation fails but the peer cannot be reported in
872	// `validate_connection()` as that is also called by
873	// `ValidateInboundSubstream` which means that the peer is still being
874	// validated and banning the peer when handling that event would
875	// result in peer getting dropped twice.
876	//
877	// The proper way to fix this is to integrate `ProtocolController` more
878	// tightly with `NotificationService` or add an additional API call for
879	// banning pre-accepted peers (which is not desirable)
880	fn validate_connection(
881		&mut self,
882		peer_id: &PeerId,
883		handshake: Vec<u8>,
884		direction: Direction,
885	) -> Result<BlockAnnouncesHandshake<B>, bool> {
886		log::trace!(target: LOG_TARGET, "New peer {peer_id} {handshake:?}");
887
888		let handshake = self.validate_handshake(peer_id, handshake)?;
889
890		if self.peers.contains_key(&peer_id) {
891			log::error!(
892				target: LOG_TARGET,
893				"Called `validate_connection()` with already connected peer {peer_id}",
894			);
895			debug_assert!(false);
896			return Err(false);
897		}
898
899		let no_slot_peer = self.default_peers_set_no_slot_peers.contains(&peer_id);
900		let this_peer_reserved_slot: usize = if no_slot_peer { 1 } else { 0 };
901
902		if handshake.roles.is_full() &&
903			self.strategy.num_peers() >=
904				self.default_peers_set_num_full +
905					self.default_peers_set_no_slot_connected_peers.len() +
906					this_peer_reserved_slot
907		{
908			log::debug!(target: LOG_TARGET, "Too many full nodes, rejecting {peer_id}");
909			return Err(false);
910		}
911
912		// make sure to accept no more than `--in-peers` many full nodes
913		if !no_slot_peer &&
914			handshake.roles.is_full() &&
915			direction.is_inbound() &&
916			self.num_in_peers == self.max_in_peers
917		{
918			log::debug!(target: LOG_TARGET, "All inbound slots have been consumed, rejecting {peer_id}");
919			return Err(false);
920		}
921
922		// make sure that all slots are not occupied by light peers
923		//
924		// `ChainSync` only accepts full peers whereas `SyncingEngine` accepts both full and light
925		// peers. Verify that there is a slot in `SyncingEngine` for the inbound light peer
926		if handshake.roles.is_light() &&
927			(self.peers.len() - self.strategy.num_peers()) >= self.default_peers_set_num_light
928		{
929			log::debug!(target: LOG_TARGET, "Too many light nodes, rejecting {peer_id}");
930			return Err(false);
931		}
932
933		Ok(handshake)
934	}
935
936	/// Called on the first connection between two peers on the default set, after their exchange
937	/// of handshake.
938	///
939	/// Returns `Ok` if the handshake is accepted and the peer added to the list of peers we sync
940	/// from.
941	fn on_sync_peer_connected(
942		&mut self,
943		peer_id: PeerId,
944		status: &BlockAnnouncesHandshake<B>,
945		direction: Direction,
946	) -> Result<(), ()> {
947		log::trace!(target: LOG_TARGET, "New peer {peer_id} {status:?}");
948
949		let peer = Peer {
950			info: ExtendedPeerInfo {
951				roles: status.roles,
952				best_hash: status.best_hash,
953				best_number: status.best_number,
954			},
955			known_blocks: LruHashSet::new(
956				NonZeroUsize::new(MAX_KNOWN_BLOCKS).expect("Constant is nonzero"),
957			),
958			inbound: direction.is_inbound(),
959		};
960
961		// Only forward full peers to syncing strategy.
962		if status.roles.is_full() {
963			self.strategy.add_peer(peer_id, peer.info.best_hash, peer.info.best_number);
964		}
965
966		log::debug!(target: LOG_TARGET, "Connected {peer_id}");
967
968		if self.peers.insert(peer_id, peer).is_none() {
969			if let Some(metrics) = &self.metrics {
970				metrics.peers.inc();
971			}
972			self.num_connected.fetch_add(1, Ordering::AcqRel);
973		}
974		self.peer_store_handle.set_peer_role(&peer_id, status.roles.into());
975
976		if self.default_peers_set_no_slot_peers.contains(&peer_id) {
977			self.default_peers_set_no_slot_connected_peers.insert(peer_id);
978		} else if direction.is_inbound() && status.roles.is_full() {
979			self.num_in_peers += 1;
980		}
981
982		self.event_streams
983			.retain(|stream| stream.unbounded_send(SyncEvent::PeerConnected(peer_id)).is_ok());
984
985		Ok(())
986	}
987
988	fn process_response_event(&mut self, response_event: ResponseEvent) {
989		let ResponseEvent { peer_id, key, response: response_result } = response_event;
990
991		match response_result {
992			Ok(Ok((response, protocol_name))) => {
993				self.strategy.on_generic_response(&peer_id, key, protocol_name, response);
994			},
995			Ok(Err(e)) => {
996				debug!(target: LOG_TARGET, "Request to peer {peer_id:?} failed: {e:?}.");
997
998				match e {
999					RequestFailure::Network(OutboundFailure::Timeout) => {
1000						self.network_service.report_peer(peer_id, rep::TIMEOUT);
1001						self.network_service
1002							.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1003					},
1004					RequestFailure::Network(OutboundFailure::UnsupportedProtocols) => {
1005						self.network_service.report_peer(peer_id, rep::BAD_PROTOCOL);
1006						self.network_service
1007							.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1008					},
1009					RequestFailure::Network(OutboundFailure::DialFailure) => {
1010						self.network_service
1011							.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1012					},
1013					RequestFailure::Refused => {
1014						self.network_service.report_peer(peer_id, rep::REFUSED);
1015						self.network_service
1016							.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1017					},
1018					RequestFailure::Network(OutboundFailure::ConnectionClosed) |
1019					RequestFailure::NotConnected => {
1020						self.network_service
1021							.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1022					},
1023					RequestFailure::UnknownProtocol => {
1024						debug_assert!(false, "Block request protocol should always be known.");
1025					},
1026					RequestFailure::Obsolete => {
1027						debug_assert!(
1028							false,
1029							"Can not receive `RequestFailure::Obsolete` after dropping the \
1030							response receiver.",
1031						);
1032					},
1033					RequestFailure::Network(OutboundFailure::Io(_)) => {
1034						self.network_service.report_peer(peer_id, rep::IO);
1035						self.network_service
1036							.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1037					},
1038				}
1039			},
1040			Err(oneshot::Canceled) => {
1041				trace!(
1042					target: LOG_TARGET,
1043					"Request to peer {peer_id:?} failed due to oneshot being canceled.",
1044				);
1045				self.network_service
1046					.disconnect_peer(peer_id, self.block_announce_protocol_name.clone());
1047			},
1048		}
1049	}
1050
1051	/// Returns the number of peers we're connected to and that are being queried.
1052	fn num_active_peers(&self) -> usize {
1053		self.pending_responses.len()
1054	}
1055
1056	/// Get config for the block announcement protocol
1057	fn get_block_announce_proto_config<N: NetworkBackend<B, <B as BlockT>::Hash>>(
1058		protocol_id: ProtocolId,
1059		fork_id: Option<&str>,
1060		roles: Roles,
1061		best_number: NumberFor<B>,
1062		best_hash: B::Hash,
1063		genesis_hash: B::Hash,
1064		set_config: &SetConfig,
1065		metrics: NotificationMetrics,
1066		peer_store_handle: Arc<dyn PeerStoreProvider>,
1067	) -> (N::NotificationProtocolConfig, Box<dyn NotificationService>) {
1068		let block_announces_protocol = {
1069			let genesis_hash = genesis_hash.as_ref();
1070			if let Some(fork_id) = fork_id {
1071				format!(
1072					"/{}/{}/block-announces/1",
1073					array_bytes::bytes2hex("", genesis_hash),
1074					fork_id
1075				)
1076			} else {
1077				format!("/{}/block-announces/1", array_bytes::bytes2hex("", genesis_hash))
1078			}
1079		};
1080
1081		N::notification_config(
1082			block_announces_protocol.into(),
1083			iter::once(format!("/{}/block-announces/1", protocol_id.as_ref()).into()).collect(),
1084			MAX_BLOCK_ANNOUNCE_SIZE,
1085			Some(NotificationHandshake::new(BlockAnnouncesHandshake::<B>::build(
1086				roles,
1087				best_number,
1088				best_hash,
1089				genesis_hash,
1090			))),
1091			set_config.clone(),
1092			metrics,
1093			peer_store_handle,
1094		)
1095	}
1096
1097	/// Import blocks.
1098	fn import_blocks(&mut self, origin: BlockOrigin, blocks: Vec<IncomingBlock<B>>) {
1099		if let Some(metrics) = &self.metrics {
1100			metrics.import_queue_blocks_submitted.inc();
1101		}
1102
1103		self.import_queue.import_blocks(origin, blocks);
1104	}
1105
1106	/// Import justifications.
1107	fn import_justifications(
1108		&mut self,
1109		peer_id: PeerId,
1110		hash: B::Hash,
1111		number: NumberFor<B>,
1112		justifications: Justifications,
1113	) {
1114		if let Some(metrics) = &self.metrics {
1115			metrics.import_queue_justifications_submitted.inc();
1116		}
1117
1118		self.import_queue.import_justifications(peer_id, hash, number, justifications);
1119	}
1120}