mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +00:00
feat: Alloy migration (#4737)
Co-authored-by: Alessandro Mazza <121622391+alessandromazza98@users.noreply.github.com> Co-authored-by: Supernovahs.eth <91280922+supernovahs@users.noreply.github.com> Co-authored-by: Oliver Nordbjerg <hi@notbjerg.me>
This commit is contained in:
@ -26,13 +26,13 @@ reth-discv4 = { path = "../discv4" }
|
||||
reth-dns-discovery = { path = "../dns" }
|
||||
reth-eth-wire = { path = "../eth-wire" }
|
||||
reth-ecies = { path = "../ecies" }
|
||||
reth-rlp.workspace = true
|
||||
reth-rlp-derive = { path = "../../rlp/rlp-derive" }
|
||||
reth-tasks.workspace = true
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-provider.workspace = true
|
||||
reth-rpc-types.workspace = true
|
||||
|
||||
alloy-rlp.workspace = true
|
||||
|
||||
# async/futures
|
||||
futures.workspace = true
|
||||
pin-project.workspace = true
|
||||
@ -63,7 +63,6 @@ rand.workspace = true
|
||||
secp256k1 = { workspace = true, features = ["global-context", "rand-std", "recovery"] }
|
||||
|
||||
enr = { workspace = true, features = ["rust-secp256k1"], optional = true }
|
||||
ethers-core = { workspace = true, default-features = false, optional = true }
|
||||
tempfile = { version = "3.3", optional = true }
|
||||
|
||||
[dev-dependencies]
|
||||
@ -88,12 +87,11 @@ ethers-middleware = { workspace = true, default-features = false }
|
||||
enr = { workspace = true, features = ["serde", "rust-secp256k1"] }
|
||||
|
||||
# misc
|
||||
hex = "0.4"
|
||||
serial_test.workspace = true
|
||||
tempfile = "3.3"
|
||||
|
||||
[features]
|
||||
default = ["serde"]
|
||||
serde = ["dep:serde", "dep:humantime-serde", "secp256k1/serde", "enr?/serde", "dep:serde_json"]
|
||||
test-utils = ["reth-provider/test-utils", "dep:enr", "dep:ethers-core", "dep:tempfile"]
|
||||
test-utils = ["reth-provider/test-utils", "dep:enr", "dep:tempfile"]
|
||||
geth-tests = []
|
||||
|
||||
@ -11,15 +11,12 @@ use reth_interfaces::p2p::{
|
||||
priority::Priority,
|
||||
};
|
||||
use reth_network_api::ReputationChangeKind;
|
||||
use reth_primitives::{Header, PeerId, H256};
|
||||
use reth_primitives::{Header, PeerId, B256};
|
||||
use std::sync::{
|
||||
atomic::{AtomicUsize, Ordering},
|
||||
Arc,
|
||||
};
|
||||
use tokio::sync::{
|
||||
mpsc::UnboundedSender,
|
||||
oneshot::{self},
|
||||
};
|
||||
use tokio::sync::{mpsc::UnboundedSender, oneshot};
|
||||
|
||||
/// Front-end API for fetching data from the network.
|
||||
///
|
||||
@ -111,7 +108,7 @@ impl BodiesClient for FetchClient {
|
||||
/// Sends a `GetBlockBodies` request to an available peer.
|
||||
fn get_block_bodies_with_priority(
|
||||
&self,
|
||||
request: Vec<H256>,
|
||||
request: Vec<B256>,
|
||||
priority: Priority,
|
||||
) -> Self::Output {
|
||||
let (response, rx) = oneshot::channel();
|
||||
|
||||
@ -9,7 +9,7 @@ use reth_interfaces::p2p::{
|
||||
priority::Priority,
|
||||
};
|
||||
use reth_network_api::ReputationChangeKind;
|
||||
use reth_primitives::{BlockBody, Header, PeerId, H256};
|
||||
use reth_primitives::{BlockBody, Header, PeerId, B256};
|
||||
use std::{
|
||||
collections::{HashMap, VecDeque},
|
||||
sync::{
|
||||
@ -37,7 +37,7 @@ pub struct StateFetcher {
|
||||
HashMap<PeerId, Request<HeadersRequest, PeerRequestResult<Vec<Header>>>>,
|
||||
/// Currently active [`GetBlockBodies`] requests
|
||||
inflight_bodies_requests:
|
||||
HashMap<PeerId, Request<Vec<H256>, PeerRequestResult<Vec<BlockBody>>>>,
|
||||
HashMap<PeerId, Request<Vec<B256>, PeerRequestResult<Vec<BlockBody>>>>,
|
||||
/// The list of _available_ peers for requests.
|
||||
peers: HashMap<PeerId, Peer>,
|
||||
/// The handle to the peers manager
|
||||
@ -73,7 +73,7 @@ impl StateFetcher {
|
||||
pub(crate) fn new_active_peer(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
best_hash: H256,
|
||||
best_hash: B256,
|
||||
best_number: u64,
|
||||
timeout: Arc<AtomicU64>,
|
||||
) {
|
||||
@ -100,7 +100,7 @@ impl StateFetcher {
|
||||
/// Updates the block information for the peer.
|
||||
///
|
||||
/// Returns `true` if this a newer block
|
||||
pub(crate) fn update_peer_block(&mut self, peer_id: &PeerId, hash: H256, number: u64) -> bool {
|
||||
pub(crate) fn update_peer_block(&mut self, peer_id: &PeerId, hash: B256, number: u64) -> bool {
|
||||
if let Some(peer) = self.peers.get_mut(peer_id) {
|
||||
if number > peer.best_number {
|
||||
peer.best_hash = hash;
|
||||
@ -302,7 +302,7 @@ struct Peer {
|
||||
/// The state this peer currently resides in.
|
||||
state: PeerState,
|
||||
/// Best known hash that the peer has
|
||||
best_hash: H256,
|
||||
best_hash: B256,
|
||||
/// Tracks the best number of the peer.
|
||||
best_number: u64,
|
||||
/// Tracks the current timeout value we use for the peer.
|
||||
@ -372,7 +372,7 @@ pub(crate) enum DownloadRequest {
|
||||
},
|
||||
/// Download the requested headers and send response through channel
|
||||
GetBlockBodies {
|
||||
request: Vec<H256>,
|
||||
request: Vec<B256>,
|
||||
response: oneshot::Sender<PeerRequestResult<Vec<BlockBody>>>,
|
||||
priority: Priority,
|
||||
},
|
||||
@ -429,7 +429,7 @@ pub(crate) enum BlockResponseOutcome {
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::{peers::PeersManager, PeersConfig};
|
||||
use reth_primitives::{SealedHeader, H256, H512};
|
||||
use reth_primitives::{SealedHeader, B256, B512};
|
||||
use std::future::poll_fn;
|
||||
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
@ -457,10 +457,10 @@ mod tests {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
// Add a few random peers
|
||||
let peer1 = H512::random();
|
||||
let peer2 = H512::random();
|
||||
fetcher.new_active_peer(peer1, H256::random(), 1, Arc::new(AtomicU64::new(1)));
|
||||
fetcher.new_active_peer(peer2, H256::random(), 2, Arc::new(AtomicU64::new(1)));
|
||||
let peer1 = B512::random();
|
||||
let peer2 = B512::random();
|
||||
fetcher.new_active_peer(peer1, B256::random(), 1, Arc::new(AtomicU64::new(1)));
|
||||
fetcher.new_active_peer(peer2, B256::random(), 2, Arc::new(AtomicU64::new(1)));
|
||||
|
||||
let first_peer = fetcher.next_peer().unwrap();
|
||||
assert!(first_peer == peer1 || first_peer == peer2);
|
||||
@ -480,15 +480,15 @@ mod tests {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
// Add a few random peers
|
||||
let peer1 = H512::random();
|
||||
let peer2 = H512::random();
|
||||
let peer3 = H512::random();
|
||||
let peer1 = B512::random();
|
||||
let peer2 = B512::random();
|
||||
let peer3 = B512::random();
|
||||
|
||||
let peer2_timeout = Arc::new(AtomicU64::new(300));
|
||||
|
||||
fetcher.new_active_peer(peer1, H256::random(), 1, Arc::new(AtomicU64::new(30)));
|
||||
fetcher.new_active_peer(peer2, H256::random(), 2, Arc::clone(&peer2_timeout));
|
||||
fetcher.new_active_peer(peer3, H256::random(), 3, Arc::new(AtomicU64::new(50)));
|
||||
fetcher.new_active_peer(peer1, B256::random(), 1, Arc::new(AtomicU64::new(30)));
|
||||
fetcher.new_active_peer(peer2, B256::random(), 2, Arc::clone(&peer2_timeout));
|
||||
fetcher.new_active_peer(peer3, B256::random(), 3, Arc::new(AtomicU64::new(50)));
|
||||
|
||||
// Must always get peer1 (lowest timeout)
|
||||
assert_eq!(fetcher.next_peer(), Some(peer1));
|
||||
@ -504,7 +504,7 @@ mod tests {
|
||||
async fn test_on_block_headers_response() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let peer_id = H512::random();
|
||||
let peer_id = B512::random();
|
||||
|
||||
assert_eq!(fetcher.on_block_headers_response(peer_id, Ok(vec![Header::default()])), None);
|
||||
|
||||
@ -534,7 +534,7 @@ mod tests {
|
||||
async fn test_header_response_outcome() {
|
||||
let manager = PeersManager::new(PeersConfig::default());
|
||||
let mut fetcher = StateFetcher::new(manager.handle(), Default::default());
|
||||
let peer_id = H512::random();
|
||||
let peer_id = B512::random();
|
||||
|
||||
let request_pair = || {
|
||||
let (tx, _rx) = oneshot::channel();
|
||||
|
||||
@ -41,7 +41,7 @@ use reth_eth_wire::{
|
||||
use reth_metrics::common::mpsc::UnboundedMeteredSender;
|
||||
use reth_net_common::bandwidth_meter::BandwidthMeter;
|
||||
use reth_network_api::ReputationChangeKind;
|
||||
use reth_primitives::{listener::EventListeners, ForkId, NodeRecord, PeerId, H256};
|
||||
use reth_primitives::{listener::EventListeners, ForkId, NodeRecord, PeerId, B256};
|
||||
use reth_provider::{BlockNumReader, BlockReader};
|
||||
use reth_rpc_types::{EthProtocolInfo, NetworkStatus};
|
||||
use std::{
|
||||
@ -301,7 +301,7 @@ where
|
||||
}
|
||||
|
||||
/// Returns the configured genesis hash
|
||||
pub fn genesis_hash(&self) -> H256 {
|
||||
pub fn genesis_hash(&self) -> B256 {
|
||||
self.swarm.state().genesis_hash()
|
||||
}
|
||||
|
||||
|
||||
@ -12,7 +12,7 @@ use reth_eth_wire::{
|
||||
};
|
||||
use reth_interfaces::p2p::error::{RequestError, RequestResult};
|
||||
use reth_primitives::{
|
||||
BlockBody, Bytes, Header, PeerId, PooledTransactionsElement, ReceiptWithBloom, H256,
|
||||
BlockBody, Bytes, Header, PeerId, PooledTransactionsElement, ReceiptWithBloom, B256,
|
||||
};
|
||||
use std::{
|
||||
fmt,
|
||||
@ -25,7 +25,7 @@ use tokio::sync::{mpsc, mpsc::error::TrySendError, oneshot};
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct NewBlockMessage {
|
||||
/// Hash of the block
|
||||
pub hash: H256,
|
||||
pub hash: B256,
|
||||
/// Raw received message
|
||||
pub block: Arc<NewBlock>,
|
||||
}
|
||||
|
||||
@ -11,7 +11,7 @@ use reth_network_api::{
|
||||
NetworkError, NetworkInfo, PeerInfo, PeerKind, Peers, PeersInfo, Reputation,
|
||||
ReputationChangeKind,
|
||||
};
|
||||
use reth_primitives::{Head, NodeRecord, PeerId, TransactionSigned, H256};
|
||||
use reth_primitives::{Head, NodeRecord, PeerId, TransactionSigned, B256};
|
||||
use reth_rpc_types::NetworkStatus;
|
||||
use std::{
|
||||
net::SocketAddr,
|
||||
@ -135,7 +135,7 @@ impl NetworkHandle {
|
||||
/// Caution: in PoS this is a noop, since new block are no longer announced over devp2p, but are
|
||||
/// instead sent to node node by the CL. However, they can still be requested over devp2p, but
|
||||
/// broadcasting them is a considered a protocol violation..
|
||||
pub fn announce_block(&self, block: NewBlock, hash: H256) {
|
||||
pub fn announce_block(&self, block: NewBlock, hash: B256) {
|
||||
self.send_message(NetworkHandleMessage::AnnounceBlock(block, hash))
|
||||
}
|
||||
|
||||
@ -332,7 +332,7 @@ pub(crate) enum NetworkHandleMessage {
|
||||
/// Add a new listener for [`NetworkEvent`].
|
||||
EventListener(UnboundedSender<NetworkEvent>),
|
||||
/// Broadcast event to announce a new block to all nodes.
|
||||
AnnounceBlock(NewBlock, H256),
|
||||
AnnounceBlock(NewBlock, B256),
|
||||
/// Sends the list of transactions to the given peer.
|
||||
SendTransaction { peer_id: PeerId, msg: SharedTransactions },
|
||||
/// Sends the list of transactions hashes to the given peer.
|
||||
|
||||
@ -1349,7 +1349,7 @@ mod test {
|
||||
};
|
||||
use reth_net_common::ban_list::BanList;
|
||||
use reth_network_api::ReputationChangeKind;
|
||||
use reth_primitives::{PeerId, H512};
|
||||
use reth_primitives::{PeerId, B512};
|
||||
use std::{
|
||||
collections::HashSet,
|
||||
future::{poll_fn, Future},
|
||||
@ -2002,7 +2002,7 @@ mod test {
|
||||
let ban_list = BanList::new(HashSet::new(), vec![ip]);
|
||||
let config = PeersConfig::default().with_ban_list(ban_list);
|
||||
let mut peer_manager = PeersManager::new(config);
|
||||
peer_manager.add_peer(H512::default(), socket_addr, None);
|
||||
peer_manager.add_peer(B512::default(), socket_addr, None);
|
||||
|
||||
assert!(peer_manager.peers.is_empty());
|
||||
}
|
||||
@ -2031,7 +2031,7 @@ mod test {
|
||||
async fn test_on_active_inbound_ban_list() {
|
||||
let ip = IpAddr::V4(Ipv4Addr::new(127, 0, 1, 2));
|
||||
let socket_addr = SocketAddr::new(ip, 8008);
|
||||
let given_peer_id: PeerId = H512::from_low_u64_ne(123403423412);
|
||||
let given_peer_id = PeerId::random();
|
||||
let ban_list = BanList::new(vec![given_peer_id], HashSet::new());
|
||||
let config = PeersConfig::default().with_ban_list(ban_list);
|
||||
let mut peer_manager = PeersManager::new(config);
|
||||
|
||||
@ -16,7 +16,7 @@ use reth_eth_wire::{
|
||||
capability::Capabilities, BlockHashNumber, DisconnectReason, NewBlockHashes, Status,
|
||||
};
|
||||
use reth_network_api::PeerKind;
|
||||
use reth_primitives::{ForkId, PeerId, H256};
|
||||
use reth_primitives::{ForkId, PeerId, B256};
|
||||
use reth_provider::BlockNumReader;
|
||||
use std::{
|
||||
collections::{HashMap, VecDeque},
|
||||
@ -60,7 +60,7 @@ pub struct NetworkState<C> {
|
||||
/// Network discovery.
|
||||
discovery: Discovery,
|
||||
/// The genesis hash of the network we're on
|
||||
genesis_hash: H256,
|
||||
genesis_hash: B256,
|
||||
/// The type that handles requests.
|
||||
///
|
||||
/// The fetcher streams RLPx related requests on a per-peer basis to this type. This type will
|
||||
@ -77,7 +77,7 @@ where
|
||||
client: C,
|
||||
discovery: Discovery,
|
||||
peers_manager: PeersManager,
|
||||
genesis_hash: H256,
|
||||
genesis_hash: B256,
|
||||
num_active_peers: Arc<AtomicUsize>,
|
||||
) -> Self {
|
||||
let state_fetcher = StateFetcher::new(peers_manager.handle(), num_active_peers);
|
||||
@ -113,7 +113,7 @@ where
|
||||
}
|
||||
|
||||
/// Configured genesis hash.
|
||||
pub fn genesis_hash(&self) -> H256 {
|
||||
pub fn genesis_hash(&self) -> B256 {
|
||||
self.genesis_hash
|
||||
}
|
||||
|
||||
@ -227,7 +227,7 @@ where
|
||||
}
|
||||
|
||||
/// Updates the block information for the peer.
|
||||
pub(crate) fn update_peer_block(&mut self, peer_id: &PeerId, hash: H256, number: u64) {
|
||||
pub(crate) fn update_peer_block(&mut self, peer_id: &PeerId, hash: B256, number: u64) {
|
||||
if let Some(peer) = self.active_peers.get_mut(peer_id) {
|
||||
peer.best_hash = hash;
|
||||
}
|
||||
@ -242,7 +242,7 @@ where
|
||||
/// Invoked after a `NewBlock` message was received by the peer.
|
||||
///
|
||||
/// This will keep track of blocks we know a peer has
|
||||
pub(crate) fn on_new_block(&mut self, peer_id: PeerId, hash: H256) {
|
||||
pub(crate) fn on_new_block(&mut self, peer_id: PeerId, hash: B256) {
|
||||
// Mark the blocks as seen
|
||||
if let Some(peer) = self.active_peers.get_mut(&peer_id) {
|
||||
peer.blocks.insert(hash);
|
||||
@ -469,7 +469,7 @@ where
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct ActivePeer {
|
||||
/// Best block of the peer.
|
||||
pub(crate) best_hash: H256,
|
||||
pub(crate) best_hash: B256,
|
||||
/// The capabilities of the remote peer.
|
||||
#[allow(unused)]
|
||||
pub(crate) capabilities: Arc<Capabilities>,
|
||||
@ -478,7 +478,7 @@ pub(crate) struct ActivePeer {
|
||||
/// The response receiver for a currently active request to that peer.
|
||||
pub(crate) pending_response: Option<PeerResponse>,
|
||||
/// Blocks we know the peer has.
|
||||
pub(crate) blocks: LruCache<H256>,
|
||||
pub(crate) blocks: LruCache<B256>,
|
||||
}
|
||||
|
||||
/// Message variants triggered by the [`NetworkState`]
|
||||
@ -530,7 +530,7 @@ mod tests {
|
||||
BlockBodies, EthVersion, Status,
|
||||
};
|
||||
use reth_interfaces::p2p::{bodies::client::BodiesClient, error::RequestError};
|
||||
use reth_primitives::{BlockBody, Header, PeerId, H256};
|
||||
use reth_primitives::{BlockBody, Header, PeerId, B256};
|
||||
use reth_provider::test_utils::NoopProvider;
|
||||
use std::{
|
||||
future::poll_fn,
|
||||
@ -606,11 +606,11 @@ mod tests {
|
||||
});
|
||||
|
||||
// send requests to the state via the client
|
||||
let (peer, bodies) = client.get_block_bodies(vec![H256::random()]).await.unwrap().split();
|
||||
let (peer, bodies) = client.get_block_bodies(vec![B256::random()]).await.unwrap().split();
|
||||
assert_eq!(peer, peer_id);
|
||||
assert_eq!(bodies, vec![body]);
|
||||
|
||||
let resp = client.get_block_bodies(vec![H256::random()]).await;
|
||||
let resp = client.get_block_bodies(vec![B256::random()]).await;
|
||||
assert!(resp.is_err());
|
||||
assert_eq!(resp.unwrap_err(), RequestError::ConnectionDropped);
|
||||
}
|
||||
|
||||
@ -20,7 +20,7 @@ use reth_metrics::common::mpsc::UnboundedMeteredReceiver;
|
||||
use reth_network_api::{Peers, ReputationChangeKind};
|
||||
use reth_primitives::{
|
||||
FromRecoveredPooledTransaction, IntoRecoveredTransaction, PeerId, PooledTransactionsElement,
|
||||
TransactionSigned, TxHash, H256,
|
||||
TransactionSigned, TxHash, B256,
|
||||
};
|
||||
use reth_transaction_pool::{
|
||||
error::PoolResult, GetPooledTransactionLimit, PoolTransaction, PropagateKind,
|
||||
@ -911,7 +911,7 @@ impl Future for GetPooledTxRequestFut {
|
||||
#[derive(Debug)]
|
||||
struct Peer {
|
||||
/// Keeps track of transactions that we know the peer has seen.
|
||||
transactions: LruCache<H256>,
|
||||
transactions: LruCache<B256>,
|
||||
/// A communication channel directly to the peer's session task.
|
||||
request_tx: PeerRequestSender,
|
||||
/// negotiated version of the session.
|
||||
@ -925,9 +925,9 @@ struct Peer {
|
||||
#[derive(Debug)]
|
||||
enum TransactionsCommand {
|
||||
/// Propagate a transaction hash to the network.
|
||||
PropagateHash(H256),
|
||||
PropagateHash(B256),
|
||||
/// Propagate transaction hashes to a specific peer.
|
||||
PropagateHashesTo(Vec<H256>, PeerId),
|
||||
PropagateHashesTo(Vec<B256>, PeerId),
|
||||
/// Request the list of active peer IDs from the [`TransactionsManager`].
|
||||
GetActivePeers,
|
||||
/// Propagate a collection of full transactions to a specific peer.
|
||||
@ -960,10 +960,11 @@ pub enum NetworkTransactionEvent {
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::{test_utils::Testnet, NetworkConfigBuilder, NetworkManager};
|
||||
use alloy_rlp::Decodable;
|
||||
use reth_interfaces::sync::{NetworkSyncUpdater, SyncState};
|
||||
use reth_network_api::NetworkInfo;
|
||||
use reth_primitives::hex;
|
||||
use reth_provider::test_utils::NoopProvider;
|
||||
use reth_rlp::Decodable;
|
||||
use reth_transaction_pool::test_utils::{testing_pool, MockTransaction};
|
||||
use secp256k1::SecretKey;
|
||||
use std::future::poll_fn;
|
||||
@ -1036,7 +1037,7 @@ mod tests {
|
||||
}
|
||||
}
|
||||
// random tx: <https://etherscan.io/getRawTx?tx=0x9448608d36e721ef403c53b00546068a6474d6cbab6816c3926de449898e7bce>
|
||||
let input = hex::decode("02f871018302a90f808504890aef60826b6c94ddf4c5025d1a5742cf12f74eec246d4432c295e487e09c3bbcc12b2b80c080a0f21a4eacd0bf8fea9c5105c543be5a1d8c796516875710fafafdf16d16d8ee23a001280915021bb446d1973501a67f93d2b38894a514b976e7b46dc2fe54598d76").unwrap();
|
||||
let input = hex!("02f871018302a90f808504890aef60826b6c94ddf4c5025d1a5742cf12f74eec246d4432c295e487e09c3bbcc12b2b80c080a0f21a4eacd0bf8fea9c5105c543be5a1d8c796516875710fafafdf16d16d8ee23a001280915021bb446d1973501a67f93d2b38894a514b976e7b46dc2fe54598d76");
|
||||
let signed_tx = TransactionSigned::decode(&mut &input[..]).unwrap();
|
||||
transactions.on_network_tx_event(NetworkTransactionEvent::IncomingTransactions {
|
||||
peer_id: *handle1.peer_id(),
|
||||
@ -1122,7 +1123,7 @@ mod tests {
|
||||
}
|
||||
}
|
||||
// random tx: <https://etherscan.io/getRawTx?tx=0x9448608d36e721ef403c53b00546068a6474d6cbab6816c3926de449898e7bce>
|
||||
let input = hex::decode("02f871018302a90f808504890aef60826b6c94ddf4c5025d1a5742cf12f74eec246d4432c295e487e09c3bbcc12b2b80c080a0f21a4eacd0bf8fea9c5105c543be5a1d8c796516875710fafafdf16d16d8ee23a001280915021bb446d1973501a67f93d2b38894a514b976e7b46dc2fe54598d76").unwrap();
|
||||
let input = hex!("02f871018302a90f808504890aef60826b6c94ddf4c5025d1a5742cf12f74eec246d4432c295e487e09c3bbcc12b2b80c080a0f21a4eacd0bf8fea9c5105c543be5a1d8c796516875710fafafdf16d16d8ee23a001280915021bb446d1973501a67f93d2b38894a514b976e7b46dc2fe54598d76");
|
||||
let signed_tx = TransactionSigned::decode(&mut &input[..]).unwrap();
|
||||
transactions.on_network_tx_event(NetworkTransactionEvent::IncomingTransactions {
|
||||
peer_id: *handle1.peer_id(),
|
||||
@ -1205,7 +1206,7 @@ mod tests {
|
||||
}
|
||||
}
|
||||
// random tx: <https://etherscan.io/getRawTx?tx=0x9448608d36e721ef403c53b00546068a6474d6cbab6816c3926de449898e7bce>
|
||||
let input = hex::decode("02f871018302a90f808504890aef60826b6c94ddf4c5025d1a5742cf12f74eec246d4432c295e487e09c3bbcc12b2b80c080a0f21a4eacd0bf8fea9c5105c543be5a1d8c796516875710fafafdf16d16d8ee23a001280915021bb446d1973501a67f93d2b38894a514b976e7b46dc2fe54598d76").unwrap();
|
||||
let input = hex!("02f871018302a90f808504890aef60826b6c94ddf4c5025d1a5742cf12f74eec246d4432c295e487e09c3bbcc12b2b80c080a0f21a4eacd0bf8fea9c5105c543be5a1d8c796516875710fafafdf16d16d8ee23a001280915021bb446d1973501a67f93d2b38894a514b976e7b46dc2fe54598d76");
|
||||
let signed_tx = TransactionSigned::decode(&mut &input[..]).unwrap();
|
||||
transactions.on_network_tx_event(NetworkTransactionEvent::IncomingTransactions {
|
||||
peer_id: *handle1.peer_id(),
|
||||
|
||||
@ -10,7 +10,7 @@ use ethers_middleware::SignerMiddleware;
|
||||
use ethers_providers::Middleware;
|
||||
use ethers_signers::Signer;
|
||||
use reth_network::test_utils::enr_to_peer_id;
|
||||
use reth_primitives::PeerId;
|
||||
use reth_primitives::{hex, PeerId};
|
||||
use thiserror::Error;
|
||||
use tracing::trace;
|
||||
|
||||
|
||||
@ -1,6 +1,6 @@
|
||||
use crate::clique::{CliqueGethInstance, CliqueMiddleware};
|
||||
use ethers_core::{
|
||||
types::{transaction::eip2718::TypedTransaction, Eip1559TransactionRequest, H160, U64},
|
||||
types::{transaction::eip2718::TypedTransaction, Address, Eip1559TransactionRequest},
|
||||
utils::Geth,
|
||||
};
|
||||
use ethers_providers::Middleware;
|
||||
@ -113,7 +113,7 @@ async fn init_geth() -> (CliqueGethInstance, Arc<ChainSpec>) {
|
||||
let txs = nonces.map(|nonce| {
|
||||
// create a tx that just sends to the zero addr
|
||||
TypedTransaction::Eip1559(
|
||||
Eip1559TransactionRequest::new().to(H160::zero()).value(1u64).nonce(nonce),
|
||||
Eip1559TransactionRequest::new().to(Address::zero()).value(1u64).nonce(nonce),
|
||||
)
|
||||
});
|
||||
tracing::info!("generated transactions for blocks");
|
||||
@ -122,7 +122,7 @@ async fn init_geth() -> (CliqueGethInstance, Arc<ChainSpec>) {
|
||||
clique.provider.send_requests(txs).await.unwrap();
|
||||
|
||||
let block = clique.provider.get_block_number().await.unwrap();
|
||||
assert!(block > U64::zero());
|
||||
assert!(block.as_u64() > 0);
|
||||
|
||||
(clique, Arc::new(chainspec))
|
||||
}
|
||||
|
||||
@ -9,7 +9,7 @@ use reth_network::test_utils::{NetworkEventStream, Testnet};
|
||||
use reth_network_api::{NetworkInfo, Peers};
|
||||
use reth_primitives::{
|
||||
Block, BlockBody, Bytes, Header, HeadersDirection, Signature, Transaction, TransactionKind,
|
||||
TransactionSigned, TxEip2930, H256, U256,
|
||||
TransactionSigned, TxEip2930, U256,
|
||||
};
|
||||
use reth_provider::test_utils::MockEthProvider;
|
||||
use std::sync::Arc;
|
||||
@ -58,7 +58,7 @@ async fn test_get_body() {
|
||||
// request some blocks
|
||||
for _ in 0..100 {
|
||||
// Set a new random block to the mock storage and request it via the network
|
||||
let block_hash = H256::random();
|
||||
let block_hash = rng.gen();
|
||||
let mut block = Block::default();
|
||||
block.body.push(rng_transaction(&mut rng));
|
||||
|
||||
@ -100,12 +100,12 @@ async fn test_get_header() {
|
||||
assert_eq!(connected, *handle1.peer_id());
|
||||
|
||||
let start: u64 = rng.gen();
|
||||
let mut hash = H256::random();
|
||||
let mut hash = rng.gen();
|
||||
// request some headers
|
||||
for idx in 0..100 {
|
||||
// Set a new random header to the mock storage and request it via the network
|
||||
let header = Header { number: start + idx, parent_hash: hash, ..Default::default() };
|
||||
hash = H256::random();
|
||||
hash = rng.gen();
|
||||
|
||||
mock_provider.add_header(hash, header.clone());
|
||||
|
||||
|
||||
Reference in New Issue
Block a user