feat: make downloaders and clients generic over block parts (#12469)

Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
Arsenii Kulikov
2024-11-12 19:13:21 +04:00
committed by GitHub
parent 3a337cd7d4
commit aece53ae88
60 changed files with 631 additions and 409 deletions

View File

@ -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

View File

@ -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(

View File

@ -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, _) => {

View File

@ -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 {

View File

@ -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

View File

@ -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>,
}