foundry_cheatcodes/
inspector.rs

1//! Cheatcode EVM inspector.
2
3use crate::{
4    CheatsConfig, CheatsCtxt, DynCheatcode, Error, Result,
5    Vm::{self, AccountAccess},
6    evm::{
7        DealRecord, GasRecord, RecordAccess, journaled_account,
8        mock::{MockCallDataContext, MockCallReturnData},
9        prank::Prank,
10    },
11    inspector::utils::CommonCreateInput,
12    script::{Broadcast, Wallets},
13    test::{
14        assume::AssumeNoRevert,
15        expect::{
16            self, ExpectedCallData, ExpectedCallTracker, ExpectedCallType, ExpectedCreate,
17            ExpectedEmitTracker, ExpectedRevert, ExpectedRevertKind,
18        },
19        revert_handlers,
20    },
21    utils::IgnoredTraces,
22};
23use alloy_consensus::BlobTransactionSidecar;
24use alloy_evm::eth::EthEvmContext;
25use alloy_network::TransactionBuilder4844;
26use alloy_primitives::{
27    Address, B256, Bytes, Log, TxKind, U256, hex,
28    map::{AddressHashMap, HashMap, HashSet},
29};
30use alloy_rpc_types::{
31    AccessList,
32    request::{TransactionInput, TransactionRequest},
33};
34use alloy_sol_types::{SolCall, SolInterface, SolValue};
35use foundry_common::{
36    SELECTOR_LEN, TransactionMaybeSigned,
37    mapping_slots::{MappingSlots, step as mapping_step},
38};
39use foundry_evm_core::{
40    Breakpoints, ContextExt, InspectorExt,
41    abi::Vm::stopExpectSafeMemoryCall,
42    backend::{DatabaseError, DatabaseExt, RevertDiagnostic},
43    constants::{CHEATCODE_ADDRESS, HARDHAT_CONSOLE_ADDRESS, MAGIC_ASSUME},
44    evm::{FoundryEvm, new_evm_with_existing_context},
45};
46use foundry_evm_traces::{
47    TracingInspector, TracingInspectorConfig, identifier::SignaturesIdentifier,
48};
49use foundry_wallets::wallet_multi::MultiWallet;
50use itertools::Itertools;
51use proptest::test_runner::{RngAlgorithm, TestRng, TestRunner};
52use rand::Rng;
53use revm::{
54    Inspector, Journal,
55    bytecode::opcode as op,
56    context::{BlockEnv, JournalTr, LocalContext, TransactionType, result::EVMError},
57    context_interface::{CreateScheme, transaction::SignedAuthorization},
58    handler::FrameResult,
59    interpreter::{
60        CallInputs, CallOutcome, CallScheme, CreateInputs, CreateOutcome, FrameInput, Gas, Host,
61        InstructionResult, Interpreter, InterpreterAction, InterpreterResult,
62        interpreter_types::{Jumps, LoopControl, MemoryTr},
63    },
64    primitives::hardfork::SpecId,
65};
66use serde_json::Value;
67use std::{
68    cmp::max,
69    collections::{BTreeMap, VecDeque},
70    fs::File,
71    io::BufReader,
72    ops::Range,
73    path::PathBuf,
74    sync::{Arc, OnceLock},
75};
76
77mod utils;
78
79pub mod analysis;
80pub use analysis::CheatcodeAnalysis;
81
82pub type Ecx<'a, 'b, 'c> = &'a mut EthEvmContext<&'b mut (dyn DatabaseExt + 'c)>;
83
84/// Helper trait for obtaining complete [revm::Inspector] instance from mutable reference to
85/// [Cheatcodes].
86///
87/// This is needed for cases when inspector itself needs mutable access to [Cheatcodes] state and
88/// allows us to correctly execute arbitrary EVM frames from inside cheatcode implementations.
89pub trait CheatcodesExecutor {
90    /// Core trait method accepting mutable reference to [Cheatcodes] and returning
91    /// [revm::Inspector].
92    fn get_inspector<'a>(&'a mut self, cheats: &'a mut Cheatcodes) -> Box<dyn InspectorExt + 'a>;
93
94    /// Obtains [FoundryEvm] instance and executes the given CREATE frame.
95    fn exec_create(
96        &mut self,
97        inputs: CreateInputs,
98        ccx: &mut CheatsCtxt,
99    ) -> Result<CreateOutcome, EVMError<DatabaseError>> {
100        with_evm(self, ccx, |evm| {
101            evm.journaled_state.depth += 1;
102
103            let frame = FrameInput::Create(Box::new(inputs));
104
105            let outcome = match evm.run_execution(frame)? {
106                FrameResult::Call(_) => unreachable!(),
107                FrameResult::Create(create) => create,
108            };
109
110            evm.journaled_state.depth -= 1;
111
112            Ok(outcome)
113        })
114    }
115
116    fn console_log(&mut self, ccx: &mut CheatsCtxt, msg: &str) {
117        self.get_inspector(ccx.state).console_log(msg);
118    }
119
120    /// Returns a mutable reference to the tracing inspector if it is available.
121    fn tracing_inspector(&mut self) -> Option<&mut TracingInspector> {
122        None
123    }
124}
125
126/// Constructs [FoundryEvm] and runs a given closure with it.
127fn with_evm<E, F, O>(
128    executor: &mut E,
129    ccx: &mut CheatsCtxt,
130    f: F,
131) -> Result<O, EVMError<DatabaseError>>
132where
133    E: CheatcodesExecutor + ?Sized,
134    F: for<'a, 'b> FnOnce(
135        &mut FoundryEvm<'a, &'b mut dyn InspectorExt>,
136    ) -> Result<O, EVMError<DatabaseError>>,
137{
138    let mut inspector = executor.get_inspector(ccx.state);
139    let error = std::mem::replace(&mut ccx.ecx.error, Ok(()));
140
141    let ctx = EthEvmContext {
142        block: ccx.ecx.block.clone(),
143        cfg: ccx.ecx.cfg.clone(),
144        tx: ccx.ecx.tx.clone(),
145        journaled_state: Journal {
146            inner: ccx.ecx.journaled_state.inner.clone(),
147            database: &mut *ccx.ecx.journaled_state.database as &mut dyn DatabaseExt,
148        },
149        local: LocalContext::default(),
150        chain: (),
151        error,
152    };
153
154    let mut evm = new_evm_with_existing_context(ctx, &mut *inspector);
155
156    let res = f(&mut evm)?;
157
158    let ctx = evm.into_context();
159    ccx.ecx.journaled_state.inner = ctx.journaled_state.inner;
160    ccx.ecx.block = ctx.block;
161    ccx.ecx.tx = ctx.tx;
162    ccx.ecx.cfg = ctx.cfg;
163    ccx.ecx.error = ctx.error;
164
165    Ok(res)
166}
167
168/// Basic implementation of [CheatcodesExecutor] that simply returns the [Cheatcodes] instance as an
169/// inspector.
170#[derive(Debug, Default, Clone, Copy)]
171struct TransparentCheatcodesExecutor;
172
173impl CheatcodesExecutor for TransparentCheatcodesExecutor {
174    fn get_inspector<'a>(&'a mut self, cheats: &'a mut Cheatcodes) -> Box<dyn InspectorExt + 'a> {
175        Box::new(cheats)
176    }
177}
178
179macro_rules! try_or_return {
180    ($e:expr) => {
181        match $e {
182            Ok(v) => v,
183            Err(_) => return,
184        }
185    };
186}
187
188/// Contains additional, test specific resources that should be kept for the duration of the test
189#[derive(Debug, Default)]
190pub struct TestContext {
191    /// Buffered readers for files opened for reading (path => BufReader mapping)
192    pub opened_read_files: HashMap<PathBuf, BufReader<File>>,
193}
194
195/// Every time we clone `Context`, we want it to be empty
196impl Clone for TestContext {
197    fn clone(&self) -> Self {
198        Default::default()
199    }
200}
201
202impl TestContext {
203    /// Clears the context.
204    pub fn clear(&mut self) {
205        self.opened_read_files.clear();
206    }
207}
208
209/// Helps collecting transactions from different forks.
210#[derive(Clone, Debug)]
211pub struct BroadcastableTransaction {
212    /// The optional RPC URL.
213    pub rpc: Option<String>,
214    /// The transaction to broadcast.
215    pub transaction: TransactionMaybeSigned,
216}
217
218#[derive(Clone, Debug, Copy)]
219pub struct RecordDebugStepInfo {
220    /// The debug trace node index when the recording starts.
221    pub start_node_idx: usize,
222    /// The original tracer config when the recording starts.
223    pub original_tracer_config: TracingInspectorConfig,
224}
225
226/// Holds gas metering state.
227#[derive(Clone, Debug, Default)]
228pub struct GasMetering {
229    /// True if gas metering is paused.
230    pub paused: bool,
231    /// True if gas metering was resumed or reset during the test.
232    /// Used to reconcile gas when frame ends (if spent less than refunded).
233    pub touched: bool,
234    /// True if gas metering should be reset to frame limit.
235    pub reset: bool,
236    /// Stores paused gas frames.
237    pub paused_frames: Vec<Gas>,
238
239    /// The group and name of the active snapshot.
240    pub active_gas_snapshot: Option<(String, String)>,
241
242    /// Cache of the amount of gas used in previous call.
243    /// This is used by the `lastCallGas` cheatcode.
244    pub last_call_gas: Option<crate::Vm::Gas>,
245
246    /// True if gas recording is enabled.
247    pub recording: bool,
248    /// The gas used in the last frame.
249    pub last_gas_used: u64,
250    /// Gas records for the active snapshots.
251    pub gas_records: Vec<GasRecord>,
252}
253
254impl GasMetering {
255    /// Start the gas recording.
256    pub fn start(&mut self) {
257        self.recording = true;
258    }
259
260    /// Stop the gas recording.
261    pub fn stop(&mut self) {
262        self.recording = false;
263    }
264
265    /// Resume paused gas metering.
266    pub fn resume(&mut self) {
267        if self.paused {
268            self.paused = false;
269            self.touched = true;
270        }
271        self.paused_frames.clear();
272    }
273
274    /// Reset gas to limit.
275    pub fn reset(&mut self) {
276        self.paused = false;
277        self.touched = true;
278        self.reset = true;
279        self.paused_frames.clear();
280    }
281}
282
283/// Holds data about arbitrary storage.
284#[derive(Clone, Debug, Default)]
285pub struct ArbitraryStorage {
286    /// Mapping of arbitrary storage addresses to generated values (slot, arbitrary value).
287    /// (SLOADs return random value if storage slot wasn't accessed).
288    /// Changed values are recorded and used to copy storage to different addresses.
289    pub values: HashMap<Address, HashMap<U256, U256>>,
290    /// Mapping of address with storage copied to arbitrary storage address source.
291    pub copies: HashMap<Address, Address>,
292    /// Address with storage slots that should be overwritten even if previously set.
293    pub overwrites: HashSet<Address>,
294}
295
296impl ArbitraryStorage {
297    /// Marks an address with arbitrary storage.
298    pub fn mark_arbitrary(&mut self, address: &Address, overwrite: bool) {
299        self.values.insert(*address, HashMap::default());
300        if overwrite {
301            self.overwrites.insert(*address);
302        } else {
303            self.overwrites.remove(address);
304        }
305    }
306
307    /// Maps an address that copies storage with the arbitrary storage address.
308    pub fn mark_copy(&mut self, from: &Address, to: &Address) {
309        if self.values.contains_key(from) {
310            self.copies.insert(*to, *from);
311        }
312    }
313
314    /// Saves arbitrary storage value for a given address:
315    /// - store value in changed values cache.
316    /// - update account's storage with given value.
317    pub fn save(&mut self, ecx: Ecx, address: Address, slot: U256, data: U256) {
318        self.values.get_mut(&address).expect("missing arbitrary address entry").insert(slot, data);
319        let (db, journal, _) = ecx.as_db_env_and_journal();
320        if journal.load_account(db, address).is_ok() {
321            journal
322                .sstore(db, address, slot, data, false)
323                .expect("could not set arbitrary storage value");
324        }
325    }
326
327    /// Copies arbitrary storage value from source address to the given target address:
328    /// - if a value is present in arbitrary values cache, then update target storage and return
329    ///   existing value.
330    /// - if no value was yet generated for given slot, then save new value in cache and update both
331    ///   source and target storages.
332    pub fn copy(&mut self, ecx: Ecx, target: Address, slot: U256, new_value: U256) -> U256 {
333        let source = self.copies.get(&target).expect("missing arbitrary copy target entry");
334        let storage_cache = self.values.get_mut(source).expect("missing arbitrary source storage");
335        let value = match storage_cache.get(&slot) {
336            Some(value) => *value,
337            None => {
338                storage_cache.insert(slot, new_value);
339                // Update source storage with new value.
340                let (db, journal, _) = ecx.as_db_env_and_journal();
341                if journal.load_account(db, *source).is_ok() {
342                    journal
343                        .sstore(db, *source, slot, new_value, false)
344                        .expect("could not copy arbitrary storage value");
345                }
346                new_value
347            }
348        };
349        // Update target storage with new value.
350        let (db, journal, _) = ecx.as_db_env_and_journal();
351        if journal.load_account(db, target).is_ok() {
352            journal.sstore(db, target, slot, value, false).expect("could not set storage");
353        }
354        value
355    }
356}
357
358/// List of transactions that can be broadcasted.
359pub type BroadcastableTransactions = VecDeque<BroadcastableTransaction>;
360
361/// An EVM inspector that handles calls to various cheatcodes, each with their own behavior.
362///
363/// Cheatcodes can be called by contracts during execution to modify the VM environment, such as
364/// mocking addresses, signatures and altering call reverts.
365///
366/// Executing cheatcodes can be very powerful. Most cheatcodes are limited to evm internals, but
367/// there are also cheatcodes like `ffi` which can execute arbitrary commands or `writeFile` and
368/// `readFile` which can manipulate files of the filesystem. Therefore, several restrictions are
369/// implemented for these cheatcodes:
370/// - `ffi`, and file cheatcodes are _always_ opt-in (via foundry config) and never enabled by
371///   default: all respective cheatcode handlers implement the appropriate checks
372/// - File cheatcodes require explicit permissions which paths are allowed for which operation, see
373///   `Config.fs_permission`
374/// - Only permitted accounts are allowed to execute cheatcodes in forking mode, this ensures no
375///   contract deployed on the live network is able to execute cheatcodes by simply calling the
376///   cheatcode address: by default, the caller, test contract and newly deployed contracts are
377///   allowed to execute cheatcodes
378#[derive(Clone, Debug)]
379pub struct Cheatcodes {
380    /// Solar compiler instance, to grant syntactic and semantic analysis capabilities
381    pub analysis: Option<CheatcodeAnalysis>,
382
383    /// The block environment
384    ///
385    /// Used in the cheatcode handler to overwrite the block environment separately from the
386    /// execution block environment.
387    pub block: Option<BlockEnv>,
388
389    /// Currently active EIP-7702 delegations that will be consumed when building the next
390    /// transaction. Set by `vm.attachDelegation()` and consumed via `.take()` during
391    /// transaction construction.
392    pub active_delegations: Vec<SignedAuthorization>,
393
394    /// The active EIP-4844 blob that will be attached to the next call.
395    pub active_blob_sidecar: Option<BlobTransactionSidecar>,
396
397    /// The gas price.
398    ///
399    /// Used in the cheatcode handler to overwrite the gas price separately from the gas price
400    /// in the execution environment.
401    pub gas_price: Option<u128>,
402
403    /// Address labels
404    pub labels: AddressHashMap<String>,
405
406    /// Prank information, mapped to the call depth where pranks were added.
407    pub pranks: BTreeMap<usize, Prank>,
408
409    /// Expected revert information
410    pub expected_revert: Option<ExpectedRevert>,
411
412    /// Assume next call can revert and discard fuzz run if it does.
413    pub assume_no_revert: Option<AssumeNoRevert>,
414
415    /// Additional diagnostic for reverts
416    pub fork_revert_diagnostic: Option<RevertDiagnostic>,
417
418    /// Recorded storage reads and writes
419    pub accesses: RecordAccess,
420
421    /// Whether storage access recording is currently active
422    pub recording_accesses: bool,
423
424    /// Recorded account accesses (calls, creates) organized by relative call depth, where the
425    /// topmost vector corresponds to accesses at the depth at which account access recording
426    /// began. Each vector in the matrix represents a list of accesses at a specific call
427    /// depth. Once that call context has ended, the last vector is removed from the matrix and
428    /// merged into the previous vector.
429    pub recorded_account_diffs_stack: Option<Vec<Vec<AccountAccess>>>,
430
431    /// The information of the debug step recording.
432    pub record_debug_steps_info: Option<RecordDebugStepInfo>,
433
434    /// Recorded logs
435    pub recorded_logs: Option<Vec<crate::Vm::Log>>,
436
437    /// Mocked calls
438    // **Note**: inner must a BTreeMap because of special `Ord` impl for `MockCallDataContext`
439    pub mocked_calls: HashMap<Address, BTreeMap<MockCallDataContext, VecDeque<MockCallReturnData>>>,
440
441    /// Mocked functions. Maps target address to be mocked to pair of (calldata, mock address).
442    pub mocked_functions: HashMap<Address, HashMap<Bytes, Address>>,
443
444    /// Expected calls
445    pub expected_calls: ExpectedCallTracker,
446    /// Expected emits
447    pub expected_emits: ExpectedEmitTracker,
448    /// Expected creates
449    pub expected_creates: Vec<ExpectedCreate>,
450
451    /// Map of context depths to memory offset ranges that may be written to within the call depth.
452    pub allowed_mem_writes: HashMap<u64, Vec<Range<u64>>>,
453
454    /// Current broadcasting information
455    pub broadcast: Option<Broadcast>,
456
457    /// Scripting based transactions
458    pub broadcastable_transactions: BroadcastableTransactions,
459
460    /// Current EIP-2930 access lists.
461    pub access_list: Option<AccessList>,
462
463    /// Additional, user configurable context this Inspector has access to when inspecting a call.
464    pub config: Arc<CheatsConfig>,
465
466    /// Test-scoped context holding data that needs to be reset every test run
467    pub test_context: TestContext,
468
469    /// Whether to commit FS changes such as file creations, writes and deletes.
470    /// Used to prevent duplicate changes file executing non-committing calls.
471    pub fs_commit: bool,
472
473    /// Serialized JSON values.
474    // **Note**: both must a BTreeMap to ensure the order of the keys is deterministic.
475    pub serialized_jsons: BTreeMap<String, BTreeMap<String, Value>>,
476
477    /// All recorded ETH `deal`s.
478    pub eth_deals: Vec<DealRecord>,
479
480    /// Gas metering state.
481    pub gas_metering: GasMetering,
482
483    /// Contains gas snapshots made over the course of a test suite.
484    // **Note**: both must a BTreeMap to ensure the order of the keys is deterministic.
485    pub gas_snapshots: BTreeMap<String, BTreeMap<String, String>>,
486
487    /// Mapping slots.
488    pub mapping_slots: Option<AddressHashMap<MappingSlots>>,
489
490    /// The current program counter.
491    pub pc: usize,
492    /// Breakpoints supplied by the `breakpoint` cheatcode.
493    /// `char -> (address, pc)`
494    pub breakpoints: Breakpoints,
495
496    /// Whether the next contract creation should be intercepted to return its initcode.
497    pub intercept_next_create_call: bool,
498
499    /// Optional cheatcodes `TestRunner`. Used for generating random values from uint and int
500    /// strategies.
501    test_runner: Option<TestRunner>,
502
503    /// Ignored traces.
504    pub ignored_traces: IgnoredTraces,
505
506    /// Addresses with arbitrary storage.
507    pub arbitrary_storage: Option<ArbitraryStorage>,
508
509    /// Deprecated cheatcodes mapped to the reason. Used to report warnings on test results.
510    pub deprecated: HashMap<&'static str, Option<&'static str>>,
511    /// Unlocked wallets used in scripts and testing of scripts.
512    pub wallets: Option<Wallets>,
513    /// Signatures identifier for decoding events and functions
514    signatures_identifier: OnceLock<Option<SignaturesIdentifier>>,
515    /// Used to determine whether the broadcasted call has dynamic gas limit.
516    pub dynamic_gas_limit: bool,
517    // Custom execution evm version.
518    pub execution_evm_version: Option<SpecId>,
519}
520
521// This is not derived because calling this in `fn new` with `..Default::default()` creates a second
522// `CheatsConfig` which is unused, and inside it `ProjectPathsConfig` is relatively expensive to
523// create.
524impl Default for Cheatcodes {
525    fn default() -> Self {
526        Self::new(Arc::default())
527    }
528}
529
530impl Cheatcodes {
531    /// Creates a new `Cheatcodes` with the given settings.
532    pub fn new(config: Arc<CheatsConfig>) -> Self {
533        Self {
534            analysis: None,
535            fs_commit: true,
536            labels: config.labels.clone(),
537            config,
538            block: Default::default(),
539            active_delegations: Default::default(),
540            active_blob_sidecar: Default::default(),
541            gas_price: Default::default(),
542            pranks: Default::default(),
543            expected_revert: Default::default(),
544            assume_no_revert: Default::default(),
545            fork_revert_diagnostic: Default::default(),
546            accesses: Default::default(),
547            recording_accesses: Default::default(),
548            recorded_account_diffs_stack: Default::default(),
549            recorded_logs: Default::default(),
550            record_debug_steps_info: Default::default(),
551            mocked_calls: Default::default(),
552            mocked_functions: Default::default(),
553            expected_calls: Default::default(),
554            expected_emits: Default::default(),
555            expected_creates: Default::default(),
556            allowed_mem_writes: Default::default(),
557            broadcast: Default::default(),
558            broadcastable_transactions: Default::default(),
559            access_list: Default::default(),
560            test_context: Default::default(),
561            serialized_jsons: Default::default(),
562            eth_deals: Default::default(),
563            gas_metering: Default::default(),
564            gas_snapshots: Default::default(),
565            mapping_slots: Default::default(),
566            pc: Default::default(),
567            breakpoints: Default::default(),
568            intercept_next_create_call: Default::default(),
569            test_runner: Default::default(),
570            ignored_traces: Default::default(),
571            arbitrary_storage: Default::default(),
572            deprecated: Default::default(),
573            wallets: Default::default(),
574            signatures_identifier: Default::default(),
575            dynamic_gas_limit: Default::default(),
576            execution_evm_version: None,
577        }
578    }
579
580    /// Enables cheatcode analysis capabilities by providing a solar compiler instance.
581    pub fn set_analysis(&mut self, analysis: CheatcodeAnalysis) {
582        self.analysis = Some(analysis);
583    }
584
585    /// Returns the configured prank at given depth or the first prank configured at a lower depth.
586    /// For example, if pranks configured for depth 1, 3 and 5, the prank for depth 4 is the one
587    /// configured at depth 3.
588    pub fn get_prank(&self, depth: usize) -> Option<&Prank> {
589        self.pranks.range(..=depth).last().map(|(_, prank)| prank)
590    }
591
592    /// Returns the configured wallets if available, else creates a new instance.
593    pub fn wallets(&mut self) -> &Wallets {
594        self.wallets.get_or_insert_with(|| Wallets::new(MultiWallet::default(), None))
595    }
596
597    /// Sets the unlocked wallets.
598    pub fn set_wallets(&mut self, wallets: Wallets) {
599        self.wallets = Some(wallets);
600    }
601
602    /// Adds a delegation to the active delegations list.
603    pub fn add_delegation(&mut self, authorization: SignedAuthorization) {
604        self.active_delegations.push(authorization);
605    }
606
607    /// Returns the signatures identifier.
608    pub fn signatures_identifier(&self) -> Option<&SignaturesIdentifier> {
609        self.signatures_identifier.get_or_init(|| SignaturesIdentifier::new(true).ok()).as_ref()
610    }
611
612    /// Decodes the input data and applies the cheatcode.
613    fn apply_cheatcode(
614        &mut self,
615        ecx: Ecx,
616        call: &CallInputs,
617        executor: &mut dyn CheatcodesExecutor,
618    ) -> Result {
619        // decode the cheatcode call
620        let decoded = Vm::VmCalls::abi_decode(&call.input.bytes(ecx)).map_err(|e| {
621            if let alloy_sol_types::Error::UnknownSelector { name: _, selector } = e {
622                let msg = format!(
623                    "unknown cheatcode with selector {selector}; \
624                     you may have a mismatch between the `Vm` interface (likely in `forge-std`) \
625                     and the `forge` version"
626                );
627                return alloy_sol_types::Error::Other(std::borrow::Cow::Owned(msg));
628            }
629            e
630        })?;
631
632        let caller = call.caller;
633
634        // ensure the caller is allowed to execute cheatcodes,
635        // but only if the backend is in forking mode
636        ecx.journaled_state.database.ensure_cheatcode_access_forking_mode(&caller)?;
637
638        apply_dispatch(
639            &decoded,
640            &mut CheatsCtxt { state: self, ecx, gas_limit: call.gas_limit, caller },
641            executor,
642        )
643    }
644
645    /// Grants cheat code access for new contracts if the caller also has
646    /// cheatcode access or the new contract is created in top most call.
647    ///
648    /// There may be cheatcodes in the constructor of the new contract, in order to allow them
649    /// automatically we need to determine the new address.
650    fn allow_cheatcodes_on_create(&self, ecx: Ecx, caller: Address, created_address: Address) {
651        if ecx.journaled_state.depth <= 1
652            || ecx.journaled_state.database.has_cheatcode_access(&caller)
653        {
654            ecx.journaled_state.database.allow_cheatcode_access(created_address);
655        }
656    }
657
658    /// Apply EIP-2930 access list.
659    ///
660    /// If the transaction type is [TransactionType::Legacy] we need to upgrade it to
661    /// [TransactionType::Eip2930] in order to use access lists. Other transaction types support
662    /// access lists themselves.
663    fn apply_accesslist(&mut self, ecx: Ecx) {
664        if let Some(access_list) = &self.access_list {
665            ecx.tx.access_list = access_list.clone();
666
667            if ecx.tx.tx_type == TransactionType::Legacy as u8 {
668                ecx.tx.tx_type = TransactionType::Eip2930 as u8;
669            }
670        }
671    }
672
673    /// Called when there was a revert.
674    ///
675    /// Cleanup any previously applied cheatcodes that altered the state in such a way that revm's
676    /// revert would run into issues.
677    pub fn on_revert(&mut self, ecx: Ecx) {
678        trace!(deals=?self.eth_deals.len(), "rolling back deals");
679
680        // Delay revert clean up until expected revert is handled, if set.
681        if self.expected_revert.is_some() {
682            return;
683        }
684
685        // we only want to apply cleanup top level
686        if ecx.journaled_state.depth() > 0 {
687            return;
688        }
689
690        // Roll back all previously applied deals
691        // This will prevent overflow issues in revm's [`JournaledState::journal_revert`] routine
692        // which rolls back any transfers.
693        while let Some(record) = self.eth_deals.pop() {
694            if let Some(acc) = ecx.journaled_state.state.get_mut(&record.address) {
695                acc.info.balance = record.old_balance;
696            }
697        }
698    }
699
700    pub fn call_with_executor(
701        &mut self,
702        ecx: Ecx,
703        call: &mut CallInputs,
704        executor: &mut dyn CheatcodesExecutor,
705    ) -> Option<CallOutcome> {
706        // Apply custom execution evm version.
707        if let Some(spec_id) = self.execution_evm_version {
708            ecx.cfg.spec = spec_id;
709        }
710
711        let gas = Gas::new(call.gas_limit);
712        let curr_depth = ecx.journaled_state.depth();
713
714        // At the root call to test function or script `run()`/`setUp()` functions, we are
715        // decreasing sender nonce to ensure that it matches on-chain nonce once we start
716        // broadcasting.
717        if curr_depth == 0 {
718            let sender = ecx.tx.caller;
719            let account = match super::evm::journaled_account(ecx, sender) {
720                Ok(account) => account,
721                Err(err) => {
722                    return Some(CallOutcome {
723                        result: InterpreterResult {
724                            result: InstructionResult::Revert,
725                            output: err.abi_encode().into(),
726                            gas,
727                        },
728                        memory_offset: call.return_memory_offset.clone(),
729                        was_precompile_called: false,
730                        precompile_call_logs: vec![],
731                    });
732                }
733            };
734            let prev = account.info.nonce;
735            account.info.nonce = prev.saturating_sub(1);
736
737            trace!(target: "cheatcodes", %sender, nonce=account.info.nonce, prev, "corrected nonce");
738        }
739
740        if call.target_address == CHEATCODE_ADDRESS {
741            return match self.apply_cheatcode(ecx, call, executor) {
742                Ok(retdata) => Some(CallOutcome {
743                    result: InterpreterResult {
744                        result: InstructionResult::Return,
745                        output: retdata.into(),
746                        gas,
747                    },
748                    memory_offset: call.return_memory_offset.clone(),
749                    was_precompile_called: true,
750                    precompile_call_logs: vec![],
751                }),
752                Err(err) => Some(CallOutcome {
753                    result: InterpreterResult {
754                        result: InstructionResult::Revert,
755                        output: err.abi_encode().into(),
756                        gas,
757                    },
758                    memory_offset: call.return_memory_offset.clone(),
759                    was_precompile_called: false,
760                    precompile_call_logs: vec![],
761                }),
762            };
763        }
764
765        if call.target_address == HARDHAT_CONSOLE_ADDRESS {
766            return None;
767        }
768
769        // Handle expected calls
770
771        // Grab the different calldatas expected.
772        if let Some(expected_calls_for_target) = self.expected_calls.get_mut(&call.bytecode_address)
773        {
774            // Match every partial/full calldata
775            for (calldata, (expected, actual_count)) in expected_calls_for_target {
776                // Increment actual times seen if...
777                // The calldata is at most, as big as this call's input, and
778                if calldata.len() <= call.input.len() &&
779                    // Both calldata match, taking the length of the assumed smaller one (which will have at least the selector), and
780                    *calldata == call.input.bytes(ecx)[..calldata.len()] &&
781                    // The value matches, if provided
782                    expected
783                        .value.is_none_or(|value| Some(value) == call.transfer_value()) &&
784                    // The gas matches, if provided
785                    expected.gas.is_none_or(|gas| gas == call.gas_limit) &&
786                    // The minimum gas matches, if provided
787                    expected.min_gas.is_none_or(|min_gas| min_gas <= call.gas_limit)
788                {
789                    *actual_count += 1;
790                }
791            }
792        }
793
794        // Handle mocked calls
795        if let Some(mocks) = self.mocked_calls.get_mut(&call.bytecode_address) {
796            let ctx = MockCallDataContext {
797                calldata: call.input.bytes(ecx),
798                value: call.transfer_value(),
799            };
800
801            if let Some(return_data_queue) = match mocks.get_mut(&ctx) {
802                Some(queue) => Some(queue),
803                None => mocks
804                    .iter_mut()
805                    .find(|(mock, _)| {
806                        call.input.bytes(ecx).get(..mock.calldata.len()) == Some(&mock.calldata[..])
807                            && mock.value.is_none_or(|value| Some(value) == call.transfer_value())
808                    })
809                    .map(|(_, v)| v),
810            } && let Some(return_data) = if return_data_queue.len() == 1 {
811                // If the mocked calls stack has a single element in it, don't empty it
812                return_data_queue.front().map(|x| x.to_owned())
813            } else {
814                // Else, we pop the front element
815                return_data_queue.pop_front()
816            } {
817                return Some(CallOutcome {
818                    result: InterpreterResult {
819                        result: return_data.ret_type,
820                        output: return_data.data,
821                        gas,
822                    },
823                    memory_offset: call.return_memory_offset.clone(),
824                    was_precompile_called: true,
825                    precompile_call_logs: vec![],
826                });
827            }
828        }
829
830        // Apply our prank
831        if let Some(prank) = &self.get_prank(curr_depth) {
832            // Apply delegate call, `call.caller`` will not equal `prank.prank_caller`
833            if prank.delegate_call
834                && curr_depth == prank.depth
835                && let CallScheme::DelegateCall = call.scheme
836            {
837                call.target_address = prank.new_caller;
838                call.caller = prank.new_caller;
839                if let Some(new_origin) = prank.new_origin {
840                    ecx.tx.caller = new_origin;
841                }
842            }
843
844            if curr_depth >= prank.depth && call.caller == prank.prank_caller {
845                let mut prank_applied = false;
846
847                // At the target depth we set `msg.sender`
848                if curr_depth == prank.depth {
849                    // Ensure new caller is loaded and touched
850                    let _ = journaled_account(ecx, prank.new_caller);
851                    call.caller = prank.new_caller;
852                    prank_applied = true;
853                }
854
855                // At the target depth, or deeper, we set `tx.origin`
856                if let Some(new_origin) = prank.new_origin {
857                    ecx.tx.caller = new_origin;
858                    prank_applied = true;
859                }
860
861                // If prank applied for first time, then update
862                if prank_applied && let Some(applied_prank) = prank.first_time_applied() {
863                    self.pranks.insert(curr_depth, applied_prank);
864                }
865            }
866        }
867
868        // Apply EIP-2930 access list
869        self.apply_accesslist(ecx);
870
871        // Apply our broadcast
872        if let Some(broadcast) = &self.broadcast {
873            // Additional check as transfers in forge scripts seem to be estimated at 2300
874            // by revm leading to "Intrinsic gas too low" failure when simulated on chain.
875            let is_fixed_gas_limit = call.gas_limit >= 21_000 && !self.dynamic_gas_limit;
876            self.dynamic_gas_limit = false;
877
878            // We only apply a broadcast *to a specific depth*.
879            //
880            // We do this because any subsequent contract calls *must* exist on chain and
881            // we only want to grab *this* call, not internal ones
882            if curr_depth == broadcast.depth && call.caller == broadcast.original_caller {
883                // At the target depth we set `msg.sender` & tx.origin.
884                // We are simulating the caller as being an EOA, so *both* must be set to the
885                // broadcast.origin.
886                ecx.tx.caller = broadcast.new_origin;
887
888                call.caller = broadcast.new_origin;
889                // Add a `legacy` transaction to the VecDeque. We use a legacy transaction here
890                // because we only need the from, to, value, and data. We can later change this
891                // into 1559, in the cli package, relatively easily once we
892                // know the target chain supports EIP-1559.
893                if !call.is_static {
894                    let (db, journal, _) = ecx.as_db_env_and_journal();
895                    if let Err(err) = journal.load_account(db, broadcast.new_origin) {
896                        return Some(CallOutcome {
897                            result: InterpreterResult {
898                                result: InstructionResult::Revert,
899                                output: Error::encode(err),
900                                gas,
901                            },
902                            memory_offset: call.return_memory_offset.clone(),
903                            was_precompile_called: false,
904                            precompile_call_logs: vec![],
905                        });
906                    }
907
908                    let input = TransactionInput::new(call.input.bytes(ecx));
909
910                    let account =
911                        ecx.journaled_state.inner.state().get_mut(&broadcast.new_origin).unwrap();
912
913                    let mut tx_req = TransactionRequest {
914                        from: Some(broadcast.new_origin),
915                        to: Some(TxKind::from(Some(call.target_address))),
916                        value: call.transfer_value(),
917                        input,
918                        nonce: Some(account.info.nonce),
919                        chain_id: Some(ecx.cfg.chain_id),
920                        gas: if is_fixed_gas_limit { Some(call.gas_limit) } else { None },
921                        ..Default::default()
922                    };
923
924                    let active_delegations = std::mem::take(&mut self.active_delegations);
925                    // Set active blob sidecar, if any.
926                    if let Some(blob_sidecar) = self.active_blob_sidecar.take() {
927                        // Ensure blob and delegation are not set for the same tx.
928                        if !active_delegations.is_empty() {
929                            let msg = "both delegation and blob are active; `attachBlob` and `attachDelegation` are not compatible";
930                            return Some(CallOutcome {
931                                result: InterpreterResult {
932                                    result: InstructionResult::Revert,
933                                    output: Error::encode(msg),
934                                    gas,
935                                },
936                                memory_offset: call.return_memory_offset.clone(),
937                                was_precompile_called: false,
938                                precompile_call_logs: vec![],
939                            });
940                        }
941                        tx_req.set_blob_sidecar(blob_sidecar);
942                    }
943
944                    // Apply active EIP-7702 delegations, if any.
945                    if !active_delegations.is_empty() {
946                        for auth in &active_delegations {
947                            let Ok(authority) = auth.recover_authority() else {
948                                continue;
949                            };
950                            if authority == broadcast.new_origin {
951                                // Increment nonce of broadcasting account to reflect signed
952                                // authorization.
953                                account.info.nonce += 1;
954                            }
955                        }
956                        tx_req.authorization_list = Some(active_delegations);
957                    }
958
959                    self.broadcastable_transactions.push_back(BroadcastableTransaction {
960                        rpc: ecx.journaled_state.database.active_fork_url(),
961                        transaction: tx_req.into(),
962                    });
963                    debug!(target: "cheatcodes", tx=?self.broadcastable_transactions.back().unwrap(), "broadcastable call");
964
965                    // Explicitly increment nonce if calls are not isolated.
966                    if !self.config.evm_opts.isolate {
967                        let prev = account.info.nonce;
968                        account.info.nonce += 1;
969                        debug!(target: "cheatcodes", address=%broadcast.new_origin, nonce=prev+1, prev, "incremented nonce");
970                    }
971                } else if broadcast.single_call {
972                    let msg = "`staticcall`s are not allowed after `broadcast`; use `startBroadcast` instead";
973                    return Some(CallOutcome {
974                        result: InterpreterResult {
975                            result: InstructionResult::Revert,
976                            output: Error::encode(msg),
977                            gas,
978                        },
979                        memory_offset: call.return_memory_offset.clone(),
980                        was_precompile_called: false,
981                        precompile_call_logs: vec![],
982                    });
983                }
984            }
985        }
986
987        // Record called accounts if `startStateDiffRecording` has been called
988        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
989            // Determine if account is "initialized," ie, it has a non-zero balance, a non-zero
990            // nonce, a non-zero KECCAK_EMPTY codehash, or non-empty code
991            let initialized;
992            let old_balance;
993            let old_nonce;
994
995            let (db, journal, _) = ecx.as_db_env_and_journal();
996            if let Ok(acc) = journal.load_account(db, call.target_address) {
997                initialized = acc.info.exists();
998                old_balance = acc.info.balance;
999                old_nonce = acc.info.nonce;
1000            } else {
1001                initialized = false;
1002                old_balance = U256::ZERO;
1003                old_nonce = 0;
1004            }
1005
1006            let kind = match call.scheme {
1007                CallScheme::Call => crate::Vm::AccountAccessKind::Call,
1008                CallScheme::CallCode => crate::Vm::AccountAccessKind::CallCode,
1009                CallScheme::DelegateCall => crate::Vm::AccountAccessKind::DelegateCall,
1010                CallScheme::StaticCall => crate::Vm::AccountAccessKind::StaticCall,
1011            };
1012
1013            // Record this call by pushing it to a new pending vector; all subsequent calls at
1014            // that depth will be pushed to the same vector. When the call ends, the
1015            // RecordedAccountAccess (and all subsequent RecordedAccountAccesses) will be
1016            // updated with the revert status of this call, since the EVM does not mark accounts
1017            // as "warm" if the call from which they were accessed is reverted
1018            recorded_account_diffs_stack.push(vec![AccountAccess {
1019                chainInfo: crate::Vm::ChainInfo {
1020                    forkId: ecx.journaled_state.db().active_fork_id().unwrap_or_default(),
1021                    chainId: U256::from(ecx.cfg.chain_id),
1022                },
1023                accessor: call.caller,
1024                account: call.bytecode_address,
1025                kind,
1026                initialized,
1027                oldBalance: old_balance,
1028                newBalance: U256::ZERO, // updated on call_end
1029                oldNonce: old_nonce,
1030                newNonce: 0, // updated on call_end
1031                value: call.call_value(),
1032                data: call.input.bytes(ecx),
1033                reverted: false,
1034                deployedCode: Bytes::new(),
1035                storageAccesses: vec![], // updated on step
1036                depth: ecx
1037                    .journaled_state
1038                    .depth()
1039                    .try_into()
1040                    .expect("journaled state depth exceeds u64"),
1041            }]);
1042        }
1043
1044        None
1045    }
1046
1047    pub fn rng(&mut self) -> &mut impl Rng {
1048        self.test_runner().rng()
1049    }
1050
1051    pub fn test_runner(&mut self) -> &mut TestRunner {
1052        self.test_runner.get_or_insert_with(|| match self.config.seed {
1053            Some(seed) => TestRunner::new_with_rng(
1054                proptest::test_runner::Config::default(),
1055                TestRng::from_seed(RngAlgorithm::ChaCha, &seed.to_be_bytes::<32>()),
1056            ),
1057            None => TestRunner::new(proptest::test_runner::Config::default()),
1058        })
1059    }
1060
1061    pub fn set_seed(&mut self, seed: U256) {
1062        self.test_runner = Some(TestRunner::new_with_rng(
1063            proptest::test_runner::Config::default(),
1064            TestRng::from_seed(RngAlgorithm::ChaCha, &seed.to_be_bytes::<32>()),
1065        ));
1066    }
1067
1068    /// Returns existing or set a default `ArbitraryStorage` option.
1069    /// Used by `setArbitraryStorage` cheatcode to track addresses with arbitrary storage.
1070    pub fn arbitrary_storage(&mut self) -> &mut ArbitraryStorage {
1071        self.arbitrary_storage.get_or_insert_with(ArbitraryStorage::default)
1072    }
1073
1074    /// Whether the given address has arbitrary storage.
1075    pub fn has_arbitrary_storage(&self, address: &Address) -> bool {
1076        match &self.arbitrary_storage {
1077            Some(storage) => storage.values.contains_key(address),
1078            None => false,
1079        }
1080    }
1081
1082    /// Whether the given slot of address with arbitrary storage should be overwritten.
1083    /// True if address is marked as and overwrite and if no value was previously generated for
1084    /// given slot.
1085    pub fn should_overwrite_arbitrary_storage(
1086        &self,
1087        address: &Address,
1088        storage_slot: U256,
1089    ) -> bool {
1090        match &self.arbitrary_storage {
1091            Some(storage) => {
1092                storage.overwrites.contains(address)
1093                    && storage
1094                        .values
1095                        .get(address)
1096                        .and_then(|arbitrary_values| arbitrary_values.get(&storage_slot))
1097                        .is_none()
1098            }
1099            None => false,
1100        }
1101    }
1102
1103    /// Whether the given address is a copy of an address with arbitrary storage.
1104    pub fn is_arbitrary_storage_copy(&self, address: &Address) -> bool {
1105        match &self.arbitrary_storage {
1106            Some(storage) => storage.copies.contains_key(address),
1107            None => false,
1108        }
1109    }
1110
1111    /// Returns struct definitions from the analysis, if available.
1112    pub fn struct_defs(&self) -> Option<&foundry_common::fmt::StructDefinitions> {
1113        self.analysis.as_ref().and_then(|analysis| analysis.struct_defs().ok())
1114    }
1115}
1116
1117impl Inspector<EthEvmContext<&mut dyn DatabaseExt>> for Cheatcodes {
1118    fn initialize_interp(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1119        // When the first interpreter is initialized we've circumvented the balance and gas checks,
1120        // so we apply our actual block data with the correct fees and all.
1121        if let Some(block) = self.block.take() {
1122            ecx.block = block;
1123        }
1124        if let Some(gas_price) = self.gas_price.take() {
1125            ecx.tx.gas_price = gas_price;
1126        }
1127
1128        // Record gas for current frame.
1129        if self.gas_metering.paused {
1130            self.gas_metering.paused_frames.push(interpreter.gas);
1131        }
1132
1133        // `expectRevert`: track the max call depth during `expectRevert`
1134        if let Some(expected) = &mut self.expected_revert {
1135            expected.max_depth = max(ecx.journaled_state.depth(), expected.max_depth);
1136        }
1137    }
1138
1139    fn step(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1140        self.pc = interpreter.bytecode.pc();
1141
1142        if self.broadcast.is_some() {
1143            self.set_gas_limit_type(interpreter);
1144        }
1145
1146        // `pauseGasMetering`: pause / resume interpreter gas.
1147        if self.gas_metering.paused {
1148            self.meter_gas(interpreter);
1149        }
1150
1151        // `resetGasMetering`: reset interpreter gas.
1152        if self.gas_metering.reset {
1153            self.meter_gas_reset(interpreter);
1154        }
1155
1156        // `record`: record storage reads and writes.
1157        if self.recording_accesses {
1158            self.record_accesses(interpreter);
1159        }
1160
1161        // `startStateDiffRecording`: record granular ordered storage accesses.
1162        if self.recorded_account_diffs_stack.is_some() {
1163            self.record_state_diffs(interpreter, ecx);
1164        }
1165
1166        // `expectSafeMemory`: check if the current opcode is allowed to interact with memory.
1167        if !self.allowed_mem_writes.is_empty() {
1168            self.check_mem_opcodes(
1169                interpreter,
1170                ecx.journaled_state.depth().try_into().expect("journaled state depth exceeds u64"),
1171            );
1172        }
1173
1174        // `startMappingRecording`: record SSTORE and KECCAK256.
1175        if let Some(mapping_slots) = &mut self.mapping_slots {
1176            mapping_step(mapping_slots, interpreter);
1177        }
1178
1179        // `snapshotGas*`: take a snapshot of the current gas.
1180        if self.gas_metering.recording {
1181            self.meter_gas_record(interpreter, ecx);
1182        }
1183    }
1184
1185    fn step_end(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1186        if self.gas_metering.paused {
1187            self.meter_gas_end(interpreter);
1188        }
1189
1190        if self.gas_metering.touched {
1191            self.meter_gas_check(interpreter);
1192        }
1193
1194        // `setArbitraryStorage` and `copyStorage`: add arbitrary values to storage.
1195        if self.arbitrary_storage.is_some() {
1196            self.arbitrary_storage_end(interpreter, ecx);
1197        }
1198    }
1199
1200    fn log(&mut self, _ecx: Ecx, log: Log) {
1201        if !self.expected_emits.is_empty()
1202            && let Some(err) = expect::handle_expect_emit(self, &log, None)
1203        {
1204            // Because we do not have access to the interpreter here, we cannot fail the test
1205            // immediately. In most cases the failure will still be caught on `call_end`.
1206            // In the rare case it is not, we log the error here.
1207            let _ = sh_err!("{err:?}");
1208        }
1209
1210        // `recordLogs`
1211        record_logs(&mut self.recorded_logs, &log);
1212    }
1213
1214    fn log_full(&mut self, interpreter: &mut Interpreter, _ecx: Ecx, log: Log) {
1215        if !self.expected_emits.is_empty() {
1216            expect::handle_expect_emit(self, &log, Some(interpreter));
1217        }
1218
1219        // `recordLogs`
1220        record_logs(&mut self.recorded_logs, &log);
1221    }
1222
1223    fn call(&mut self, ecx: Ecx, inputs: &mut CallInputs) -> Option<CallOutcome> {
1224        Self::call_with_executor(self, ecx, inputs, &mut TransparentCheatcodesExecutor)
1225    }
1226
1227    fn call_end(&mut self, ecx: Ecx, call: &CallInputs, outcome: &mut CallOutcome) {
1228        let cheatcode_call = call.target_address == CHEATCODE_ADDRESS
1229            || call.target_address == HARDHAT_CONSOLE_ADDRESS;
1230
1231        // Clean up pranks/broadcasts if it's not a cheatcode call end. We shouldn't do
1232        // it for cheatcode calls because they are not applied for cheatcodes in the `call` hook.
1233        // This should be placed before the revert handling, because we might exit early there
1234        if !cheatcode_call {
1235            // Clean up pranks
1236            let curr_depth = ecx.journaled_state.depth();
1237            if let Some(prank) = &self.get_prank(curr_depth)
1238                && curr_depth == prank.depth
1239            {
1240                ecx.tx.caller = prank.prank_origin;
1241
1242                // Clean single-call prank once we have returned to the original depth
1243                if prank.single_call {
1244                    self.pranks.remove(&curr_depth);
1245                }
1246            }
1247
1248            // Clean up broadcast
1249            if let Some(broadcast) = &self.broadcast
1250                && curr_depth == broadcast.depth
1251            {
1252                ecx.tx.caller = broadcast.original_origin;
1253
1254                // Clean single-call broadcast once we have returned to the original depth
1255                if broadcast.single_call {
1256                    let _ = self.broadcast.take();
1257                }
1258            }
1259        }
1260
1261        // Handle assume no revert cheatcode.
1262        if let Some(assume_no_revert) = &mut self.assume_no_revert {
1263            // Record current reverter address before processing the expect revert if call reverted,
1264            // expect revert is set with expected reverter address and no actual reverter set yet.
1265            if outcome.result.is_revert() && assume_no_revert.reverted_by.is_none() {
1266                assume_no_revert.reverted_by = Some(call.target_address);
1267            }
1268
1269            // allow multiple cheatcode calls at the same depth
1270            let curr_depth = ecx.journaled_state.depth();
1271            if curr_depth <= assume_no_revert.depth && !cheatcode_call {
1272                // Discard run if we're at the same depth as cheatcode, call reverted, and no
1273                // specific reason was supplied
1274                if outcome.result.is_revert() {
1275                    let assume_no_revert = std::mem::take(&mut self.assume_no_revert).unwrap();
1276                    return match revert_handlers::handle_assume_no_revert(
1277                        &assume_no_revert,
1278                        outcome.result.result,
1279                        &outcome.result.output,
1280                        &self.config.available_artifacts,
1281                    ) {
1282                        // if result is Ok, it was an anticipated revert; return an "assume" error
1283                        // to reject this run
1284                        Ok(_) => {
1285                            outcome.result.output = Error::from(MAGIC_ASSUME).abi_encode().into();
1286                        }
1287                        // if result is Error, it was an unanticipated revert; should revert
1288                        // normally
1289                        Err(error) => {
1290                            trace!(expected=?assume_no_revert, ?error, status=?outcome.result.result, "Expected revert mismatch");
1291                            outcome.result.result = InstructionResult::Revert;
1292                            outcome.result.output = error.abi_encode().into();
1293                        }
1294                    };
1295                } else {
1296                    // Call didn't revert, reset `assume_no_revert` state.
1297                    self.assume_no_revert = None;
1298                }
1299            }
1300        }
1301
1302        // Handle expected reverts.
1303        if let Some(expected_revert) = &mut self.expected_revert {
1304            // Record current reverter address and call scheme before processing the expect revert
1305            // if call reverted.
1306            if outcome.result.is_revert() {
1307                // Record current reverter address if expect revert is set with expected reverter
1308                // address and no actual reverter was set yet or if we're expecting more than one
1309                // revert.
1310                if expected_revert.reverter.is_some()
1311                    && (expected_revert.reverted_by.is_none() || expected_revert.count > 1)
1312                {
1313                    expected_revert.reverted_by = Some(call.target_address);
1314                }
1315            }
1316
1317            let curr_depth = ecx.journaled_state.depth();
1318            if curr_depth <= expected_revert.depth {
1319                let needs_processing = match expected_revert.kind {
1320                    ExpectedRevertKind::Default => !cheatcode_call,
1321                    // `pending_processing` == true means that we're in the `call_end` hook for
1322                    // `vm.expectCheatcodeRevert` and shouldn't expect revert here
1323                    ExpectedRevertKind::Cheatcode { pending_processing } => {
1324                        cheatcode_call && !pending_processing
1325                    }
1326                };
1327
1328                if needs_processing {
1329                    let mut expected_revert = std::mem::take(&mut self.expected_revert).unwrap();
1330                    return match revert_handlers::handle_expect_revert(
1331                        cheatcode_call,
1332                        false,
1333                        self.config.internal_expect_revert,
1334                        &expected_revert,
1335                        outcome.result.result,
1336                        outcome.result.output.clone(),
1337                        &self.config.available_artifacts,
1338                    ) {
1339                        Err(error) => {
1340                            trace!(expected=?expected_revert, ?error, status=?outcome.result.result, "Expected revert mismatch");
1341                            outcome.result.result = InstructionResult::Revert;
1342                            outcome.result.output = error.abi_encode().into();
1343                        }
1344                        Ok((_, retdata)) => {
1345                            expected_revert.actual_count += 1;
1346                            if expected_revert.actual_count < expected_revert.count {
1347                                self.expected_revert = Some(expected_revert);
1348                            }
1349                            outcome.result.result = InstructionResult::Return;
1350                            outcome.result.output = retdata;
1351                        }
1352                    };
1353                }
1354
1355                // Flip `pending_processing` flag for cheatcode revert expectations, marking that
1356                // we've exited the `expectCheatcodeRevert` call scope
1357                if let ExpectedRevertKind::Cheatcode { pending_processing } =
1358                    &mut self.expected_revert.as_mut().unwrap().kind
1359                {
1360                    *pending_processing = false;
1361                }
1362            }
1363        }
1364
1365        // Exit early for calls to cheatcodes as other logic is not relevant for cheatcode
1366        // invocations
1367        if cheatcode_call {
1368            return;
1369        }
1370
1371        // Record the gas usage of the call, this allows the `lastCallGas` cheatcode to
1372        // retrieve the gas usage of the last call.
1373        let gas = outcome.result.gas;
1374        self.gas_metering.last_call_gas = Some(crate::Vm::Gas {
1375            gasLimit: gas.limit(),
1376            gasTotalUsed: gas.spent(),
1377            gasMemoryUsed: 0,
1378            gasRefunded: gas.refunded(),
1379            gasRemaining: gas.remaining(),
1380        });
1381
1382        // If `startStateDiffRecording` has been called, update the `reverted` status of the
1383        // previous call depth's recorded accesses, if any
1384        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
1385            // The root call cannot be recorded.
1386            if ecx.journaled_state.depth() > 0
1387                && let Some(mut last_recorded_depth) = recorded_account_diffs_stack.pop()
1388            {
1389                // Update the reverted status of all deeper calls if this call reverted, in
1390                // accordance with EVM behavior
1391                if outcome.result.is_revert() {
1392                    last_recorded_depth.iter_mut().for_each(|element| {
1393                        element.reverted = true;
1394                        element
1395                            .storageAccesses
1396                            .iter_mut()
1397                            .for_each(|storage_access| storage_access.reverted = true);
1398                    })
1399                }
1400
1401                if let Some(call_access) = last_recorded_depth.first_mut() {
1402                    // Assert that we're at the correct depth before recording post-call state
1403                    // changes. Depending on the depth the cheat was
1404                    // called at, there may not be any pending
1405                    // calls to update if execution has percolated up to a higher depth.
1406                    let (db, journal, _) = ecx.as_db_env_and_journal();
1407                    let curr_depth = journal.depth;
1408                    if call_access.depth == curr_depth as u64
1409                        && let Ok(acc) = journal.load_account(db, call.target_address)
1410                    {
1411                        debug_assert!(access_is_call(call_access.kind));
1412                        call_access.newBalance = acc.info.balance;
1413                        call_access.newNonce = acc.info.nonce;
1414                    }
1415                    // Merge the last depth's AccountAccesses into the AccountAccesses at the
1416                    // current depth, or push them back onto the pending
1417                    // vector if higher depths were not recorded. This
1418                    // preserves ordering of accesses.
1419                    if let Some(last) = recorded_account_diffs_stack.last_mut() {
1420                        last.extend(last_recorded_depth);
1421                    } else {
1422                        recorded_account_diffs_stack.push(last_recorded_depth);
1423                    }
1424                }
1425            }
1426        }
1427
1428        // At the end of the call,
1429        // we need to check if we've found all the emits.
1430        // We know we've found all the expected emits in the right order
1431        // if the queue is fully matched.
1432        // If it's not fully matched, then either:
1433        // 1. Not enough events were emitted (we'll know this because the amount of times we
1434        // inspected events will be less than the size of the queue) 2. The wrong events
1435        // were emitted (The inspected events should match the size of the queue, but still some
1436        // events will not be matched)
1437
1438        // First, check that we're at the call depth where the emits were declared from.
1439        let should_check_emits = self
1440            .expected_emits
1441            .iter()
1442            .any(|(expected, _)| {
1443                let curr_depth = ecx.journaled_state.depth();
1444                expected.depth == curr_depth
1445            }) &&
1446            // Ignore staticcalls
1447            !call.is_static;
1448        if should_check_emits {
1449            let expected_counts = self
1450                .expected_emits
1451                .iter()
1452                .filter_map(|(expected, count_map)| {
1453                    let count = match expected.address {
1454                        Some(emitter) => match count_map.get(&emitter) {
1455                            Some(log_count) => expected
1456                                .log
1457                                .as_ref()
1458                                .map(|l| log_count.count(l))
1459                                .unwrap_or_else(|| log_count.count_unchecked()),
1460                            None => 0,
1461                        },
1462                        None => match &expected.log {
1463                            Some(log) => count_map.values().map(|logs| logs.count(log)).sum(),
1464                            None => count_map.values().map(|logs| logs.count_unchecked()).sum(),
1465                        },
1466                    };
1467
1468                    if count != expected.count { Some((expected, count)) } else { None }
1469                })
1470                .collect::<Vec<_>>();
1471
1472            // Revert if not all emits expected were matched.
1473            if let Some((expected, _)) = self
1474                .expected_emits
1475                .iter()
1476                .find(|(expected, _)| !expected.found && expected.count > 0)
1477            {
1478                outcome.result.result = InstructionResult::Revert;
1479                let error_msg = expected.mismatch_error.as_deref().unwrap_or("log != expected log");
1480                outcome.result.output = error_msg.abi_encode().into();
1481                return;
1482            }
1483
1484            if !expected_counts.is_empty() {
1485                let msg = if outcome.result.is_ok() {
1486                    let (expected, count) = expected_counts.first().unwrap();
1487                    format!("log emitted {count} times, expected {}", expected.count)
1488                } else {
1489                    "expected an emit, but the call reverted instead. \
1490                     ensure you're testing the happy path when using `expectEmit`"
1491                        .to_string()
1492                };
1493
1494                outcome.result.result = InstructionResult::Revert;
1495                outcome.result.output = Error::encode(msg);
1496                return;
1497            }
1498
1499            // All emits were found, we're good.
1500            // Clear the queue, as we expect the user to declare more events for the next call
1501            // if they wanna match further events.
1502            self.expected_emits.clear()
1503        }
1504
1505        // this will ensure we don't have false positives when trying to diagnose reverts in fork
1506        // mode
1507        let diag = self.fork_revert_diagnostic.take();
1508
1509        // if there's a revert and a previous call was diagnosed as fork related revert then we can
1510        // return a better error here
1511        if outcome.result.is_revert()
1512            && let Some(err) = diag
1513        {
1514            outcome.result.output = Error::encode(err.to_error_msg(&self.labels));
1515            return;
1516        }
1517
1518        // try to diagnose reverts in multi-fork mode where a call is made to an address that does
1519        // not exist
1520        if let TxKind::Call(test_contract) = ecx.tx.kind {
1521            // if a call to a different contract than the original test contract returned with
1522            // `Stop` we check if the contract actually exists on the active fork
1523            if ecx.journaled_state.db().is_forked_mode()
1524                && outcome.result.result == InstructionResult::Stop
1525                && call.target_address != test_contract
1526            {
1527                let journaled_state = ecx.journaled_state.clone();
1528                self.fork_revert_diagnostic =
1529                    ecx.journaled_state.db().diagnose_revert(call.target_address, &journaled_state);
1530            }
1531        }
1532
1533        // If the depth is 0, then this is the root call terminating
1534        if ecx.journaled_state.depth() == 0 {
1535            // If we already have a revert, we shouldn't run the below logic as it can obfuscate an
1536            // earlier error that happened first with unrelated information about
1537            // another error when using cheatcodes.
1538            if outcome.result.is_revert() {
1539                return;
1540            }
1541
1542            // If there's not a revert, we can continue on to run the last logic for expect*
1543            // cheatcodes.
1544
1545            // Match expected calls
1546            for (address, calldatas) in &self.expected_calls {
1547                // Loop over each address, and for each address, loop over each calldata it expects.
1548                for (calldata, (expected, actual_count)) in calldatas {
1549                    // Grab the values we expect to see
1550                    let ExpectedCallData { gas, min_gas, value, count, call_type } = expected;
1551
1552                    let failed = match call_type {
1553                        // If the cheatcode was called with a `count` argument,
1554                        // we must check that the EVM performed a CALL with this calldata exactly
1555                        // `count` times.
1556                        ExpectedCallType::Count => *count != *actual_count,
1557                        // If the cheatcode was called without a `count` argument,
1558                        // we must check that the EVM performed a CALL with this calldata at least
1559                        // `count` times. The amount of times to check was
1560                        // the amount of time the cheatcode was called.
1561                        ExpectedCallType::NonCount => *count > *actual_count,
1562                    };
1563                    if failed {
1564                        let expected_values = [
1565                            Some(format!("data {}", hex::encode_prefixed(calldata))),
1566                            value.as_ref().map(|v| format!("value {v}")),
1567                            gas.map(|g| format!("gas {g}")),
1568                            min_gas.map(|g| format!("minimum gas {g}")),
1569                        ]
1570                        .into_iter()
1571                        .flatten()
1572                        .join(", ");
1573                        let but = if outcome.result.is_ok() {
1574                            let s = if *actual_count == 1 { "" } else { "s" };
1575                            format!("was called {actual_count} time{s}")
1576                        } else {
1577                            "the call reverted instead; \
1578                             ensure you're testing the happy path when using `expectCall`"
1579                                .to_string()
1580                        };
1581                        let s = if *count == 1 { "" } else { "s" };
1582                        let msg = format!(
1583                            "expected call to {address} with {expected_values} \
1584                             to be called {count} time{s}, but {but}"
1585                        );
1586                        outcome.result.result = InstructionResult::Revert;
1587                        outcome.result.output = Error::encode(msg);
1588
1589                        return;
1590                    }
1591                }
1592            }
1593
1594            // Check if we have any leftover expected emits
1595            // First, if any emits were found at the root call, then we its ok and we remove them.
1596            // For count=0 expectations, NOT being found is success, so mark them as found
1597            for (expected, _) in &mut self.expected_emits {
1598                if expected.count == 0 && !expected.found {
1599                    expected.found = true;
1600                }
1601            }
1602            self.expected_emits.retain(|(expected, _)| !expected.found);
1603            // If not empty, we got mismatched emits
1604            if !self.expected_emits.is_empty() {
1605                let msg = if outcome.result.is_ok() {
1606                    "expected an emit, but no logs were emitted afterwards. \
1607                     you might have mismatched events or not enough events were emitted"
1608                } else {
1609                    "expected an emit, but the call reverted instead. \
1610                     ensure you're testing the happy path when using `expectEmit`"
1611                };
1612                outcome.result.result = InstructionResult::Revert;
1613                outcome.result.output = Error::encode(msg);
1614                return;
1615            }
1616
1617            // Check for leftover expected creates
1618            if let Some(expected_create) = self.expected_creates.first() {
1619                let msg = format!(
1620                    "expected {} call by address {} for bytecode {} but not found",
1621                    expected_create.create_scheme,
1622                    hex::encode_prefixed(expected_create.deployer),
1623                    hex::encode_prefixed(&expected_create.bytecode),
1624                );
1625                outcome.result.result = InstructionResult::Revert;
1626                outcome.result.output = Error::encode(msg);
1627            }
1628        }
1629    }
1630
1631    fn create(&mut self, ecx: Ecx, mut input: &mut CreateInputs) -> Option<CreateOutcome> {
1632        // Apply custom execution evm version.
1633        if let Some(spec_id) = self.execution_evm_version {
1634            ecx.cfg.spec = spec_id;
1635        }
1636
1637        let gas = Gas::new(input.gas_limit());
1638        // Check if we should intercept this create
1639        if self.intercept_next_create_call {
1640            // Reset the flag
1641            self.intercept_next_create_call = false;
1642
1643            // Get initcode from the input
1644            let output = input.init_code();
1645
1646            // Return a revert with the initcode as error data
1647            return Some(CreateOutcome {
1648                result: InterpreterResult { result: InstructionResult::Revert, output, gas },
1649                address: None,
1650            });
1651        }
1652
1653        let curr_depth = ecx.journaled_state.depth();
1654
1655        // Apply our prank
1656        if let Some(prank) = &self.get_prank(curr_depth)
1657            && curr_depth >= prank.depth
1658            && input.caller() == prank.prank_caller
1659        {
1660            let mut prank_applied = false;
1661
1662            // At the target depth we set `msg.sender`
1663            if curr_depth == prank.depth {
1664                // Ensure new caller is loaded and touched
1665                let _ = journaled_account(ecx, prank.new_caller);
1666                input.set_caller(prank.new_caller);
1667                prank_applied = true;
1668            }
1669
1670            // At the target depth, or deeper, we set `tx.origin`
1671            if let Some(new_origin) = prank.new_origin {
1672                ecx.tx.caller = new_origin;
1673                prank_applied = true;
1674            }
1675
1676            // If prank applied for first time, then update
1677            if prank_applied && let Some(applied_prank) = prank.first_time_applied() {
1678                self.pranks.insert(curr_depth, applied_prank);
1679            }
1680        }
1681
1682        // Apply EIP-2930 access list
1683        self.apply_accesslist(ecx);
1684
1685        // Apply our broadcast
1686        if let Some(broadcast) = &mut self.broadcast
1687            && curr_depth >= broadcast.depth
1688            && input.caller() == broadcast.original_caller
1689        {
1690            let (db, journal, _) = ecx.as_db_env_and_journal();
1691            if let Err(err) = journal.load_account(db, broadcast.new_origin) {
1692                return Some(CreateOutcome {
1693                    result: InterpreterResult {
1694                        result: InstructionResult::Revert,
1695                        output: Error::encode(err),
1696                        gas,
1697                    },
1698                    address: None,
1699                });
1700            }
1701
1702            ecx.tx.caller = broadcast.new_origin;
1703
1704            if curr_depth == broadcast.depth || broadcast.deploy_from_code {
1705                // Reset deploy from code flag for upcoming calls;
1706                broadcast.deploy_from_code = false;
1707
1708                input.set_caller(broadcast.new_origin);
1709
1710                let account = &ecx.journaled_state.inner.state()[&broadcast.new_origin];
1711                self.broadcastable_transactions.push_back(BroadcastableTransaction {
1712                    rpc: ecx.journaled_state.database.active_fork_url(),
1713                    transaction: TransactionRequest {
1714                        from: Some(broadcast.new_origin),
1715                        to: None,
1716                        value: Some(input.value()),
1717                        input: TransactionInput::new(input.init_code()),
1718                        nonce: Some(account.info.nonce),
1719                        ..Default::default()
1720                    }
1721                    .into(),
1722                });
1723
1724                input.log_debug(self, &input.scheme().unwrap_or(CreateScheme::Create));
1725            }
1726        }
1727
1728        // Allow cheatcodes from the address of the new contract
1729        let address = input.allow_cheatcodes(self, ecx);
1730
1731        // If `recordAccountAccesses` has been called, record the create
1732        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
1733            recorded_account_diffs_stack.push(vec![AccountAccess {
1734                chainInfo: crate::Vm::ChainInfo {
1735                    forkId: ecx.journaled_state.db().active_fork_id().unwrap_or_default(),
1736                    chainId: U256::from(ecx.cfg.chain_id),
1737                },
1738                accessor: input.caller(),
1739                account: address,
1740                kind: crate::Vm::AccountAccessKind::Create,
1741                initialized: true,
1742                oldBalance: U256::ZERO, // updated on create_end
1743                newBalance: U256::ZERO, // updated on create_end
1744                oldNonce: 0,            // new contract starts with nonce 0
1745                newNonce: 1,            // updated on create_end (contracts start with nonce 1)
1746                value: input.value(),
1747                data: input.init_code(),
1748                reverted: false,
1749                deployedCode: Bytes::new(), // updated on create_end
1750                storageAccesses: vec![],    // updated on create_end
1751                depth: curr_depth as u64,
1752            }]);
1753        }
1754
1755        None
1756    }
1757
1758    fn create_end(&mut self, ecx: Ecx, call: &CreateInputs, outcome: &mut CreateOutcome) {
1759        let call = Some(call);
1760        let curr_depth = ecx.journaled_state.depth();
1761
1762        // Clean up pranks
1763        if let Some(prank) = &self.get_prank(curr_depth)
1764            && curr_depth == prank.depth
1765        {
1766            ecx.tx.caller = prank.prank_origin;
1767
1768            // Clean single-call prank once we have returned to the original depth
1769            if prank.single_call {
1770                std::mem::take(&mut self.pranks);
1771            }
1772        }
1773
1774        // Clean up broadcasts
1775        if let Some(broadcast) = &self.broadcast
1776            && curr_depth == broadcast.depth
1777        {
1778            ecx.tx.caller = broadcast.original_origin;
1779
1780            // Clean single-call broadcast once we have returned to the original depth
1781            if broadcast.single_call {
1782                std::mem::take(&mut self.broadcast);
1783            }
1784        }
1785
1786        // Handle expected reverts
1787        if let Some(expected_revert) = &self.expected_revert
1788            && curr_depth <= expected_revert.depth
1789            && matches!(expected_revert.kind, ExpectedRevertKind::Default)
1790        {
1791            let mut expected_revert = std::mem::take(&mut self.expected_revert).unwrap();
1792            return match revert_handlers::handle_expect_revert(
1793                false,
1794                true,
1795                self.config.internal_expect_revert,
1796                &expected_revert,
1797                outcome.result.result,
1798                outcome.result.output.clone(),
1799                &self.config.available_artifacts,
1800            ) {
1801                Ok((address, retdata)) => {
1802                    expected_revert.actual_count += 1;
1803                    if expected_revert.actual_count < expected_revert.count {
1804                        self.expected_revert = Some(expected_revert.clone());
1805                    }
1806
1807                    outcome.result.result = InstructionResult::Return;
1808                    outcome.result.output = retdata;
1809                    outcome.address = address;
1810                }
1811                Err(err) => {
1812                    outcome.result.result = InstructionResult::Revert;
1813                    outcome.result.output = err.abi_encode().into();
1814                }
1815            };
1816        }
1817
1818        // If `startStateDiffRecording` has been called, update the `reverted` status of the
1819        // previous call depth's recorded accesses, if any
1820        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
1821            // The root call cannot be recorded.
1822            if curr_depth > 0
1823                && let Some(last_depth) = &mut recorded_account_diffs_stack.pop()
1824            {
1825                // Update the reverted status of all deeper calls if this call reverted, in
1826                // accordance with EVM behavior
1827                if outcome.result.is_revert() {
1828                    last_depth.iter_mut().for_each(|element| {
1829                        element.reverted = true;
1830                        element
1831                            .storageAccesses
1832                            .iter_mut()
1833                            .for_each(|storage_access| storage_access.reverted = true);
1834                    })
1835                }
1836
1837                if let Some(create_access) = last_depth.first_mut() {
1838                    // Assert that we're at the correct depth before recording post-create state
1839                    // changes. Depending on what depth the cheat was called at, there
1840                    // may not be any pending calls to update if execution has
1841                    // percolated up to a higher depth.
1842                    let depth = ecx.journaled_state.depth();
1843                    if create_access.depth == depth as u64 {
1844                        debug_assert_eq!(
1845                            create_access.kind as u8,
1846                            crate::Vm::AccountAccessKind::Create as u8
1847                        );
1848                        let (db, journal, _) = ecx.as_db_env_and_journal();
1849                        if let Some(address) = outcome.address
1850                            && let Ok(created_acc) = journal.load_account(db, address)
1851                        {
1852                            create_access.newBalance = created_acc.info.balance;
1853                            create_access.newNonce = created_acc.info.nonce;
1854                            create_access.deployedCode =
1855                                created_acc.info.code.clone().unwrap_or_default().original_bytes();
1856                        }
1857                    }
1858                    // Merge the last depth's AccountAccesses into the AccountAccesses at the
1859                    // current depth, or push them back onto the pending
1860                    // vector if higher depths were not recorded. This
1861                    // preserves ordering of accesses.
1862                    if let Some(last) = recorded_account_diffs_stack.last_mut() {
1863                        last.append(last_depth);
1864                    } else {
1865                        recorded_account_diffs_stack.push(last_depth.clone());
1866                    }
1867                }
1868            }
1869        }
1870
1871        // Match the create against expected_creates
1872        let (db, journal, _) = ecx.as_db_env_and_journal();
1873        if !self.expected_creates.is_empty()
1874            && let (Some(address), Some(call)) = (outcome.address, call)
1875            && let Ok(created_acc) = journal.load_account(db, address)
1876        {
1877            let bytecode = created_acc.info.code.clone().unwrap_or_default().original_bytes();
1878            if let Some((index, _)) =
1879                self.expected_creates.iter().find_position(|expected_create| {
1880                    expected_create.deployer == call.caller
1881                        && expected_create.create_scheme.eq(call.scheme.into())
1882                        && expected_create.bytecode == bytecode
1883                })
1884            {
1885                self.expected_creates.swap_remove(index);
1886            }
1887        }
1888    }
1889}
1890
1891impl InspectorExt for Cheatcodes {
1892    fn should_use_create2_factory(&mut self, ecx: Ecx, inputs: &CreateInputs) -> bool {
1893        if let CreateScheme::Create2 { .. } = inputs.scheme {
1894            let depth = ecx.journaled_state.depth();
1895            let target_depth = if let Some(prank) = &self.get_prank(depth) {
1896                prank.depth
1897            } else if let Some(broadcast) = &self.broadcast {
1898                broadcast.depth
1899            } else {
1900                1
1901            };
1902
1903            depth == target_depth
1904                && (self.broadcast.is_some() || self.config.always_use_create_2_factory)
1905        } else {
1906            false
1907        }
1908    }
1909
1910    fn create2_deployer(&self) -> Address {
1911        self.config.evm_opts.create2_deployer
1912    }
1913}
1914
1915impl Cheatcodes {
1916    #[cold]
1917    fn meter_gas(&mut self, interpreter: &mut Interpreter) {
1918        if let Some(paused_gas) = self.gas_metering.paused_frames.last() {
1919            // Keep gas constant if paused.
1920            // Make sure we record the memory changes so that memory expansion is not paused.
1921            let memory = *interpreter.gas.memory();
1922            interpreter.gas = *paused_gas;
1923            interpreter.gas.memory_mut().words_num = memory.words_num;
1924            interpreter.gas.memory_mut().expansion_cost = memory.expansion_cost;
1925        } else {
1926            // Record frame paused gas.
1927            self.gas_metering.paused_frames.push(interpreter.gas);
1928        }
1929    }
1930
1931    #[cold]
1932    fn meter_gas_record(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1933        if interpreter.bytecode.action.as_ref().and_then(|i| i.instruction_result()).is_none() {
1934            self.gas_metering.gas_records.iter_mut().for_each(|record| {
1935                let curr_depth = ecx.journaled_state.depth();
1936                if curr_depth == record.depth {
1937                    // Skip the first opcode of the first call frame as it includes the gas cost of
1938                    // creating the snapshot.
1939                    if self.gas_metering.last_gas_used != 0 {
1940                        let gas_diff =
1941                            interpreter.gas.spent().saturating_sub(self.gas_metering.last_gas_used);
1942                        record.gas_used = record.gas_used.saturating_add(gas_diff);
1943                    }
1944
1945                    // Update `last_gas_used` to the current spent gas for the next iteration to
1946                    // compare against.
1947                    self.gas_metering.last_gas_used = interpreter.gas.spent();
1948                }
1949            });
1950        }
1951    }
1952
1953    #[cold]
1954    fn meter_gas_end(&mut self, interpreter: &mut Interpreter) {
1955        // Remove recorded gas if we exit frame.
1956        if let Some(interpreter_action) = interpreter.bytecode.action.as_ref()
1957            && will_exit(interpreter_action)
1958        {
1959            self.gas_metering.paused_frames.pop();
1960        }
1961    }
1962
1963    #[cold]
1964    fn meter_gas_reset(&mut self, interpreter: &mut Interpreter) {
1965        let mut gas = Gas::new(interpreter.gas.limit());
1966        gas.memory_mut().words_num = interpreter.gas.memory().words_num;
1967        gas.memory_mut().expansion_cost = interpreter.gas.memory().expansion_cost;
1968        interpreter.gas = gas;
1969        self.gas_metering.reset = false;
1970    }
1971
1972    #[cold]
1973    fn meter_gas_check(&mut self, interpreter: &mut Interpreter) {
1974        if let Some(interpreter_action) = interpreter.bytecode.action.as_ref()
1975            && will_exit(interpreter_action)
1976        {
1977            // Reset gas if spent is less than refunded.
1978            // This can happen if gas was paused / resumed or reset.
1979            // https://github.com/foundry-rs/foundry/issues/4370
1980            if interpreter.gas.spent()
1981                < u64::try_from(interpreter.gas.refunded()).unwrap_or_default()
1982            {
1983                interpreter.gas = Gas::new(interpreter.gas.limit());
1984            }
1985        }
1986    }
1987
1988    /// Generates or copies arbitrary values for storage slots.
1989    /// Invoked in inspector `step_end` (when the current opcode is not executed), if current opcode
1990    /// to execute is `SLOAD` and storage slot is cold.
1991    /// Ensures that in next step (when `SLOAD` opcode is executed) an arbitrary value is returned:
1992    /// - copies the existing arbitrary storage value (or the new generated one if no value in
1993    ///   cache) from mapped source address to the target address.
1994    /// - generates arbitrary value and saves it in target address storage.
1995    #[cold]
1996    fn arbitrary_storage_end(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1997        let (key, target_address) = if interpreter.bytecode.opcode() == op::SLOAD {
1998            (try_or_return!(interpreter.stack.peek(0)), interpreter.input.target_address)
1999        } else {
2000            return;
2001        };
2002
2003        let Some(value) = ecx.sload(target_address, key) else {
2004            return;
2005        };
2006
2007        if (value.is_cold && value.data.is_zero())
2008            || self.should_overwrite_arbitrary_storage(&target_address, key)
2009        {
2010            if self.has_arbitrary_storage(&target_address) {
2011                let arbitrary_value = self.rng().random();
2012                self.arbitrary_storage.as_mut().unwrap().save(
2013                    ecx,
2014                    target_address,
2015                    key,
2016                    arbitrary_value,
2017                );
2018            } else if self.is_arbitrary_storage_copy(&target_address) {
2019                let arbitrary_value = self.rng().random();
2020                self.arbitrary_storage.as_mut().unwrap().copy(
2021                    ecx,
2022                    target_address,
2023                    key,
2024                    arbitrary_value,
2025                );
2026            }
2027        }
2028    }
2029
2030    /// Records storage slots reads and writes.
2031    #[cold]
2032    fn record_accesses(&mut self, interpreter: &mut Interpreter) {
2033        let access = &mut self.accesses;
2034        match interpreter.bytecode.opcode() {
2035            op::SLOAD => {
2036                let key = try_or_return!(interpreter.stack.peek(0));
2037                access.record_read(interpreter.input.target_address, key);
2038            }
2039            op::SSTORE => {
2040                let key = try_or_return!(interpreter.stack.peek(0));
2041                access.record_write(interpreter.input.target_address, key);
2042            }
2043            _ => {}
2044        }
2045    }
2046
2047    #[cold]
2048    fn record_state_diffs(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
2049        let Some(account_accesses) = &mut self.recorded_account_diffs_stack else { return };
2050        match interpreter.bytecode.opcode() {
2051            op::SELFDESTRUCT => {
2052                // Ensure that we're not selfdestructing a context recording was initiated on
2053                let Some(last) = account_accesses.last_mut() else { return };
2054
2055                // get previous balance, nonce and initialized status of the target account
2056                let target = try_or_return!(interpreter.stack.peek(0));
2057                let target = Address::from_word(B256::from(target));
2058                let (db, journal, _) = ecx.as_db_env_and_journal();
2059                let (initialized, old_balance, old_nonce) = journal
2060                    .load_account(db, target)
2061                    .map(|account| {
2062                        (account.info.exists(), account.info.balance, account.info.nonce)
2063                    })
2064                    .unwrap_or_default();
2065
2066                // load balance of this account
2067                let value = ecx
2068                    .balance(interpreter.input.target_address)
2069                    .map(|b| b.data)
2070                    .unwrap_or(U256::ZERO);
2071
2072                // register access for the target account
2073                last.push(crate::Vm::AccountAccess {
2074                    chainInfo: crate::Vm::ChainInfo {
2075                        forkId: ecx.journaled_state.database.active_fork_id().unwrap_or_default(),
2076                        chainId: U256::from(ecx.cfg.chain_id),
2077                    },
2078                    accessor: interpreter.input.target_address,
2079                    account: target,
2080                    kind: crate::Vm::AccountAccessKind::SelfDestruct,
2081                    initialized,
2082                    oldBalance: old_balance,
2083                    newBalance: old_balance + value,
2084                    oldNonce: old_nonce,
2085                    newNonce: old_nonce, // nonce doesn't change on selfdestruct
2086                    value,
2087                    data: Bytes::new(),
2088                    reverted: false,
2089                    deployedCode: Bytes::new(),
2090                    storageAccesses: vec![],
2091                    depth: ecx
2092                        .journaled_state
2093                        .depth()
2094                        .try_into()
2095                        .expect("journaled state depth exceeds u64"),
2096                });
2097            }
2098
2099            op::SLOAD => {
2100                let Some(last) = account_accesses.last_mut() else { return };
2101
2102                let key = try_or_return!(interpreter.stack.peek(0));
2103                let address = interpreter.input.target_address;
2104
2105                // Try to include present value for informational purposes, otherwise assume
2106                // it's not set (zero value)
2107                let mut present_value = U256::ZERO;
2108                // Try to load the account and the slot's present value
2109                let (db, journal, _) = ecx.as_db_env_and_journal();
2110                if journal.load_account(db, address).is_ok()
2111                    && let Some(previous) = ecx.sload(address, key)
2112                {
2113                    present_value = previous.data;
2114                }
2115                let access = crate::Vm::StorageAccess {
2116                    account: interpreter.input.target_address,
2117                    slot: key.into(),
2118                    isWrite: false,
2119                    previousValue: present_value.into(),
2120                    newValue: present_value.into(),
2121                    reverted: false,
2122                };
2123                let curr_depth = ecx
2124                    .journaled_state
2125                    .depth()
2126                    .try_into()
2127                    .expect("journaled state depth exceeds u64");
2128                append_storage_access(last, access, curr_depth);
2129            }
2130            op::SSTORE => {
2131                let Some(last) = account_accesses.last_mut() else { return };
2132
2133                let key = try_or_return!(interpreter.stack.peek(0));
2134                let value = try_or_return!(interpreter.stack.peek(1));
2135                let address = interpreter.input.target_address;
2136                // Try to load the account and the slot's previous value, otherwise, assume it's
2137                // not set (zero value)
2138                let mut previous_value = U256::ZERO;
2139                let (db, journal, _) = ecx.as_db_env_and_journal();
2140                if journal.load_account(db, address).is_ok()
2141                    && let Some(previous) = ecx.sload(address, key)
2142                {
2143                    previous_value = previous.data;
2144                }
2145
2146                let access = crate::Vm::StorageAccess {
2147                    account: address,
2148                    slot: key.into(),
2149                    isWrite: true,
2150                    previousValue: previous_value.into(),
2151                    newValue: value.into(),
2152                    reverted: false,
2153                };
2154                let curr_depth = ecx
2155                    .journaled_state
2156                    .depth()
2157                    .try_into()
2158                    .expect("journaled state depth exceeds u64");
2159                append_storage_access(last, access, curr_depth);
2160            }
2161
2162            // Record account accesses via the EXT family of opcodes
2163            op::EXTCODECOPY | op::EXTCODESIZE | op::EXTCODEHASH | op::BALANCE => {
2164                let kind = match interpreter.bytecode.opcode() {
2165                    op::EXTCODECOPY => crate::Vm::AccountAccessKind::Extcodecopy,
2166                    op::EXTCODESIZE => crate::Vm::AccountAccessKind::Extcodesize,
2167                    op::EXTCODEHASH => crate::Vm::AccountAccessKind::Extcodehash,
2168                    op::BALANCE => crate::Vm::AccountAccessKind::Balance,
2169                    _ => unreachable!(),
2170                };
2171                let address =
2172                    Address::from_word(B256::from(try_or_return!(interpreter.stack.peek(0))));
2173                let initialized;
2174                let balance;
2175                let nonce;
2176                let (db, journal, _) = ecx.as_db_env_and_journal();
2177                if let Ok(acc) = journal.load_account(db, address) {
2178                    initialized = acc.info.exists();
2179                    balance = acc.info.balance;
2180                    nonce = acc.info.nonce;
2181                } else {
2182                    initialized = false;
2183                    balance = U256::ZERO;
2184                    nonce = 0;
2185                }
2186                let curr_depth = ecx
2187                    .journaled_state
2188                    .depth()
2189                    .try_into()
2190                    .expect("journaled state depth exceeds u64");
2191                let account_access = crate::Vm::AccountAccess {
2192                    chainInfo: crate::Vm::ChainInfo {
2193                        forkId: ecx.journaled_state.database.active_fork_id().unwrap_or_default(),
2194                        chainId: U256::from(ecx.cfg.chain_id),
2195                    },
2196                    accessor: interpreter.input.target_address,
2197                    account: address,
2198                    kind,
2199                    initialized,
2200                    oldBalance: balance,
2201                    newBalance: balance,
2202                    oldNonce: nonce,
2203                    newNonce: nonce, // EXT* operations don't change nonce
2204                    value: U256::ZERO,
2205                    data: Bytes::new(),
2206                    reverted: false,
2207                    deployedCode: Bytes::new(),
2208                    storageAccesses: vec![],
2209                    depth: curr_depth,
2210                };
2211                // Record the EXT* call as an account access at the current depth
2212                // (future storage accesses will be recorded in a new "Resume" context)
2213                if let Some(last) = account_accesses.last_mut() {
2214                    last.push(account_access);
2215                } else {
2216                    account_accesses.push(vec![account_access]);
2217                }
2218            }
2219            _ => {}
2220        }
2221    }
2222
2223    /// Checks to see if the current opcode can either mutate directly or expand memory.
2224    ///
2225    /// If the opcode at the current program counter is a match, check if the modified memory lies
2226    /// within the allowed ranges. If not, revert and fail the test.
2227    #[cold]
2228    fn check_mem_opcodes(&self, interpreter: &mut Interpreter, depth: u64) {
2229        let Some(ranges) = self.allowed_mem_writes.get(&depth) else {
2230            return;
2231        };
2232
2233        // The `mem_opcode_match` macro is used to match the current opcode against a list of
2234        // opcodes that can mutate memory (either directly or expansion via reading). If the
2235        // opcode is a match, the memory offsets that are being written to are checked to be
2236        // within the allowed ranges. If not, the test is failed and the transaction is
2237        // reverted. For all opcodes that can mutate memory aside from MSTORE,
2238        // MSTORE8, and MLOAD, the size and destination offset are on the stack, and
2239        // the macro expands all of these cases. For MSTORE, MSTORE8, and MLOAD, the
2240        // size of the memory write is implicit, so these cases are hard-coded.
2241        macro_rules! mem_opcode_match {
2242            ($(($opcode:ident, $offset_depth:expr, $size_depth:expr, $writes:expr)),* $(,)?) => {
2243                match interpreter.bytecode.opcode() {
2244                    ////////////////////////////////////////////////////////////////
2245                    //    OPERATIONS THAT CAN EXPAND/MUTATE MEMORY BY WRITING     //
2246                    ////////////////////////////////////////////////////////////////
2247
2248                    op::MSTORE => {
2249                        // The offset of the mstore operation is at the top of the stack.
2250                        let offset = try_or_return!(interpreter.stack.peek(0)).saturating_to::<u64>();
2251
2252                        // If none of the allowed ranges contain [offset, offset + 32), memory has been
2253                        // unexpectedly mutated.
2254                        if !ranges.iter().any(|range| {
2255                            range.contains(&offset) && range.contains(&(offset + 31))
2256                        }) {
2257                            // SPECIAL CASE: When the compiler attempts to store the selector for
2258                            // `stopExpectSafeMemory`, this is allowed. It will do so at the current free memory
2259                            // pointer, which could have been updated to the exclusive upper bound during
2260                            // execution.
2261                            let value = try_or_return!(interpreter.stack.peek(1)).to_be_bytes::<32>();
2262                            if value[..SELECTOR_LEN] == stopExpectSafeMemoryCall::SELECTOR {
2263                                return
2264                            }
2265
2266                            disallowed_mem_write(offset, 32, interpreter, ranges);
2267                            return
2268                        }
2269                    }
2270                    op::MSTORE8 => {
2271                        // The offset of the mstore8 operation is at the top of the stack.
2272                        let offset = try_or_return!(interpreter.stack.peek(0)).saturating_to::<u64>();
2273
2274                        // If none of the allowed ranges contain the offset, memory has been
2275                        // unexpectedly mutated.
2276                        if !ranges.iter().any(|range| range.contains(&offset)) {
2277                            disallowed_mem_write(offset, 1, interpreter, ranges);
2278                            return
2279                        }
2280                    }
2281
2282                    ////////////////////////////////////////////////////////////////
2283                    //        OPERATIONS THAT CAN EXPAND MEMORY BY READING        //
2284                    ////////////////////////////////////////////////////////////////
2285
2286                    op::MLOAD => {
2287                        // The offset of the mload operation is at the top of the stack
2288                        let offset = try_or_return!(interpreter.stack.peek(0)).saturating_to::<u64>();
2289
2290                        // If the offset being loaded is >= than the memory size, the
2291                        // memory is being expanded. If none of the allowed ranges contain
2292                        // [offset, offset + 32), memory has been unexpectedly mutated.
2293                        if offset >= interpreter.memory.size() as u64 && !ranges.iter().any(|range| {
2294                            range.contains(&offset) && range.contains(&(offset + 31))
2295                        }) {
2296                            disallowed_mem_write(offset, 32, interpreter, ranges);
2297                            return
2298                        }
2299                    }
2300
2301                    ////////////////////////////////////////////////////////////////
2302                    //          OPERATIONS WITH OFFSET AND SIZE ON STACK          //
2303                    ////////////////////////////////////////////////////////////////
2304
2305                    op::CALL => {
2306                        // The destination offset of the operation is the fifth element on the stack.
2307                        let dest_offset = try_or_return!(interpreter.stack.peek(5)).saturating_to::<u64>();
2308
2309                        // The size of the data that will be copied is the sixth element on the stack.
2310                        let size = try_or_return!(interpreter.stack.peek(6)).saturating_to::<u64>();
2311
2312                        // If none of the allowed ranges contain [dest_offset, dest_offset + size),
2313                        // memory outside of the expected ranges has been touched. If the opcode
2314                        // only reads from memory, this is okay as long as the memory is not expanded.
2315                        let fail_cond = !ranges.iter().any(|range| {
2316                            range.contains(&dest_offset) &&
2317                                range.contains(&(dest_offset + size.saturating_sub(1)))
2318                        });
2319
2320                        // If the failure condition is met, set the output buffer to a revert string
2321                        // that gives information about the allowed ranges and revert.
2322                        if fail_cond {
2323                            // SPECIAL CASE: When a call to `stopExpectSafeMemory` is performed, this is allowed.
2324                            // It allocated calldata at the current free memory pointer, and will attempt to read
2325                            // from this memory region to perform the call.
2326                            let to = Address::from_word(try_or_return!(interpreter.stack.peek(1)).to_be_bytes::<32>().into());
2327                            if to == CHEATCODE_ADDRESS {
2328                                let args_offset = try_or_return!(interpreter.stack.peek(3)).saturating_to::<usize>();
2329                                let args_size = try_or_return!(interpreter.stack.peek(4)).saturating_to::<usize>();
2330                                let memory_word = interpreter.memory.slice_len(args_offset, args_size);
2331                                if memory_word[..SELECTOR_LEN] == stopExpectSafeMemoryCall::SELECTOR {
2332                                    return
2333                                }
2334                            }
2335
2336                            disallowed_mem_write(dest_offset, size, interpreter, ranges);
2337                            return
2338                        }
2339                    }
2340
2341                    $(op::$opcode => {
2342                        // The destination offset of the operation.
2343                        let dest_offset = try_or_return!(interpreter.stack.peek($offset_depth)).saturating_to::<u64>();
2344
2345                        // The size of the data that will be copied.
2346                        let size = try_or_return!(interpreter.stack.peek($size_depth)).saturating_to::<u64>();
2347
2348                        // If none of the allowed ranges contain [dest_offset, dest_offset + size),
2349                        // memory outside of the expected ranges has been touched. If the opcode
2350                        // only reads from memory, this is okay as long as the memory is not expanded.
2351                        let fail_cond = !ranges.iter().any(|range| {
2352                                range.contains(&dest_offset) &&
2353                                    range.contains(&(dest_offset + size.saturating_sub(1)))
2354                            }) && ($writes ||
2355                                [dest_offset, (dest_offset + size).saturating_sub(1)].into_iter().any(|offset| {
2356                                    offset >= interpreter.memory.size() as u64
2357                                })
2358                            );
2359
2360                        // If the failure condition is met, set the output buffer to a revert string
2361                        // that gives information about the allowed ranges and revert.
2362                        if fail_cond {
2363                            disallowed_mem_write(dest_offset, size, interpreter, ranges);
2364                            return
2365                        }
2366                    })*
2367
2368                    _ => {}
2369                }
2370            }
2371        }
2372
2373        // Check if the current opcode can write to memory, and if so, check if the memory
2374        // being written to is registered as safe to modify.
2375        mem_opcode_match!(
2376            (CALLDATACOPY, 0, 2, true),
2377            (CODECOPY, 0, 2, true),
2378            (RETURNDATACOPY, 0, 2, true),
2379            (EXTCODECOPY, 1, 3, true),
2380            (CALLCODE, 5, 6, true),
2381            (STATICCALL, 4, 5, true),
2382            (DELEGATECALL, 4, 5, true),
2383            (KECCAK256, 0, 1, false),
2384            (LOG0, 0, 1, false),
2385            (LOG1, 0, 1, false),
2386            (LOG2, 0, 1, false),
2387            (LOG3, 0, 1, false),
2388            (LOG4, 0, 1, false),
2389            (CREATE, 1, 2, false),
2390            (CREATE2, 1, 2, false),
2391            (RETURN, 0, 1, false),
2392            (REVERT, 0, 1, false),
2393        );
2394    }
2395
2396    #[cold]
2397    fn set_gas_limit_type(&mut self, interpreter: &mut Interpreter) {
2398        match interpreter.bytecode.opcode() {
2399            op::CREATE2 => self.dynamic_gas_limit = true,
2400            op::CALL => {
2401                // If first element of the stack is close to current remaining gas then assume
2402                // dynamic gas limit.
2403                self.dynamic_gas_limit =
2404                    try_or_return!(interpreter.stack.peek(0)) >= interpreter.gas.remaining() - 100
2405            }
2406            _ => self.dynamic_gas_limit = false,
2407        }
2408    }
2409}
2410
2411/// Helper that expands memory, stores a revert string pertaining to a disallowed memory write,
2412/// and sets the return range to the revert string's location in memory.
2413///
2414/// This will set the interpreter's next action to a return with the revert string as the output.
2415/// And trigger a revert.
2416fn disallowed_mem_write(
2417    dest_offset: u64,
2418    size: u64,
2419    interpreter: &mut Interpreter,
2420    ranges: &[Range<u64>],
2421) {
2422    let revert_string = format!(
2423        "memory write at offset 0x{:02X} of size 0x{:02X} not allowed; safe range: {}",
2424        dest_offset,
2425        size,
2426        ranges.iter().map(|r| format!("(0x{:02X}, 0x{:02X}]", r.start, r.end)).join(" U ")
2427    );
2428
2429    interpreter.bytecode.set_action(InterpreterAction::new_return(
2430        InstructionResult::Revert,
2431        Bytes::from(revert_string.into_bytes()),
2432        interpreter.gas,
2433    ));
2434}
2435
2436/// Returns true if the kind of account access is a call.
2437fn access_is_call(kind: crate::Vm::AccountAccessKind) -> bool {
2438    matches!(
2439        kind,
2440        crate::Vm::AccountAccessKind::Call
2441            | crate::Vm::AccountAccessKind::StaticCall
2442            | crate::Vm::AccountAccessKind::CallCode
2443            | crate::Vm::AccountAccessKind::DelegateCall
2444    )
2445}
2446
2447/// Records a log into the recorded logs vector, if it exists.
2448fn record_logs(recorded_logs: &mut Option<Vec<Vm::Log>>, log: &Log) {
2449    if let Some(storage_recorded_logs) = recorded_logs {
2450        storage_recorded_logs.push(Vm::Log {
2451            topics: log.data.topics().to_vec(),
2452            data: log.data.data.clone(),
2453            emitter: log.address,
2454        });
2455    }
2456}
2457
2458/// Appends an AccountAccess that resumes the recording of the current context.
2459fn append_storage_access(
2460    last: &mut Vec<AccountAccess>,
2461    storage_access: crate::Vm::StorageAccess,
2462    storage_depth: u64,
2463) {
2464    // Assert that there's an existing record for the current context.
2465    if !last.is_empty() && last.first().unwrap().depth < storage_depth {
2466        // Three cases to consider:
2467        // 1. If there hasn't been a context switch since the start of this context, then add the
2468        //    storage access to the current context record.
2469        // 2. If there's an existing Resume record, then add the storage access to it.
2470        // 3. Otherwise, create a new Resume record based on the current context.
2471        if last.len() == 1 {
2472            last.first_mut().unwrap().storageAccesses.push(storage_access);
2473        } else {
2474            let last_record = last.last_mut().unwrap();
2475            if last_record.kind as u8 == crate::Vm::AccountAccessKind::Resume as u8 {
2476                last_record.storageAccesses.push(storage_access);
2477            } else {
2478                let entry = last.first().unwrap();
2479                let resume_record = crate::Vm::AccountAccess {
2480                    chainInfo: crate::Vm::ChainInfo {
2481                        forkId: entry.chainInfo.forkId,
2482                        chainId: entry.chainInfo.chainId,
2483                    },
2484                    accessor: entry.accessor,
2485                    account: entry.account,
2486                    kind: crate::Vm::AccountAccessKind::Resume,
2487                    initialized: entry.initialized,
2488                    storageAccesses: vec![storage_access],
2489                    reverted: entry.reverted,
2490                    // The remaining fields are defaults
2491                    oldBalance: U256::ZERO,
2492                    newBalance: U256::ZERO,
2493                    oldNonce: 0,
2494                    newNonce: 0,
2495                    value: U256::ZERO,
2496                    data: Bytes::new(),
2497                    deployedCode: Bytes::new(),
2498                    depth: entry.depth,
2499                };
2500                last.push(resume_record);
2501            }
2502        }
2503    }
2504}
2505
2506/// Dispatches the cheatcode call to the appropriate function.
2507fn apply_dispatch(
2508    calls: &Vm::VmCalls,
2509    ccx: &mut CheatsCtxt,
2510    executor: &mut dyn CheatcodesExecutor,
2511) -> Result {
2512    let cheat = calls_as_dyn_cheatcode(calls);
2513
2514    let _guard = debug_span!(target: "cheatcodes", "apply", id = %cheat.id()).entered();
2515    trace!(target: "cheatcodes", ?cheat, "applying");
2516
2517    if let spec::Status::Deprecated(replacement) = *cheat.status() {
2518        ccx.state.deprecated.insert(cheat.signature(), replacement);
2519    }
2520
2521    // Apply the cheatcode.
2522    let mut result = cheat.dyn_apply(ccx, executor);
2523
2524    // Format the error message to include the cheatcode name.
2525    if let Err(e) = &mut result
2526        && e.is_str()
2527    {
2528        let name = cheat.name();
2529        // Skip showing the cheatcode name for:
2530        // - assertions: too verbose, and can already be inferred from the error message
2531        // - `rpcUrl`: forge-std relies on it in `getChainWithUpdatedRpcUrl`
2532        if !name.contains("assert") && name != "rpcUrl" {
2533            *e = fmt_err!("vm.{name}: {e}");
2534        }
2535    }
2536
2537    trace!(
2538        target: "cheatcodes",
2539        return = %match &result {
2540            Ok(b) => hex::encode(b),
2541            Err(e) => e.to_string(),
2542        }
2543    );
2544
2545    result
2546}
2547
2548fn calls_as_dyn_cheatcode(calls: &Vm::VmCalls) -> &dyn DynCheatcode {
2549    macro_rules! as_dyn {
2550        ($($variant:ident),*) => {
2551            match calls {
2552                $(Vm::VmCalls::$variant(cheat) => cheat,)*
2553            }
2554        };
2555    }
2556    vm_calls!(as_dyn)
2557}
2558
2559/// Helper function to check if frame execution will exit.
2560fn will_exit(action: &InterpreterAction) -> bool {
2561    match action {
2562        InterpreterAction::Return(result) => {
2563            result.result.is_ok_or_revert() || result.result.is_error()
2564        }
2565        _ => false,
2566    }
2567}