chore: add and fix more lints, improve docs (#4765)

This commit is contained in:
DaniPopes
2023-09-25 17:46:46 +02:00
committed by GitHub
parent b701cbc9a3
commit 8f9d2908ca
134 changed files with 709 additions and 625 deletions

View File

@ -12,6 +12,7 @@ use tokio::sync::mpsc;
pub(crate) const ETH_REQUEST_CHANNEL_CAPACITY: usize = 256;
/// A builder that can configure all components of the network.
#[allow(missing_debug_implementations)]
pub struct NetworkBuilder<C, Tx, Eth> {
pub(crate) network: NetworkManager<C>,
pub(crate) transactions: Tx,

View File

@ -27,6 +27,7 @@ pub fn rng_secret_key() -> SecretKey {
}
/// All network related initialization settings.
#[derive(Debug)]
pub struct NetworkConfig<C> {
/// The client type that can interact with the chain.
///

View File

@ -23,8 +23,8 @@ use tokio_stream::wrappers::ReceiverStream;
/// An abstraction over the configured discovery protocol.
///
/// Listens for new discovered nodes and emits events for discovered nodes and their
/// address.#[derive(Debug, Clone)]
/// address.
#[derive(Debug)]
pub struct Discovery {
/// All nodes discovered via discovery protocol.
///

View File

@ -53,6 +53,7 @@ const APPROX_HEADER_SIZE: usize = 500;
/// Manages eth related requests on top of the p2p network.
///
/// This can be spawned to another task and is supposed to be run as background service.
#[derive(Debug)]
#[must_use = "Manager does nothing unless polled."]
pub struct EthRequestHandler<C> {
/// The client type that can interact with the chain.

View File

@ -30,6 +30,7 @@ pub use client::FetchClient;
/// peers and sends the response once ready.
///
/// This type maintains a list of connected peers that are available for requests.
#[derive(Debug)]
pub struct StateFetcher {
/// Currently active [`GetBlockHeaders`] requests
inflight_headers_requests:
@ -296,6 +297,7 @@ enum PollAction {
}
/// Represents a connected peer
#[derive(Debug)]
struct Peer {
/// The state this peer currently resides in.
state: PeerState,
@ -314,6 +316,7 @@ impl Peer {
}
/// Tracks the state of an individual peer
#[derive(Debug)]
enum PeerState {
/// Peer is currently not handling requests and is available.
Idle,
@ -349,6 +352,7 @@ impl PeerState {
/// A request that waits for a response from the network, so it can send it back through the
/// response channel.
#[derive(Debug)]
struct Request<Req, Resp> {
/// The issued request object
/// TODO: this can be attached to the response in error case
@ -358,6 +362,7 @@ struct Request<Req, Resp> {
}
/// Requests that can be sent to the Syncer from a [`FetchClient`]
#[derive(Debug)]
pub(crate) enum DownloadRequest {
/// Download the requested headers and send response through channel
GetBlockHeaders {

View File

@ -3,7 +3,7 @@ use reth_primitives::PeerId;
use std::task::{Context, Poll};
/// Abstraction over block import.
pub trait BlockImport: Send + Sync {
pub trait BlockImport: std::fmt::Debug + Send + Sync {
/// Invoked for a received `NewBlock` broadcast message from the peer.
///
/// > When a `NewBlock` announcement message is received from a peer, the client first verifies
@ -18,6 +18,7 @@ pub trait BlockImport: Send + Sync {
}
/// Outcome of the [`BlockImport`]'s block handling.
#[derive(Debug)]
pub struct BlockImportOutcome {
/// Sender of the `NewBlock` message.
pub peer: PeerId,

View File

@ -1,17 +1,3 @@
#![cfg_attr(docsrs, feature(doc_cfg))]
#![doc(
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
issue_tracker_base_url = "https://github.com/paradigmxzy/reth/issues/"
)]
#![warn(missing_docs)]
#![deny(unused_must_use, rust_2018_idioms, rustdoc::broken_intra_doc_links)]
#![allow(rustdoc::private_intra_doc_links)]
#![doc(test(
no_crate_inject,
attr(deny(warnings, rust_2018_idioms), allow(dead_code, unused_variables))
))]
//! reth P2P networking.
//!
//! Ethereum's networking protocol is specified in [devp2p](https://github.com/ethereum/devp2p).
@ -119,6 +105,15 @@
//! - `test-utils`: Various utilities helpful for writing tests
//! - `geth-tests`: Runs tests that require Geth to be installed locally.
#![doc(
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
issue_tracker_base_url = "https://github.com/paradigmxzy/reth/issues/"
)]
#![warn(missing_debug_implementations, missing_docs, rustdoc::all)] // TODO(danipopes): unreachable_pub
#![deny(unused_must_use, rust_2018_idioms)]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
#[cfg(any(test, feature = "test-utils"))]
/// Common helpers for network testing.
pub mod test_utils;
@ -145,7 +140,7 @@ pub mod transactions;
pub use builder::NetworkBuilder;
pub use config::{NetworkConfig, NetworkConfigBuilder};
pub use discovery::Discovery;
pub use discovery::{Discovery, DiscoveryEvent};
pub use fetch::FetchClient;
pub use manager::{NetworkEvent, NetworkManager};
pub use message::PeerRequest;

View File

@ -85,6 +85,7 @@ use tracing::{debug, error, trace, warn};
/// ethrequest <--> |ETH request handing| NetworkManager
/// discovery --> |Discovered peers| NetworkManager
/// ```
#[derive(Debug)]
#[must_use = "The NetworkManager does nothing unless polled"]
pub struct NetworkManager<C> {
/// The type that manages the actual network part, which includes connections.

View File

@ -157,7 +157,7 @@ impl NetworkHandle {
})
}
/// Provides a shareable reference to the [`BandwidthMeter`] stored on the [`NetworkInner`]
/// Provides a shareable reference to the [`BandwidthMeter`] stored on the `NetworkInner`.
pub fn bandwidth_meter(&self) -> &BandwidthMeter {
&self.inner.bandwidth_meter
}
@ -321,6 +321,7 @@ struct NetworkInner {
/// Internal messages that can be passed to the [`NetworkManager`](crate::NetworkManager).
#[allow(missing_docs)]
#[derive(Debug)]
pub(crate) enum NetworkHandleMessage {
/// Adds an address for a peer.
AddPeerAddress(PeerId, PeerKind, SocketAddr),

View File

@ -81,7 +81,8 @@ impl PeersHandle {
/// From this type, connections to peers are established or disconnected, see [`PeerAction`].
///
/// The [`PeersManager`] will be notified on peer related changes
pub(crate) struct PeersManager {
#[derive(Debug)]
pub struct PeersManager {
/// All peers known to the network
peers: HashMap<PeerId, Peer>,
/// Copy of the sender half, so new [`PeersHandle`] can be created on demand.
@ -117,7 +118,7 @@ pub(crate) struct PeersManager {
impl PeersManager {
/// Create a new instance with the given config
pub(crate) fn new(config: PeersConfig) -> Self {
pub fn new(config: PeersConfig) -> Self {
let PeersConfig {
refill_slots_interval,
connection_info,
@ -1020,6 +1021,7 @@ impl PeerConnectionState {
}
/// Commands the [`PeersManager`] listens for.
#[derive(Debug)]
pub(crate) enum PeerCommand {
/// Command for manually add
Add(PeerId, SocketAddr),
@ -1046,28 +1048,47 @@ pub enum PeerAction {
remote_addr: SocketAddr,
},
/// Disconnect an existing connection.
Disconnect { peer_id: PeerId, reason: Option<DisconnectReason> },
Disconnect {
/// The peer ID of the established connection.
peer_id: PeerId,
/// An optional reason for the disconnect.
reason: Option<DisconnectReason>,
},
/// Disconnect an existing incoming connection, because the peers reputation is below the
/// banned threshold or is on the [`BanList`]
DisconnectBannedIncoming {
/// Peer id of the established connection.
/// The peer ID of the established connection.
peer_id: PeerId,
},
/// Ban the peer in discovery.
DiscoveryBanPeerId { peer_id: PeerId, ip_addr: IpAddr },
DiscoveryBanPeerId {
/// The peer ID.
peer_id: PeerId,
/// The IP address.
ip_addr: IpAddr,
},
/// Ban the IP in discovery.
DiscoveryBanIp { ip_addr: IpAddr },
DiscoveryBanIp {
/// The IP address.
ip_addr: IpAddr,
},
/// Ban the peer temporarily
BanPeer { peer_id: PeerId },
BanPeer {
/// The peer ID.
peer_id: PeerId,
},
/// Unban the peer temporarily
UnBanPeer { peer_id: PeerId },
UnBanPeer {
/// The peer ID.
peer_id: PeerId,
},
/// Emit peerAdded event
PeerAdded(PeerId),
/// Emit peerRemoved event
PeerRemoved(PeerId),
}
/// Config type for initiating a [`PeersManager`] instance
/// Config type for initiating a [`PeersManager`] instance.
#[derive(Debug, Clone, PartialEq)]
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
#[cfg_attr(feature = "serde", serde(default))]

View File

@ -3,8 +3,8 @@
mod manager;
mod reputation;
pub(crate) use manager::{InboundConnectionError, PeerAction, PeersManager};
pub use manager::{Peer, PeersConfig, PeersHandle};
pub(crate) use manager::InboundConnectionError;
pub use manager::{Peer, PeerAction, PeersConfig, PeersHandle, PeersManager};
pub use reputation::ReputationChangeWeights;
pub use reth_network_api::PeerKind;

View File

@ -44,6 +44,7 @@ const PEER_BLOCK_CACHE_LIMIT: usize = 512;
/// then send to the session of the peer.
///
/// This type is also responsible for responding for received request.
#[derive(Debug)]
pub struct NetworkState<C> {
/// All active peers and their state.
active_peers: HashMap<PeerId, ActivePeer>,
@ -465,6 +466,7 @@ where
/// Tracks the state of a Peer with an active Session.
///
/// For example known blocks,so we can decide what to announce.
#[derive(Debug)]
pub(crate) struct ActivePeer {
/// Best block of the peer.
pub(crate) best_hash: H256,
@ -480,6 +482,7 @@ pub(crate) struct ActivePeer {
}
/// Message variants triggered by the [`NetworkState`]
#[derive(Debug)]
pub(crate) enum StateAction {
/// Dispatch a `NewBlock` message to the peer
NewBlock {

View File

@ -61,6 +61,7 @@ use tracing::{debug, trace};
/// fetchRequest --> |request Headers, Bodies| StateFetch
/// State --> |poll pending requests| StateFetch
/// ```
#[derive(Debug)]
#[must_use = "Swarm does nothing unless polled"]
pub(crate) struct Swarm<C> {
/// Listens for new incoming connections.
@ -428,7 +429,7 @@ pub(crate) enum SwarmEvent {
/// Represents the state of the connection of the node. If shutting down,
/// new connections won't be established.
#[derive(Default)]
#[derive(Debug, Default)]
pub(crate) enum NetworkConnectionState {
#[default]
Active,

View File

@ -1,5 +1,3 @@
#![warn(missing_docs, unreachable_pub)]
//! Common helpers for network testing.
mod init;
@ -9,4 +7,4 @@ pub use init::{
enr_to_peer_id, unused_port, unused_tcp_addr, unused_tcp_and_udp_port, unused_tcp_udp,
unused_udp_addr, unused_udp_port, GETH_TIMEOUT,
};
pub use testnet::{NetworkEventStream, PeerConfig, Testnet};
pub use testnet::{NetworkEventStream, Peer, PeerConfig, Testnet};

View File

@ -187,6 +187,7 @@ where
}
/// A handle to a [`Testnet`] that can be shared.
#[derive(Debug)]
pub struct TestnetHandle<C> {
_handle: JoinHandle<()>,
terminate: oneshot::Sender<oneshot::Sender<Testnet<C>>>,
@ -203,7 +204,9 @@ impl<C> TestnetHandle<C> {
}
}
/// A peer in the [`Testnet`].
#[pin_project]
#[derive(Debug)]
pub struct Peer<C> {
#[pin]
network: NetworkManager<C>,
@ -262,6 +265,7 @@ where
}
/// A helper config for setting up the reth networking stack.
#[derive(Debug)]
pub struct PeerConfig<C = NoopProvider> {
config: NetworkConfig<C>,
client: C,
@ -327,6 +331,7 @@ impl Default for PeerConfig {
/// A helper type to await network events
///
/// This makes it easier to await established connections
#[derive(Debug)]
pub struct NetworkEventStream {
inner: UnboundedReceiverStream<NetworkEvent>,
}

View File

@ -59,6 +59,7 @@ const GET_POOLED_TRANSACTION_SOFT_LIMIT_SIZE: GetPooledTransactionLimit =
pub type PoolImportFuture = Pin<Box<dyn Future<Output = PoolResult<TxHash>> + Send + 'static>>;
/// Api to interact with [`TransactionsManager`] task.
#[derive(Debug)]
pub struct TransactionsHandle {
/// Command channel to the [`TransactionsManager`]
manager_tx: mpsc::UnboundedSender<TransactionsCommand>,
@ -117,6 +118,7 @@ impl TransactionsHandle {
///
/// It is directly connected to the [`TransactionPool`] to retrieve requested transactions and
/// propagate new transactions over the network.
#[derive(Debug)]
#[must_use = "Manager does nothing unless polled."]
pub struct TransactionsManager<Pool> {
/// Access to the transaction pool.
@ -844,6 +846,7 @@ impl Future for GetPooledTxRequestFut {
}
/// Tracks a single peer
#[derive(Debug)]
struct Peer {
/// Keeps track of transactions that we know the peer has seen.
transactions: LruCache<H256>,
@ -857,6 +860,7 @@ struct Peer {
}
/// Commands to send to the [`TransactionsManager`]
#[derive(Debug)]
enum TransactionsCommand {
/// Propagate a transaction hash to the network.
PropagateHash(H256),