mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +00:00
feat: make downloaders and clients generic over block parts (#12469)
Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
@ -34,6 +34,7 @@ reth-network-peers = { workspace = true, features = ["net"] }
|
||||
reth-network-types.workspace = true
|
||||
|
||||
# ethereum
|
||||
alloy-consensus.workspace = true
|
||||
alloy-eips.workspace = true
|
||||
alloy-primitives.workspace = true
|
||||
alloy-rlp.workspace = true
|
||||
|
||||
@ -7,6 +7,7 @@ use std::sync::{
|
||||
|
||||
use alloy_primitives::B256;
|
||||
use futures::{future, future::Either};
|
||||
use reth_eth_wire::{EthNetworkPrimitives, NetworkPrimitives};
|
||||
use reth_network_api::test_utils::PeersHandle;
|
||||
use reth_network_p2p::{
|
||||
bodies::client::{BodiesClient, BodiesFut},
|
||||
@ -17,7 +18,6 @@ use reth_network_p2p::{
|
||||
};
|
||||
use reth_network_peers::PeerId;
|
||||
use reth_network_types::ReputationChangeKind;
|
||||
use reth_primitives::Header;
|
||||
use tokio::sync::{mpsc::UnboundedSender, oneshot};
|
||||
|
||||
use crate::{fetch::DownloadRequest, flattened_response::FlattenedResponse};
|
||||
@ -30,16 +30,16 @@ use crate::{fetch::DownloadRequest, flattened_response::FlattenedResponse};
|
||||
///
|
||||
/// include_mmd!("docs/mermaid/fetch-client.mmd")
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct FetchClient {
|
||||
pub struct FetchClient<N: NetworkPrimitives = EthNetworkPrimitives> {
|
||||
/// Sender half of the request channel.
|
||||
pub(crate) request_tx: UnboundedSender<DownloadRequest>,
|
||||
pub(crate) request_tx: UnboundedSender<DownloadRequest<N>>,
|
||||
/// The handle to the peers
|
||||
pub(crate) peers_handle: PeersHandle,
|
||||
/// Number of active peer sessions the node's currently handling.
|
||||
pub(crate) num_active_peers: Arc<AtomicUsize>,
|
||||
}
|
||||
|
||||
impl DownloadClient for FetchClient {
|
||||
impl<N: NetworkPrimitives> DownloadClient for FetchClient<N> {
|
||||
fn report_bad_message(&self, peer_id: PeerId) {
|
||||
self.peers_handle.reputation_change(peer_id, ReputationChangeKind::BadMessage);
|
||||
}
|
||||
@ -53,8 +53,9 @@ impl DownloadClient for FetchClient {
|
||||
// or an error.
|
||||
type HeadersClientFuture<T> = Either<FlattenedResponse<T>, future::Ready<T>>;
|
||||
|
||||
impl HeadersClient for FetchClient {
|
||||
type Output = HeadersClientFuture<PeerRequestResult<Vec<Header>>>;
|
||||
impl<N: NetworkPrimitives> HeadersClient for FetchClient<N> {
|
||||
type Header = N::BlockHeader;
|
||||
type Output = HeadersClientFuture<PeerRequestResult<Vec<N::BlockHeader>>>;
|
||||
|
||||
/// Sends a `GetBlockHeaders` request to an available peer.
|
||||
fn get_headers_with_priority(
|
||||
@ -75,8 +76,9 @@ impl HeadersClient for FetchClient {
|
||||
}
|
||||
}
|
||||
|
||||
impl BodiesClient for FetchClient {
|
||||
type Output = BodiesFut;
|
||||
impl<N: NetworkPrimitives> BodiesClient for FetchClient<N> {
|
||||
type Body = N::BlockBody;
|
||||
type Output = BodiesFut<N::BlockBody>;
|
||||
|
||||
/// Sends a `GetBlockBodies` request to an available peer.
|
||||
fn get_block_bodies_with_priority(
|
||||
|
||||
@ -15,7 +15,7 @@ use std::{
|
||||
|
||||
use alloy_primitives::B256;
|
||||
use futures::StreamExt;
|
||||
use reth_eth_wire::{GetBlockBodies, GetBlockHeaders};
|
||||
use reth_eth_wire::{EthNetworkPrimitives, GetBlockBodies, GetBlockHeaders, NetworkPrimitives};
|
||||
use reth_network_api::test_utils::PeersHandle;
|
||||
use reth_network_p2p::{
|
||||
error::{EthResponseValidator, PeerRequestResult, RequestError, RequestResult},
|
||||
@ -24,12 +24,14 @@ use reth_network_p2p::{
|
||||
};
|
||||
use reth_network_peers::PeerId;
|
||||
use reth_network_types::ReputationChangeKind;
|
||||
use reth_primitives::{BlockBody, Header};
|
||||
use tokio::sync::{mpsc, mpsc::UnboundedSender, oneshot};
|
||||
use tokio_stream::wrappers::UnboundedReceiverStream;
|
||||
|
||||
use crate::message::BlockRequest;
|
||||
|
||||
type InflightHeadersRequest<H> = Request<HeadersRequest, PeerRequestResult<Vec<H>>>;
|
||||
type InflightBodiesRequest<B> = Request<Vec<B256>, PeerRequestResult<Vec<B>>>;
|
||||
|
||||
/// Manages data fetching operations.
|
||||
///
|
||||
/// This type is hooked into the staged sync pipeline and delegates download request to available
|
||||
@ -37,13 +39,11 @@ use crate::message::BlockRequest;
|
||||
///
|
||||
/// This type maintains a list of connected peers that are available for requests.
|
||||
#[derive(Debug)]
|
||||
pub struct StateFetcher {
|
||||
pub struct StateFetcher<N: NetworkPrimitives = EthNetworkPrimitives> {
|
||||
/// Currently active [`GetBlockHeaders`] requests
|
||||
inflight_headers_requests:
|
||||
HashMap<PeerId, Request<HeadersRequest, PeerRequestResult<Vec<Header>>>>,
|
||||
inflight_headers_requests: HashMap<PeerId, InflightHeadersRequest<N::BlockHeader>>,
|
||||
/// Currently active [`GetBlockBodies`] requests
|
||||
inflight_bodies_requests:
|
||||
HashMap<PeerId, Request<Vec<B256>, PeerRequestResult<Vec<BlockBody>>>>,
|
||||
inflight_bodies_requests: HashMap<PeerId, InflightBodiesRequest<N::BlockBody>>,
|
||||
/// The list of _available_ peers for requests.
|
||||
peers: HashMap<PeerId, Peer>,
|
||||
/// The handle to the peers manager
|
||||
@ -51,16 +51,16 @@ pub struct StateFetcher {
|
||||
/// Number of active peer sessions the node's currently handling.
|
||||
num_active_peers: Arc<AtomicUsize>,
|
||||
/// Requests queued for processing
|
||||
queued_requests: VecDeque<DownloadRequest>,
|
||||
queued_requests: VecDeque<DownloadRequest<N>>,
|
||||
/// Receiver for new incoming download requests
|
||||
download_requests_rx: UnboundedReceiverStream<DownloadRequest>,
|
||||
download_requests_rx: UnboundedReceiverStream<DownloadRequest<N>>,
|
||||
/// Sender for download requests, used to detach a [`FetchClient`]
|
||||
download_requests_tx: UnboundedSender<DownloadRequest>,
|
||||
download_requests_tx: UnboundedSender<DownloadRequest<N>>,
|
||||
}
|
||||
|
||||
// === impl StateSyncer ===
|
||||
|
||||
impl StateFetcher {
|
||||
impl<N: NetworkPrimitives> StateFetcher<N> {
|
||||
pub(crate) fn new(peers_handle: PeersHandle, num_active_peers: Arc<AtomicUsize>) -> Self {
|
||||
let (download_requests_tx, download_requests_rx) = mpsc::unbounded_channel();
|
||||
Self {
|
||||
@ -217,7 +217,7 @@ impl StateFetcher {
|
||||
/// Handles a new request to a peer.
|
||||
///
|
||||
/// Caution: this assumes the peer exists and is idle
|
||||
fn prepare_block_request(&mut self, peer_id: PeerId, req: DownloadRequest) -> BlockRequest {
|
||||
fn prepare_block_request(&mut self, peer_id: PeerId, req: DownloadRequest<N>) -> BlockRequest {
|
||||
// update the peer's state
|
||||
if let Some(peer) = self.peers.get_mut(&peer_id) {
|
||||
peer.state = req.peer_state();
|
||||
@ -260,7 +260,7 @@ impl StateFetcher {
|
||||
pub(crate) fn on_block_headers_response(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
res: RequestResult<Vec<Header>>,
|
||||
res: RequestResult<Vec<N::BlockHeader>>,
|
||||
) -> Option<BlockResponseOutcome> {
|
||||
let is_error = res.is_err();
|
||||
let maybe_reputation_change = res.reputation_change_err();
|
||||
@ -296,7 +296,7 @@ impl StateFetcher {
|
||||
pub(crate) fn on_block_bodies_response(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
res: RequestResult<Vec<BlockBody>>,
|
||||
res: RequestResult<Vec<N::BlockBody>>,
|
||||
) -> Option<BlockResponseOutcome> {
|
||||
let is_likely_bad_response = res.as_ref().map_or(true, |bodies| bodies.is_empty());
|
||||
|
||||
@ -315,7 +315,7 @@ impl StateFetcher {
|
||||
}
|
||||
|
||||
/// Returns a new [`FetchClient`] that can send requests to this type.
|
||||
pub(crate) fn client(&self) -> FetchClient {
|
||||
pub(crate) fn client(&self) -> FetchClient<N> {
|
||||
FetchClient {
|
||||
request_tx: self.download_requests_tx.clone(),
|
||||
peers_handle: self.peers_handle.clone(),
|
||||
@ -405,24 +405,24 @@ struct Request<Req, Resp> {
|
||||
|
||||
/// Requests that can be sent to the Syncer from a [`FetchClient`]
|
||||
#[derive(Debug)]
|
||||
pub(crate) enum DownloadRequest {
|
||||
pub(crate) enum DownloadRequest<N: NetworkPrimitives> {
|
||||
/// Download the requested headers and send response through channel
|
||||
GetBlockHeaders {
|
||||
request: HeadersRequest,
|
||||
response: oneshot::Sender<PeerRequestResult<Vec<Header>>>,
|
||||
response: oneshot::Sender<PeerRequestResult<Vec<N::BlockHeader>>>,
|
||||
priority: Priority,
|
||||
},
|
||||
/// Download the requested headers and send response through channel
|
||||
GetBlockBodies {
|
||||
request: Vec<B256>,
|
||||
response: oneshot::Sender<PeerRequestResult<Vec<BlockBody>>>,
|
||||
response: oneshot::Sender<PeerRequestResult<Vec<N::BlockBody>>>,
|
||||
priority: Priority,
|
||||
},
|
||||
}
|
||||
|
||||
// === impl DownloadRequest ===
|
||||
|
||||
impl DownloadRequest {
|
||||
impl<N: NetworkPrimitives> DownloadRequest<N> {
|
||||
/// Returns the corresponding state for a peer that handles the request.
|
||||
const fn peer_state(&self) -> PeerState {
|
||||
match self {
|
||||
@ -472,13 +472,14 @@ pub(crate) enum BlockResponseOutcome {
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::{peers::PeersManager, PeersConfig};
|
||||
use alloy_consensus::Header;
|
||||
use alloy_primitives::B512;
|
||||
use std::future::poll_fn;
|
||||
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
async fn test_poll_fetcher() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let mut fetcher: StateFetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
|
||||
poll_fn(move |cx| {
|
||||
assert!(fetcher.poll(cx).is_pending());
|
||||
@ -498,7 +499,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_peer_rotation() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let mut fetcher: StateFetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
// Add a few random peers
|
||||
let peer1 = B512::random();
|
||||
let peer2 = B512::random();
|
||||
@ -521,7 +522,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_peer_prioritization() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let mut fetcher: StateFetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
// Add a few random peers
|
||||
let peer1 = B512::random();
|
||||
let peer2 = B512::random();
|
||||
@ -546,7 +547,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_on_block_headers_response() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let mut fetcher: StateFetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let peer_id = B512::random();
|
||||
|
||||
assert_eq!(fetcher.on_block_headers_response(peer_id, Ok(vec![Header::default()])), None);
|
||||
@ -576,7 +577,7 @@ mod tests {
|
||||
#[tokio::test]
|
||||
async fn test_header_response_outcome() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let mut fetcher: StateFetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let peer_id = B512::random();
|
||||
|
||||
let request_pair = || {
|
||||
@ -610,7 +611,10 @@ mod tests {
|
||||
let outcome =
|
||||
fetcher.on_block_headers_response(peer_id, Err(RequestError::Timeout)).unwrap();
|
||||
|
||||
assert!(EthResponseValidator::reputation_change_err(&Err(RequestError::Timeout)).is_some());
|
||||
assert!(EthResponseValidator::reputation_change_err(&Err::<Vec<Header>, _>(
|
||||
RequestError::Timeout
|
||||
))
|
||||
.is_some());
|
||||
|
||||
match outcome {
|
||||
BlockResponseOutcome::BadResponse(peer, _) => {
|
||||
|
||||
@ -12,12 +12,13 @@ use alloy_primitives::{Bytes, B256};
|
||||
use futures::FutureExt;
|
||||
use reth_eth_wire::{
|
||||
capability::RawCapabilityMessage, message::RequestPair, BlockBodies, BlockHeaders, EthMessage,
|
||||
GetBlockBodies, GetBlockHeaders, NewBlock, NewBlockHashes, NewPooledTransactionHashes,
|
||||
NodeData, PooledTransactions, Receipts, SharedTransactions, Transactions,
|
||||
EthNetworkPrimitives, GetBlockBodies, GetBlockHeaders, NetworkPrimitives, NewBlock,
|
||||
NewBlockHashes, NewPooledTransactionHashes, NodeData, PooledTransactions, Receipts,
|
||||
SharedTransactions, Transactions,
|
||||
};
|
||||
use reth_network_api::PeerRequest;
|
||||
use reth_network_p2p::error::{RequestError, RequestResult};
|
||||
use reth_primitives::{BlockBody, Header, PooledTransactionsElement, ReceiptWithBloom};
|
||||
use reth_primitives::{PooledTransactionsElement, ReceiptWithBloom};
|
||||
use tokio::sync::oneshot;
|
||||
|
||||
/// Internal form of a `NewBlock` message
|
||||
@ -74,16 +75,16 @@ pub enum BlockRequest {
|
||||
|
||||
/// Corresponding variant for [`PeerRequest`].
|
||||
#[derive(Debug)]
|
||||
pub enum PeerResponse {
|
||||
pub enum PeerResponse<N: NetworkPrimitives = EthNetworkPrimitives> {
|
||||
/// Represents a response to a request for block headers.
|
||||
BlockHeaders {
|
||||
/// The receiver channel for the response to a block headers request.
|
||||
response: oneshot::Receiver<RequestResult<BlockHeaders>>,
|
||||
response: oneshot::Receiver<RequestResult<BlockHeaders<N::BlockHeader>>>,
|
||||
},
|
||||
/// Represents a response to a request for block bodies.
|
||||
BlockBodies {
|
||||
/// The receiver channel for the response to a block bodies request.
|
||||
response: oneshot::Receiver<RequestResult<BlockBodies>>,
|
||||
response: oneshot::Receiver<RequestResult<BlockBodies<N::BlockBody>>>,
|
||||
},
|
||||
/// Represents a response to a request for pooled transactions.
|
||||
PooledTransactions {
|
||||
@ -104,9 +105,9 @@ pub enum PeerResponse {
|
||||
|
||||
// === impl PeerResponse ===
|
||||
|
||||
impl PeerResponse {
|
||||
impl<N: NetworkPrimitives> PeerResponse<N> {
|
||||
/// Polls the type to completion.
|
||||
pub(crate) fn poll(&mut self, cx: &mut Context<'_>) -> Poll<PeerResponseResult> {
|
||||
pub(crate) fn poll(&mut self, cx: &mut Context<'_>) -> Poll<PeerResponseResult<N>> {
|
||||
macro_rules! poll_request {
|
||||
($response:ident, $item:ident, $cx:ident) => {
|
||||
match ready!($response.poll_unpin($cx)) {
|
||||
@ -139,11 +140,11 @@ impl PeerResponse {
|
||||
|
||||
/// All response variants for [`PeerResponse`]
|
||||
#[derive(Debug)]
|
||||
pub enum PeerResponseResult {
|
||||
pub enum PeerResponseResult<N: NetworkPrimitives = EthNetworkPrimitives> {
|
||||
/// Represents a result containing block headers or an error.
|
||||
BlockHeaders(RequestResult<Vec<Header>>),
|
||||
BlockHeaders(RequestResult<Vec<N::BlockHeader>>),
|
||||
/// Represents a result containing block bodies or an error.
|
||||
BlockBodies(RequestResult<Vec<BlockBody>>),
|
||||
BlockBodies(RequestResult<Vec<N::BlockBody>>),
|
||||
/// Represents a result containing pooled transactions or an error.
|
||||
PooledTransactions(RequestResult<Vec<PooledTransactionsElement>>),
|
||||
/// Represents a result containing node data or an error.
|
||||
@ -154,9 +155,9 @@ pub enum PeerResponseResult {
|
||||
|
||||
// === impl PeerResponseResult ===
|
||||
|
||||
impl PeerResponseResult {
|
||||
impl<N: NetworkPrimitives> PeerResponseResult<N> {
|
||||
/// Converts this response into an [`EthMessage`]
|
||||
pub fn try_into_message(self, id: u64) -> RequestResult<EthMessage> {
|
||||
pub fn try_into_message(self, id: u64) -> RequestResult<EthMessage<N>> {
|
||||
macro_rules! to_message {
|
||||
($response:ident, $item:ident, $request_id:ident) => {
|
||||
match $response {
|
||||
|
||||
@ -18,10 +18,7 @@ use reth_network_api::{
|
||||
NetworkEventListenerProvider, NetworkInfo, NetworkStatus, PeerInfo, PeerRequest, Peers,
|
||||
PeersInfo,
|
||||
};
|
||||
use reth_network_p2p::{
|
||||
sync::{NetworkSyncUpdater, SyncState, SyncStateProvider},
|
||||
BlockClient,
|
||||
};
|
||||
use reth_network_p2p::sync::{NetworkSyncUpdater, SyncState, SyncStateProvider};
|
||||
use reth_network_peers::{NodeRecord, PeerId};
|
||||
use reth_network_types::{PeerAddr, PeerKind, Reputation, ReputationChangeKind};
|
||||
use reth_primitives::{Head, TransactionSigned};
|
||||
@ -400,7 +397,9 @@ impl NetworkSyncUpdater for NetworkHandle {
|
||||
}
|
||||
|
||||
impl BlockDownloaderProvider for NetworkHandle {
|
||||
async fn fetch_client(&self) -> Result<impl BlockClient + 'static, oneshot::error::RecvError> {
|
||||
type Client = FetchClient;
|
||||
|
||||
async fn fetch_client(&self) -> Result<Self::Client, oneshot::error::RecvError> {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
let _ = self.manager().send(NetworkHandleMessage::FetchClient(tx));
|
||||
rx.await
|
||||
|
||||
@ -14,7 +14,10 @@ use std::{
|
||||
|
||||
use alloy_primitives::B256;
|
||||
use rand::seq::SliceRandom;
|
||||
use reth_eth_wire::{BlockHashNumber, Capabilities, DisconnectReason, NewBlockHashes, Status};
|
||||
use reth_eth_wire::{
|
||||
BlockHashNumber, Capabilities, DisconnectReason, EthNetworkPrimitives, NetworkPrimitives,
|
||||
NewBlockHashes, Status,
|
||||
};
|
||||
use reth_network_api::{DiscoveredEvent, DiscoveryEvent, PeerRequest, PeerRequestSender};
|
||||
use reth_network_peers::PeerId;
|
||||
use reth_network_types::{PeerAddr, PeerKind};
|
||||
@ -69,9 +72,9 @@ impl Deref for BlockNumReader {
|
||||
///
|
||||
/// This type is also responsible for responding for received request.
|
||||
#[derive(Debug)]
|
||||
pub struct NetworkState {
|
||||
pub struct NetworkState<N: NetworkPrimitives = EthNetworkPrimitives> {
|
||||
/// All active peers and their state.
|
||||
active_peers: HashMap<PeerId, ActivePeer>,
|
||||
active_peers: HashMap<PeerId, ActivePeer<N>>,
|
||||
/// Manages connections to peers.
|
||||
peers_manager: PeersManager,
|
||||
/// Buffered messages until polled.
|
||||
@ -88,10 +91,10 @@ pub struct NetworkState {
|
||||
/// The fetcher streams `RLPx` related requests on a per-peer basis to this type. This type
|
||||
/// will then queue in the request and notify the fetcher once the result has been
|
||||
/// received.
|
||||
state_fetcher: StateFetcher,
|
||||
state_fetcher: StateFetcher<N>,
|
||||
}
|
||||
|
||||
impl NetworkState {
|
||||
impl<N: NetworkPrimitives> NetworkState<N> {
|
||||
/// Create a new state instance with the given params
|
||||
pub(crate) fn new(
|
||||
client: BlockNumReader,
|
||||
@ -126,7 +129,7 @@ impl NetworkState {
|
||||
}
|
||||
|
||||
/// Returns a new [`FetchClient`]
|
||||
pub(crate) fn fetch_client(&self) -> FetchClient {
|
||||
pub(crate) fn fetch_client(&self) -> FetchClient<N> {
|
||||
self.state_fetcher.client()
|
||||
}
|
||||
|
||||
@ -144,7 +147,7 @@ impl NetworkState {
|
||||
peer: PeerId,
|
||||
capabilities: Arc<Capabilities>,
|
||||
status: Arc<Status>,
|
||||
request_tx: PeerRequestSender,
|
||||
request_tx: PeerRequestSender<PeerRequest<N>>,
|
||||
timeout: Arc<AtomicU64>,
|
||||
) {
|
||||
debug_assert!(!self.active_peers.contains_key(&peer), "Already connected; not possible");
|
||||
@ -399,7 +402,11 @@ impl NetworkState {
|
||||
/// Delegates the response result to the fetcher which may return an outcome specific
|
||||
/// instruction that needs to be handled in [`Self::on_block_response_outcome`]. This could be
|
||||
/// a follow-up request or an instruction to slash the peer's reputation.
|
||||
fn on_eth_response(&mut self, peer: PeerId, resp: PeerResponseResult) -> Option<StateAction> {
|
||||
fn on_eth_response(
|
||||
&mut self,
|
||||
peer: PeerId,
|
||||
resp: PeerResponseResult<N>,
|
||||
) -> Option<StateAction> {
|
||||
match resp {
|
||||
PeerResponseResult::BlockHeaders(res) => {
|
||||
let outcome = self.state_fetcher.on_block_headers_response(peer, res)?;
|
||||
@ -492,16 +499,16 @@ impl NetworkState {
|
||||
///
|
||||
/// For example known blocks,so we can decide what to announce.
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct ActivePeer {
|
||||
pub(crate) struct ActivePeer<N: NetworkPrimitives> {
|
||||
/// Best block of the peer.
|
||||
pub(crate) best_hash: B256,
|
||||
/// The capabilities of the remote peer.
|
||||
#[allow(dead_code)]
|
||||
pub(crate) capabilities: Arc<Capabilities>,
|
||||
/// A communication channel directly to the session task.
|
||||
pub(crate) request_tx: PeerRequestSender,
|
||||
pub(crate) request_tx: PeerRequestSender<PeerRequest<N>>,
|
||||
/// The response receiver for a currently active request to that peer.
|
||||
pub(crate) pending_response: Option<PeerResponse>,
|
||||
pub(crate) pending_response: Option<PeerResponse<N>>,
|
||||
/// Blocks we know the peer has.
|
||||
pub(crate) blocks: LruCache<B256>,
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user