Move reputation types from reth-network-api to reth-network-types (#9914)

Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
Emilia Hane
2024-07-31 22:08:38 +02:00
committed by GitHub
parent 02d25304f9
commit 78703b5729
16 changed files with 144 additions and 133 deletions

4
Cargo.lock generated
View File

@ -7565,6 +7565,7 @@ dependencies = [
"enr",
"reth-eth-wire",
"reth-network-peers",
"reth-network-types",
"serde",
"thiserror",
"tokio",
@ -7579,8 +7580,8 @@ dependencies = [
"parking_lot 0.12.3",
"reth-consensus",
"reth-eth-wire-types",
"reth-network-api",
"reth-network-peers",
"reth-network-types",
"reth-primitives",
"reth-storage-errors",
"thiserror",
@ -7610,7 +7611,6 @@ version = "1.0.3"
dependencies = [
"humantime-serde",
"reth-net-banlist",
"reth-network-api",
"reth-network-peers",
"serde",
"serde_json",

View File

@ -127,7 +127,7 @@ pub mod tasks {
/// Re-exported from `reth_network`.
pub mod network {
pub use reth_network::*;
pub use reth_network_api::{noop, reputation, NetworkInfo, PeerKind, Peers, PeersInfo};
pub use reth_network_api::{noop, NetworkInfo, PeerKind, Peers, PeersInfo};
}
/// Re-exported from `reth_transaction_pool`.

View File

@ -16,6 +16,7 @@ workspace = true
reth-eth-wire.workspace = true
alloy-rpc-types-admin.workspace = true
reth-network-peers.workspace = true
reth-network-types.workspace = true
# ethereum
alloy-primitives.workspace = true

View File

@ -13,20 +13,20 @@
)]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
use std::{future::Future, net::SocketAddr, sync::Arc, time::Instant};
pub use alloy_rpc_types_admin::EthProtocolInfo;
pub use error::NetworkError;
pub use reputation::{Reputation, ReputationChangeKind};
use reth_eth_wire::{capability::Capabilities, DisconnectReason, EthVersion, Status};
use reth_network_peers::NodeRecord;
use std::{future::Future, net::SocketAddr, sync::Arc, time::Instant};
pub use reth_network_types::{Reputation, ReputationChangeKind};
/// The `PeerId` type.
pub type PeerId = alloy_primitives::B512;
/// Network Error
pub mod error;
/// Reputation score
pub mod reputation;
/// Implementation of network traits for that does nothing.
pub mod noop;

View File

@ -1,52 +0,0 @@
/// The type that tracks the reputation score.
pub type Reputation = i32;
/// Various kinds of reputation changes.
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum ReputationChangeKind {
/// Received an unspecific bad message from the peer
BadMessage,
/// Peer sent a bad block.
///
/// Note: this will we only used in pre-merge, pow consensus, since after no more block announcements are sent via devp2p: [EIP-3675](https://eips.ethereum.org/EIPS/eip-3675#devp2p)
BadBlock,
/// Peer sent a bad transaction message. E.g. Transactions which weren't recoverable.
BadTransactions,
/// Peer sent a bad announcement message, e.g. invalid transaction type for the configured
/// network.
BadAnnouncement,
/// Peer sent a message that included a hash or transaction that we already received from the
/// peer.
///
/// According to the [Eth spec](https://github.com/ethereum/devp2p/blob/master/caps/eth.md):
///
/// > A node should never send a transaction back to a peer that it can determine already knows
/// > of it (either because it was previously sent or because it was informed from this peer
/// > originally). This is usually achieved by remembering a set of transaction hashes recently
/// > relayed by the peer.
AlreadySeenTransaction,
/// Peer failed to respond in time.
Timeout,
/// Peer does not adhere to network protocol rules.
BadProtocol,
/// Failed to establish a connection to the peer.
FailedToConnect,
/// Connection dropped by peer.
Dropped,
/// Reset the reputation to the default value.
Reset,
/// Apply a reputation change by value
Other(Reputation),
}
impl ReputationChangeKind {
/// Returns true if the reputation change is a [`ReputationChangeKind::Reset`].
pub const fn is_reset(&self) -> bool {
matches!(self, Self::Reset)
}
/// Returns true if the reputation change is [`ReputationChangeKind::Dropped`].
pub const fn is_dropped(&self) -> bool {
matches!(self, Self::Dropped)
}
}

View File

@ -13,7 +13,6 @@ workspace = true
[dependencies]
# reth
reth-network-api.workspace = true
reth-network-peers.workspace = true
reth-net-banlist.workspace = true

View File

@ -14,7 +14,10 @@
/// Types related to peering.
pub mod peers;
pub use peers::{ConnectionsConfig, PeersConfig, ReputationChangeWeights};
pub use peers::{
reputation::{Reputation, ReputationChangeKind, ReputationChangeWeights},
ConnectionsConfig, PeersConfig,
};
pub mod session;
pub use session::{SessionLimits, SessionsConfig};

View File

@ -1,7 +1,5 @@
//! Peer reputation management
use reth_network_api::{Reputation, ReputationChangeKind};
/// The default reputation of a peer
pub const DEFAULT_REPUTATION: Reputation = 0;
@ -50,6 +48,59 @@ pub const fn is_banned_reputation(reputation: i32) -> bool {
reputation < BANNED_REPUTATION
}
/// The type that tracks the reputation score.
pub type Reputation = i32;
/// Various kinds of reputation changes.
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum ReputationChangeKind {
/// Received an unspecific bad message from the peer
BadMessage,
/// Peer sent a bad block.
///
/// Note: this will we only used in pre-merge, pow consensus, since after no more block announcements are sent via devp2p: [EIP-3675](https://eips.ethereum.org/EIPS/eip-3675#devp2p)
BadBlock,
/// Peer sent a bad transaction message. E.g. Transactions which weren't recoverable.
BadTransactions,
/// Peer sent a bad announcement message, e.g. invalid transaction type for the configured
/// network.
BadAnnouncement,
/// Peer sent a message that included a hash or transaction that we already received from the
/// peer.
///
/// According to the [Eth spec](https://github.com/ethereum/devp2p/blob/master/caps/eth.md):
///
/// > A node should never send a transaction back to a peer that it can determine already knows
/// > of it (either because it was previously sent or because it was informed from this peer
/// > originally). This is usually achieved by remembering a set of transaction hashes recently
/// > relayed by the peer.
AlreadySeenTransaction,
/// Peer failed to respond in time.
Timeout,
/// Peer does not adhere to network protocol rules.
BadProtocol,
/// Failed to establish a connection to the peer.
FailedToConnect,
/// Connection dropped by peer.
Dropped,
/// Reset the reputation to the default value.
Reset,
/// Apply a reputation change by value
Other(Reputation),
}
impl ReputationChangeKind {
/// Returns true if the reputation change is a [`ReputationChangeKind::Reset`].
pub const fn is_reset(&self) -> bool {
matches!(self, Self::Reset)
}
/// Returns true if the reputation change is [`ReputationChangeKind::Dropped`].
pub const fn is_dropped(&self) -> bool {
matches!(self, Self::Dropped)
}
}
/// How the [`ReputationChangeKind`] are weighted.
#[derive(Debug, Clone, PartialEq, Eq)]
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]

View File

@ -3,7 +3,6 @@
use crate::{fetch::DownloadRequest, flattened_response::FlattenedResponse, peers::PeersHandle};
use futures::{future, future::Either};
use reth_network_api::ReputationChangeKind;
use reth_network_p2p::{
bodies::client::{BodiesClient, BodiesFut},
download::DownloadClient,
@ -12,6 +11,7 @@ use reth_network_p2p::{
priority::Priority,
};
use reth_network_peers::PeerId;
use reth_network_types::ReputationChangeKind;
use reth_primitives::{Header, B256};
use std::sync::{
atomic::{AtomicUsize, Ordering},

View File

@ -3,13 +3,13 @@
use crate::{message::BlockRequest, peers::PeersHandle};
use futures::StreamExt;
use reth_eth_wire::{GetBlockBodies, GetBlockHeaders};
use reth_network_api::ReputationChangeKind;
use reth_network_p2p::{
error::{EthResponseValidator, PeerRequestResult, RequestError, RequestResult},
headers::client::HeadersRequest,
priority::Priority,
};
use reth_network_peers::PeerId;
use reth_network_types::ReputationChangeKind;
use reth_primitives::{BlockBody, Header, B256};
use std::{
collections::{HashMap, VecDeque},

View File

@ -15,6 +15,38 @@
//! (IP+port) of our node is published via discovery, remote peers can initiate inbound connections
//! to the local node. Once a (tcp) connection is established, both peers start to authenticate a [RLPx session](https://github.com/ethereum/devp2p/blob/master/rlpx.md) via a handshake. If the handshake was successful, both peers announce their capabilities and are now ready to exchange sub-protocol messages via the `RLPx` session.
use std::{
net::SocketAddr,
path::Path,
pin::Pin,
sync::{
atomic::{AtomicU64, AtomicUsize, Ordering},
Arc,
},
task::{Context, Poll},
time::{Duration, Instant},
};
use futures::{Future, StreamExt};
use parking_lot::Mutex;
use reth_eth_wire::{
capability::{Capabilities, CapabilityMessage},
DisconnectReason, EthVersion, Status,
};
use reth_fs_util::{self as fs, FsPathError};
use reth_metrics::common::mpsc::UnboundedMeteredSender;
use reth_network_api::{EthProtocolInfo, NetworkStatus, PeerInfo};
use reth_network_peers::{NodeRecord, PeerId};
use reth_network_types::ReputationChangeKind;
use reth_primitives::ForkId;
use reth_storage_api::BlockNumReader;
use reth_tasks::shutdown::GracefulShutdown;
use reth_tokio_util::EventSender;
use secp256k1::SecretKey;
use tokio::sync::mpsc::{self, error::TrySendError};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::{debug, error, trace, warn};
use crate::{
budget::{DEFAULT_BUDGET_TRY_DRAIN_NETWORK_HANDLE_CHANNEL, DEFAULT_BUDGET_TRY_DRAIN_SWARM},
config::NetworkConfig,
@ -35,35 +67,6 @@ use crate::{
transactions::NetworkTransactionEvent,
FetchClient, NetworkBuilder,
};
use futures::{Future, StreamExt};
use parking_lot::Mutex;
use reth_eth_wire::{
capability::{Capabilities, CapabilityMessage},
DisconnectReason, EthVersion, Status,
};
use reth_fs_util::{self as fs, FsPathError};
use reth_metrics::common::mpsc::UnboundedMeteredSender;
use reth_network_api::{EthProtocolInfo, NetworkStatus, PeerInfo, ReputationChangeKind};
use reth_network_peers::{NodeRecord, PeerId};
use reth_primitives::ForkId;
use reth_storage_api::BlockNumReader;
use reth_tasks::shutdown::GracefulShutdown;
use reth_tokio_util::EventSender;
use secp256k1::SecretKey;
use std::{
net::SocketAddr,
path::Path,
pin::Pin,
sync::{
atomic::{AtomicU64, AtomicUsize, Ordering},
Arc,
},
task::{Context, Poll},
time::{Duration, Instant},
};
use tokio::sync::mpsc::{self, error::TrySendError};
use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::{debug, error, trace, warn};
#[cfg_attr(doc, aquamarine::aquamarine)]
/// Manages the _entire_ state of the network.

View File

@ -14,11 +14,11 @@ use parking_lot::Mutex;
use reth_discv4::Discv4;
use reth_eth_wire::{DisconnectReason, NewBlock, NewPooledTransactionHashes, SharedTransactions};
use reth_network_api::{
NetworkError, NetworkInfo, NetworkStatus, PeerInfo, PeerKind, Peers, PeersInfo, Reputation,
ReputationChangeKind,
NetworkError, NetworkInfo, NetworkStatus, PeerInfo, PeerKind, Peers, PeersInfo,
};
use reth_network_p2p::sync::{NetworkSyncUpdater, SyncState, SyncStateProvider};
use reth_network_peers::{NodeRecord, PeerId};
use reth_network_types::{Reputation, ReputationChangeKind};
use reth_primitives::{Head, TransactionSigned, B256};
use reth_tokio_util::{EventSender, EventStream};
use secp256k1::SecretKey;

View File

@ -8,7 +8,7 @@ use crate::{
use futures::StreamExt;
use reth_eth_wire::{errors::EthStreamError, DisconnectReason};
use reth_net_banlist::BanList;
use reth_network_api::{PeerKind, ReputationChangeKind};
use reth_network_api::PeerKind;
use reth_network_peers::{NodeRecord, PeerId};
use reth_network_types::{
peers::{
@ -17,7 +17,7 @@ use reth_network_types::{
is_banned_reputation, DEFAULT_REPUTATION, MAX_TRUSTED_PEER_REPUTATION_CHANGE,
},
},
ConnectionsConfig, PeersConfig, ReputationChangeWeights,
ConnectionsConfig, PeersConfig, ReputationChangeKind, ReputationChangeWeights,
};
use reth_primitives::ForkId;
use std::{
@ -1326,24 +1326,6 @@ impl Display for InboundConnectionError {
#[cfg(test)]
mod tests {
use super::PeersManager;
use crate::{
peers::{
ConnectionInfo, InboundConnectionError, PeerAction, PeerAddr, PeerBackoffDurations,
PeerConnectionState,
},
session::PendingSessionHandshakeError,
PeersConfig,
};
use reth_eth_wire::{
errors::{EthHandshakeError, EthStreamError, P2PHandshakeError, P2PStreamError},
DisconnectReason,
};
use reth_net_banlist::BanList;
use reth_network_api::{Direction, ReputationChangeKind};
use reth_network_peers::{PeerId, TrustedPeer};
use reth_network_types::{peers::reputation::DEFAULT_REPUTATION, BackoffKind};
use reth_primitives::B512;
use std::{
collections::HashSet,
future::{poll_fn, Future},
@ -1353,8 +1335,30 @@ mod tests {
task::{Context, Poll},
time::Duration,
};
use reth_eth_wire::{
errors::{EthHandshakeError, EthStreamError, P2PHandshakeError, P2PStreamError},
DisconnectReason,
};
use reth_net_banlist::BanList;
use reth_network_api::Direction;
use reth_network_peers::{PeerId, TrustedPeer};
use reth_network_types::{
peers::reputation::DEFAULT_REPUTATION, BackoffKind, ReputationChangeKind,
};
use reth_primitives::B512;
use url::Host;
use super::PeersManager;
use crate::{
peers::{
ConnectionInfo, InboundConnectionError, PeerAction, PeerAddr, PeerBackoffDurations,
PeerConnectionState,
},
session::PendingSessionHandshakeError,
PeersConfig,
};
struct PeerActionFuture<'a> {
peers: &'a mut PeersManager,
}

View File

@ -1,19 +1,5 @@
//! Transactions management for the p2p network.
use crate::{
budget::{
DEFAULT_BUDGET_TRY_DRAIN_NETWORK_TRANSACTION_EVENTS,
DEFAULT_BUDGET_TRY_DRAIN_PENDING_POOL_IMPORTS, DEFAULT_BUDGET_TRY_DRAIN_POOL_IMPORTS,
DEFAULT_BUDGET_TRY_DRAIN_STREAM,
},
cache::LruCache,
duration_metered_exec,
manager::NetworkEvent,
message::{PeerRequest, PeerRequestSender},
metered_poll_nested_stream_with_budget,
metrics::{TransactionsManagerMetrics, NETWORK_POOL_TRANSACTIONS_SCOPE},
NetworkEvents, NetworkHandle,
};
use futures::{stream::FuturesUnordered, Future, StreamExt};
use reth_eth_wire::{
EthVersion, GetPooledTransactions, HandleMempoolData, HandleVersionedMempoolData,
@ -21,12 +7,13 @@ use reth_eth_wire::{
PooledTransactions, RequestTxHashes, Transactions,
};
use reth_metrics::common::mpsc::UnboundedMeteredReceiver;
use reth_network_api::{Peers, ReputationChangeKind};
use reth_network_api::Peers;
use reth_network_p2p::{
error::{RequestError, RequestResult},
sync::SyncStateProvider,
};
use reth_network_peers::PeerId;
use reth_network_types::ReputationChangeKind;
use reth_primitives::{
FromRecoveredPooledTransaction, PooledTransactionsElement, TransactionSigned, TxHash, B256,
};
@ -50,6 +37,21 @@ use tokio::sync::{mpsc, oneshot, oneshot::error::RecvError};
use tokio_stream::wrappers::{ReceiverStream, UnboundedReceiverStream};
use tracing::{debug, trace};
use crate::{
budget::{
DEFAULT_BUDGET_TRY_DRAIN_NETWORK_TRANSACTION_EVENTS,
DEFAULT_BUDGET_TRY_DRAIN_PENDING_POOL_IMPORTS, DEFAULT_BUDGET_TRY_DRAIN_POOL_IMPORTS,
DEFAULT_BUDGET_TRY_DRAIN_STREAM,
},
cache::LruCache,
duration_metered_exec,
manager::NetworkEvent,
message::{PeerRequest, PeerRequestSender},
metered_poll_nested_stream_with_budget,
metrics::{TransactionsManagerMetrics, NETWORK_POOL_TRANSACTIONS_SCOPE},
NetworkEvents, NetworkHandle,
};
/// Aggregation on configurable parameters for [`TransactionsManager`].
pub mod config;
/// Default and spec'd bounds.

View File

@ -13,11 +13,11 @@ workspace = true
[dependencies]
reth-primitives.workspace = true
reth-network-api.workspace = true
reth-eth-wire-types.workspace = true
reth-consensus.workspace = true
reth-network-peers.workspace = true
reth-storage-errors.workspace = true
reth-network-types.workspace = true
# async
futures.workspace = true

View File

@ -1,7 +1,7 @@
use super::headers::client::HeadersRequest;
use reth_consensus::ConsensusError;
use reth_network_api::ReputationChangeKind;
use reth_network_peers::WithPeerId;
use reth_network_types::ReputationChangeKind;
use reth_primitives::{
BlockHashOrNumber, BlockNumber, GotExpected, GotExpectedBoxed, Header, B256,
};