diff --git a/Cargo.lock b/Cargo.lock index d29023f330ce1..062195c70f8ab 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1717,6 +1717,12 @@ version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" +[[package]] +name = "downcast" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" + [[package]] name = "downcast-rs" version = "1.2.0" @@ -2093,6 +2099,15 @@ dependencies = [ "miniz_oxide", ] +[[package]] +name = "float-cmp" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "98de4bbd547a563b716d8dfa9aad1cb19bfab00f4fa09a6a4ed21dbcf44ce9c4" +dependencies = [ + "num-traits", +] + [[package]] name = "fnv" version = "1.0.7" @@ -2116,6 +2131,12 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "fragile" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85dcb89d2b10c5f6133de2efd8c11959ce9dbb46a2f7a4cab208c4eeda6ce1ab" + [[package]] name = "frame-benchmarking" version = "4.0.0-dev" @@ -4395,6 +4416,33 @@ dependencies = [ "windows-sys 0.36.1", ] +[[package]] +name = "mockall" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2be9a9090bc1cac2930688fa9478092a64c6a92ddc6ae0692d46b37d9cab709" +dependencies = [ + "cfg-if 1.0.0", + "downcast", + "fragile", + "lazy_static", + "mockall_derive", + "predicates", + "predicates-tree", +] + +[[package]] +name = "mockall_derive" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86d702a0530a0141cf4ed147cf5ec7be6f2c187d4e37fcbefc39cf34116bfe8f" +dependencies = [ + "cfg-if 1.0.0", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "more-asserts" version = "0.2.1" @@ -4970,6 +5018,12 @@ dependencies = [ "version_check", ] +[[package]] +name = "normalize-line-endings" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61807f77802ff30975e01f4f071c8ba10c022052f98b3294119f3e615d13e5be" + [[package]] name = "num-bigint" version = "0.2.6" @@ -6942,8 +6996,11 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c143348f141cc87aab5b950021bac6145d0e5ae754b0591de23244cee42c9308" dependencies = [ "difflib", + "float-cmp", "itertools", + "normalize-line-endings", "predicates-core", + "regex", ] [[package]] @@ -8586,6 +8643,7 @@ dependencies = [ "libp2p", "log", "lru", + "mockall", "parity-scale-codec", "prost 0.11.0", "prost-build 0.11.1", diff --git a/client/network/common/src/sync.rs b/client/network/common/src/sync.rs index 020b2c9efa4c7..d3603c6792c84 100644 --- a/client/network/common/src/sync.rs +++ b/client/network/common/src/sync.rs @@ -269,12 +269,14 @@ pub trait ChainSync: Send { ); /// Get an iterator over all scheduled justification requests. - fn justification_requests( - &mut self, - ) -> Box)> + '_>; + fn justification_requests<'a>( + &'a mut self, + ) -> Box)> + 'a>; /// Get an iterator over all block requests of all peers. - fn block_requests(&mut self) -> Box)> + '_>; + fn block_requests<'a>( + &'a mut self, + ) -> Box)> + 'a>; /// Get a state request, if any. fn state_request(&mut self) -> Option<(PeerId, OpaqueStateRequest)>; @@ -359,9 +361,9 @@ pub trait ChainSync: Send { /// /// If [`PollBlockAnnounceValidation::ImportHeader`] is returned, then the caller MUST try to /// import passed header (call `on_block_data`). The network request isn't sent in this case. - fn poll_block_announce_validation( + fn poll_block_announce_validation<'a>( &mut self, - cx: &mut std::task::Context, + cx: &mut std::task::Context<'a>, ) -> Poll>; /// Call when a peer has disconnected. diff --git a/client/network/src/protocol.rs b/client/network/src/protocol.rs index 325e044527efa..c3def8adc6cfe 100644 --- a/client/network/src/protocol.rs +++ b/client/network/src/protocol.rs @@ -1440,7 +1440,7 @@ where for (id, request) in self .chain_sync .block_requests() - .map(|(peer_id, request)| (*peer_id, request)) + .map(|(peer_id, request)| (peer_id, request)) .collect::>() { let event = diff --git a/client/network/src/service.rs b/client/network/src/service.rs index 28e479b702779..25916041285a3 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -92,6 +92,8 @@ use std::{ pub use behaviour::{InboundFailure, OutboundFailure, ResponseFailure}; +#[cfg(test)] +mod chainsync_tests; mod metrics; mod out_events; #[cfg(test)] diff --git a/client/network/src/service/chainsync_tests.rs b/client/network/src/service/chainsync_tests.rs new file mode 100644 index 0000000000000..ca44c65d267f4 --- /dev/null +++ b/client/network/src/service/chainsync_tests.rs @@ -0,0 +1,339 @@ +// This file is part of Substrate. + +// Copyright (C) 2022 Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +use crate::{config, NetworkWorker}; + +use futures::prelude::*; +use libp2p::PeerId; +use sc_block_builder::BlockBuilderProvider; +use sc_client_api::{BlockBackend, HeaderBackend}; +use sc_consensus::JustificationSyncLink; +use sc_network_common::{ + config::{ + NonDefaultSetConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, SetConfig, + TransportConfig, + }, + protocol::role::Roles, + service::NetworkSyncForkRequest, + sync::{message::BlockAnnouncesHandshake, ChainSync as ChainSyncT, SyncState, SyncStatus}, +}; +use sc_network_light::light_client_requests::handler::LightClientRequestHandler; +use sc_network_sync::{ + block_request_handler::BlockRequestHandler, mock::MockChainSync, + state_request_handler::StateRequestHandler, +}; +use sp_core::H256; +use sp_runtime::{ + generic::BlockId, + traits::{Block as BlockT, Header as _, Zero}, +}; +use std::{iter, sync::Arc, task::Poll}; +use substrate_test_runtime_client::{TestClientBuilder, TestClientBuilderExt as _}; + +type TestNetworkWorker = NetworkWorker< + substrate_test_runtime_client::runtime::Block, + substrate_test_runtime_client::runtime::Hash, + substrate_test_runtime_client::TestClient, +>; + +const BLOCK_ANNOUNCE_PROTO_NAME: &str = "/block-announces"; +const PROTOCOL_NAME: &str = "/foo"; + +fn make_network( + chain_sync: Box>, + client: Arc, +) -> (TestNetworkWorker, Arc) { + let network_config = config::NetworkConfiguration { + extra_sets: vec![NonDefaultSetConfig { + notifications_protocol: PROTOCOL_NAME.into(), + fallback_names: Vec::new(), + max_notification_size: 1024 * 1024, + handshake: None, + set_config: Default::default(), + }], + listen_addresses: vec![config::build_multiaddr![Memory(rand::random::())]], + transport: TransportConfig::MemoryOnly, + ..config::NetworkConfiguration::new_local() + }; + + #[derive(Clone)] + struct PassThroughVerifier(bool); + + #[async_trait::async_trait] + impl sc_consensus::Verifier for PassThroughVerifier { + async fn verify( + &mut self, + mut block: sc_consensus::BlockImportParams, + ) -> Result< + ( + sc_consensus::BlockImportParams, + Option)>>, + ), + String, + > { + let maybe_keys = block + .header + .digest() + .log(|l| { + l.try_as_raw(sp_runtime::generic::OpaqueDigestItemId::Consensus(b"aura")) + .or_else(|| { + l.try_as_raw(sp_runtime::generic::OpaqueDigestItemId::Consensus( + b"babe", + )) + }) + }) + .map(|blob| { + vec![(sp_blockchain::well_known_cache_keys::AUTHORITIES, blob.to_vec())] + }); + + block.finalized = self.0; + block.fork_choice = Some(sc_consensus::ForkChoiceStrategy::LongestChain); + Ok((block, maybe_keys)) + } + } + + let import_queue = Box::new(sc_consensus::BasicQueue::new( + PassThroughVerifier(false), + Box::new(client.clone()), + None, + &sp_core::testing::TaskExecutor::new(), + None, + )); + + let protocol_id = ProtocolId::from("/test-protocol-name"); + + let fork_id = Some(String::from("test-fork-id")); + + let block_request_protocol_config = { + let (handler, protocol_config) = + BlockRequestHandler::new(&protocol_id, None, client.clone(), 50); + async_std::task::spawn(handler.run().boxed()); + protocol_config + }; + + let state_request_protocol_config = { + let (handler, protocol_config) = + StateRequestHandler::new(&protocol_id, None, client.clone(), 50); + async_std::task::spawn(handler.run().boxed()); + protocol_config + }; + + let light_client_request_protocol_config = { + let (handler, protocol_config) = + LightClientRequestHandler::new(&protocol_id, None, client.clone()); + async_std::task::spawn(handler.run().boxed()); + protocol_config + }; + + let block_announce_config = NonDefaultSetConfig { + notifications_protocol: BLOCK_ANNOUNCE_PROTO_NAME.into(), + fallback_names: vec![], + max_notification_size: 1024 * 1024, + handshake: Some(NotificationHandshake::new(BlockAnnouncesHandshake::< + substrate_test_runtime_client::runtime::Block, + >::build( + Roles::from(&config::Role::Full), + client.info().best_number, + client.info().best_hash, + client + .block_hash(Zero::zero()) + .ok() + .flatten() + .expect("Genesis block exists; qed"), + ))), + set_config: SetConfig { + in_peers: 0, + out_peers: 0, + reserved_nodes: Vec::new(), + non_reserved_mode: NonReservedPeerMode::Deny, + }, + }; + + let worker = NetworkWorker::new(config::Params { + block_announce_config, + role: config::Role::Full, + executor: None, + network_config, + chain: client.clone(), + protocol_id, + fork_id, + import_queue, + chain_sync, + metrics_registry: None, + block_request_protocol_config, + state_request_protocol_config, + light_client_request_protocol_config, + warp_sync_protocol_config: None, + request_response_protocol_configs: Vec::new(), + }) + .unwrap(); + + (worker, client) +} + +fn set_default_expecations_no_peers( + chain_sync: &mut MockChainSync, +) { + chain_sync.expect_block_requests().returning(|| Box::new(iter::empty())); + chain_sync.expect_state_request().returning(|| None); + chain_sync.expect_justification_requests().returning(|| Box::new(iter::empty())); + chain_sync.expect_warp_sync_request().returning(|| None); + chain_sync.expect_poll_block_announce_validation().returning(|_| Poll::Pending); + chain_sync.expect_status().returning(|| SyncStatus { + state: SyncState::Idle, + best_seen_block: None, + num_peers: 0u32, + queued_blocks: 0u32, + state_sync: None, + warp_sync: None, + }); +} + +#[async_std::test] +async fn normal_network_poll_no_peers() { + let client = Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0); + let mut chain_sync = + Box::new(MockChainSync::::new()); + set_default_expecations_no_peers(&mut chain_sync); + + let (mut network, _) = make_network(chain_sync, client); + + // poll the network once + futures::future::poll_fn(|cx| { + let _ = network.poll_unpin(cx); + Poll::Ready(()) + }) + .await; +} + +#[async_std::test] +async fn request_justification() { + let client = Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0); + let mut chain_sync = + Box::new(MockChainSync::::new()); + + let hash = H256::random(); + let number = 1337u64; + + chain_sync + .expect_request_justification() + .withf(move |in_hash, in_number| &hash == in_hash && &number == in_number) + .once() + .returning(|_, _| ()); + + set_default_expecations_no_peers(&mut chain_sync); + let (mut network, _) = make_network(chain_sync, client); + + // send "request justifiction" message and poll the network + network.service().request_justification(&hash, number); + + futures::future::poll_fn(|cx| { + let _ = network.poll_unpin(cx); + Poll::Ready(()) + }) + .await; +} + +#[async_std::test] +async fn clear_justification_requests(&mut self) { + let client = Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0); + let mut chain_sync = + Box::new(MockChainSync::::new()); + + chain_sync.expect_clear_justification_requests().once().returning(|| ()); + + set_default_expecations_no_peers(&mut chain_sync); + let (mut network, _) = make_network(chain_sync, client); + + // send "request justifiction" message and poll the network + network.service().clear_justification_requests(); + + futures::future::poll_fn(|cx| { + let _ = network.poll_unpin(cx); + Poll::Ready(()) + }) + .await; +} + +#[async_std::test] +async fn set_sync_fork_request() { + let client = Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0); + let mut chain_sync = + Box::new(MockChainSync::::new()); + + let hash = H256::random(); + let number = 1337u64; + let peers = (0..3).map(|_| PeerId::random()).collect::>(); + let copy_peers = peers.clone(); + + chain_sync + .expect_set_sync_fork_request() + .withf(move |in_peers, in_hash, in_number| { + &peers == in_peers && &hash == in_hash && &number == in_number + }) + .once() + .returning(|_, _, _| ()); + + set_default_expecations_no_peers(&mut chain_sync); + let (mut network, _) = make_network(chain_sync, client); + + // send "set sync fork request" message and poll the network + network.service().set_sync_fork_request(copy_peers, hash, number); + + futures::future::poll_fn(|cx| { + let _ = network.poll_unpin(cx); + Poll::Ready(()) + }) + .await; +} + +#[async_std::test] +async fn on_block_finalized() { + let client = Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0); + let mut chain_sync = + Box::new(MockChainSync::::new()); + + let at = client.header(&BlockId::Hash(client.info().best_hash)).unwrap().unwrap().hash(); + let block = client + .new_block_at(&BlockId::Hash(at), Default::default(), false) + .unwrap() + .build() + .unwrap() + .block; + let header = block.header.clone(); + let block_number = *header.number(); + let hash = block.hash(); + + chain_sync + .expect_on_block_finalized() + .withf(move |in_hash, in_number| &hash == in_hash && &block_number == in_number) + .once() + .returning(|_, _| ()); + + set_default_expecations_no_peers(&mut chain_sync); + let (mut network, _) = make_network(chain_sync, client); + + // send "set sync fork request" message and poll the network + network.on_block_finalized(hash, header); + + futures::future::poll_fn(|cx| { + let _ = network.poll_unpin(cx); + Poll::Ready(()) + }) + .await; +} diff --git a/client/network/sync/Cargo.toml b/client/network/sync/Cargo.toml index 24d418f7233d7..9d032f5cca96c 100644 --- a/client/network/sync/Cargo.toml +++ b/client/network/sync/Cargo.toml @@ -25,6 +25,7 @@ futures = "0.3.21" libp2p = "0.46.1" log = "0.4.17" lru = "0.7.5" +mockall = "0.11.2" prost = "0.11" smallvec = "1.8.0" thiserror = "1.0" diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index 280e530eca9a9..84998c747b3cc 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -30,6 +30,7 @@ pub mod block_request_handler; pub mod blocks; +pub mod mock; mod schema; pub mod state; pub mod state_request_handler; @@ -643,9 +644,9 @@ where .extend(peers); } - fn justification_requests( - &mut self, - ) -> Box)> + '_> { + fn justification_requests<'a>( + &'a mut self, + ) -> Box)> + 'a> { let peers = &mut self.peers; let mut matcher = self.extra_justifications.matcher(); Box::new(std::iter::from_fn(move || { @@ -670,7 +671,9 @@ where })) } - fn block_requests(&mut self) -> Box)> + '_> { + fn block_requests<'a>( + &'a mut self, + ) -> Box)> + 'a> { if self.mode == SyncMode::Warp { return Box::new(std::iter::once(self.warp_target_block_request()).flatten()) } @@ -695,8 +698,8 @@ where let allowed_requests = self.allowed_requests.take(); let max_parallel = if major_sync { 1 } else { self.max_parallel_downloads }; let gap_sync = &mut self.gap_sync; - let iter = self.peers.iter_mut().filter_map(move |(id, peer)| { - if !peer.state.is_available() || !allowed_requests.contains(id) { + let iter = self.peers.iter_mut().filter_map(move |(&id, peer)| { + if !peer.state.is_available() || !allowed_requests.contains(&id) { return None } @@ -725,7 +728,7 @@ where }; Some((id, ancestry_request::(current))) } else if let Some((range, req)) = peer_block_request( - id, + &id, peer, blocks, attrs, @@ -744,7 +747,7 @@ where ); Some((id, req)) } else if let Some((hash, req)) = - fork_sync_request(id, fork_targets, best_queued, last_finalized, attrs, |hash| { + fork_sync_request(&id, fork_targets, best_queued, last_finalized, attrs, |hash| { if queue.contains(hash) { BlockStatus::Queued } else { @@ -756,7 +759,7 @@ where Some((id, req)) } else if let Some((range, req)) = gap_sync.as_mut().and_then(|sync| { peer_gap_block_request( - id, + &id, peer, &mut sync.blocks, attrs, @@ -2216,7 +2219,7 @@ where } /// Generate block request for downloading of the target block body during warp sync. - fn warp_target_block_request(&mut self) -> Option<(&PeerId, BlockRequest)> { + fn warp_target_block_request(&mut self) -> Option<(PeerId, BlockRequest)> { if let Some(sync) = &self.warp_sync { if self.allowed_requests.is_empty() || sync.is_complete() || @@ -2234,7 +2237,7 @@ where trace!(target: "sync", "New warp target block request for {}", id); peer.state = PeerSyncState::DownloadingWarpTargetBlock; self.allowed_requests.clear(); - return Some((id, request)) + return Some((*id, request)) } } } @@ -2482,7 +2485,7 @@ fn fork_sync_request( true }); for (hash, r) in targets { - if !r.peers.contains(id) { + if !r.peers.contains(&id) { continue } // Download the fork only if it is behind or not too far ahead our tip of the chain @@ -2740,7 +2743,7 @@ mod test { // we wil send block requests to these peers // for these blocks we don't know about - assert!(sync.block_requests().all(|(p, _)| { *p == peer_id1 || *p == peer_id2 })); + assert!(sync.block_requests().all(|(p, _)| { p == peer_id1 || p == peer_id2 })); // add a new peer at a known block sync.new_peer(peer_id3, b1_hash, b1_number).unwrap(); @@ -2835,7 +2838,7 @@ mod test { log::trace!(target: "sync", "Requests: {:?}", requests); assert_eq!(1, requests.len()); - assert_eq!(peer, requests[0].0); + assert_eq!(*peer, requests[0].0); let request = requests[0].1.clone(); @@ -3065,9 +3068,9 @@ mod test { send_block_announce(best_block.header().clone(), &peer_id2, &mut sync); let (peer1_req, peer2_req) = sync.block_requests().fold((None, None), |res, req| { - if req.0 == &peer_id1 { + if req.0 == peer_id1 { (Some(req.1), res.1) - } else if req.0 == &peer_id2 { + } else if req.0 == peer_id2 { (res.0, Some(req.1)) } else { panic!("Unexpected req: {:?}", req) diff --git a/client/network/sync/src/mock.rs b/client/network/sync/src/mock.rs new file mode 100644 index 0000000000000..2a3b059f735b2 --- /dev/null +++ b/client/network/sync/src/mock.rs @@ -0,0 +1,118 @@ +// This file is part of Substrate. + +// Copyright (C) 2022 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 . + +//! Contains a mock implementation of `ChainSync` that can be used +//! for testing calls made to `ChainSync`. + +use futures::task::Poll; +use libp2p::PeerId; +use sc_consensus::{BlockImportError, BlockImportStatus}; +use sc_network_common::sync::{ + message::{BlockAnnounce, BlockData, BlockRequest, BlockResponse}, + warp::{EncodedProof, WarpProofRequest}, + BadPeer, ChainSync as ChainSyncT, Metrics, OnBlockData, OnBlockJustification, OnStateData, + OpaqueBlockRequest, OpaqueBlockResponse, OpaqueStateRequest, OpaqueStateResponse, PeerInfo, + PollBlockAnnounceValidation, SyncStatus, +}; +use sp_runtime::traits::{Block as BlockT, NumberFor}; + +mockall::mock! { + pub ChainSync {} + + impl ChainSyncT for ChainSync { + fn peer_info(&self, who: &PeerId) -> Option>; + fn status(&self) -> SyncStatus; + fn num_sync_requests(&self) -> usize; + fn num_downloaded_blocks(&self) -> usize; + fn num_peers(&self) -> usize; + fn new_peer( + &mut self, + who: PeerId, + best_hash: Block::Hash, + best_number: NumberFor, + ) -> Result>, BadPeer>; + fn update_chain_info(&mut self, best_hash: &Block::Hash, best_number: NumberFor); + fn request_justification(&mut self, hash: &Block::Hash, number: NumberFor); + fn clear_justification_requests(&mut self); + fn set_sync_fork_request( + &mut self, + peers: Vec, + hash: &Block::Hash, + number: NumberFor, + ); + fn justification_requests<'a>( + &'a mut self, + ) -> Box)> + 'a>; + fn block_requests<'a>(&'a mut self) -> Box)> + 'a>; + fn state_request(&mut self) -> Option<(PeerId, OpaqueStateRequest)>; + fn warp_sync_request(&mut self) -> Option<(PeerId, WarpProofRequest)>; + fn on_block_data( + &mut self, + who: &PeerId, + request: Option>, + response: BlockResponse, + ) -> Result, BadPeer>; + fn on_state_data( + &mut self, + who: &PeerId, + response: OpaqueStateResponse, + ) -> Result, BadPeer>; + fn on_warp_sync_data(&mut self, who: &PeerId, response: EncodedProof) -> Result<(), BadPeer>; + fn on_block_justification( + &mut self, + who: PeerId, + response: BlockResponse, + ) -> Result, BadPeer>; + fn on_blocks_processed( + &mut self, + imported: usize, + count: usize, + results: Vec<(Result>, BlockImportError>, Block::Hash)>, + ) -> Box), BadPeer>>>; + fn on_justification_import( + &mut self, + hash: Block::Hash, + number: NumberFor, + 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) -> Option>; + fn metrics(&self) -> Metrics; + fn create_opaque_block_request(&self, request: &BlockRequest) -> OpaqueBlockRequest; + fn encode_block_request(&self, request: &OpaqueBlockRequest) -> Result, String>; + fn decode_block_response(&self, response: &[u8]) -> Result; + fn block_response_into_blocks( + &self, + request: &BlockRequest, + response: OpaqueBlockResponse, + ) -> Result>, String>; + fn encode_state_request(&self, request: &OpaqueStateRequest) -> Result, String>; + fn decode_state_response(&self, response: &[u8]) -> Result; + } +}