anvil/eth/backend/
executor.rs

1use crate::{
2    PrecompileFactory,
3    eth::{
4        backend::{
5            cheats::{CheatEcrecover, CheatsManager},
6            db::Db,
7            env::Env,
8            mem::op_haltreason_to_instruction_result,
9            validate::TransactionValidator,
10        },
11        error::InvalidTransactionError,
12        pool::transactions::PoolTransaction,
13    },
14    mem::inspector::AnvilInspector,
15};
16use alloy_consensus::{
17    Header, Receipt, ReceiptWithBloom, Transaction, constants::EMPTY_WITHDRAWALS,
18    proofs::calculate_receipt_root, transaction::Either,
19};
20use alloy_eips::{
21    eip7685::EMPTY_REQUESTS_HASH,
22    eip7702::{RecoveredAuthority, RecoveredAuthorization},
23    eip7840::BlobParams,
24};
25use alloy_evm::{
26    EthEvmFactory, Evm, EvmEnv, EvmFactory, FromRecoveredTx,
27    eth::EthEvmContext,
28    precompiles::{DynPrecompile, Precompile, PrecompilesMap},
29};
30use alloy_op_evm::OpEvmFactory;
31use alloy_primitives::{B256, Bloom, BloomInput, Log};
32use anvil_core::eth::{
33    block::{BlockInfo, create_block},
34    transaction::{PendingTransaction, TransactionInfo},
35};
36use foundry_evm::{
37    backend::DatabaseError,
38    core::{either_evm::EitherEvm, precompiles::EC_RECOVER},
39    traces::{CallTraceDecoder, CallTraceNode},
40};
41use foundry_evm_networks::NetworkConfigs;
42use foundry_primitives::{FoundryReceiptEnvelope, FoundryTxEnvelope};
43use op_revm::{OpContext, OpTransaction};
44use revm::{
45    Database, Inspector,
46    context::{Block as RevmBlock, Cfg, TxEnv},
47    context_interface::result::{EVMError, ExecutionResult, Output},
48    interpreter::InstructionResult,
49    primitives::hardfork::SpecId,
50};
51use std::{fmt::Debug, sync::Arc};
52
53/// Represents an executed transaction (transacted on the DB)
54#[derive(Debug)]
55pub struct ExecutedTransaction {
56    transaction: Arc<PoolTransaction>,
57    exit_reason: InstructionResult,
58    out: Option<Output>,
59    gas_used: u64,
60    logs: Vec<Log>,
61    traces: Vec<CallTraceNode>,
62    nonce: u64,
63}
64
65// == impl ExecutedTransaction ==
66
67impl ExecutedTransaction {
68    /// Creates the receipt for the transaction
69    fn create_receipt(&self, cumulative_gas_used: &mut u64) -> FoundryReceiptEnvelope {
70        let logs = self.logs.clone();
71        *cumulative_gas_used = cumulative_gas_used.saturating_add(self.gas_used);
72
73        // successful return see [Return]
74        let status_code = u8::from(self.exit_reason as u8 <= InstructionResult::SelfDestruct as u8);
75        let receipt_with_bloom: ReceiptWithBloom = Receipt {
76            status: (status_code == 1).into(),
77            cumulative_gas_used: *cumulative_gas_used,
78            logs,
79        }
80        .into();
81
82        match self.transaction.pending_transaction.transaction.as_ref() {
83            FoundryTxEnvelope::Legacy(_) => FoundryReceiptEnvelope::Legacy(receipt_with_bloom),
84            FoundryTxEnvelope::Eip2930(_) => FoundryReceiptEnvelope::Eip2930(receipt_with_bloom),
85            FoundryTxEnvelope::Eip1559(_) => FoundryReceiptEnvelope::Eip1559(receipt_with_bloom),
86            FoundryTxEnvelope::Eip4844(_) => FoundryReceiptEnvelope::Eip4844(receipt_with_bloom),
87            FoundryTxEnvelope::Eip7702(_) => FoundryReceiptEnvelope::Eip7702(receipt_with_bloom),
88            FoundryTxEnvelope::Deposit(_tx) => {
89                FoundryReceiptEnvelope::Deposit(op_alloy_consensus::OpDepositReceiptWithBloom {
90                    receipt: op_alloy_consensus::OpDepositReceipt {
91                        inner: receipt_with_bloom.receipt,
92                        deposit_nonce: Some(0),
93                        deposit_receipt_version: Some(1),
94                    },
95                    logs_bloom: receipt_with_bloom.logs_bloom,
96                })
97            }
98            // TODO(onbjerg): we should impl support for Tempo transactions
99            FoundryTxEnvelope::Tempo(_) => todo!(),
100        }
101    }
102}
103
104/// Represents the outcome of mining a new block
105#[derive(Clone, Debug)]
106pub struct ExecutedTransactions {
107    /// The block created after executing the `included` transactions
108    pub block: BlockInfo,
109    /// All transactions included in the block
110    pub included: Vec<Arc<PoolTransaction>>,
111    /// All transactions that were invalid at the point of their execution and were not included in
112    /// the block
113    pub invalid: Vec<Arc<PoolTransaction>>,
114}
115
116/// An executor for a series of transactions
117pub struct TransactionExecutor<'a, Db: ?Sized, V: TransactionValidator> {
118    /// where to insert the transactions
119    pub db: &'a mut Db,
120    /// type used to validate before inclusion
121    pub validator: &'a V,
122    /// all pending transactions
123    pub pending: std::vec::IntoIter<Arc<PoolTransaction>>,
124    pub evm_env: EvmEnv,
125    pub parent_hash: B256,
126    /// Cumulative gas used by all executed transactions
127    pub gas_used: u64,
128    /// Cumulative blob gas used by all executed transactions
129    pub blob_gas_used: u64,
130    pub enable_steps_tracing: bool,
131    pub networks: NetworkConfigs,
132    pub print_logs: bool,
133    pub print_traces: bool,
134    /// Recorder used for decoding traces, used together with print_traces
135    pub call_trace_decoder: Arc<CallTraceDecoder>,
136    /// Precompiles to inject to the EVM.
137    pub precompile_factory: Option<Arc<dyn PrecompileFactory>>,
138    pub blob_params: BlobParams,
139    pub cheats: CheatsManager,
140}
141
142impl<DB: Db + ?Sized, V: TransactionValidator> TransactionExecutor<'_, DB, V> {
143    /// Executes all transactions and puts them in a new block with the provided `timestamp`
144    pub fn execute(mut self) -> ExecutedTransactions {
145        let mut transactions = Vec::new();
146        let mut transaction_infos = Vec::new();
147        let mut receipts = Vec::new();
148        let mut bloom = Bloom::default();
149        let mut cumulative_gas_used = 0u64;
150        let mut invalid = Vec::new();
151        let mut included = Vec::new();
152        let gas_limit = self.evm_env.block_env().gas_limit;
153        let parent_hash = self.parent_hash;
154        let block_number = self.evm_env.block_env().number;
155        let difficulty = self.evm_env.block_env().difficulty;
156        let mix_hash = self.evm_env.block_env().prevrandao;
157        let beneficiary = self.evm_env.block_env().beneficiary;
158        let timestamp = self.evm_env.block_env().timestamp;
159        let base_fee = if self.evm_env.cfg_env().spec.is_enabled_in(SpecId::LONDON) {
160            Some(self.evm_env.block_env().basefee)
161        } else {
162            None
163        };
164
165        let is_shanghai = self.evm_env.cfg_env().spec >= SpecId::SHANGHAI;
166        let is_cancun = self.evm_env.cfg_env().spec >= SpecId::CANCUN;
167        let is_prague = self.evm_env.cfg_env().spec >= SpecId::PRAGUE;
168        let excess_blob_gas =
169            if is_cancun { self.evm_env.block_env().blob_excess_gas() } else { None };
170        let mut cumulative_blob_gas_used = if is_cancun { Some(0u64) } else { None };
171
172        for tx in self.into_iter() {
173            let tx = match tx {
174                TransactionExecutionOutcome::Executed(tx) => {
175                    included.push(tx.transaction.clone());
176                    tx
177                }
178                TransactionExecutionOutcome::BlockGasExhausted(tx) => {
179                    trace!(target: "backend",  tx_gas_limit = %tx.pending_transaction.transaction.gas_limit(), ?tx,  "block gas limit exhausting, skipping transaction");
180                    continue;
181                }
182                TransactionExecutionOutcome::BlobGasExhausted(tx) => {
183                    trace!(target: "backend",  blob_gas = %tx.pending_transaction.transaction.blob_gas_used().unwrap_or_default(), ?tx,  "block blob gas limit exhausting, skipping transaction");
184                    continue;
185                }
186                TransactionExecutionOutcome::TransactionGasExhausted(tx) => {
187                    trace!(target: "backend",  tx_gas_limit = %tx.pending_transaction.transaction.gas_limit(), ?tx,  "transaction gas limit exhausting, skipping transaction");
188                    continue;
189                }
190                TransactionExecutionOutcome::Invalid(tx, _) => {
191                    trace!(target: "backend", ?tx,  "skipping invalid transaction");
192                    invalid.push(tx);
193                    continue;
194                }
195                TransactionExecutionOutcome::DatabaseError(_, err) => {
196                    // Note: this is only possible in forking mode, if for example a rpc request
197                    // failed
198                    trace!(target: "backend", ?err,  "Failed to execute transaction due to database error");
199                    continue;
200                }
201            };
202            if is_cancun {
203                let tx_blob_gas =
204                    tx.transaction.pending_transaction.transaction.blob_gas_used().unwrap_or(0);
205                cumulative_blob_gas_used =
206                    Some(cumulative_blob_gas_used.unwrap_or(0u64).saturating_add(tx_blob_gas));
207            }
208            let receipt = tx.create_receipt(&mut cumulative_gas_used);
209
210            let ExecutedTransaction { transaction, logs, out, traces, exit_reason: exit, .. } = tx;
211            build_logs_bloom(&logs, &mut bloom);
212
213            let contract_address = out.as_ref().and_then(|out| {
214                if let Output::Create(_, contract_address) = out {
215                    trace!(target: "backend", "New contract deployed: at {:?}", contract_address);
216                    *contract_address
217                } else {
218                    None
219                }
220            });
221
222            let transaction_index = transaction_infos.len() as u64;
223            let info = TransactionInfo {
224                transaction_hash: transaction.hash(),
225                transaction_index,
226                from: *transaction.pending_transaction.sender(),
227                to: transaction.pending_transaction.transaction.to(),
228                contract_address,
229                traces,
230                exit,
231                out: out.map(Output::into_data),
232                nonce: tx.nonce,
233                gas_used: tx.gas_used,
234            };
235
236            transaction_infos.push(info);
237            receipts.push(receipt);
238            transactions.push(transaction.pending_transaction.transaction.clone());
239        }
240
241        let receipts_root = calculate_receipt_root(&receipts);
242
243        let header = Header {
244            parent_hash,
245            ommers_hash: Default::default(),
246            beneficiary,
247            state_root: self.db.maybe_state_root().unwrap_or_default(),
248            transactions_root: Default::default(), // Will be computed by create_block
249            receipts_root,
250            logs_bloom: bloom,
251            difficulty,
252            number: block_number.saturating_to(),
253            gas_limit,
254            gas_used: cumulative_gas_used,
255            timestamp: timestamp.saturating_to(),
256            extra_data: Default::default(),
257            mix_hash: mix_hash.unwrap_or_default(),
258            nonce: Default::default(),
259            base_fee_per_gas: base_fee,
260            parent_beacon_block_root: is_cancun.then_some(Default::default()),
261            blob_gas_used: cumulative_blob_gas_used,
262            excess_blob_gas,
263            withdrawals_root: is_shanghai.then_some(EMPTY_WITHDRAWALS),
264            requests_hash: is_prague.then_some(EMPTY_REQUESTS_HASH),
265        };
266
267        let block = create_block(header, transactions);
268        let block = BlockInfo { block, transactions: transaction_infos, receipts };
269        ExecutedTransactions { block, included, invalid }
270    }
271
272    fn env_for(&self, tx: &PendingTransaction) -> Env {
273        let mut tx_env: OpTransaction<TxEnv> =
274            FromRecoveredTx::from_recovered_tx(tx.transaction.as_ref(), *tx.sender());
275
276        if let FoundryTxEnvelope::Eip7702(tx_7702) = tx.transaction.as_ref()
277            && self.cheats.has_recover_overrides()
278        {
279            // Override invalid recovered authorizations with signature overrides from cheat manager
280            let cheated_auths = tx_7702
281                .tx()
282                .authorization_list
283                .iter()
284                .zip(tx_env.base.authorization_list)
285                .map(|(signed_auth, either_auth)| {
286                    either_auth.right_and_then(|recovered_auth| {
287                        if recovered_auth.authority().is_none()
288                            && let Ok(signature) = signed_auth.signature()
289                            && let Some(override_addr) =
290                                self.cheats.get_recover_override(&signature.as_bytes().into())
291                        {
292                            Either::Right(RecoveredAuthorization::new_unchecked(
293                                recovered_auth.into_parts().0,
294                                RecoveredAuthority::Valid(override_addr),
295                            ))
296                        } else {
297                            Either::Right(recovered_auth)
298                        }
299                    })
300                })
301                .collect();
302            tx_env.base.authorization_list = cheated_auths;
303        }
304
305        if self.networks.is_optimism() {
306            tx_env.enveloped_tx = Some(alloy_rlp::encode(tx.transaction.as_ref()).into());
307        }
308
309        Env::new(self.evm_env.clone(), tx_env, self.networks)
310    }
311}
312
313/// Represents the result of a single transaction execution attempt
314#[derive(Debug)]
315pub enum TransactionExecutionOutcome {
316    /// Transaction successfully executed
317    Executed(ExecutedTransaction),
318    /// Invalid transaction not executed
319    Invalid(Arc<PoolTransaction>, InvalidTransactionError),
320    /// Execution skipped because could exceed block gas limit
321    BlockGasExhausted(Arc<PoolTransaction>),
322    /// Execution skipped because it exceeded the blob gas limit
323    BlobGasExhausted(Arc<PoolTransaction>),
324    /// Execution skipped because it exceeded the transaction gas limit
325    TransactionGasExhausted(Arc<PoolTransaction>),
326    /// When an error occurred during execution
327    DatabaseError(Arc<PoolTransaction>, DatabaseError),
328}
329
330impl<DB: Db + ?Sized, V: TransactionValidator> Iterator for &mut TransactionExecutor<'_, DB, V> {
331    type Item = TransactionExecutionOutcome;
332
333    fn next(&mut self) -> Option<Self::Item> {
334        let transaction = self.pending.next()?;
335        let sender = *transaction.pending_transaction.sender();
336        let account = match self.db.basic(sender).map(|acc| acc.unwrap_or_default()) {
337            Ok(account) => account,
338            Err(err) => return Some(TransactionExecutionOutcome::DatabaseError(transaction, err)),
339        };
340        let env = self.env_for(&transaction.pending_transaction);
341
342        // check that we comply with the block's gas limit, if not disabled
343        let max_block_gas = self.gas_used.saturating_add(env.tx.base.gas_limit);
344        if !env.evm_env.cfg_env.disable_block_gas_limit
345            && max_block_gas > env.evm_env.block_env.gas_limit
346        {
347            return Some(TransactionExecutionOutcome::BlockGasExhausted(transaction));
348        }
349
350        // check that we comply with the transaction's gas limit as imposed by Osaka (EIP-7825)
351        if env.evm_env.cfg_env.tx_gas_limit_cap.is_none()
352            && transaction.pending_transaction.transaction.gas_limit()
353                > env.evm_env.cfg_env().tx_gas_limit_cap()
354        {
355            return Some(TransactionExecutionOutcome::TransactionGasExhausted(transaction));
356        }
357
358        // check that we comply with the block's blob gas limit
359        let max_blob_gas = self.blob_gas_used.saturating_add(
360            transaction.pending_transaction.transaction.blob_gas_used().unwrap_or(0),
361        );
362        if max_blob_gas > self.blob_params.max_blob_gas_per_block() {
363            return Some(TransactionExecutionOutcome::BlobGasExhausted(transaction));
364        }
365
366        // validate before executing
367        if let Err(err) = self.validator.validate_pool_transaction_for(
368            &transaction.pending_transaction,
369            &account,
370            &env,
371        ) {
372            warn!(target: "backend", "Skipping invalid tx execution [{:?}] {}", transaction.hash(), err);
373            return Some(TransactionExecutionOutcome::Invalid(transaction, err));
374        }
375
376        let nonce = account.nonce;
377
378        let mut inspector = AnvilInspector::default().with_tracing();
379        if self.enable_steps_tracing {
380            inspector = inspector.with_steps_tracing();
381        }
382        if self.print_logs {
383            inspector = inspector.with_log_collector();
384        }
385        if self.print_traces {
386            inspector = inspector.with_trace_printer();
387        }
388
389        let exec_result = {
390            let mut evm = new_evm_with_inspector(&mut *self.db, &env, &mut inspector);
391            self.networks.inject_precompiles(evm.precompiles_mut());
392
393            if let Some(factory) = &self.precompile_factory {
394                evm.precompiles_mut().extend_precompiles(factory.precompiles());
395            }
396
397            let cheats = Arc::new(self.cheats.clone());
398            if cheats.has_recover_overrides() {
399                let cheat_ecrecover = CheatEcrecover::new(Arc::clone(&cheats));
400                evm.precompiles_mut().apply_precompile(&EC_RECOVER, move |_| {
401                    Some(DynPrecompile::new_stateful(
402                        cheat_ecrecover.precompile_id().clone(),
403                        move |input| cheat_ecrecover.call(input),
404                    ))
405                });
406            }
407
408            trace!(target: "backend", "[{:?}] executing", transaction.hash());
409            // transact and commit the transaction
410            match evm.transact_commit(env.tx) {
411                Ok(exec_result) => exec_result,
412                Err(err) => {
413                    warn!(target: "backend", "[{:?}] failed to execute: {:?}", transaction.hash(), err);
414                    match err {
415                        EVMError::Database(err) => {
416                            return Some(TransactionExecutionOutcome::DatabaseError(
417                                transaction,
418                                err,
419                            ));
420                        }
421                        EVMError::Transaction(err) => {
422                            return Some(TransactionExecutionOutcome::Invalid(
423                                transaction,
424                                err.into(),
425                            ));
426                        }
427                        // This will correspond to prevrandao not set, and it should never happen.
428                        // If it does, it's a bug.
429                        e => panic!("failed to execute transaction: {e}"),
430                    }
431                }
432            }
433        };
434
435        if self.print_traces {
436            inspector.print_traces(self.call_trace_decoder.clone());
437        }
438        inspector.print_logs();
439
440        let (exit_reason, gas_used, out, logs) = match exec_result {
441            ExecutionResult::Success { reason, gas_used, logs, output, .. } => {
442                (reason.into(), gas_used, Some(output), Some(logs))
443            }
444            ExecutionResult::Revert { gas_used, output } => {
445                (InstructionResult::Revert, gas_used, Some(Output::Call(output)), None)
446            }
447            ExecutionResult::Halt { reason, gas_used } => {
448                (op_haltreason_to_instruction_result(reason), gas_used, None, None)
449            }
450        };
451
452        if exit_reason == InstructionResult::OutOfGas {
453            // this currently useful for debugging estimations
454            warn!(target: "backend", "[{:?}] executed with out of gas", transaction.hash())
455        }
456
457        trace!(target: "backend", ?exit_reason, ?gas_used, "[{:?}] executed with out={:?}", transaction.hash(), out);
458
459        // Track the total gas used for total gas per block checks
460        self.gas_used = self.gas_used.saturating_add(gas_used);
461
462        // Track the total blob gas used for total blob gas per blob checks
463        if let Some(blob_gas) = transaction.pending_transaction.transaction.blob_gas_used() {
464            self.blob_gas_used = self.blob_gas_used.saturating_add(blob_gas);
465        }
466
467        trace!(target: "backend::executor", "transacted [{:?}], result: {:?} gas {}", transaction.hash(), exit_reason, gas_used);
468
469        let tx = ExecutedTransaction {
470            transaction,
471            exit_reason,
472            out,
473            gas_used,
474            logs: logs.unwrap_or_default(),
475            traces: inspector.tracer.map(|t| t.into_traces().into_nodes()).unwrap_or_default(),
476            nonce,
477        };
478
479        Some(TransactionExecutionOutcome::Executed(tx))
480    }
481}
482
483/// Inserts all logs into the bloom
484fn build_logs_bloom(logs: &[Log], bloom: &mut Bloom) {
485    for log in logs {
486        bloom.accrue(BloomInput::Raw(&log.address[..]));
487        for topic in log.topics() {
488            bloom.accrue(BloomInput::Raw(&topic[..]));
489        }
490    }
491}
492
493/// Creates a database with given database and inspector.
494pub fn new_evm_with_inspector<DB, I>(
495    db: DB,
496    env: &Env,
497    inspector: I,
498) -> EitherEvm<DB, I, PrecompilesMap>
499where
500    DB: Database<Error = DatabaseError> + Debug,
501    I: Inspector<EthEvmContext<DB>> + Inspector<OpContext<DB>>,
502{
503    if env.networks.is_optimism() {
504        let evm_env = EvmEnv::new(
505            env.evm_env.cfg_env.clone().with_spec(op_revm::OpSpecId::ISTHMUS),
506            env.evm_env.block_env.clone(),
507        );
508        EitherEvm::Op(OpEvmFactory::default().create_evm_with_inspector(db, evm_env, inspector))
509    } else {
510        EitherEvm::Eth(EthEvmFactory::default().create_evm_with_inspector(
511            db,
512            env.evm_env.clone(),
513            inspector,
514        ))
515    }
516}