refactor: BlockExecutionStrategy API (#14480)

This commit is contained in:
Arsenii Kulikov
2025-02-19 17:10:50 +04:00
committed by GitHub
parent 1ae733a7d1
commit 4f454c91d6
30 changed files with 423 additions and 555 deletions

View File

@ -7,7 +7,7 @@ use crate::{
};
use alloy_consensus::BlockHeader;
use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult};
use alloy_primitives::{Address, Bytes, B256, U256};
use alloy_primitives::{Bytes, B256, U256};
use alloy_rpc_types_eth::{
simulate::{SimBlock, SimulatePayload, SimulatedBlock},
state::{EvmOverrides, StateOverride},
@ -22,6 +22,7 @@ use reth_evm::{
TransactionEnv,
};
use reth_node_api::BlockBody;
use reth_primitives::Recovered;
use reth_primitives_traits::SignedTransaction;
use reth_provider::{BlockIdReader, ChainSpecProvider, ProviderHeader};
use reth_revm::{database::StateProviderDatabase, db::CacheDB, DatabaseRef};
@ -172,8 +173,6 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
let mut results = Vec::with_capacity(calls.len());
while let Some(call) = calls.next() {
let sender = call.from.unwrap_or_default();
// Resolve transaction, populate missing fields and enforce calls
// correctness.
let tx = simulate::resolve_transaction(
@ -185,7 +184,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
this.tx_resp_builder(),
)?;
let tx_env = this.evm_config().tx_env(&tx, sender);
let tx_env = this.evm_config().tx_env(&tx);
let (res, (_, tx_env)) = {
if trace_transfers {
@ -324,9 +323,9 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
if replay_block_txs {
// only need to replay the transactions in the block if not all transactions are
// to be replayed
let transactions = block.transactions_with_sender().take(num_txs);
for (signer, tx) in transactions {
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx, *signer);
let transactions = block.transactions_recovered().take(num_txs);
for tx in transactions {
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
let (res, _) = this.transact(&mut db, evm_env.clone(), tx_env)?;
db.commit(res.state);
}
@ -671,12 +670,12 @@ pub trait Call:
let this = self.clone();
self.spawn_with_state_at_block(parent_block.into(), move |state| {
let mut db = CacheDB::new(StateProviderDatabase::new(state));
let block_txs = block.transactions_with_sender();
let block_txs = block.transactions_recovered();
// replay all transactions prior to the targeted transaction
this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx.tx(), tx.signer());
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
let (res, _) = this.transact(&mut db, evm_env, tx_env)?;
f(tx_info, res, db)
@ -702,18 +701,18 @@ pub trait Call:
) -> Result<usize, Self::Error>
where
DB: Database<Error = ProviderError> + DatabaseCommit,
I: IntoIterator<Item = (&'a Address, &'a <Self::Evm as ConfigureEvmEnv>::Transaction)>,
I: IntoIterator<Item = Recovered<&'a <Self::Evm as ConfigureEvmEnv>::Transaction>>,
<Self::Evm as ConfigureEvmEnv>::Transaction: SignedTransaction,
{
let mut evm = self.evm_config().evm_with_env(db, evm_env);
let mut index = 0;
for (sender, tx) in transactions {
for tx in transactions {
if *tx.tx_hash() == target_tx_hash {
// reached the target transaction
break
}
let tx_env = self.evm_config().tx_env(tx, *sender);
let tx_env = self.evm_config().tx_env(tx);
evm.transact_commit(tx_env).map_err(Self::Error::from_evm_err)?;
index += 1;
}

View File

@ -3,7 +3,7 @@
use super::SpawnBlocking;
use crate::{types::RpcTypes, EthApiTypes, FromEthApiError, FromEvmError, RpcNodeCore};
use alloy_consensus::{BlockHeader, Transaction};
use alloy_consensus::{transaction::Recovered, BlockHeader, Transaction};
use alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK;
use alloy_primitives::B256;
use alloy_rpc_types_eth::BlockNumberOrTag;
@ -209,7 +209,7 @@ pub trait LoadPendingBlock:
block_env: &BlockEnv,
parent_hash: B256,
state_root: B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> ProviderBlock<Self::Provider>;
@ -219,7 +219,7 @@ pub trait LoadPendingBlock:
block_env: &BlockEnv,
parent_hash: B256,
state_root: B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
results: Vec<ExecutionResult<HaltReasonFor<Self::Evm>>>,
) -> (ProviderBlock<Self::Provider>, Vec<ProviderReceipt<Self::Provider>>) {
let mut cumulative_gas_used = 0;
@ -267,7 +267,6 @@ pub trait LoadPendingBlock:
let base_fee = evm_env.block_env.basefee;
let mut executed_txs = Vec::new();
let mut senders = Vec::new();
let mut best_txs =
self.pool().best_transactions_with_attributes(BestTransactionsAttributes::new(
base_fee,
@ -336,7 +335,7 @@ pub trait LoadPendingBlock:
}
}
let tx_env = self.evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = self.evm_config().tx_env(&tx);
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
Ok(res) => res,
@ -377,9 +376,7 @@ pub trait LoadPendingBlock:
cumulative_gas_used += gas_used;
// append transaction to the list of executed transactions
let (tx, sender) = tx.into_parts();
executed_txs.push(tx);
senders.push(sender);
results.push(result);
}
@ -405,6 +402,8 @@ pub trait LoadPendingBlock:
// calculate the state root
let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?;
let senders = executed_txs.iter().map(|tx| tx.signer()).collect();
let (block, receipts) = self.assemble_block_and_receipts(
&evm_env.block_env,
parent_hash,

View File

@ -204,14 +204,14 @@ pub trait Trace:
let this = self.clone();
self.spawn_with_state_at_block(parent_block.into(), move |state| {
let mut db = CacheDB::new(StateProviderDatabase::new(state));
let block_txs = block.transactions_with_sender();
let block_txs = block.transactions_recovered();
this.apply_pre_execution_changes(&block, &mut db, &evm_env)?;
// replay all transactions prior to the targeted transaction
this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
let tx_env = this.evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = this.evm_config().tx_env(tx);
let (res, _) = this.inspect(
StateCacheDbRefMutWrapper(&mut db),
evm_env,
@ -339,10 +339,10 @@ pub trait Trace:
let mut results = Vec::with_capacity(max_transactions);
let mut transactions = block
.transactions_with_sender()
.transactions_recovered()
.take(max_transactions)
.enumerate()
.map(|(idx, (signer, tx))| {
.map(|(idx, tx)| {
let tx_info = TransactionInfo {
hash: Some(*tx.tx_hash()),
index: Some(idx as u64),
@ -350,7 +350,7 @@ pub trait Trace:
block_number: Some(block_number),
base_fee: Some(base_fee),
};
let tx_env = this.evm_config().tx_env(tx, *signer);
let tx_env = this.evm_config().tx_env(tx);
(tx_info, tx_env)
})
.peekable();

View File

@ -7,7 +7,7 @@ use alloy_rpc_types_eth::{
Block, BlockTransactionsKind, Header,
};
use jsonrpsee_types::ErrorObject;
use reth_primitives::RecoveredBlock;
use reth_primitives::{Recovered, RecoveredBlock};
use reth_primitives_traits::{block::BlockTx, BlockBody as _, SignedTransaction};
use reth_rpc_server_types::result::rpc_err;
use reth_rpc_types_compat::{block::from_block, TransactionCompat};
@ -60,7 +60,7 @@ pub fn resolve_transaction<DB: Database, Tx, T: TransactionCompat<Tx>>(
chain_id: u64,
db: &mut DB,
tx_resp_builder: &T,
) -> Result<Tx, EthApiError>
) -> Result<Recovered<Tx>, EthApiError>
where
EthApiError: From<DB::Error>,
{
@ -108,7 +108,11 @@ where
}
}
tx_resp_builder.build_simulate_v1_transaction(tx).map_err(|e| EthApiError::other(e.into()))
let tx = tx_resp_builder
.build_simulate_v1_transaction(tx)
.map_err(|e| EthApiError::other(e.into()))?;
Ok(Recovered::new_unchecked(tx, from))
}
/// Handles outputs of the calls execution and builds a [`SimulatedBlock`].

View File

@ -109,12 +109,12 @@ where
this.eth_api().apply_pre_execution_changes(&block, &mut db, &evm_env)?;
let mut transactions = block.transactions_with_sender().enumerate().peekable();
let mut transactions = block.transactions_recovered().enumerate().peekable();
let mut inspector = None;
while let Some((index, (signer, tx))) = transactions.next() {
while let Some((index, tx)) = transactions.next() {
let tx_hash = *tx.tx_hash();
let tx_env = this.eth_api().evm_config().tx_env(tx, *signer);
let tx_env = this.eth_api().evm_config().tx_env(tx);
let (result, state_changes) = this.trace_transaction(
&opts,
@ -229,7 +229,7 @@ where
let this = self.clone();
self.eth_api()
.spawn_with_state_at_block(state_at, move |state| {
let block_txs = block.transactions_with_sender();
let block_txs = block.transactions_recovered();
// configure env for the target transaction
let tx = transaction.into_recovered();
@ -246,7 +246,7 @@ where
*tx.tx_hash(),
)?;
let tx_env = this.eth_api().evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = this.eth_api().evm_config().tx_env(&tx);
this.trace_transaction(
&opts,
@ -530,11 +530,11 @@ where
if replay_block_txs {
// only need to replay the transactions in the block if not all transactions are
// to be replayed
let transactions = block.transactions_with_sender().take(num_txs);
let transactions = block.transactions_recovered().take(num_txs);
// Execute all transactions until index
for (signer, tx) in transactions {
let tx_env = this.eth_api().evm_config().tx_env(tx, *signer);
for tx in transactions {
let tx_env = this.eth_api().evm_config().tx_env(tx);
let (res, _) = this.eth_api().transact(&mut db, evm_env.clone(), tx_env)?;
db.commit(res.state);
}

View File

@ -176,7 +176,7 @@ where
hasher.update(*tx.tx_hash());
let gas_price = tx.effective_gas_price(basefee);
let ResultAndState { result, state } = evm
.transact(eth_api.evm_config().tx_env(&tx, signer))
.transact(eth_api.evm_config().tx_env(&tx))
.map_err(Eth::Error::from_evm_err)?;
let gas_used = result.gas_used();

View File

@ -1,6 +1,9 @@
//! Support for building a pending block with transactions from local view of mempool.
use alloy_consensus::{constants::EMPTY_WITHDRAWALS, Header, Transaction, EMPTY_OMMER_ROOT_HASH};
use alloy_consensus::{
constants::EMPTY_WITHDRAWALS, transaction::Recovered, Header, Transaction,
EMPTY_OMMER_ROOT_HASH,
};
use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE};
use alloy_primitives::U256;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
@ -61,7 +64,7 @@ where
block_env: &BlockEnv,
parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> reth_provider::ProviderBlock<Self::Provider> {
let chain_spec = self.provider().chain_spec();
@ -105,7 +108,11 @@ where
// seal the block
reth_primitives::Block {
header,
body: BlockBody { transactions, ommers: vec![], withdrawals: None },
body: BlockBody {
transactions: transactions.into_iter().map(|tx| tx.into_tx()).collect(),
ommers: vec![],
withdrawals: None,
},
}
}

View File

@ -10,6 +10,7 @@ use alloy_rpc_types_mev::{
};
use jsonrpsee::core::RpcResult;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, Evm};
use reth_primitives::Recovered;
use reth_provider::ProviderTx;
use reth_revm::{database::StateProviderDatabase, db::CacheDB};
use reth_rpc_api::MevSimApiServer;
@ -21,9 +22,8 @@ use reth_rpc_eth_types::{
revm_utils::apply_block_overrides, utils::recover_raw_transaction, EthApiError,
};
use reth_tasks::pool::BlockingTaskGuard;
use reth_transaction_pool::{PoolConsensusTx, PoolPooledTx, PoolTransaction, TransactionPool};
use reth_transaction_pool::{PoolPooledTx, PoolTransaction, TransactionPool};
use revm::{context_interface::result::ResultAndState, DatabaseCommit, DatabaseRef};
use revm_primitives::Address;
use std::{sync::Arc, time::Duration};
use tracing::info;
@ -46,9 +46,7 @@ const SBUNDLE_PAYOUT_MAX_COST: u64 = 30_000;
#[derive(Clone, Debug)]
pub struct FlattenedBundleItem<T> {
/// The signed transaction
pub tx: T,
/// The address that signed the transaction
pub signer: Address,
pub tx: Recovered<T>,
/// Whether the transaction is allowed to revert
pub can_revert: bool,
/// Item-level inclusion constraints
@ -169,10 +167,10 @@ where
while idx < body.len() {
match &body[idx] {
BundleItem::Tx { tx, can_revert } => {
let recovered_tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?;
let (tx, signer) = recovered_tx.into_parts();
let tx: PoolConsensusTx<Eth::Pool> =
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus(tx);
let tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?;
let tx = tx.map_transaction(
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus,
);
let refund_percent =
validity.as_ref().and_then(|v| v.refund.as_ref()).and_then(|refunds| {
@ -186,7 +184,6 @@ where
// Create FlattenedBundleItem with current inclusion, validity, and privacy
let flattened_item = FlattenedBundleItem {
tx,
signer,
can_revert: *can_revert,
inclusion: inclusion.clone(),
validity: validity.clone(),
@ -282,7 +279,7 @@ where
}
let ResultAndState { result, state } = evm
.transact(eth_api.evm_config().tx_env(&item.tx, item.signer))
.transact(eth_api.evm_config().tx_env(&item.tx))
.map_err(Eth::Error::from_evm_err)?;
if !result.is_success() && !item.can_revert {
@ -330,7 +327,7 @@ where
if let Some(refund_percent) = item.refund_percent {
// Get refund configurations
let refund_configs = item.refund_configs.clone().unwrap_or_else(|| {
vec![RefundConfig { address: item.signer, percent: 100 }]
vec![RefundConfig { address: item.tx.signer(), percent: 100 }]
});
// Calculate payout transaction fee

View File

@ -113,7 +113,7 @@ where
.map_transaction(<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus);
let (evm_env, at) = self.eth_api().evm_env_at(block_id.unwrap_or_default()).await?;
let tx_env = self.eth_api().evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = self.eth_api().evm_config().tx_env(tx);
let config = TracingInspectorConfig::from_parity_config(&trace_types);