feat: make more EVM and RPC conversions fallible (#18685)

This commit is contained in:
Mablr
2025-09-26 15:56:46 +02:00
committed by GitHub
parent 057c71281f
commit 6a50aa3ea5
16 changed files with 145 additions and 78 deletions

View File

@@ -224,14 +224,14 @@ where
pub fn evm_env_for<T: PayloadTypes<BuiltPayload: BuiltPayload<Primitives = N>>>(
&self,
input: &BlockOrPayload<T>,
) -> EvmEnvFor<Evm>
) -> Result<EvmEnvFor<Evm>, Evm::Error>
where
V: PayloadValidator<T, Block = N::Block>,
Evm: ConfigureEngineEvm<T::ExecutionData, Primitives = N>,
{
match input {
BlockOrPayload::Payload(payload) => self.evm_config.evm_env_for_payload(payload),
BlockOrPayload::Block(block) => self.evm_config.evm_env(block.header()),
BlockOrPayload::Payload(payload) => Ok(self.evm_config.evm_env_for_payload(payload)),
BlockOrPayload::Block(block) => Ok(self.evm_config.evm_env(block.header())?),
}
}
@@ -259,14 +259,14 @@ where
pub fn execution_ctx_for<'a, T: PayloadTypes<BuiltPayload: BuiltPayload<Primitives = N>>>(
&self,
input: &'a BlockOrPayload<T>,
) -> ExecutionCtxFor<'a, Evm>
) -> Result<ExecutionCtxFor<'a, Evm>, Evm::Error>
where
V: PayloadValidator<T, Block = N::Block>,
Evm: ConfigureEngineEvm<T::ExecutionData, Primitives = N>,
{
match input {
BlockOrPayload::Payload(payload) => self.evm_config.context_for_payload(payload),
BlockOrPayload::Block(block) => self.evm_config.context_for_block(block),
BlockOrPayload::Payload(payload) => Ok(self.evm_config.context_for_payload(payload)),
BlockOrPayload::Block(block) => Ok(self.evm_config.context_for_block(block)?),
}
}
@@ -370,7 +370,7 @@ where
.into())
};
let evm_env = self.evm_env_for(&input);
let evm_env = self.evm_env_for(&input).map_err(NewPayloadError::other)?;
let env = ExecutionEnv { evm_env, hash: input.hash(), parent_hash: input.parent_hash() };
@@ -740,7 +740,8 @@ where
.build();
let evm = self.evm_config.evm_with_env(&mut db, env.evm_env.clone());
let ctx = self.execution_ctx_for(input);
let ctx =
self.execution_ctx_for(input).map_err(|e| InsertBlockErrorKind::Other(Box::new(e)))?;
let mut executor = self.evm_config.create_executor(evm, ctx);
if !self.config.precompile_cache_disabled() {

View File

@@ -285,8 +285,8 @@ where
.with_bundle_update()
.build();
let ctx = evm_config.context_for_block(&reorg_target);
let evm = evm_config.evm_for_block(&mut state, &reorg_target);
let ctx = evm_config.context_for_block(&reorg_target).map_err(RethError::other)?;
let evm = evm_config.evm_for_block(&mut state, &reorg_target).map_err(RethError::other)?;
let mut builder = evm_config.create_block_builder(evm, &reorg_target_parent, ctx);
builder.apply_pre_execution_changes()?;

View File

@@ -154,7 +154,7 @@ where
&self.block_assembler
}
fn evm_env(&self, header: &Header) -> EvmEnv {
fn evm_env(&self, header: &Header) -> Result<EvmEnv, Self::Error> {
let blob_params = self.chain_spec().blob_params_at_timestamp(header.timestamp);
let spec = config::revm_spec(self.chain_spec(), header);
@@ -189,7 +189,7 @@ where
blob_excess_gas_and_price,
};
EvmEnv { cfg_env, block_env }
Ok(EvmEnv { cfg_env, block_env })
}
fn next_evm_env(
@@ -265,26 +265,29 @@ where
Ok((cfg, block_env).into())
}
fn context_for_block<'a>(&self, block: &'a SealedBlock<Block>) -> EthBlockExecutionCtx<'a> {
EthBlockExecutionCtx {
fn context_for_block<'a>(
&self,
block: &'a SealedBlock<Block>,
) -> Result<EthBlockExecutionCtx<'a>, Self::Error> {
Ok(EthBlockExecutionCtx {
parent_hash: block.header().parent_hash,
parent_beacon_block_root: block.header().parent_beacon_block_root,
ommers: &block.body().ommers,
withdrawals: block.body().withdrawals.as_ref().map(Cow::Borrowed),
}
})
}
fn context_for_next_block(
&self,
parent: &SealedHeader,
attributes: Self::NextBlockEnvCtx,
) -> EthBlockExecutionCtx<'_> {
EthBlockExecutionCtx {
) -> Result<EthBlockExecutionCtx<'_>, Self::Error> {
Ok(EthBlockExecutionCtx {
parent_hash: parent.hash(),
parent_beacon_block_root: attributes.parent_beacon_block_root,
ommers: &[],
withdrawals: attributes.withdrawals.map(Cow::Owned),
}
})
}
}
@@ -401,7 +404,7 @@ mod tests {
// Use the `EthEvmConfig` to fill the `cfg_env` and `block_env` based on the ChainSpec,
// Header, and total difficulty
let EvmEnv { cfg_env, .. } =
EthEvmConfig::new(Arc::new(chain_spec.clone())).evm_env(&header);
EthEvmConfig::new(Arc::new(chain_spec.clone())).evm_env(&header).unwrap();
// Assert that the chain ID in the `cfg_env` is correctly set to the chain ID of the
// ChainSpec

View File

@@ -160,7 +160,7 @@ impl ConfigureEvm for MockEvmConfig {
self.inner.block_assembler()
}
fn evm_env(&self, header: &Header) -> EvmEnvFor<Self> {
fn evm_env(&self, header: &Header) -> Result<EvmEnvFor<Self>, Self::Error> {
self.inner.evm_env(header)
}
@@ -175,7 +175,7 @@ impl ConfigureEvm for MockEvmConfig {
fn context_for_block<'a>(
&self,
block: &'a SealedBlock<BlockTy<Self::Primitives>>,
) -> reth_evm::ExecutionCtxFor<'a, Self> {
) -> Result<reth_evm::ExecutionCtxFor<'a, Self>, Self::Error> {
self.inner.context_for_block(block)
}
@@ -183,7 +183,7 @@ impl ConfigureEvm for MockEvmConfig {
&self,
parent: &SealedHeader,
attributes: Self::NextBlockEnvCtx,
) -> reth_evm::ExecutionCtxFor<'_, Self> {
) -> Result<reth_evm::ExecutionCtxFor<'_, Self>, Self::Error> {
self.inner.context_for_next_block(parent, attributes)
}
}

View File

@@ -559,6 +559,7 @@ where
let result = self
.strategy_factory
.executor_for_block(&mut self.db, block)
.map_err(BlockExecutionError::other)?
.execute_block(block.transactions_recovered())?;
self.db.merge_transitions(BundleRetention::Reverts);
@@ -577,6 +578,7 @@ where
let result = self
.strategy_factory
.executor_for_block(&mut self.db, block)
.map_err(BlockExecutionError::other)?
.with_state_hook(Some(Box::new(state_hook)))
.execute_block(block.transactions_recovered())?;

View File

@@ -219,7 +219,7 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
fn block_assembler(&self) -> &Self::BlockAssembler;
/// Creates a new [`EvmEnv`] for the given header.
fn evm_env(&self, header: &HeaderTy<Self::Primitives>) -> EvmEnvFor<Self>;
fn evm_env(&self, header: &HeaderTy<Self::Primitives>) -> Result<EvmEnvFor<Self>, Self::Error>;
/// Returns the configured [`EvmEnv`] for `parent + 1` block.
///
@@ -246,7 +246,7 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
fn context_for_block<'a>(
&self,
block: &'a SealedBlock<BlockTy<Self::Primitives>>,
) -> ExecutionCtxFor<'a, Self>;
) -> Result<ExecutionCtxFor<'a, Self>, Self::Error>;
/// Returns the configured [`BlockExecutorFactory::ExecutionCtx`] for `parent + 1`
/// block.
@@ -254,7 +254,7 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
&self,
parent: &SealedHeader<HeaderTy<Self::Primitives>>,
attributes: Self::NextBlockEnvCtx,
) -> ExecutionCtxFor<'_, Self>;
) -> Result<ExecutionCtxFor<'_, Self>, Self::Error>;
/// Returns a [`TxEnv`] from a transaction and [`Address`].
fn tx_env(&self, transaction: impl IntoTxEnv<TxEnvFor<Self>>) -> TxEnvFor<Self> {
@@ -285,9 +285,9 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
&self,
db: DB,
header: &HeaderTy<Self::Primitives>,
) -> EvmFor<Self, DB> {
let evm_env = self.evm_env(header);
self.evm_with_env(db, evm_env)
) -> Result<EvmFor<Self, DB>, Self::Error> {
let evm_env = self.evm_env(header)?;
Ok(self.evm_with_env(db, evm_env))
}
/// Returns a new EVM with the given database configured with the given environment settings,
@@ -327,10 +327,10 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
&'a self,
db: &'a mut State<DB>,
block: &'a SealedBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> impl BlockExecutorFor<'a, Self::BlockExecutorFactory, DB> {
let evm = self.evm_for_block(db, block.header());
let ctx = self.context_for_block(block);
self.create_executor(evm, ctx)
) -> Result<impl BlockExecutorFor<'a, Self::BlockExecutorFactory, DB>, Self::Error> {
let evm = self.evm_for_block(db, block.header())?;
let ctx = self.context_for_block(block)?;
Ok(self.create_executor(evm, ctx))
}
/// Creates a [`BlockBuilder`]. Should be used when building a new block.
@@ -407,7 +407,7 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
) -> Result<impl BlockBuilder<Primitives = Self::Primitives>, Self::Error> {
let evm_env = self.next_evm_env(parent, &attributes)?;
let evm = self.evm_with_env(db, evm_env);
let ctx = self.context_for_next_block(parent, attributes);
let ctx = self.context_for_next_block(parent, attributes)?;
Ok(self.create_block_builder(evm, parent, ctx))
}

View File

@@ -43,7 +43,7 @@ where
self.inner().block_assembler()
}
fn evm_env(&self, header: &HeaderTy<Self::Primitives>) -> EvmEnvFor<Self> {
fn evm_env(&self, header: &HeaderTy<Self::Primitives>) -> Result<EvmEnvFor<Self>, Self::Error> {
self.inner().evm_env(header)
}
@@ -58,7 +58,7 @@ where
fn context_for_block<'a>(
&self,
block: &'a SealedBlock<BlockTy<Self::Primitives>>,
) -> crate::ExecutionCtxFor<'a, Self> {
) -> Result<crate::ExecutionCtxFor<'a, Self>, Self::Error> {
self.inner().context_for_block(block)
}
@@ -66,7 +66,7 @@ where
&self,
parent: &SealedHeader<HeaderTy<Self::Primitives>>,
attributes: Self::NextBlockEnvCtx,
) -> crate::ExecutionCtxFor<'_, Self> {
) -> Result<crate::ExecutionCtxFor<'_, Self>, Self::Error> {
self.inner().context_for_next_block(parent, attributes)
}
}

View File

@@ -151,7 +151,7 @@ where
&self.block_assembler
}
fn evm_env(&self, header: &Header) -> EvmEnv<OpSpecId> {
fn evm_env(&self, header: &Header) -> Result<EvmEnv<OpSpecId>, Self::Error> {
let spec = config::revm_spec(self.chain_spec(), header);
let cfg_env = CfgEnv::new().with_chain_id(self.chain_spec().chain().id()).with_spec(spec);
@@ -181,7 +181,7 @@ where
blob_excess_gas_and_price,
};
EvmEnv { cfg_env, block_env }
Ok(EvmEnv { cfg_env, block_env })
}
fn next_evm_env(
@@ -222,24 +222,27 @@ where
Ok(EvmEnv { cfg_env, block_env })
}
fn context_for_block(&self, block: &'_ SealedBlock<N::Block>) -> OpBlockExecutionCtx {
OpBlockExecutionCtx {
fn context_for_block(
&self,
block: &'_ SealedBlock<N::Block>,
) -> Result<OpBlockExecutionCtx, Self::Error> {
Ok(OpBlockExecutionCtx {
parent_hash: block.header().parent_hash(),
parent_beacon_block_root: block.header().parent_beacon_block_root(),
extra_data: block.header().extra_data().clone(),
}
})
}
fn context_for_next_block(
&self,
parent: &SealedHeader<N::BlockHeader>,
attributes: Self::NextBlockEnvCtx,
) -> OpBlockExecutionCtx {
OpBlockExecutionCtx {
) -> Result<OpBlockExecutionCtx, Self::Error> {
Ok(OpBlockExecutionCtx {
parent_hash: parent.hash(),
parent_beacon_block_root: attributes.parent_beacon_block_root,
extra_data: attributes.extra_data,
}
})
}
}
@@ -359,7 +362,8 @@ mod tests {
// Header, and total difficulty
let EvmEnv { cfg_env, .. } =
OpEvmConfig::optimism(Arc::new(OpChainSpec { inner: chain_spec.clone() }))
.evm_env(&header);
.evm_env(&header)
.unwrap();
// Assert that the chain ID in the `cfg_env` is correctly set to the chain ID of the
// ChainSpec

View File

@@ -70,8 +70,15 @@ pub trait ReceiptConverter<N: NodePrimitives>: Debug + 'static {
/// A type that knows how to convert a consensus header into an RPC header.
pub trait HeaderConverter<Consensus, Rpc>: Debug + Send + Sync + Unpin + Clone + 'static {
/// An associated RPC conversion error.
type Err: error::Error;
/// Converts a consensus header into an RPC header.
fn convert_header(&self, header: SealedHeader<Consensus>, block_size: usize) -> Rpc;
fn convert_header(
&self,
header: SealedHeader<Consensus>,
block_size: usize,
) -> Result<Rpc, Self::Err>;
}
/// Default implementation of [`HeaderConverter`] that uses [`FromConsensusHeader`] to convert
@@ -80,8 +87,14 @@ impl<Consensus, Rpc> HeaderConverter<Consensus, Rpc> for ()
where
Rpc: FromConsensusHeader<Consensus>,
{
fn convert_header(&self, header: SealedHeader<Consensus>, block_size: usize) -> Rpc {
Rpc::from_consensus_header(header, block_size)
type Err = Infallible;
fn convert_header(
&self,
header: SealedHeader<Consensus>,
block_size: usize,
) -> Result<Rpc, Self::Err> {
Ok(Rpc::from_consensus_header(header, block_size))
}
}
@@ -205,10 +218,12 @@ pub trait IntoRpcTx<T> {
/// An additional context, usually [`TransactionInfo`] in a wrapper that carries some
/// implementation specific extra information.
type TxInfo;
/// An associated RPC conversion error.
type Err: error::Error;
/// Performs the conversion consuming `self` with `signer` and `tx_info`. See [`IntoRpcTx`]
/// for details.
fn into_rpc_tx(self, signer: Address, tx_info: Self::TxInfo) -> T;
fn into_rpc_tx(self, signer: Address, tx_info: Self::TxInfo) -> Result<T, Self::Err>;
}
/// Converts `T` into `self`. It is reciprocal of [`IntoRpcTx`].
@@ -222,23 +237,30 @@ pub trait IntoRpcTx<T> {
/// Prefer using [`IntoRpcTx`] over using [`FromConsensusTx`] when specifying trait bounds on a
/// generic function. This way, types that directly implement [`IntoRpcTx`] can be used as arguments
/// as well.
pub trait FromConsensusTx<T> {
pub trait FromConsensusTx<T>: Sized {
/// An additional context, usually [`TransactionInfo`] in a wrapper that carries some
/// implementation specific extra information.
type TxInfo;
/// An associated RPC conversion error.
type Err: error::Error;
/// Performs the conversion consuming `tx` with `signer` and `tx_info`. See [`FromConsensusTx`]
/// for details.
fn from_consensus_tx(tx: T, signer: Address, tx_info: Self::TxInfo) -> Self;
fn from_consensus_tx(tx: T, signer: Address, tx_info: Self::TxInfo) -> Result<Self, Self::Err>;
}
impl<TxIn: alloy_consensus::Transaction, T: alloy_consensus::Transaction + From<TxIn>>
FromConsensusTx<TxIn> for Transaction<T>
{
type TxInfo = TransactionInfo;
type Err = Infallible;
fn from_consensus_tx(tx: TxIn, signer: Address, tx_info: Self::TxInfo) -> Self {
Self::from_transaction(Recovered::new_unchecked(tx.into(), signer), tx_info)
fn from_consensus_tx(
tx: TxIn,
signer: Address,
tx_info: Self::TxInfo,
) -> Result<Self, Self::Err> {
Ok(Self::from_transaction(Recovered::new_unchecked(tx.into(), signer), tx_info))
}
}
@@ -246,10 +268,12 @@ impl<ConsensusTx, RpcTx> IntoRpcTx<RpcTx> for ConsensusTx
where
ConsensusTx: alloy_consensus::Transaction,
RpcTx: FromConsensusTx<Self>,
<RpcTx as FromConsensusTx<ConsensusTx>>::Err: Debug,
{
type TxInfo = RpcTx::TxInfo;
type Err = <RpcTx as FromConsensusTx<ConsensusTx>>::Err;
fn into_rpc_tx(self, signer: Address, tx_info: Self::TxInfo) -> RpcTx {
fn into_rpc_tx(self, signer: Address, tx_info: Self::TxInfo) -> Result<RpcTx, Self::Err> {
RpcTx::from_consensus_tx(self, signer, tx_info)
}
}
@@ -285,7 +309,7 @@ impl<Tx, RpcTx> RpcTxConverter<Tx, RpcTx, Tx::TxInfo> for ()
where
Tx: IntoRpcTx<RpcTx>,
{
type Err = Infallible;
type Err = Tx::Err;
fn convert_rpc_tx(
&self,
@@ -293,7 +317,7 @@ where
signer: Address,
tx_info: Tx::TxInfo,
) -> Result<RpcTx, Self::Err> {
Ok(tx.into_rpc_tx(signer, tx_info))
tx.into_rpc_tx(signer, tx_info)
}
}
@@ -893,6 +917,7 @@ where
+ From<TxEnv::Error>
+ From<<Map as TxInfoMapper<TxTy<N>>>::Err>
+ From<RpcTx::Err>
+ From<Header::Err>
+ Error
+ Unpin
+ Sync
@@ -924,7 +949,7 @@ where
let (tx, signer) = tx.into_parts();
let tx_info = self.mapper.try_map(&tx, tx_info)?;
Ok(self.rpc_tx_converter.convert_rpc_tx(tx, signer, tx_info)?)
self.rpc_tx_converter.convert_rpc_tx(tx, signer, tx_info).map_err(Into::into)
}
fn build_simulate_v1_transaction(
@@ -966,7 +991,7 @@ where
header: SealedHeaderFor<Self::Primitives>,
block_size: usize,
) -> Result<RpcHeader<Self::Network>, Self::Error> {
Ok(self.header_converter.convert_header(header, block_size))
Ok(self.header_converter.convert_header(header, block_size)?)
}
}
@@ -1016,9 +1041,14 @@ pub mod op {
for op_alloy_rpc_types::Transaction<T>
{
type TxInfo = OpTransactionInfo;
type Err = Infallible;
fn from_consensus_tx(tx: T, signer: Address, tx_info: Self::TxInfo) -> Self {
Self::from_transaction(Recovered::new_unchecked(tx, signer), tx_info)
fn from_consensus_tx(
tx: T,
signer: Address,
tx_info: Self::TxInfo,
) -> Result<Self, Self::Err> {
Ok(Self::from_transaction(Recovered::new_unchecked(tx, signer), tx_info))
}
}

View File

@@ -160,7 +160,9 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
let ctx = this
.evm_config()
.context_for_next_block(&parent, this.next_env_attributes(&parent)?);
.context_for_next_block(&parent, this.next_env_attributes(&parent)?)
.map_err(RethError::other)
.map_err(Self::Error::from_eth_err)?;
let (result, results) = if trace_transfers {
// prepare inspector to capture transfer inside the evm so they are recorded
// and included in logs

View File

@@ -94,8 +94,9 @@ where
return Err(RethError::msg("cancun has not been activated"))
}
let current_precompiles =
evm_to_precompiles_map(self.evm_config.evm_for_block(EmptyDB::default(), &latest));
let current_precompiles = evm_to_precompiles_map(
self.evm_config.evm_for_block(EmptyDB::default(), &latest).map_err(RethError::other)?,
);
let mut fork_timestamps =
chain_spec.forks_iter().filter_map(|(_, cond)| cond.as_timestamp()).collect::<Vec<_>>();
@@ -124,7 +125,9 @@ where
header
};
let last_precompiles = evm_to_precompiles_map(
self.evm_config.evm_for_block(EmptyDB::default(), &fake_header),
self.evm_config
.evm_for_block(EmptyDB::default(), &fake_header)
.map_err(RethError::other)?,
);
config.last = self.build_fork_config_at(last_fork_timestamp, last_precompiles);
@@ -137,7 +140,9 @@ where
header
};
let next_precompiles = evm_to_precompiles_map(
self.evm_config.evm_for_block(EmptyDB::default(), &fake_header),
self.evm_config
.evm_for_block(EmptyDB::default(), &fake_header)
.map_err(RethError::other)?,
);
config.next = self.build_fork_config_at(next_fork_timestamp, next_precompiles);

View File

@@ -81,7 +81,11 @@ pub trait LoadPendingBlock:
// Note: for the PENDING block we assume it is past the known merge block and
// thus this will not fail when looking up the total
// difficulty value for the blockenv.
let evm_env = self.evm_config().evm_env(block.header());
let evm_env = self
.evm_config()
.evm_env(block.header())
.map_err(RethError::other)
.map_err(Self::Error::from_eth_err)?;
return Ok(PendingBlockEnv::new(
evm_env,

View File

@@ -281,7 +281,11 @@ pub trait LoadState:
let header =
self.cache().get_header(block_hash).await.map_err(Self::Error::from_eth_err)?;
let evm_env = self.evm_config().evm_env(&header);
let evm_env = self
.evm_config()
.evm_env(&header)
.map_err(RethError::other)
.map_err(Self::Error::from_eth_err)?;
Ok((evm_env, block_hash.into()))
}

View File

@@ -18,6 +18,7 @@ use alloy_rpc_types_trace::geth::{
use async_trait::async_trait;
use jsonrpsee::core::RpcResult;
use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks};
use reth_errors::RethError;
use reth_evm::{execute::Executor, ConfigureEvm, EvmEnvFor, TxEnvFor};
use reth_primitives_traits::{Block as _, BlockBody, ReceiptWithBloom, RecoveredBlock};
use reth_revm::{
@@ -151,7 +152,12 @@ where
.map_err(BlockError::RlpDecodeRawBlock)
.map_err(Eth::Error::from_eth_err)?;
let evm_env = self.eth_api().evm_config().evm_env(block.header());
let evm_env = self
.eth_api()
.evm_config()
.evm_env(block.header())
.map_err(RethError::other)
.map_err(Eth::Error::from_eth_err)?;
// Depending on EIP-2 we need to recover the transactions differently
let senders =

View File

@@ -135,7 +135,7 @@ impl ConfigureEvm for CustomEvmConfig {
self.inner.block_assembler()
}
fn evm_env(&self, header: &Header) -> EvmEnv<SpecId> {
fn evm_env(&self, header: &Header) -> Result<EvmEnv<SpecId>, Self::Error> {
self.inner.evm_env(header)
}
@@ -147,7 +147,10 @@ impl ConfigureEvm for CustomEvmConfig {
self.inner.next_evm_env(parent, attributes)
}
fn context_for_block<'a>(&self, block: &'a SealedBlock<Block>) -> EthBlockExecutionCtx<'a> {
fn context_for_block<'a>(
&self,
block: &'a SealedBlock<Block>,
) -> Result<EthBlockExecutionCtx<'a>, Self::Error> {
self.inner.context_for_block(block)
}
@@ -155,7 +158,7 @@ impl ConfigureEvm for CustomEvmConfig {
&self,
parent: &SealedHeader,
attributes: Self::NextBlockEnvCtx,
) -> EthBlockExecutionCtx<'_> {
) -> Result<EthBlockExecutionCtx<'_>, Self::Error> {
self.inner.context_for_next_block(parent, attributes)
}
}

View File

@@ -62,7 +62,7 @@ impl ConfigureEvm for CustomEvmConfig {
&self.block_assembler
}
fn evm_env(&self, header: &CustomHeader) -> EvmEnv<OpSpecId> {
fn evm_env(&self, header: &CustomHeader) -> Result<EvmEnv<OpSpecId>, Self::Error> {
self.inner.evm_env(header)
}
@@ -74,30 +74,33 @@ impl ConfigureEvm for CustomEvmConfig {
self.inner.next_evm_env(parent, &attributes.inner)
}
fn context_for_block(&self, block: &SealedBlock<Block>) -> CustomBlockExecutionCtx {
CustomBlockExecutionCtx {
fn context_for_block(
&self,
block: &SealedBlock<Block>,
) -> Result<CustomBlockExecutionCtx, Self::Error> {
Ok(CustomBlockExecutionCtx {
inner: OpBlockExecutionCtx {
parent_hash: block.header().parent_hash(),
parent_beacon_block_root: block.header().parent_beacon_block_root(),
extra_data: block.header().extra_data().clone(),
},
extension: block.extension,
}
})
}
fn context_for_next_block(
&self,
parent: &SealedHeader<CustomHeader>,
attributes: Self::NextBlockEnvCtx,
) -> CustomBlockExecutionCtx {
CustomBlockExecutionCtx {
) -> Result<CustomBlockExecutionCtx, Self::Error> {
Ok(CustomBlockExecutionCtx {
inner: OpBlockExecutionCtx {
parent_hash: parent.hash(),
parent_beacon_block_root: attributes.inner.parent_beacon_block_root,
extra_data: attributes.inner.extra_data,
},
extension: attributes.extension,
}
})
}
}