diff --git a/Cargo.lock b/Cargo.lock index 2a61c35f0a5e..accaf097a898 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7680,6 +7680,7 @@ dependencies = [ "reth-evm", "reth-execution-types", "reth-primitives", + "reth-primitives-traits", "reth-revm", "reth-testing-utils", "revm-primitives", @@ -8407,6 +8408,7 @@ dependencies = [ "alloy-eips", "alloy-primitives", "alloy-trie", + "op-alloy-consensus", "reth-chainspec", "reth-consensus", "reth-consensus-common", @@ -8611,6 +8613,7 @@ dependencies = [ "reth-optimism-payload-builder", "reth-optimism-primitives", "reth-primitives", + "reth-primitives-traits", "reth-provider", "reth-rpc", "reth-rpc-api", diff --git a/crates/e2e-test-utils/src/engine_api.rs b/crates/e2e-test-utils/src/engine_api.rs index 8c0f03bafd3a..d20456c171bd 100644 --- a/crates/e2e-test-utils/src/engine_api.rs +++ b/crates/e2e-test-utils/src/engine_api.rs @@ -6,7 +6,7 @@ use jsonrpsee::{ http_client::{transport::HttpBackend, HttpClient}, }; use reth_chainspec::EthereumHardforks; -use reth_node_api::EngineTypes; +use reth_node_api::{EngineTypes, NodePrimitives}; use reth_node_builder::BuiltPayload; use reth_payload_builder::PayloadId; use reth_payload_primitives::PayloadBuilderAttributes; @@ -17,14 +17,16 @@ use std::{marker::PhantomData, sync::Arc}; /// Helper for engine api operations #[derive(Debug)] -pub struct EngineApiTestContext { +pub struct EngineApiTestContext { pub chain_spec: Arc, - pub canonical_stream: CanonStateNotificationStream, + pub canonical_stream: CanonStateNotificationStream, pub engine_api_client: HttpClient>, pub _marker: PhantomData, } -impl EngineApiTestContext { +impl + EngineApiTestContext +{ /// Retrieves a v3 payload from the engine api pub async fn get_payload_v3( &self, diff --git a/crates/e2e-test-utils/src/lib.rs b/crates/e2e-test-utils/src/lib.rs index ae87fde48969..44e518eec5c8 100644 --- a/crates/e2e-test-utils/src/lib.rs +++ b/crates/e2e-test-utils/src/lib.rs @@ -5,7 +5,6 @@ use reth_chainspec::EthChainSpec; use reth_db::{test_utils::TempDatabase, DatabaseEnv}; use reth_engine_local::LocalPayloadAttributesBuilder; use reth_network_api::test_utils::PeersHandleProvider; -use reth_node_api::EngineValidator; use reth_node_builder::{ components::NodeComponentsBuilder, rpc::{EngineValidatorAddOn, RethRpcAddOns}, @@ -14,7 +13,6 @@ use reth_node_builder::{ PayloadTypes, }; use reth_node_core::args::{DiscoveryArgs, NetworkArgs, RpcServerArgs}; -use reth_primitives::EthPrimitives; use reth_provider::providers::{ BlockchainProvider, BlockchainProvider2, NodeTypesForProvider, NodeTypesForTree, }; @@ -122,7 +120,7 @@ pub async fn setup_engine( where N: Default + Node>>> - + NodeTypesWithEngine + + NodeTypesWithEngine + NodeTypesForProvider, N::ComponentsBuilder: NodeComponentsBuilder< TmpNodeAdapter>>, @@ -132,10 +130,7 @@ where >, >, N::AddOns: RethRpcAddOns>>> - + EngineValidatorAddOn< - Adapter>>, - Validator: EngineValidator, - >, + + EngineValidatorAddOn>>>, LocalPayloadAttributesBuilder: PayloadAttributesBuilder< <::Engine as PayloadTypes>::PayloadAttributes, >, diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index e64892c0e939..29dbc9274591 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -4,17 +4,17 @@ use crate::{ }; use alloy_consensus::BlockHeader; use alloy_eips::BlockId; -use alloy_primitives::{BlockHash, BlockNumber, Bytes, B256}; +use alloy_primitives::{BlockHash, BlockNumber, Bytes, Sealable, B256}; use alloy_rpc_types_engine::PayloadStatusEnum; use alloy_rpc_types_eth::BlockNumberOrTag; use eyre::Ok; use futures_util::Future; use reth_chainspec::EthereumHardforks; use reth_network_api::test_utils::PeersHandleProvider; -use reth_node_api::{Block, EngineTypes, FullNodeComponents}; +use reth_node_api::{Block, BlockTy, EngineTypes, FullNodeComponents}; use reth_node_builder::{rpc::RethRpcAddOns, FullNode, NodeTypes, NodeTypesWithEngine}; +use reth_node_core::primitives::SignedTransaction; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes}; -use reth_primitives::EthPrimitives; use reth_provider::{ BlockReader, BlockReaderIdExt, CanonStateSubscriptions, StageCheckpointReader, }; @@ -25,7 +25,7 @@ use tokio_stream::StreamExt; use url::Url; /// An helper struct to handle node actions -#[allow(missing_debug_implementations)] +#[expect(missing_debug_implementations, clippy::complexity)] pub struct NodeTestContext where Node: FullNodeComponents, @@ -41,6 +41,7 @@ where pub engine_api: EngineApiTestContext< ::Engine, ::ChainSpec, + ::Primitives, >, /// Context for testing RPC features. pub rpc: RpcTestContext, @@ -50,11 +51,7 @@ impl NodeTestContext where Engine: EngineTypes, Node: FullNodeComponents, - Node::Types: NodeTypesWithEngine< - ChainSpec: EthereumHardforks, - Engine = Engine, - Primitives = EthPrimitives, - >, + Node::Types: NodeTypesWithEngine, Node::Network: PeersHandleProvider, AddOns: RethRpcAddOns, { @@ -97,7 +94,7 @@ where where Engine::ExecutionPayloadEnvelopeV3: From + PayloadEnvelopeExt, Engine::ExecutionPayloadEnvelopeV4: From + PayloadEnvelopeExt, - AddOns::EthApi: EthApiSpec> + AddOns::EthApi: EthApiSpec>> + EthTransactions + TraceExt, { @@ -236,7 +233,7 @@ where // pool is actually present in the canonical block let head = self.engine_api.canonical_stream.next().await.unwrap(); let tx = head.tip().transactions().first(); - assert_eq!(tx.unwrap().hash().as_slice(), tip_tx_hash.as_slice()); + assert_eq!(tx.unwrap().tx_hash().as_slice(), tip_tx_hash.as_slice()); loop { // wait for the block to commit diff --git a/crates/e2e-test-utils/src/rpc.rs b/crates/e2e-test-utils/src/rpc.rs index cdc72a29538b..3a8d902441f8 100644 --- a/crates/e2e-test-utils/src/rpc.rs +++ b/crates/e2e-test-utils/src/rpc.rs @@ -2,7 +2,7 @@ use alloy_consensus::TxEnvelope; use alloy_network::eip2718::Decodable2718; use alloy_primitives::{Bytes, B256}; use reth_chainspec::EthereumHardforks; -use reth_node_api::{FullNodeComponents, NodePrimitives}; +use reth_node_api::{BlockTy, FullNodeComponents}; use reth_node_builder::{rpc::RpcRegistry, NodeTypes}; use reth_provider::BlockReader; use reth_rpc_api::DebugApiServer; @@ -18,16 +18,8 @@ pub struct RpcTestContext { impl RpcTestContext where - Node: FullNodeComponents< - Types: NodeTypes< - ChainSpec: EthereumHardforks, - Primitives: NodePrimitives< - Block = reth_primitives::Block, - Receipt = reth_primitives::Receipt, - >, - >, - >, - EthApi: EthApiSpec> + Node: FullNodeComponents>, + EthApi: EthApiSpec>> + EthTransactions + TraceExt, { diff --git a/crates/ethereum/evm/Cargo.toml b/crates/ethereum/evm/Cargo.toml index d12d4d6d7b30..f55a74adb7ad 100644 --- a/crates/ethereum/evm/Cargo.toml +++ b/crates/ethereum/evm/Cargo.toml @@ -22,6 +22,7 @@ reth-consensus.workspace = true # Ethereum revm-primitives.workspace = true +reth-primitives-traits.workspace = true # Alloy alloy-primitives.workspace = true @@ -52,5 +53,6 @@ std = [ "revm-primitives/std", "secp256k1/std", "reth-ethereum-forks/std", - "serde_json/std" + "serde_json/std", + "reth-primitives-traits/std" ] diff --git a/crates/ethereum/evm/src/lib.rs b/crates/ethereum/evm/src/lib.rs index 5af8f8c024a8..ebdf3d22c2bd 100644 --- a/crates/ethereum/evm/src/lib.rs +++ b/crates/ethereum/evm/src/lib.rs @@ -24,7 +24,8 @@ use alloy_consensus::{BlockHeader, Header}; use alloy_primitives::{Address, Bytes, TxKind, U256}; use reth_chainspec::ChainSpec; use reth_evm::{env::EvmEnv, ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes}; -use reth_primitives::{transaction::FillTxEnv, TransactionSigned}; +use reth_primitives::TransactionSigned; +use reth_primitives_traits::transaction::execute::FillTxEnv; use revm_primitives::{ AnalysisKind, BlobExcessGasAndPrice, BlockEnv, CfgEnv, CfgEnvWithHandlerCfg, Env, SpecId, TxEnv, }; diff --git a/crates/evm/src/lib.rs b/crates/evm/src/lib.rs index c539735b553d..540eb419acfd 100644 --- a/crates/evm/src/lib.rs +++ b/crates/evm/src/lib.rs @@ -20,7 +20,7 @@ extern crate alloc; use crate::builder::RethEvmBuilder; use alloy_consensus::BlockHeader as _; use alloy_primitives::{Address, Bytes, B256, U256}; -use reth_primitives_traits::BlockHeader; +use reth_primitives_traits::{BlockHeader, SignedTransaction}; use revm::{Database, Evm, GetInspector}; use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, Env, EnvWithHandlerCfg, SpecId, TxEnv}; @@ -119,7 +119,7 @@ pub trait ConfigureEvmEnv: Send + Sync + Unpin + Clone + 'static { type Header: BlockHeader; /// The transaction type. - type Transaction; + type Transaction: SignedTransaction; /// The error type that is returned by [`Self::next_cfg_and_block_env`]. type Error: core::error::Error + Send + Sync; diff --git a/crates/node/builder/src/launch/engine.rs b/crates/node/builder/src/launch/engine.rs index 9d69a29d114e..e4e247e239d3 100644 --- a/crates/node/builder/src/launch/engine.rs +++ b/crates/node/builder/src/launch/engine.rs @@ -23,8 +23,8 @@ use reth_exex::ExExManagerHandle; use reth_network::{NetworkSyncUpdater, SyncState}; use reth_network_api::BlockDownloaderProvider; use reth_node_api::{ - BlockTy, BuiltPayload, EngineValidator, FullNodeTypes, NodeTypesWithDBAdapter, - NodeTypesWithEngine, PayloadAttributesBuilder, PayloadBuilder, PayloadTypes, + BuiltPayload, FullNodeTypes, NodeTypesWithDBAdapter, NodeTypesWithEngine, + PayloadAttributesBuilder, PayloadBuilder, PayloadTypes, }; use reth_node_core::{ dirs::{ChainPath, DataDirPath}, @@ -32,7 +32,7 @@ use reth_node_core::{ primitives::Head, }; use reth_node_events::{cl::ConsensusLayerHealthEvents, node}; -use reth_primitives::{EthPrimitives, EthereumHardforks}; +use reth_primitives::EthereumHardforks; use reth_provider::providers::{BlockchainProvider2, NodeTypesForProvider}; use reth_tasks::TaskExecutor; use reth_tokio_util::EventSender; @@ -74,7 +74,7 @@ impl EngineNodeLauncher { impl LaunchNode> for EngineNodeLauncher where - Types: NodeTypesForProvider + NodeTypesWithEngine, + Types: NodeTypesForProvider + NodeTypesWithEngine, DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static, T: FullNodeTypes< Types = Types, @@ -83,11 +83,7 @@ where >, CB: NodeComponentsBuilder, AO: RethRpcAddOns> - + EngineValidatorAddOn< - NodeAdapter, - Validator: EngineValidator>, - >, - + + EngineValidatorAddOn>, LocalPayloadAttributesBuilder: PayloadAttributesBuilder< <::Engine as PayloadTypes>::PayloadAttributes, >, @@ -157,13 +153,13 @@ where let consensus_engine_stream = UnboundedReceiverStream::from(consensus_engine_rx) .maybe_skip_fcu(node_config.debug.skip_fcu) .maybe_skip_new_payload(node_config.debug.skip_new_payload) - .maybe_reorg( - ctx.blockchain_db().clone(), - ctx.components().evm_config().clone(), - reth_payload_validator::ExecutionPayloadValidator::new(ctx.chain_spec()), - node_config.debug.reorg_frequency, - node_config.debug.reorg_depth, - ) + // .maybe_reorg( + // ctx.blockchain_db().clone(), + // ctx.components().evm_config().clone(), + // reth_payload_validator::ExecutionPayloadValidator::new(ctx.chain_spec()), + // node_config.debug.reorg_frequency, + // node_config.debug.reorg_depth, + // ) // Store messages _after_ skipping so that `replay-engine` command // would replay only the messages that were observed by the engine // during this run. diff --git a/crates/optimism/bin/src/main.rs b/crates/optimism/bin/src/main.rs index f19c38f92a60..db4fd9ec01f2 100644 --- a/crates/optimism/bin/src/main.rs +++ b/crates/optimism/bin/src/main.rs @@ -23,40 +23,26 @@ fn main() { if let Err(err) = Cli::::parse().run(|builder, rollup_args| async move { - if rollup_args.experimental { - tracing::warn!(target: "reth::cli", "Experimental engine is default now, and the --engine.experimental flag is deprecated. To enable the legacy functionality, use --engine.legacy."); - } - let use_legacy_engine = rollup_args.legacy; - match use_legacy_engine { - false => { - let engine_tree_config = TreeConfig::default() - .with_persistence_threshold(rollup_args.persistence_threshold) - .with_memory_block_buffer_target(rollup_args.memory_block_buffer_target); - - let op_node = OpNode::new(rollup_args.clone()); - let handle = builder - .with_types_and_provider::>() - .with_components(op_node.components()) - .with_add_ons(op_node.add_ons()) - .launch_with_fn(|builder| { - let launcher = EngineNodeLauncher::new( - builder.task_executor().clone(), - builder.config().datadir(), - engine_tree_config, - ); - builder.launch_with(launcher) - }) - .await?; - - handle.node_exit_future.await - } - true => { - let handle = - builder.node(OpNode::new(rollup_args.clone())).launch().await?; - - handle.node_exit_future.await - } - } + let engine_tree_config = TreeConfig::default() + .with_persistence_threshold(rollup_args.persistence_threshold) + .with_memory_block_buffer_target(rollup_args.memory_block_buffer_target); + + let op_node = OpNode::new(rollup_args.clone()); + let handle = builder + .with_types_and_provider::>() + .with_components(op_node.components()) + .with_add_ons(op_node.add_ons()) + .launch_with_fn(|builder| { + let launcher = EngineNodeLauncher::new( + builder.task_executor().clone(), + builder.config().datadir(), + engine_tree_config, + ); + builder.launch_with(launcher) + }) + .await?; + + handle.node_exit_future.await }) { eprintln!("Error: {err:?}"); diff --git a/crates/optimism/cli/Cargo.toml b/crates/optimism/cli/Cargo.toml index 5fab2b93d4f0..3cd2edeecc5a 100644 --- a/crates/optimism/cli/Cargo.toml +++ b/crates/optimism/cli/Cargo.toml @@ -14,7 +14,7 @@ workspace = true reth-static-file-types = { workspace = true, features = ["clap"] } reth-cli-commands.workspace = true reth-consensus.workspace = true -reth-db = { workspace = true, features = ["mdbx"] } +reth-db = { workspace = true, features = ["mdbx", "op"] } reth-db-api.workspace = true reth-db-common.workspace = true reth-downloaders.workspace = true diff --git a/crates/optimism/cli/src/commands/import.rs b/crates/optimism/cli/src/commands/import.rs index d8e41f07df7e..90a2acdec0d5 100644 --- a/crates/optimism/cli/src/commands/import.rs +++ b/crates/optimism/cli/src/commands/import.rs @@ -12,6 +12,7 @@ use reth_db_api::transaction::DbTx; use reth_downloaders::file_client::{ ChunkedFileReader, FileClient, DEFAULT_BYTE_LEN_CHUNK_CHAIN_FILE, }; +use reth_node_builder::BlockTy; use reth_node_core::version::SHORT_VERSION; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_evm::OpExecutorProvider; @@ -69,7 +70,7 @@ impl> ImportOpCommand { let mut total_decoded_txns = 0; let mut total_filtered_out_dup_txns = 0; - while let Some(mut file_client) = reader.next_chunk::().await? { + while let Some(mut file_client) = reader.next_chunk::>>().await? { // create a new FileClient from chunk read from file info!(target: "reth::cli", "Importing chain file chunk" diff --git a/crates/optimism/cli/src/commands/import_receipts.rs b/crates/optimism/cli/src/commands/import_receipts.rs index 0cf6bdb723e2..040ecdc00357 100644 --- a/crates/optimism/cli/src/commands/import_receipts.rs +++ b/crates/optimism/cli/src/commands/import_receipts.rs @@ -12,9 +12,10 @@ use reth_downloaders::{ receipt_file_client::ReceiptFileClient, }; use reth_execution_types::ExecutionOutcome; +use reth_node_builder::ReceiptTy; use reth_node_core::version::SHORT_VERSION; use reth_optimism_chainspec::OpChainSpec; -use reth_optimism_primitives::{bedrock::is_dup_tx, OpPrimitives}; +use reth_optimism_primitives::{bedrock::is_dup_tx, OpPrimitives, OpReceipt}; use reth_primitives::{NodePrimitives, Receipts}; use reth_provider::{ providers::ProviderNodeTypes, writer::UnifiedStorageWriter, DatabaseProviderFactory, @@ -63,7 +64,7 @@ impl> ImportReceiptsOpCommand { provider_factory, self.path, self.chunk_len, - |first_block, receipts: &mut Receipts| { + |first_block, receipts| { let mut total_filtered_out_dup_txns = 0; for (index, receipts_for_block) in receipts.iter_mut().enumerate() { if is_dup_tx(first_block + index as u64) { @@ -87,12 +88,9 @@ pub async fn import_receipts_from_file( filter: F, ) -> eyre::Result<()> where - N: ProviderNodeTypes< - ChainSpec = OpChainSpec, - Primitives: NodePrimitives, - >, + N: ProviderNodeTypes>, P: AsRef, - F: FnMut(u64, &mut Receipts) -> usize, + F: FnMut(u64, &mut Receipts) -> usize, { for stage in StageId::ALL { let checkpoint = provider_factory.database_provider_ro()?.get_stage_checkpoint(stage)?; @@ -128,8 +126,8 @@ pub async fn import_receipts_from_reader( mut filter: F, ) -> eyre::Result where - N: ProviderNodeTypes>, - F: FnMut(u64, &mut Receipts) -> usize, + N: ProviderNodeTypes>, + F: FnMut(u64, &mut Receipts>) -> usize, { let static_file_provider = provider_factory.static_file_provider(); @@ -163,7 +161,7 @@ where .expect("transaction static files must exist before importing receipts"); while let Some(file_client) = - reader.next_receipts_chunk::>().await? + reader.next_receipts_chunk::>>().await? { if highest_block_receipts == highest_block_transactions { warn!(target: "reth::cli", highest_block_receipts, highest_block_transactions, "Ignoring all other blocks in the file since we have reached the desired height"); @@ -267,6 +265,9 @@ pub struct ImportReceiptsResult { mod test { use alloy_primitives::hex; use reth_db_common::init::init_genesis; + use reth_optimism_chainspec::OP_MAINNET; + use reth_optimism_node::OpNode; + use reth_provider::test_utils::create_test_provider_factory_with_node_types; use reth_stages::test_utils::TestStageDB; use tempfile::tempfile; use tokio::{ @@ -301,11 +302,10 @@ mod test { init_genesis(&db.factory).unwrap(); // todo: where does import command init receipts ? probably somewhere in pipeline - + let provider_factory = + create_test_provider_factory_with_node_types::(OP_MAINNET.clone()); let ImportReceiptsResult { total_decoded_receipts, total_filtered_out_dup_txns } = - import_receipts_from_reader(&TestStageDB::default().factory, reader, |_, _| 0) - .await - .unwrap(); + import_receipts_from_reader(&provider_factory, reader, |_, _| 0).await.unwrap(); assert_eq!(total_decoded_receipts, 3); assert_eq!(total_filtered_out_dup_txns, 0); diff --git a/crates/optimism/consensus/Cargo.toml b/crates/optimism/consensus/Cargo.toml index 4f4868a454dc..b9060c5e62ae 100644 --- a/crates/optimism/consensus/Cargo.toml +++ b/crates/optimism/consensus/Cargo.toml @@ -35,6 +35,7 @@ tracing.workspace = true [dev-dependencies] alloy-primitives.workspace = true +op-alloy-consensus.workspace = true reth-optimism-chainspec.workspace = true [features] diff --git a/crates/optimism/consensus/src/lib.rs b/crates/optimism/consensus/src/lib.rs index 78e0748ca395..cb942729aa58 100644 --- a/crates/optimism/consensus/src/lib.rs +++ b/crates/optimism/consensus/src/lib.rs @@ -23,8 +23,8 @@ use reth_consensus_common::validation::{ }; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_forks::OpHardforks; -use reth_optimism_primitives::OpPrimitives; -use reth_primitives::{BlockBody, BlockWithSenders, GotExpected, SealedBlock, SealedHeader}; +use reth_optimism_primitives::{OpBlock, OpBlockBody, OpPrimitives, OpReceipt}; +use reth_primitives::{BlockWithSenders, GotExpected, SealedBlockFor, SealedHeader}; use std::{sync::Arc, time::SystemTime}; mod proof; @@ -52,23 +52,26 @@ impl OpBeaconConsensus { impl FullConsensus for OpBeaconConsensus { fn validate_block_post_execution( &self, - block: &BlockWithSenders, - input: PostExecutionInput<'_>, + block: &BlockWithSenders, + input: PostExecutionInput<'_, OpReceipt>, ) -> Result<(), ConsensusError> { validate_block_post_execution(block, &self.chain_spec, input.receipts) } } -impl Consensus for OpBeaconConsensus { +impl Consensus for OpBeaconConsensus { fn validate_body_against_header( &self, - body: &BlockBody, + body: &OpBlockBody, header: &SealedHeader, ) -> Result<(), ConsensusError> { validate_body_against_header(body, header.header()) } - fn validate_block_pre_execution(&self, block: &SealedBlock) -> Result<(), ConsensusError> { + fn validate_block_pre_execution( + &self, + block: &SealedBlockFor, + ) -> Result<(), ConsensusError> { // Check ommers hash let ommers_hash = reth_primitives::proofs::calculate_ommers_root(&block.body.ommers); if block.header.ommers_hash != ommers_hash { diff --git a/crates/optimism/consensus/src/proof.rs b/crates/optimism/consensus/src/proof.rs index df0669568b3d..6f86e70f9c33 100644 --- a/crates/optimism/consensus/src/proof.rs +++ b/crates/optimism/consensus/src/proof.rs @@ -1,15 +1,17 @@ //! Helper function for Receipt root calculation for Optimism hardforks. +use alloy_consensus::TxReceipt; use alloy_eips::eip2718::Encodable2718; use alloy_primitives::B256; use alloy_trie::root::ordered_trie_root_with_encoder; use reth_chainspec::ChainSpec; use reth_optimism_forks::OpHardfork; -use reth_primitives::{Receipt, ReceiptWithBloom}; +use reth_optimism_primitives::OpReceipt; +use reth_primitives::ReceiptWithBloom; /// Calculates the receipt root for a header. pub(crate) fn calculate_receipt_root_optimism( - receipts: &[ReceiptWithBloom], + receipts: &[ReceiptWithBloom], chain_spec: &ChainSpec, timestamp: u64, ) -> B256 { @@ -25,7 +27,9 @@ pub(crate) fn calculate_receipt_root_optimism( .iter() .cloned() .map(|mut r| { - r.receipt.deposit_nonce = None; + if let OpReceipt::Deposit(receipt) = &mut r.receipt { + receipt.deposit_nonce = None; + } r }) .collect::>(); @@ -36,11 +40,11 @@ pub(crate) fn calculate_receipt_root_optimism( ordered_trie_root_with_encoder(receipts, |r, buf| r.encode_2718(buf)) } -/// Calculates the receipt root for a header for the reference type of [Receipt]. +/// Calculates the receipt root for a header for the reference type of [`OpReceipt`]. /// /// NOTE: Prefer calculate receipt root optimism if you have log blooms memoized. pub fn calculate_receipt_root_no_memo_optimism( - receipts: &[&Receipt], + receipts: &[&OpReceipt], chain_spec: impl reth_chainspec::Hardforks, timestamp: u64, ) -> B256 { @@ -56,7 +60,9 @@ pub fn calculate_receipt_root_no_memo_optimism( .iter() .map(|r| { let mut r = (*r).clone(); - r.deposit_nonce = None; + if let OpReceipt::Deposit(r) = &mut r { + r.deposit_nonce = None; + } r }) .collect::>(); @@ -74,9 +80,11 @@ pub fn calculate_receipt_root_no_memo_optimism( #[cfg(test)] mod tests { use super::*; + use alloy_consensus::Receipt; use alloy_primitives::{b256, bloom, hex, Address, Bloom, Bytes, Log, LogData}; + use op_alloy_consensus::OpDepositReceipt; use reth_optimism_chainspec::BASE_SEPOLIA; - use reth_primitives::{Receipt, ReceiptWithBloom, TxType}; + use reth_primitives::ReceiptWithBloom; /// Tests that the receipt root is computed correctly for the regolith block. /// This was implemented due to a minor bug in op-geth and op-erigon where in @@ -114,21 +122,21 @@ mod tests { let receipts = vec![ // 0xb0d6ee650637911394396d81172bd1c637d568ed1fbddab0daddfca399c58b53 ReceiptWithBloom { - receipt: Receipt { - tx_type: TxType::Deposit, - success: true, - cumulative_gas_used: 46913, - logs: vec![], + receipt: OpReceipt::Deposit(OpDepositReceipt { + inner: Receipt { + status: true.into(), + cumulative_gas_used: 46913, + logs: vec![], + }, deposit_nonce: Some(4012991u64), deposit_receipt_version: None, - }, + }), logs_bloom: Bloom(hex!("00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000").into()), }, // 0x2f433586bae30573c393adfa02bc81d2a1888a3d6c9869f473fb57245166bd9a ReceiptWithBloom { - receipt: Receipt { - tx_type: TxType::Eip1559, - success: true, + receipt: OpReceipt::Eip1559(Receipt { + status: true.into(), cumulative_gas_used: 118083, logs: vec![ Log { @@ -164,17 +172,13 @@ mod tests { b256!("000000000000000000000000c498902843af527e674846bb7edefa8ad62b8fb9"), ], Bytes::from_static(&hex!("0000000000000000000000000000000000000000000000000000000000000003"))) }, - ], - deposit_nonce: None, - deposit_receipt_version: None, - }, + ]}), logs_bloom: Bloom(hex!("00001000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000002000000000000000000000000000000000000000000000000000000000000000000000000000020000000000000000000800000000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000040000000000004000000000080000000000000000000000000000000000000000000000000000008000000000000080020000000000000000000000000002000000000000000000000000000080000010000").into()), }, // 0x6c33676e8f6077f46a62eabab70bc6d1b1b18a624b0739086d77093a1ecf8266 ReceiptWithBloom { - receipt: Receipt { - tx_type: TxType::Eip1559, - success: true, + receipt: OpReceipt::Eip1559(Receipt { + status: true.into(), cumulative_gas_used: 189253, logs: vec![ Log { @@ -207,16 +211,13 @@ mod tests { Bytes::from_static(&hex!("0000000000000000000000000000000000000000000000000000000000000003"))) }, ], - deposit_nonce: None, - deposit_receipt_version: None, - }, + }), logs_bloom: Bloom(hex!("00000000000000000000200000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000002000000000020000000000000000000000000000000000000000000000000000000000000000020000000000000000000800000000000000000000000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000040000000000004000000000080000000000000000000000000000000000000000000000000000008000000000000080020000000000000000000000000002000000000000000000000000000080000000000").into()), }, // 0x4d3ecbef04ba7ce7f5ab55be0c61978ca97c117d7da448ed9771d4ff0c720a3f ReceiptWithBloom { - receipt: Receipt { - tx_type: TxType::Eip1559, - success: true, + receipt: OpReceipt::Eip1559(Receipt { + status: true.into(), cumulative_gas_used: 346969, logs: vec![ Log { @@ -279,16 +280,13 @@ mod tests { Bytes::from_static(&hex!("0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002d24d8e9ac1aa79e200000000000000000000000000000000000000000000000014bc73062aea80930000000000000000000000000000000000000000000000000000000000000000"))) }, ], - deposit_nonce: None, - deposit_receipt_version: None, - }, + }), logs_bloom: Bloom(hex!("00200000000000000000000080000000000000000000000000040000100004000000000000000000000000100000000000000000000000000000100000000000000000000000000002000008000000200000000200000000020000000000000040000000000000000400000200000000000000000000000000000010000000000400000000010400000000000000000000000000002000c80000004080002000000000000000400200000000800000000000000000000000000000000000000000000002000000000000000000000000000000000100001000000000000000000000002000000000000000000000010000000000000000000000800000800000").into()), }, // 0xf738af5eb00ba23dbc1be2dbce41dbc0180f0085b7fb46646e90bf737af90351 ReceiptWithBloom { - receipt: Receipt { - tx_type: TxType::Eip1559, - success: true, + receipt: OpReceipt::Eip1559(Receipt { + status: true.into(), cumulative_gas_used: 623249, logs: vec![ Log { @@ -321,9 +319,7 @@ mod tests { Bytes::from_static(&hex!("0000000000000000000000000000000000000000000000000000000000000020000000000000000000000000a4fa7f3fbf0677f254ebdb1646146864c305b76e00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001400000000000000000000000000000000000000000000000000000000000000000000000000000000000000000eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000007717500762343034303661353035646234633961386163316433306335633332303265370000000000000000000000000000000000000000000000000000000000000037697066733a2f2f516d515141646b33736538396b47716577395256567a316b68643548375562476d4d4a485a62566f386a6d346f4a2f30000000000000000000"))) }, ], - deposit_nonce: None, - deposit_receipt_version: None, - }, + }), logs_bloom: Bloom(hex!("00000000000000000000000000000000400000000000000000000000000000000000004000000000000001000000000000000002000000000100000000000000000000000000000000000008000000000000000000000000000000000000000004000000020000000000000000000800000000000000000000000010200100200008000002000000000000000000800000000000000000000002000000000000000000000000000000080000000000000000000000004000000000000000000000000002000000000000000000000000000000000000200000000000000020002000000000000000002000000000000000000000000000000000000000000000").into()), }, ]; @@ -340,14 +336,11 @@ mod tests { }]; let logs_bloom = bloom!("00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001"); let receipt = ReceiptWithBloom { - receipt: Receipt { - tx_type: TxType::Eip2930, - success: true, + receipt: OpReceipt::Eip2930(Receipt { + status: true.into(), cumulative_gas_used: 102068, logs, - deposit_nonce: None, - deposit_receipt_version: None, - }, + }), logs_bloom, }; let receipt = vec![receipt]; diff --git a/crates/optimism/consensus/src/validation.rs b/crates/optimism/consensus/src/validation.rs index 5290603e7b89..d5b88f03d4b6 100644 --- a/crates/optimism/consensus/src/validation.rs +++ b/crates/optimism/consensus/src/validation.rs @@ -3,16 +3,17 @@ use alloy_consensus::TxReceipt; use alloy_primitives::{Bloom, B256}; use reth_chainspec::{ChainSpec, EthereumHardforks}; use reth_consensus::ConsensusError; -use reth_primitives::{gas_spent_by_transactions, BlockWithSenders, GotExpected, Receipt}; +use reth_optimism_primitives::{OpBlock, OpReceipt}; +use reth_primitives::{gas_spent_by_transactions, BlockWithSenders, GotExpected}; /// Validate a block with regard to execution results: /// /// - Compares the receipts root in the block header to the block body /// - Compares the gas used in the block header to the actual gas usage after execution pub fn validate_block_post_execution( - block: &BlockWithSenders, + block: &BlockWithSenders, chain_spec: &ChainSpec, - receipts: &[Receipt], + receipts: &[OpReceipt], ) -> Result<(), ConsensusError> { // Before Byzantium, receipts contained state root that would mean that expensive // operation as hashing that is required for state root got calculated in every @@ -33,10 +34,10 @@ pub fn validate_block_post_execution( // Check if gas used matches the value set in header. let cumulative_gas_used = - receipts.last().map(|receipt| receipt.cumulative_gas_used).unwrap_or(0); - if block.gas_used != cumulative_gas_used { + receipts.last().map(|receipt| receipt.cumulative_gas_used()).unwrap_or(0); + if block.gas_used as u128 != cumulative_gas_used { return Err(ConsensusError::BlockGasUsed { - gas: GotExpected { got: cumulative_gas_used, expected: block.gas_used }, + gas: GotExpected { got: cumulative_gas_used as u64, expected: block.gas_used }, gas_spent_by_tx: gas_spent_by_transactions(receipts), }) } @@ -48,12 +49,12 @@ pub fn validate_block_post_execution( fn verify_receipts( expected_receipts_root: B256, expected_logs_bloom: Bloom, - receipts: &[Receipt], + receipts: &[OpReceipt], chain_spec: &ChainSpec, timestamp: u64, ) -> Result<(), ConsensusError> { // Calculate receipts root. - let receipts_with_bloom = receipts.iter().cloned().map(Receipt::with_bloom).collect::>(); + let receipts_with_bloom = receipts.iter().cloned().map(Into::into).collect::>(); let receipts_root = calculate_receipt_root_optimism(&receipts_with_bloom, chain_spec, timestamp); diff --git a/crates/optimism/evm/src/execute.rs b/crates/optimism/evm/src/execute.rs index 8cc4a2c35525..4fbde7b66fa2 100644 --- a/crates/optimism/evm/src/execute.rs +++ b/crates/optimism/evm/src/execute.rs @@ -2,10 +2,10 @@ use crate::{l1::ensure_create2_deployer, OpBlockExecutionError, OpEvmConfig}; use alloc::{boxed::Box, sync::Arc, vec::Vec}; -use alloy_consensus::{Header, Transaction as _}; +use alloy_consensus::{Eip658Value, Header, Receipt, Transaction as _}; use alloy_eips::eip7685::Requests; use core::fmt::Display; -use op_alloy_consensus::DepositTransaction; +use op_alloy_consensus::{OpDepositReceipt, OpTxType}; use reth_chainspec::EthereumHardforks; use reth_consensus::ConsensusError; use reth_evm::{ @@ -22,8 +22,9 @@ use reth_evm::{ use reth_optimism_chainspec::OpChainSpec; use reth_optimism_consensus::validate_block_post_execution; use reth_optimism_forks::OpHardfork; -use reth_optimism_primitives::OpPrimitives; -use reth_primitives::{BlockWithSenders, Receipt, TransactionSigned, TxType}; +use reth_optimism_primitives::{OpBlock, OpPrimitives, OpReceipt, OpTransactionSigned}; +use reth_primitives::BlockWithSenders; +use reth_primitives_traits::SignedTransaction; use reth_revm::{Database, State}; use revm_primitives::{db::DatabaseCommit, EnvWithHandlerCfg, ResultAndState}; use tracing::trace; @@ -58,7 +59,7 @@ where + Sync + Send + 'static - + ConfigureEvm
, + + ConfigureEvm
, { type Primitives = OpPrimitives; type Strategy + Display>> = @@ -121,7 +122,7 @@ where impl BlockExecutionStrategy for OpExecutionStrategy where DB: Database + Display>, - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvm
, { type DB = DB; type Primitives = OpPrimitives; @@ -131,7 +132,10 @@ where self.tx_env_overrides = Some(tx_env_overrides); } - fn apply_pre_execution_changes(&mut self, block: &BlockWithSenders) -> Result<(), Self::Error> { + fn apply_pre_execution_changes( + &mut self, + block: &BlockWithSenders, + ) -> Result<(), Self::Error> { // Set state clear flag if the block is after the Spurious Dragon hardfork. let state_clear_flag = (*self.chain_spec).is_spurious_dragon_active_at_block(block.header.number); @@ -159,8 +163,8 @@ where fn execute_transactions( &mut self, - block: &BlockWithSenders, - ) -> Result, Self::Error> { + block: &BlockWithSenders, + ) -> Result, Self::Error> { let env = self.evm_env_for_block(&block.header); let mut evm = self.evm_config.evm_with_env(&mut self.state, env); @@ -174,7 +178,7 @@ where // must be no greater than the block’s gasLimit. let block_available_gas = block.header.gas_limit - cumulative_gas_used; if transaction.gas_limit() > block_available_gas && - (is_regolith || !transaction.is_system_transaction()) + (is_regolith || !transaction.is_deposit()) { return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas { transaction_gas_limit: transaction.gas_limit(), @@ -183,11 +187,6 @@ where .into()) } - // An optimism block should never contain blob transactions. - if matches!(transaction.tx_type(), TxType::Eip4844) { - return Err(OpBlockExecutionError::BlobTransactionRejected.into()) - } - // Cache the depositor account prior to the state transition for the deposit nonce. // // Note that this *only* needs to be done post-regolith hardfork, as deposit nonces @@ -230,22 +229,32 @@ where // append gas used cumulative_gas_used += result.gas_used(); - // Push transaction changeset and calculate header bloom filter for receipt. - receipts.push(Receipt { - tx_type: transaction.tx_type(), + let receipt = Receipt { // Success flag was added in `EIP-658: Embedding transaction status code in // receipts`. - success: result.is_success(), - cumulative_gas_used, + status: Eip658Value::Eip658(result.is_success()), + cumulative_gas_used: cumulative_gas_used as u128, logs: result.into_logs(), - deposit_nonce: depositor.map(|account| account.nonce), - // The deposit receipt version was introduced in Canyon to indicate an update to how - // receipt hashes should be computed when set. The state transition process ensures - // this is only set for post-Canyon deposit transactions. - deposit_receipt_version: (transaction.is_deposit() && - self.chain_spec - .is_fork_active_at_timestamp(OpHardfork::Canyon, block.timestamp)) - .then_some(1), + }; + + // Push transaction changeset and calculate header bloom filter for receipt. + receipts.push(match transaction.tx_type() { + OpTxType::Legacy => OpReceipt::Legacy(receipt), + OpTxType::Eip2930 => OpReceipt::Eip2930(receipt), + OpTxType::Eip1559 => OpReceipt::Eip1559(receipt), + OpTxType::Eip7702 => OpReceipt::Eip7702(receipt), + OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt { + inner: receipt, + deposit_nonce: depositor.map(|account| account.nonce), + // The deposit receipt version was introduced in Canyon to indicate an update to + // how receipt hashes should be computed when set. The state + // transition process ensures this is only set for + // post-Canyon deposit transactions. + deposit_receipt_version: (transaction.is_deposit() && + self.chain_spec + .is_fork_active_at_timestamp(OpHardfork::Canyon, block.timestamp)) + .then_some(1), + }), }); } @@ -254,8 +263,8 @@ where fn apply_post_execution_changes( &mut self, - block: &BlockWithSenders, - _receipts: &[Receipt], + block: &BlockWithSenders, + _receipts: &[OpReceipt], ) -> Result { let balance_increments = post_block_balance_increments(&self.chain_spec.clone(), &block.block); @@ -284,8 +293,8 @@ where fn validate_block_post_execution( &self, - block: &BlockWithSenders, - receipts: &[Receipt], + block: &BlockWithSenders, + receipts: &[OpReceipt], _requests: &Requests, ) -> Result<(), ConsensusError> { validate_block_post_execution(block, &self.chain_spec.clone(), receipts) @@ -313,11 +322,11 @@ mod tests { use alloy_primitives::{ b256, Address, PrimitiveSignature as Signature, StorageKey, StorageValue, U256, }; - use op_alloy_consensus::TxDeposit; + use op_alloy_consensus::{OpTypedTransaction, TxDeposit}; use reth_chainspec::MIN_TRANSACTION_GAS; use reth_evm::execute::{BasicBlockExecutorProvider, BatchExecutor, BlockExecutorProvider}; use reth_optimism_chainspec::OpChainSpecBuilder; - use reth_primitives::{Account, Block, BlockBody, Transaction, TransactionSigned}; + use reth_primitives::{Account, Block, BlockBody}; use reth_revm::{ database::StateProviderDatabase, test_utils::StateProviderTest, L1_BLOCK_CONTRACT, }; @@ -380,8 +389,8 @@ mod tests { let chain_spec = Arc::new(OpChainSpecBuilder::base_mainnet().regolith_activated().build()); - let tx = TransactionSigned::new_unhashed( - Transaction::Eip1559(TxEip1559 { + let tx = OpTransactionSigned::new_unhashed( + OpTypedTransaction::Eip1559(TxEip1559 { chain_id: chain_spec.chain.id(), nonce: 0, gas_limit: MIN_TRANSACTION_GAS, @@ -391,8 +400,8 @@ mod tests { Signature::test_signature(), ); - let tx_deposit = TransactionSigned::new_unhashed( - Transaction::Deposit(op_alloy_consensus::TxDeposit { + let tx_deposit = OpTransactionSigned::new_unhashed( + OpTypedTransaction::Deposit(op_alloy_consensus::TxDeposit { from: addr, to: addr.into(), gas_limit: MIN_TRANSACTION_GAS, @@ -424,13 +433,14 @@ mod tests { let tx_receipt = receipts[0][0].as_ref().unwrap(); let deposit_receipt = receipts[0][1].as_ref().unwrap(); - // deposit_receipt_version is not present in pre canyon transactions - assert!(deposit_receipt.deposit_receipt_version.is_none()); - assert!(tx_receipt.deposit_receipt_version.is_none()); - + assert!(!matches!(tx_receipt, OpReceipt::Deposit(_))); // deposit_nonce is present only in deposit transactions + let OpReceipt::Deposit(deposit_receipt) = deposit_receipt else { + panic!("expected deposit") + }; assert!(deposit_receipt.deposit_nonce.is_some()); - assert!(tx_receipt.deposit_nonce.is_none()); + // deposit_receipt_version is not present in pre canyon transactions + assert!(deposit_receipt.deposit_receipt_version.is_none()); } #[test] @@ -455,8 +465,8 @@ mod tests { let chain_spec = Arc::new(OpChainSpecBuilder::base_mainnet().canyon_activated().build()); - let tx = TransactionSigned::new_unhashed( - Transaction::Eip1559(TxEip1559 { + let tx = OpTransactionSigned::new_unhashed( + OpTypedTransaction::Eip1559(TxEip1559 { chain_id: chain_spec.chain.id(), nonce: 0, gas_limit: MIN_TRANSACTION_GAS, @@ -466,8 +476,8 @@ mod tests { Signature::test_signature(), ); - let tx_deposit = TransactionSigned::new_unhashed( - Transaction::Deposit(op_alloy_consensus::TxDeposit { + let tx_deposit = OpTransactionSigned::new_unhashed( + OpTypedTransaction::Deposit(op_alloy_consensus::TxDeposit { from: addr, to: addr.into(), gas_limit: MIN_TRANSACTION_GAS, @@ -500,11 +510,13 @@ mod tests { let deposit_receipt = receipts[0][1].as_ref().unwrap(); // deposit_receipt_version is set to 1 for post canyon deposit transactions + assert!(!matches!(tx_receipt, OpReceipt::Deposit(_))); + let OpReceipt::Deposit(deposit_receipt) = deposit_receipt else { + panic!("expected deposit") + }; assert_eq!(deposit_receipt.deposit_receipt_version, Some(1)); - assert!(tx_receipt.deposit_receipt_version.is_none()); // deposit_nonce is present only in deposit transactions assert!(deposit_receipt.deposit_nonce.is_some()); - assert!(tx_receipt.deposit_nonce.is_none()); } } diff --git a/crates/optimism/evm/src/lib.rs b/crates/optimism/evm/src/lib.rs index 6b367b8f3e7f..733f78a1d57e 100644 --- a/crates/optimism/evm/src/lib.rs +++ b/crates/optimism/evm/src/lib.rs @@ -18,7 +18,8 @@ use alloy_primitives::{Address, U256}; use op_alloy_consensus::EIP1559ParamError; use reth_evm::{env::EvmEnv, ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes}; use reth_optimism_chainspec::OpChainSpec; -use reth_primitives::{transaction::FillTxEnv, TransactionSigned}; +use reth_optimism_primitives::OpTransactionSigned; +use reth_primitives_traits::FillTxEnv; use reth_revm::{ inspector_handle_register, primitives::{AnalysisKind, CfgEnvWithHandlerCfg, TxEnv}, @@ -58,10 +59,10 @@ impl OpEvmConfig { impl ConfigureEvmEnv for OpEvmConfig { type Header = Header; - type Transaction = TransactionSigned; + type Transaction = OpTransactionSigned; type Error = EIP1559ParamError; - fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) { + fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &OpTransactionSigned, sender: Address) { transaction.fill_tx_env(tx_env, sender); } @@ -189,7 +190,7 @@ impl ConfigureEvm for OpEvmConfig { #[cfg(test)] mod tests { use super::*; - use alloy_consensus::{constants::KECCAK_EMPTY, Header}; + use alloy_consensus::{constants::KECCAK_EMPTY, Header, Receipt}; use alloy_eips::eip7685::Requests; use alloy_genesis::Genesis; use alloy_primitives::{bytes, Address, LogData, B256, U256}; @@ -199,8 +200,8 @@ mod tests { AccountRevertInit, BundleStateInit, Chain, ExecutionOutcome, RevertsInit, }; use reth_optimism_chainspec::BASE_MAINNET; - use reth_optimism_primitives::OpPrimitives; - use reth_primitives::{Account, Log, Receipt, Receipts, SealedBlockWithSenders, TxType}; + use reth_optimism_primitives::{OpBlock, OpPrimitives, OpReceipt}; + use reth_primitives::{Account, Log, Receipts, SealedBlockWithSenders}; use reth_revm::{ db::{BundleState, CacheDB, EmptyDBTyped}, inspectors::NoOpInspector, @@ -530,7 +531,7 @@ mod tests { #[test] fn receipts_by_block_hash() { // Create a default SealedBlockWithSenders object - let block: SealedBlockWithSenders = Default::default(); + let block: SealedBlockWithSenders = Default::default(); // Define block hashes for block1 and block2 let block1_hash = B256::new([0x01; 32]); @@ -548,24 +549,18 @@ mod tests { block2.block.header.set_hash(block2_hash); // Create a random receipt object, receipt1 - let receipt1 = Receipt { - tx_type: TxType::Legacy, + let receipt1 = OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - }; + status: true.into(), + }); // Create another random receipt object, receipt2 - let receipt2 = Receipt { - tx_type: TxType::Legacy, + let receipt2 = OpReceipt::Legacy(Receipt { cumulative_gas_used: 1325345, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - }; + status: true.into(), + }); // Create a Receipts object with a vector of receipt vectors let receipts = @@ -573,7 +568,7 @@ mod tests { // Create an ExecutionOutcome object with the created bundle, receipts, an empty requests // vector, and first_block set to 10 - let execution_outcome = ExecutionOutcome { + let execution_outcome = ExecutionOutcome:: { bundle: Default::default(), receipts, requests: vec![], @@ -614,14 +609,11 @@ mod tests { // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { - receipt_vec: vec![vec![Some(Receipt { - tx_type: TxType::Legacy, + receipt_vec: vec![vec![Some(OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - })]], + status: true.into(), + }))]], }; // Create a Requests object with a vector of requests @@ -677,14 +669,11 @@ mod tests { fn test_block_number_to_index() { // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { - receipt_vec: vec![vec![Some(Receipt { - tx_type: TxType::Legacy, + receipt_vec: vec![vec![Some(OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - })]], + status: true.into(), + }))]], }; // Define the first block number @@ -713,14 +702,11 @@ mod tests { fn test_get_logs() { // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { - receipt_vec: vec![vec![Some(Receipt { - tx_type: TxType::Legacy, + receipt_vec: vec![vec![Some(OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![Log::::default()], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - })]], + status: true.into(), + }))]], }; // Define the first block number @@ -746,14 +732,11 @@ mod tests { fn test_receipts_by_block() { // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { - receipt_vec: vec![vec![Some(Receipt { - tx_type: TxType::Legacy, + receipt_vec: vec![vec![Some(OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![Log::::default()], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - })]], + status: true.into(), + }))]], }; // Define the first block number @@ -774,14 +757,11 @@ mod tests { // Assert that the receipts for block number 123 match the expected receipts assert_eq!( receipts_by_block, - vec![&Some(Receipt { - tx_type: TxType::Legacy, + vec![&Some(OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![Log::::default()], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - })] + status: true.into(), + }))] ); } @@ -789,14 +769,11 @@ mod tests { fn test_receipts_len() { // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { - receipt_vec: vec![vec![Some(Receipt { - tx_type: TxType::Legacy, + receipt_vec: vec![vec![Some(OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![Log::::default()], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - })]], + status: true.into(), + }))]], }; // Create an empty Receipts object @@ -838,14 +815,11 @@ mod tests { #[test] fn test_revert_to() { // Create a random receipt object - let receipt = Receipt { - tx_type: TxType::Legacy, + let receipt = OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - }; + status: true.into(), + }); // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { @@ -888,14 +862,11 @@ mod tests { #[test] fn test_extend_execution_outcome() { // Create a Receipt object with specific attributes. - let receipt = Receipt { - tx_type: TxType::Legacy, + let receipt = OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - }; + status: true.into(), + }); // Create a Receipts object containing the receipt. let receipts = Receipts { receipt_vec: vec![vec![Some(receipt.clone())]] }; @@ -933,14 +904,11 @@ mod tests { #[test] fn test_split_at_execution_outcome() { // Create a random receipt object - let receipt = Receipt { - tx_type: TxType::Legacy, + let receipt = OpReceipt::Legacy(Receipt { cumulative_gas_used: 46913, logs: vec![], - success: true, - deposit_nonce: Some(18), - deposit_receipt_version: Some(34), - }; + status: true.into(), + }); // Create a Receipts object with a vector of receipt vectors let receipts = Receipts { diff --git a/crates/optimism/node/Cargo.toml b/crates/optimism/node/Cargo.toml index d1fcfd950de1..cad466dfc8da 100644 --- a/crates/optimism/node/Cargo.toml +++ b/crates/optimism/node/Cargo.toml @@ -43,7 +43,7 @@ reth-optimism-rpc.workspace = true reth-optimism-chainspec.workspace = true reth-optimism-consensus.workspace = true reth-optimism-forks.workspace = true -reth-optimism-primitives = { workspace = true, features = ["serde"] } +reth-optimism-primitives = { workspace = true, features = ["serde", "serde-bincode-compat"] } # revm with required optimism features revm = { workspace = true, features = ["secp256k1", "blst", "c-kzg"] } @@ -88,6 +88,7 @@ alloy-consensus.workspace = true futures.workspace = true [features] +default = ["reth-codec"] optimism = [ "reth-primitives/optimism", "reth-provider/optimism", diff --git a/crates/optimism/node/src/args.rs b/crates/optimism/node/src/args.rs index b84e98d28b19..658748c9c441 100644 --- a/crates/optimism/node/src/args.rs +++ b/crates/optimism/node/src/args.rs @@ -38,23 +38,12 @@ pub struct RollupArgs { #[arg(long = "rollup.discovery.v4", default_value = "false")] pub discovery_v4: bool, - /// Enable the experimental engine features on reth binary - /// - /// DEPRECATED: experimental engine is default now, use --engine.legacy to enable the legacy - /// functionality - #[arg(long = "engine.experimental", default_value = "false")] - pub experimental: bool, - - /// Enable the legacy engine on reth binary - #[arg(long = "engine.legacy", default_value = "false")] - pub legacy: bool, - /// Configure persistence threshold for engine experimental. - #[arg(long = "engine.persistence-threshold", conflicts_with = "legacy", default_value_t = DEFAULT_PERSISTENCE_THRESHOLD)] + #[arg(long = "engine.persistence-threshold", default_value_t = DEFAULT_PERSISTENCE_THRESHOLD)] pub persistence_threshold: u64, /// Configure the target number of blocks to keep in memory. - #[arg(long = "engine.memory-block-buffer-target", conflicts_with = "legacy", default_value_t = DEFAULT_MEMORY_BLOCK_BUFFER_TARGET)] + #[arg(long = "engine.memory-block-buffer-target", default_value_t = DEFAULT_MEMORY_BLOCK_BUFFER_TARGET)] pub memory_block_buffer_target: u64, } @@ -66,8 +55,6 @@ impl Default for RollupArgs { enable_genesis_walkback: false, compute_pending_block: false, discovery_v4: false, - experimental: false, - legacy: false, persistence_threshold: DEFAULT_PERSISTENCE_THRESHOLD, memory_block_buffer_target: DEFAULT_MEMORY_BLOCK_BUFFER_TARGET, } diff --git a/crates/optimism/node/src/engine.rs b/crates/optimism/node/src/engine.rs index c8fcad324ad8..70802f5adf59 100644 --- a/crates/optimism/node/src/engine.rs +++ b/crates/optimism/node/src/engine.rs @@ -18,8 +18,9 @@ use reth_node_api::{ use reth_optimism_chainspec::OpChainSpec; use reth_optimism_forks::{OpHardfork, OpHardforks}; use reth_optimism_payload_builder::{OpBuiltPayload, OpPayloadBuilderAttributes}; +use reth_optimism_primitives::OpBlock; use reth_payload_validator::ExecutionPayloadValidator; -use reth_primitives::{Block, SealedBlockFor}; +use reth_primitives::SealedBlockFor; use reth_rpc_types_compat::engine::payload::block_to_payload; use std::sync::Arc; @@ -38,7 +39,7 @@ impl PayloadTypes for OpEngineTypes { impl EngineTypes for OpEngineTypes where - T::BuiltPayload: BuiltPayload> + T::BuiltPayload: BuiltPayload> + TryInto + TryInto + TryInto @@ -89,7 +90,7 @@ impl OpEngineValidator { } impl PayloadValidator for OpEngineValidator { - type Block = Block; + type Block = OpBlock; fn ensure_well_formed_payload( &self, diff --git a/crates/optimism/node/src/node.rs b/crates/optimism/node/src/node.rs index 8d153e0c4223..568d1bfdb9fc 100644 --- a/crates/optimism/node/src/node.rs +++ b/crates/optimism/node/src/node.rs @@ -6,12 +6,12 @@ use crate::{ txpool::{OpTransactionPool, OpTransactionValidator}, OpEngineTypes, }; -use alloy_consensus::Header; +use op_alloy_consensus::OpPooledTransaction; use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig}; use reth_chainspec::{EthChainSpec, Hardforks}; use reth_evm::{execute::BasicBlockExecutorProvider, ConfigureEvm}; use reth_network::{NetworkConfig, NetworkHandle, NetworkManager, NetworkPrimitives, PeersInfo}; -use reth_node_api::{AddOnsContext, EngineValidator, FullNodeComponents, NodeAddOns, TxTy}; +use reth_node_api::{AddOnsContext, FullNodeComponents, HeaderTy, NodeAddOns, TxTy}; use reth_node_builder::{ components::{ ComponentsBuilder, ConsensusBuilder, ExecutorBuilder, NetworkBuilder, @@ -28,14 +28,13 @@ use reth_optimism_payload_builder::{ builder::OpPayloadTransactions, config::{OpBuilderConfig, OpDAConfig}, }; -use reth_optimism_primitives::OpPrimitives; +use reth_optimism_primitives::{OpPrimitives, OpReceipt, OpTransactionSigned}; use reth_optimism_rpc::{ miner::{MinerApiExtServer, OpMinerExtApi}, witness::{DebugExecutionWitnessApiServer, OpDebugWitnessApi}, OpEthApi, SequencerClient, }; use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService}; -use reth_primitives::{PooledTransaction, TransactionSigned}; use reth_provider::{CanonStateSubscriptions, EthStorage}; use reth_rpc_server_types::RethRpcModule; use reth_tracing::tracing::{debug, info}; @@ -47,7 +46,7 @@ use reth_trie_db::MerklePatriciaTrie; use std::sync::Arc; /// Storage implementation for Optimism. -pub type OpStorage = EthStorage; +pub type OpStorage = EthStorage; /// Type configuration for a regular Optimism node. #[derive(Debug, Default, Clone)] @@ -192,7 +191,6 @@ where Engine = OpEngineTypes, >, >, - OpEngineValidator: EngineValidator<::Engine>, { type Handle = RpcHandle>; @@ -242,7 +240,6 @@ where Engine = OpEngineTypes, >, >, - OpEngineValidator: EngineValidator<::Engine>, { type EthApi = OpEthApi; @@ -488,7 +485,7 @@ where Pool: TransactionPool>> + Unpin + 'static, - Evm: ConfigureEvm
, + Evm: ConfigureEvm
, Transaction = TxTy>, { let payload_builder = reth_optimism_payload_builder::OpPayloadBuilder::with_builder_config( evm_config, @@ -603,7 +600,10 @@ impl NetworkBuilder for OpNetworkBuilder where Node: FullNodeTypes>, Pool: TransactionPool< - Transaction: PoolTransaction, Pooled = PooledTransaction>, + Transaction: PoolTransaction< + Consensus = TxTy, + Pooled = OpPooledTransaction, + >, > + Unpin + 'static, { @@ -667,9 +667,9 @@ pub struct OpNetworkPrimitives; impl NetworkPrimitives for OpNetworkPrimitives { type BlockHeader = alloy_consensus::Header; - type BlockBody = reth_primitives::BlockBody; - type Block = reth_primitives::Block; - type BroadcastedTransaction = reth_primitives::TransactionSigned; - type PooledTransaction = reth_primitives::PooledTransaction; - type Receipt = reth_primitives::Receipt; + type BlockBody = reth_primitives::BlockBody; + type Block = reth_primitives::Block; + type BroadcastedTransaction = OpTransactionSigned; + type PooledTransaction = OpPooledTransaction; + type Receipt = OpReceipt; } diff --git a/crates/optimism/node/src/txpool.rs b/crates/optimism/node/src/txpool.rs index 4e2a947ed171..b598fc3a7efa 100644 --- a/crates/optimism/node/src/txpool.rs +++ b/crates/optimism/node/src/txpool.rs @@ -9,10 +9,10 @@ use parking_lot::RwLock; use reth_chainspec::ChainSpec; use reth_node_api::{Block, BlockBody}; use reth_optimism_evm::RethL1BlockInfo; -use reth_optimism_primitives::OpTransactionSigned; +use reth_optimism_primitives::{OpBlock, OpTransactionSigned}; use reth_primitives::{ transaction::TransactionConversionError, GotExpected, InvalidTransactionError, RecoveredTx, - SealedBlock, TransactionSigned, + SealedBlock, }; use reth_primitives_traits::SignedTransaction; use reth_provider::{BlockReaderIdExt, StateProviderFactory}; @@ -30,8 +30,8 @@ use std::sync::{ /// Type alias for default optimism transaction pool pub type OpTransactionPool = Pool< - TransactionValidationTaskExecutor>, - CoinbaseTipOrdering, + TransactionValidationTaskExecutor>, + CoinbaseTipOrdering, S, >; @@ -39,6 +39,13 @@ pub type OpTransactionPool = Pool< #[derive(Debug, Clone, derive_more::Deref)] pub struct OpPooledTransaction(EthPooledTransaction); +impl OpPooledTransaction { + /// Create new instance of [Self]. + pub fn new(transaction: RecoveredTx, encoded_length: usize) -> Self { + Self(EthPooledTransaction::new(transaction, encoded_length)) + } +} + impl From> for OpPooledTransaction { fn from(tx: RecoveredTx) -> Self { let encoded_len = tx.encode_2718_len(); @@ -241,7 +248,7 @@ impl OpTransactionValidator { impl OpTransactionValidator where Client: StateProviderFactory + BlockReaderIdExt, - Tx: EthPoolTransaction, + Tx: EthPoolTransaction, { /// Create a new [`OpTransactionValidator`]. pub fn new(inner: EthTransactionValidator) -> Self { @@ -373,8 +380,8 @@ where impl TransactionValidator for OpTransactionValidator where - Client: StateProviderFactory + BlockReaderIdExt, - Tx: EthPoolTransaction, + Client: StateProviderFactory + BlockReaderIdExt, + Tx: EthPoolTransaction, { type Transaction = Tx; @@ -417,16 +424,17 @@ pub struct OpL1BlockInfo { #[cfg(test)] mod tests { - use crate::txpool::OpTransactionValidator; + use crate::txpool::{OpPooledTransaction, OpTransactionValidator}; use alloy_eips::eip2718::Encodable2718; use alloy_primitives::{PrimitiveSignature as Signature, TxKind, U256}; - use op_alloy_consensus::TxDeposit; + use op_alloy_consensus::{OpTypedTransaction, TxDeposit}; use reth_chainspec::MAINNET; - use reth_primitives::{RecoveredTx, Transaction, TransactionSigned}; + use reth_optimism_primitives::OpTransactionSigned; + use reth_primitives::RecoveredTx; use reth_provider::test_utils::MockEthProvider; use reth_transaction_pool::{ - blobstore::InMemoryBlobStore, validate::EthTransactionValidatorBuilder, - EthPooledTransaction, TransactionOrigin, TransactionValidationOutcome, + blobstore::InMemoryBlobStore, validate::EthTransactionValidatorBuilder, TransactionOrigin, + TransactionValidationOutcome, }; #[test] fn validate_optimism_transaction() { @@ -439,7 +447,7 @@ mod tests { let origin = TransactionOrigin::External; let signer = Default::default(); - let deposit_tx = Transaction::Deposit(TxDeposit { + let deposit_tx = OpTypedTransaction::Deposit(TxDeposit { source_hash: Default::default(), from: signer, to: TxKind::Create, @@ -450,10 +458,10 @@ mod tests { input: Default::default(), }); let signature = Signature::test_signature(); - let signed_tx = TransactionSigned::new_unhashed(deposit_tx, signature); + let signed_tx = OpTransactionSigned::new_unhashed(deposit_tx, signature); let signed_recovered = RecoveredTx::from_signed_transaction(signed_tx, signer); let len = signed_recovered.encode_2718_len(); - let pooled_tx = EthPooledTransaction::new(signed_recovered, len); + let pooled_tx = OpPooledTransaction::new(signed_recovered, len); let outcome = validator.validate_one(origin, pooled_tx); let err = match outcome { diff --git a/crates/optimism/node/tests/it/builder.rs b/crates/optimism/node/tests/it/builder.rs index d6486881765d..fc0016fbcaf7 100644 --- a/crates/optimism/node/tests/it/builder.rs +++ b/crates/optimism/node/tests/it/builder.rs @@ -1,10 +1,11 @@ //! Node builder setup tests. use reth_db::test_utils::create_test_rw_db; -use reth_node_api::FullNodeComponents; +use reth_node_api::{FullNodeComponents, NodeTypesWithDBAdapter}; use reth_node_builder::{Node, NodeBuilder, NodeConfig}; use reth_optimism_chainspec::BASE_MAINNET; use reth_optimism_node::{args::RollupArgs, OpNode}; +use reth_provider::providers::BlockchainProvider2; #[test] fn test_basic_setup() { @@ -15,7 +16,7 @@ fn test_basic_setup() { let op_node = OpNode::new(args); let _builder = NodeBuilder::new(config) .with_database(db) - .with_types::() + .with_types_and_provider::>>() .with_components(op_node.components()) .with_add_ons(op_node.add_ons()) .on_component_initialized(move |ctx| { diff --git a/crates/optimism/node/tests/it/priority.rs b/crates/optimism/node/tests/it/priority.rs index 1b49ed684bfc..031300aba3e0 100644 --- a/crates/optimism/node/tests/it/priority.rs +++ b/crates/optimism/node/tests/it/priority.rs @@ -4,6 +4,7 @@ use alloy_consensus::TxEip1559; use alloy_genesis::Genesis; use alloy_network::TxSignerSync; use alloy_primitives::{Address, ChainId, TxKind}; +use op_alloy_consensus::OpTypedTransaction; use reth_chainspec::EthChainSpec; use reth_db::test_utils::create_test_rw_db_with_path; use reth_e2e_test_utils::{ @@ -25,9 +26,9 @@ use reth_optimism_node::{ OpEngineTypes, OpNode, }; use reth_optimism_payload_builder::builder::OpPayloadTransactions; -use reth_optimism_primitives::OpPrimitives; +use reth_optimism_primitives::{OpPrimitives, OpTransactionSigned}; use reth_payload_util::{PayloadTransactions, PayloadTransactionsChain, PayloadTransactionsFixed}; -use reth_primitives::{RecoveredTx, SealedBlock, Transaction, TransactionSigned}; +use reth_primitives::RecoveredTx; use reth_provider::providers::BlockchainProvider2; use reth_tasks::TaskManager; use reth_transaction_pool::{pool::BestPayloadTransactions, PoolTransaction}; @@ -44,10 +45,10 @@ impl OpPayloadTransactions for CustomTxPriority { &self, pool: Pool, attr: reth_transaction_pool::BestTransactionsAttributes, - ) -> impl PayloadTransactions + ) -> impl PayloadTransactions where Pool: reth_transaction_pool::TransactionPool< - Transaction: PoolTransaction, + Transaction: PoolTransaction, >, { // Block composition: @@ -67,7 +68,10 @@ impl OpPayloadTransactions for CustomTxPriority { }; let signature = sender.sign_transaction_sync(&mut end_of_block_tx).unwrap(); let end_of_block_tx = RecoveredTx::from_signed_transaction( - TransactionSigned::new_unhashed(Transaction::Eip1559(end_of_block_tx), signature), + OpTransactionSigned::new_unhashed( + OpTypedTransaction::Eip1559(end_of_block_tx), + signature, + ), sender.address(), ); @@ -182,12 +186,14 @@ async fn test_custom_block_priority_config() { .unwrap(); assert_eq!(block_payloads.len(), 1); let (block_payload, _) = block_payloads.first().unwrap(); - let block_payload: SealedBlock = block_payload.block().clone(); + let block_payload = block_payload.block().clone(); assert_eq!(block_payload.body.transactions.len(), 2); // L1 block info tx + end-of-block custom tx // Check that last transaction in the block looks like a transfer to a random address. let end_of_block_tx = block_payload.body.transactions.last().unwrap(); - let end_of_block_tx = end_of_block_tx.transaction.as_eip1559().unwrap(); + let OpTypedTransaction::Eip1559(end_of_block_tx) = &end_of_block_tx.transaction else { + panic!("expected EIP-1559 transaction"); + }; assert_eq!(end_of_block_tx.nonce, 1); assert_eq!(end_of_block_tx.gas_limit, 21_000); assert!(end_of_block_tx.input.is_empty()); diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 94bc853fc36a..3daff091c4f0 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -5,12 +5,12 @@ use crate::{ error::OpPayloadBuilderError, payload::{OpBuiltPayload, OpPayloadBuilderAttributes}, }; -use alloy_consensus::{Header, Transaction, EMPTY_OMMER_ROOT_HASH}; +use alloy_consensus::{Eip658Value, Header, Transaction, Typed2718, EMPTY_OMMER_ROOT_HASH}; use alloy_eips::{eip4895::Withdrawals, merge::BEACON_NONCE}; use alloy_primitives::{Address, Bytes, B256, U256}; use alloy_rpc_types_debug::ExecutionWitness; use alloy_rpc_types_engine::PayloadId; -use op_alloy_consensus::DepositTransaction; +use op_alloy_consensus::{OpDepositReceipt, OpTxType}; use op_alloy_rpc_types_engine::OpPayloadAttributes; use reth_basic_payload_builder::*; use reth_chain_state::ExecutedBlock; @@ -20,12 +20,13 @@ use reth_execution_types::ExecutionOutcome; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism; use reth_optimism_forks::OpHardforks; +use reth_optimism_primitives::{OpPrimitives, OpReceipt, OpTransactionSigned}; use reth_payload_builder_primitives::PayloadBuilderError; use reth_payload_primitives::PayloadBuilderAttributes; use reth_payload_util::{NoopPayloadTransactions, PayloadTransactions}; use reth_primitives::{ - proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt, Receipt, - SealedHeader, TransactionSigned, TxType, + proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt, + SealedHeader, TxType, }; use reth_provider::{ HashedPostStateProvider, ProviderError, StateProofProvider, StateProviderFactory, @@ -104,7 +105,7 @@ impl OpPayloadBuilder { } impl OpPayloadBuilder where - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvm
, { /// Constructs an Optimism payload from the transactions sent via the /// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in @@ -121,7 +122,7 @@ where ) -> Result, PayloadBuilderError> where Client: StateProviderFactory + ChainSpecProvider, - Txs: PayloadTransactions, + Txs: PayloadTransactions, { let evm_env = self .cfg_and_block_env(&args.config.attributes, &args.config.parent_header) @@ -217,8 +218,8 @@ where impl PayloadBuilder for OpPayloadBuilder where Client: StateProviderFactory + ChainSpecProvider, - Pool: TransactionPool>, - EvmConfig: ConfigureEvm
, + Pool: TransactionPool>, + EvmConfig: ConfigureEvm
, Txs: OpPayloadTransactions, { type Attributes = OpPayloadBuilderAttributes; @@ -293,7 +294,7 @@ impl<'a, Txs> OpBuilder<'a, Txs> { impl OpBuilder<'_, Txs> where - Txs: PayloadTransactions, + Txs: PayloadTransactions, { /// Executes the payload and returns the outcome. pub fn execute( @@ -302,7 +303,7 @@ where ctx: &OpPayloadBuilderCtx, ) -> Result, PayloadBuilderError> where - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvm
, DB: Database, { let Self { best } = self; @@ -347,7 +348,7 @@ where ctx: OpPayloadBuilderCtx, ) -> Result, PayloadBuilderError> where - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvm
, DB: Database + AsRef

, P: StateRootProvider + HashedPostStateProvider, { @@ -360,7 +361,7 @@ where let block_number = ctx.block_number(); let execution_outcome = ExecutionOutcome::new( state.take_bundle(), - vec![info.receipts].into(), + info.receipts.into(), block_number, Vec::new(), ); @@ -437,7 +438,7 @@ where debug!(target: "payload_builder", id=%ctx.attributes().payload_id(), sealed_block_header = ?sealed_block.header, "sealed built block"); // create the executed block data - let executed = ExecutedBlock { + let executed: ExecutedBlock = ExecutedBlock { block: sealed_block.clone(), senders: Arc::new(info.executed_senders), execution_output: Arc::new(execution_outcome), @@ -473,7 +474,7 @@ where ctx: &OpPayloadBuilderCtx, ) -> Result where - EvmConfig: ConfigureEvm

, + EvmConfig: ConfigureEvm
, DB: Database + AsRef

, P: StateProofProvider, { @@ -490,22 +491,22 @@ pub trait OpPayloadTransactions: Clone + Send + Sync + Unpin + 'static { /// Returns an iterator that yields the transaction in the order they should get included in the /// new payload. fn best_transactions< - Pool: TransactionPool>, + Pool: TransactionPool>, >( &self, pool: Pool, attr: BestTransactionsAttributes, - ) -> impl PayloadTransactions; + ) -> impl PayloadTransactions; } impl OpPayloadTransactions for () { fn best_transactions< - Pool: TransactionPool>, + Pool: TransactionPool>, >( &self, pool: Pool, attr: BestTransactionsAttributes, - ) -> impl PayloadTransactions { + ) -> impl PayloadTransactions { BestPayloadTransactions::new(pool.best_transactions_with_attributes(attr)) } } @@ -523,11 +524,11 @@ pub struct ExecutedPayload { #[derive(Default, Debug)] pub struct ExecutionInfo { /// All executed transactions (unrecovered). - pub executed_transactions: Vec, + pub executed_transactions: Vec, /// The recovered senders for the executed transactions. pub executed_senders: Vec

, /// The transaction receipts - pub receipts: Vec>, + pub receipts: Vec, /// All gas used so far pub cumulative_gas_used: u64, /// Tracks fees from executed mempool transactions @@ -708,7 +709,7 @@ impl OpPayloadBuilderCtx { impl OpPayloadBuilderCtx where - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvm
, { /// apply eip-4788 pre block contract call pub fn apply_pre_beacon_root_contract_call( @@ -816,18 +817,28 @@ where // add gas used by the transaction to cumulative gas used, before creating the receipt info.cumulative_gas_used += gas_used; - // Push transaction changeset and calculate header bloom filter for receipt. - info.receipts.push(Some(Receipt { - tx_type: sequencer_tx.tx_type(), - success: result.is_success(), - cumulative_gas_used: info.cumulative_gas_used, + let receipt = alloy_consensus::Receipt { + status: Eip658Value::Eip658(result.is_success()), + cumulative_gas_used: info.cumulative_gas_used as u128, logs: result.into_logs().into_iter().collect(), - deposit_nonce: depositor.map(|account| account.nonce), - // The deposit receipt version was introduced in Canyon to indicate an update to how - // receipt hashes should be computed when set. The state transition process - // ensures this is only set for post-Canyon deposit transactions. - deposit_receipt_version: self.is_canyon_active().then_some(1), - })); + }; + + // Push transaction changeset and calculate header bloom filter for receipt. + info.receipts.push(match sequencer_tx.tx_type() { + OpTxType::Legacy => OpReceipt::Legacy(receipt), + OpTxType::Eip2930 => OpReceipt::Eip2930(receipt), + OpTxType::Eip1559 => OpReceipt::Eip1559(receipt), + OpTxType::Eip7702 => OpReceipt::Eip7702(receipt), + OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt { + inner: receipt, + deposit_nonce: depositor.map(|account| account.nonce), + // The deposit receipt version was introduced in Canyon to indicate an update to + // how receipt hashes should be computed when set. The state + // transition process ensures this is only set for + // post-Canyon deposit transactions. + deposit_receipt_version: self.is_canyon_active().then_some(1), + }), + }); // append sender and transaction to the respective lists info.executed_senders.push(sequencer_tx.signer()); @@ -844,7 +855,7 @@ where &self, info: &mut ExecutionInfo, db: &mut State, - mut best_txs: impl PayloadTransactions, + mut best_txs: impl PayloadTransactions, ) -> Result, PayloadBuilderError> where DB: Database, @@ -917,15 +928,24 @@ where // receipt info.cumulative_gas_used += gas_used; - // Push transaction changeset and calculate header bloom filter for receipt. - info.receipts.push(Some(Receipt { - tx_type: tx.tx_type(), - success: result.is_success(), - cumulative_gas_used: info.cumulative_gas_used, + let receipt = alloy_consensus::Receipt { + status: Eip658Value::Eip658(result.is_success()), + cumulative_gas_used: info.cumulative_gas_used as u128, logs: result.into_logs().into_iter().collect(), - deposit_nonce: None, - deposit_receipt_version: None, - })); + }; + + // Push transaction changeset and calculate header bloom filter for receipt. + info.receipts.push(match tx.tx_type() { + OpTxType::Legacy => OpReceipt::Legacy(receipt), + OpTxType::Eip2930 => OpReceipt::Eip2930(receipt), + OpTxType::Eip1559 => OpReceipt::Eip1559(receipt), + OpTxType::Eip7702 => OpReceipt::Eip7702(receipt), + OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt { + inner: receipt, + deposit_nonce: None, + deposit_receipt_version: None, + }), + }); // update add to total fees let miner_fee = tx diff --git a/crates/optimism/payload/src/payload.rs b/crates/optimism/payload/src/payload.rs index d5ce7f9dadf1..ebf8fc66783c 100644 --- a/crates/optimism/payload/src/payload.rs +++ b/crates/optimism/payload/src/payload.rs @@ -14,10 +14,10 @@ use op_alloy_rpc_types_engine::{OpExecutionPayloadEnvelopeV3, OpExecutionPayload use reth_chain_state::ExecutedBlock; use reth_chainspec::EthereumHardforks; use reth_optimism_chainspec::OpChainSpec; -use reth_optimism_primitives::OpPrimitives; +use reth_optimism_primitives::{OpBlock, OpPrimitives, OpTransactionSigned}; use reth_payload_builder::EthPayloadBuilderAttributes; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes}; -use reth_primitives::{transaction::WithEncoded, SealedBlock, TransactionSigned}; +use reth_primitives::{transaction::WithEncoded, SealedBlockFor}; use reth_rpc_types_compat::engine::payload::{ block_to_payload_v1, block_to_payload_v3, convert_block_to_payload_field_v2, }; @@ -32,7 +32,7 @@ pub struct OpPayloadBuilderAttributes { pub no_tx_pool: bool, /// Decoded transactions and the original EIP-2718 encoded bytes as received in the payload /// attributes. - pub transactions: Vec>, + pub transactions: Vec>, /// The gas limit for the generated payload pub gas_limit: Option, /// EIP-1559 parameters for the generated payload @@ -71,8 +71,7 @@ impl PayloadBuilderAttributes for OpPayloadBuilderAttributes { .into_iter() .map(|data| { let mut buf = data.as_ref(); - let tx = - TransactionSigned::decode_2718(&mut buf).map_err(alloy_rlp::Error::from)?; + let tx = Decodable2718::decode_2718(&mut buf).map_err(alloy_rlp::Error::from)?; if !buf.is_empty() { return Err(alloy_rlp::Error::UnexpectedLength); @@ -136,9 +135,9 @@ pub struct OpBuiltPayload { /// Identifier of the payload pub(crate) id: PayloadId, /// The built block - pub(crate) block: Arc, + pub(crate) block: Arc>, /// Block execution data for the payload, if any. - pub(crate) executed_block: Option, + pub(crate) executed_block: Option>, /// The fees of the block pub(crate) fees: U256, /// The blobs, proofs, and commitments in the block. If the block is pre-cancun, this will be @@ -156,11 +155,11 @@ impl OpBuiltPayload { /// Initializes the payload with the given initial block. pub const fn new( id: PayloadId, - block: Arc, + block: Arc>, fees: U256, chain_spec: Arc, attributes: OpPayloadBuilderAttributes, - executed_block: Option, + executed_block: Option>, ) -> Self { Self { id, block, executed_block, fees, sidecars: Vec::new(), chain_spec, attributes } } @@ -171,7 +170,7 @@ impl OpBuiltPayload { } /// Returns the built block(sealed) - pub fn block(&self) -> &SealedBlock { + pub fn block(&self) -> &SealedBlockFor { &self.block } @@ -189,7 +188,7 @@ impl OpBuiltPayload { impl BuiltPayload for OpBuiltPayload { type Primitives = OpPrimitives; - fn block(&self) -> &SealedBlock { + fn block(&self) -> &SealedBlockFor { &self.block } @@ -197,7 +196,7 @@ impl BuiltPayload for OpBuiltPayload { self.fees } - fn executed_block(&self) -> Option { + fn executed_block(&self) -> Option> { self.executed_block.clone() } @@ -209,7 +208,7 @@ impl BuiltPayload for OpBuiltPayload { impl BuiltPayload for &OpBuiltPayload { type Primitives = OpPrimitives; - fn block(&self) -> &SealedBlock { + fn block(&self) -> &SealedBlockFor { (**self).block() } @@ -217,7 +216,7 @@ impl BuiltPayload for &OpBuiltPayload { (**self).fees() } - fn executed_block(&self) -> Option { + fn executed_block(&self) -> Option> { self.executed_block.clone() } diff --git a/crates/optimism/primitives/src/lib.rs b/crates/optimism/primitives/src/lib.rs index cb955b9cf080..337d8da77beb 100644 --- a/crates/optimism/primitives/src/lib.rs +++ b/crates/optimism/primitives/src/lib.rs @@ -14,25 +14,27 @@ extern crate alloc; pub mod bedrock; pub mod transaction; +use reth_primitives_traits::Block; pub use transaction::{signed::OpTransactionSigned, tx_type::OpTxType}; mod receipt; pub use receipt::OpReceipt; -/// Optimism primitive types. -pub type OpPrimitives = reth_primitives::EthPrimitives; - -// TODO: once we are ready for separating primitive types, introduce a separate `NodePrimitives` -// implementation used exclusively by legacy engine. -// -// #[derive(Debug, Default, Clone, PartialEq, Eq)] -// pub struct OpPrimitives; -// -// impl NodePrimitives for OpPrimitives { -// type Block = Block; -// type BlockHeader = Header; -// type BlockBody = BlockBody; -// type SignedTx = TransactionSigned; -// type TxType = OpTxType; -// type Receipt = Receipt; -// } +/// Optimism-specific block type. +pub type OpBlock = reth_primitives::Block; + +/// Optimism-specific block body type. +pub type OpBlockBody = ::Body; + +/// Primitive types for Optimism Node. +#[derive(Debug, Default, Clone, PartialEq, Eq)] +pub struct OpPrimitives; + +#[cfg(feature = "optimism")] +impl reth_primitives::NodePrimitives for OpPrimitives { + type Block = OpBlock; + type BlockHeader = alloy_consensus::Header; + type BlockBody = OpBlockBody; + type SignedTx = OpTransactionSigned; + type Receipt = OpReceipt; +} diff --git a/crates/optimism/primitives/src/transaction/signed.rs b/crates/optimism/primitives/src/transaction/signed.rs index 6f1b7f90e25f..5ebb4b7f6151 100644 --- a/crates/optimism/primitives/src/transaction/signed.rs +++ b/crates/optimism/primitives/src/transaction/signed.rs @@ -66,6 +66,11 @@ impl OpTransactionSigned { pub fn new_unhashed(transaction: OpTypedTransaction, signature: Signature) -> Self { Self { hash: Default::default(), signature, transaction } } + + /// Returns whether this transaction is a deposit. + pub const fn is_deposit(&self) -> bool { + matches!(self.transaction, OpTypedTransaction::Deposit(_)) + } } impl SignedTransaction for OpTransactionSigned { diff --git a/crates/optimism/rpc/Cargo.toml b/crates/optimism/rpc/Cargo.toml index d4a0b1fce273..1bc4071f16d0 100644 --- a/crates/optimism/rpc/Cargo.toml +++ b/crates/optimism/rpc/Cargo.toml @@ -15,6 +15,7 @@ workspace = true # reth reth-evm.workspace = true reth-primitives.workspace = true +reth-primitives-traits.workspace = true reth-provider.workspace = true reth-rpc-eth-api.workspace = true reth-rpc-eth-types.workspace = true diff --git a/crates/optimism/rpc/src/eth/block.rs b/crates/optimism/rpc/src/eth/block.rs index 3899e0b7f5c2..a4806f096b1a 100644 --- a/crates/optimism/rpc/src/eth/block.rs +++ b/crates/optimism/rpc/src/eth/block.rs @@ -7,7 +7,9 @@ use op_alloy_rpc_types::OpTransactionReceipt; use reth_chainspec::ChainSpecProvider; use reth_node_api::BlockBody; use reth_optimism_chainspec::OpChainSpec; -use reth_primitives::{Receipt, TransactionMeta, TransactionSigned}; +use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; +use reth_primitives::TransactionMeta; +use reth_primitives_traits::SignedTransaction; use reth_provider::{BlockReader, HeaderProvider}; use reth_rpc_eth_api::{ helpers::{EthBlocks, LoadBlock, LoadPendingBlock, LoadReceipt, SpawnBlocking}, @@ -21,7 +23,7 @@ where Self: LoadBlock< Error = OpEthApiError, NetworkTypes: Network, - Provider: BlockReader, + Provider: BlockReader, >, N: OpNodeCore + HeaderProvider>, { @@ -50,7 +52,7 @@ where .enumerate() .map(|(idx, (tx, receipt))| -> Result<_, _> { let meta = TransactionMeta { - tx_hash: tx.hash(), + tx_hash: *tx.tx_hash(), index: idx as u64, block_hash, block_number, diff --git a/crates/optimism/rpc/src/eth/pending_block.rs b/crates/optimism/rpc/src/eth/pending_block.rs index 535850c7f5ee..b8eb76197810 100644 --- a/crates/optimism/rpc/src/eth/pending_block.rs +++ b/crates/optimism/rpc/src/eth/pending_block.rs @@ -2,15 +2,18 @@ use crate::OpEthApi; use alloy_consensus::{ - constants::EMPTY_WITHDRAWALS, proofs::calculate_transaction_root, Header, EMPTY_OMMER_ROOT_HASH, + constants::EMPTY_WITHDRAWALS, proofs::calculate_transaction_root, Eip658Value, Header, + Transaction as _, TxReceipt, EMPTY_OMMER_ROOT_HASH, }; use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE, BlockNumberOrTag}; use alloy_primitives::{B256, U256}; +use op_alloy_consensus::{OpDepositReceipt, OpTxType}; use op_alloy_network::Network; use reth_chainspec::{EthChainSpec, EthereumHardforks}; use reth_evm::ConfigureEvm; use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism; -use reth_primitives::{logs_bloom, BlockBody, Receipt, SealedBlockWithSenders, TransactionSigned}; +use reth_optimism_primitives::{OpBlock, OpReceipt, OpTransactionSigned}; +use reth_primitives::{logs_bloom, BlockBody, SealedBlockWithSenders}; use reth_provider::{ BlockReader, BlockReaderIdExt, ChainSpecProvider, ProviderBlock, ProviderHeader, ProviderReceipt, ProviderTx, ReceiptProvider, StateProviderFactory, @@ -33,14 +36,17 @@ where >, N: RpcNodeCore< Provider: BlockReaderIdExt< - Transaction = reth_primitives::TransactionSigned, - Block = reth_primitives::Block, - Receipt = reth_primitives::Receipt, + Transaction = OpTransactionSigned, + Block = OpBlock, + Receipt = OpReceipt, Header = reth_primitives::Header, > + ChainSpecProvider + StateProviderFactory, Pool: TransactionPool>>, - Evm: ConfigureEvm
, + Evm: ConfigureEvm< + Header = ProviderHeader, + Transaction = ProviderTx, + >, >, { #[inline] @@ -55,7 +61,13 @@ where /// Returns the locally built pending block async fn local_pending_block( &self, - ) -> Result)>, Self::Error> { + ) -> Result< + Option<( + SealedBlockWithSenders>, + Vec>, + )>, + Self::Error, + > { // See: let latest = self .provider() @@ -97,7 +109,7 @@ where timestamp, ); - let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| &r.logs)); + let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| r.logs())); let is_cancun = chain_spec.is_cancun_active_at_timestamp(timestamp); let is_prague = chain_spec.is_prague_active_at_timestamp(timestamp); let is_shanghai = chain_spec.is_shanghai_active_at_timestamp(timestamp); @@ -118,7 +130,7 @@ where number: block_env.number.to::(), gas_limit: block_env.gas_limit.to::(), difficulty: U256::ZERO, - gas_used: receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default(), + gas_used: receipts.last().map(|r| r.cumulative_gas_used()).unwrap_or_default() as u64, blob_gas_used: is_cancun.then(|| { transactions.iter().map(|tx| tx.blob_gas_used().unwrap_or_default()).sum::() }), @@ -142,13 +154,22 @@ where result: ExecutionResult, cumulative_gas_used: u64, ) -> reth_provider::ProviderReceipt { - #[allow(clippy::needless_update)] - Receipt { - tx_type: tx.tx_type(), - success: result.is_success(), - cumulative_gas_used, + let receipt = alloy_consensus::Receipt { + status: Eip658Value::Eip658(result.is_success()), + cumulative_gas_used: cumulative_gas_used as u128, logs: result.into_logs().into_iter().collect(), - ..Default::default() + }; + + match tx.tx_type() { + OpTxType::Legacy => OpReceipt::Legacy(receipt), + OpTxType::Eip2930 => OpReceipt::Eip2930(receipt), + OpTxType::Eip1559 => OpReceipt::Eip1559(receipt), + OpTxType::Eip7702 => OpReceipt::Eip7702(receipt), + OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt { + inner: receipt, + deposit_nonce: None, + deposit_receipt_version: None, + }), } } } diff --git a/crates/optimism/rpc/src/eth/receipt.rs b/crates/optimism/rpc/src/eth/receipt.rs index 2a4df1ada49d..5cf389bafb86 100644 --- a/crates/optimism/rpc/src/eth/receipt.rs +++ b/crates/optimism/rpc/src/eth/receipt.rs @@ -2,15 +2,14 @@ use alloy_eips::eip2718::Encodable2718; use alloy_rpc_types_eth::{Log, TransactionReceipt}; -use op_alloy_consensus::{ - DepositTransaction, OpDepositReceipt, OpDepositReceiptWithBloom, OpReceiptEnvelope, -}; +use op_alloy_consensus::{OpDepositReceipt, OpDepositReceiptWithBloom, OpReceiptEnvelope}; use op_alloy_rpc_types::{L1BlockInfo, OpTransactionReceipt, OpTransactionReceiptFields}; use reth_node_api::{FullNodeComponents, NodeTypes}; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_evm::RethL1BlockInfo; use reth_optimism_forks::OpHardforks; -use reth_primitives::{Receipt, TransactionMeta, TransactionSigned, TxType}; +use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; +use reth_primitives::TransactionMeta; use reth_provider::{ChainSpecProvider, ReceiptProvider, TransactionsProvider}; use reth_rpc_eth_api::{helpers::LoadReceipt, FromEthApiError, RpcReceipt}; use reth_rpc_eth_types::{receipt::build_receipt, EthApiError}; @@ -21,14 +20,14 @@ impl LoadReceipt for OpEthApi where Self: Send + Sync, N: FullNodeComponents>, - Self::Provider: - TransactionsProvider + ReceiptProvider, + Self::Provider: TransactionsProvider + + ReceiptProvider, { async fn build_transaction_receipt( &self, - tx: TransactionSigned, + tx: OpTransactionSigned, meta: TransactionMeta, - receipt: Receipt, + receipt: OpReceipt, ) -> Result, Self::Error> { let (block, receipts) = self .inner @@ -107,7 +106,7 @@ impl OpReceiptFieldsBuilder { pub fn l1_block_info( mut self, chain_spec: &OpChainSpec, - tx: &TransactionSigned, + tx: &OpTransactionSigned, l1_block_info: revm::L1BlockInfo, ) -> Result { let raw_tx = tx.encoded_2718(); @@ -196,25 +195,21 @@ impl OpReceiptBuilder { /// Returns a new builder. pub fn new( chain_spec: &OpChainSpec, - transaction: &TransactionSigned, + transaction: &OpTransactionSigned, meta: TransactionMeta, - receipt: &Receipt, - all_receipts: &[Receipt], + receipt: &OpReceipt, + all_receipts: &[OpReceipt], l1_block_info: revm::L1BlockInfo, ) -> Result { let timestamp = meta.timestamp; let core_receipt = build_receipt(transaction, meta, receipt, all_receipts, |receipt_with_bloom| { - match receipt.tx_type { - TxType::Legacy => OpReceiptEnvelope::::Legacy(receipt_with_bloom), - TxType::Eip2930 => OpReceiptEnvelope::::Eip2930(receipt_with_bloom), - TxType::Eip1559 => OpReceiptEnvelope::::Eip1559(receipt_with_bloom), - TxType::Eip4844 => { - // TODO: unreachable - OpReceiptEnvelope::::Eip1559(receipt_with_bloom) - } - TxType::Eip7702 => OpReceiptEnvelope::::Eip7702(receipt_with_bloom), - TxType::Deposit => { + match receipt { + OpReceipt::Legacy(_) => OpReceiptEnvelope::::Legacy(receipt_with_bloom), + OpReceipt::Eip2930(_) => OpReceiptEnvelope::::Eip2930(receipt_with_bloom), + OpReceipt::Eip1559(_) => OpReceiptEnvelope::::Eip1559(receipt_with_bloom), + OpReceipt::Eip7702(_) => OpReceiptEnvelope::::Eip7702(receipt_with_bloom), + OpReceipt::Deposit(receipt) => { OpReceiptEnvelope::::Deposit(OpDepositReceiptWithBloom:: { receipt: OpDepositReceipt:: { inner: receipt_with_bloom.receipt, @@ -229,8 +224,6 @@ impl OpReceiptBuilder { let op_receipt_fields = OpReceiptFieldsBuilder::new(timestamp) .l1_block_info(chain_spec, transaction, l1_block_info)? - .deposit_nonce(receipt.deposit_nonce) - .deposit_version(receipt.deposit_receipt_version) .build(); Ok(Self { core_receipt, op_receipt_fields }) @@ -291,13 +284,14 @@ mod test { #[test] fn op_receipt_fields_from_block_and_tx() { // rig - let tx_0 = TransactionSigned::decode_2718( + let tx_0 = OpTransactionSigned::decode_2718( &mut TX_SET_L1_BLOCK_OP_MAINNET_BLOCK_124665056.as_slice(), ) .unwrap(); - let tx_1 = TransactionSigned::decode_2718(&mut TX_1_OP_MAINNET_BLOCK_124665056.as_slice()) - .unwrap(); + let tx_1 = + OpTransactionSigned::decode_2718(&mut TX_1_OP_MAINNET_BLOCK_124665056.as_slice()) + .unwrap(); let block = Block { body: BlockBody { transactions: [tx_0, tx_1.clone()].to_vec(), ..Default::default() }, @@ -363,7 +357,7 @@ mod test { fn base_receipt_gas_fields() { // https://basescan.org/tx/0x510fd4c47d78ba9f97c91b0f2ace954d5384c169c9545a77a373cf3ef8254e6e let system = hex!("7ef8f8a0389e292420bcbf9330741f72074e39562a09ff5a00fd22e4e9eee7e34b81bca494deaddeaddeaddeaddeaddeaddeaddeaddead00019442000000000000000000000000000000000000158080830f424080b8a4440a5e20000008dd00101c120000000000000004000000006721035b00000000014189960000000000000000000000000000000000000000000000000000000349b4dcdc000000000000000000000000000000000000000000000000000000004ef9325cc5991ce750960f636ca2ffbb6e209bb3ba91412f21dd78c14ff154d1930f1f9a0000000000000000000000005050f69a9786f081509234f1a7f4684b5e5b76c9"); - let tx_0 = TransactionSigned::decode_2718(&mut &system[..]).unwrap(); + let tx_0 = OpTransactionSigned::decode_2718(&mut &system[..]).unwrap(); let block = Block { body: BlockBody { transactions: vec![tx_0], ..Default::default() }, @@ -374,7 +368,7 @@ mod test { // https://basescan.org/tx/0xf9420cbaf66a2dda75a015488d37262cbfd4abd0aad7bb2be8a63e14b1fa7a94 let tx = hex!("02f86c8221058034839a4ae283021528942f16386bb37709016023232523ff6d9daf444be380841249c58bc080a001b927eda2af9b00b52a57be0885e0303c39dd2831732e14051c2336470fd468a0681bf120baf562915841a48601c2b54a6742511e535cf8f71c95115af7ff63bd"); - let tx_1 = TransactionSigned::decode_2718(&mut &tx[..]).unwrap(); + let tx_1 = OpTransactionSigned::decode_2718(&mut &tx[..]).unwrap(); let receipt_meta = OpReceiptFieldsBuilder::new(1730216981) .l1_block_info(&BASE_MAINNET, &tx_1, l1_block_info) diff --git a/crates/optimism/rpc/src/eth/transaction.rs b/crates/optimism/rpc/src/eth/transaction.rs index 468b46d97eba..05fbb0a95349 100644 --- a/crates/optimism/rpc/src/eth/transaction.rs +++ b/crates/optimism/rpc/src/eth/transaction.rs @@ -3,10 +3,12 @@ use alloy_consensus::{Signed, Transaction as _}; use alloy_primitives::{Bytes, PrimitiveSignature as Signature, Sealable, Sealed, B256}; use alloy_rpc_types_eth::TransactionInfo; -use op_alloy_consensus::OpTxEnvelope; -use op_alloy_rpc_types::Transaction; +use op_alloy_consensus::{OpTxEnvelope, OpTypedTransaction}; +use op_alloy_rpc_types::{OpTransactionRequest, Transaction}; use reth_node_api::FullNodeComponents; -use reth_primitives::{RecoveredTx, TransactionSigned}; +use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; +use reth_primitives::RecoveredTx; +use reth_primitives_traits::transaction::signed::SignedTransaction; use reth_provider::{ BlockReader, BlockReaderIdExt, ProviderTx, ReceiptProvider, TransactionsProvider, }; @@ -73,47 +75,40 @@ where } } -impl TransactionCompat for OpEthApi +impl TransactionCompat for OpEthApi where - N: FullNodeComponents>, + N: FullNodeComponents>, { type Transaction = Transaction; type Error = OpEthApiError; fn fill( &self, - tx: RecoveredTx, + tx: RecoveredTx, tx_info: TransactionInfo, ) -> Result { let from = tx.signer(); - let hash = tx.hash(); - let TransactionSigned { transaction, signature, .. } = tx.into_signed(); + let hash = *tx.tx_hash(); + let OpTransactionSigned { transaction, signature, .. } = tx.into_signed(); let mut deposit_receipt_version = None; let mut deposit_nonce = None; let inner = match transaction { - reth_primitives::Transaction::Legacy(tx) => { - Signed::new_unchecked(tx, signature, hash).into() - } - reth_primitives::Transaction::Eip2930(tx) => { - Signed::new_unchecked(tx, signature, hash).into() - } - reth_primitives::Transaction::Eip1559(tx) => { - Signed::new_unchecked(tx, signature, hash).into() - } - reth_primitives::Transaction::Eip4844(_) => unreachable!(), - reth_primitives::Transaction::Eip7702(tx) => { - Signed::new_unchecked(tx, signature, hash).into() - } - reth_primitives::Transaction::Deposit(tx) => { + OpTypedTransaction::Legacy(tx) => Signed::new_unchecked(tx, signature, hash).into(), + OpTypedTransaction::Eip2930(tx) => Signed::new_unchecked(tx, signature, hash).into(), + OpTypedTransaction::Eip1559(tx) => Signed::new_unchecked(tx, signature, hash).into(), + OpTypedTransaction::Eip7702(tx) => Signed::new_unchecked(tx, signature, hash).into(), + OpTypedTransaction::Deposit(tx) => { self.inner .eth_api .provider() .receipt_by_hash(hash) .map_err(Self::Error::from_eth_err)? .inspect(|receipt| { - deposit_receipt_version = receipt.deposit_receipt_version; - deposit_nonce = receipt.deposit_nonce; + if let OpReceipt::Deposit(receipt) = receipt { + deposit_receipt_version = receipt.deposit_receipt_version; + deposit_nonce = receipt.deposit_nonce; + } }); OpTxEnvelope::Deposit(tx.seal_unchecked(hash)) @@ -154,14 +149,15 @@ where fn build_simulate_v1_transaction( &self, request: alloy_rpc_types_eth::TransactionRequest, - ) -> Result { + ) -> Result { + let request: OpTransactionRequest = request.into(); let Ok(tx) = request.build_typed_tx() else { return Err(OpEthApiError::Eth(EthApiError::TransactionConversionError)) }; // Create an empty signature for the transaction. let signature = Signature::new(Default::default(), Default::default(), false); - Ok(TransactionSigned::new_unhashed(tx.into(), signature)) + Ok(OpTransactionSigned::new_unhashed(tx, signature)) } fn otterscan_api_truncate_input(tx: &mut Self::Transaction) { diff --git a/crates/optimism/rpc/src/witness.rs b/crates/optimism/rpc/src/witness.rs index d533bb187d94..387e6597edc2 100644 --- a/crates/optimism/rpc/src/witness.rs +++ b/crates/optimism/rpc/src/witness.rs @@ -9,7 +9,8 @@ use reth_chainspec::ChainSpecProvider; use reth_evm::ConfigureEvm; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_payload_builder::OpPayloadBuilder; -use reth_primitives::{SealedHeader, TransactionSigned}; +use reth_optimism_primitives::OpTransactionSigned; +use reth_primitives::SealedHeader; use reth_provider::{BlockReaderIdExt, ProviderError, ProviderResult, StateProviderFactory}; pub use reth_rpc_api::DebugExecutionWitnessApiServer; use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult}; @@ -58,7 +59,7 @@ where + ChainSpecProvider + Clone + 'static, - EvmConfig: ConfigureEvm
+ 'static, + EvmConfig: ConfigureEvm
+ 'static, { async fn execute_payload( &self, diff --git a/crates/payload/util/src/traits.rs b/crates/payload/util/src/traits.rs index f3a609a7abf5..3baed7d9da25 100644 --- a/crates/payload/util/src/traits.rs +++ b/crates/payload/util/src/traits.rs @@ -23,9 +23,15 @@ pub trait PayloadTransactions { } /// [`PayloadTransactions`] implementation that produces nothing. -#[derive(Debug, Default, Clone, Copy)] +#[derive(Debug, Clone, Copy)] pub struct NoopPayloadTransactions(core::marker::PhantomData); +impl Default for NoopPayloadTransactions { + fn default() -> Self { + Self(Default::default()) + } +} + impl PayloadTransactions for NoopPayloadTransactions { type Transaction = T; diff --git a/crates/rpc/rpc-types-compat/src/engine/payload.rs b/crates/rpc/rpc-types-compat/src/engine/payload.rs index 61f24fbd7261..99de9c063a7d 100644 --- a/crates/rpc/rpc-types-compat/src/engine/payload.rs +++ b/crates/rpc/rpc-types-compat/src/engine/payload.rs @@ -207,7 +207,9 @@ pub fn block_to_payload_v3( } /// Converts [`SealedBlock`] to [`ExecutionPayloadFieldV2`] -pub fn convert_block_to_payload_field_v2(value: SealedBlock) -> ExecutionPayloadFieldV2 { +pub fn convert_block_to_payload_field_v2( + value: SealedBlock>, +) -> ExecutionPayloadFieldV2 { // if there are withdrawals, return V2 if value.body.withdrawals.is_some() { ExecutionPayloadFieldV2::V2(block_to_payload_v2(value)) diff --git a/crates/storage/db-api/Cargo.toml b/crates/storage/db-api/Cargo.toml index 0fa030a8cd30..671b67d6e5cb 100644 --- a/crates/storage/db-api/Cargo.toml +++ b/crates/storage/db-api/Cargo.toml @@ -15,7 +15,6 @@ workspace = true # reth reth-codecs.workspace = true reth-db-models.workspace = true -reth-optimism-primitives = { workspace = true, optional = true } reth-primitives = { workspace = true, features = ["reth-codec"] } reth-primitives-traits = { workspace = true, features = ["serde", "reth-codec"] } reth-prune-types.workspace = true @@ -28,6 +27,9 @@ alloy-primitives.workspace = true alloy-genesis.workspace = true alloy-consensus.workspace = true +# optimism +reth-optimism-primitives = { workspace = true, optional = true } + # codecs modular-bitfield.workspace = true roaring = "0.10.2" @@ -87,6 +89,7 @@ arbitrary = [ optimism = [ "reth-primitives/optimism", "reth-codecs/op", - "reth-optimism-primitives?/optimism" + "reth-optimism-primitives?/optimism", + "op", ] op = ["dep:reth-optimism-primitives", "reth-codecs/op"] diff --git a/crates/storage/db/Cargo.toml b/crates/storage/db/Cargo.toml index b7114a430663..ab1608eddeb7 100644 --- a/crates/storage/db/Cargo.toml +++ b/crates/storage/db/Cargo.toml @@ -112,6 +112,7 @@ arbitrary = [ "alloy-consensus/arbitrary", ] optimism = ["reth-primitives/optimism", "reth-db-api/optimism"] +op = ["reth-db-api/op"] disable-lock = [] [[bench]] diff --git a/crates/storage/provider/src/providers/blockchain_provider.rs b/crates/storage/provider/src/providers/blockchain_provider.rs index 6d5413cfe95a..dd23128cb1c4 100644 --- a/crates/storage/provider/src/providers/blockchain_provider.rs +++ b/crates/storage/provider/src/providers/blockchain_provider.rs @@ -706,9 +706,7 @@ where } } -impl> CanonStateSubscriptions - for BlockchainProvider2 -{ +impl CanonStateSubscriptions for BlockchainProvider2 { fn subscribe_to_canonical_state(&self) -> CanonStateNotifications { self.canonical_in_memory_state.subscribe_canon_state() } diff --git a/crates/storage/provider/src/providers/database/chain.rs b/crates/storage/provider/src/providers/database/chain.rs index 57bc2e0b5ce6..24f4888ec397 100644 --- a/crates/storage/provider/src/providers/database/chain.rs +++ b/crates/storage/provider/src/providers/database/chain.rs @@ -1,7 +1,6 @@ use crate::{providers::NodeTypesForProvider, DatabaseProvider}; use reth_db::transaction::{DbTx, DbTxMut}; -use reth_node_types::FullNodePrimitives; -use reth_primitives::EthPrimitives; +use reth_node_types::{FullNodePrimitives, FullSignedTx}; use reth_storage_api::{ChainStorageReader, ChainStorageWriter, EthStorage}; /// Trait that provides access to implementations of [`ChainStorage`] @@ -19,23 +18,27 @@ pub trait ChainStorage: Send + Sync { Types: NodeTypesForProvider; } -impl ChainStorage for EthStorage { - fn reader( - &self, - ) -> impl ChainStorageReader, EthPrimitives> +impl ChainStorage for EthStorage +where + T: FullSignedTx, + N: FullNodePrimitives< + Block = reth_primitives::Block, + BlockBody = reth_primitives::BlockBody, + SignedTx = T, + >, +{ + fn reader(&self) -> impl ChainStorageReader, N> where TX: DbTx + 'static, - Types: NodeTypesForProvider, + Types: NodeTypesForProvider, { self } - fn writer( - &self, - ) -> impl ChainStorageWriter, EthPrimitives> + fn writer(&self) -> impl ChainStorageWriter, N> where TX: DbTxMut + DbTx + 'static, - Types: NodeTypesForProvider, + Types: NodeTypesForProvider, { self } diff --git a/crates/storage/provider/src/test_utils/mod.rs b/crates/storage/provider/src/test_utils/mod.rs index b6788914ee83..1fe4404cd860 100644 --- a/crates/storage/provider/src/test_utils/mod.rs +++ b/crates/storage/provider/src/test_utils/mod.rs @@ -9,7 +9,7 @@ use reth_db::{ DatabaseEnv, }; use reth_errors::ProviderResult; -use reth_node_types::NodeTypesWithDBAdapter; +use reth_node_types::{NodeTypes, NodeTypesWithDBAdapter}; use reth_primitives::{Account, StorageEntry}; use reth_trie::StateRoot; use reth_trie_db::DatabaseStateRoot; @@ -45,6 +45,13 @@ pub fn create_test_provider_factory() -> ProviderFactory { pub fn create_test_provider_factory_with_chain_spec( chain_spec: Arc, ) -> ProviderFactory { + create_test_provider_factory_with_node_types::(chain_spec) +} + +/// Creates test provider factory with provided chain spec. +pub fn create_test_provider_factory_with_node_types( + chain_spec: Arc, +) -> ProviderFactory>>> { let (static_dir, _) = create_test_static_files_dir(); let db = create_test_rw_db(); ProviderFactory::new( diff --git a/crates/transaction-pool/src/traits.rs b/crates/transaction-pool/src/traits.rs index 9db8c64d37a7..3ce92576d134 100644 --- a/crates/transaction-pool/src/traits.rs +++ b/crates/transaction-pool/src/traits.rs @@ -978,7 +978,7 @@ pub trait PoolTransaction: type TryFromConsensusError: fmt::Display; /// Associated type representing the raw consensus variant of the transaction. - type Consensus: From; + type Consensus: SignedTransaction + From; /// Associated type representing the recovered pooled variant of the transaction. type Pooled: SignedTransaction;