mirror of
https://github.com/paradigmxyz/reth.git
synced 2026-01-09 15:28:01 -05:00
feat: make more EVM and RPC conversions fallible (#18685)
This commit is contained in:
@@ -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() {
|
||||
|
||||
@@ -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()?;
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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())?;
|
||||
|
||||
|
||||
@@ -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))
|
||||
}
|
||||
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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()))
|
||||
}
|
||||
|
||||
@@ -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 =
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user