From 7250ca5d77eadafe70569ac6cb891e2682836b5b Mon Sep 17 00:00:00 2001 From: Dan Cline <6798349+Rjected@users.noreply.github.com> Date: Thu, 19 Dec 2024 12:39:11 +0200 Subject: [PATCH 01/30] chore: add docs for removing from state in system calls (#13447) --- crates/evm/src/system_calls/eip2935.rs | 5 +++++ crates/evm/src/system_calls/eip4788.rs | 5 +++++ crates/evm/src/system_calls/eip7002.rs | 6 +++++- crates/evm/src/system_calls/eip7251.rs | 6 +++++- 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/crates/evm/src/system_calls/eip2935.rs b/crates/evm/src/system_calls/eip2935.rs index 0cc2b83a3ca5..05637d4e1d6c 100644 --- a/crates/evm/src/system_calls/eip2935.rs +++ b/crates/evm/src/system_calls/eip2935.rs @@ -65,6 +65,11 @@ where } }; + // NOTE: Revm currently marks these accounts as "touched" when we do the above transact calls, + // and includes them in the result. + // + // There should be no state changes to these addresses anyways as a result of this system call, + // so we can just remove them from the state returned. res.state.remove(&alloy_eips::eip4788::SYSTEM_ADDRESS); res.state.remove(&evm.block().coinbase); diff --git a/crates/evm/src/system_calls/eip4788.rs b/crates/evm/src/system_calls/eip4788.rs index bfd5797214e5..4c78ff059c19 100644 --- a/crates/evm/src/system_calls/eip4788.rs +++ b/crates/evm/src/system_calls/eip4788.rs @@ -75,6 +75,11 @@ where } }; + // NOTE: Revm currently marks these accounts as "touched" when we do the above transact calls, + // and includes them in the result. + // + // There should be no state changes to these addresses anyways as a result of this system call, + // so we can just remove them from the state returned. res.state.remove(&alloy_eips::eip4788::SYSTEM_ADDRESS); res.state.remove(&evm.block().coinbase); diff --git a/crates/evm/src/system_calls/eip7002.rs b/crates/evm/src/system_calls/eip7002.rs index d3c6d84903ed..d949dd5a54a0 100644 --- a/crates/evm/src/system_calls/eip7002.rs +++ b/crates/evm/src/system_calls/eip7002.rs @@ -51,7 +51,11 @@ where } }; - // cleanup the state + // NOTE: Revm currently marks these accounts as "touched" when we do the above transact calls, + // and includes them in the result. + // + // There should be no state changes to these addresses anyways as a result of this system call, + // so we can just remove them from the state returned. res.state.remove(&alloy_eips::eip7002::SYSTEM_ADDRESS); res.state.remove(&evm.block().coinbase); diff --git a/crates/evm/src/system_calls/eip7251.rs b/crates/evm/src/system_calls/eip7251.rs index 28ae0160cdf6..86a1bee53e61 100644 --- a/crates/evm/src/system_calls/eip7251.rs +++ b/crates/evm/src/system_calls/eip7251.rs @@ -53,7 +53,11 @@ where } }; - // cleanup the state + // NOTE: Revm currently marks these accounts as "touched" when we do the above transact calls, + // and includes them in the result. + // + // There should be no state changes to these addresses anyways as a result of this system call, + // so we can just remove them from the state returned. res.state.remove(&alloy_eips::eip7002::SYSTEM_ADDRESS); res.state.remove(&evm.block().coinbase); From b26e9e0d1ca1eb40bba7b76192b4b5f5030e674b Mon Sep 17 00:00:00 2001 From: Dan Cline <6798349+Rjected@users.noreply.github.com> Date: Thu, 19 Dec 2024 12:46:40 +0200 Subject: [PATCH 02/30] chore: add helper trait for BlockReader tied to NetworkPrimitives (#13449) --- crates/node/builder/src/builder/mod.rs | 10 ++++----- crates/node/builder/src/node.rs | 29 ++++++++++++++++++++++++-- 2 files changed, 31 insertions(+), 8 deletions(-) diff --git a/crates/node/builder/src/builder/mod.rs b/crates/node/builder/src/builder/mod.rs index fe36d4b7ec55..8ffe357fd521 100644 --- a/crates/node/builder/src/builder/mod.rs +++ b/crates/node/builder/src/builder/mod.rs @@ -7,7 +7,7 @@ use crate::{ components::NodeComponentsBuilder, node::FullNode, rpc::{RethRpcAddOns, RethRpcServerHandles, RpcContext}, - DefaultNodeLauncher, LaunchNode, Node, NodeHandle, + BlockReaderFor, DefaultNodeLauncher, LaunchNode, Node, NodeHandle, }; use alloy_eips::eip4844::env_settings::EnvKzgSettings; use futures::Future; @@ -35,7 +35,7 @@ use reth_node_core::{ }; use reth_provider::{ providers::{BlockchainProvider, NodeTypesForProvider}, - BlockReader, ChainSpecProvider, FullProvider, + ChainSpecProvider, FullProvider, }; use reth_tasks::TaskExecutor; use reth_transaction_pool::{PoolConfig, PoolTransaction, TransactionPool}; @@ -658,8 +658,7 @@ impl BuilderContext { >, > + Unpin + 'static, - Node::Provider: - BlockReader, + Node::Provider: BlockReaderFor, { self.start_network_with(builder, pool, Default::default()) } @@ -685,8 +684,7 @@ impl BuilderContext { >, > + Unpin + 'static, - Node::Provider: - BlockReader, + Node::Provider: BlockReaderFor, { let (handle, network, txpool, eth) = builder .transactions(pool, tx_config) diff --git a/crates/node/builder/src/node.rs b/crates/node/builder/src/node.rs index ce7d12fee3d3..93deb47a0110 100644 --- a/crates/node/builder/src/node.rs +++ b/crates/node/builder/src/node.rs @@ -7,13 +7,14 @@ use std::{ sync::Arc, }; -use reth_node_api::{EngineTypes, FullNodeComponents}; +use reth_network::NetworkPrimitives; +use reth_node_api::{BlockBody, EngineTypes, FullNodeComponents}; use reth_node_core::{ dirs::{ChainPath, DataDirPath}, node_config::NodeConfig, }; use reth_payload_builder::PayloadBuilderHandle; -use reth_provider::ChainSpecProvider; +use reth_provider::{BlockReader, ChainSpecProvider}; use reth_rpc_api::EngineApiClient; use reth_rpc_builder::{auth::AuthServerHandle, RpcServerHandle}; use reth_tasks::TaskExecutor; @@ -210,3 +211,27 @@ impl> DerefMut for FullNode: + BlockReader< + Block = N::Block, + Header = N::BlockHeader, + Transaction = ::Transaction, + Receipt = N::Receipt, +> +{ +} + +impl BlockReaderFor for T +where + N: NetworkPrimitives, + T: BlockReader< + Block = N::Block, + Header = N::BlockHeader, + Transaction = ::Transaction, + Receipt = N::Receipt, + >, +{ +} From 1364e25468438226ef4783ab0a517d4e3003f9c4 Mon Sep 17 00:00:00 2001 From: Dan Cline <6798349+Rjected@users.noreply.github.com> Date: Thu, 19 Dec 2024 12:47:39 +0200 Subject: [PATCH 03/30] chore: add BlockBody associated type helpers (#13451) --- crates/primitives-traits/src/block/body.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/crates/primitives-traits/src/block/body.rs b/crates/primitives-traits/src/block/body.rs index 4546e855427b..8c8804b593a0 100644 --- a/crates/primitives-traits/src/block/body.rs +++ b/crates/primitives-traits/src/block/body.rs @@ -98,3 +98,11 @@ pub trait BlockBody: self.encoded_2718_transactions_iter().map(Into::into).collect() } } + +/// This is a helper alias to make it easy to refer to the inner `Transaction` associated type of a +/// given type that implements [`BlockBody`]. +pub type BodyTx = ::Transaction; + +/// This is a helper alias to make it easy to refer to the inner `OmmerHeader` associated type of a +/// given type that implements [`BlockBody`]. +pub type BodyOmmer = ::OmmerHeader; From d5a096f52f0ffec07ccc28d231c423caf5f65184 Mon Sep 17 00:00:00 2001 From: Dan Cline <6798349+Rjected@users.noreply.github.com> Date: Thu, 19 Dec 2024 12:48:41 +0200 Subject: [PATCH 04/30] chore: remove redundant bound in EthRequestHandler (#13450) --- crates/net/network/src/eth_requests.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/net/network/src/eth_requests.rs b/crates/net/network/src/eth_requests.rs index 47dc8a89e992..fe411912089b 100644 --- a/crates/net/network/src/eth_requests.rs +++ b/crates/net/network/src/eth_requests.rs @@ -4,7 +4,7 @@ use crate::{ budget::DEFAULT_BUDGET_TRY_DRAIN_DOWNLOADERS, metered_poll_nested_stream_with_budget, metrics::EthRequestHandlerMetrics, }; -use alloy_consensus::{BlockHeader, ReceiptWithBloom, TxReceipt}; +use alloy_consensus::{BlockHeader, ReceiptWithBloom}; use alloy_eips::BlockHashOrNumber; use alloy_rlp::Encodable; use futures::StreamExt; @@ -16,7 +16,7 @@ use reth_network_api::test_utils::PeersHandle; use reth_network_p2p::error::RequestResult; use reth_network_peers::PeerId; use reth_primitives_traits::Block; -use reth_storage_api::{BlockReader, HeaderProvider, ReceiptProvider}; +use reth_storage_api::{BlockReader, HeaderProvider}; use std::{ future::Future, pin::Pin, @@ -81,7 +81,7 @@ impl EthRequestHandler { impl EthRequestHandler where N: NetworkPrimitives, - C: BlockReader + HeaderProvider + ReceiptProvider, + C: BlockReader, { /// Returns the list of requested headers fn get_headers_response(&self, request: GetBlockHeaders) -> Vec { From 2eecf75c35bcabb32e9ecda4a13cd1b03c343b8f Mon Sep 17 00:00:00 2001 From: Dan Cline <6798349+Rjected@users.noreply.github.com> Date: Thu, 19 Dec 2024 12:57:13 +0200 Subject: [PATCH 05/30] chore: remove redundant Send bounds for BlockReader types (#13452) --- crates/exex/exex/src/backfill/stream.rs | 10 +++++----- crates/rpc/rpc/src/eth/core.rs | 4 ++-- crates/storage/storage-api/src/transactions.rs | 2 +- crates/transaction-pool/src/maintain.rs | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/crates/exex/exex/src/backfill/stream.rs b/crates/exex/exex/src/backfill/stream.rs index 0e27954eb41d..b36841540799 100644 --- a/crates/exex/exex/src/backfill/stream.rs +++ b/crates/exex/exex/src/backfill/stream.rs @@ -8,7 +8,7 @@ use futures::{ use reth_evm::execute::{BlockExecutionError, BlockExecutionOutput, BlockExecutorProvider}; use reth_node_api::NodePrimitives; use reth_primitives::{BlockWithSenders, EthPrimitives}; -use reth_provider::{BlockReader, Chain, HeaderProvider, StateProviderFactory}; +use reth_provider::{BlockReader, Chain, StateProviderFactory}; use reth_prune_types::PruneModes; use reth_stages_api::ExecutionStageThresholds; use reth_tracing::tracing::debug; @@ -114,8 +114,8 @@ where impl Stream for StreamBackfillJob> where - E: BlockExecutorProvider> + Clone + Send + 'static, - P: HeaderProvider + BlockReader + StateProviderFactory + Clone + Send + Unpin + 'static, + E: BlockExecutorProvider> + Clone + 'static, + P: BlockReader + StateProviderFactory + Clone + Unpin + 'static, { type Item = BackfillJobResult>; @@ -147,8 +147,8 @@ where impl Stream for StreamBackfillJob> where - E: BlockExecutorProvider> + Clone + Send + 'static, - P: HeaderProvider + BlockReader + StateProviderFactory + Clone + Send + Unpin + 'static, + E: BlockExecutorProvider> + Clone + 'static, + P: BlockReader + StateProviderFactory + Clone + Unpin + 'static, { type Item = BackfillJobResult>; diff --git a/crates/rpc/rpc/src/eth/core.rs b/crates/rpc/rpc/src/eth/core.rs index 8341742e4d17..48320de2b3ad 100644 --- a/crates/rpc/rpc/src/eth/core.rs +++ b/crates/rpc/rpc/src/eth/core.rs @@ -158,7 +158,7 @@ where impl RpcNodeCore for EthApi where - Provider: BlockReader + Send + Sync + Clone + Unpin, + Provider: BlockReader + Clone + Unpin, Pool: Send + Sync + Clone + Unpin, Network: Send + Sync + Clone, EvmConfig: Send + Sync + Clone + Unpin, @@ -193,7 +193,7 @@ where impl RpcNodeCoreExt for EthApi where - Provider: BlockReader + Send + Sync + Clone + Unpin, + Provider: BlockReader + Clone + Unpin, Pool: Send + Sync + Clone + Unpin, Network: Send + Sync + Clone, EvmConfig: Send + Sync + Clone + Unpin, diff --git a/crates/storage/storage-api/src/transactions.rs b/crates/storage/storage-api/src/transactions.rs index ca2bcaeb4690..ed44b4dd1322 100644 --- a/crates/storage/storage-api/src/transactions.rs +++ b/crates/storage/storage-api/src/transactions.rs @@ -86,7 +86,7 @@ pub type ProviderTx

=

::Transaction; /// Client trait for fetching additional transactions related data. #[auto_impl::auto_impl(&, Arc)] -pub trait TransactionsProviderExt: BlockReader + Send + Sync { +pub trait TransactionsProviderExt: BlockReader { /// Get transactions range by block range. fn transaction_range_by_block_range( &self, diff --git a/crates/transaction-pool/src/maintain.rs b/crates/transaction-pool/src/maintain.rs index a0c5571a4abb..23bf75b01d66 100644 --- a/crates/transaction-pool/src/maintain.rs +++ b/crates/transaction-pool/src/maintain.rs @@ -83,7 +83,7 @@ where BlockBody = reth_primitives::BlockBody, SignedTx = TransactionSigned, >, - Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + Send + 'static, + Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + 'static, P: TransactionPoolExt> + 'static, St: Stream> + Send + Unpin + 'static, Tasks: TaskSpawner + 'static, @@ -109,7 +109,7 @@ pub async fn maintain_transaction_pool( BlockBody = reth_primitives::BlockBody, SignedTx = TransactionSigned, >, - Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + Send + 'static, + Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + 'static, P: TransactionPoolExt> + 'static, St: Stream> + Send + Unpin + 'static, Tasks: TaskSpawner + 'static, From 726d064afbe59a866623e069f22d57568967d692 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 19 Dec 2024 12:33:26 +0100 Subject: [PATCH 06/30] chore: add missing getters (#13444) --- crates/optimism/rpc/src/eth/mod.rs | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/crates/optimism/rpc/src/eth/mod.rs b/crates/optimism/rpc/src/eth/mod.rs index 4304a2a37411..f244d114ed93 100644 --- a/crates/optimism/rpc/src/eth/mod.rs +++ b/crates/optimism/rpc/src/eth/mod.rs @@ -79,6 +79,16 @@ where + 'static, >, { + /// Returns a reference to the [`EthApiNodeBackend`]. + pub fn eth_api(&self) -> &EthApiNodeBackend { + self.inner.eth_api() + } + + /// Returns the configured sequencer client, if any. + pub fn sequencer_client(&self) -> Option<&SequencerClient> { + self.inner.sequencer_client() + } + /// Build a [`OpEthApi`] using [`OpEthApiBuilder`]. pub const fn builder() -> OpEthApiBuilder { OpEthApiBuilder::new() @@ -273,6 +283,18 @@ struct OpEthApiInner { sequencer_client: Option, } +impl OpEthApiInner { + /// Returns a reference to the [`EthApiNodeBackend`]. + const fn eth_api(&self) -> &EthApiNodeBackend { + &self.eth_api + } + + /// Returns the configured sequencer client, if any. + const fn sequencer_client(&self) -> Option<&SequencerClient> { + self.sequencer_client.as_ref() + } +} + /// A type that knows how to build a [`OpEthApi`]. #[derive(Debug, Default)] pub struct OpEthApiBuilder { From 320a0b9af96b3ef71d40f5f794bbb7a15622e064 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 19 Dec 2024 12:44:20 +0100 Subject: [PATCH 07/30] feat: add manual broadcast in full (#13453) --- crates/net/network/src/transactions/mod.rs | 55 ++++++++++++++++------ 1 file changed, 40 insertions(+), 15 deletions(-) diff --git a/crates/net/network/src/transactions/mod.rs b/crates/net/network/src/transactions/mod.rs index badb20c302ce..67a033e70537 100644 --- a/crates/net/network/src/transactions/mod.rs +++ b/crates/net/network/src/transactions/mod.rs @@ -136,7 +136,7 @@ impl TransactionsHandle { rx.await } - /// Manually propagate full transactions to a specific peer. + /// Manually propagate full transaction hashes to a specific peer. /// /// Do nothing if transactions are empty. pub fn propagate_transactions_to(&self, transactions: Vec, peer: PeerId) { @@ -146,12 +146,10 @@ impl TransactionsHandle { self.send(TransactionsCommand::PropagateTransactionsTo(transactions, peer)) } - /// Manually propagate the given transactions to all peers. + /// Manually propagate the given transaction hashes to all peers. /// /// It's up to the [`TransactionsManager`] whether the transactions are sent as hashes or in /// full. - /// - /// Do nothing if transactions are empty. pub fn propagate_transactions(&self, transactions: Vec) { if transactions.is_empty() { return @@ -159,6 +157,22 @@ impl TransactionsHandle { self.send(TransactionsCommand::PropagateTransactions(transactions)) } + /// Manually propagate the given transactions to all peers. + /// + /// It's up to the [`TransactionsManager`] whether the transactions are sent as hashes or in + /// full. + pub fn broadcast_transactions( + &self, + transactions: impl IntoIterator, + ) { + let transactions = + transactions.into_iter().map(PropagateTransaction::new).collect::>(); + if transactions.is_empty() { + return + } + self.send(TransactionsCommand::BroadcastTransactions(transactions)) + } + /// Request the transaction hashes known by specific peers. pub async fn get_transaction_hashes( &self, @@ -735,7 +749,7 @@ where // filter all transactions unknown to the peer let mut full_transactions = FullTransactionsBuilder::new(peer.version); - let to_propagate = self.pool.get_all(txs).into_iter().map(PropagateTransaction::new); + let to_propagate = self.pool.get_all(txs).into_iter().map(PropagateTransaction::pool_tx); if propagation_mode.is_forced() { // skip cache check if forced @@ -811,7 +825,7 @@ where .pool .get_all(hashes) .into_iter() - .map(PropagateTransaction::new) + .map(PropagateTransaction::pool_tx) .collect::>(); let mut propagated = PropagatedTransactions::default(); @@ -956,7 +970,7 @@ where /// __without__ their sidecar, because 4844 transactions are only ever announced as hashes. fn propagate_all(&mut self, hashes: Vec) { let propagated = self.propagate_transactions( - self.pool.get_all(hashes).into_iter().map(PropagateTransaction::new).collect(), + self.pool.get_all(hashes).into_iter().map(PropagateTransaction::pool_tx).collect(), PropagationMode::Basic, ); @@ -1014,6 +1028,9 @@ where } } TransactionsCommand::PropagateTransactions(txs) => self.propagate_all(txs), + TransactionsCommand::BroadcastTransactions(txs) => { + self.propagate_transactions(txs, PropagationMode::Forced); + } TransactionsCommand::GetTransactionHashes { peers, tx } => { let mut res = HashMap::with_capacity(peers.len()); for peer_id in peers { @@ -1508,8 +1525,14 @@ struct PropagateTransaction { } impl PropagateTransaction { + /// Create a new instance from a transaction. + pub fn new(transaction: T) -> Self { + let size = transaction.length(); + Self { size, transaction: Arc::new(transaction) } + } + /// Create a new instance from a pooled transaction - fn new

(tx: Arc>) -> Self + fn pool_tx

(tx: Arc>) -> Self where P: PoolTransaction, { @@ -1797,8 +1820,10 @@ enum TransactionsCommand { GetActivePeers(oneshot::Sender>), /// Propagate a collection of full transactions to a specific peer. PropagateTransactionsTo(Vec, PeerId), - /// Propagate a collection of full transactions to all peers. + /// Propagate a collection of hashes to all peers. PropagateTransactions(Vec), + /// Propagate a collection of broadcastable transactions in full to all peers. + BroadcastTransactions(Vec>), /// Request transaction hashes known by specific peers from the [`TransactionsManager`]. GetTransactionHashes { peers: Vec, @@ -2370,7 +2395,7 @@ mod tests { assert!(builder.is_empty()); let mut factory = MockTransactionFactory::default(); - let tx = PropagateTransaction::new(Arc::new(factory.create_eip1559())); + let tx = PropagateTransaction::pool_tx(Arc::new(factory.create_eip1559())); builder.push(&tx); assert!(!builder.is_empty()); @@ -2391,7 +2416,7 @@ mod tests { // create a transaction that still fits tx.transaction.set_size(DEFAULT_SOFT_LIMIT_BYTE_SIZE_TRANSACTIONS_BROADCAST_MESSAGE + 1); let tx = Arc::new(tx); - let tx = PropagateTransaction::new(tx); + let tx = PropagateTransaction::pool_tx(tx); builder.push(&tx); assert!(!builder.is_empty()); @@ -2416,7 +2441,7 @@ mod tests { assert!(builder.is_empty()); let mut factory = MockTransactionFactory::default(); - let tx = PropagateTransaction::new(Arc::new(factory.create_eip4844())); + let tx = PropagateTransaction::pool_tx(Arc::new(factory.create_eip4844())); builder.push(&tx); assert!(!builder.is_empty()); @@ -2425,7 +2450,7 @@ mod tests { let txs = txs.pooled.unwrap(); assert_eq!(txs.len(), 1); - let tx = PropagateTransaction::new(Arc::new(factory.create_eip1559())); + let tx = PropagateTransaction::pool_tx(Arc::new(factory.create_eip1559())); builder.push(&tx); let txs = builder.clone().build(); @@ -2461,9 +2486,9 @@ mod tests { let mut propagate = vec![]; let mut factory = MockTransactionFactory::default(); let eip1559_tx = Arc::new(factory.create_eip1559()); - propagate.push(PropagateTransaction::new(eip1559_tx.clone())); + propagate.push(PropagateTransaction::pool_tx(eip1559_tx.clone())); let eip4844_tx = Arc::new(factory.create_eip4844()); - propagate.push(PropagateTransaction::new(eip4844_tx.clone())); + propagate.push(PropagateTransaction::pool_tx(eip4844_tx.clone())); let propagated = tx_manager.propagate_transactions(propagate.clone(), PropagationMode::Basic); From 5639552f95f10f90e8520909899614019dd942fe Mon Sep 17 00:00:00 2001 From: bendanzhentan <455462586@qq.com> Date: Thu, 19 Dec 2024 19:34:00 +0800 Subject: [PATCH 08/30] fix: drop and remove lockfile when test (#13437) --- crates/storage/db/src/lockfile.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/db/src/lockfile.rs b/crates/storage/db/src/lockfile.rs index 63962bc4a4a8..15ddee2f0fef 100644 --- a/crates/storage/db/src/lockfile.rs +++ b/crates/storage/db/src/lockfile.rs @@ -64,7 +64,7 @@ impl StorageLock { impl Drop for StorageLock { fn drop(&mut self) { // The lockfile is not created in disable-lock mode, so we don't need to delete it. - #[cfg(not(feature = "disable-lock"))] + #[cfg(any(test, not(feature = "disable-lock")))] if Arc::strong_count(&self.0) == 1 && self.0.file_path.exists() { // TODO: should only happen during tests that the file does not exist: tempdir is // getting dropped first. However, tempdir shouldn't be dropped From d1b3dee5cbc74b35908152fcb5b90aeddecb6e8f Mon Sep 17 00:00:00 2001 From: Federico Gimenez Date: Thu, 19 Dec 2024 13:48:12 +0200 Subject: [PATCH 09/30] perf(trie): use local ThreadPool in Parallel::multiproof (#13416) --- crates/trie/parallel/src/proof.rs | 89 ++++++++++++++++++++++++++----- 1 file changed, 75 insertions(+), 14 deletions(-) diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 1d9daff5c920..149a53a1e4b2 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -24,15 +24,15 @@ use reth_trie::{ }; use reth_trie_common::proof::ProofRetainer; use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; -use std::sync::Arc; -use tracing::{debug, error}; +use std::{sync::Arc, time::Instant}; +use tracing::{debug, error, trace}; #[cfg(feature = "metrics")] use crate::metrics::ParallelStateRootMetrics; /// TODO: #[derive(Debug)] -pub struct ParallelProof { +pub struct ParallelProof<'env, Factory> { /// Consistent view of the database. view: ConsistentDbView, /// The sorted collection of cached in-memory intermediate trie nodes that @@ -46,18 +46,21 @@ pub struct ParallelProof { pub prefix_sets: Arc, /// Flag indicating whether to include branch node hash masks in the proof. collect_branch_node_hash_masks: bool, + /// Thread pool for local tasks + thread_pool: &'env rayon::ThreadPool, /// Parallel state root metrics. #[cfg(feature = "metrics")] metrics: ParallelStateRootMetrics, } -impl ParallelProof { +impl<'env, Factory> ParallelProof<'env, Factory> { /// Create new state proof generator. pub fn new( view: ConsistentDbView, nodes_sorted: Arc, state_sorted: Arc, prefix_sets: Arc, + thread_pool: &'env rayon::ThreadPool, ) -> Self { Self { view, @@ -65,6 +68,7 @@ impl ParallelProof { state_sorted, prefix_sets, collect_branch_node_hash_masks: false, + thread_pool, #[cfg(feature = "metrics")] metrics: ParallelStateRootMetrics::default(), } @@ -77,7 +81,7 @@ impl ParallelProof { } } -impl ParallelProof +impl ParallelProof<'_, Factory> where Factory: DatabaseProviderFactory + StateCommitmentProvider @@ -112,26 +116,50 @@ where prefix_sets.account_prefix_set.iter().map(|nibbles| B256::from_slice(&nibbles.pack())), prefix_sets.storage_prefix_sets.clone(), ); + let storage_root_targets_len = storage_root_targets.len(); + + debug!( + target: "trie::parallel_state_root", + total_targets = storage_root_targets_len, + "Starting parallel proof generation" + ); // Pre-calculate storage roots for accounts which were changed. tracker.set_precomputed_storage_roots(storage_root_targets.len() as u64); - debug!(target: "trie::parallel_state_root", len = storage_root_targets.len(), "pre-generating storage proofs"); + let mut storage_proofs = B256HashMap::with_capacity_and_hasher(storage_root_targets.len(), Default::default()); + for (hashed_address, prefix_set) in storage_root_targets.into_iter().sorted_unstable_by_key(|(address, _)| *address) { let view = self.view.clone(); let target_slots = targets.get(&hashed_address).cloned().unwrap_or_default(); - let trie_nodes_sorted = self.nodes_sorted.clone(); let hashed_state_sorted = self.state_sorted.clone(); + let collect_masks = self.collect_branch_node_hash_masks; let (tx, rx) = std::sync::mpsc::sync_channel(1); - rayon::spawn_fifo(move || { + self.thread_pool.spawn_fifo(move || { + debug!( + target: "trie::parallel", + ?hashed_address, + "Starting proof calculation" + ); + + let task_start = Instant::now(); let result = (|| -> Result<_, ParallelStateRootError> { + let provider_start = Instant::now(); let provider_ro = view.provider_ro()?; + trace!( + target: "trie::parallel", + ?hashed_address, + provider_time = ?provider_start.elapsed(), + "Got provider" + ); + + let cursor_start = Instant::now(); let trie_cursor_factory = InMemoryTrieCursorFactory::new( DatabaseTrieCursorFactory::new(provider_ro.tx_ref()), &trie_nodes_sorted, @@ -140,19 +168,42 @@ where DatabaseHashedCursorFactory::new(provider_ro.tx_ref()), &hashed_state_sorted, ); + trace!( + target: "trie::parallel", + ?hashed_address, + cursor_time = ?cursor_start.elapsed(), + "Created cursors" + ); - StorageProof::new_hashed( + let proof_start = Instant::now(); + let proof_result = StorageProof::new_hashed( trie_cursor_factory, hashed_cursor_factory, hashed_address, ) .with_prefix_set_mut(PrefixSetMut::from(prefix_set.iter().cloned())) - .with_branch_node_hash_masks(self.collect_branch_node_hash_masks) + .with_branch_node_hash_masks(collect_masks) .storage_multiproof(target_slots) - .map_err(|e| ParallelStateRootError::Other(e.to_string())) + .map_err(|e| ParallelStateRootError::Other(e.to_string())); + + trace!( + target: "trie::parallel", + ?hashed_address, + proof_time = ?proof_start.elapsed(), + "Completed proof calculation" + ); + + proof_result })(); - if let Err(err) = tx.send(result) { - error!(target: "trie::parallel", ?hashed_address, err_content = ?err.0, "Failed to send proof result"); + + if let Err(e) = tx.send(result) { + error!( + target: "trie::parallel", + ?hashed_address, + error = ?e, + task_time = ?task_start.elapsed(), + "Failed to send proof result" + ); } }); storage_proofs.insert(hashed_address, rx); @@ -338,12 +389,22 @@ mod tests { let trie_cursor_factory = DatabaseTrieCursorFactory::new(provider_rw.tx_ref()); let hashed_cursor_factory = DatabaseHashedCursorFactory::new(provider_rw.tx_ref()); + let num_threads = + std::thread::available_parallelism().map_or(1, |num| (num.get() / 2).max(1)); + + let state_root_task_pool = rayon::ThreadPoolBuilder::new() + .num_threads(num_threads) + .thread_name(|i| format!("proof-worker-{}", i)) + .build() + .expect("Failed to create proof worker thread pool"); + assert_eq!( ParallelProof::new( consistent_view, Default::default(), Default::default(), - Default::default() + Default::default(), + &state_root_task_pool ) .multiproof(targets.clone()) .unwrap(), From 790a1e2898090899eaae20feb0cbe40d19e5f17a Mon Sep 17 00:00:00 2001 From: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com> Date: Thu, 19 Dec 2024 13:51:37 +0200 Subject: [PATCH 10/30] fix(trie): memoize the hash of a revealed sparse node (#13233) --- Cargo.lock | 162 +++++++++++++++++---------------- crates/trie/sparse/src/trie.rs | 28 ++++-- 2 files changed, 105 insertions(+), 85 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 554b8742b25d..299f84e8784f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -100,15 +100,15 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.21" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" +checksum = "45862d1c77f2228b9e10bc609d5bc203d86ebc9b87ad8d5d5167a6c9abf739d9" [[package]] name = "alloy-chains" -version = "0.1.48" +version = "0.1.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0161082e0edd9013d23083465cc04b20e44b7a15646d36ba7b0cdb7cd6fe18f" +checksum = "18c5c520273946ecf715c0010b4e3503d7eba9893cd9ce6b7fff5654c4a3c470" dependencies = [ "alloy-primitives", "alloy-rlp", @@ -351,7 +351,7 @@ dependencies = [ "getrandom 0.2.15", "hashbrown 0.15.2", "hex-literal", - "indexmap 2.7.0", + "indexmap 2.6.0", "itoa", "k256", "keccak-asm", @@ -697,7 +697,7 @@ dependencies = [ "alloy-sol-macro-input", "const-hex", "heck", - "indexmap 2.7.0", + "indexmap 2.6.0", "proc-macro-error2", "proc-macro2", "quote", @@ -908,9 +908,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.94" +version = "1.0.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1fd03a028ef38ba2276dce7e33fcd6369c158a1bca17946c4b1b701891c1ff7" +checksum = "4c95c10ba0b00a02636238b814946408b1322d5ac4760326e6fb8ec956d85775" [[package]] name = "aquamarine" @@ -1407,7 +1407,7 @@ dependencies = [ "bitflags 2.6.0", "boa_interner", "boa_macros", - "indexmap 2.7.0", + "indexmap 2.6.0", "num-bigint", "rustc-hash 2.1.0", ] @@ -1433,7 +1433,7 @@ dependencies = [ "fast-float", "hashbrown 0.14.5", "icu_normalizer", - "indexmap 2.7.0", + "indexmap 2.6.0", "intrusive-collections", "itertools 0.13.0", "num-bigint", @@ -1479,7 +1479,7 @@ dependencies = [ "boa_gc", "boa_macros", "hashbrown 0.14.5", - "indexmap 2.7.0", + "indexmap 2.6.0", "once_cell", "phf", "rustc-hash 2.1.0", @@ -1805,9 +1805,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.23" +version = "4.5.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3135e7ec2ef7b10c6ed8950f0f792ed96ee093fa088608f1c76e569722700c84" +checksum = "fb3b4b9e5a7c7514dfa52869339ee98b3156b0bfb4e8a77c4ff4babb64b1604f" dependencies = [ "clap_builder", "clap_derive", @@ -1815,9 +1815,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.23" +version = "4.5.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30582fc632330df2bd26877bde0c1f4470d57c582bbc070376afcd04d8cb4838" +checksum = "b17a95aa67cc7b5ebd32aa5370189aa0d79069ef1c64ce893bd30fb24bff20ec" dependencies = [ "anstream", "anstyle", @@ -1839,9 +1839,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.4" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" +checksum = "afb84c814227b90d6895e01398aee0d8033c00e7466aca416fb6a8e0eb19d8a7" [[package]] name = "coins-bip32" @@ -1987,9 +1987,9 @@ checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" [[package]] name = "const_format" -version = "0.2.34" +version = "0.2.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "126f97965c8ad46d6d9163268ff28432e8f6a1196a55578867832e3049df63dd" +checksum = "50c655d81ff1114fb0dcdea9225ea9f0cc712a6f8d189378e82bdf62a473a64b" dependencies = [ "const_format_proc_macros", "konst", @@ -1997,9 +1997,9 @@ dependencies = [ [[package]] name = "const_format_proc_macros" -version = "0.2.34" +version = "0.2.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d57c2eccfb16dbac1f4e61e206105db5820c9d26c3c472bc17c774259ef7744" +checksum = "eff1a44b93f47b1bac19a27932f5c591e43d1ba357ee4f61526c8a25603f0eb1" dependencies = [ "proc-macro2", "quote", @@ -2797,11 +2797,12 @@ dependencies = [ [[package]] name = "ethereum_ssz" -version = "0.8.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "036c84bd29bff35e29bbee3c8fc0e2fb95db12b6f2f3cae82a827fbc97256f3a" +checksum = "bfbba28f4f3f32d92c06a64f5bf6c4537b5d4e21f28c689bd2bbaecfea4e0d3e" dependencies = [ "alloy-primitives", + "derivative", "ethereum_serde_utils", "itertools 0.13.0", "serde", @@ -2812,9 +2813,9 @@ dependencies = [ [[package]] name = "ethereum_ssz_derive" -version = "0.8.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9dc8e67e1f770f5aa4c2c2069aaaf9daee7ac21bed357a71b911b37a58966cfb" +checksum = "0d37845ba7c16bf4be8be4b5786f03a2ba5f2fda0d7f9e7cb2282f69cff420d7" dependencies = [ "darling", "proc-macro2", @@ -3434,9 +3435,9 @@ checksum = "42012b0f064e01aa58b545fe3727f90f7dd4020f4a3ea735b50344965f5a57e9" [[package]] name = "generator" -version = "0.8.4" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6bd114ceda131d3b1d665eba35788690ad37f5916457286b32ab6fd3c438dd" +checksum = "dbb949699c3e4df3a183b1d2142cb24277057055ed23c68ed58894f76c517223" dependencies = [ "cfg-if", "libc", @@ -3572,7 +3573,7 @@ dependencies = [ "futures-core", "futures-sink", "http", - "indexmap 2.7.0", + "indexmap 2.6.0", "slab", "tokio", "tokio-util", @@ -3781,9 +3782,9 @@ dependencies = [ [[package]] name = "http" -version = "1.2.0" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f16ca2af56261c99fba8bac40a10251ce8188205a4c448fbb745a2e4daa76fea" +checksum = "21b9ddb458710bc376481b842f5da65cdf31522de232c1ca8146abce2a358258" dependencies = [ "bytes", "fnv", @@ -4205,9 +4206,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.7.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f822373a4fe84d4bb149bf54e584a7f4abec90e072ed49cda0edea5b95471f" +checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" dependencies = [ "arbitrary", "equivalent", @@ -4234,7 +4235,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ "ahash", - "indexmap 2.7.0", + "indexmap 2.6.0", "is-terminal", "itoa", "log", @@ -4422,9 +4423,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.76" +version = "0.3.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6717b6b5b077764fb5966237269cb3c64edddde4b14ce42647430a78ced9e7b7" +checksum = "a865e038f7f6ed956f788f0d7d60c541fff74c7bd74272c5d4cf15c63743e705" dependencies = [ "once_cell", "wasm-bindgen", @@ -4955,7 +4956,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "85b6f8152da6d7892ff1b7a1c0fa3f435e92b5918ad67035c3bb432111d9a29b" dependencies = [ "base64 0.22.1", - "indexmap 2.7.0", + "indexmap 2.6.0", "metrics", "metrics-util", "quanta", @@ -4987,7 +4988,7 @@ dependencies = [ "crossbeam-epoch", "crossbeam-utils", "hashbrown 0.15.2", - "indexmap 2.7.0", + "indexmap 2.6.0", "metrics", "ordered-float", "quanta", @@ -5174,9 +5175,9 @@ dependencies = [ [[package]] name = "multihash" -version = "0.19.3" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b430e7953c29dd6a09afc29ff0bb69c6e306329ee6794700aee27b76a1aea8d" +checksum = "cc41f430805af9d1cf4adae4ed2149c759b877b01d909a1f40256188d09345d2" dependencies = [ "core2", "unsigned-varint", @@ -5612,9 +5613,9 @@ dependencies = [ [[package]] name = "parity-scale-codec" -version = "3.6.12" +version = "3.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "306800abfa29c7f16596b5970a588435e3d5b3149683d00c12b699cc19f895ee" +checksum = "8be4817d39f3272f69c59fe05d0535ae6456c2dc2fa1ba02910296c7e0a5c590" dependencies = [ "arbitrary", "arrayvec", @@ -5623,19 +5624,20 @@ dependencies = [ "bytes", "impl-trait-for-tuples", "parity-scale-codec-derive", + "rustversion", "serde", ] [[package]] name = "parity-scale-codec-derive" -version = "3.6.12" +version = "3.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d830939c76d294956402033aee57a6da7b438f2294eb94864c37b0569053a42c" +checksum = "8781a75c6205af67215f382092b6e0a4ff3734798523e69073d4bcd294ec767b" dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.90", ] [[package]] @@ -5701,12 +5703,12 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.7.15" +version = "2.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b7cafe60d6cf8e62e1b9b2ea516a089c008945bb5a275416789e7db0bc199dc" +checksum = "879952a81a83930934cbf1786752d6dedc3b1f29e8f8fb2ad1d0a36f377cf442" dependencies = [ "memchr", - "thiserror 2.0.7", + "thiserror 1.0.69", "ucd-trie", ] @@ -7913,7 +7915,7 @@ dependencies = [ "criterion", "dashmap 6.1.0", "derive_more", - "indexmap 2.7.0", + "indexmap 2.6.0", "parking_lot", "pprof", "rand 0.8.5", @@ -10329,7 +10331,7 @@ version = "1.0.133" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c7fceb2473b9166b2294ef05efcb65a3db80803f0b03ef86a5fc88a2b85ee377" dependencies = [ - "indexmap 2.7.0", + "indexmap 2.6.0", "itoa", "memchr", "ryu", @@ -10389,7 +10391,7 @@ dependencies = [ "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.7.0", + "indexmap 2.6.0", "serde", "serde_derive", "serde_json", @@ -10643,9 +10645,9 @@ dependencies = [ [[package]] name = "soketto" -version = "0.8.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e859df029d160cb88608f5d7df7fb4753fd20fdfb4de5644f3d8b8440841721" +checksum = "37468c595637c10857701c990f93a40ce0e357cedb0953d1c26c8d8027f9bb53" dependencies = [ "base64 0.22.1", "bytes", @@ -11015,9 +11017,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.37" +version = "0.3.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35e7868883861bd0e56d9ac6efcaaca0d6d5d82a2a7ec8209ff492c07cf37b21" +checksum = "5dfd88e563464686c916c7e46e623e520ddc6d79fa6641390f2e3fa86e83e885" dependencies = [ "deranged", "itoa", @@ -11039,9 +11041,9 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.19" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2834e6017e3e5e4b9834939793b282bc03b37a3336245fa820e35e233e2a85de" +checksum = "3f252a68540fde3a3877aeea552b832b40ab9a69e318efd078774a01ddee1ccf" dependencies = [ "num-conv", "time-core", @@ -11093,9 +11095,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.42.0" +version = "1.41.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cec9b21b0450273377fc97bd4c33a8acffc8c996c987a7c5b319a0083707551" +checksum = "22cfb5bee7a6a52939ca9224d6ac897bb669134078daa8735560897f69de4d33" dependencies = [ "backtrace", "bytes", @@ -11122,19 +11124,20 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.1" +version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" +checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" dependencies = [ "rustls", + "rustls-pki-types", "tokio", ] [[package]] name = "tokio-stream" -version = "0.1.17" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eca58d7bba4a75707817a2c44174253f9236b2d5fbd055602e9d5c07c139a047" +checksum = "4f4e6ce100d0eb49a2734f8c0812bcd324cf357d21810932c5df6b96ef2b86f1" dependencies = [ "futures-core", "pin-project-lite", @@ -11160,9 +11163,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.13" +version = "0.7.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7fcaa8d55a2bdd6b83ace262b016eca0d79ee02818c5c1bcdf0305114081078" +checksum = "61e7c3654c13bcd040d4a03abee2c75b1d14a37b423cf5a813ceae1cc903ec6a" dependencies = [ "bytes", "futures-core", @@ -11200,7 +11203,7 @@ version = "0.22.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ - "indexmap 2.7.0", + "indexmap 2.6.0", "serde", "serde_spanned", "toml_datetime", @@ -11710,9 +11713,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.99" +version = "0.2.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a474f6281d1d70c17ae7aa6a613c87fce69a127e2624002df63dcb39d6cf6396" +checksum = "d15e63b4482863c109d70a7b8706c1e364eb6ea449b201a76c5b89cedcec2d5c" dependencies = [ "cfg-if", "once_cell", @@ -11721,12 +11724,13 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.99" +version = "0.2.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f89bb38646b4f81674e8f5c3fb81b562be1fd936d84320f3264486418519c79" +checksum = "8d36ef12e3aaca16ddd3f67922bc63e48e953f126de60bd33ccc0101ef9998cd" dependencies = [ "bumpalo", "log", + "once_cell", "proc-macro2", "quote", "syn 2.0.90", @@ -11735,9 +11739,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.49" +version = "0.4.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38176d9b44ea84e9184eff0bc34cc167ed044f816accfe5922e54d84cf48eca2" +checksum = "9dfaf8f50e5f293737ee323940c7d8b08a66a95a419223d9f41610ca08b0833d" dependencies = [ "cfg-if", "js-sys", @@ -11748,9 +11752,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.99" +version = "0.2.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cc6181fd9a7492eef6fef1f33961e3695e4579b9872a6f7c83aee556666d4fe" +checksum = "705440e08b42d3e4b36de7d66c944be628d579796b8090bfa3471478a2260051" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -11758,9 +11762,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.99" +version = "0.2.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" +checksum = "98c9ae5a76e46f4deecd0f0255cc223cfa18dc9b261213b8aa0c7b36f61b3f1d" dependencies = [ "proc-macro2", "quote", @@ -11771,9 +11775,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.99" +version = "0.2.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943aab3fdaaa029a6e0271b35ea10b72b943135afe9bffca82384098ad0e06a6" +checksum = "6ee99da9c5ba11bd675621338ef6fa52296b76b83305e9b6e5c77d4c286d6d49" [[package]] name = "wasm-streams" @@ -11804,9 +11808,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.76" +version = "0.3.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04dd7223427d52553d3702c004d3b2fe07c148165faa56313cb00211e31c12bc" +checksum = "a98bc3c33f0fe7e59ad7cd041b89034fa82a7c2d4365ca538dda6cdaf513863c" dependencies = [ "js-sys", "wasm-bindgen", diff --git a/crates/trie/sparse/src/trie.rs b/crates/trie/sparse/src/trie.rs index cae03892bf05..04f034e1c8a5 100644 --- a/crates/trie/sparse/src/trie.rs +++ b/crates/trie/sparse/src/trie.rs @@ -305,8 +305,14 @@ impl

RevealedSparseTrie

{ match self.nodes.entry(path) { Entry::Occupied(mut entry) => match entry.get() { // Blinded nodes can be replaced. - SparseNode::Hash(_) => { - entry.insert(SparseNode::new_branch(branch.state_mask)); + SparseNode::Hash(hash) => { + entry.insert(SparseNode::Branch { + state_mask: branch.state_mask, + // Memoize the hash of a previously blinded node in a new branch + // node. + hash: Some(*hash), + store_in_db_trie: None, + }); } // Branch node already exists, or an extension node was placed where a // branch node was before. @@ -327,10 +333,15 @@ impl

RevealedSparseTrie

{ } TrieNode::Extension(ext) => match self.nodes.entry(path) { Entry::Occupied(mut entry) => match entry.get() { - SparseNode::Hash(_) => { + SparseNode::Hash(hash) => { let mut child_path = entry.key().clone(); child_path.extend_from_slice_unchecked(&ext.key); - entry.insert(SparseNode::new_ext(ext.key)); + entry.insert(SparseNode::Extension { + key: ext.key, + // Memoize the hash of a previously blinded node in a new extension + // node. + hash: Some(*hash), + }); self.reveal_node_or_hash(child_path, &ext.child)?; } // Extension node already exists, or an extension node was placed where a branch @@ -354,11 +365,16 @@ impl

RevealedSparseTrie

{ }, TrieNode::Leaf(leaf) => match self.nodes.entry(path) { Entry::Occupied(mut entry) => match entry.get() { - SparseNode::Hash(_) => { + SparseNode::Hash(hash) => { let mut full = entry.key().clone(); full.extend_from_slice_unchecked(&leaf.key); - entry.insert(SparseNode::new_leaf(leaf.key)); self.values.insert(full, leaf.value); + entry.insert(SparseNode::Leaf { + key: leaf.key, + // Memoize the hash of a previously blinded node in a new leaf + // node. + hash: Some(*hash), + }); } // Left node already exists. SparseNode::Leaf { .. } => {} From 0a0a2d425d19f0615a74ddb5eb11d616946647a5 Mon Sep 17 00:00:00 2001 From: Federico Gimenez Date: Thu, 19 Dec 2024 14:28:15 +0200 Subject: [PATCH 11/30] perf(engine): use ParallelProof::multiproof in StateRootTask (#13260) --- crates/engine/tree/Cargo.toml | 4 +- crates/engine/tree/benches/state_root_task.rs | 36 ++++++- crates/engine/tree/src/tree/root.rs | 99 +++++++++++-------- crates/trie/parallel/src/proof.rs | 2 +- 4 files changed, 94 insertions(+), 47 deletions(-) diff --git a/crates/engine/tree/Cargo.toml b/crates/engine/tree/Cargo.toml index 6a6a67a5e36b..b5b8fc743645 100644 --- a/crates/engine/tree/Cargo.toml +++ b/crates/engine/tree/Cargo.toml @@ -32,7 +32,6 @@ reth-prune.workspace = true reth-revm.workspace = true reth-stages-api.workspace = true reth-tasks.workspace = true -reth-trie-db.workspace = true reth-trie-parallel.workspace = true reth-trie-sparse.workspace = true reth-trie.workspace = true @@ -82,6 +81,7 @@ reth-stages = { workspace = true, features = ["test-utils"] } reth-static-file.workspace = true reth-testing-utils.workspace = true reth-tracing.workspace = true +reth-trie-db.workspace = true # alloy alloy-rlp.workspace = true @@ -120,6 +120,6 @@ test-utils = [ "reth-static-file", "reth-tracing", "reth-trie/test-utils", - "reth-prune-types?/test-utils", "reth-trie-db/test-utils", + "reth-prune-types?/test-utils", ] diff --git a/crates/engine/tree/benches/state_root_task.rs b/crates/engine/tree/benches/state_root_task.rs index f6a6a4adce78..93f1ff4ec330 100644 --- a/crates/engine/tree/benches/state_root_task.rs +++ b/crates/engine/tree/benches/state_root_task.rs @@ -145,10 +145,34 @@ fn bench_state_root(c: &mut Criterion) { let nodes_sorted = config.nodes_sorted.clone(); let state_sorted = config.state_sorted.clone(); let prefix_sets = config.prefix_sets.clone(); - - (config, state_updates, provider, nodes_sorted, state_sorted, prefix_sets) + let num_threads = std::thread::available_parallelism() + .map_or(1, |num| (num.get() / 2).max(1)); + + let state_root_task_pool = rayon::ThreadPoolBuilder::new() + .num_threads(num_threads) + .thread_name(|i| format!("proof-worker-{}", i)) + .build() + .expect("Failed to create proof worker thread pool"); + + ( + config, + state_updates, + provider, + nodes_sorted, + state_sorted, + prefix_sets, + state_root_task_pool, + ) }, - |(config, state_updates, provider, nodes_sorted, state_sorted, prefix_sets)| { + |( + config, + state_updates, + provider, + nodes_sorted, + state_sorted, + prefix_sets, + state_root_task_pool, + )| { let blinded_provider_factory = ProofBlindedProviderFactory::new( InMemoryTrieCursorFactory::new( DatabaseTrieCursorFactory::new(provider.tx_ref()), @@ -162,7 +186,11 @@ fn bench_state_root(c: &mut Criterion) { ); black_box(std::thread::scope(|scope| { - let task = StateRootTask::new(config, blinded_provider_factory); + let task = StateRootTask::new( + config, + blinded_provider_factory, + &state_root_task_pool, + ); let mut hook = task.state_hook(); let handle = task.spawn(scope); diff --git a/crates/engine/tree/src/tree/root.rs b/crates/engine/tree/src/tree/root.rs index cb64d95d8f92..dd931a8e40b8 100644 --- a/crates/engine/tree/src/tree/root.rs +++ b/crates/engine/tree/src/tree/root.rs @@ -6,20 +6,15 @@ use rayon::iter::{ParallelBridge, ParallelIterator}; use reth_errors::{ProviderError, ProviderResult}; use reth_evm::system_calls::OnStateHook; use reth_provider::{ - providers::ConsistentDbView, BlockReader, DBProvider, DatabaseProviderFactory, - StateCommitmentProvider, + providers::ConsistentDbView, BlockReader, DatabaseProviderFactory, StateCommitmentProvider, }; use reth_trie::{ - hashed_cursor::HashedPostStateCursorFactory, prefix_set::TriePrefixSetsMut, - proof::Proof, - trie_cursor::InMemoryTrieCursorFactory, updates::{TrieUpdates, TrieUpdatesSorted}, HashedPostState, HashedPostStateSorted, HashedStorage, MultiProof, MultiProofTargets, Nibbles, TrieInput, }; -use reth_trie_db::{DatabaseHashedCursorFactory, DatabaseProof, DatabaseTrieCursorFactory}; -use reth_trie_parallel::root::ParallelStateRootError; +use reth_trie_parallel::{proof::ParallelProof, root::ParallelStateRootError}; use reth_trie_sparse::{ blinded::{BlindedProvider, BlindedProviderFactory}, errors::{SparseStateTrieError, SparseStateTrieResult, SparseTrieError, SparseTrieErrorKind}, @@ -269,7 +264,7 @@ fn evm_state_to_hashed_post_state(update: EvmState) -> HashedPostState { /// to the tree. /// Then it updates relevant leaves according to the result of the transaction. #[derive(Debug)] -pub struct StateRootTask { +pub struct StateRootTask<'env, Factory, BPF: BlindedProviderFactory> { /// Task configuration. config: StateRootConfig, /// Receiver for state root related messages. @@ -283,10 +278,12 @@ pub struct StateRootTask { /// The sparse trie used for the state root calculation. If [`None`], then update is in /// progress. sparse_trie: Option>>, + /// Reference to the shared thread pool for parallel proof generation + thread_pool: &'env rayon::ThreadPool, } #[allow(dead_code)] -impl<'env, Factory, ABP, SBP, BPF> StateRootTask +impl<'env, Factory, ABP, SBP, BPF> StateRootTask<'env, Factory, BPF> where Factory: DatabaseProviderFactory + StateCommitmentProvider @@ -302,7 +299,11 @@ where + 'env, { /// Creates a new state root task with the unified message channel - pub fn new(config: StateRootConfig, blinded_provider: BPF) -> Self { + pub fn new( + config: StateRootConfig, + blinded_provider: BPF, + thread_pool: &'env rayon::ThreadPool, + ) -> Self { let (tx, rx) = channel(); Self { @@ -312,6 +313,7 @@ where fetched_proof_targets: Default::default(), proof_sequencer: ProofSequencer::new(), sparse_trie: Some(Box::new(SparseStateTrie::new(blinded_provider).with_updates(true))), + thread_pool, } } @@ -350,6 +352,7 @@ where fetched_proof_targets: &mut MultiProofTargets, proof_sequence_number: u64, state_root_message_sender: Sender>, + thread_pool: &'env rayon::ThreadPool, ) { let proof_targets = targets.into_iter().map(|address| (keccak256(address), Default::default())).collect(); @@ -362,6 +365,7 @@ where proof_targets, proof_sequence_number, state_root_message_sender, + thread_pool, ); } @@ -375,6 +379,7 @@ where fetched_proof_targets: &mut MultiProofTargets, proof_sequence_number: u64, state_root_message_sender: Sender>, + thread_pool: &'env rayon::ThreadPool, ) { let hashed_state_update = evm_state_to_hashed_post_state(update); @@ -388,6 +393,7 @@ where proof_targets, proof_sequence_number, state_root_message_sender, + thread_pool, ); } @@ -398,22 +404,27 @@ where proof_targets: MultiProofTargets, proof_sequence_number: u64, state_root_message_sender: Sender>, + thread_pool: &'env rayon::ThreadPool, ) { // Dispatch proof gathering for this state update - scope.spawn(move |_| match calculate_multiproof(config, proof_targets.clone()) { - Ok(proof) => { - let _ = state_root_message_sender.send(StateRootMessage::ProofCalculated( - Box::new(ProofCalculated { - state_update: hashed_state_update, - targets: proof_targets, - proof, - sequence_number: proof_sequence_number, - }), - )); - } - Err(error) => { - let _ = - state_root_message_sender.send(StateRootMessage::ProofCalculationError(error)); + scope.spawn(move |_| { + let result = calculate_multiproof(thread_pool, config, proof_targets.clone()); + + match result { + Ok(proof) => { + let _ = state_root_message_sender.send(StateRootMessage::ProofCalculated( + Box::new(ProofCalculated { + state_update: hashed_state_update, + targets: proof_targets, + proof, + sequence_number: proof_sequence_number, + }), + )); + } + Err(error) => { + let _ = state_root_message_sender + .send(StateRootMessage::ProofCalculationError(error)); + } } }); } @@ -517,6 +528,7 @@ where &mut self.fetched_proof_targets, self.proof_sequencer.next_sequence(), self.tx.clone(), + self.thread_pool, ); } StateRootMessage::StateUpdate(update) => { @@ -540,6 +552,7 @@ where &mut self.fetched_proof_targets, self.proof_sequencer.next_sequence(), self.tx.clone(), + self.thread_pool, ); } StateRootMessage::FinishedStateUpdates => { @@ -717,26 +730,23 @@ fn get_proof_targets( /// Calculate multiproof for the targets. #[inline] fn calculate_multiproof( + thread_pool: &rayon::ThreadPool, config: StateRootConfig, proof_targets: MultiProofTargets, ) -> ProviderResult where - Factory: DatabaseProviderFactory + StateCommitmentProvider, + Factory: + DatabaseProviderFactory + StateCommitmentProvider + Clone + 'static, { - let provider = config.consistent_view.provider_ro()?; - - Ok(Proof::from_tx(provider.tx_ref()) - .with_trie_cursor_factory(InMemoryTrieCursorFactory::new( - DatabaseTrieCursorFactory::new(provider.tx_ref()), - &config.nodes_sorted, - )) - .with_hashed_cursor_factory(HashedPostStateCursorFactory::new( - DatabaseHashedCursorFactory::new(provider.tx_ref()), - &config.state_sorted, - )) - .with_prefix_sets_mut(config.prefix_sets.as_ref().clone()) - .with_branch_node_hash_masks(true) - .multiproof(proof_targets)?) + Ok(ParallelProof::new( + config.consistent_view, + config.nodes_sorted, + config.state_sorted, + config.prefix_sets, + thread_pool, + ) + .with_branch_node_hash_masks(true) + .multiproof(proof_targets)?) } /// Updates the sparse trie with the given proofs and state, and returns the updated trie and the @@ -967,8 +977,17 @@ mod tests { ), config.prefix_sets.clone(), ); + let num_threads = + std::thread::available_parallelism().map_or(1, |num| (num.get() / 2).max(1)); + + let state_root_task_pool = rayon::ThreadPoolBuilder::new() + .num_threads(num_threads) + .thread_name(|i| format!("proof-worker-{}", i)) + .build() + .expect("Failed to create proof worker thread pool"); + let (root_from_task, _) = std::thread::scope(|std_scope| { - let task = StateRootTask::new(config, blinded_provider_factory); + let task = StateRootTask::new(config, blinded_provider_factory, &state_root_task_pool); let mut state_hook = task.state_hook(); let handle = task.spawn(std_scope); diff --git a/crates/trie/parallel/src/proof.rs b/crates/trie/parallel/src/proof.rs index 149a53a1e4b2..ef7e34b1970a 100644 --- a/crates/trie/parallel/src/proof.rs +++ b/crates/trie/parallel/src/proof.rs @@ -125,7 +125,7 @@ where ); // Pre-calculate storage roots for accounts which were changed. - tracker.set_precomputed_storage_roots(storage_root_targets.len() as u64); + tracker.set_precomputed_storage_roots(storage_root_targets_len as u64); let mut storage_proofs = B256HashMap::with_capacity_and_hasher(storage_root_targets.len(), Default::default()); From 7a3e6d83196c56e7ba0ed1568e48639d39e84f81 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 19 Dec 2024 15:16:08 +0100 Subject: [PATCH 12/30] chore: use op feature for engine local (#13454) --- crates/engine/local/Cargo.toml | 6 ++---- crates/engine/local/src/payload.rs | 2 +- crates/optimism/node/Cargo.toml | 2 +- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/crates/engine/local/Cargo.toml b/crates/engine/local/Cargo.toml index b3ad169e3189..7e29dbf9af0e 100644 --- a/crates/engine/local/Cargo.toml +++ b/crates/engine/local/Cargo.toml @@ -48,8 +48,6 @@ op-alloy-rpc-types-engine = { workspace = true, optional = true } workspace = true [features] -optimism = [ - "op-alloy-rpc-types-engine", - "reth-beacon-consensus/optimism", - "reth-provider/optimism", +op = [ + "dep:op-alloy-rpc-types-engine" ] diff --git a/crates/engine/local/src/payload.rs b/crates/engine/local/src/payload.rs index 6355a2a00af2..090b4d910b31 100644 --- a/crates/engine/local/src/payload.rs +++ b/crates/engine/local/src/payload.rs @@ -45,7 +45,7 @@ where } } -#[cfg(feature = "optimism")] +#[cfg(feature = "op")] impl PayloadAttributesBuilder for LocalPayloadAttributesBuilder where diff --git a/crates/optimism/node/Cargo.toml b/crates/optimism/node/Cargo.toml index 3c3ebb57039f..ee6c969fd390 100644 --- a/crates/optimism/node/Cargo.toml +++ b/crates/optimism/node/Cargo.toml @@ -93,7 +93,7 @@ optimism = [ "reth-beacon-consensus/optimism", "revm/optimism", "reth-optimism-rpc/optimism", - "reth-engine-local/optimism", + "reth-engine-local/op", "reth-optimism-consensus/optimism", "reth-db/optimism", "reth-optimism-node/optimism", From f3b1d0fbe7a6277e8d659110b846252ec9f8714f Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 19 Dec 2024 15:48:22 +0100 Subject: [PATCH 13/30] meta: update code owners (#13457) --- .github/CODEOWNERS | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 5a1d1df72611..6e2024754d47 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -2,7 +2,7 @@ bin/ @onbjerg crates/blockchain-tree/ @rakita @rkrasiuk @mattsse @Rjected crates/blockchain-tree-api/ @rakita @rkrasiuk @mattsse @Rjected -crates/chainspec/ @Rjected @joshieDo @mattsse @emhane +crates/chainspec/ @Rjected @joshieDo @mattsse crates/chain-state/ @fgimenez @mattsse @rkrasiuk crates/cli/ @onbjerg @mattsse crates/config/ @onbjerg @@ -16,18 +16,18 @@ crates/ethereum-forks/ @mattsse @Rjected crates/etl/ @joshieDo @shekhirin crates/evm/ @rakita @mattsse @Rjected crates/exex/ @onbjerg @shekhirin -crates/fs-util/ @onbjerg @emhane +crates/fs-util/ @onbjerg crates/metrics/ @onbjerg -crates/net/ @emhane @mattsse @Rjected -crates/net/downloaders/ @onbjerg @rkrasiuk @emhane -crates/node/ @mattsse @Rjected @onbjerg @emhane @klkvr -crates/optimism/ @mattsse @Rjected @fgimenez @emhane +crates/net/ @mattsse @Rjected +crates/net/downloaders/ @onbjerg @rkrasiuk +crates/node/ @mattsse @Rjected @onbjerg @klkvr +crates/optimism/ @mattsse @Rjected @fgimenez crates/payload/ @mattsse @Rjected -crates/primitives/ @Rjected @emhane @mattsse @klkvr -crates/primitives-traits/ @Rjected @joshieDo @emhane @mattsse @klkvr +crates/primitives/ @Rjected @mattsse @klkvr +crates/primitives-traits/ @Rjected @joshieDo @mattsse @klkvr crates/prune/ @shekhirin @joshieDo crates/revm/ @mattsse @rakita -crates/rpc/ @mattsse @Rjected @emhane +crates/rpc/ @mattsse @Rjected crates/stages/ @onbjerg @rkrasiuk @shekhirin crates/static-file/ @joshieDo @shekhirin crates/storage/codecs/ @joshieDo @@ -40,7 +40,7 @@ crates/storage/nippy-jar/ @joshieDo @shekhirin crates/storage/provider/ @rakita @joshieDo @shekhirin crates/storage/storage-api/ @joshieDo @rkrasiuk crates/tasks/ @mattsse -crates/tokio-util/ @fgimenez @emhane +crates/tokio-util/ @fgimenez crates/tracing/ @onbjerg crates/transaction-pool/ @mattsse crates/trie/ @rkrasiuk @Rjected @shekhirin From b271f0cc6dbdcbbc2e5438dda1cd5e0dedd83079 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Thu, 19 Dec 2024 16:57:51 +0200 Subject: [PATCH 14/30] chore: relax pool maintain future (#13455) --- crates/transaction-pool/src/maintain.rs | 24 +++++++----------------- 1 file changed, 7 insertions(+), 17 deletions(-) diff --git a/crates/transaction-pool/src/maintain.rs b/crates/transaction-pool/src/maintain.rs index 23bf75b01d66..dfcd22fd68bd 100644 --- a/crates/transaction-pool/src/maintain.rs +++ b/crates/transaction-pool/src/maintain.rs @@ -19,9 +19,7 @@ use reth_chain_state::CanonStateNotification; use reth_chainspec::{ChainSpecProvider, EthChainSpec}; use reth_execution_types::ChangedAccount; use reth_fs_util::FsPathError; -use reth_primitives::{ - transaction::SignedTransactionIntoRecoveredExt, SealedHeader, TransactionSigned, -}; +use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, SealedHeader}; use reth_primitives_traits::{NodePrimitives, SignedTransaction}; use reth_storage_api::{errors::provider::ProviderError, BlockReaderIdExt, StateProviderFactory}; use reth_tasks::TaskSpawner; @@ -78,13 +76,9 @@ pub fn maintain_transaction_pool_future( config: MaintainPoolConfig, ) -> BoxFuture<'static, ()> where - N: NodePrimitives< - BlockHeader = reth_primitives::Header, - BlockBody = reth_primitives::BlockBody, - SignedTx = TransactionSigned, - >, + N: NodePrimitives, Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + 'static, - P: TransactionPoolExt> + 'static, + P: TransactionPoolExt> + 'static, St: Stream> + Send + Unpin + 'static, Tasks: TaskSpawner + 'static, { @@ -104,13 +98,9 @@ pub async fn maintain_transaction_pool( task_spawner: Tasks, config: MaintainPoolConfig, ) where - N: NodePrimitives< - BlockHeader = reth_primitives::Header, - BlockBody = reth_primitives::BlockBody, - SignedTx = TransactionSigned, - >, + N: NodePrimitives, Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + 'static, - P: TransactionPoolExt> + 'static, + P: TransactionPoolExt> + 'static, St: Stream> + Send + Unpin + 'static, Tasks: TaskSpawner + 'static, { @@ -339,7 +329,7 @@ pub async fn maintain_transaction_pool( // been validated previously, we still need the blob in order to // accurately set the transaction's // encoded-length which is propagated over the network. - pool.get_blob(TransactionSigned::hash(&tx)) + pool.get_blob(*tx.tx_hash()) .ok() .flatten() .map(Arc::unwrap_or_clone) @@ -680,7 +670,7 @@ mod tests { use alloy_primitives::{hex, U256}; use reth_chainspec::MAINNET; use reth_fs_util as fs; - use reth_primitives::PooledTransaction; + use reth_primitives::{PooledTransaction, TransactionSigned}; use reth_provider::test_utils::{ExtendedAccount, MockEthProvider}; use reth_tasks::TaskManager; From 058cfe2f98952633da53f877b538a1ea3f9d657f Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Thu, 19 Dec 2024 16:58:07 +0200 Subject: [PATCH 15/30] chore: relax PayloadBuilderService bound (#13456) --- Cargo.lock | 1 + crates/payload/basic/src/lib.rs | 4 ++-- crates/payload/builder/Cargo.toml | 10 ++++++---- crates/payload/builder/src/service.rs | 6 ++++-- crates/payload/builder/src/traits.rs | 3 ++- 5 files changed, 15 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 299f84e8784f..35f1a2ed0853 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8686,6 +8686,7 @@ dependencies = [ "reth-payload-builder-primitives", "reth-payload-primitives", "reth-primitives", + "reth-primitives-traits", "revm", "tokio", "tokio-stream", diff --git a/crates/payload/basic/src/lib.rs b/crates/payload/basic/src/lib.rs index 43292385661c..1b8bc6ba7c0b 100644 --- a/crates/payload/basic/src/lib.rs +++ b/crates/payload/basic/src/lib.rs @@ -19,7 +19,7 @@ use reth_evm::state_change::post_block_withdrawals_balance_increments; use reth_payload_builder::{KeepPayloadJobAlive, PayloadId, PayloadJob, PayloadJobGenerator}; use reth_payload_builder_primitives::PayloadBuilderError; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes, PayloadKind}; -use reth_primitives::{proofs, SealedHeader}; +use reth_primitives::{proofs, NodePrimitives, SealedHeader}; use reth_provider::{BlockReaderIdExt, CanonStateNotification, StateProviderFactory}; use reth_revm::cached::CachedReads; use reth_tasks::TaskSpawner; @@ -191,7 +191,7 @@ where Ok(job) } - fn on_new_state(&mut self, new_state: CanonStateNotification) { + fn on_new_state(&mut self, new_state: CanonStateNotification) { let mut cached = CachedReads::default(); // extract the state from the notification and put it into the cache diff --git a/crates/payload/builder/Cargo.toml b/crates/payload/builder/Cargo.toml index 78814da50664..d709c1208e2b 100644 --- a/crates/payload/builder/Cargo.toml +++ b/crates/payload/builder/Cargo.toml @@ -14,6 +14,7 @@ workspace = true [dependencies] # reth reth-primitives = { workspace = true, optional = true } +reth-primitives-traits.workspace = true reth-chain-state.workspace = true reth-payload-builder-primitives.workspace = true reth-payload-primitives.workspace = true @@ -44,8 +45,9 @@ alloy-consensus.workspace = true [features] test-utils = [ - "alloy-primitives", - "reth-chain-state/test-utils", - "reth-primitives/test-utils", - "revm/test-utils", + "alloy-primitives", + "reth-chain-state/test-utils", + "reth-primitives/test-utils", + "revm/test-utils", + "reth-primitives-traits/test-utils" ] diff --git a/crates/payload/builder/src/service.rs b/crates/payload/builder/src/service.rs index af11ba75ce6a..aaff23dda82d 100644 --- a/crates/payload/builder/src/service.rs +++ b/crates/payload/builder/src/service.rs @@ -14,6 +14,7 @@ use reth_payload_builder_primitives::{ Events, PayloadBuilder, PayloadBuilderError, PayloadEvents, PayloadStoreExt, }; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes, PayloadKind, PayloadTypes}; +use reth_primitives_traits::NodePrimitives; use std::{ fmt, future::Future, @@ -352,12 +353,13 @@ where } } -impl Future for PayloadBuilderService +impl Future for PayloadBuilderService where T: PayloadTypes, + N: NodePrimitives, Gen: PayloadJobGenerator + Unpin + 'static, ::Job: Unpin + 'static, - St: Stream + Send + Unpin + 'static, + St: Stream> + Send + Unpin + 'static, Gen::Job: PayloadJob, ::BuiltPayload: Into, { diff --git a/crates/payload/builder/src/traits.rs b/crates/payload/builder/src/traits.rs index d9d54ccd0e45..34a756e6059b 100644 --- a/crates/payload/builder/src/traits.rs +++ b/crates/payload/builder/src/traits.rs @@ -3,6 +3,7 @@ use reth_chain_state::CanonStateNotification; use reth_payload_builder_primitives::PayloadBuilderError; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes, PayloadKind}; +use reth_primitives_traits::NodePrimitives; use std::future::Future; /// A type that can build a payload. @@ -105,7 +106,7 @@ pub trait PayloadJobGenerator: Send + Sync { /// /// This is intended for any logic that needs to be run when the chain state changes or used to /// use the in memory state for the head block. - fn on_new_state(&mut self, new_state: CanonStateNotification) { + fn on_new_state(&mut self, new_state: CanonStateNotification) { let _ = new_state; } } From 431c993cee8c36f91c32878e1208ac12322e59ee Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 08:56:01 +0100 Subject: [PATCH 16/30] chore: bump alloy 0.8.2 (#13459) --- Cargo.lock | 108 ++++++++++++++--------------- Cargo.toml | 56 +++++++-------- crates/net/network-api/src/noop.rs | 3 +- crates/net/network/src/manager.rs | 3 +- crates/rpc/rpc/src/admin.rs | 3 +- 5 files changed, 88 insertions(+), 85 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 35f1a2ed0853..087ecef86878 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -121,9 +121,9 @@ dependencies = [ [[package]] name = "alloy-consensus" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a205d0cbb7bfdf9f4fd4b0ec842bc4c5f926e8c14ec3072d3fd75dd363baf1e0" +checksum = "bf9f8fb3895d5a526b6b8cae7e1bba96ba350545852f0b0ab51041136785ac95" dependencies = [ "alloy-eips", "alloy-primitives", @@ -155,9 +155,9 @@ dependencies = [ [[package]] name = "alloy-contract" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aec7945dff98ba68489aa6da455bf66f6c0fee8157df06747fbae7cb03c368e2" +checksum = "5a1240738b2c2d5a45153722fc50f28cd6e3129283b5f3e039031577c71eb70d" dependencies = [ "alloy-dyn-abi", "alloy-json-abi", @@ -222,9 +222,9 @@ dependencies = [ [[package]] name = "alloy-eips" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1d9907c29ce622946759bf4fd3418166bfeae76c1c544b8081c7be3acd9b4be" +checksum = "5061487522a347cc84b282ce507dfca70a34d0ef173c1af643b47f77897e38c9" dependencies = [ "alloy-eip2930", "alloy-eip7702", @@ -243,9 +243,9 @@ dependencies = [ [[package]] name = "alloy-genesis" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68f13f7405a8eb8021258994ed1beab490c3e509ebbe2c18e1c24ae10749d56b" +checksum = "62acff796b1cfc301f09eba904b8299b2e243069ddf33584941843e3a1c5a6a5" dependencies = [ "alloy-primitives", "alloy-serde", @@ -267,9 +267,9 @@ dependencies = [ [[package]] name = "alloy-json-rpc" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39a786ce6bc7539dc30cabac6b7875644247c9e7d780e71a9f254d42ebdc013c" +checksum = "73c81f9b96b5ed5c32508ca494d74cdc693e534535b0cf9bf2a5b94a2d3d3a27" dependencies = [ "alloy-primitives", "alloy-sol-types", @@ -281,9 +281,9 @@ dependencies = [ [[package]] name = "alloy-network" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99051f82f77159d5bee06108f33cffee02849e2861fc500bf74213aa2ae8a26e" +checksum = "01e53bad8c26999c2577e395e5cea1cee39966247218206ecc694544f0853993" dependencies = [ "alloy-consensus", "alloy-consensus-any", @@ -319,9 +319,9 @@ dependencies = [ [[package]] name = "alloy-node-bindings" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb130be1b7cfca7355710808392a793768bd055e5a28e1fed9d03ec7fe8fde2c" +checksum = "7b19414ac7f0cc153fc000a540babd450fe7cbd3ccb4b5ddb62b4fa5a72126fb" dependencies = [ "alloy-genesis", "alloy-primitives", @@ -368,9 +368,9 @@ dependencies = [ [[package]] name = "alloy-provider" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0280a4f68e0cefde9449ee989a248230efbe3f95255299d2a7a92009e154629d" +checksum = "8a9cda73601658d30295f1a4e93614166da9ad925f26196157f48e0e225ef70a" dependencies = [ "alloy-chains", "alloy-consensus", @@ -409,9 +409,9 @@ dependencies = [ [[package]] name = "alloy-pubsub" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9475dc1a835bd8bb77275b6bccf8e177e7e669ba81277ce6bea0016ce994fafe" +checksum = "2a83226edba216f2464b1bd48a06dd8e8d836427883568ec63ecda65d770f830" dependencies = [ "alloy-json-rpc", "alloy-primitives", @@ -450,9 +450,9 @@ dependencies = [ [[package]] name = "alloy-rpc-client" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6fc8b0f68619cfab3a2e15dca7b80ab266f78430bb4353dec546528e04b7449" +checksum = "318806a3a1f5e8c2f110de2bdf5aa14faf9f2f19a4f24603a8cb968b35319a6c" dependencies = [ "alloy-json-rpc", "alloy-primitives", @@ -475,9 +475,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "986f23fe42ac95832901a24b93c20f7ed2b9644394c02b86222801230da60041" +checksum = "a97b37bdea65460684bcfdbd6c3bac43dec7316fa634da2d96f16df9b970482f" dependencies = [ "alloy-primitives", "alloy-rpc-types-engine", @@ -488,9 +488,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-admin" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c739a734da7f41054aeff8abddbf66ae44f2e624ce7a7ccd9bd84c76f7c24902" +checksum = "972a7682802bb02ce42a5b32a5b5d6952717159c0007c20834c3e40aebc94349" dependencies = [ "alloy-genesis", "alloy-primitives", @@ -500,9 +500,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-anvil" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83ac5e71dd1a25029ec565ea34aaf95515f4168192c2843efe198fa490d58dd7" +checksum = "fa29f70eb194f0e61df4ac7d6c5226ed5588c386f0ebdcf7c43d7623e05d755a" dependencies = [ "alloy-primitives", "alloy-rpc-types-eth", @@ -523,9 +523,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-beacon" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4612f586da13ac81c75bbbd04f6371bb34d47f0650920fca68636a0b9177bc4" +checksum = "5c9d4cdff2aef8bc2a2b6b172a87bd205ed293ad23aa21d86a2eb82dbfc88231" dependencies = [ "alloy-eips", "alloy-primitives", @@ -539,9 +539,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-debug" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cda556c1c69af9222dec46dd3496b622e9a249ef19a7bd0e80e5995a4c81b3" +checksum = "893786e4bd2acb2884fc5a708a64e9a64f1e233c40cc2f8583219f6f5ff67f6c" dependencies = [ "alloy-primitives", "serde", @@ -549,9 +549,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-engine" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30814f8b9ac10219fb77fe42c277a0ffa1c369fbc3961f14d159f51fb221966e" +checksum = "90ebd3bd158875c60bef2167e858ef004455f17ee19f19670149b4fc6567fb7f" dependencies = [ "alloy-consensus", "alloy-eips", @@ -570,9 +570,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-eth" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0643cc497a71941f526454fe4fecb47e9307d3a7b6c05f70718a0341643bcc79" +checksum = "43495c55aeaac24faa4f8b315ea8b7ee6e49985bd1edfcb1e156177de59b55a6" dependencies = [ "alloy-consensus", "alloy-consensus-any", @@ -592,9 +592,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-mev" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "268058db229a3b3beb8185dac46cd2149efe54afa9d788d5e995008099811cc2" +checksum = "a3cd734a907a7d6c26a2f054a13ee194212143b50ffc34d4d3505d38b05fce88" dependencies = [ "alloy-eips", "alloy-primitives", @@ -606,9 +606,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-trace" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc57a00e8de2f8e45f988fdfd1e9d08784f563553d7717924170f324a1ab7459" +checksum = "3d1dcc3dae728f6b576ca96b055f3ee91e1e9f88a2d36bc8d578fafef803c356" dependencies = [ "alloy-primitives", "alloy-rpc-types-eth", @@ -620,9 +620,9 @@ dependencies = [ [[package]] name = "alloy-rpc-types-txpool" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a6c604b8ce6393e75726c15eb2306fe004bca4da07383548158626fe7636ae8" +checksum = "da98336d7b576d85df704c678043bf21d65f8e15512f217838584257c7354a21" dependencies = [ "alloy-primitives", "alloy-rpc-types-eth", @@ -632,9 +632,9 @@ dependencies = [ [[package]] name = "alloy-serde" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea61b049d7ecc66a29f107970dae493d0908e366048f7484a1ca9b02c85f9b2b" +checksum = "514e925398442f5ac13d33ba98e26bd99925131edd4928a9a15658ec67f2bf92" dependencies = [ "alloy-primitives", "arbitrary", @@ -644,9 +644,9 @@ dependencies = [ [[package]] name = "alloy-signer" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93461b0e79c2ddd791fec5f369ab5c2686a33bbb03530144972edf5248f8a2c7" +checksum = "39d611775721dfe9f993d94308ac6c84ea5b1880e4c91e8b38c86d3e610ae41c" dependencies = [ "alloy-primitives", "async-trait", @@ -658,9 +658,9 @@ dependencies = [ [[package]] name = "alloy-signer-local" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f08ec1bfa433f9e9f7c5af05af07e5cf86d27d93170de76b760e63b925f1c9c" +checksum = "bccfe7997ffd0a721995c61cda28b842d0fb5231ddc51b8a4c1ee1c9a9bbb6ed" dependencies = [ "alloy-consensus", "alloy-network", @@ -746,9 +746,9 @@ dependencies = [ [[package]] name = "alloy-transport" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf656f983e14812df65b5aee37e7b37535f68a848295e6ed736b2054a405cb7" +checksum = "3df16f5842a8484d8523297a5c1d574ec0f43500b1baa0566ff1ea4190883c5c" dependencies = [ "alloy-json-rpc", "base64 0.22.1", @@ -766,9 +766,9 @@ dependencies = [ [[package]] name = "alloy-transport-http" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec938d51a47b7953b1c0fd8ddeb89a29eb113cd4908dfc4e01c7893b252d669f" +checksum = "b7c4c662b3f9d5d4470ea7f0d60ad00c9332315001d0acbe91f6b0ca0df535f4" dependencies = [ "alloy-json-rpc", "alloy-transport", @@ -781,9 +781,9 @@ dependencies = [ [[package]] name = "alloy-transport-ipc" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9df0d2e1b24dd029641bd21ef783491c42af87b162968be94f0443c1eb72c8e0" +checksum = "60c8d8a4c922358c27da4cc639cb8bba527376e1e268beac230751aec80231b9" dependencies = [ "alloy-json-rpc", "alloy-pubsub", @@ -800,9 +800,9 @@ dependencies = [ [[package]] name = "alloy-transport-ws" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fabdf2d18c0c87b6cfcf6a067f1d5a7db378f103faeb16130d6d174c73d006b" +checksum = "7dff9302c8430fe84e88c445f243f99d7f189e4c8a78ab8a56312bfa5ad0801f" dependencies = [ "alloy-pubsub", "alloy-transport", diff --git a/Cargo.toml b/Cargo.toml index 9f2f3500b82d..4abed9a85f51 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -441,40 +441,40 @@ alloy-rlp = { version = "0.3.10", default-features = false } alloy-sol-types = "0.8.15" alloy-trie = { version = "0.7", default-features = false } -alloy-consensus = { version = "0.8.1", default-features = false } -alloy-contract = { version = "0.8.1", default-features = false } -alloy-eips = { version = "0.8.1", default-features = false } -alloy-genesis = { version = "0.8.1", default-features = false } -alloy-json-rpc = { version = "0.8.1", default-features = false } -alloy-network = { version = "0.8.1", default-features = false } -alloy-network-primitives = { version = "0.8.1", default-features = false } -alloy-node-bindings = { version = "0.8.1", default-features = false } -alloy-provider = { version = "0.8.1", features = [ +alloy-consensus = { version = "0.8.2", default-features = false } +alloy-contract = { version = "0.8.2", default-features = false } +alloy-eips = { version = "0.8.2", default-features = false } +alloy-genesis = { version = "0.8.2", default-features = false } +alloy-json-rpc = { version = "0.8.2", default-features = false } +alloy-network = { version = "0.8.2", default-features = false } +alloy-network-primitives = { version = "0.8.2", default-features = false } +alloy-node-bindings = { version = "0.8.2", default-features = false } +alloy-provider = { version = "0.8.2", features = [ "reqwest", ], default-features = false } -alloy-pubsub = { version = "0.8.1", default-features = false } -alloy-rpc-client = { version = "0.8.1", default-features = false } -alloy-rpc-types = { version = "0.8.1", features = [ +alloy-pubsub = { version = "0.8.2", default-features = false } +alloy-rpc-client = { version = "0.8.2", default-features = false } +alloy-rpc-types = { version = "0.8.2", features = [ "eth", ], default-features = false } -alloy-rpc-types-admin = { version = "0.8.1", default-features = false } -alloy-rpc-types-anvil = { version = "0.8.1", default-features = false } -alloy-rpc-types-beacon = { version = "0.8.1", default-features = false } -alloy-rpc-types-debug = { version = "0.8.1", default-features = false } -alloy-rpc-types-engine = { version = "0.8.1", default-features = false } -alloy-rpc-types-eth = { version = "0.8.1", default-features = false } -alloy-rpc-types-mev = { version = "0.8.1", default-features = false } -alloy-rpc-types-trace = { version = "0.8.1", default-features = false } -alloy-rpc-types-txpool = { version = "0.8.1", default-features = false } -alloy-serde = { version = "0.8.1", default-features = false } -alloy-signer = { version = "0.8.1", default-features = false } -alloy-signer-local = { version = "0.8.1", default-features = false } -alloy-transport = { version = "0.8.1" } -alloy-transport-http = { version = "0.8.1", features = [ +alloy-rpc-types-admin = { version = "0.8.2", default-features = false } +alloy-rpc-types-anvil = { version = "0.8.2", default-features = false } +alloy-rpc-types-beacon = { version = "0.8.2", default-features = false } +alloy-rpc-types-debug = { version = "0.8.2", default-features = false } +alloy-rpc-types-engine = { version = "0.8.2", default-features = false } +alloy-rpc-types-eth = { version = "0.8.2", default-features = false } +alloy-rpc-types-mev = { version = "0.8.2", default-features = false } +alloy-rpc-types-trace = { version = "0.8.2", default-features = false } +alloy-rpc-types-txpool = { version = "0.8.2", default-features = false } +alloy-serde = { version = "0.8.2", default-features = false } +alloy-signer = { version = "0.8.2", default-features = false } +alloy-signer-local = { version = "0.8.2", default-features = false } +alloy-transport = { version = "0.8.2" } +alloy-transport-http = { version = "0.8.2", features = [ "reqwest-rustls-tls", ], default-features = false } -alloy-transport-ipc = { version = "0.8.1", default-features = false } -alloy-transport-ws = { version = "0.8.1", default-features = false } +alloy-transport-ipc = { version = "0.8.2", default-features = false } +alloy-transport-ws = { version = "0.8.2", default-features = false } # op op-alloy-rpc-types = "0.8.4" diff --git a/crates/net/network-api/src/noop.rs b/crates/net/network-api/src/noop.rs index 6036b7173dda..e21df81305dc 100644 --- a/crates/net/network-api/src/noop.rs +++ b/crates/net/network-api/src/noop.rs @@ -26,12 +26,13 @@ impl NetworkInfo for NoopNetwork { } async fn network_status(&self) -> Result { + #[allow(deprecated)] Ok(NetworkStatus { client_version: "reth-test".to_string(), protocol_version: ProtocolVersion::V5 as u64, eth_protocol_info: EthProtocolInfo { - difficulty: Default::default(), network: 1, + difficulty: None, genesis: Default::default(), config: Default::default(), head: Default::default(), diff --git a/crates/net/network/src/manager.rs b/crates/net/network/src/manager.rs index 54018676d102..c63f5025b560 100644 --- a/crates/net/network/src/manager.rs +++ b/crates/net/network/src/manager.rs @@ -409,11 +409,12 @@ impl NetworkManager { let status = sessions.status(); let hello_message = sessions.hello_message(); + #[allow(deprecated)] NetworkStatus { client_version: hello_message.client_version, protocol_version: hello_message.protocol_version as u64, eth_protocol_info: EthProtocolInfo { - difficulty: status.total_difficulty, + difficulty: None, head: status.blockhash, network: status.chain.id(), genesis: status.genesis, diff --git a/crates/rpc/rpc/src/admin.rs b/crates/rpc/rpc/src/admin.rs index 0358aa3a8d43..75cfece9d564 100644 --- a/crates/rpc/rpc/src/admin.rs +++ b/crates/rpc/rpc/src/admin.rs @@ -166,13 +166,14 @@ where ip: enode.address, ports: Ports { discovery: enode.udp_port, listener: enode.tcp_port }, listen_addr: enode.tcp_addr(), + #[allow(deprecated)] protocols: ProtocolInfo { eth: Some(EthProtocolInfo { network: status.eth_protocol_info.network, - difficulty: status.eth_protocol_info.difficulty, genesis: status.eth_protocol_info.genesis, config, head: status.eth_protocol_info.head, + difficulty: None, }), snap: None, }, From 1009607c4910598ab304739327bd05dfcbdce843 Mon Sep 17 00:00:00 2001 From: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com> Date: Fri, 20 Dec 2024 10:21:49 +0200 Subject: [PATCH 17/30] fix(tree): destroyed accounts should not have storage in state root task (#13460) Co-authored-by: Federico Gimenez --- crates/engine/tree/src/tree/root.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/crates/engine/tree/src/tree/root.rs b/crates/engine/tree/src/tree/root.rs index dd931a8e40b8..6ebb509da85b 100644 --- a/crates/engine/tree/src/tree/root.rs +++ b/crates/engine/tree/src/tree/root.rs @@ -243,11 +243,12 @@ fn evm_state_to_hashed_post_state(update: EvmState) -> HashedPostState { }) .peekable(); - if destroyed || changed_storage_iter.peek().is_some() { - hashed_state.storages.insert( - hashed_address, - HashedStorage::from_iter(destroyed, changed_storage_iter), - ); + if destroyed { + hashed_state.storages.insert(hashed_address, HashedStorage::new(true)); + } else if changed_storage_iter.peek().is_some() { + hashed_state + .storages + .insert(hashed_address, HashedStorage::from_iter(false, changed_storage_iter)); } } } From 04af67879c6dac9a9984dedc0e512d16b8eebb1f Mon Sep 17 00:00:00 2001 From: Kero Date: Fri, 20 Dec 2024 16:30:41 +0800 Subject: [PATCH 18/30] chore: assert etl_config.file_size != 0 (#13435) --- crates/storage/db-common/src/init.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/storage/db-common/src/init.rs b/crates/storage/db-common/src/init.rs index 95b2a5d5c4a9..1c5ce30ce97f 100644 --- a/crates/storage/db-common/src/init.rs +++ b/crates/storage/db-common/src/init.rs @@ -373,6 +373,10 @@ where + StateWriter + AsRef, { + if etl_config.file_size == 0 { + return Err(eyre::eyre!("ETL file size cannot be zero")) + } + let block = provider_rw.last_block_number()?; let hash = provider_rw.block_hash(block)?.unwrap(); let expected_state_root = provider_rw From 2183e4955264123b19096a56edcf599bb5bde30e Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 09:42:54 +0100 Subject: [PATCH 19/30] fix: use correct merge activation block for sepolia (#13466) --- crates/ethereum-forks/src/hardfork/ethereum.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/ethereum-forks/src/hardfork/ethereum.rs b/crates/ethereum-forks/src/hardfork/ethereum.rs index 4e13b001786c..c728f6ed6634 100644 --- a/crates/ethereum-forks/src/hardfork/ethereum.rs +++ b/crates/ethereum-forks/src/hardfork/ethereum.rs @@ -96,7 +96,7 @@ impl EthereumHardfork { /// Retrieves the activation block for the specified hardfork on the Sepolia testnet. pub const fn sepolia_activation_block(&self) -> Option { match self { - Self::Paris => Some(1735371), + Self::Paris => Some(1450409), Self::Shanghai => Some(2990908), Self::Cancun => Some(5187023), Self::Frontier | From 3c4a0dcfbd11b7f843350a5b1f1a20160bf6d6ba Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 09:43:41 +0100 Subject: [PATCH 20/30] chore: fix reth-primitives traits test build (#13467) --- crates/primitives-traits/Cargo.toml | 10 +++++++--- crates/primitives-traits/src/account.rs | 2 +- crates/primitives-traits/src/crypto.rs | 10 +++++----- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/crates/primitives-traits/Cargo.toml b/crates/primitives-traits/Cargo.toml index 296597d78284..995da07f934c 100644 --- a/crates/primitives-traits/Cargo.toml +++ b/crates/primitives-traits/Cargo.toml @@ -51,22 +51,26 @@ proptest = { workspace = true, optional = true } proptest-arbitrary-interop = { workspace = true, optional = true } [dev-dependencies] -alloy-primitives = { workspace = true, features = ["arbitrary"] } -alloy-consensus = { workspace = true, features = ["arbitrary"] } +reth-codecs.workspace = true +alloy-primitives = { workspace = true, features = ["arbitrary", "serde"] } +alloy-consensus = { workspace = true, features = ["arbitrary", "serde"] } + +arbitrary = { workspace = true, features = ["derive"] } secp256k1 = { workspace = true, features = [ "recovery", "global-context", "rand" ] } bincode.workspace = true +byteorder.workspace = true proptest-arbitrary-interop.workspace = true proptest.workspace = true rand.workspace = true +serde.workspace = true serde_json.workspace = true test-fuzz.workspace = true modular-bitfield.workspace = true -serde.workspace = true [features] default = ["std"] diff --git a/crates/primitives-traits/src/account.rs b/crates/primitives-traits/src/account.rs index 7ea618315fd9..64ce209e7f5e 100644 --- a/crates/primitives-traits/src/account.rs +++ b/crates/primitives-traits/src/account.rs @@ -25,7 +25,7 @@ pub mod compact_ids { } /// An Ethereum account. -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +#[cfg_attr(any(test, feature = "serde"), derive(serde::Serialize, serde::Deserialize))] #[derive(Clone, Copy, Debug, PartialEq, Eq, Default)] #[cfg_attr(any(test, feature = "arbitrary"), derive(arbitrary::Arbitrary))] #[cfg_attr(any(test, feature = "reth-codec"), derive(reth_codecs::Compact))] diff --git a/crates/primitives-traits/src/crypto.rs b/crates/primitives-traits/src/crypto.rs index aba6107272e3..99a6521cd3c5 100644 --- a/crates/primitives-traits/src/crypto.rs +++ b/crates/primitives-traits/src/crypto.rs @@ -56,8 +56,8 @@ pub mod secp256k1 { } } -#[cfg(feature = "secp256k1")] -#[allow(unused)] +#[cfg(any(test, feature = "secp256k1"))] +#[allow(unused, unreachable_pub)] mod impl_secp256k1 { use super::*; pub(crate) use ::secp256k1::Error; @@ -196,9 +196,9 @@ mod tests { sign_message(B256::from_slice(&secret.to_bytes()[..]), hash).expect("sign message"); let mut sig: [u8; 65] = [0; 65]; - sig[0..32].copy_from_slice(&signature.r.to_be_bytes::<32>()); - sig[32..64].copy_from_slice(&signature.s.to_be_bytes::<32>()); - sig[64] = signature.odd_y_parity as u8; + sig[0..32].copy_from_slice(&signature.r().to_be_bytes::<32>()); + sig[32..64].copy_from_slice(&signature.s().to_be_bytes::<32>()); + sig[64] = signature.v() as u8; assert_eq!(recover_signer_unchecked(&sig, &hash).ok(), Some(signer)); } From 37ef2010f61b5c8fe650601a965c737ee5210fa1 Mon Sep 17 00:00:00 2001 From: Federico Gimenez Date: Fri, 20 Dec 2024 11:22:48 +0200 Subject: [PATCH 21/30] perf(engine): include selfdestructs in StateRootTask bench input (#13293) Co-authored-by: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com> --- crates/engine/tree/benches/state_root_task.rs | 168 ++++++++++++------ 1 file changed, 117 insertions(+), 51 deletions(-) diff --git a/crates/engine/tree/benches/state_root_task.rs b/crates/engine/tree/benches/state_root_task.rs index 93f1ff4ec330..7a84a4c62d67 100644 --- a/crates/engine/tree/benches/state_root_task.rs +++ b/crates/engine/tree/benches/state_root_task.rs @@ -10,7 +10,7 @@ use reth_primitives::{Account as RethAccount, StorageEntry}; use reth_provider::{ providers::ConsistentDbView, test_utils::{create_test_provider_factory, MockNodeTypesWithDB}, - HashingWriter, ProviderFactory, + AccountReader, HashingWriter, ProviderFactory, }; use reth_testing_utils::generators::{self, Rng}; use reth_trie::{ @@ -28,38 +28,54 @@ struct BenchParams { num_accounts: usize, updates_per_account: usize, storage_slots_per_account: usize, + selfdestructs_per_update: usize, } +/// Generates a series of random state updates with configurable accounts, +/// storage, and self-destructs fn create_bench_state_updates(params: &BenchParams) -> Vec { let mut rng = generators::rng(); let all_addresses: Vec

= (0..params.num_accounts).map(|_| rng.gen()).collect(); let mut updates = Vec::new(); for _ in 0..params.updates_per_account { - let num_accounts_in_update = rng.gen_range(1..=params.num_accounts); let mut state_update = EvmState::default(); + let num_accounts_in_update = rng.gen_range(1..=params.num_accounts); - let selected_addresses = &all_addresses[0..num_accounts_in_update]; - - for &address in selected_addresses { - let mut storage = HashMap::default(); - for _ in 0..params.storage_slots_per_account { - let slot = U256::from(rng.gen::()); - storage.insert( - slot, - EvmStorageSlot::new_changed(U256::ZERO, U256::from(rng.gen::())), - ); - } + // regular updates for randomly selected accounts + for &address in &all_addresses[0..num_accounts_in_update] { + // randomly choose to self-destruct with probability + // (selfdestructs/accounts) + let is_selfdestruct = + rng.gen_bool(params.selfdestructs_per_update as f64 / params.num_accounts as f64); - let account = RevmAccount { - info: AccountInfo { - balance: U256::from(rng.gen::()), - nonce: rng.gen::(), - code_hash: KECCAK_EMPTY, - code: Some(Default::default()), - }, - storage, - status: AccountStatus::Touched, + let account = if is_selfdestruct { + RevmAccount { + info: AccountInfo::default(), + storage: HashMap::default(), + status: AccountStatus::SelfDestructed, + } + } else { + RevmAccount { + info: AccountInfo { + balance: U256::from(rng.gen::()), + nonce: rng.gen::(), + code_hash: KECCAK_EMPTY, + code: Some(Default::default()), + }, + storage: (0..rng.gen_range(0..=params.storage_slots_per_account)) + .map(|_| { + ( + U256::from(rng.gen::()), + EvmStorageSlot::new_changed( + U256::ZERO, + U256::from(rng.gen::()), + ), + ) + }) + .collect(), + status: AccountStatus::Touched, + } }; state_update.insert(address, account); @@ -71,50 +87,99 @@ fn create_bench_state_updates(params: &BenchParams) -> Vec { updates } -fn convert_revm_to_reth_account(revm_account: &RevmAccount) -> RethAccount { - RethAccount { - balance: revm_account.info.balance, - nonce: revm_account.info.nonce, - bytecode_hash: if revm_account.info.code_hash == KECCAK_EMPTY { - None - } else { - Some(revm_account.info.code_hash) - }, +fn convert_revm_to_reth_account(revm_account: &RevmAccount) -> Option { + match revm_account.status { + AccountStatus::SelfDestructed => None, + _ => Some(RethAccount { + balance: revm_account.info.balance, + nonce: revm_account.info.nonce, + bytecode_hash: if revm_account.info.code_hash == KECCAK_EMPTY { + None + } else { + Some(revm_account.info.code_hash) + }, + }), } } +/// Applies state updates to the provider, ensuring self-destructs only affect +/// existing accounts fn setup_provider( factory: &ProviderFactory, state_updates: &[EvmState], ) -> Result<(), Box> { - let provider_rw = factory.provider_rw()?; - for update in state_updates { - let account_updates = update - .iter() - .map(|(address, account)| (*address, Some(convert_revm_to_reth_account(account)))); - provider_rw.insert_account_for_hashing(account_updates)?; - - let storage_updates = update.iter().map(|(address, account)| { - let storage_entries = account.storage.iter().map(|(slot, value)| StorageEntry { - key: B256::from(*slot), - value: value.present_value, - }); - (*address, storage_entries) - }); - provider_rw.insert_storage_for_hashing(storage_updates)?; + let provider_rw = factory.provider_rw()?; + + let mut account_updates = Vec::new(); + + for (address, account) in update { + // only process self-destructs if account exists, always process + // other updates + let should_process = match account.status { + AccountStatus::SelfDestructed => { + provider_rw.basic_account(*address).ok().flatten().is_some() + } + _ => true, + }; + + if should_process { + account_updates.push(( + *address, + convert_revm_to_reth_account(account), + (account.status == AccountStatus::Touched).then(|| { + account + .storage + .iter() + .map(|(slot, value)| StorageEntry { + key: B256::from(*slot), + value: value.present_value, + }) + .collect::>() + }), + )); + } + } + + // update in the provider account and its storage (if available) + for (address, account, maybe_storage) in account_updates { + provider_rw.insert_account_for_hashing(std::iter::once((address, account)))?; + if let Some(storage) = maybe_storage { + provider_rw + .insert_storage_for_hashing(std::iter::once((address, storage.into_iter())))?; + } + } + + provider_rw.commit()?; } - provider_rw.commit()?; Ok(()) } fn bench_state_root(c: &mut Criterion) { + reth_tracing::init_test_tracing(); + let mut group = c.benchmark_group("state_root"); let scenarios = vec![ - BenchParams { num_accounts: 100, updates_per_account: 5, storage_slots_per_account: 10 }, - BenchParams { num_accounts: 1000, updates_per_account: 10, storage_slots_per_account: 20 }, + BenchParams { + num_accounts: 100, + updates_per_account: 5, + storage_slots_per_account: 10, + selfdestructs_per_update: 2, + }, + BenchParams { + num_accounts: 1000, + updates_per_account: 10, + storage_slots_per_account: 20, + selfdestructs_per_update: 5, + }, + BenchParams { + num_accounts: 500, + updates_per_account: 8, + storage_slots_per_account: 15, + selfdestructs_per_update: 20, + }, ]; for params in scenarios { @@ -122,10 +187,11 @@ fn bench_state_root(c: &mut Criterion) { BenchmarkId::new( "state_root_task", format!( - "accounts_{}_updates_{}_slots_{}", + "accounts_{}_updates_{}_slots_{}_selfdestructs_{}", params.num_accounts, params.updates_per_account, - params.storage_slots_per_account + params.storage_slots_per_account, + params.selfdestructs_per_update ), ), ¶ms, From 378d80a22a46084b192d4497f937d9c398d2962d Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 11:44:59 +0100 Subject: [PATCH 22/30] chore: sprinkle a few track_caller (#13471) --- crates/storage/db/src/lib.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/storage/db/src/lib.rs b/crates/storage/db/src/lib.rs index 7090b4262fd7..113e8661f83d 100644 --- a/crates/storage/db/src/lib.rs +++ b/crates/storage/db/src/lib.rs @@ -162,6 +162,7 @@ pub mod test_utils { } /// Create `static_files` path for testing + #[track_caller] pub fn create_test_static_files_dir() -> (TempDir, PathBuf) { let temp_dir = TempDir::with_prefix("reth-test-static-").expect(ERROR_TEMPDIR); let path = temp_dir.path().to_path_buf(); @@ -175,6 +176,7 @@ pub mod test_utils { } /// Create read/write database for testing + #[track_caller] pub fn create_test_rw_db() -> Arc> { let path = tempdir_path(); let emsg = format!("{ERROR_DB_CREATION}: {path:?}"); @@ -190,6 +192,7 @@ pub mod test_utils { } /// Create read/write database for testing + #[track_caller] pub fn create_test_rw_db_with_path>(path: P) -> Arc> { let path = path.as_ref().to_path_buf(); let db = init_db( @@ -202,6 +205,7 @@ pub mod test_utils { } /// Create read only database for testing + #[track_caller] pub fn create_test_ro_db() -> Arc> { let args = DatabaseArguments::new(ClientVersion::default()) .with_max_read_transaction_duration(Some(MaxReadTransactionDuration::Unbounded)); From 30e8c7817186bf654fbe3e186a6fd54f6788499a Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 11:50:03 +0100 Subject: [PATCH 23/30] chore: rm async from state root from bench (#13472) --- Cargo.lock | 1 - crates/trie/parallel/Cargo.toml | 7 +------ crates/trie/parallel/benches/root.rs | 10 ++++------ crates/trie/parallel/src/root.rs | 4 ++-- 4 files changed, 7 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 087ecef86878..3c6e3ed92f49 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9660,7 +9660,6 @@ dependencies = [ "reth-trie-common", "reth-trie-db", "thiserror 2.0.7", - "tokio", "tracing", ] diff --git a/crates/trie/parallel/Cargo.toml b/crates/trie/parallel/Cargo.toml index 1b3e2d59be10..489d59fc92fe 100644 --- a/crates/trie/parallel/Cargo.toml +++ b/crates/trie/parallel/Cargo.toml @@ -46,13 +46,8 @@ reth-trie = { workspace = true, features = ["test-utils"] } # misc rand.workspace = true -tokio = { workspace = true, default-features = false, features = [ - "sync", - "rt", - "macros", -] } rayon.workspace = true -criterion = { workspace = true, features = ["async_tokio"] } +criterion.workspace = true proptest.workspace = true proptest-arbitrary-interop.workspace = true diff --git a/crates/trie/parallel/benches/root.rs b/crates/trie/parallel/benches/root.rs index a9300efa9b0d..37da18039050 100644 --- a/crates/trie/parallel/benches/root.rs +++ b/crates/trie/parallel/benches/root.rs @@ -19,8 +19,6 @@ pub fn calculate_state_root(c: &mut Criterion) { let mut group = c.benchmark_group("Calculate State Root"); group.sample_size(20); - let runtime = tokio::runtime::Runtime::new().unwrap(); - for size in [1_000, 3_000, 5_000, 10_000] { let (db_state, updated_state) = generate_test_data(size); let provider_factory = create_test_provider_factory(); @@ -37,14 +35,14 @@ pub fn calculate_state_root(c: &mut Criterion) { // state root group.bench_function(BenchmarkId::new("sync root", size), |b| { - b.to_async(&runtime).iter_with_setup( + b.iter_with_setup( || { let sorted_state = updated_state.clone().into_sorted(); let prefix_sets = updated_state.construct_prefix_sets().freeze(); let provider = provider_factory.provider().unwrap(); (provider, sorted_state, prefix_sets) }, - |(provider, sorted_state, prefix_sets)| async move { + |(provider, sorted_state, prefix_sets)| { let hashed_cursor_factory = HashedPostStateCursorFactory::new( DatabaseHashedCursorFactory::new(provider.tx_ref()), &sorted_state, @@ -59,14 +57,14 @@ pub fn calculate_state_root(c: &mut Criterion) { // parallel root group.bench_function(BenchmarkId::new("parallel root", size), |b| { - b.to_async(&runtime).iter_with_setup( + b.iter_with_setup( || { ParallelStateRoot::new( view.clone(), TrieInput::from_state(updated_state.clone()), ) }, - |calculator| async { calculator.incremental_root() }, + |calculator| calculator.incremental_root(), ); }); } diff --git a/crates/trie/parallel/src/root.rs b/crates/trie/parallel/src/root.rs index 1ae1a6026c8f..9ee8ed71e3c4 100644 --- a/crates/trie/parallel/src/root.rs +++ b/crates/trie/parallel/src/root.rs @@ -259,8 +259,8 @@ mod tests { use reth_provider::{test_utils::create_test_provider_factory, HashingWriter}; use reth_trie::{test_utils, HashedPostState, HashedStorage}; - #[tokio::test] - async fn random_parallel_root() { + #[test] + fn random_parallel_root() { let factory = create_test_provider_factory(); let consistent_view = ConsistentDbView::new(factory.clone(), None); From 39661308446bc4e612f83f2b108a094aaaafc139 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 12:16:09 +0100 Subject: [PATCH 24/30] chore: make trie-common no-std (#13473) --- Cargo.toml | 2 +- crates/chainspec/Cargo.toml | 3 ++- crates/evm/execution-types/Cargo.toml | 3 ++- crates/primitives/Cargo.toml | 3 ++- crates/trie/common/Cargo.toml | 16 ++++++++++++++++ crates/trie/common/src/hash_builder/state.rs | 1 + crates/trie/common/src/lib.rs | 3 +++ crates/trie/common/src/nibbles.rs | 3 ++- crates/trie/common/src/prefix_set.rs | 4 ++-- crates/trie/common/src/proofs.rs | 1 + crates/trie/common/src/root.rs | 1 + crates/trie/common/src/subnode.rs | 1 + crates/trie/common/src/updates.rs | 15 ++++++++++++--- 13 files changed, 46 insertions(+), 10 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 4abed9a85f51..2ad2bc9f65ae 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -419,7 +419,7 @@ reth-tokio-util = { path = "crates/tokio-util" } reth-tracing = { path = "crates/tracing" } reth-transaction-pool = { path = "crates/transaction-pool" } reth-trie = { path = "crates/trie/trie" } -reth-trie-common = { path = "crates/trie/common" } +reth-trie-common = { path = "crates/trie/common", default-features = false } reth-trie-db = { path = "crates/trie/db" } reth-trie-parallel = { path = "crates/trie/parallel" } reth-trie-sparse = { path = "crates/trie/sparse" } diff --git a/crates/chainspec/Cargo.toml b/crates/chainspec/Cargo.toml index 0e56cf2d3d94..9e38feb8bc2b 100644 --- a/crates/chainspec/Cargo.toml +++ b/crates/chainspec/Cargo.toml @@ -51,7 +51,8 @@ std = [ "alloy-rlp/std", "reth-ethereum-forks/std", "derive_more/std", - "reth-network-peers/std" + "reth-network-peers/std", + "reth-trie-common/std" ] arbitrary = [ "alloy-chains/arbitrary", diff --git a/crates/evm/execution-types/Cargo.toml b/crates/evm/execution-types/Cargo.toml index c0ef2c5a694d..5d872846a2fa 100644 --- a/crates/evm/execution-types/Cargo.toml +++ b/crates/evm/execution-types/Cargo.toml @@ -64,5 +64,6 @@ std = [ "serde?/std", "reth-primitives-traits/std", "alloy-consensus/std", - "serde_with?/std" + "serde_with?/std", + "reth-trie-common?/std" ] diff --git a/crates/primitives/Cargo.toml b/crates/primitives/Cargo.toml index 712abd6523e8..5f7194a6a526 100644 --- a/crates/primitives/Cargo.toml +++ b/crates/primitives/Cargo.toml @@ -114,7 +114,8 @@ std = [ "bytes/std", "derive_more/std", "reth-zstd-compressors?/std", - "secp256k1?/std" + "secp256k1?/std", + "reth-trie-common/std" ] reth-codec = [ "dep:reth-codecs", diff --git a/crates/trie/common/Cargo.toml b/crates/trie/common/Cargo.toml index acebc2a7d77c..73d9a3a8577e 100644 --- a/crates/trie/common/Cargo.toml +++ b/crates/trie/common/Cargo.toml @@ -58,6 +58,22 @@ serde_json.workspace = true serde_with.workspace = true [features] +default = ["std"] +std = [ + "alloy-consensus/std", + "alloy-genesis/std", + "alloy-primitives/std", + "alloy-rlp/std", + "alloy-rpc-types-eth?/std", + "alloy-serde?/std", + "alloy-trie/std", + "bytes?/std", + "derive_more/std", + "nybbles/std", + "reth-primitives-traits/std", + "serde?/std", + "serde_with?/std" +] eip1186 = [ "alloy-rpc-types-eth/serde", "dep:alloy-serde", diff --git a/crates/trie/common/src/hash_builder/state.rs b/crates/trie/common/src/hash_builder/state.rs index 4bf3bade3986..7ed369491c8e 100644 --- a/crates/trie/common/src/hash_builder/state.rs +++ b/crates/trie/common/src/hash_builder/state.rs @@ -1,4 +1,5 @@ use crate::TrieMask; +use alloc::vec::Vec; use alloy_trie::{hash_builder::HashBuilderValue, nodes::RlpNode, HashBuilder}; use nybbles::Nibbles; diff --git a/crates/trie/common/src/lib.rs b/crates/trie/common/src/lib.rs index 6647de67811c..093c2969bd5e 100644 --- a/crates/trie/common/src/lib.rs +++ b/crates/trie/common/src/lib.rs @@ -7,6 +7,9 @@ )] #![cfg_attr(not(test), warn(unused_crate_dependencies))] #![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] +#![cfg_attr(not(feature = "std"), no_std)] + +extern crate alloc; /// The implementation of hash builder. pub mod hash_builder; diff --git a/crates/trie/common/src/nibbles.rs b/crates/trie/common/src/nibbles.rs index b1cc2f10c56f..a7db55b854bb 100644 --- a/crates/trie/common/src/nibbles.rs +++ b/crates/trie/common/src/nibbles.rs @@ -1,3 +1,4 @@ +use alloc::vec::Vec; use derive_more::Deref; pub use nybbles::Nibbles; @@ -30,7 +31,7 @@ impl PartialEq<[u8]> for StoredNibbles { impl PartialOrd<[u8]> for StoredNibbles { #[inline] - fn partial_cmp(&self, other: &[u8]) -> Option { + fn partial_cmp(&self, other: &[u8]) -> Option { self.0.as_slice().partial_cmp(other) } } diff --git a/crates/trie/common/src/prefix_set.rs b/crates/trie/common/src/prefix_set.rs index d58531f12daf..7c6a2c3a717e 100644 --- a/crates/trie/common/src/prefix_set.rs +++ b/crates/trie/common/src/prefix_set.rs @@ -1,6 +1,6 @@ use crate::Nibbles; +use alloc::{sync::Arc, vec::Vec}; use alloy_primitives::map::{B256HashMap, B256HashSet}; -use std::sync::Arc; /// Collection of mutable prefix sets. #[derive(Clone, Default, Debug)] @@ -209,7 +209,7 @@ impl PrefixSet { impl<'a> IntoIterator for &'a PrefixSet { type Item = &'a Nibbles; - type IntoIter = std::slice::Iter<'a, Nibbles>; + type IntoIter = core::slice::Iter<'a, Nibbles>; fn into_iter(self) -> Self::IntoIter { self.iter() } diff --git a/crates/trie/common/src/proofs.rs b/crates/trie/common/src/proofs.rs index 12419ec62180..54171710761e 100644 --- a/crates/trie/common/src/proofs.rs +++ b/crates/trie/common/src/proofs.rs @@ -1,6 +1,7 @@ //! Merkle trie proofs. use crate::{Nibbles, TrieAccount}; +use alloc::vec::Vec; use alloy_consensus::constants::KECCAK_EMPTY; use alloy_primitives::{ keccak256, diff --git a/crates/trie/common/src/root.rs b/crates/trie/common/src/root.rs index 982dec98837f..b23d3b42dfe3 100644 --- a/crates/trie/common/src/root.rs +++ b/crates/trie/common/src/root.rs @@ -1,6 +1,7 @@ //! Common root computation functions. use crate::TrieAccount; +use alloc::vec::Vec; use alloy_primitives::{keccak256, Address, B256, U256}; use alloy_rlp::Encodable; use alloy_trie::HashBuilder; diff --git a/crates/trie/common/src/subnode.rs b/crates/trie/common/src/subnode.rs index de65a7887806..7ef8a349fa79 100644 --- a/crates/trie/common/src/subnode.rs +++ b/crates/trie/common/src/subnode.rs @@ -1,4 +1,5 @@ use super::BranchNodeCompact; +use alloc::vec::Vec; /// Walker sub node for storing intermediate state root calculation state in the database. #[derive(Debug, Clone, PartialEq, Eq, Default)] diff --git a/crates/trie/common/src/updates.rs b/crates/trie/common/src/updates.rs index 1f50462507b5..99e2c908ca80 100644 --- a/crates/trie/common/src/updates.rs +++ b/crates/trie/common/src/updates.rs @@ -1,4 +1,5 @@ use crate::{BranchNodeCompact, HashBuilder, Nibbles}; +use alloc::vec::Vec; use alloy_primitives::{ map::{B256HashMap, B256HashSet, HashMap, HashSet}, B256, @@ -230,6 +231,10 @@ impl StorageTrieUpdates { #[cfg(any(test, feature = "serde"))] mod serde_nibbles_set { use crate::Nibbles; + use alloc::{ + string::{String, ToString}, + vec::Vec, + }; use alloy_primitives::map::HashSet; use serde::{de::Error, Deserialize, Deserializer, Serialize, Serializer}; @@ -266,13 +271,17 @@ mod serde_nibbles_set { #[cfg(any(test, feature = "serde"))] mod serde_nibbles_map { use crate::Nibbles; + use alloc::{ + string::{String, ToString}, + vec::Vec, + }; use alloy_primitives::{hex, map::HashMap}; + use core::marker::PhantomData; use serde::{ de::{Error, MapAccess, Visitor}, ser::SerializeMap, Deserialize, Deserializer, Serialize, Serializer, }; - use std::marker::PhantomData; pub(super) fn serialize( map: &HashMap, @@ -308,7 +317,7 @@ mod serde_nibbles_map { { type Value = HashMap; - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + fn expecting(&self, formatter: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { formatter.write_str("a map with hex-encoded Nibbles keys") } @@ -411,10 +420,10 @@ fn exclude_empty_from_pair( #[cfg(feature = "serde-bincode-compat")] pub mod serde_bincode_compat { use crate::{BranchNodeCompact, Nibbles}; + use alloc::borrow::Cow; use alloy_primitives::map::{B256HashMap, HashMap, HashSet}; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use serde_with::{DeserializeAs, SerializeAs}; - use std::borrow::Cow; /// Bincode-compatible [`super::TrieUpdates`] serde implementation. /// From a4f86b0e2d4bdefa4139a9e5161f1349ca06e3b0 Mon Sep 17 00:00:00 2001 From: DaniPopes <57450786+DaniPopes@users.noreply.github.com> Date: Fri, 20 Dec 2024 13:21:51 +0200 Subject: [PATCH 25/30] feat: set up codspeed (#13372) --- .github/scripts/codspeed-build.sh | 23 ++++ .github/workflows/bench.yml | 50 +++----- .gitignore | 2 +- Cargo.lock | 93 +++++++------- Cargo.toml | 3 +- crates/blockchain-tree/Cargo.toml | 3 +- crates/engine/tree/Cargo.toml | 4 +- crates/engine/tree/benches/channel_perf.rs | 6 +- crates/engine/tree/benches/state_root_task.rs | 3 +- crates/net/ecies/src/algorithm.rs | 1 - crates/primitives/benches/validate_blob_tx.rs | 8 +- crates/stages/stages/benches/criterion.rs | 36 +++--- crates/stages/stages/benches/setup/mod.rs | 6 + crates/storage/db/Cargo.toml | 6 - crates/storage/db/benches/criterion.rs | 116 ++++++------------ crates/storage/db/benches/hash_keys.rs | 9 +- crates/storage/db/benches/iai.rs | 102 --------------- crates/storage/db/src/lib.rs | 10 +- crates/storage/libmdbx-rs/benches/cursor.rs | 4 +- .../storage/libmdbx-rs/benches/transaction.rs | 4 +- crates/transaction-pool/Cargo.toml | 70 +++++------ crates/transaction-pool/benches/priority.rs | 3 +- crates/transaction-pool/benches/reorder.rs | 24 ++-- crates/transaction-pool/benches/truncate.rs | 6 +- crates/trie/common/benches/prefix_set.rs | 10 +- crates/trie/parallel/Cargo.toml | 8 ++ crates/trie/parallel/benches/root.rs | 6 + crates/trie/sparse/Cargo.toml | 13 ++ crates/trie/sparse/benches/rlp_node.rs | 25 ++-- crates/trie/sparse/benches/root.rs | 26 +++- crates/trie/trie/Cargo.toml | 21 ++-- crates/trie/trie/benches/hash_post_state.rs | 6 + crates/trie/trie/benches/trie_root.rs | 3 +- docs/design/database.md | 2 +- 34 files changed, 316 insertions(+), 396 deletions(-) create mode 100755 .github/scripts/codspeed-build.sh delete mode 100644 crates/storage/db/benches/iai.rs diff --git a/.github/scripts/codspeed-build.sh b/.github/scripts/codspeed-build.sh new file mode 100755 index 000000000000..188ecfb883f7 --- /dev/null +++ b/.github/scripts/codspeed-build.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +set -eo pipefail + +# TODO: Benchmarks run WAY too slow due to excessive amount of iterations. + +cmd=(cargo codspeed build --profile profiling) +excludes=( + # Unnecessary + --exclude reth-libmdbx + # Build is too slow + --exclude reth-network + # Built separately + --exclude reth-transaction-pool + # TODO: some benchmarks panic: https://github.com/paradigmxyz/reth/actions/runs/12307046814/job/34349955788 + --exclude reth-db + --exclude reth-trie-parallel + --exclude reth-engine-tree +) + +"${cmd[@]}" --features test-utils --workspace "${excludes[@]}" + +# TODO: Slow benchmarks due to too many iterations +## "${cmd[@]}" -p reth-transaction-pool --features test-utils,arbitrary diff --git a/.github/workflows/bench.yml b/.github/workflows/bench.yml index 8ddc97441c2a..34c98d472bdd 100644 --- a/.github/workflows/bench.yml +++ b/.github/workflows/bench.yml @@ -1,4 +1,4 @@ -# Runs benchmarks on serialization/deserialization of storage values and keys. +# Runs benchmarks. on: pull_request: @@ -9,7 +9,6 @@ on: env: CARGO_TERM_COLOR: always BASELINE: base - IAI_CALLGRIND_RUNNER: iai-callgrind-runner concurrency: group: ${{ github.workflow }}-${{ github.head_ref || github.run_id }} @@ -17,47 +16,26 @@ concurrency: name: bench jobs: - iai: + codspeed: runs-on: group: Reth - # Only run benchmarks in merge groups and on main - if: github.event_name != 'pull_request' steps: - uses: actions/checkout@v4 - - name: Install Valgrind - run: sudo apt update && sudo apt install valgrind + with: + submodules: true + - uses: rui314/setup-mold@v1 - uses: dtolnay/rust-toolchain@stable - uses: Swatinem/rust-cache@v2 with: cache-on-failure: true - - name: Install cargo-binstall - uses: taiki-e/install-action@cargo-binstall - - name: Install iai-callgrind-runner - run: | - echo "::group::Install" - version=$(cargo metadata --format-version=1 |\ - jq '.packages[] | select(.name == "iai-callgrind").version' |\ - tr -d '"' - ) - cargo binstall iai-callgrind-runner --version $version --no-confirm --no-symlinks --force - echo "::endgroup::" - echo "::group::Verification" - which iai-callgrind-runner - echo "::endgroup::" - - name: Checkout base - uses: actions/checkout@v4 + - name: Install cargo-codspeed + uses: taiki-e/install-action@v2 with: - ref: ${{ github.base_ref || 'main' }} - # On `main` branch, generates test vectors and serializes them to disk using `serde-json`. - - name: Generate test vectors - run: cargo run --bin reth --features dev -- test-vectors tables - # Runs iai and stores `main` baseline report for comparison later on $BASELINE. - - name: Save baseline - run: cargo bench -p reth-db --bench iai --profile profiling --features test-utils -- --save-baseline=$BASELINE - - name: Checkout PR - uses: actions/checkout@v4 + tool: cargo-codspeed + - name: Build the benchmark target(s) + run: ./.github/scripts/codspeed-build.sh + - name: Run the benchmarks + uses: CodSpeedHQ/action@v3 with: - clean: false - # Runs iai on incoming merge using previously generated test-vectors and compares the report against `main` report. - - name: Compare PR benchmarks - run: cargo bench -p reth-db --bench iai --profile profiling --features test-utils -- --baseline=$BASELINE + run: cargo codspeed run --workspace + token: ${{ secrets.CODSPEED_TOKEN }} diff --git a/.gitignore b/.gitignore index 00f776542422..424bdd938e08 100644 --- a/.gitignore +++ b/.gitignore @@ -19,7 +19,7 @@ target/ testdata/micro/db # Generated data for stage benchmarks -crates/stages/testdata +crates/stages/stages/testdata # Prometheus data dir data/ diff --git a/Cargo.lock b/Cargo.lock index 3c6e3ed92f49..78ff0d78cd1b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1843,6 +1843,30 @@ version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "afb84c814227b90d6895e01398aee0d8033c00e7466aca416fb6a8e0eb19d8a7" +[[package]] +name = "codspeed" +version = "2.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "450a0e9df9df1c154156f4344f99d8f6f6e69d0fc4de96ef6e2e68b2ec3bce97" +dependencies = [ + "colored", + "libc", + "serde_json", +] + +[[package]] +name = "codspeed-criterion-compat" +version = "2.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eb1a6cb9c20e177fde58cdef97c1c7c9264eb1424fe45c4fccedc2fb078a569" +dependencies = [ + "codspeed", + "colored", + "criterion", + "futures", + "tokio", +] + [[package]] name = "coins-bip32" version = "0.12.0" @@ -1900,6 +1924,16 @@ version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" +[[package]] +name = "colored" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cbf2150cce219b664a8a70df7a1f933836724b503f8a413af9365b4dcc4d90b8" +dependencies = [ + "lazy_static", + "windows-sys 0.48.0", +] + [[package]] name = "combine" version = "4.6.7" @@ -3934,42 +3968,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "iai-callgrind" -version = "0.14.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22275f8051874cd2f05b2aa1e0098d5cbec34df30ff92f1a1e2686a4cefed870" -dependencies = [ - "bincode", - "derive_more", - "iai-callgrind-macros", - "iai-callgrind-runner", -] - -[[package]] -name = "iai-callgrind-macros" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8e6677dc52bd798b988e62ffd6831bf7eb46e4348cb1c74c1164954ebd0e5a1" -dependencies = [ - "derive_more", - "proc-macro-error2", - "proc-macro2", - "quote", - "serde", - "serde_json", - "syn 2.0.90", -] - -[[package]] -name = "iai-callgrind-runner" -version = "0.14.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a02dd95fe4949513b45a328b5b18f527ee02e96f3428b48090aa7cf9043ab0b8" -dependencies = [ - "serde", -] - [[package]] name = "iana-time-zone" version = "0.1.61" @@ -6998,10 +6996,9 @@ dependencies = [ "arbitrary", "assert_matches", "bytes", - "criterion", + "codspeed-criterion-compat", "derive_more", "eyre", - "iai-callgrind", "metrics", "page_size", "parking_lot", @@ -7391,7 +7388,7 @@ dependencies = [ "alloy-rlp", "alloy-rpc-types-engine", "assert_matches", - "criterion", + "codspeed-criterion-compat", "crossbeam-channel", "derive_more", "futures", @@ -7912,7 +7909,7 @@ version = "1.1.4" dependencies = [ "bitflags 2.6.0", "byteorder", - "criterion", + "codspeed-criterion-compat", "dashmap 6.1.0", "derive_more", "indexmap 2.6.0", @@ -7979,7 +7976,7 @@ dependencies = [ "alloy-rlp", "aquamarine", "auto_impl", - "criterion", + "codspeed-criterion-compat", "derive_more", "discv5", "enr", @@ -8762,7 +8759,7 @@ dependencies = [ "bincode", "bytes", "c-kzg", - "criterion", + "codspeed-criterion-compat", "derive_more", "modular-bitfield", "once_cell", @@ -9287,7 +9284,7 @@ dependencies = [ "alloy-rlp", "assert_matches", "bincode", - "criterion", + "codspeed-criterion-compat", "futures-util", "itertools 0.13.0", "num-traits", @@ -9514,7 +9511,7 @@ dependencies = [ "assert_matches", "auto_impl", "bitflags 2.6.0", - "criterion", + "codspeed-criterion-compat", "futures-util", "metrics", "parking_lot", @@ -9560,7 +9557,7 @@ dependencies = [ "alloy-rlp", "alloy-trie", "auto_impl", - "criterion", + "codspeed-criterion-compat", "itertools 0.13.0", "metrics", "proptest", @@ -9593,7 +9590,7 @@ dependencies = [ "arbitrary", "bincode", "bytes", - "criterion", + "codspeed-criterion-compat", "derive_more", "hash-db", "itertools 0.13.0", @@ -9643,7 +9640,7 @@ version = "1.1.4" dependencies = [ "alloy-primitives", "alloy-rlp", - "criterion", + "codspeed-criterion-compat", "derive_more", "itertools 0.13.0", "metrics", @@ -9671,7 +9668,7 @@ dependencies = [ "alloy-rlp", "arbitrary", "assert_matches", - "criterion", + "codspeed-criterion-compat", "itertools 0.13.0", "pretty_assertions", "proptest", diff --git a/Cargo.toml b/Cargo.toml index 2ad2bc9f65ae..62b54a3bdb4e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -601,8 +601,7 @@ toml = "0.8" # misc-testing arbitrary = "1.3" assert_matches = "1.5.0" -criterion = "0.5" -iai-callgrind = "0.14" +criterion = { package = "codspeed-criterion-compat", version = "2.7" } pprof = "0.14" proptest = "1.4" proptest-derive = "0.5" diff --git a/crates/blockchain-tree/Cargo.toml b/crates/blockchain-tree/Cargo.toml index 07ecedf882f2..1c42a292aea7 100644 --- a/crates/blockchain-tree/Cargo.toml +++ b/crates/blockchain-tree/Cargo.toml @@ -77,7 +77,8 @@ test-utils = [ "reth-db-api/test-utils", "reth-provider/test-utils", "reth-trie-db/test-utils", - "reth-trie/test-utils" + "reth-trie/test-utils", + "reth-trie-parallel/test-utils" ] optimism = [ "reth-primitives/optimism", diff --git a/crates/engine/tree/Cargo.toml b/crates/engine/tree/Cargo.toml index b5b8fc743645..84ca3291a33f 100644 --- a/crates/engine/tree/Cargo.toml +++ b/crates/engine/tree/Cargo.toml @@ -120,6 +120,8 @@ test-utils = [ "reth-static-file", "reth-tracing", "reth-trie/test-utils", - "reth-trie-db/test-utils", + "reth-trie-sparse/test-utils", "reth-prune-types?/test-utils", + "reth-trie-db/test-utils", + "reth-trie-parallel/test-utils", ] diff --git a/crates/engine/tree/benches/channel_perf.rs b/crates/engine/tree/benches/channel_perf.rs index c1c65e0a68e1..50c48a391f65 100644 --- a/crates/engine/tree/benches/channel_perf.rs +++ b/crates/engine/tree/benches/channel_perf.rs @@ -7,7 +7,7 @@ use revm_primitives::{ Account, AccountInfo, AccountStatus, Address, EvmState, EvmStorage, EvmStorageSlot, HashMap, B256, U256, }; -use std::thread; +use std::{hint::black_box, thread}; /// Creates a mock state with the specified number of accounts for benchmarking fn create_bench_state(num_accounts: usize) -> EvmState { @@ -47,7 +47,7 @@ impl StdStateRootTask { fn run(self) { while let Ok(state) = self.rx.recv() { - criterion::black_box(state); + black_box(state); } } } @@ -64,7 +64,7 @@ impl CrossbeamStateRootTask { fn run(self) { while let Ok(state) = self.rx.recv() { - criterion::black_box(state); + black_box(state); } } } diff --git a/crates/engine/tree/benches/state_root_task.rs b/crates/engine/tree/benches/state_root_task.rs index 7a84a4c62d67..b88c8578c5db 100644 --- a/crates/engine/tree/benches/state_root_task.rs +++ b/crates/engine/tree/benches/state_root_task.rs @@ -3,7 +3,7 @@ #![allow(missing_docs)] -use criterion::{black_box, criterion_group, criterion_main, BenchmarkId, Criterion}; +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use reth_engine_tree::tree::root::{StateRootConfig, StateRootTask}; use reth_evm::system_calls::OnStateHook; use reth_primitives::{Account as RethAccount, StorageEntry}; @@ -22,6 +22,7 @@ use revm_primitives::{ Account as RevmAccount, AccountInfo, AccountStatus, Address, EvmState, EvmStorageSlot, HashMap, B256, KECCAK_EMPTY, U256, }; +use std::hint::black_box; #[derive(Debug, Clone)] struct BenchParams { diff --git a/crates/net/ecies/src/algorithm.rs b/crates/net/ecies/src/algorithm.rs index f799b6c7f6c0..3abc7cbb2671 100644 --- a/crates/net/ecies/src/algorithm.rs +++ b/crates/net/ecies/src/algorithm.rs @@ -975,6 +975,5 @@ mod tests { &mut dest, ); } - std::hint::black_box(()); } } diff --git a/crates/primitives/benches/validate_blob_tx.rs b/crates/primitives/benches/validate_blob_tx.rs index 453381366e14..b38937508f8d 100644 --- a/crates/primitives/benches/validate_blob_tx.rs +++ b/crates/primitives/benches/validate_blob_tx.rs @@ -69,12 +69,10 @@ fn validate_blob_tx( // for now we just use the default SubPoolLimit group.bench_function(group_id, |b| { + let kzg_settings = kzg_settings.get(); b.iter_with_setup(setup, |(tx, blob_sidecar)| { - if let Err(err) = - std::hint::black_box(tx.validate_blob(&blob_sidecar, kzg_settings.get())) - { - println!("Validation failed: {err:?}"); - } + let r = tx.validate_blob(&blob_sidecar, kzg_settings); + (r, tx, blob_sidecar) }); }); } diff --git a/crates/stages/stages/benches/criterion.rs b/crates/stages/stages/benches/criterion.rs index 0f876dd7011a..2bbb5ce0a542 100644 --- a/crates/stages/stages/benches/criterion.rs +++ b/crates/stages/stages/benches/criterion.rs @@ -1,11 +1,10 @@ #![allow(missing_docs)] +#![allow(unexpected_cfgs)] + +use alloy_primitives::BlockNumber; use criterion::{criterion_main, measurement::WallTime, BenchmarkGroup, Criterion}; -#[cfg(not(target_os = "windows"))] -use pprof::criterion::{Output, PProfProfiler}; use reth_config::config::{EtlConfig, TransactionLookupConfig}; use reth_db::{test_utils::TempDatabase, Database, DatabaseEnv}; - -use alloy_primitives::BlockNumber; use reth_provider::{test_utils::MockNodeTypesWithDB, DatabaseProvider, DatabaseProviderFactory}; use reth_stages::{ stages::{MerkleStage, SenderRecoveryStage, TransactionLookupStage}, @@ -22,25 +21,30 @@ use setup::StageRange; // Expanded form of `criterion_group!` // // This is currently needed to only instantiate the tokio runtime once. +#[cfg(not(codspeed))] fn benches() { - #[cfg(not(target_os = "windows"))] - let mut criterion = Criterion::default() - .with_profiler(PProfProfiler::new(1000, Output::Flamegraph(None))) - .configure_from_args(); + #[cfg(not(windows))] + use pprof::criterion::{Output, PProfProfiler}; + let criterion = Criterion::default(); + #[cfg(not(windows))] + let criterion = criterion.with_profiler(PProfProfiler::new(1000, Output::Flamegraph(None))); + run_benches(&mut criterion.configure_from_args()); +} + +fn run_benches(criterion: &mut Criterion) { let runtime = Runtime::new().unwrap(); let _guard = runtime.enter(); - - #[cfg(target_os = "windows")] - let mut criterion = Criterion::default().configure_from_args(); - - transaction_lookup(&mut criterion, &runtime); - account_hashing(&mut criterion, &runtime); - senders(&mut criterion, &runtime); - merkle(&mut criterion, &runtime); + transaction_lookup(criterion, &runtime); + account_hashing(criterion, &runtime); + senders(criterion, &runtime); + merkle(criterion, &runtime); } +#[cfg(not(codspeed))] criterion_main!(benches); +#[cfg(codspeed)] +criterion_main!(run_benches); const DEFAULT_NUM_BLOCKS: u64 = 10_000; diff --git a/crates/stages/stages/benches/setup/mod.rs b/crates/stages/stages/benches/setup/mod.rs index c1c3ff89d727..b2e6d4efcea0 100644 --- a/crates/stages/stages/benches/setup/mod.rs +++ b/crates/stages/stages/benches/setup/mod.rs @@ -88,6 +88,12 @@ where // Helper for generating testdata for the benchmarks. // Returns the path to the database file. pub(crate) fn txs_testdata(num_blocks: u64) -> TestStageDB { + // This is way too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) { + std::process::exit(0); + } + let txs_range = 100..150; // number of storage changes per transition diff --git a/crates/storage/db/Cargo.toml b/crates/storage/db/Cargo.toml index fd313a40ae53..b7114a430663 100644 --- a/crates/storage/db/Cargo.toml +++ b/crates/storage/db/Cargo.toml @@ -69,7 +69,6 @@ pprof = { workspace = true, features = [ "criterion", ] } criterion.workspace = true -iai-callgrind.workspace = true arbitrary = { workspace = true, features = ["derive"] } proptest.workspace = true @@ -125,11 +124,6 @@ name = "criterion" required-features = ["test-utils"] harness = false -[[bench]] -name = "iai" -required-features = ["test-utils"] -harness = false - [[bench]] name = "get" required-features = ["test-utils"] diff --git a/crates/storage/db/benches/criterion.rs b/crates/storage/db/benches/criterion.rs index 7ac9566d80c5..b8102326d0a2 100644 --- a/crates/storage/db/benches/criterion.rs +++ b/crates/storage/db/benches/criterion.rs @@ -3,7 +3,7 @@ use std::{path::Path, sync::Arc}; use criterion::{ - black_box, criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, + criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, }; use pprof::criterion::{Output, PProfProfiler}; use reth_db::{tables::*, test_utils::create_test_rw_db_with_path}; @@ -71,12 +71,9 @@ where b.iter_with_setup( || input.clone(), |input| { - { - for (k, _, _, _) in input { - k.encode(); - } - }; - black_box(()); + for (k, _, _, _) in input { + k.encode(); + } }, ) }); @@ -85,12 +82,9 @@ where b.iter_with_setup( || input.clone(), |input| { - { - for (_, k, _, _) in input { - let _ = ::Key::decode(&k); - } - }; - black_box(()); + for (_, k, _, _) in input { + let _ = ::Key::decode(&k); + } }, ) }); @@ -99,12 +93,9 @@ where b.iter_with_setup( || input.clone(), |input| { - { - for (_, _, v, _) in input { - v.compress(); - } - }; - black_box(()); + for (_, _, v, _) in input { + v.compress(); + } }, ) }); @@ -113,12 +104,9 @@ where b.iter_with_setup( || input.clone(), |input| { - { - for (_, _, _, v) in input { - let _ = ::Value::decompress(&v); - } - }; - black_box(()); + for (_, _, _, v) in input { + let _ = ::Value::decompress(&v); + } }, ) }); @@ -148,14 +136,10 @@ where // Create TX let tx = db.tx_mut().expect("tx"); let mut crsr = tx.cursor_write::().expect("cursor"); - - black_box({ - for (k, _, v, _) in input { - crsr.append(k, v).expect("submit"); - } - - tx.inner.commit().unwrap() - }); + for (k, _, v, _) in input { + crsr.append(k, v).expect("submit"); + } + tx.inner.commit().unwrap() }, ) }); @@ -171,15 +155,12 @@ where // Create TX let tx = db.tx_mut().expect("tx"); let mut crsr = tx.cursor_write::().expect("cursor"); + for index in RANDOM_INDEXES { + let (k, _, v, _) = input.get(index).unwrap().clone(); + crsr.insert(k, v).expect("submit"); + } - black_box({ - for index in RANDOM_INDEXES { - let (k, _, v, _) = input.get(index).unwrap().clone(); - crsr.insert(k, v).expect("submit"); - } - - tx.inner.commit().unwrap() - }); + tx.inner.commit().unwrap() }, ) }); @@ -190,15 +171,11 @@ where b.iter(|| { // Create TX let tx = db.tx().expect("tx"); - - { - let mut cursor = tx.cursor_read::().expect("cursor"); - let walker = cursor.walk(Some(input.first().unwrap().0.clone())).unwrap(); - for element in walker { - element.unwrap(); - } - }; - black_box(()); + let mut cursor = tx.cursor_read::().expect("cursor"); + let walker = cursor.walk(Some(input.first().unwrap().0.clone())).unwrap(); + for element in walker { + element.unwrap(); + } }) }); @@ -208,14 +185,10 @@ where b.iter(|| { // Create TX let tx = db.tx().expect("tx"); - - { - for index in RANDOM_INDEXES { - let mut cursor = tx.cursor_read::().expect("cursor"); - cursor.seek_exact(input.get(index).unwrap().0.clone()).unwrap(); - } - }; - black_box(()); + for index in RANDOM_INDEXES { + let mut cursor = tx.cursor_read::().expect("cursor"); + cursor.seek_exact(input.get(index).unwrap().0.clone()).unwrap(); + } }) }); } @@ -245,14 +218,10 @@ where // Create TX let tx = db.tx_mut().expect("tx"); let mut crsr = tx.cursor_dup_write::().expect("cursor"); - - black_box({ - for (k, _, v, _) in input { - crsr.append_dup(k, v).expect("submit"); - } - - tx.inner.commit().unwrap() - }); + for (k, _, v, _) in input { + crsr.append_dup(k, v).expect("submit"); + } + tx.inner.commit().unwrap() }, ) }); @@ -268,12 +237,10 @@ where |(input, db)| { // Create TX let tx = db.tx_mut().expect("tx"); - for index in RANDOM_INDEXES { let (k, _, v, _) = input.get(index).unwrap().clone(); tx.put::(k, v).unwrap(); } - tx.inner.commit().unwrap(); }, ) @@ -286,14 +253,11 @@ where // Create TX let tx = db.tx().expect("tx"); - { - let mut cursor = tx.cursor_dup_read::().expect("cursor"); - let walker = cursor.walk_dup(None, Some(T::SubKey::default())).unwrap(); - for element in walker { - element.unwrap(); - } - }; - black_box(()); + let mut cursor = tx.cursor_dup_read::().expect("cursor"); + let walker = cursor.walk_dup(None, Some(T::SubKey::default())).unwrap(); + for element in walker { + element.unwrap(); + } }) }); diff --git a/crates/storage/db/benches/hash_keys.rs b/crates/storage/db/benches/hash_keys.rs index cd0bfcb5be06..0150354a7afb 100644 --- a/crates/storage/db/benches/hash_keys.rs +++ b/crates/storage/db/benches/hash_keys.rs @@ -3,7 +3,7 @@ use std::{collections::HashSet, path::Path, sync::Arc}; use criterion::{ - black_box, criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, + criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, }; use pprof::criterion::{Output, PProfProfiler}; use proptest::{ @@ -20,6 +20,7 @@ use reth_db_api::{ transaction::DbTxMut, }; use reth_fs_util as fs; +use std::hint::black_box; mod utils; use utils::*; @@ -46,6 +47,12 @@ pub fn hash_keys(c: &mut Criterion) { group.sample_size(10); for size in [10_000, 100_000, 1_000_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && size > 10_000 { + continue; + } + measure_table_insertion::(&mut group, size); } } diff --git a/crates/storage/db/benches/iai.rs b/crates/storage/db/benches/iai.rs deleted file mode 100644 index 167cd0860e26..000000000000 --- a/crates/storage/db/benches/iai.rs +++ /dev/null @@ -1,102 +0,0 @@ -#![allow(missing_docs, non_snake_case, unreachable_pub)] - -use iai_callgrind::{ - library_benchmark, library_benchmark_group, LibraryBenchmarkConfig, RegressionConfig, -}; -use paste::paste; -use reth_db_api::table::{Compress, Decode, Decompress, Encode, Table}; - -mod utils; -use utils::*; - -macro_rules! impl_iai_callgrind_inner { - ( - $(($name:ident, $group_name:ident, $mod:ident, $compress:ident, $decompress:ident, $encode:ident, $decode:ident, $seqread:ident, $randread:ident, $seqwrite:ident, $randwrite:ident))+ - ) => { - use std::hint::black_box; - $( - #[library_benchmark] - pub fn $compress() { - for (_, _, v, _) in black_box(load_vectors::()) { - black_box(v.compress()); - } - } - - #[library_benchmark] - pub fn $decompress() { - for (_, _, _, comp) in black_box(load_vectors::()) { - let _ = black_box(::Value::decompress(&comp)); - } - } - - #[library_benchmark] - pub fn $encode() { - for (k, _, _, _) in black_box(load_vectors::()) { - black_box(k.encode()); - } - } - - #[library_benchmark] - pub fn $decode() { - for (_, enc, _, _) in black_box(load_vectors::()) { - let _ = black_box(::Key::decode(&enc)); - } - } - - #[allow(dead_code)] - pub const fn $seqread() {} - - #[allow(dead_code)] - pub const fn $randread() {} - - #[allow(dead_code)] - pub const fn $seqwrite() {} - - #[allow(dead_code)] - pub const fn $randwrite() {} - - - library_benchmark_group!( - name = $group_name; - config = LibraryBenchmarkConfig::default() - .regression( - RegressionConfig::default().fail_fast(false) - ); - benchmarks = - $compress, - $decompress, - $encode, - $decode, - ); - )+ - - iai_callgrind::main!( - config = LibraryBenchmarkConfig::default(); - library_benchmark_groups = $($group_name),+); - }; -} - -macro_rules! impl_iai_callgrind { - ($($name:ident),+) => { - paste! { - impl_iai_callgrind_inner!( - $( - ( $name, [<$name _group>],[<$name _mod>], [<$name _ValueCompress>], [<$name _ValueDecompress>], [<$name _ValueEncode>], [<$name _ValueDecode>], [<$name _SeqRead>], [<$name _RandomRead>], [<$name _SeqWrite>], [<$name _RandomWrite>]) - )+ - ); - } - }; -} - -impl_iai_callgrind!( - CanonicalHeaders, - HeaderTerminalDifficulties, - HeaderNumbers, - Headers, - BlockBodyIndices, - BlockOmmers, - TransactionHashNumbers, - Transactions, - PlainStorageState, - PlainAccountState -); diff --git a/crates/storage/db/src/lib.rs b/crates/storage/db/src/lib.rs index 113e8661f83d..7e6a6932bdd7 100644 --- a/crates/storage/db/src/lib.rs +++ b/crates/storage/db/src/lib.rs @@ -60,15 +60,15 @@ pub mod test_utils { use tempfile::TempDir; /// Error during database open - pub const ERROR_DB_OPEN: &str = "Not able to open the database file."; + pub const ERROR_DB_OPEN: &str = "could not open the database file"; /// Error during database creation - pub const ERROR_DB_CREATION: &str = "Not able to create the database file."; + pub const ERROR_DB_CREATION: &str = "could not create the database file"; /// Error during database creation - pub const ERROR_STATIC_FILES_CREATION: &str = "Not able to create the static file path."; + pub const ERROR_STATIC_FILES_CREATION: &str = "could not create the static file path"; /// Error during table creation - pub const ERROR_TABLE_CREATION: &str = "Not able to create tables in the database."; + pub const ERROR_TABLE_CREATION: &str = "could not create tables in the database"; /// Error during tempdir creation - pub const ERROR_TEMPDIR: &str = "Not able to create a temporary directory."; + pub const ERROR_TEMPDIR: &str = "could not create a temporary directory"; /// A database will delete the db dir when dropped. pub struct TempDatabase { diff --git a/crates/storage/libmdbx-rs/benches/cursor.rs b/crates/storage/libmdbx-rs/benches/cursor.rs index acd7d9a72c72..d43c5182983b 100644 --- a/crates/storage/libmdbx-rs/benches/cursor.rs +++ b/crates/storage/libmdbx-rs/benches/cursor.rs @@ -1,10 +1,10 @@ #![allow(missing_docs)] mod utils; -use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use criterion::{criterion_group, criterion_main, Criterion}; use pprof::criterion::{Output, PProfProfiler}; use reth_libmdbx::{ffi::*, *}; -use std::ptr; +use std::{hint::black_box, ptr}; use utils::*; /// Benchmark of iterator sequential read performance. diff --git a/crates/storage/libmdbx-rs/benches/transaction.rs b/crates/storage/libmdbx-rs/benches/transaction.rs index 33d25cdaa68a..eb4b0671b7fa 100644 --- a/crates/storage/libmdbx-rs/benches/transaction.rs +++ b/crates/storage/libmdbx-rs/benches/transaction.rs @@ -1,11 +1,11 @@ #![allow(missing_docs, unreachable_pub)] mod utils; -use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use criterion::{criterion_group, criterion_main, Criterion}; use rand::{prelude::SliceRandom, SeedableRng}; use rand_xorshift::XorShiftRng; use reth_libmdbx::{ffi::*, ObjectLength, WriteFlags}; -use std::ptr; +use std::{hint::black_box, ptr}; use utils::*; fn bench_get_rand(c: &mut Criterion) { diff --git a/crates/transaction-pool/Cargo.toml b/crates/transaction-pool/Cargo.toml index f2586059967e..238a0e1d15ee 100644 --- a/crates/transaction-pool/Cargo.toml +++ b/crates/transaction-pool/Cargo.toml @@ -76,45 +76,45 @@ serde_json.workspace = true [features] default = ["serde"] serde = [ - "dep:serde", - "reth-execution-types/serde", - "reth-eth-wire-types/serde", - "reth-provider/serde", - "alloy-consensus/serde", - "alloy-eips/serde", - "alloy-primitives/serde", - "bitflags/serde", - "parking_lot/serde", - "rand?/serde", - "smallvec/serde", - "reth-primitives-traits/serde", - "revm-interpreter/serde", - "revm-primitives/serde" + "dep:serde", + "reth-execution-types/serde", + "reth-eth-wire-types/serde", + "reth-provider/serde", + "alloy-consensus/serde", + "alloy-eips/serde", + "alloy-primitives/serde", + "bitflags/serde", + "parking_lot/serde", + "rand?/serde", + "smallvec/serde", + "reth-primitives-traits/serde", + "revm-interpreter/serde", + "revm-primitives/serde", ] test-utils = [ - "rand", - "paste", - "serde", - "reth-chain-state/test-utils", - "reth-chainspec/test-utils", - "reth-primitives/test-utils", - "reth-provider/test-utils", - "reth-primitives-traits/test-utils", + "rand", + "paste", + "serde", + "reth-chain-state/test-utils", + "reth-chainspec/test-utils", + "reth-primitives/test-utils", + "reth-provider/test-utils", + "reth-primitives-traits/test-utils", ] arbitrary = [ - "proptest", - "reth-primitives/arbitrary", - "proptest-arbitrary-interop", - "reth-chainspec/arbitrary", - "reth-eth-wire-types/arbitrary", - "alloy-consensus/arbitrary", - "alloy-eips/arbitrary", - "alloy-primitives/arbitrary", - "bitflags/arbitrary", - "reth-primitives-traits/arbitrary", - "smallvec/arbitrary", - "revm-interpreter/arbitrary", - "revm-primitives/arbitrary" + "proptest", + "reth-primitives/arbitrary", + "proptest-arbitrary-interop", + "reth-chainspec/arbitrary", + "reth-eth-wire-types/arbitrary", + "alloy-consensus/arbitrary", + "alloy-eips/arbitrary", + "alloy-primitives/arbitrary", + "bitflags/arbitrary", + "reth-primitives-traits/arbitrary", + "smallvec/arbitrary", + "revm-interpreter/arbitrary", + "revm-primitives/arbitrary", ] [[bench]] diff --git a/crates/transaction-pool/benches/priority.rs b/crates/transaction-pool/benches/priority.rs index 8a63da3d4c2d..196e76ed0294 100644 --- a/crates/transaction-pool/benches/priority.rs +++ b/crates/transaction-pool/benches/priority.rs @@ -1,9 +1,10 @@ #![allow(missing_docs)] use criterion::{ - black_box, criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, + criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, }; use proptest::{prelude::*, strategy::ValueTree, test_runner::TestRunner}; use reth_transaction_pool::{blob_tx_priority, fee_delta}; +use std::hint::black_box; fn generate_test_data_fee_delta() -> (u128, u128) { let config = ProptestConfig::default(); diff --git a/crates/transaction-pool/benches/reorder.rs b/crates/transaction-pool/benches/reorder.rs index 9fc21629753f..e20a598d6b46 100644 --- a/crates/transaction-pool/benches/reorder.rs +++ b/crates/transaction-pool/benches/reorder.rs @@ -75,19 +75,17 @@ fn txpool_reordering_bench( ); group.bench_function(group_id, |b| { b.iter_with_setup(setup, |(mut txpool, new_txs)| { - { - // Reorder with new base fee - let bigger_base_fee = base_fee.saturating_add(10); - txpool.reorder(bigger_base_fee); - - // Reorder with new base fee after adding transactions. - for new_tx in new_txs { - txpool.add_transaction(new_tx); - } - let smaller_base_fee = base_fee.saturating_sub(10); - txpool.reorder(smaller_base_fee) - }; - std::hint::black_box(()); + // Reorder with new base fee + let bigger_base_fee = base_fee.saturating_add(10); + txpool.reorder(bigger_base_fee); + + // Reorder with new base fee after adding transactions. + for new_tx in new_txs { + txpool.add_transaction(new_tx); + } + let smaller_base_fee = base_fee.saturating_sub(10); + txpool.reorder(smaller_base_fee); + txpool }); }); } diff --git a/crates/transaction-pool/benches/truncate.rs b/crates/transaction-pool/benches/truncate.rs index 1ca6f98499cd..3c8f807efc3d 100644 --- a/crates/transaction-pool/benches/truncate.rs +++ b/crates/transaction-pool/benches/truncate.rs @@ -161,7 +161,7 @@ fn truncate_pending( group.bench_function(group_id, |b| { b.iter_with_setup(setup, |mut txpool| { txpool.truncate_pool(SubPoolLimit::default()); - std::hint::black_box(()); + txpool }); }); } @@ -195,7 +195,7 @@ fn truncate_queued( group.bench_function(group_id, |b| { b.iter_with_setup(setup, |mut txpool| { txpool.truncate_pool(SubPoolLimit::default()); - std::hint::black_box(()); + txpool }); }); } @@ -229,7 +229,7 @@ fn truncate_basefee( group.bench_function(group_id, |b| { b.iter_with_setup(setup, |mut txpool| { txpool.truncate_pool(SubPoolLimit::default()); - std::hint::black_box(()); + txpool }); }); } diff --git a/crates/trie/common/benches/prefix_set.rs b/crates/trie/common/benches/prefix_set.rs index b61d58e02729..ee0eba46a25a 100644 --- a/crates/trie/common/benches/prefix_set.rs +++ b/crates/trie/common/benches/prefix_set.rs @@ -1,6 +1,6 @@ #![allow(missing_docs, unreachable_pub)] use criterion::{ - black_box, criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, + criterion_group, criterion_main, measurement::WallTime, BenchmarkGroup, Criterion, }; use proptest::{ prelude::*, @@ -11,7 +11,7 @@ use reth_trie_common::{ prefix_set::{PrefixSet, PrefixSetMut}, Nibbles, }; -use std::collections::BTreeSet; +use std::{collections::BTreeSet, hint::black_box}; /// Abstraction for aggregating nibbles and freezing it to a type /// that can be later used for benching. @@ -48,6 +48,12 @@ pub fn prefix_set_lookups(c: &mut Criterion) { let mut group = c.benchmark_group("Prefix Set Lookups"); for size in [10, 100, 1_000, 10_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && size > 1_000 { + continue; + } + let test_data = generate_test_data(size); use implementations::*; diff --git a/crates/trie/parallel/Cargo.toml b/crates/trie/parallel/Cargo.toml index 489d59fc92fe..3390a00c868a 100644 --- a/crates/trie/parallel/Cargo.toml +++ b/crates/trie/parallel/Cargo.toml @@ -54,6 +54,14 @@ proptest-arbitrary-interop.workspace = true [features] default = ["metrics"] metrics = ["reth-metrics", "dep:metrics", "reth-trie/metrics"] +test-utils = [ + "reth-trie/test-utils", + "reth-trie-common/test-utils", + "reth-db/test-utils", + "reth-primitives/test-utils", + "reth-provider/test-utils", + "reth-trie-db/test-utils", +] [[bench]] name = "root" diff --git a/crates/trie/parallel/benches/root.rs b/crates/trie/parallel/benches/root.rs index 37da18039050..fd2c86e7c5a9 100644 --- a/crates/trie/parallel/benches/root.rs +++ b/crates/trie/parallel/benches/root.rs @@ -20,6 +20,12 @@ pub fn calculate_state_root(c: &mut Criterion) { group.sample_size(20); for size in [1_000, 3_000, 5_000, 10_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && size > 3_000 { + continue; + } + let (db_state, updated_state) = generate_test_data(size); let provider_factory = create_test_provider_factory(); { diff --git a/crates/trie/sparse/Cargo.toml b/crates/trie/sparse/Cargo.toml index 205451ef72a8..38a71432b4f2 100644 --- a/crates/trie/sparse/Cargo.toml +++ b/crates/trie/sparse/Cargo.toml @@ -41,6 +41,19 @@ proptest-arbitrary-interop.workspace = true proptest.workspace = true rand.workspace = true +[features] +test-utils = [ + "reth-primitives-traits/test-utils", + "reth-trie/test-utils", + "reth-trie-common/test-utils", +] +arbitrary = [ + "reth-primitives-traits/arbitrary", + "reth-trie-common/arbitrary", + "alloy-primitives/arbitrary", + "smallvec/arbitrary", +] + [[bench]] name = "root" harness = false diff --git a/crates/trie/sparse/benches/rlp_node.rs b/crates/trie/sparse/benches/rlp_node.rs index 57ab52978b64..25d4e64af141 100644 --- a/crates/trie/sparse/benches/rlp_node.rs +++ b/crates/trie/sparse/benches/rlp_node.rs @@ -1,6 +1,4 @@ -#![allow(missing_docs, unreachable_pub)] - -use std::time::{Duration, Instant}; +#![allow(missing_docs)] use alloy_primitives::{B256, U256}; use criterion::{criterion_group, criterion_main, Criterion}; @@ -11,7 +9,7 @@ use reth_testing_utils::generators; use reth_trie::Nibbles; use reth_trie_sparse::RevealedSparseTrie; -pub fn update_rlp_node_level(c: &mut Criterion) { +fn update_rlp_node_level(c: &mut Criterion) { let mut rng = generators::rng(); let mut group = c.benchmark_group("update rlp node level"); @@ -53,20 +51,11 @@ pub fn update_rlp_node_level(c: &mut Criterion) { group.bench_function( format!("size {size} | updated {updated_leaves}% | depth {depth}"), |b| { - // Use `iter_custom` to avoid measuring clones and drops - b.iter_custom(|iters| { - let mut elapsed = Duration::ZERO; - - let mut cloned = sparse.clone(); - for _ in 0..iters { - let start = Instant::now(); - cloned.update_rlp_node_level(depth); - elapsed += start.elapsed(); - cloned = sparse.clone(); - } - - elapsed - }) + b.iter_batched_ref( + || sparse.clone(), + |cloned| cloned.update_rlp_node_level(depth), + criterion::BatchSize::PerIteration, + ) }, ); } diff --git a/crates/trie/sparse/benches/root.rs b/crates/trie/sparse/benches/root.rs index c9f5d655d3e1..9e9d2aec12c7 100644 --- a/crates/trie/sparse/benches/root.rs +++ b/crates/trie/sparse/benches/root.rs @@ -1,4 +1,4 @@ -#![allow(missing_docs, unreachable_pub)] +#![allow(missing_docs)] use alloy_primitives::{map::B256HashMap, B256, U256}; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; @@ -15,11 +15,17 @@ use reth_trie::{ use reth_trie_common::{HashBuilder, Nibbles}; use reth_trie_sparse::SparseTrie; -pub fn calculate_root_from_leaves(c: &mut Criterion) { +fn calculate_root_from_leaves(c: &mut Criterion) { let mut group = c.benchmark_group("calculate root from leaves"); group.sample_size(20); for size in [1_000, 5_000, 10_000, 100_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && size > 5_000 { + continue; + } + let state = generate_test_data(size); // hash builder @@ -29,6 +35,7 @@ pub fn calculate_root_from_leaves(c: &mut Criterion) { hb.add_leaf(Nibbles::unpack(key), &alloy_rlp::encode_fixed_size(value)); } hb.root(); + hb }) }); @@ -44,19 +51,32 @@ pub fn calculate_root_from_leaves(c: &mut Criterion) { .unwrap(); } sparse.root().unwrap(); + sparse }) }); } } -pub fn calculate_root_from_leaves_repeated(c: &mut Criterion) { +fn calculate_root_from_leaves_repeated(c: &mut Criterion) { let mut group = c.benchmark_group("calculate root from leaves repeated"); group.sample_size(20); for init_size in [1_000, 10_000, 100_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && init_size > 10_000 { + continue; + } + let init_state = generate_test_data(init_size); for update_size in [100, 1_000, 5_000, 10_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && update_size > 1_000 { + continue; + } + for num_updates in [1, 3, 5, 10] { let updates = (0..num_updates).map(|_| generate_test_data(update_size)).collect::>(); diff --git a/crates/trie/trie/Cargo.toml b/crates/trie/trie/Cargo.toml index cfce88fa0201..4fa525a74bb3 100644 --- a/crates/trie/trie/Cargo.toml +++ b/crates/trie/trie/Cargo.toml @@ -61,19 +61,20 @@ criterion.workspace = true [features] metrics = ["reth-metrics", "dep:metrics"] serde = [ - "alloy-primitives/serde", - "alloy-consensus/serde", - "alloy-trie/serde", + "alloy-primitives/serde", + "alloy-consensus/serde", + "alloy-trie/serde", "alloy-eips/serde", - "revm/serde", - "reth-trie-common/serde" + "revm/serde", + "reth-trie-common/serde", ] test-utils = [ - "triehash", - "revm/test-utils", - "reth-primitives/test-utils", - "reth-trie-common/test-utils", - "reth-stages-types/test-utils" + "triehash", + "revm/test-utils", + "reth-primitives/test-utils", + "reth-trie-common/test-utils", + "reth-trie-sparse/test-utils", + "reth-stages-types/test-utils", ] [[bench]] diff --git a/crates/trie/trie/benches/hash_post_state.rs b/crates/trie/trie/benches/hash_post_state.rs index da47d01e15c5..129bad5a6780 100644 --- a/crates/trie/trie/benches/hash_post_state.rs +++ b/crates/trie/trie/benches/hash_post_state.rs @@ -10,6 +10,12 @@ pub fn hash_post_state(c: &mut Criterion) { group.sample_size(20); for size in [100, 1_000, 3_000, 5_000, 10_000] { + // Too slow. + #[allow(unexpected_cfgs)] + if cfg!(codspeed) && size > 1_000 { + continue; + } + let state = generate_test_data(size); // sequence diff --git a/crates/trie/trie/benches/trie_root.rs b/crates/trie/trie/benches/trie_root.rs index be6e49545799..840edd32d050 100644 --- a/crates/trie/trie/benches/trie_root.rs +++ b/crates/trie/trie/benches/trie_root.rs @@ -1,10 +1,11 @@ #![allow(missing_docs, unreachable_pub)] use alloy_primitives::B256; -use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use criterion::{criterion_group, criterion_main, Criterion}; use proptest::{prelude::*, strategy::ValueTree, test_runner::TestRunner}; use proptest_arbitrary_interop::arb; use reth_primitives::{Receipt, ReceiptWithBloom}; use reth_trie::triehash::KeccakHasher; +use std::hint::black_box; /// Benchmarks different implementations of the root calculation. pub fn trie_root_benchmark(c: &mut Criterion) { diff --git a/docs/design/database.md b/docs/design/database.md index 48fc8612cbaa..b45c783bc5f3 100644 --- a/docs/design/database.md +++ b/docs/design/database.md @@ -9,7 +9,7 @@ - We want Reth's serialized format to be able to trade off read/write speed for size, depending on who the user is. - To achieve that, we created the [Encode/Decode/Compress/Decompress traits](https://github.com/paradigmxyz/reth/blob/0d9b9a392d4196793736522f3fc2ac804991b45d/crates/interfaces/src/db/table.rs#L9-L36) to make the (de)serialization of database `Table::Key` and `Table::Values` generic. - - This allows for [out-of-the-box benchmarking](https://github.com/paradigmxyz/reth/blob/0d9b9a392d4196793736522f3fc2ac804991b45d/crates/db/benches/encoding_iai.rs#L5) (using [Criterion](https://github.com/bheisler/criterion.rs) and [Iai](https://github.com/bheisler/iai)) + - This allows for [out-of-the-box benchmarking](https://github.com/paradigmxyz/reth/blob/0d9b9a392d4196793736522f3fc2ac804991b45d/crates/db/benches/encoding_iai.rs#L5) (using [Criterion](https://github.com/bheisler/criterion.rs)) - It also enables [out-of-the-box fuzzing](https://github.com/paradigmxyz/reth/blob/0d9b9a392d4196793736522f3fc2ac804991b45d/crates/interfaces/src/db/codecs/fuzz/mod.rs) using [trailofbits/test-fuzz](https://github.com/trailofbits/test-fuzz). - We implemented that trait for the following encoding formats: - [Ethereum-specific Compact Encoding](https://github.com/paradigmxyz/reth/blob/0d9b9a392d4196793736522f3fc2ac804991b45d/crates/codecs/derive/src/compact/mod.rs): A lot of Ethereum datatypes have unnecessary zeros when serialized, or optional (e.g. on empty hashes) which would be nice not to pay in storage costs. From 09c64202b639521d1d58d688f06b50efa55f93db Mon Sep 17 00:00:00 2001 From: DaniPopes <57450786+DaniPopes@users.noreply.github.com> Date: Fri, 20 Dec 2024 13:36:26 +0200 Subject: [PATCH 26/30] chore: fix unused import in net fetcher (#13474) --- crates/net/network/src/transactions/fetcher.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/net/network/src/transactions/fetcher.rs b/crates/net/network/src/transactions/fetcher.rs index e52c73f03e00..b79e78461d89 100644 --- a/crates/net/network/src/transactions/fetcher.rs +++ b/crates/net/network/src/transactions/fetcher.rs @@ -61,7 +61,7 @@ use std::{ time::Duration, }; use tokio::sync::{mpsc::error::TrySendError, oneshot, oneshot::error::RecvError}; -use tracing::{debug, trace}; +use tracing::trace; use validation::FilterOutcome; /// The type responsible for fetching missing transactions from peers. @@ -626,7 +626,7 @@ impl TransactionFetcher { { for hash in &new_announced_hashes { if self.hashes_pending_fetch.contains(hash) { - debug!(target: "net::tx", "`{}` should have been taken out of buffer before packing in a request, breaks invariant `@hashes_pending_fetch` and `@inflight_requests`, `@hashes_fetch_inflight_and_pending_fetch` for `{}`: {:?}", + tracing::debug!(target: "net::tx", "`{}` should have been taken out of buffer before packing in a request, breaks invariant `@hashes_pending_fetch` and `@inflight_requests`, `@hashes_fetch_inflight_and_pending_fetch` for `{}`: {:?}", format!("{:?}", new_announced_hashes), // Assuming new_announced_hashes can be debug-printed directly format!("{:?}", new_announced_hashes), new_announced_hashes.iter().map(|hash| { From 557ebef283a65cf13a1afd885151a6a9896cd0b8 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 12:36:51 +0100 Subject: [PATCH 27/30] chore: use try into directly (#13476) --- crates/ethereum/node/tests/e2e/rpc.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/crates/ethereum/node/tests/e2e/rpc.rs b/crates/ethereum/node/tests/e2e/rpc.rs index 86777c79373d..00070e354e82 100644 --- a/crates/ethereum/node/tests/e2e/rpc.rs +++ b/crates/ethereum/node/tests/e2e/rpc.rs @@ -239,11 +239,7 @@ async fn test_flashbots_validate_v4() -> eyre::Result<()> { }, execution_payload: block_to_payload_v3(payload.block().clone()), blobs_bundle: BlobsBundleV1::new([]), - execution_requests: payload - .requests() - .map(|reqs| alloy_rpc_types_beacon::requests::ExecutionRequestsV4::try_from(&reqs)) - .unwrap() - .unwrap(), + execution_requests: payload.requests().unwrap().try_into().unwrap(), target_blobs_per_block: eip4844::TARGET_BLOBS_PER_BLOCK, signature: Default::default(), }, From d1dcc0b14b35dc7952290fa510473ec9326aecad Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 12:37:58 +0100 Subject: [PATCH 28/30] chore: remove revm default std (#13478) --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 62b54a3bdb4e..e23c3724f3cf 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -426,7 +426,7 @@ reth-trie-sparse = { path = "crates/trie/sparse" } reth-zstd-compressors = { path = "crates/storage/zstd-compressors", default-features = false } # revm -revm = { version = "18.0.0", features = ["std"], default-features = false } +revm = { version = "18.0.0", default-features = false } revm-inspectors = "0.13.0" revm-primitives = { version = "14.0.0", default-features = false } revm-interpreter = { version = "14.0.0", default-features = false } From 06806758db95c0ab88eee67a08422cfda572523a Mon Sep 17 00:00:00 2001 From: Emilia Hane Date: Fri, 20 Dec 2024 13:15:59 +0100 Subject: [PATCH 29/30] chore(consensus): Add docs ab soon deprecated `BeaconConsensusEngine` (#13439) --- crates/consensus/beacon/src/engine/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index dc9e757c2d48..3d70888fdad3 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -167,6 +167,8 @@ type PendingForkchoiceUpdate = /// # Panics /// /// If the future is polled more than once. Leads to undefined state. +/// +/// Note: soon deprecated. See `reth_engine_service::EngineService`. #[must_use = "Future does nothing unless polled"] #[allow(missing_debug_implementations)] pub struct BeaconConsensusEngine From 42af22fc7c574e6d9f95bc0f5c00678288842b07 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Fri, 20 Dec 2024 13:16:42 +0100 Subject: [PATCH 30/30] chore: enable std with arbitrary (#13479) --- crates/trie/common/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/trie/common/Cargo.toml b/crates/trie/common/Cargo.toml index 73d9a3a8577e..4486573b0e7c 100644 --- a/crates/trie/common/Cargo.toml +++ b/crates/trie/common/Cargo.toml @@ -107,6 +107,7 @@ test-utils = [ "reth-codecs/test-utils", ] arbitrary = [ + "std", "dep:reth-codecs", "alloy-trie/arbitrary", "dep:arbitrary",