mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: integrate OpPrimitives (#13556)
This commit is contained in:
3
Cargo.lock
generated
3
Cargo.lock
generated
@ -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",
|
||||
|
||||
@ -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<E, ChainSpec> {
|
||||
pub struct EngineApiTestContext<E, ChainSpec, N: NodePrimitives> {
|
||||
pub chain_spec: Arc<ChainSpec>,
|
||||
pub canonical_stream: CanonStateNotificationStream,
|
||||
pub canonical_stream: CanonStateNotificationStream<N>,
|
||||
pub engine_api_client: HttpClient<AuthClientService<HttpBackend>>,
|
||||
pub _marker: PhantomData<E>,
|
||||
}
|
||||
|
||||
impl<E: EngineTypes, ChainSpec: EthereumHardforks> EngineApiTestContext<E, ChainSpec> {
|
||||
impl<E: EngineTypes, ChainSpec: EthereumHardforks, N: NodePrimitives>
|
||||
EngineApiTestContext<E, ChainSpec, N>
|
||||
{
|
||||
/// Retrieves a v3 payload from the engine api
|
||||
pub async fn get_payload_v3(
|
||||
&self,
|
||||
|
||||
@ -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<N>(
|
||||
where
|
||||
N: Default
|
||||
+ Node<TmpNodeAdapter<N, BlockchainProvider2<NodeTypesWithDBAdapter<N, TmpDB>>>>
|
||||
+ NodeTypesWithEngine<Primitives = EthPrimitives>
|
||||
+ NodeTypesWithEngine
|
||||
+ NodeTypesForProvider,
|
||||
N::ComponentsBuilder: NodeComponentsBuilder<
|
||||
TmpNodeAdapter<N, BlockchainProvider2<NodeTypesWithDBAdapter<N, TmpDB>>>,
|
||||
@ -132,10 +130,7 @@ where
|
||||
>,
|
||||
>,
|
||||
N::AddOns: RethRpcAddOns<Adapter<N, BlockchainProvider2<NodeTypesWithDBAdapter<N, TmpDB>>>>
|
||||
+ EngineValidatorAddOn<
|
||||
Adapter<N, BlockchainProvider2<NodeTypesWithDBAdapter<N, TmpDB>>>,
|
||||
Validator: EngineValidator<N::Engine, Block = reth_primitives::Block>,
|
||||
>,
|
||||
+ EngineValidatorAddOn<Adapter<N, BlockchainProvider2<NodeTypesWithDBAdapter<N, TmpDB>>>>,
|
||||
LocalPayloadAttributesBuilder<N::ChainSpec>: PayloadAttributesBuilder<
|
||||
<<N as NodeTypesWithEngine>::Engine as PayloadTypes>::PayloadAttributes,
|
||||
>,
|
||||
|
||||
@ -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<Node, AddOns>
|
||||
where
|
||||
Node: FullNodeComponents,
|
||||
@ -41,6 +41,7 @@ where
|
||||
pub engine_api: EngineApiTestContext<
|
||||
<Node::Types as NodeTypesWithEngine>::Engine,
|
||||
<Node::Types as NodeTypes>::ChainSpec,
|
||||
<Node::Types as NodeTypes>::Primitives,
|
||||
>,
|
||||
/// Context for testing RPC features.
|
||||
pub rpc: RpcTestContext<Node, AddOns::EthApi>,
|
||||
@ -50,11 +51,7 @@ impl<Node, Engine, AddOns> NodeTestContext<Node, AddOns>
|
||||
where
|
||||
Engine: EngineTypes,
|
||||
Node: FullNodeComponents,
|
||||
Node::Types: NodeTypesWithEngine<
|
||||
ChainSpec: EthereumHardforks,
|
||||
Engine = Engine,
|
||||
Primitives = EthPrimitives,
|
||||
>,
|
||||
Node::Types: NodeTypesWithEngine<ChainSpec: EthereumHardforks, Engine = Engine>,
|
||||
Node::Network: PeersHandleProvider,
|
||||
AddOns: RethRpcAddOns<Node>,
|
||||
{
|
||||
@ -97,7 +94,7 @@ where
|
||||
where
|
||||
Engine::ExecutionPayloadEnvelopeV3: From<Engine::BuiltPayload> + PayloadEnvelopeExt,
|
||||
Engine::ExecutionPayloadEnvelopeV4: From<Engine::BuiltPayload> + PayloadEnvelopeExt,
|
||||
AddOns::EthApi: EthApiSpec<Provider: BlockReader<Block = reth_primitives::Block>>
|
||||
AddOns::EthApi: EthApiSpec<Provider: BlockReader<Block = BlockTy<Node::Types>>>
|
||||
+ 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
|
||||
|
||||
@ -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<Node: FullNodeComponents, EthApi: EthApiTypes> {
|
||||
|
||||
impl<Node, EthApi> RpcTestContext<Node, EthApi>
|
||||
where
|
||||
Node: FullNodeComponents<
|
||||
Types: NodeTypes<
|
||||
ChainSpec: EthereumHardforks,
|
||||
Primitives: NodePrimitives<
|
||||
Block = reth_primitives::Block,
|
||||
Receipt = reth_primitives::Receipt,
|
||||
>,
|
||||
>,
|
||||
>,
|
||||
EthApi: EthApiSpec<Provider: BlockReader<Block = reth_primitives::Block>>
|
||||
Node: FullNodeComponents<Types: NodeTypes<ChainSpec: EthereumHardforks>>,
|
||||
EthApi: EthApiSpec<Provider: BlockReader<Block = BlockTy<Node::Types>>>
|
||||
+ EthTransactions
|
||||
+ TraceExt,
|
||||
{
|
||||
|
||||
@ -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"
|
||||
]
|
||||
|
||||
@ -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,
|
||||
};
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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<Types, DB, T, CB, AO> LaunchNode<NodeBuilderWithComponents<T, CB, AO>> for EngineNodeLauncher
|
||||
where
|
||||
Types: NodeTypesForProvider + NodeTypesWithEngine<Primitives = EthPrimitives>,
|
||||
Types: NodeTypesForProvider + NodeTypesWithEngine,
|
||||
DB: Database + DatabaseMetrics + DatabaseMetadata + Clone + Unpin + 'static,
|
||||
T: FullNodeTypes<
|
||||
Types = Types,
|
||||
@ -83,11 +83,7 @@ where
|
||||
>,
|
||||
CB: NodeComponentsBuilder<T>,
|
||||
AO: RethRpcAddOns<NodeAdapter<T, CB::Components>>
|
||||
+ EngineValidatorAddOn<
|
||||
NodeAdapter<T, CB::Components>,
|
||||
Validator: EngineValidator<Types::Engine, Block = BlockTy<Types>>,
|
||||
>,
|
||||
|
||||
+ EngineValidatorAddOn<NodeAdapter<T, CB::Components>>,
|
||||
LocalPayloadAttributesBuilder<Types::ChainSpec>: PayloadAttributesBuilder<
|
||||
<<Types as NodeTypesWithEngine>::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.
|
||||
|
||||
@ -23,12 +23,6 @@ fn main() {
|
||||
|
||||
if let Err(err) =
|
||||
Cli::<OpChainSpecParser, RollupArgs>::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);
|
||||
@ -49,14 +43,6 @@ fn main() {
|
||||
.await?;
|
||||
|
||||
handle.node_exit_future.await
|
||||
}
|
||||
true => {
|
||||
let handle =
|
||||
builder.node(OpNode::new(rollup_args.clone())).launch().await?;
|
||||
|
||||
handle.node_exit_future.await
|
||||
}
|
||||
}
|
||||
})
|
||||
{
|
||||
eprintln!("Error: {err:?}");
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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<C: ChainSpecParser<ChainSpec = OpChainSpec>> ImportOpCommand<C> {
|
||||
let mut total_decoded_txns = 0;
|
||||
let mut total_filtered_out_dup_txns = 0;
|
||||
|
||||
while let Some(mut file_client) = reader.next_chunk::<FileClient>().await? {
|
||||
while let Some(mut file_client) = reader.next_chunk::<FileClient<BlockTy<N>>>().await? {
|
||||
// create a new FileClient from chunk read from file
|
||||
info!(target: "reth::cli",
|
||||
"Importing chain file chunk"
|
||||
|
||||
@ -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<C: ChainSpecParser<ChainSpec = OpChainSpec>> ImportReceiptsOpCommand<C> {
|
||||
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<N, P, F>(
|
||||
filter: F,
|
||||
) -> eyre::Result<()>
|
||||
where
|
||||
N: ProviderNodeTypes<
|
||||
ChainSpec = OpChainSpec,
|
||||
Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>,
|
||||
>,
|
||||
N: ProviderNodeTypes<ChainSpec = OpChainSpec, Primitives: NodePrimitives<Receipt = OpReceipt>>,
|
||||
P: AsRef<Path>,
|
||||
F: FnMut(u64, &mut Receipts) -> usize,
|
||||
F: FnMut(u64, &mut Receipts<OpReceipt>) -> 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<N, F>(
|
||||
mut filter: F,
|
||||
) -> eyre::Result<ImportReceiptsResult>
|
||||
where
|
||||
N: ProviderNodeTypes<Primitives: NodePrimitives<Receipt = reth_primitives::Receipt>>,
|
||||
F: FnMut(u64, &mut Receipts) -> usize,
|
||||
N: ProviderNodeTypes<Primitives: NodePrimitives<Receipt = OpReceipt>>,
|
||||
F: FnMut(u64, &mut Receipts<ReceiptTy<N>>) -> 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::<ReceiptFileClient<HackReceiptFileCodec>>().await?
|
||||
reader.next_receipts_chunk::<ReceiptFileClient<HackReceiptFileCodec<OpReceipt>>>().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::<OpNode>(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);
|
||||
|
||||
@ -35,6 +35,7 @@ tracing.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
alloy-primitives.workspace = true
|
||||
op-alloy-consensus.workspace = true
|
||||
reth-optimism-chainspec.workspace = true
|
||||
|
||||
[features]
|
||||
|
||||
@ -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<OpPrimitives> for OpBeaconConsensus {
|
||||
fn validate_block_post_execution(
|
||||
&self,
|
||||
block: &BlockWithSenders,
|
||||
input: PostExecutionInput<'_>,
|
||||
block: &BlockWithSenders<OpBlock>,
|
||||
input: PostExecutionInput<'_, OpReceipt>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
validate_block_post_execution(block, &self.chain_spec, input.receipts)
|
||||
}
|
||||
}
|
||||
|
||||
impl Consensus for OpBeaconConsensus {
|
||||
impl Consensus<Header, OpBlockBody> 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<OpBlock>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
// Check ommers hash
|
||||
let ommers_hash = reth_primitives::proofs::calculate_ommers_root(&block.body.ommers);
|
||||
if block.header.ommers_hash != ommers_hash {
|
||||
|
||||
@ -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<Receipt>],
|
||||
receipts: &[ReceiptWithBloom<OpReceipt>],
|
||||
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::<Vec<_>>();
|
||||
@ -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();
|
||||
if let OpReceipt::Deposit(r) = &mut r {
|
||||
r.deposit_nonce = None;
|
||||
}
|
||||
r
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
@ -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,
|
||||
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];
|
||||
|
||||
@ -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<OpBlock>,
|
||||
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::<Vec<_>>();
|
||||
let receipts_with_bloom = receipts.iter().cloned().map(Into::into).collect::<Vec<_>>();
|
||||
let receipts_root =
|
||||
calculate_receipt_root_optimism(&receipts_with_bloom, chain_spec, timestamp);
|
||||
|
||||
|
||||
@ -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<Header = alloy_consensus::Header, Transaction = TransactionSigned>,
|
||||
+ ConfigureEvm<Header = alloy_consensus::Header, Transaction = OpTransactionSigned>,
|
||||
{
|
||||
type Primitives = OpPrimitives;
|
||||
type Strategy<DB: Database<Error: Into<ProviderError> + Display>> =
|
||||
@ -121,7 +122,7 @@ where
|
||||
impl<DB, EvmConfig> BlockExecutionStrategy for OpExecutionStrategy<DB, EvmConfig>
|
||||
where
|
||||
DB: Database<Error: Into<ProviderError> + Display>,
|
||||
EvmConfig: ConfigureEvm<Header = alloy_consensus::Header, Transaction = TransactionSigned>,
|
||||
EvmConfig: ConfigureEvm<Header = alloy_consensus::Header, Transaction = OpTransactionSigned>,
|
||||
{
|
||||
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<OpBlock>,
|
||||
) -> 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<ExecuteOutput<Receipt>, Self::Error> {
|
||||
block: &BlockWithSenders<OpBlock>,
|
||||
) -> Result<ExecuteOutput<OpReceipt>, 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(),
|
||||
};
|
||||
|
||||
// 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.
|
||||
// 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<OpBlock>,
|
||||
_receipts: &[OpReceipt],
|
||||
) -> Result<Requests, Self::Error> {
|
||||
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<OpBlock>,
|
||||
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();
|
||||
|
||||
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());
|
||||
// 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());
|
||||
|
||||
// deposit_nonce is present only in deposit transactions
|
||||
assert!(deposit_receipt.deposit_nonce.is_some());
|
||||
assert!(tx_receipt.deposit_nonce.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());
|
||||
}
|
||||
}
|
||||
|
||||
@ -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<OpBlock> = 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::<OpReceipt> {
|
||||
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::<LogData>::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::<LogData>::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::<LogData>::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::<LogData>::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 {
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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,
|
||||
}
|
||||
|
||||
@ -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<T: PayloadTypes> PayloadTypes for OpEngineTypes<T> {
|
||||
|
||||
impl<T: PayloadTypes> EngineTypes for OpEngineTypes<T>
|
||||
where
|
||||
T::BuiltPayload: BuiltPayload<Primitives: NodePrimitives<Block = reth_primitives::Block>>
|
||||
T::BuiltPayload: BuiltPayload<Primitives: NodePrimitives<Block = OpBlock>>
|
||||
+ TryInto<ExecutionPayloadV1>
|
||||
+ TryInto<ExecutionPayloadEnvelopeV2>
|
||||
+ TryInto<OpExecutionPayloadEnvelopeV3>
|
||||
@ -89,7 +90,7 @@ impl OpEngineValidator {
|
||||
}
|
||||
|
||||
impl PayloadValidator for OpEngineValidator {
|
||||
type Block = Block;
|
||||
type Block = OpBlock;
|
||||
|
||||
fn ensure_well_formed_payload(
|
||||
&self,
|
||||
|
||||
@ -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<OpTransactionSigned>;
|
||||
|
||||
/// Type configuration for a regular Optimism node.
|
||||
#[derive(Debug, Default, Clone)]
|
||||
@ -192,7 +191,6 @@ where
|
||||
Engine = OpEngineTypes,
|
||||
>,
|
||||
>,
|
||||
OpEngineValidator: EngineValidator<<N::Types as NodeTypesWithEngine>::Engine>,
|
||||
{
|
||||
type Handle = RpcHandle<N, OpEthApi<N>>;
|
||||
|
||||
@ -242,7 +240,6 @@ where
|
||||
Engine = OpEngineTypes,
|
||||
>,
|
||||
>,
|
||||
OpEngineValidator: EngineValidator<<N::Types as NodeTypesWithEngine>::Engine>,
|
||||
{
|
||||
type EthApi = OpEthApi<N>;
|
||||
|
||||
@ -488,7 +485,7 @@ where
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
|
||||
+ Unpin
|
||||
+ 'static,
|
||||
Evm: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
Evm: ConfigureEvm<Header = HeaderTy<Node::Types>, Transaction = TxTy<Node::Types>>,
|
||||
{
|
||||
let payload_builder = reth_optimism_payload_builder::OpPayloadBuilder::with_builder_config(
|
||||
evm_config,
|
||||
@ -603,7 +600,10 @@ impl<Node, Pool> NetworkBuilder<Node, Pool> for OpNetworkBuilder
|
||||
where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = OpChainSpec, Primitives = OpPrimitives>>,
|
||||
Pool: TransactionPool<
|
||||
Transaction: PoolTransaction<Consensus = TxTy<Node::Types>, Pooled = PooledTransaction>,
|
||||
Transaction: PoolTransaction<
|
||||
Consensus = TxTy<Node::Types>,
|
||||
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<OpTransactionSigned>;
|
||||
type Block = reth_primitives::Block<OpTransactionSigned>;
|
||||
type BroadcastedTransaction = OpTransactionSigned;
|
||||
type PooledTransaction = OpPooledTransaction;
|
||||
type Receipt = OpReceipt;
|
||||
}
|
||||
|
||||
@ -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<Client, S> = Pool<
|
||||
TransactionValidationTaskExecutor<OpTransactionValidator<Client, EthPooledTransaction>>,
|
||||
CoinbaseTipOrdering<EthPooledTransaction>,
|
||||
TransactionValidationTaskExecutor<OpTransactionValidator<Client, OpPooledTransaction>>,
|
||||
CoinbaseTipOrdering<OpPooledTransaction>,
|
||||
S,
|
||||
>;
|
||||
|
||||
@ -39,6 +39,13 @@ pub type OpTransactionPool<Client, S> = Pool<
|
||||
#[derive(Debug, Clone, derive_more::Deref)]
|
||||
pub struct OpPooledTransaction(EthPooledTransaction<OpTransactionSigned>);
|
||||
|
||||
impl OpPooledTransaction {
|
||||
/// Create new instance of [Self].
|
||||
pub fn new(transaction: RecoveredTx<OpTransactionSigned>, encoded_length: usize) -> Self {
|
||||
Self(EthPooledTransaction::new(transaction, encoded_length))
|
||||
}
|
||||
}
|
||||
|
||||
impl From<RecoveredTx<op_alloy_consensus::OpPooledTransaction>> for OpPooledTransaction {
|
||||
fn from(tx: RecoveredTx<op_alloy_consensus::OpPooledTransaction>) -> Self {
|
||||
let encoded_len = tx.encode_2718_len();
|
||||
@ -241,7 +248,7 @@ impl<Client, Tx> OpTransactionValidator<Client, Tx> {
|
||||
impl<Client, Tx> OpTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt,
|
||||
Tx: EthPoolTransaction<Consensus = TransactionSigned>,
|
||||
Tx: EthPoolTransaction<Consensus = OpTransactionSigned>,
|
||||
{
|
||||
/// Create a new [`OpTransactionValidator`].
|
||||
pub fn new(inner: EthTransactionValidator<Client, Tx>) -> Self {
|
||||
@ -373,8 +380,8 @@ where
|
||||
|
||||
impl<Client, Tx> TransactionValidator for OpTransactionValidator<Client, Tx>
|
||||
where
|
||||
Client: StateProviderFactory + BlockReaderIdExt<Block = reth_primitives::Block>,
|
||||
Tx: EthPoolTransaction<Consensus = TransactionSigned>,
|
||||
Client: StateProviderFactory + BlockReaderIdExt<Block = OpBlock>,
|
||||
Tx: EthPoolTransaction<Consensus = OpTransactionSigned>,
|
||||
{
|
||||
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 {
|
||||
|
||||
@ -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::<OpNode>()
|
||||
.with_types_and_provider::<OpNode, BlockchainProvider2<NodeTypesWithDBAdapter<OpNode, _>>>()
|
||||
.with_components(op_node.components())
|
||||
.with_add_ons(op_node.add_ons())
|
||||
.on_component_initialized(move |ctx| {
|
||||
|
||||
@ -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<Transaction = TransactionSigned>
|
||||
) -> impl PayloadTransactions<Transaction = OpTransactionSigned>
|
||||
where
|
||||
Pool: reth_transaction_pool::TransactionPool<
|
||||
Transaction: PoolTransaction<Consensus = TransactionSigned>,
|
||||
Transaction: PoolTransaction<Consensus = OpTransactionSigned>,
|
||||
>,
|
||||
{
|
||||
// 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());
|
||||
|
||||
@ -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<EvmConfig, Txs> OpPayloadBuilder<EvmConfig, Txs> {
|
||||
}
|
||||
impl<EvmConfig, T> OpPayloadBuilder<EvmConfig, T>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
|
||||
{
|
||||
/// 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<BuildOutcome<OpBuiltPayload>, PayloadBuilderError>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
|
||||
Txs: PayloadTransactions<Transaction = TransactionSigned>,
|
||||
Txs: PayloadTransactions<Transaction = OpTransactionSigned>,
|
||||
{
|
||||
let evm_env = self
|
||||
.cfg_and_block_env(&args.config.attributes, &args.config.parent_header)
|
||||
@ -217,8 +218,8 @@ where
|
||||
impl<Pool, Client, EvmConfig, Txs> PayloadBuilder<Pool, Client> for OpPayloadBuilder<EvmConfig, Txs>
|
||||
where
|
||||
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = EvmConfig::Transaction>>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
|
||||
Txs: OpPayloadTransactions,
|
||||
{
|
||||
type Attributes = OpPayloadBuilderAttributes;
|
||||
@ -293,7 +294,7 @@ impl<'a, Txs> OpBuilder<'a, Txs> {
|
||||
|
||||
impl<Txs> OpBuilder<'_, Txs>
|
||||
where
|
||||
Txs: PayloadTransactions<Transaction = TransactionSigned>,
|
||||
Txs: PayloadTransactions<Transaction = OpTransactionSigned>,
|
||||
{
|
||||
/// Executes the payload and returns the outcome.
|
||||
pub fn execute<EvmConfig, DB>(
|
||||
@ -302,7 +303,7 @@ where
|
||||
ctx: &OpPayloadBuilderCtx<EvmConfig>,
|
||||
) -> Result<BuildOutcomeKind<ExecutedPayload>, PayloadBuilderError>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
|
||||
DB: Database<Error = ProviderError>,
|
||||
{
|
||||
let Self { best } = self;
|
||||
@ -347,7 +348,7 @@ where
|
||||
ctx: OpPayloadBuilderCtx<EvmConfig>,
|
||||
) -> Result<BuildOutcomeKind<OpBuiltPayload>, PayloadBuilderError>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
|
||||
DB: Database<Error = ProviderError> + AsRef<P>,
|
||||
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<OpPrimitives> = ExecutedBlock {
|
||||
block: sealed_block.clone(),
|
||||
senders: Arc::new(info.executed_senders),
|
||||
execution_output: Arc::new(execution_outcome),
|
||||
@ -473,7 +474,7 @@ where
|
||||
ctx: &OpPayloadBuilderCtx<EvmConfig>,
|
||||
) -> Result<ExecutionWitness, PayloadBuilderError>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
|
||||
DB: Database<Error = ProviderError> + AsRef<P>,
|
||||
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<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = OpTransactionSigned>>,
|
||||
>(
|
||||
&self,
|
||||
pool: Pool,
|
||||
attr: BestTransactionsAttributes,
|
||||
) -> impl PayloadTransactions<Transaction = TransactionSigned>;
|
||||
) -> impl PayloadTransactions<Transaction = OpTransactionSigned>;
|
||||
}
|
||||
|
||||
impl OpPayloadTransactions for () {
|
||||
fn best_transactions<
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = OpTransactionSigned>>,
|
||||
>(
|
||||
&self,
|
||||
pool: Pool,
|
||||
attr: BestTransactionsAttributes,
|
||||
) -> impl PayloadTransactions<Transaction = TransactionSigned> {
|
||||
) -> impl PayloadTransactions<Transaction = OpTransactionSigned> {
|
||||
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<TransactionSigned>,
|
||||
pub executed_transactions: Vec<OpTransactionSigned>,
|
||||
/// The recovered senders for the executed transactions.
|
||||
pub executed_senders: Vec<Address>,
|
||||
/// The transaction receipts
|
||||
pub receipts: Vec<Option<Receipt>>,
|
||||
pub receipts: Vec<OpReceipt>,
|
||||
/// All gas used so far
|
||||
pub cumulative_gas_used: u64,
|
||||
/// Tracks fees from executed mempool transactions
|
||||
@ -708,7 +709,7 @@ impl<EvmConfig> OpPayloadBuilderCtx<EvmConfig> {
|
||||
|
||||
impl<EvmConfig> OpPayloadBuilderCtx<EvmConfig>
|
||||
where
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned>,
|
||||
{
|
||||
/// apply eip-4788 pre block contract call
|
||||
pub fn apply_pre_beacon_root_contract_call<DB>(
|
||||
@ -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(),
|
||||
};
|
||||
|
||||
// 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.
|
||||
// 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<DB>,
|
||||
mut best_txs: impl PayloadTransactions<Transaction = TransactionSigned>,
|
||||
mut best_txs: impl PayloadTransactions<Transaction = EvmConfig::Transaction>,
|
||||
) -> Result<Option<()>, PayloadBuilderError>
|
||||
where
|
||||
DB: Database<Error = ProviderError>,
|
||||
@ -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(),
|
||||
};
|
||||
|
||||
// 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
|
||||
|
||||
@ -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<WithEncoded<TransactionSigned>>,
|
||||
pub transactions: Vec<WithEncoded<OpTransactionSigned>>,
|
||||
/// The gas limit for the generated payload
|
||||
pub gas_limit: Option<u64>,
|
||||
/// 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<SealedBlock>,
|
||||
pub(crate) block: Arc<SealedBlockFor<OpBlock>>,
|
||||
/// Block execution data for the payload, if any.
|
||||
pub(crate) executed_block: Option<ExecutedBlock>,
|
||||
pub(crate) executed_block: Option<ExecutedBlock<OpPrimitives>>,
|
||||
/// 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<SealedBlock>,
|
||||
block: Arc<SealedBlockFor<OpBlock>>,
|
||||
fees: U256,
|
||||
chain_spec: Arc<OpChainSpec>,
|
||||
attributes: OpPayloadBuilderAttributes,
|
||||
executed_block: Option<ExecutedBlock>,
|
||||
executed_block: Option<ExecutedBlock<OpPrimitives>>,
|
||||
) -> 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<OpBlock> {
|
||||
&self.block
|
||||
}
|
||||
|
||||
@ -189,7 +188,7 @@ impl OpBuiltPayload {
|
||||
impl BuiltPayload for OpBuiltPayload {
|
||||
type Primitives = OpPrimitives;
|
||||
|
||||
fn block(&self) -> &SealedBlock {
|
||||
fn block(&self) -> &SealedBlockFor<OpBlock> {
|
||||
&self.block
|
||||
}
|
||||
|
||||
@ -197,7 +196,7 @@ impl BuiltPayload for OpBuiltPayload {
|
||||
self.fees
|
||||
}
|
||||
|
||||
fn executed_block(&self) -> Option<ExecutedBlock> {
|
||||
fn executed_block(&self) -> Option<ExecutedBlock<OpPrimitives>> {
|
||||
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<OpBlock> {
|
||||
(**self).block()
|
||||
}
|
||||
|
||||
@ -217,7 +216,7 @@ impl BuiltPayload for &OpBuiltPayload {
|
||||
(**self).fees()
|
||||
}
|
||||
|
||||
fn executed_block(&self) -> Option<ExecutedBlock> {
|
||||
fn executed_block(&self) -> Option<ExecutedBlock<OpPrimitives>> {
|
||||
self.executed_block.clone()
|
||||
}
|
||||
|
||||
|
||||
@ -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;
|
||||
/// Optimism-specific block type.
|
||||
pub type OpBlock = reth_primitives::Block<OpTransactionSigned>;
|
||||
|
||||
// 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 body type.
|
||||
pub type OpBlockBody = <OpBlock as Block>::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;
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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<ReceiptResponse = OpTransactionReceipt>,
|
||||
Provider: BlockReader<Receipt = Receipt, Transaction = TransactionSigned>,
|
||||
Provider: BlockReader<Receipt = OpReceipt, Transaction = OpTransactionSigned>,
|
||||
>,
|
||||
N: OpNodeCore<Provider: ChainSpecProvider<ChainSpec = OpChainSpec> + 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,
|
||||
|
||||
@ -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<ChainSpec: EthChainSpec + EthereumHardforks>
|
||||
+ StateProviderFactory,
|
||||
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = ProviderTx<N::Provider>>>,
|
||||
Evm: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
|
||||
Evm: ConfigureEvm<
|
||||
Header = ProviderHeader<Self::Provider>,
|
||||
Transaction = ProviderTx<Self::Provider>,
|
||||
>,
|
||||
>,
|
||||
{
|
||||
#[inline]
|
||||
@ -55,7 +61,13 @@ where
|
||||
/// Returns the locally built pending block
|
||||
async fn local_pending_block(
|
||||
&self,
|
||||
) -> Result<Option<(SealedBlockWithSenders, Vec<Receipt>)>, Self::Error> {
|
||||
) -> Result<
|
||||
Option<(
|
||||
SealedBlockWithSenders<ProviderBlock<Self::Provider>>,
|
||||
Vec<ProviderReceipt<Self::Provider>>,
|
||||
)>,
|
||||
Self::Error,
|
||||
> {
|
||||
// See: <https://github.com/ethereum-optimism/op-geth/blob/f2e69450c6eec9c35d56af91389a1c47737206ca/miner/worker.go#L367-L375>
|
||||
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::<u64>(),
|
||||
gas_limit: block_env.gas_limit.to::<u64>(),
|
||||
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::<u64>()
|
||||
}),
|
||||
@ -142,13 +154,22 @@ where
|
||||
result: ExecutionResult,
|
||||
cumulative_gas_used: u64,
|
||||
) -> reth_provider::ProviderReceipt<Self::Provider> {
|
||||
#[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,
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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<N> LoadReceipt for OpEthApi<N>
|
||||
where
|
||||
Self: Send + Sync,
|
||||
N: FullNodeComponents<Types: NodeTypes<ChainSpec = OpChainSpec>>,
|
||||
Self::Provider:
|
||||
TransactionsProvider<Transaction = TransactionSigned> + ReceiptProvider<Receipt = Receipt>,
|
||||
Self::Provider: TransactionsProvider<Transaction = OpTransactionSigned>
|
||||
+ ReceiptProvider<Receipt = OpReceipt>,
|
||||
{
|
||||
async fn build_transaction_receipt(
|
||||
&self,
|
||||
tx: TransactionSigned,
|
||||
tx: OpTransactionSigned,
|
||||
meta: TransactionMeta,
|
||||
receipt: Receipt,
|
||||
receipt: OpReceipt,
|
||||
) -> Result<RpcReceipt<Self::NetworkTypes>, 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<Self, OpEthApiError> {
|
||||
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<Self, OpEthApiError> {
|
||||
let timestamp = meta.timestamp;
|
||||
let core_receipt =
|
||||
build_receipt(transaction, meta, receipt, all_receipts, |receipt_with_bloom| {
|
||||
match receipt.tx_type {
|
||||
TxType::Legacy => OpReceiptEnvelope::<Log>::Legacy(receipt_with_bloom),
|
||||
TxType::Eip2930 => OpReceiptEnvelope::<Log>::Eip2930(receipt_with_bloom),
|
||||
TxType::Eip1559 => OpReceiptEnvelope::<Log>::Eip1559(receipt_with_bloom),
|
||||
TxType::Eip4844 => {
|
||||
// TODO: unreachable
|
||||
OpReceiptEnvelope::<Log>::Eip1559(receipt_with_bloom)
|
||||
}
|
||||
TxType::Eip7702 => OpReceiptEnvelope::<Log>::Eip7702(receipt_with_bloom),
|
||||
TxType::Deposit => {
|
||||
match receipt {
|
||||
OpReceipt::Legacy(_) => OpReceiptEnvelope::<Log>::Legacy(receipt_with_bloom),
|
||||
OpReceipt::Eip2930(_) => OpReceiptEnvelope::<Log>::Eip2930(receipt_with_bloom),
|
||||
OpReceipt::Eip1559(_) => OpReceiptEnvelope::<Log>::Eip1559(receipt_with_bloom),
|
||||
OpReceipt::Eip7702(_) => OpReceiptEnvelope::<Log>::Eip7702(receipt_with_bloom),
|
||||
OpReceipt::Deposit(receipt) => {
|
||||
OpReceiptEnvelope::<Log>::Deposit(OpDepositReceiptWithBloom::<Log> {
|
||||
receipt: OpDepositReceipt::<Log> {
|
||||
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,12 +284,13 @@ 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())
|
||||
let tx_1 =
|
||||
OpTransactionSigned::decode_2718(&mut TX_1_OP_MAINNET_BLOCK_124665056.as_slice())
|
||||
.unwrap();
|
||||
|
||||
let block = Block {
|
||||
@ -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)
|
||||
|
||||
@ -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<N> TransactionCompat for OpEthApi<N>
|
||||
impl<N> TransactionCompat<OpTransactionSigned> for OpEthApi<N>
|
||||
where
|
||||
N: FullNodeComponents<Provider: ReceiptProvider<Receipt = reth_primitives::Receipt>>,
|
||||
N: FullNodeComponents<Provider: ReceiptProvider<Receipt = OpReceipt>>,
|
||||
{
|
||||
type Transaction = Transaction;
|
||||
type Error = OpEthApiError;
|
||||
|
||||
fn fill(
|
||||
&self,
|
||||
tx: RecoveredTx,
|
||||
tx: RecoveredTx<OpTransactionSigned>,
|
||||
tx_info: TransactionInfo,
|
||||
) -> Result<Self::Transaction, Self::Error> {
|
||||
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| {
|
||||
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<TransactionSigned, Self::Error> {
|
||||
) -> Result<OpTransactionSigned, Self::Error> {
|
||||
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) {
|
||||
|
||||
@ -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<ChainSpec = OpChainSpec>
|
||||
+ Clone
|
||||
+ 'static,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned> + 'static,
|
||||
EvmConfig: ConfigureEvm<Header = Header, Transaction = OpTransactionSigned> + 'static,
|
||||
{
|
||||
async fn execute_payload(
|
||||
&self,
|
||||
|
||||
@ -23,9 +23,15 @@ pub trait PayloadTransactions {
|
||||
}
|
||||
|
||||
/// [`PayloadTransactions`] implementation that produces nothing.
|
||||
#[derive(Debug, Default, Clone, Copy)]
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub struct NoopPayloadTransactions<T>(core::marker::PhantomData<T>);
|
||||
|
||||
impl<T> Default for NoopPayloadTransactions<T> {
|
||||
fn default() -> Self {
|
||||
Self(Default::default())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> PayloadTransactions for NoopPayloadTransactions<T> {
|
||||
type Transaction = T;
|
||||
|
||||
|
||||
@ -207,7 +207,9 @@ pub fn block_to_payload_v3<T: Encodable2718>(
|
||||
}
|
||||
|
||||
/// Converts [`SealedBlock`] to [`ExecutionPayloadFieldV2`]
|
||||
pub fn convert_block_to_payload_field_v2(value: SealedBlock) -> ExecutionPayloadFieldV2 {
|
||||
pub fn convert_block_to_payload_field_v2<T: Encodable2718>(
|
||||
value: SealedBlock<Header, BlockBody<T>>,
|
||||
) -> ExecutionPayloadFieldV2 {
|
||||
// if there are withdrawals, return V2
|
||||
if value.body.withdrawals.is_some() {
|
||||
ExecutionPayloadFieldV2::V2(block_to_payload_v2(value))
|
||||
|
||||
@ -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"]
|
||||
|
||||
@ -112,6 +112,7 @@ arbitrary = [
|
||||
"alloy-consensus/arbitrary",
|
||||
]
|
||||
optimism = ["reth-primitives/optimism", "reth-db-api/optimism"]
|
||||
op = ["reth-db-api/op"]
|
||||
disable-lock = []
|
||||
|
||||
[[bench]]
|
||||
|
||||
@ -706,9 +706,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<N: NodeTypesWithDB<Primitives = EthPrimitives>> CanonStateSubscriptions
|
||||
for BlockchainProvider2<N>
|
||||
{
|
||||
impl<N: ProviderNodeTypes> CanonStateSubscriptions for BlockchainProvider2<N> {
|
||||
fn subscribe_to_canonical_state(&self) -> CanonStateNotifications<Self::Primitives> {
|
||||
self.canonical_in_memory_state.subscribe_canon_state()
|
||||
}
|
||||
|
||||
@ -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<Primitives: FullNodePrimitives>: Send + Sync {
|
||||
Types: NodeTypesForProvider<Primitives = Primitives>;
|
||||
}
|
||||
|
||||
impl ChainStorage<EthPrimitives> for EthStorage {
|
||||
fn reader<TX, Types>(
|
||||
&self,
|
||||
) -> impl ChainStorageReader<DatabaseProvider<TX, Types>, EthPrimitives>
|
||||
impl<N, T> ChainStorage<N> for EthStorage<T>
|
||||
where
|
||||
T: FullSignedTx,
|
||||
N: FullNodePrimitives<
|
||||
Block = reth_primitives::Block<T>,
|
||||
BlockBody = reth_primitives::BlockBody<T>,
|
||||
SignedTx = T,
|
||||
>,
|
||||
{
|
||||
fn reader<TX, Types>(&self) -> impl ChainStorageReader<DatabaseProvider<TX, Types>, N>
|
||||
where
|
||||
TX: DbTx + 'static,
|
||||
Types: NodeTypesForProvider<Primitives = EthPrimitives>,
|
||||
Types: NodeTypesForProvider<Primitives = N>,
|
||||
{
|
||||
self
|
||||
}
|
||||
|
||||
fn writer<TX, Types>(
|
||||
&self,
|
||||
) -> impl ChainStorageWriter<DatabaseProvider<TX, Types>, EthPrimitives>
|
||||
fn writer<TX, Types>(&self) -> impl ChainStorageWriter<DatabaseProvider<TX, Types>, N>
|
||||
where
|
||||
TX: DbTxMut + DbTx + 'static,
|
||||
Types: NodeTypesForProvider<Primitives = EthPrimitives>,
|
||||
Types: NodeTypesForProvider<Primitives = N>,
|
||||
{
|
||||
self
|
||||
}
|
||||
|
||||
@ -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<MockNodeTypesWithDB> {
|
||||
pub fn create_test_provider_factory_with_chain_spec(
|
||||
chain_spec: Arc<ChainSpec>,
|
||||
) -> ProviderFactory<MockNodeTypesWithDB> {
|
||||
create_test_provider_factory_with_node_types::<MockNodeTypes>(chain_spec)
|
||||
}
|
||||
|
||||
/// Creates test provider factory with provided chain spec.
|
||||
pub fn create_test_provider_factory_with_node_types<N: NodeTypes>(
|
||||
chain_spec: Arc<N::ChainSpec>,
|
||||
) -> ProviderFactory<NodeTypesWithDBAdapter<N, Arc<TempDatabase<DatabaseEnv>>>> {
|
||||
let (static_dir, _) = create_test_static_files_dir();
|
||||
let db = create_test_rw_db();
|
||||
ProviderFactory::new(
|
||||
|
||||
@ -978,7 +978,7 @@ pub trait PoolTransaction:
|
||||
type TryFromConsensusError: fmt::Display;
|
||||
|
||||
/// Associated type representing the raw consensus variant of the transaction.
|
||||
type Consensus: From<Self::Pooled>;
|
||||
type Consensus: SignedTransaction + From<Self::Pooled>;
|
||||
|
||||
/// Associated type representing the recovered pooled variant of the transaction.
|
||||
type Pooled: SignedTransaction;
|
||||
|
||||
Reference in New Issue
Block a user