mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: abstract EthTransactionValidator over ChainSpec (#14162)
This commit is contained in:
@ -196,16 +196,12 @@ where
|
||||
let data_dir = ctx.config().datadir();
|
||||
let pool_config = ctx.pool_config();
|
||||
let blob_store = DiskFileBlobStore::open(data_dir.blobstore(), Default::default())?;
|
||||
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.chain_spec())
|
||||
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.provider().clone())
|
||||
.with_head_timestamp(ctx.head().timestamp)
|
||||
.kzg_settings(ctx.kzg_settings()?)
|
||||
.with_local_transactions_config(pool_config.local_transactions_config.clone())
|
||||
.with_additional_tasks(ctx.config().txpool.additional_validation_tasks)
|
||||
.build_with_tasks(
|
||||
ctx.provider().clone(),
|
||||
ctx.task_executor().clone(),
|
||||
blob_store.clone(),
|
||||
);
|
||||
.build_with_tasks(ctx.task_executor().clone(), blob_store.clone());
|
||||
|
||||
let transaction_pool =
|
||||
reth_transaction_pool::Pool::eth_pool(validator, blob_store, pool_config);
|
||||
|
||||
@ -10,7 +10,7 @@ use crate::{
|
||||
};
|
||||
use futures::{FutureExt, StreamExt};
|
||||
use pin_project::pin_project;
|
||||
use reth_chainspec::{ChainSpecProvider, Hardforks, MAINNET};
|
||||
use reth_chainspec::{ChainSpecProvider, EthereumHardforks, Hardforks};
|
||||
use reth_eth_wire::{
|
||||
protocol::Protocol, DisconnectReason, EthNetworkPrimitives, HelloMessageWithProtocols,
|
||||
};
|
||||
@ -175,7 +175,12 @@ where
|
||||
|
||||
impl<C, Pool> Testnet<C, Pool>
|
||||
where
|
||||
C: StateProviderFactory + BlockReaderIdExt + HeaderProvider + Clone + 'static,
|
||||
C: ChainSpecProvider<ChainSpec: EthereumHardforks>
|
||||
+ StateProviderFactory
|
||||
+ BlockReaderIdExt
|
||||
+ HeaderProvider
|
||||
+ Clone
|
||||
+ 'static,
|
||||
Pool: TransactionPool,
|
||||
{
|
||||
/// Installs an eth pool on each peer
|
||||
@ -184,7 +189,6 @@ where
|
||||
let blob_store = InMemoryBlobStore::default();
|
||||
let pool = TransactionValidationTaskExecutor::eth(
|
||||
peer.client.clone(),
|
||||
MAINNET.clone(),
|
||||
blob_store.clone(),
|
||||
TokioTaskExecutor::default(),
|
||||
);
|
||||
@ -205,7 +209,6 @@ where
|
||||
let blob_store = InMemoryBlobStore::default();
|
||||
let pool = TransactionValidationTaskExecutor::eth(
|
||||
peer.client.clone(),
|
||||
MAINNET.clone(),
|
||||
blob_store.clone(),
|
||||
TokioTaskExecutor::default(),
|
||||
);
|
||||
|
||||
@ -4,10 +4,9 @@ use crate::OpBlockExecutionError;
|
||||
use alloc::{string::ToString, sync::Arc};
|
||||
use alloy_consensus::Transaction;
|
||||
use alloy_primitives::{address, b256, hex, Address, Bytes, B256, U256};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_execution_errors::BlockExecutionError;
|
||||
use reth_optimism_chainspec::OpChainSpec;
|
||||
use reth_optimism_forks::OpHardfork;
|
||||
use reth_optimism_forks::{OpHardfork, OpHardforks};
|
||||
use reth_primitives_traits::BlockBody;
|
||||
use revm::{
|
||||
primitives::{Bytecode, HashMap, SpecId},
|
||||
@ -186,19 +185,20 @@ pub fn parse_l1_info_tx_ecotone(data: &[u8]) -> Result<L1BlockInfo, OpBlockExecu
|
||||
}
|
||||
|
||||
/// An extension trait for [`L1BlockInfo`] that allows us to calculate the L1 cost of a transaction
|
||||
/// based off of the [`ChainSpec`]'s activated hardfork.
|
||||
/// based off of the chain spec's activated hardfork.
|
||||
pub trait RethL1BlockInfo {
|
||||
/// Forwards an L1 transaction calculation to revm and returns the gas cost.
|
||||
///
|
||||
/// ### Takes
|
||||
/// - `chain_spec`: The [`ChainSpec`] for the node.
|
||||
/// - `chain_spec`: The chain spec for the node.
|
||||
/// - `timestamp`: The timestamp of the current block.
|
||||
/// - `input`: The calldata of the transaction.
|
||||
/// - `is_deposit`: Whether or not the transaction is a deposit.
|
||||
fn l1_tx_data_fee(
|
||||
&mut self,
|
||||
chain_spec: &ChainSpec,
|
||||
chain_spec: impl OpHardforks,
|
||||
timestamp: u64,
|
||||
block: u64,
|
||||
input: &[u8],
|
||||
is_deposit: bool,
|
||||
) -> Result<U256, BlockExecutionError>;
|
||||
@ -206,13 +206,14 @@ pub trait RethL1BlockInfo {
|
||||
/// Computes the data gas cost for an L2 transaction.
|
||||
///
|
||||
/// ### Takes
|
||||
/// - `chain_spec`: The [`ChainSpec`] for the node.
|
||||
/// - `chain_spec`: The chain spec for the node.
|
||||
/// - `timestamp`: The timestamp of the current block.
|
||||
/// - `input`: The calldata of the transaction.
|
||||
fn l1_data_gas(
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
chain_spec: impl OpHardforks,
|
||||
timestamp: u64,
|
||||
block_number: u64,
|
||||
input: &[u8],
|
||||
) -> Result<U256, BlockExecutionError>;
|
||||
}
|
||||
@ -220,8 +221,9 @@ pub trait RethL1BlockInfo {
|
||||
impl RethL1BlockInfo for L1BlockInfo {
|
||||
fn l1_tx_data_fee(
|
||||
&mut self,
|
||||
chain_spec: &ChainSpec,
|
||||
chain_spec: impl OpHardforks,
|
||||
timestamp: u64,
|
||||
block_number: u64,
|
||||
input: &[u8],
|
||||
is_deposit: bool,
|
||||
) -> Result<U256, BlockExecutionError> {
|
||||
@ -229,13 +231,13 @@ impl RethL1BlockInfo for L1BlockInfo {
|
||||
return Ok(U256::ZERO)
|
||||
}
|
||||
|
||||
let spec_id = if chain_spec.is_fork_active_at_timestamp(OpHardfork::Fjord, timestamp) {
|
||||
let spec_id = if chain_spec.is_fjord_active_at_timestamp(timestamp) {
|
||||
SpecId::FJORD
|
||||
} else if chain_spec.is_fork_active_at_timestamp(OpHardfork::Ecotone, timestamp) {
|
||||
} else if chain_spec.is_ecotone_active_at_timestamp(timestamp) {
|
||||
SpecId::ECOTONE
|
||||
} else if chain_spec.is_fork_active_at_timestamp(OpHardfork::Regolith, timestamp) {
|
||||
} else if chain_spec.is_regolith_active_at_timestamp(timestamp) {
|
||||
SpecId::REGOLITH
|
||||
} else if chain_spec.is_fork_active_at_timestamp(OpHardfork::Bedrock, timestamp) {
|
||||
} else if chain_spec.is_bedrock_active_at_block(block_number) {
|
||||
SpecId::BEDROCK
|
||||
} else {
|
||||
return Err(OpBlockExecutionError::L1BlockInfoError {
|
||||
@ -248,15 +250,16 @@ impl RethL1BlockInfo for L1BlockInfo {
|
||||
|
||||
fn l1_data_gas(
|
||||
&self,
|
||||
chain_spec: &ChainSpec,
|
||||
chain_spec: impl OpHardforks,
|
||||
timestamp: u64,
|
||||
block_number: u64,
|
||||
input: &[u8],
|
||||
) -> Result<U256, BlockExecutionError> {
|
||||
let spec_id = if chain_spec.is_fork_active_at_timestamp(OpHardfork::Fjord, timestamp) {
|
||||
let spec_id = if chain_spec.is_fjord_active_at_timestamp(timestamp) {
|
||||
SpecId::FJORD
|
||||
} else if chain_spec.is_fork_active_at_timestamp(OpHardfork::Regolith, timestamp) {
|
||||
} else if chain_spec.is_regolith_active_at_timestamp(timestamp) {
|
||||
SpecId::REGOLITH
|
||||
} else if chain_spec.is_fork_active_at_timestamp(OpHardfork::Bedrock, timestamp) {
|
||||
} else if chain_spec.is_bedrock_active_at_block(block_number) {
|
||||
SpecId::BEDROCK
|
||||
} else {
|
||||
return Err(OpBlockExecutionError::L1BlockInfoError {
|
||||
|
||||
@ -13,7 +13,9 @@ use reth_evm::{
|
||||
execute::BasicBlockExecutorProvider, ConfigureEvm, ConfigureEvmEnv, ConfigureEvmFor,
|
||||
};
|
||||
use reth_network::{NetworkConfig, NetworkHandle, NetworkManager, NetworkPrimitives, PeersInfo};
|
||||
use reth_node_api::{AddOnsContext, FullNodeComponents, NodeAddOns, PrimitivesTy, TxTy};
|
||||
use reth_node_api::{
|
||||
AddOnsContext, FullNodeComponents, NodeAddOns, NodePrimitives, PrimitivesTy, TxTy,
|
||||
};
|
||||
use reth_node_builder::{
|
||||
components::{
|
||||
ComponentsBuilder, ConsensusBuilder, ExecutorBuilder, NetworkBuilder,
|
||||
@ -26,6 +28,7 @@ use reth_node_builder::{
|
||||
use reth_optimism_chainspec::OpChainSpec;
|
||||
use reth_optimism_consensus::OpBeaconConsensus;
|
||||
use reth_optimism_evm::{BasicOpReceiptBuilder, OpEvmConfig, OpExecutionStrategyFactory};
|
||||
use reth_optimism_forks::OpHardforks;
|
||||
use reth_optimism_payload_builder::{
|
||||
builder::OpPayloadTransactions,
|
||||
config::{OpBuilderConfig, OpDAConfig},
|
||||
@ -357,7 +360,12 @@ pub struct OpPoolBuilder {
|
||||
|
||||
impl<Node> PoolBuilder<Node> for OpPoolBuilder
|
||||
where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = OpChainSpec, Primitives = OpPrimitives>>,
|
||||
Node: FullNodeTypes<
|
||||
Types: NodeTypes<
|
||||
ChainSpec: OpHardforks,
|
||||
Primitives: NodePrimitives<SignedTx = OpTransactionSigned>,
|
||||
>,
|
||||
>,
|
||||
{
|
||||
type Pool = OpTransactionPool<Node::Provider, DiskFileBlobStore>;
|
||||
|
||||
@ -366,24 +374,22 @@ where
|
||||
let data_dir = ctx.config().datadir();
|
||||
let blob_store = DiskFileBlobStore::open(data_dir.blobstore(), Default::default())?;
|
||||
|
||||
let validator = TransactionValidationTaskExecutor::eth_builder(Arc::new(
|
||||
ctx.chain_spec().inner.clone(),
|
||||
))
|
||||
.no_eip4844()
|
||||
.with_head_timestamp(ctx.head().timestamp)
|
||||
.kzg_settings(ctx.kzg_settings()?)
|
||||
.with_additional_tasks(
|
||||
pool_config_overrides
|
||||
.additional_validation_tasks
|
||||
.unwrap_or_else(|| ctx.config().txpool.additional_validation_tasks),
|
||||
)
|
||||
.build_with_tasks(ctx.provider().clone(), ctx.task_executor().clone(), blob_store.clone())
|
||||
.map(|validator| {
|
||||
OpTransactionValidator::new(validator)
|
||||
// In --dev mode we can't require gas fees because we're unable to decode
|
||||
// the L1 block info
|
||||
.require_l1_data_gas_fee(!ctx.config().dev.dev)
|
||||
});
|
||||
let validator = TransactionValidationTaskExecutor::eth_builder(ctx.provider().clone())
|
||||
.no_eip4844()
|
||||
.with_head_timestamp(ctx.head().timestamp)
|
||||
.kzg_settings(ctx.kzg_settings()?)
|
||||
.with_additional_tasks(
|
||||
pool_config_overrides
|
||||
.additional_validation_tasks
|
||||
.unwrap_or_else(|| ctx.config().txpool.additional_validation_tasks),
|
||||
)
|
||||
.build_with_tasks(ctx.task_executor().clone(), blob_store.clone())
|
||||
.map(|validator| {
|
||||
OpTransactionValidator::new(validator)
|
||||
// In --dev mode we can't require gas fees because we're unable to decode
|
||||
// the L1 block info
|
||||
.require_l1_data_gas_fee(!ctx.config().dev.dev)
|
||||
});
|
||||
|
||||
let transaction_pool = reth_transaction_pool::Pool::new(
|
||||
validator,
|
||||
|
||||
@ -6,16 +6,16 @@ use alloy_eips::eip2718::Encodable2718;
|
||||
use alloy_primitives::{Address, TxHash, TxKind, U256};
|
||||
use op_alloy_consensus::OpTypedTransaction;
|
||||
use parking_lot::RwLock;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_node_api::{Block, BlockBody};
|
||||
use reth_optimism_evm::RethL1BlockInfo;
|
||||
use reth_optimism_primitives::{OpBlock, OpTransactionSigned};
|
||||
use reth_optimism_forks::OpHardforks;
|
||||
use reth_optimism_primitives::OpTransactionSigned;
|
||||
use reth_primitives::{
|
||||
transaction::TransactionConversionError, GotExpected, InvalidTransactionError, Recovered,
|
||||
SealedBlock,
|
||||
};
|
||||
use reth_primitives_traits::SignedTransaction;
|
||||
use reth_provider::{BlockReaderIdExt, StateProviderFactory};
|
||||
use reth_provider::{BlockReaderIdExt, ChainSpecProvider, StateProviderFactory};
|
||||
use reth_revm::L1BlockInfo;
|
||||
use reth_transaction_pool::{
|
||||
CoinbaseTipOrdering, EthBlobTransactionSidecar, EthPoolTransaction, EthPooledTransaction,
|
||||
@ -242,7 +242,10 @@ pub struct OpTransactionValidator<Client, Tx> {
|
||||
|
||||
impl<Client, Tx> OpTransactionValidator<Client, Tx> {
|
||||
/// Returns the configured chain spec
|
||||
pub fn chain_spec(&self) -> &Arc<ChainSpec> {
|
||||
pub fn chain_spec(&self) -> Arc<Client::ChainSpec>
|
||||
where
|
||||
Client: ChainSpecProvider,
|
||||
{
|
||||
self.inner.chain_spec()
|
||||
}
|
||||
|
||||
@ -256,6 +259,11 @@ impl<Client, Tx> OpTransactionValidator<Client, Tx> {
|
||||
self.block_info.timestamp.load(Ordering::Relaxed)
|
||||
}
|
||||
|
||||
/// Returns the current block number.
|
||||
fn block_number(&self) -> u64 {
|
||||
self.block_info.number.load(Ordering::Relaxed)
|
||||
}
|
||||
|
||||
/// Whether to ensure that the transaction's sender has enough balance to also cover the L1 gas
|
||||
/// fee.
|
||||
pub fn require_l1_data_gas_fee(self, require_l1_data_gas_fee: bool) -> Self {
|
||||
@ -271,7 +279,7 @@ impl<Client, Tx> OpTransactionValidator<Client, Tx> {
|
||||
|
||||
impl<Client, Tx> OpTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt,
|
||||
Client: ChainSpecProvider<ChainSpec: OpHardforks> + StateProviderFactory + BlockReaderIdExt,
|
||||
Tx: EthPoolTransaction<Consensus = OpTransactionSigned>,
|
||||
{
|
||||
/// Create a new [`OpTransactionValidator`].
|
||||
@ -284,6 +292,7 @@ where
|
||||
// so that we will accept txs into the pool before the first block
|
||||
if block.header().number() == 0 {
|
||||
this.block_info.timestamp.store(block.header().timestamp(), Ordering::Relaxed);
|
||||
this.block_info.number.store(block.header().number(), Ordering::Relaxed);
|
||||
} else {
|
||||
this.update_l1_block_info(block.header(), block.body().transactions().first());
|
||||
}
|
||||
@ -309,6 +318,7 @@ where
|
||||
T: Transaction,
|
||||
{
|
||||
self.block_info.timestamp.store(header.timestamp(), Ordering::Relaxed);
|
||||
self.block_info.number.store(header.number(), Ordering::Relaxed);
|
||||
|
||||
if let Some(Ok(cost_addition)) = tx.map(reth_optimism_evm::extract_l1_info_from_tx) {
|
||||
*self.block_info.l1_block_info.write() = cost_addition;
|
||||
@ -357,6 +367,7 @@ where
|
||||
let cost_addition = match l1_block_info.l1_tx_data_fee(
|
||||
self.chain_spec(),
|
||||
self.block_timestamp(),
|
||||
self.block_number(),
|
||||
&encoded,
|
||||
false,
|
||||
) {
|
||||
@ -404,7 +415,7 @@ where
|
||||
|
||||
impl<Client, Tx> TransactionValidator for OpTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt<Block = OpBlock>,
|
||||
Client: ChainSpecProvider<ChainSpec: OpHardforks> + StateProviderFactory + BlockReaderIdExt,
|
||||
Tx: EthPoolTransaction<Consensus = OpTransactionSigned>,
|
||||
{
|
||||
type Transaction = Tx;
|
||||
@ -443,6 +454,8 @@ pub struct OpL1BlockInfo {
|
||||
l1_block_info: RwLock<L1BlockInfo>,
|
||||
/// Current block timestamp.
|
||||
timestamp: AtomicU64,
|
||||
/// Current block number.
|
||||
number: AtomicU64,
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@ -451,7 +464,7 @@ mod tests {
|
||||
use alloy_eips::eip2718::Encodable2718;
|
||||
use alloy_primitives::{PrimitiveSignature as Signature, TxKind, U256};
|
||||
use op_alloy_consensus::{OpTypedTransaction, TxDeposit};
|
||||
use reth_chainspec::MAINNET;
|
||||
use reth_optimism_chainspec::OP_MAINNET;
|
||||
use reth_optimism_primitives::OpTransactionSigned;
|
||||
use reth_primitives::Recovered;
|
||||
use reth_provider::test_utils::MockEthProvider;
|
||||
@ -461,11 +474,11 @@ mod tests {
|
||||
};
|
||||
#[test]
|
||||
fn validate_optimism_transaction() {
|
||||
let client = MockEthProvider::default();
|
||||
let validator = EthTransactionValidatorBuilder::new(MAINNET.clone())
|
||||
let client = MockEthProvider::default().with_chain_spec(OP_MAINNET.clone());
|
||||
let validator = EthTransactionValidatorBuilder::new(client)
|
||||
.no_shanghai()
|
||||
.no_cancun()
|
||||
.build(client, InMemoryBlobStore::default());
|
||||
.build(InMemoryBlobStore::default());
|
||||
let validator = OpTransactionValidator::new(validator);
|
||||
|
||||
let origin = TransactionOrigin::External;
|
||||
|
||||
@ -59,6 +59,8 @@ where
|
||||
/// deposit transaction.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct OpReceiptFieldsBuilder {
|
||||
/// Block number.
|
||||
pub block_number: u64,
|
||||
/// Block timestamp.
|
||||
pub block_timestamp: u64,
|
||||
/// The L1 fee for transaction.
|
||||
@ -87,8 +89,9 @@ pub struct OpReceiptFieldsBuilder {
|
||||
|
||||
impl OpReceiptFieldsBuilder {
|
||||
/// Returns a new builder.
|
||||
pub const fn new(block_timestamp: u64) -> Self {
|
||||
pub const fn new(block_timestamp: u64, block_number: u64) -> Self {
|
||||
Self {
|
||||
block_number,
|
||||
block_timestamp,
|
||||
l1_fee: None,
|
||||
l1_data_gas: None,
|
||||
@ -110,18 +113,19 @@ impl OpReceiptFieldsBuilder {
|
||||
l1_block_info: &mut revm::L1BlockInfo,
|
||||
) -> Result<Self, OpEthApiError> {
|
||||
let raw_tx = tx.encoded_2718();
|
||||
let block_number = self.block_number;
|
||||
let timestamp = self.block_timestamp;
|
||||
|
||||
self.l1_fee = Some(
|
||||
l1_block_info
|
||||
.l1_tx_data_fee(chain_spec, timestamp, &raw_tx, tx.is_deposit())
|
||||
.l1_tx_data_fee(chain_spec, timestamp, block_number, &raw_tx, tx.is_deposit())
|
||||
.map_err(|_| OpEthApiError::L1BlockFeeError)?
|
||||
.saturating_to(),
|
||||
);
|
||||
|
||||
self.l1_data_gas = Some(
|
||||
l1_block_info
|
||||
.l1_data_gas(chain_spec, timestamp, &raw_tx)
|
||||
.l1_data_gas(chain_spec, timestamp, block_number, &raw_tx)
|
||||
.map_err(|_| OpEthApiError::L1BlockGasError)?
|
||||
.saturating_add(l1_block_info.l1_fee_overhead.unwrap_or_default())
|
||||
.saturating_to(),
|
||||
@ -154,6 +158,7 @@ impl OpReceiptFieldsBuilder {
|
||||
/// Builds the [`OpTransactionReceiptFields`] object.
|
||||
pub const fn build(self) -> OpTransactionReceiptFields {
|
||||
let Self {
|
||||
block_number: _, // used to compute other fields
|
||||
block_timestamp: _, // used to compute other fields
|
||||
l1_fee,
|
||||
l1_data_gas: l1_gas_used,
|
||||
@ -202,6 +207,7 @@ impl OpReceiptBuilder {
|
||||
l1_block_info: &mut revm::L1BlockInfo,
|
||||
) -> Result<Self, OpEthApiError> {
|
||||
let timestamp = meta.timestamp;
|
||||
let block_number = meta.block_number;
|
||||
let core_receipt =
|
||||
build_receipt(transaction, meta, receipt, all_receipts, None, |receipt_with_bloom| {
|
||||
match receipt {
|
||||
@ -222,7 +228,7 @@ impl OpReceiptBuilder {
|
||||
}
|
||||
})?;
|
||||
|
||||
let op_receipt_fields = OpReceiptFieldsBuilder::new(timestamp)
|
||||
let op_receipt_fields = OpReceiptFieldsBuilder::new(timestamp, block_number)
|
||||
.l1_block_info(chain_spec, transaction, l1_block_info)?
|
||||
.build();
|
||||
|
||||
@ -304,7 +310,7 @@ mod test {
|
||||
// test
|
||||
assert!(OP_MAINNET.is_fjord_active_at_timestamp(BLOCK_124665056_TIMESTAMP));
|
||||
|
||||
let receipt_meta = OpReceiptFieldsBuilder::new(BLOCK_124665056_TIMESTAMP)
|
||||
let receipt_meta = OpReceiptFieldsBuilder::new(BLOCK_124665056_TIMESTAMP, 124665056)
|
||||
.l1_block_info(&OP_MAINNET, &tx_1, &mut l1_block_info)
|
||||
.expect("should parse revm l1 info")
|
||||
.build();
|
||||
@ -370,7 +376,7 @@ mod test {
|
||||
let tx = hex!("02f86c8221058034839a4ae283021528942f16386bb37709016023232523ff6d9daf444be380841249c58bc080a001b927eda2af9b00b52a57be0885e0303c39dd2831732e14051c2336470fd468a0681bf120baf562915841a48601c2b54a6742511e535cf8f71c95115af7ff63bd");
|
||||
let tx_1 = OpTransactionSigned::decode_2718(&mut &tx[..]).unwrap();
|
||||
|
||||
let receipt_meta = OpReceiptFieldsBuilder::new(1730216981)
|
||||
let receipt_meta = OpReceiptFieldsBuilder::new(1730216981, 21713817)
|
||||
.l1_block_info(&BASE_MAINNET, &tx_1, &mut l1_block_info)
|
||||
.expect("should parse revm l1 info")
|
||||
.build();
|
||||
|
||||
@ -15,7 +15,7 @@ use alloy_primitives::{
|
||||
Address, BlockHash, BlockNumber, Bytes, StorageKey, StorageValue, TxHash, TxNumber, B256, U256,
|
||||
};
|
||||
use parking_lot::Mutex;
|
||||
use reth_chainspec::{ChainInfo, ChainSpec};
|
||||
use reth_chainspec::{ChainInfo, EthChainSpec};
|
||||
use reth_db::mock::{DatabaseMock, TxMock};
|
||||
use reth_db_api::models::{AccountBeforeTx, StoredBlockBodyIndices};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
@ -43,8 +43,8 @@ use std::{
|
||||
};
|
||||
|
||||
/// A mock implementation for Provider interfaces.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct MockEthProvider<T = TransactionSigned> {
|
||||
#[derive(Debug)]
|
||||
pub struct MockEthProvider<T = TransactionSigned, ChainSpec = reth_chainspec::ChainSpec> {
|
||||
/// Local block store
|
||||
pub blocks: Arc<Mutex<HashMap<B256, Block<T>>>>,
|
||||
/// Local header store
|
||||
@ -57,6 +57,18 @@ pub struct MockEthProvider<T = TransactionSigned> {
|
||||
pub state_roots: Arc<Mutex<Vec<B256>>>,
|
||||
}
|
||||
|
||||
impl<T, ChainSpec> Clone for MockEthProvider<T, ChainSpec> {
|
||||
fn clone(&self) -> Self {
|
||||
Self {
|
||||
blocks: self.blocks.clone(),
|
||||
headers: self.headers.clone(),
|
||||
accounts: self.accounts.clone(),
|
||||
chain_spec: self.chain_spec.clone(),
|
||||
state_roots: self.state_roots.clone(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> MockEthProvider<T> {
|
||||
/// Create a new, empty instance
|
||||
pub fn new() -> Self {
|
||||
@ -68,7 +80,9 @@ impl<T> MockEthProvider<T> {
|
||||
state_roots: Default::default(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T, ChainSpec> MockEthProvider<T, ChainSpec> {
|
||||
/// Add block to local block store
|
||||
pub fn add_block(&self, hash: B256, block: Block<T>) {
|
||||
self.add_header(hash, block.header.clone());
|
||||
@ -111,6 +125,17 @@ impl<T> MockEthProvider<T> {
|
||||
pub fn add_state_root(&self, state_root: B256) {
|
||||
self.state_roots.lock().push(state_root);
|
||||
}
|
||||
|
||||
/// Set chain spec.
|
||||
pub fn with_chain_spec<C>(self, chain_spec: C) -> MockEthProvider<T, C> {
|
||||
MockEthProvider {
|
||||
blocks: self.blocks,
|
||||
headers: self.headers,
|
||||
accounts: self.accounts,
|
||||
chain_spec: Arc::new(chain_spec),
|
||||
state_roots: self.state_roots,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for MockEthProvider {
|
||||
@ -162,16 +187,20 @@ pub struct MockNode;
|
||||
|
||||
impl NodeTypes for MockNode {
|
||||
type Primitives = EthPrimitives;
|
||||
type ChainSpec = ChainSpec;
|
||||
type ChainSpec = reth_chainspec::ChainSpec;
|
||||
type StateCommitment = MerklePatriciaTrie;
|
||||
type Storage = EthStorage;
|
||||
}
|
||||
|
||||
impl<T: Transaction> StateCommitmentProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> StateCommitmentProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
type StateCommitment = <MockNode as NodeTypes>::StateCommitment;
|
||||
}
|
||||
|
||||
impl<T: Transaction> DatabaseProviderFactory for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> DatabaseProviderFactory
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
type DB = DatabaseMock;
|
||||
type Provider = DatabaseProvider<TxMock, MockNode>;
|
||||
type ProviderRW = DatabaseProvider<TxMock, MockNode>;
|
||||
@ -185,7 +214,7 @@ impl<T: Transaction> DatabaseProviderFactory for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> HeaderProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> HeaderProvider for MockEthProvider<T, ChainSpec> {
|
||||
type Header = Header;
|
||||
|
||||
fn header(&self, block_hash: &BlockHash) -> ProviderResult<Option<Header>> {
|
||||
@ -244,7 +273,9 @@ impl<T: Transaction> HeaderProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> ChainSpecProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec + 'static> ChainSpecProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
type ChainSpec = ChainSpec;
|
||||
|
||||
fn chain_spec(&self) -> Arc<ChainSpec> {
|
||||
@ -252,7 +283,9 @@ impl<T: Transaction> ChainSpecProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: SignedTransaction> TransactionsProvider for MockEthProvider<T> {
|
||||
impl<T: SignedTransaction, ChainSpec: EthChainSpec> TransactionsProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
type Transaction = T;
|
||||
|
||||
fn transaction_id(&self, tx_hash: TxHash) -> ProviderResult<Option<TxNumber>> {
|
||||
@ -391,7 +424,7 @@ impl<T: SignedTransaction> TransactionsProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> ReceiptProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> ReceiptProvider for MockEthProvider<T, ChainSpec> {
|
||||
type Receipt = Receipt;
|
||||
|
||||
fn receipt(&self, _id: TxNumber) -> ProviderResult<Option<Receipt>> {
|
||||
@ -414,9 +447,12 @@ impl<T: Transaction> ReceiptProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> ReceiptProviderIdExt for MockEthProvider<T> {}
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> ReceiptProviderIdExt
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
}
|
||||
|
||||
impl<T: Transaction> BlockHashReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> BlockHashReader for MockEthProvider<T, ChainSpec> {
|
||||
fn block_hash(&self, number: u64) -> ProviderResult<Option<B256>> {
|
||||
let lock = self.blocks.lock();
|
||||
|
||||
@ -440,7 +476,7 @@ impl<T: Transaction> BlockHashReader for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> BlockNumReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> BlockNumReader for MockEthProvider<T, ChainSpec> {
|
||||
fn chain_info(&self) -> ProviderResult<ChainInfo> {
|
||||
let best_block_number = self.best_block_number()?;
|
||||
let lock = self.headers.lock();
|
||||
@ -471,7 +507,7 @@ impl<T: Transaction> BlockNumReader for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> BlockIdReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> BlockIdReader for MockEthProvider<T, ChainSpec> {
|
||||
fn pending_block_num_hash(&self) -> ProviderResult<Option<alloy_eips::BlockNumHash>> {
|
||||
Ok(None)
|
||||
}
|
||||
@ -485,7 +521,7 @@ impl<T: Transaction> BlockIdReader for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: SignedTransaction> BlockReader for MockEthProvider<T> {
|
||||
impl<T: SignedTransaction, ChainSpec: EthChainSpec> BlockReader for MockEthProvider<T, ChainSpec> {
|
||||
type Block = Block<T>;
|
||||
|
||||
fn find_block_by_hash(
|
||||
@ -559,7 +595,9 @@ impl<T: SignedTransaction> BlockReader for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: SignedTransaction> BlockReaderIdExt for MockEthProvider<T> {
|
||||
impl<T: SignedTransaction, ChainSpec: EthChainSpec> BlockReaderIdExt
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn block_by_id(&self, id: BlockId) -> ProviderResult<Option<Block<T>>> {
|
||||
match id {
|
||||
BlockId::Number(num) => self.block_by_number_or_tag(num),
|
||||
@ -586,13 +624,15 @@ impl<T: SignedTransaction> BlockReaderIdExt for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> AccountReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> AccountReader for MockEthProvider<T, ChainSpec> {
|
||||
fn basic_account(&self, address: &Address) -> ProviderResult<Option<Account>> {
|
||||
Ok(self.accounts.lock().get(address).cloned().map(|a| a.account))
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> StageCheckpointReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> StageCheckpointReader
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn get_stage_checkpoint(&self, _id: StageId) -> ProviderResult<Option<StageCheckpoint>> {
|
||||
Ok(None)
|
||||
}
|
||||
@ -606,7 +646,7 @@ impl<T: Transaction> StageCheckpointReader for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> StateRootProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> StateRootProvider for MockEthProvider<T, ChainSpec> {
|
||||
fn state_root(&self, _state: HashedPostState) -> ProviderResult<B256> {
|
||||
Ok(self.state_roots.lock().pop().unwrap_or_default())
|
||||
}
|
||||
@ -632,7 +672,9 @@ impl<T: Transaction> StateRootProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> StorageRootProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> StorageRootProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn storage_root(
|
||||
&self,
|
||||
_address: Address,
|
||||
@ -660,7 +702,7 @@ impl<T: Transaction> StorageRootProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> StateProofProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> StateProofProvider for MockEthProvider<T, ChainSpec> {
|
||||
fn proof(
|
||||
&self,
|
||||
_input: TrieInput,
|
||||
@ -687,13 +729,17 @@ impl<T: Transaction> StateProofProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> HashedPostStateProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> HashedPostStateProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn hashed_post_state(&self, _state: &revm::db::BundleState) -> HashedPostState {
|
||||
HashedPostState::default()
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> StateProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec + 'static> StateProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn storage(
|
||||
&self,
|
||||
account: Address,
|
||||
@ -716,7 +762,9 @@ impl<T: Transaction> StateProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: SignedTransaction> StateProviderFactory for MockEthProvider<T> {
|
||||
impl<T: SignedTransaction, ChainSpec: EthChainSpec + 'static> StateProviderFactory
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn latest(&self) -> ProviderResult<StateProviderBox> {
|
||||
Ok(Box::new(self.clone()))
|
||||
}
|
||||
@ -768,7 +816,9 @@ impl<T: SignedTransaction> StateProviderFactory for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> WithdrawalsProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> WithdrawalsProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn withdrawals_by_block(
|
||||
&self,
|
||||
_id: BlockHashOrNumber,
|
||||
@ -778,13 +828,15 @@ impl<T: Transaction> WithdrawalsProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> OmmersProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> OmmersProvider for MockEthProvider<T, ChainSpec> {
|
||||
fn ommers(&self, _id: BlockHashOrNumber) -> ProviderResult<Option<Vec<Header>>> {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> BlockBodyIndicesProvider for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> BlockBodyIndicesProvider
|
||||
for MockEthProvider<T, ChainSpec>
|
||||
{
|
||||
fn block_body_indices(&self, _num: u64) -> ProviderResult<Option<StoredBlockBodyIndices>> {
|
||||
Ok(None)
|
||||
}
|
||||
@ -796,7 +848,7 @@ impl<T: Transaction> BlockBodyIndicesProvider for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> ChangeSetReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> ChangeSetReader for MockEthProvider<T, ChainSpec> {
|
||||
fn account_block_changeset(
|
||||
&self,
|
||||
_block_number: BlockNumber,
|
||||
@ -805,7 +857,7 @@ impl<T: Transaction> ChangeSetReader for MockEthProvider<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Transaction> StateReader for MockEthProvider<T> {
|
||||
impl<T: Transaction, ChainSpec: EthChainSpec> StateReader for MockEthProvider<T, ChainSpec> {
|
||||
type Receipt = Receipt;
|
||||
|
||||
fn get_state(&self, _block: BlockNumber) -> ProviderResult<Option<ExecutionOutcome>> {
|
||||
|
||||
@ -82,12 +82,14 @@
|
||||
//! use reth_chainspec::MAINNET;
|
||||
//! use reth_storage_api::StateProviderFactory;
|
||||
//! use reth_tasks::TokioTaskExecutor;
|
||||
//! use reth_chainspec::ChainSpecProvider;
|
||||
//! use reth_transaction_pool::{TransactionValidationTaskExecutor, Pool, TransactionPool};
|
||||
//! use reth_transaction_pool::blobstore::InMemoryBlobStore;
|
||||
//! async fn t<C>(client: C) where C: StateProviderFactory + Clone + 'static{
|
||||
//! use reth_chainspec::EthereumHardforks;
|
||||
//! async fn t<C>(client: C) where C: ChainSpecProvider<ChainSpec: EthereumHardforks> + StateProviderFactory + Clone + 'static{
|
||||
//! let blob_store = InMemoryBlobStore::default();
|
||||
//! let pool = Pool::eth_pool(
|
||||
//! TransactionValidationTaskExecutor::eth(client, MAINNET.clone(), blob_store.clone(), TokioTaskExecutor::default()),
|
||||
//! TransactionValidationTaskExecutor::eth(client, blob_store.clone(), TokioTaskExecutor::default()),
|
||||
//! blob_store,
|
||||
//! Default::default(),
|
||||
//! );
|
||||
@ -126,7 +128,7 @@
|
||||
//! let manager = TaskManager::new(rt.handle().clone());
|
||||
//! let executor = manager.executor();
|
||||
//! let pool = Pool::eth_pool(
|
||||
//! TransactionValidationTaskExecutor::eth(client.clone(), MAINNET.clone(), blob_store.clone(), executor.clone()),
|
||||
//! TransactionValidationTaskExecutor::eth(client.clone(), blob_store.clone(), executor.clone()),
|
||||
//! blob_store,
|
||||
//! Default::default(),
|
||||
//! );
|
||||
@ -174,6 +176,7 @@ use crate::{identifier::TransactionId, pool::PoolInner};
|
||||
use alloy_eips::eip4844::{BlobAndProofV1, BlobTransactionSidecar};
|
||||
use alloy_primitives::{Address, TxHash, B256, U256};
|
||||
use aquamarine as _;
|
||||
use reth_chainspec::{ChainSpecProvider, EthereumHardforks};
|
||||
use reth_eth_wire_types::HandleMempoolData;
|
||||
use reth_execution_types::ChangedAccount;
|
||||
use reth_primitives::Recovered;
|
||||
@ -280,7 +283,8 @@ where
|
||||
|
||||
impl<Client, S> EthTransactionPool<Client, S>
|
||||
where
|
||||
Client: StateProviderFactory + Clone + 'static,
|
||||
Client:
|
||||
ChainSpecProvider<ChainSpec: EthereumHardforks> + StateProviderFactory + Clone + 'static,
|
||||
S: BlobStore,
|
||||
{
|
||||
/// Returns a new [`Pool`] that uses the default [`TransactionValidationTaskExecutor`] when
|
||||
@ -292,15 +296,16 @@ where
|
||||
/// use reth_chainspec::MAINNET;
|
||||
/// use reth_storage_api::StateProviderFactory;
|
||||
/// use reth_tasks::TokioTaskExecutor;
|
||||
/// use reth_chainspec::ChainSpecProvider;
|
||||
/// use reth_transaction_pool::{
|
||||
/// blobstore::InMemoryBlobStore, Pool, TransactionValidationTaskExecutor,
|
||||
/// };
|
||||
/// # fn t<C>(client: C) where C: StateProviderFactory + Clone + 'static {
|
||||
/// use reth_chainspec::EthereumHardforks;
|
||||
/// # fn t<C>(client: C) where C: ChainSpecProvider<ChainSpec: EthereumHardforks> + StateProviderFactory + Clone + 'static {
|
||||
/// let blob_store = InMemoryBlobStore::default();
|
||||
/// let pool = Pool::eth_pool(
|
||||
/// TransactionValidationTaskExecutor::eth(
|
||||
/// client,
|
||||
/// MAINNET.clone(),
|
||||
/// blob_store.clone(),
|
||||
/// TokioTaskExecutor::default(),
|
||||
/// ),
|
||||
|
||||
@ -677,7 +677,6 @@ mod tests {
|
||||
};
|
||||
use alloy_eips::eip2718::Decodable2718;
|
||||
use alloy_primitives::{hex, U256};
|
||||
use reth_chainspec::MAINNET;
|
||||
use reth_fs_util as fs;
|
||||
use reth_primitives::{PooledTransaction, TransactionSigned};
|
||||
use reth_provider::test_utils::{ExtendedAccount, MockEthProvider};
|
||||
@ -706,8 +705,7 @@ mod tests {
|
||||
let sender = hex!("1f9090aaE28b8a3dCeaDf281B0F12828e676c326").into();
|
||||
provider.add_account(sender, ExtendedAccount::new(42, U256::MAX));
|
||||
let blob_store = InMemoryBlobStore::default();
|
||||
let validator = EthTransactionValidatorBuilder::new(MAINNET.clone())
|
||||
.build(provider, blob_store.clone());
|
||||
let validator = EthTransactionValidatorBuilder::new(provider).build(blob_store.clone());
|
||||
|
||||
let txpool = Pool::new(
|
||||
validator.clone(),
|
||||
|
||||
@ -22,7 +22,7 @@ use alloy_eips::{
|
||||
eip1559::ETHEREUM_BLOCK_GAS_LIMIT,
|
||||
eip4844::{env_settings::EnvKzgSettings, MAX_BLOBS_PER_BLOCK},
|
||||
};
|
||||
use reth_chainspec::{ChainSpec, EthereumHardforks};
|
||||
use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks};
|
||||
use reth_primitives::{InvalidTransactionError, SealedBlock};
|
||||
use reth_primitives_traits::{Block, GotExpected};
|
||||
use reth_storage_api::{StateProvider, StateProviderFactory};
|
||||
@ -45,8 +45,11 @@ pub struct EthTransactionValidator<Client, T> {
|
||||
|
||||
impl<Client, Tx> EthTransactionValidator<Client, Tx> {
|
||||
/// Returns the configured chain spec
|
||||
pub fn chain_spec(&self) -> &Arc<ChainSpec> {
|
||||
&self.inner.chain_spec
|
||||
pub fn chain_spec(&self) -> Arc<Client::ChainSpec>
|
||||
where
|
||||
Client: ChainSpecProvider,
|
||||
{
|
||||
self.client().chain_spec()
|
||||
}
|
||||
|
||||
/// Returns the configured client
|
||||
@ -57,7 +60,7 @@ impl<Client, Tx> EthTransactionValidator<Client, Tx> {
|
||||
|
||||
impl<Client, Tx> EthTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory,
|
||||
Client: ChainSpecProvider<ChainSpec: EthereumHardforks> + StateProviderFactory,
|
||||
Tx: EthPoolTransaction,
|
||||
{
|
||||
/// Validates a single transaction.
|
||||
@ -86,7 +89,7 @@ where
|
||||
|
||||
impl<Client, Tx> TransactionValidator for EthTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory,
|
||||
Client: ChainSpecProvider<ChainSpec: EthereumHardforks> + StateProviderFactory,
|
||||
Tx: EthPoolTransaction,
|
||||
{
|
||||
type Transaction = Tx;
|
||||
@ -130,8 +133,6 @@ where
|
||||
/// And adheres to the configured [`LocalTransactionConfig`].
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct EthTransactionValidatorInner<Client, T> {
|
||||
/// Spec of the chain
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
/// This type fetches account info from the db
|
||||
client: Client,
|
||||
/// Blobstore used for fetching re-injected blob transactions.
|
||||
@ -162,18 +163,23 @@ pub(crate) struct EthTransactionValidatorInner<Client, T> {
|
||||
|
||||
// === impl EthTransactionValidatorInner ===
|
||||
|
||||
impl<Client, Tx> EthTransactionValidatorInner<Client, Tx> {
|
||||
impl<Client: ChainSpecProvider, Tx> EthTransactionValidatorInner<Client, Tx> {
|
||||
/// Returns the configured chain id
|
||||
pub(crate) fn chain_id(&self) -> u64 {
|
||||
self.chain_spec.chain().id()
|
||||
self.client.chain_spec().chain().id()
|
||||
}
|
||||
}
|
||||
|
||||
impl<Client, Tx> EthTransactionValidatorInner<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory,
|
||||
Client: ChainSpecProvider<ChainSpec: EthereumHardforks> + StateProviderFactory,
|
||||
Tx: EthPoolTransaction,
|
||||
{
|
||||
/// Returns the configured chain spec
|
||||
fn chain_spec(&self) -> Arc<Client::ChainSpec> {
|
||||
self.client.chain_spec()
|
||||
}
|
||||
|
||||
/// Validates a single transaction using an optional cached state provider.
|
||||
/// If no provider is passed, a new one will be created. This allows reusing
|
||||
/// the same provider across multiple txs.
|
||||
@ -509,15 +515,15 @@ where
|
||||
|
||||
fn on_new_head_block<T: BlockHeader>(&self, new_tip_block: &T) {
|
||||
// update all forks
|
||||
if self.chain_spec.is_cancun_active_at_timestamp(new_tip_block.timestamp()) {
|
||||
if self.chain_spec().is_cancun_active_at_timestamp(new_tip_block.timestamp()) {
|
||||
self.fork_tracker.cancun.store(true, std::sync::atomic::Ordering::Relaxed);
|
||||
}
|
||||
|
||||
if self.chain_spec.is_shanghai_active_at_timestamp(new_tip_block.timestamp()) {
|
||||
if self.chain_spec().is_shanghai_active_at_timestamp(new_tip_block.timestamp()) {
|
||||
self.fork_tracker.shanghai.store(true, std::sync::atomic::Ordering::Relaxed);
|
||||
}
|
||||
|
||||
if self.chain_spec.is_prague_active_at_timestamp(new_tip_block.timestamp()) {
|
||||
if self.chain_spec().is_prague_active_at_timestamp(new_tip_block.timestamp()) {
|
||||
self.fork_tracker.prague.store(true, std::sync::atomic::Ordering::Relaxed);
|
||||
}
|
||||
|
||||
@ -531,8 +537,8 @@ where
|
||||
|
||||
/// A builder for [`TransactionValidationTaskExecutor`]
|
||||
#[derive(Debug)]
|
||||
pub struct EthTransactionValidatorBuilder {
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
pub struct EthTransactionValidatorBuilder<Client> {
|
||||
client: Client,
|
||||
/// Fork indicator whether we are in the Shanghai stage.
|
||||
shanghai: bool,
|
||||
/// Fork indicator whether we are in the Cancun hardfork.
|
||||
@ -564,8 +570,8 @@ pub struct EthTransactionValidatorBuilder {
|
||||
max_tx_input_bytes: usize,
|
||||
}
|
||||
|
||||
impl EthTransactionValidatorBuilder {
|
||||
/// Creates a new builder for the given [`ChainSpec`]
|
||||
impl<Client> EthTransactionValidatorBuilder<Client> {
|
||||
/// Creates a new builder for the given client
|
||||
///
|
||||
/// By default this assumes the network is on the `Cancun` hardfork and the following
|
||||
/// transactions are allowed:
|
||||
@ -573,10 +579,10 @@ impl EthTransactionValidatorBuilder {
|
||||
/// - EIP-2718
|
||||
/// - EIP-1559
|
||||
/// - EIP-4844
|
||||
pub fn new(chain_spec: Arc<ChainSpec>) -> Self {
|
||||
pub fn new(client: Client) -> Self {
|
||||
Self {
|
||||
block_gas_limit: ETHEREUM_BLOCK_GAS_LIMIT.into(),
|
||||
chain_spec,
|
||||
client,
|
||||
minimum_priority_fee: None,
|
||||
additional_tasks: 1,
|
||||
kzg_settings: EnvKzgSettings::Default,
|
||||
@ -696,10 +702,13 @@ impl EthTransactionValidatorBuilder {
|
||||
/// Configures validation rules based on the head block's timestamp.
|
||||
///
|
||||
/// For example, whether the Shanghai and Cancun hardfork is activated at launch.
|
||||
pub fn with_head_timestamp(mut self, timestamp: u64) -> Self {
|
||||
self.cancun = self.chain_spec.is_cancun_active_at_timestamp(timestamp);
|
||||
self.shanghai = self.chain_spec.is_shanghai_active_at_timestamp(timestamp);
|
||||
self.prague = self.chain_spec.is_prague_active_at_timestamp(timestamp);
|
||||
pub fn with_head_timestamp(mut self, timestamp: u64) -> Self
|
||||
where
|
||||
Client: ChainSpecProvider<ChainSpec: EthereumHardforks>,
|
||||
{
|
||||
self.cancun = self.client.chain_spec().is_cancun_active_at_timestamp(timestamp);
|
||||
self.shanghai = self.client.chain_spec().is_shanghai_active_at_timestamp(timestamp);
|
||||
self.prague = self.client.chain_spec().is_prague_active_at_timestamp(timestamp);
|
||||
self
|
||||
}
|
||||
|
||||
@ -718,16 +727,12 @@ impl EthTransactionValidatorBuilder {
|
||||
}
|
||||
|
||||
/// Builds a the [`EthTransactionValidator`] without spawning validator tasks.
|
||||
pub fn build<Client, Tx, S>(
|
||||
self,
|
||||
client: Client,
|
||||
blob_store: S,
|
||||
) -> EthTransactionValidator<Client, Tx>
|
||||
pub fn build<Tx, S>(self, blob_store: S) -> EthTransactionValidator<Client, Tx>
|
||||
where
|
||||
S: BlobStore,
|
||||
{
|
||||
let Self {
|
||||
chain_spec,
|
||||
client,
|
||||
shanghai,
|
||||
cancun,
|
||||
prague,
|
||||
@ -750,7 +755,6 @@ impl EthTransactionValidatorBuilder {
|
||||
};
|
||||
|
||||
let inner = EthTransactionValidatorInner {
|
||||
chain_spec,
|
||||
client,
|
||||
eip2718,
|
||||
eip1559,
|
||||
@ -775,9 +779,8 @@ impl EthTransactionValidatorBuilder {
|
||||
/// The validator will spawn `additional_tasks` additional tasks for validation.
|
||||
///
|
||||
/// By default this will spawn 1 additional task.
|
||||
pub fn build_with_tasks<Client, Tx, T, S>(
|
||||
pub fn build_with_tasks<Tx, T, S>(
|
||||
self,
|
||||
client: Client,
|
||||
tasks: T,
|
||||
blob_store: S,
|
||||
) -> TransactionValidationTaskExecutor<EthTransactionValidator<Client, Tx>>
|
||||
@ -786,7 +789,7 @@ impl EthTransactionValidatorBuilder {
|
||||
S: BlobStore,
|
||||
{
|
||||
let additional_tasks = self.additional_tasks;
|
||||
let validator = self.build(client, blob_store);
|
||||
let validator = self.build(blob_store);
|
||||
|
||||
let (tx, task) = ValidationTask::new();
|
||||
|
||||
@ -882,7 +885,6 @@ mod tests {
|
||||
};
|
||||
use alloy_eips::eip2718::Decodable2718;
|
||||
use alloy_primitives::{hex, U256};
|
||||
use reth_chainspec::MAINNET;
|
||||
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, PooledTransaction};
|
||||
use reth_provider::test_utils::{ExtendedAccount, MockEthProvider};
|
||||
|
||||
@ -915,8 +917,7 @@ mod tests {
|
||||
ExtendedAccount::new(transaction.nonce(), U256::MAX),
|
||||
);
|
||||
let blob_store = InMemoryBlobStore::default();
|
||||
let validator = EthTransactionValidatorBuilder::new(MAINNET.clone())
|
||||
.build(provider, blob_store.clone());
|
||||
let validator = EthTransactionValidatorBuilder::new(provider).build(blob_store.clone());
|
||||
|
||||
let outcome = validator.validate_one(TransactionOrigin::External, transaction.clone());
|
||||
|
||||
@ -943,9 +944,9 @@ mod tests {
|
||||
);
|
||||
|
||||
let blob_store = InMemoryBlobStore::default();
|
||||
let validator = EthTransactionValidatorBuilder::new(MAINNET.clone())
|
||||
let validator = EthTransactionValidatorBuilder::new(provider)
|
||||
.set_block_gas_limit(1_000_000) // tx gas limit is 1_015_288
|
||||
.build(provider, blob_store.clone());
|
||||
.build(blob_store.clone());
|
||||
|
||||
let outcome = validator.validate_one(TransactionOrigin::External, transaction.clone());
|
||||
|
||||
|
||||
@ -7,7 +7,6 @@ use crate::{
|
||||
TransactionValidator,
|
||||
};
|
||||
use futures_util::{lock::Mutex, StreamExt};
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_primitives::SealedBlock;
|
||||
use reth_primitives_traits::Block;
|
||||
use reth_tasks::TaskSpawner;
|
||||
@ -93,8 +92,8 @@ pub struct TransactionValidationTaskExecutor<V> {
|
||||
|
||||
impl TransactionValidationTaskExecutor<()> {
|
||||
/// Convenience method to create a [`EthTransactionValidatorBuilder`]
|
||||
pub fn eth_builder(chain_spec: Arc<ChainSpec>) -> EthTransactionValidatorBuilder {
|
||||
EthTransactionValidatorBuilder::new(chain_spec)
|
||||
pub fn eth_builder<Client>(client: Client) -> EthTransactionValidatorBuilder<Client> {
|
||||
EthTransactionValidatorBuilder::new(client)
|
||||
}
|
||||
}
|
||||
|
||||
@ -112,23 +111,18 @@ impl<V> TransactionValidationTaskExecutor<V> {
|
||||
}
|
||||
|
||||
impl<Client, Tx> TransactionValidationTaskExecutor<EthTransactionValidator<Client, Tx>> {
|
||||
/// Creates a new instance for the given [`ChainSpec`]
|
||||
/// Creates a new instance for the given client
|
||||
///
|
||||
/// This will spawn a single validation tasks that performs the actual validation.
|
||||
/// See [`TransactionValidationTaskExecutor::eth_with_additional_tasks`]
|
||||
pub fn eth<T, S: BlobStore>(
|
||||
client: Client,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
blob_store: S,
|
||||
tasks: T,
|
||||
) -> Self
|
||||
pub fn eth<T, S: BlobStore>(client: Client, blob_store: S, tasks: T) -> Self
|
||||
where
|
||||
T: TaskSpawner,
|
||||
{
|
||||
Self::eth_with_additional_tasks(client, chain_spec, blob_store, tasks, 0)
|
||||
Self::eth_with_additional_tasks(client, blob_store, tasks, 0)
|
||||
}
|
||||
|
||||
/// Creates a new instance for the given [`ChainSpec`]
|
||||
/// Creates a new instance for the given client
|
||||
///
|
||||
/// By default this will enable support for:
|
||||
/// - shanghai
|
||||
@ -139,7 +133,6 @@ impl<Client, Tx> TransactionValidationTaskExecutor<EthTransactionValidator<Clien
|
||||
/// `num_additional_tasks` additional tasks.
|
||||
pub fn eth_with_additional_tasks<T, S: BlobStore>(
|
||||
client: Client,
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
blob_store: S,
|
||||
tasks: T,
|
||||
num_additional_tasks: usize,
|
||||
@ -147,9 +140,9 @@ impl<Client, Tx> TransactionValidationTaskExecutor<EthTransactionValidator<Clien
|
||||
where
|
||||
T: TaskSpawner,
|
||||
{
|
||||
EthTransactionValidatorBuilder::new(chain_spec)
|
||||
EthTransactionValidatorBuilder::new(client)
|
||||
.with_additional_tasks(num_additional_tasks)
|
||||
.build_with_tasks::<Client, Tx, T, S>(client, tasks, blob_store)
|
||||
.build_with_tasks::<Tx, T, S>(tasks, blob_store)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user