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::BlobTransactionSidecarVariant;
24use alloy_evm::eth::EthEvmContext;
25use alloy_network::{TransactionBuilder4844, TransactionBuilder7594};
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<BlobTransactionSidecarVariant>,
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                        if blob_sidecar.is_eip4844() {
942                            tx_req.set_blob_sidecar(blob_sidecar.into_eip4844().unwrap());
943                        } else if blob_sidecar.is_eip7594() {
944                            tx_req.set_blob_sidecar_7594(blob_sidecar.into_eip7594().unwrap());
945                        }
946                    }
947
948                    // Apply active EIP-7702 delegations, if any.
949                    if !active_delegations.is_empty() {
950                        for auth in &active_delegations {
951                            let Ok(authority) = auth.recover_authority() else {
952                                continue;
953                            };
954                            if authority == broadcast.new_origin {
955                                // Increment nonce of broadcasting account to reflect signed
956                                // authorization.
957                                account.info.nonce += 1;
958                            }
959                        }
960                        tx_req.authorization_list = Some(active_delegations);
961                    }
962
963                    self.broadcastable_transactions.push_back(BroadcastableTransaction {
964                        rpc: ecx.journaled_state.database.active_fork_url(),
965                        transaction: tx_req.into(),
966                    });
967                    debug!(target: "cheatcodes", tx=?self.broadcastable_transactions.back().unwrap(), "broadcastable call");
968
969                    // Explicitly increment nonce if calls are not isolated.
970                    if !self.config.evm_opts.isolate {
971                        let prev = account.info.nonce;
972                        account.info.nonce += 1;
973                        debug!(target: "cheatcodes", address=%broadcast.new_origin, nonce=prev+1, prev, "incremented nonce");
974                    }
975                } else if broadcast.single_call {
976                    let msg = "`staticcall`s are not allowed after `broadcast`; use `startBroadcast` instead";
977                    return Some(CallOutcome {
978                        result: InterpreterResult {
979                            result: InstructionResult::Revert,
980                            output: Error::encode(msg),
981                            gas,
982                        },
983                        memory_offset: call.return_memory_offset.clone(),
984                        was_precompile_called: false,
985                        precompile_call_logs: vec![],
986                    });
987                }
988            }
989        }
990
991        // Record called accounts if `startStateDiffRecording` has been called
992        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
993            // Determine if account is "initialized," ie, it has a non-zero balance, a non-zero
994            // nonce, a non-zero KECCAK_EMPTY codehash, or non-empty code
995            let initialized;
996            let old_balance;
997            let old_nonce;
998
999            let (db, journal, _) = ecx.as_db_env_and_journal();
1000            if let Ok(acc) = journal.load_account(db, call.target_address) {
1001                initialized = acc.info.exists();
1002                old_balance = acc.info.balance;
1003                old_nonce = acc.info.nonce;
1004            } else {
1005                initialized = false;
1006                old_balance = U256::ZERO;
1007                old_nonce = 0;
1008            }
1009
1010            let kind = match call.scheme {
1011                CallScheme::Call => crate::Vm::AccountAccessKind::Call,
1012                CallScheme::CallCode => crate::Vm::AccountAccessKind::CallCode,
1013                CallScheme::DelegateCall => crate::Vm::AccountAccessKind::DelegateCall,
1014                CallScheme::StaticCall => crate::Vm::AccountAccessKind::StaticCall,
1015            };
1016
1017            // Record this call by pushing it to a new pending vector; all subsequent calls at
1018            // that depth will be pushed to the same vector. When the call ends, the
1019            // RecordedAccountAccess (and all subsequent RecordedAccountAccesses) will be
1020            // updated with the revert status of this call, since the EVM does not mark accounts
1021            // as "warm" if the call from which they were accessed is reverted
1022            recorded_account_diffs_stack.push(vec![AccountAccess {
1023                chainInfo: crate::Vm::ChainInfo {
1024                    forkId: ecx.journaled_state.db().active_fork_id().unwrap_or_default(),
1025                    chainId: U256::from(ecx.cfg.chain_id),
1026                },
1027                accessor: call.caller,
1028                account: call.bytecode_address,
1029                kind,
1030                initialized,
1031                oldBalance: old_balance,
1032                newBalance: U256::ZERO, // updated on call_end
1033                oldNonce: old_nonce,
1034                newNonce: 0, // updated on call_end
1035                value: call.call_value(),
1036                data: call.input.bytes(ecx),
1037                reverted: false,
1038                deployedCode: Bytes::new(),
1039                storageAccesses: vec![], // updated on step
1040                depth: ecx
1041                    .journaled_state
1042                    .depth()
1043                    .try_into()
1044                    .expect("journaled state depth exceeds u64"),
1045            }]);
1046        }
1047
1048        None
1049    }
1050
1051    pub fn rng(&mut self) -> &mut impl Rng {
1052        self.test_runner().rng()
1053    }
1054
1055    pub fn test_runner(&mut self) -> &mut TestRunner {
1056        self.test_runner.get_or_insert_with(|| match self.config.seed {
1057            Some(seed) => TestRunner::new_with_rng(
1058                proptest::test_runner::Config::default(),
1059                TestRng::from_seed(RngAlgorithm::ChaCha, &seed.to_be_bytes::<32>()),
1060            ),
1061            None => TestRunner::new(proptest::test_runner::Config::default()),
1062        })
1063    }
1064
1065    pub fn set_seed(&mut self, seed: U256) {
1066        self.test_runner = Some(TestRunner::new_with_rng(
1067            proptest::test_runner::Config::default(),
1068            TestRng::from_seed(RngAlgorithm::ChaCha, &seed.to_be_bytes::<32>()),
1069        ));
1070    }
1071
1072    /// Returns existing or set a default `ArbitraryStorage` option.
1073    /// Used by `setArbitraryStorage` cheatcode to track addresses with arbitrary storage.
1074    pub fn arbitrary_storage(&mut self) -> &mut ArbitraryStorage {
1075        self.arbitrary_storage.get_or_insert_with(ArbitraryStorage::default)
1076    }
1077
1078    /// Whether the given address has arbitrary storage.
1079    pub fn has_arbitrary_storage(&self, address: &Address) -> bool {
1080        match &self.arbitrary_storage {
1081            Some(storage) => storage.values.contains_key(address),
1082            None => false,
1083        }
1084    }
1085
1086    /// Whether the given slot of address with arbitrary storage should be overwritten.
1087    /// True if address is marked as and overwrite and if no value was previously generated for
1088    /// given slot.
1089    pub fn should_overwrite_arbitrary_storage(
1090        &self,
1091        address: &Address,
1092        storage_slot: U256,
1093    ) -> bool {
1094        match &self.arbitrary_storage {
1095            Some(storage) => {
1096                storage.overwrites.contains(address)
1097                    && storage
1098                        .values
1099                        .get(address)
1100                        .and_then(|arbitrary_values| arbitrary_values.get(&storage_slot))
1101                        .is_none()
1102            }
1103            None => false,
1104        }
1105    }
1106
1107    /// Whether the given address is a copy of an address with arbitrary storage.
1108    pub fn is_arbitrary_storage_copy(&self, address: &Address) -> bool {
1109        match &self.arbitrary_storage {
1110            Some(storage) => storage.copies.contains_key(address),
1111            None => false,
1112        }
1113    }
1114
1115    /// Returns struct definitions from the analysis, if available.
1116    pub fn struct_defs(&self) -> Option<&foundry_common::fmt::StructDefinitions> {
1117        self.analysis.as_ref().and_then(|analysis| analysis.struct_defs().ok())
1118    }
1119}
1120
1121impl Inspector<EthEvmContext<&mut dyn DatabaseExt>> for Cheatcodes {
1122    fn initialize_interp(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1123        // When the first interpreter is initialized we've circumvented the balance and gas checks,
1124        // so we apply our actual block data with the correct fees and all.
1125        if let Some(block) = self.block.take() {
1126            ecx.block = block;
1127        }
1128        if let Some(gas_price) = self.gas_price.take() {
1129            ecx.tx.gas_price = gas_price;
1130        }
1131
1132        // Record gas for current frame.
1133        if self.gas_metering.paused {
1134            self.gas_metering.paused_frames.push(interpreter.gas);
1135        }
1136
1137        // `expectRevert`: track the max call depth during `expectRevert`
1138        if let Some(expected) = &mut self.expected_revert {
1139            expected.max_depth = max(ecx.journaled_state.depth(), expected.max_depth);
1140        }
1141    }
1142
1143    fn step(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1144        self.pc = interpreter.bytecode.pc();
1145
1146        if self.broadcast.is_some() {
1147            self.set_gas_limit_type(interpreter);
1148        }
1149
1150        // `pauseGasMetering`: pause / resume interpreter gas.
1151        if self.gas_metering.paused {
1152            self.meter_gas(interpreter);
1153        }
1154
1155        // `resetGasMetering`: reset interpreter gas.
1156        if self.gas_metering.reset {
1157            self.meter_gas_reset(interpreter);
1158        }
1159
1160        // `record`: record storage reads and writes.
1161        if self.recording_accesses {
1162            self.record_accesses(interpreter);
1163        }
1164
1165        // `startStateDiffRecording`: record granular ordered storage accesses.
1166        if self.recorded_account_diffs_stack.is_some() {
1167            self.record_state_diffs(interpreter, ecx);
1168        }
1169
1170        // `expectSafeMemory`: check if the current opcode is allowed to interact with memory.
1171        if !self.allowed_mem_writes.is_empty() {
1172            self.check_mem_opcodes(
1173                interpreter,
1174                ecx.journaled_state.depth().try_into().expect("journaled state depth exceeds u64"),
1175            );
1176        }
1177
1178        // `startMappingRecording`: record SSTORE and KECCAK256.
1179        if let Some(mapping_slots) = &mut self.mapping_slots {
1180            mapping_step(mapping_slots, interpreter);
1181        }
1182
1183        // `snapshotGas*`: take a snapshot of the current gas.
1184        if self.gas_metering.recording {
1185            self.meter_gas_record(interpreter, ecx);
1186        }
1187    }
1188
1189    fn step_end(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1190        if self.gas_metering.paused {
1191            self.meter_gas_end(interpreter);
1192        }
1193
1194        if self.gas_metering.touched {
1195            self.meter_gas_check(interpreter);
1196        }
1197
1198        // `setArbitraryStorage` and `copyStorage`: add arbitrary values to storage.
1199        if self.arbitrary_storage.is_some() {
1200            self.arbitrary_storage_end(interpreter, ecx);
1201        }
1202    }
1203
1204    fn log(&mut self, _ecx: Ecx, log: Log) {
1205        if !self.expected_emits.is_empty()
1206            && let Some(err) = expect::handle_expect_emit(self, &log, None)
1207        {
1208            // Because we do not have access to the interpreter here, we cannot fail the test
1209            // immediately. In most cases the failure will still be caught on `call_end`.
1210            // In the rare case it is not, we log the error here.
1211            let _ = sh_err!("{err:?}");
1212        }
1213
1214        // `recordLogs`
1215        record_logs(&mut self.recorded_logs, &log);
1216    }
1217
1218    fn log_full(&mut self, interpreter: &mut Interpreter, _ecx: Ecx, log: Log) {
1219        if !self.expected_emits.is_empty() {
1220            expect::handle_expect_emit(self, &log, Some(interpreter));
1221        }
1222
1223        // `recordLogs`
1224        record_logs(&mut self.recorded_logs, &log);
1225    }
1226
1227    fn call(&mut self, ecx: Ecx, inputs: &mut CallInputs) -> Option<CallOutcome> {
1228        Self::call_with_executor(self, ecx, inputs, &mut TransparentCheatcodesExecutor)
1229    }
1230
1231    fn call_end(&mut self, ecx: Ecx, call: &CallInputs, outcome: &mut CallOutcome) {
1232        let cheatcode_call = call.target_address == CHEATCODE_ADDRESS
1233            || call.target_address == HARDHAT_CONSOLE_ADDRESS;
1234
1235        // Clean up pranks/broadcasts if it's not a cheatcode call end. We shouldn't do
1236        // it for cheatcode calls because they are not applied for cheatcodes in the `call` hook.
1237        // This should be placed before the revert handling, because we might exit early there
1238        if !cheatcode_call {
1239            // Clean up pranks
1240            let curr_depth = ecx.journaled_state.depth();
1241            if let Some(prank) = &self.get_prank(curr_depth)
1242                && curr_depth == prank.depth
1243            {
1244                ecx.tx.caller = prank.prank_origin;
1245
1246                // Clean single-call prank once we have returned to the original depth
1247                if prank.single_call {
1248                    self.pranks.remove(&curr_depth);
1249                }
1250            }
1251
1252            // Clean up broadcast
1253            if let Some(broadcast) = &self.broadcast
1254                && curr_depth == broadcast.depth
1255            {
1256                ecx.tx.caller = broadcast.original_origin;
1257
1258                // Clean single-call broadcast once we have returned to the original depth
1259                if broadcast.single_call {
1260                    let _ = self.broadcast.take();
1261                }
1262            }
1263        }
1264
1265        // Handle assume no revert cheatcode.
1266        if let Some(assume_no_revert) = &mut self.assume_no_revert {
1267            // Record current reverter address before processing the expect revert if call reverted,
1268            // expect revert is set with expected reverter address and no actual reverter set yet.
1269            if outcome.result.is_revert() && assume_no_revert.reverted_by.is_none() {
1270                assume_no_revert.reverted_by = Some(call.target_address);
1271            }
1272
1273            // allow multiple cheatcode calls at the same depth
1274            let curr_depth = ecx.journaled_state.depth();
1275            if curr_depth <= assume_no_revert.depth && !cheatcode_call {
1276                // Discard run if we're at the same depth as cheatcode, call reverted, and no
1277                // specific reason was supplied
1278                if outcome.result.is_revert() {
1279                    let assume_no_revert = std::mem::take(&mut self.assume_no_revert).unwrap();
1280                    return match revert_handlers::handle_assume_no_revert(
1281                        &assume_no_revert,
1282                        outcome.result.result,
1283                        &outcome.result.output,
1284                        &self.config.available_artifacts,
1285                    ) {
1286                        // if result is Ok, it was an anticipated revert; return an "assume" error
1287                        // to reject this run
1288                        Ok(_) => {
1289                            outcome.result.output = Error::from(MAGIC_ASSUME).abi_encode().into();
1290                        }
1291                        // if result is Error, it was an unanticipated revert; should revert
1292                        // normally
1293                        Err(error) => {
1294                            trace!(expected=?assume_no_revert, ?error, status=?outcome.result.result, "Expected revert mismatch");
1295                            outcome.result.result = InstructionResult::Revert;
1296                            outcome.result.output = error.abi_encode().into();
1297                        }
1298                    };
1299                } else {
1300                    // Call didn't revert, reset `assume_no_revert` state.
1301                    self.assume_no_revert = None;
1302                }
1303            }
1304        }
1305
1306        // Handle expected reverts.
1307        if let Some(expected_revert) = &mut self.expected_revert {
1308            // Record current reverter address and call scheme before processing the expect revert
1309            // if call reverted.
1310            if outcome.result.is_revert() {
1311                // Record current reverter address if expect revert is set with expected reverter
1312                // address and no actual reverter was set yet or if we're expecting more than one
1313                // revert.
1314                if expected_revert.reverter.is_some()
1315                    && (expected_revert.reverted_by.is_none() || expected_revert.count > 1)
1316                {
1317                    expected_revert.reverted_by = Some(call.target_address);
1318                }
1319            }
1320
1321            let curr_depth = ecx.journaled_state.depth();
1322            if curr_depth <= expected_revert.depth {
1323                let needs_processing = match expected_revert.kind {
1324                    ExpectedRevertKind::Default => !cheatcode_call,
1325                    // `pending_processing` == true means that we're in the `call_end` hook for
1326                    // `vm.expectCheatcodeRevert` and shouldn't expect revert here
1327                    ExpectedRevertKind::Cheatcode { pending_processing } => {
1328                        cheatcode_call && !pending_processing
1329                    }
1330                };
1331
1332                if needs_processing {
1333                    let mut expected_revert = std::mem::take(&mut self.expected_revert).unwrap();
1334                    return match revert_handlers::handle_expect_revert(
1335                        cheatcode_call,
1336                        false,
1337                        self.config.internal_expect_revert,
1338                        &expected_revert,
1339                        outcome.result.result,
1340                        outcome.result.output.clone(),
1341                        &self.config.available_artifacts,
1342                    ) {
1343                        Err(error) => {
1344                            trace!(expected=?expected_revert, ?error, status=?outcome.result.result, "Expected revert mismatch");
1345                            outcome.result.result = InstructionResult::Revert;
1346                            outcome.result.output = error.abi_encode().into();
1347                        }
1348                        Ok((_, retdata)) => {
1349                            expected_revert.actual_count += 1;
1350                            if expected_revert.actual_count < expected_revert.count {
1351                                self.expected_revert = Some(expected_revert);
1352                            }
1353                            outcome.result.result = InstructionResult::Return;
1354                            outcome.result.output = retdata;
1355                        }
1356                    };
1357                }
1358
1359                // Flip `pending_processing` flag for cheatcode revert expectations, marking that
1360                // we've exited the `expectCheatcodeRevert` call scope
1361                if let ExpectedRevertKind::Cheatcode { pending_processing } =
1362                    &mut self.expected_revert.as_mut().unwrap().kind
1363                {
1364                    *pending_processing = false;
1365                }
1366            }
1367        }
1368
1369        // Exit early for calls to cheatcodes as other logic is not relevant for cheatcode
1370        // invocations
1371        if cheatcode_call {
1372            return;
1373        }
1374
1375        // Record the gas usage of the call, this allows the `lastCallGas` cheatcode to
1376        // retrieve the gas usage of the last call.
1377        let gas = outcome.result.gas;
1378        self.gas_metering.last_call_gas = Some(crate::Vm::Gas {
1379            gasLimit: gas.limit(),
1380            gasTotalUsed: gas.spent(),
1381            gasMemoryUsed: 0,
1382            gasRefunded: gas.refunded(),
1383            gasRemaining: gas.remaining(),
1384        });
1385
1386        // If `startStateDiffRecording` has been called, update the `reverted` status of the
1387        // previous call depth's recorded accesses, if any
1388        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
1389            // The root call cannot be recorded.
1390            if ecx.journaled_state.depth() > 0
1391                && let Some(mut last_recorded_depth) = recorded_account_diffs_stack.pop()
1392            {
1393                // Update the reverted status of all deeper calls if this call reverted, in
1394                // accordance with EVM behavior
1395                if outcome.result.is_revert() {
1396                    last_recorded_depth.iter_mut().for_each(|element| {
1397                        element.reverted = true;
1398                        element
1399                            .storageAccesses
1400                            .iter_mut()
1401                            .for_each(|storage_access| storage_access.reverted = true);
1402                    })
1403                }
1404
1405                if let Some(call_access) = last_recorded_depth.first_mut() {
1406                    // Assert that we're at the correct depth before recording post-call state
1407                    // changes. Depending on the depth the cheat was
1408                    // called at, there may not be any pending
1409                    // calls to update if execution has percolated up to a higher depth.
1410                    let (db, journal, _) = ecx.as_db_env_and_journal();
1411                    let curr_depth = journal.depth;
1412                    if call_access.depth == curr_depth as u64
1413                        && let Ok(acc) = journal.load_account(db, call.target_address)
1414                    {
1415                        debug_assert!(access_is_call(call_access.kind));
1416                        call_access.newBalance = acc.info.balance;
1417                        call_access.newNonce = acc.info.nonce;
1418                    }
1419                    // Merge the last depth's AccountAccesses into the AccountAccesses at the
1420                    // current depth, or push them back onto the pending
1421                    // vector if higher depths were not recorded. This
1422                    // preserves ordering of accesses.
1423                    if let Some(last) = recorded_account_diffs_stack.last_mut() {
1424                        last.extend(last_recorded_depth);
1425                    } else {
1426                        recorded_account_diffs_stack.push(last_recorded_depth);
1427                    }
1428                }
1429            }
1430        }
1431
1432        // At the end of the call,
1433        // we need to check if we've found all the emits.
1434        // We know we've found all the expected emits in the right order
1435        // if the queue is fully matched.
1436        // If it's not fully matched, then either:
1437        // 1. Not enough events were emitted (we'll know this because the amount of times we
1438        // inspected events will be less than the size of the queue) 2. The wrong events
1439        // were emitted (The inspected events should match the size of the queue, but still some
1440        // events will not be matched)
1441
1442        // First, check that we're at the call depth where the emits were declared from.
1443        let should_check_emits = self
1444            .expected_emits
1445            .iter()
1446            .any(|(expected, _)| {
1447                let curr_depth = ecx.journaled_state.depth();
1448                expected.depth == curr_depth
1449            }) &&
1450            // Ignore staticcalls
1451            !call.is_static;
1452        if should_check_emits {
1453            let expected_counts = self
1454                .expected_emits
1455                .iter()
1456                .filter_map(|(expected, count_map)| {
1457                    let count = match expected.address {
1458                        Some(emitter) => match count_map.get(&emitter) {
1459                            Some(log_count) => expected
1460                                .log
1461                                .as_ref()
1462                                .map(|l| log_count.count(l))
1463                                .unwrap_or_else(|| log_count.count_unchecked()),
1464                            None => 0,
1465                        },
1466                        None => match &expected.log {
1467                            Some(log) => count_map.values().map(|logs| logs.count(log)).sum(),
1468                            None => count_map.values().map(|logs| logs.count_unchecked()).sum(),
1469                        },
1470                    };
1471
1472                    if count != expected.count { Some((expected, count)) } else { None }
1473                })
1474                .collect::<Vec<_>>();
1475
1476            // Revert if not all emits expected were matched.
1477            if let Some((expected, _)) = self
1478                .expected_emits
1479                .iter()
1480                .find(|(expected, _)| !expected.found && expected.count > 0)
1481            {
1482                outcome.result.result = InstructionResult::Revert;
1483                let error_msg = expected.mismatch_error.as_deref().unwrap_or("log != expected log");
1484                outcome.result.output = error_msg.abi_encode().into();
1485                return;
1486            }
1487
1488            if !expected_counts.is_empty() {
1489                let msg = if outcome.result.is_ok() {
1490                    let (expected, count) = expected_counts.first().unwrap();
1491                    format!("log emitted {count} times, expected {}", expected.count)
1492                } else {
1493                    "expected an emit, but the call reverted instead. \
1494                     ensure you're testing the happy path when using `expectEmit`"
1495                        .to_string()
1496                };
1497
1498                outcome.result.result = InstructionResult::Revert;
1499                outcome.result.output = Error::encode(msg);
1500                return;
1501            }
1502
1503            // All emits were found, we're good.
1504            // Clear the queue, as we expect the user to declare more events for the next call
1505            // if they wanna match further events.
1506            self.expected_emits.clear()
1507        }
1508
1509        // this will ensure we don't have false positives when trying to diagnose reverts in fork
1510        // mode
1511        let diag = self.fork_revert_diagnostic.take();
1512
1513        // if there's a revert and a previous call was diagnosed as fork related revert then we can
1514        // return a better error here
1515        if outcome.result.is_revert()
1516            && let Some(err) = diag
1517        {
1518            outcome.result.output = Error::encode(err.to_error_msg(&self.labels));
1519            return;
1520        }
1521
1522        // try to diagnose reverts in multi-fork mode where a call is made to an address that does
1523        // not exist
1524        if let TxKind::Call(test_contract) = ecx.tx.kind {
1525            // if a call to a different contract than the original test contract returned with
1526            // `Stop` we check if the contract actually exists on the active fork
1527            if ecx.journaled_state.db().is_forked_mode()
1528                && outcome.result.result == InstructionResult::Stop
1529                && call.target_address != test_contract
1530            {
1531                let journaled_state = ecx.journaled_state.clone();
1532                self.fork_revert_diagnostic =
1533                    ecx.journaled_state.db().diagnose_revert(call.target_address, &journaled_state);
1534            }
1535        }
1536
1537        // If the depth is 0, then this is the root call terminating
1538        if ecx.journaled_state.depth() == 0 {
1539            // If we already have a revert, we shouldn't run the below logic as it can obfuscate an
1540            // earlier error that happened first with unrelated information about
1541            // another error when using cheatcodes.
1542            if outcome.result.is_revert() {
1543                return;
1544            }
1545
1546            // If there's not a revert, we can continue on to run the last logic for expect*
1547            // cheatcodes.
1548
1549            // Match expected calls
1550            for (address, calldatas) in &self.expected_calls {
1551                // Loop over each address, and for each address, loop over each calldata it expects.
1552                for (calldata, (expected, actual_count)) in calldatas {
1553                    // Grab the values we expect to see
1554                    let ExpectedCallData { gas, min_gas, value, count, call_type } = expected;
1555
1556                    let failed = match call_type {
1557                        // If the cheatcode was called with a `count` argument,
1558                        // we must check that the EVM performed a CALL with this calldata exactly
1559                        // `count` times.
1560                        ExpectedCallType::Count => *count != *actual_count,
1561                        // If the cheatcode was called without a `count` argument,
1562                        // we must check that the EVM performed a CALL with this calldata at least
1563                        // `count` times. The amount of times to check was
1564                        // the amount of time the cheatcode was called.
1565                        ExpectedCallType::NonCount => *count > *actual_count,
1566                    };
1567                    if failed {
1568                        let expected_values = [
1569                            Some(format!("data {}", hex::encode_prefixed(calldata))),
1570                            value.as_ref().map(|v| format!("value {v}")),
1571                            gas.map(|g| format!("gas {g}")),
1572                            min_gas.map(|g| format!("minimum gas {g}")),
1573                        ]
1574                        .into_iter()
1575                        .flatten()
1576                        .join(", ");
1577                        let but = if outcome.result.is_ok() {
1578                            let s = if *actual_count == 1 { "" } else { "s" };
1579                            format!("was called {actual_count} time{s}")
1580                        } else {
1581                            "the call reverted instead; \
1582                             ensure you're testing the happy path when using `expectCall`"
1583                                .to_string()
1584                        };
1585                        let s = if *count == 1 { "" } else { "s" };
1586                        let msg = format!(
1587                            "expected call to {address} with {expected_values} \
1588                             to be called {count} time{s}, but {but}"
1589                        );
1590                        outcome.result.result = InstructionResult::Revert;
1591                        outcome.result.output = Error::encode(msg);
1592
1593                        return;
1594                    }
1595                }
1596            }
1597
1598            // Check if we have any leftover expected emits
1599            // First, if any emits were found at the root call, then we its ok and we remove them.
1600            // For count=0 expectations, NOT being found is success, so mark them as found
1601            for (expected, _) in &mut self.expected_emits {
1602                if expected.count == 0 && !expected.found {
1603                    expected.found = true;
1604                }
1605            }
1606            self.expected_emits.retain(|(expected, _)| !expected.found);
1607            // If not empty, we got mismatched emits
1608            if !self.expected_emits.is_empty() {
1609                let msg = if outcome.result.is_ok() {
1610                    "expected an emit, but no logs were emitted afterwards. \
1611                     you might have mismatched events or not enough events were emitted"
1612                } else {
1613                    "expected an emit, but the call reverted instead. \
1614                     ensure you're testing the happy path when using `expectEmit`"
1615                };
1616                outcome.result.result = InstructionResult::Revert;
1617                outcome.result.output = Error::encode(msg);
1618                return;
1619            }
1620
1621            // Check for leftover expected creates
1622            if let Some(expected_create) = self.expected_creates.first() {
1623                let msg = format!(
1624                    "expected {} call by address {} for bytecode {} but not found",
1625                    expected_create.create_scheme,
1626                    hex::encode_prefixed(expected_create.deployer),
1627                    hex::encode_prefixed(&expected_create.bytecode),
1628                );
1629                outcome.result.result = InstructionResult::Revert;
1630                outcome.result.output = Error::encode(msg);
1631            }
1632        }
1633    }
1634
1635    fn create(&mut self, ecx: Ecx, mut input: &mut CreateInputs) -> Option<CreateOutcome> {
1636        // Apply custom execution evm version.
1637        if let Some(spec_id) = self.execution_evm_version {
1638            ecx.cfg.spec = spec_id;
1639        }
1640
1641        let gas = Gas::new(input.gas_limit());
1642        // Check if we should intercept this create
1643        if self.intercept_next_create_call {
1644            // Reset the flag
1645            self.intercept_next_create_call = false;
1646
1647            // Get initcode from the input
1648            let output = input.init_code();
1649
1650            // Return a revert with the initcode as error data
1651            return Some(CreateOutcome {
1652                result: InterpreterResult { result: InstructionResult::Revert, output, gas },
1653                address: None,
1654            });
1655        }
1656
1657        let curr_depth = ecx.journaled_state.depth();
1658
1659        // Apply our prank
1660        if let Some(prank) = &self.get_prank(curr_depth)
1661            && curr_depth >= prank.depth
1662            && input.caller() == prank.prank_caller
1663        {
1664            let mut prank_applied = false;
1665
1666            // At the target depth we set `msg.sender`
1667            if curr_depth == prank.depth {
1668                // Ensure new caller is loaded and touched
1669                let _ = journaled_account(ecx, prank.new_caller);
1670                input.set_caller(prank.new_caller);
1671                prank_applied = true;
1672            }
1673
1674            // At the target depth, or deeper, we set `tx.origin`
1675            if let Some(new_origin) = prank.new_origin {
1676                ecx.tx.caller = new_origin;
1677                prank_applied = true;
1678            }
1679
1680            // If prank applied for first time, then update
1681            if prank_applied && let Some(applied_prank) = prank.first_time_applied() {
1682                self.pranks.insert(curr_depth, applied_prank);
1683            }
1684        }
1685
1686        // Apply EIP-2930 access list
1687        self.apply_accesslist(ecx);
1688
1689        // Apply our broadcast
1690        if let Some(broadcast) = &mut self.broadcast
1691            && curr_depth >= broadcast.depth
1692            && input.caller() == broadcast.original_caller
1693        {
1694            let (db, journal, _) = ecx.as_db_env_and_journal();
1695            if let Err(err) = journal.load_account(db, broadcast.new_origin) {
1696                return Some(CreateOutcome {
1697                    result: InterpreterResult {
1698                        result: InstructionResult::Revert,
1699                        output: Error::encode(err),
1700                        gas,
1701                    },
1702                    address: None,
1703                });
1704            }
1705
1706            ecx.tx.caller = broadcast.new_origin;
1707
1708            if curr_depth == broadcast.depth || broadcast.deploy_from_code {
1709                // Reset deploy from code flag for upcoming calls;
1710                broadcast.deploy_from_code = false;
1711
1712                input.set_caller(broadcast.new_origin);
1713
1714                let account = &ecx.journaled_state.inner.state()[&broadcast.new_origin];
1715                self.broadcastable_transactions.push_back(BroadcastableTransaction {
1716                    rpc: ecx.journaled_state.database.active_fork_url(),
1717                    transaction: TransactionRequest {
1718                        from: Some(broadcast.new_origin),
1719                        to: None,
1720                        value: Some(input.value()),
1721                        input: TransactionInput::new(input.init_code()),
1722                        nonce: Some(account.info.nonce),
1723                        ..Default::default()
1724                    }
1725                    .into(),
1726                });
1727
1728                input.log_debug(self, &input.scheme().unwrap_or(CreateScheme::Create));
1729            }
1730        }
1731
1732        // Allow cheatcodes from the address of the new contract
1733        let address = input.allow_cheatcodes(self, ecx);
1734
1735        // If `recordAccountAccesses` has been called, record the create
1736        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
1737            recorded_account_diffs_stack.push(vec![AccountAccess {
1738                chainInfo: crate::Vm::ChainInfo {
1739                    forkId: ecx.journaled_state.db().active_fork_id().unwrap_or_default(),
1740                    chainId: U256::from(ecx.cfg.chain_id),
1741                },
1742                accessor: input.caller(),
1743                account: address,
1744                kind: crate::Vm::AccountAccessKind::Create,
1745                initialized: true,
1746                oldBalance: U256::ZERO, // updated on create_end
1747                newBalance: U256::ZERO, // updated on create_end
1748                oldNonce: 0,            // new contract starts with nonce 0
1749                newNonce: 1,            // updated on create_end (contracts start with nonce 1)
1750                value: input.value(),
1751                data: input.init_code(),
1752                reverted: false,
1753                deployedCode: Bytes::new(), // updated on create_end
1754                storageAccesses: vec![],    // updated on create_end
1755                depth: curr_depth as u64,
1756            }]);
1757        }
1758
1759        None
1760    }
1761
1762    fn create_end(&mut self, ecx: Ecx, call: &CreateInputs, outcome: &mut CreateOutcome) {
1763        let call = Some(call);
1764        let curr_depth = ecx.journaled_state.depth();
1765
1766        // Clean up pranks
1767        if let Some(prank) = &self.get_prank(curr_depth)
1768            && curr_depth == prank.depth
1769        {
1770            ecx.tx.caller = prank.prank_origin;
1771
1772            // Clean single-call prank once we have returned to the original depth
1773            if prank.single_call {
1774                std::mem::take(&mut self.pranks);
1775            }
1776        }
1777
1778        // Clean up broadcasts
1779        if let Some(broadcast) = &self.broadcast
1780            && curr_depth == broadcast.depth
1781        {
1782            ecx.tx.caller = broadcast.original_origin;
1783
1784            // Clean single-call broadcast once we have returned to the original depth
1785            if broadcast.single_call {
1786                std::mem::take(&mut self.broadcast);
1787            }
1788        }
1789
1790        // Handle expected reverts
1791        if let Some(expected_revert) = &self.expected_revert
1792            && curr_depth <= expected_revert.depth
1793            && matches!(expected_revert.kind, ExpectedRevertKind::Default)
1794        {
1795            let mut expected_revert = std::mem::take(&mut self.expected_revert).unwrap();
1796            return match revert_handlers::handle_expect_revert(
1797                false,
1798                true,
1799                self.config.internal_expect_revert,
1800                &expected_revert,
1801                outcome.result.result,
1802                outcome.result.output.clone(),
1803                &self.config.available_artifacts,
1804            ) {
1805                Ok((address, retdata)) => {
1806                    expected_revert.actual_count += 1;
1807                    if expected_revert.actual_count < expected_revert.count {
1808                        self.expected_revert = Some(expected_revert.clone());
1809                    }
1810
1811                    outcome.result.result = InstructionResult::Return;
1812                    outcome.result.output = retdata;
1813                    outcome.address = address;
1814                }
1815                Err(err) => {
1816                    outcome.result.result = InstructionResult::Revert;
1817                    outcome.result.output = err.abi_encode().into();
1818                }
1819            };
1820        }
1821
1822        // If `startStateDiffRecording` has been called, update the `reverted` status of the
1823        // previous call depth's recorded accesses, if any
1824        if let Some(recorded_account_diffs_stack) = &mut self.recorded_account_diffs_stack {
1825            // The root call cannot be recorded.
1826            if curr_depth > 0
1827                && let Some(last_depth) = &mut recorded_account_diffs_stack.pop()
1828            {
1829                // Update the reverted status of all deeper calls if this call reverted, in
1830                // accordance with EVM behavior
1831                if outcome.result.is_revert() {
1832                    last_depth.iter_mut().for_each(|element| {
1833                        element.reverted = true;
1834                        element
1835                            .storageAccesses
1836                            .iter_mut()
1837                            .for_each(|storage_access| storage_access.reverted = true);
1838                    })
1839                }
1840
1841                if let Some(create_access) = last_depth.first_mut() {
1842                    // Assert that we're at the correct depth before recording post-create state
1843                    // changes. Depending on what depth the cheat was called at, there
1844                    // may not be any pending calls to update if execution has
1845                    // percolated up to a higher depth.
1846                    let depth = ecx.journaled_state.depth();
1847                    if create_access.depth == depth as u64 {
1848                        debug_assert_eq!(
1849                            create_access.kind as u8,
1850                            crate::Vm::AccountAccessKind::Create as u8
1851                        );
1852                        let (db, journal, _) = ecx.as_db_env_and_journal();
1853                        if let Some(address) = outcome.address
1854                            && let Ok(created_acc) = journal.load_account(db, address)
1855                        {
1856                            create_access.newBalance = created_acc.info.balance;
1857                            create_access.newNonce = created_acc.info.nonce;
1858                            create_access.deployedCode =
1859                                created_acc.info.code.clone().unwrap_or_default().original_bytes();
1860                        }
1861                    }
1862                    // Merge the last depth's AccountAccesses into the AccountAccesses at the
1863                    // current depth, or push them back onto the pending
1864                    // vector if higher depths were not recorded. This
1865                    // preserves ordering of accesses.
1866                    if let Some(last) = recorded_account_diffs_stack.last_mut() {
1867                        last.append(last_depth);
1868                    } else {
1869                        recorded_account_diffs_stack.push(last_depth.clone());
1870                    }
1871                }
1872            }
1873        }
1874
1875        // Match the create against expected_creates
1876        let (db, journal, _) = ecx.as_db_env_and_journal();
1877        if !self.expected_creates.is_empty()
1878            && let (Some(address), Some(call)) = (outcome.address, call)
1879            && let Ok(created_acc) = journal.load_account(db, address)
1880        {
1881            let bytecode = created_acc.info.code.clone().unwrap_or_default().original_bytes();
1882            if let Some((index, _)) =
1883                self.expected_creates.iter().find_position(|expected_create| {
1884                    expected_create.deployer == call.caller()
1885                        && expected_create.create_scheme.eq(call.scheme().into())
1886                        && expected_create.bytecode == bytecode
1887                })
1888            {
1889                self.expected_creates.swap_remove(index);
1890            }
1891        }
1892    }
1893}
1894
1895impl InspectorExt for Cheatcodes {
1896    fn should_use_create2_factory(&mut self, ecx: Ecx, inputs: &CreateInputs) -> bool {
1897        if let CreateScheme::Create2 { .. } = inputs.scheme() {
1898            let depth = ecx.journaled_state.depth();
1899            let target_depth = if let Some(prank) = &self.get_prank(depth) {
1900                prank.depth
1901            } else if let Some(broadcast) = &self.broadcast {
1902                broadcast.depth
1903            } else {
1904                1
1905            };
1906
1907            depth == target_depth
1908                && (self.broadcast.is_some() || self.config.always_use_create_2_factory)
1909        } else {
1910            false
1911        }
1912    }
1913
1914    fn create2_deployer(&self) -> Address {
1915        self.config.evm_opts.create2_deployer
1916    }
1917}
1918
1919impl Cheatcodes {
1920    #[cold]
1921    fn meter_gas(&mut self, interpreter: &mut Interpreter) {
1922        if let Some(paused_gas) = self.gas_metering.paused_frames.last() {
1923            // Keep gas constant if paused.
1924            // Make sure we record the memory changes so that memory expansion is not paused.
1925            let memory = *interpreter.gas.memory();
1926            interpreter.gas = *paused_gas;
1927            interpreter.gas.memory_mut().words_num = memory.words_num;
1928            interpreter.gas.memory_mut().expansion_cost = memory.expansion_cost;
1929        } else {
1930            // Record frame paused gas.
1931            self.gas_metering.paused_frames.push(interpreter.gas);
1932        }
1933    }
1934
1935    #[cold]
1936    fn meter_gas_record(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
1937        if interpreter.bytecode.action.as_ref().and_then(|i| i.instruction_result()).is_none() {
1938            self.gas_metering.gas_records.iter_mut().for_each(|record| {
1939                let curr_depth = ecx.journaled_state.depth();
1940                if curr_depth == record.depth {
1941                    // Skip the first opcode of the first call frame as it includes the gas cost of
1942                    // creating the snapshot.
1943                    if self.gas_metering.last_gas_used != 0 {
1944                        let gas_diff =
1945                            interpreter.gas.spent().saturating_sub(self.gas_metering.last_gas_used);
1946                        record.gas_used = record.gas_used.saturating_add(gas_diff);
1947                    }
1948
1949                    // Update `last_gas_used` to the current spent gas for the next iteration to
1950                    // compare against.
1951                    self.gas_metering.last_gas_used = interpreter.gas.spent();
1952                }
1953            });
1954        }
1955    }
1956
1957    #[cold]
1958    fn meter_gas_end(&mut self, interpreter: &mut Interpreter) {
1959        // Remove recorded gas if we exit frame.
1960        if let Some(interpreter_action) = interpreter.bytecode.action.as_ref()
1961            && will_exit(interpreter_action)
1962        {
1963            self.gas_metering.paused_frames.pop();
1964        }
1965    }
1966
1967    #[cold]
1968    fn meter_gas_reset(&mut self, interpreter: &mut Interpreter) {
1969        let mut gas = Gas::new(interpreter.gas.limit());
1970        gas.memory_mut().words_num = interpreter.gas.memory().words_num;
1971        gas.memory_mut().expansion_cost = interpreter.gas.memory().expansion_cost;
1972        interpreter.gas = gas;
1973        self.gas_metering.reset = false;
1974    }
1975
1976    #[cold]
1977    fn meter_gas_check(&mut self, interpreter: &mut Interpreter) {
1978        if let Some(interpreter_action) = interpreter.bytecode.action.as_ref()
1979            && will_exit(interpreter_action)
1980        {
1981            // Reset gas if spent is less than refunded.
1982            // This can happen if gas was paused / resumed or reset.
1983            // https://github.com/foundry-rs/foundry/issues/4370
1984            if interpreter.gas.spent()
1985                < u64::try_from(interpreter.gas.refunded()).unwrap_or_default()
1986            {
1987                interpreter.gas = Gas::new(interpreter.gas.limit());
1988            }
1989        }
1990    }
1991
1992    /// Generates or copies arbitrary values for storage slots.
1993    /// Invoked in inspector `step_end` (when the current opcode is not executed), if current opcode
1994    /// to execute is `SLOAD` and storage slot is cold.
1995    /// Ensures that in next step (when `SLOAD` opcode is executed) an arbitrary value is returned:
1996    /// - copies the existing arbitrary storage value (or the new generated one if no value in
1997    ///   cache) from mapped source address to the target address.
1998    /// - generates arbitrary value and saves it in target address storage.
1999    #[cold]
2000    fn arbitrary_storage_end(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
2001        let (key, target_address) = if interpreter.bytecode.opcode() == op::SLOAD {
2002            (try_or_return!(interpreter.stack.peek(0)), interpreter.input.target_address)
2003        } else {
2004            return;
2005        };
2006
2007        let Some(value) = ecx.sload(target_address, key) else {
2008            return;
2009        };
2010
2011        if (value.is_cold && value.data.is_zero())
2012            || self.should_overwrite_arbitrary_storage(&target_address, key)
2013        {
2014            if self.has_arbitrary_storage(&target_address) {
2015                let arbitrary_value = self.rng().random();
2016                self.arbitrary_storage.as_mut().unwrap().save(
2017                    ecx,
2018                    target_address,
2019                    key,
2020                    arbitrary_value,
2021                );
2022            } else if self.is_arbitrary_storage_copy(&target_address) {
2023                let arbitrary_value = self.rng().random();
2024                self.arbitrary_storage.as_mut().unwrap().copy(
2025                    ecx,
2026                    target_address,
2027                    key,
2028                    arbitrary_value,
2029                );
2030            }
2031        }
2032    }
2033
2034    /// Records storage slots reads and writes.
2035    #[cold]
2036    fn record_accesses(&mut self, interpreter: &mut Interpreter) {
2037        let access = &mut self.accesses;
2038        match interpreter.bytecode.opcode() {
2039            op::SLOAD => {
2040                let key = try_or_return!(interpreter.stack.peek(0));
2041                access.record_read(interpreter.input.target_address, key);
2042            }
2043            op::SSTORE => {
2044                let key = try_or_return!(interpreter.stack.peek(0));
2045                access.record_write(interpreter.input.target_address, key);
2046            }
2047            _ => {}
2048        }
2049    }
2050
2051    #[cold]
2052    fn record_state_diffs(&mut self, interpreter: &mut Interpreter, ecx: Ecx) {
2053        let Some(account_accesses) = &mut self.recorded_account_diffs_stack else { return };
2054        match interpreter.bytecode.opcode() {
2055            op::SELFDESTRUCT => {
2056                // Ensure that we're not selfdestructing a context recording was initiated on
2057                let Some(last) = account_accesses.last_mut() else { return };
2058
2059                // get previous balance, nonce and initialized status of the target account
2060                let target = try_or_return!(interpreter.stack.peek(0));
2061                let target = Address::from_word(B256::from(target));
2062                let (db, journal, _) = ecx.as_db_env_and_journal();
2063                let (initialized, old_balance, old_nonce) = journal
2064                    .load_account(db, target)
2065                    .map(|account| {
2066                        (account.info.exists(), account.info.balance, account.info.nonce)
2067                    })
2068                    .unwrap_or_default();
2069
2070                // load balance of this account
2071                let value = ecx
2072                    .balance(interpreter.input.target_address)
2073                    .map(|b| b.data)
2074                    .unwrap_or(U256::ZERO);
2075
2076                // register access for the target account
2077                last.push(crate::Vm::AccountAccess {
2078                    chainInfo: crate::Vm::ChainInfo {
2079                        forkId: ecx.journaled_state.database.active_fork_id().unwrap_or_default(),
2080                        chainId: U256::from(ecx.cfg.chain_id),
2081                    },
2082                    accessor: interpreter.input.target_address,
2083                    account: target,
2084                    kind: crate::Vm::AccountAccessKind::SelfDestruct,
2085                    initialized,
2086                    oldBalance: old_balance,
2087                    newBalance: old_balance + value,
2088                    oldNonce: old_nonce,
2089                    newNonce: old_nonce, // nonce doesn't change on selfdestruct
2090                    value,
2091                    data: Bytes::new(),
2092                    reverted: false,
2093                    deployedCode: Bytes::new(),
2094                    storageAccesses: vec![],
2095                    depth: ecx
2096                        .journaled_state
2097                        .depth()
2098                        .try_into()
2099                        .expect("journaled state depth exceeds u64"),
2100                });
2101            }
2102
2103            op::SLOAD => {
2104                let Some(last) = account_accesses.last_mut() else { return };
2105
2106                let key = try_or_return!(interpreter.stack.peek(0));
2107                let address = interpreter.input.target_address;
2108
2109                // Try to include present value for informational purposes, otherwise assume
2110                // it's not set (zero value)
2111                let mut present_value = U256::ZERO;
2112                // Try to load the account and the slot's present value
2113                let (db, journal, _) = ecx.as_db_env_and_journal();
2114                if journal.load_account(db, address).is_ok()
2115                    && let Some(previous) = ecx.sload(address, key)
2116                {
2117                    present_value = previous.data;
2118                }
2119                let access = crate::Vm::StorageAccess {
2120                    account: interpreter.input.target_address,
2121                    slot: key.into(),
2122                    isWrite: false,
2123                    previousValue: present_value.into(),
2124                    newValue: present_value.into(),
2125                    reverted: false,
2126                };
2127                let curr_depth = ecx
2128                    .journaled_state
2129                    .depth()
2130                    .try_into()
2131                    .expect("journaled state depth exceeds u64");
2132                append_storage_access(last, access, curr_depth);
2133            }
2134            op::SSTORE => {
2135                let Some(last) = account_accesses.last_mut() else { return };
2136
2137                let key = try_or_return!(interpreter.stack.peek(0));
2138                let value = try_or_return!(interpreter.stack.peek(1));
2139                let address = interpreter.input.target_address;
2140                // Try to load the account and the slot's previous value, otherwise, assume it's
2141                // not set (zero value)
2142                let mut previous_value = U256::ZERO;
2143                let (db, journal, _) = ecx.as_db_env_and_journal();
2144                if journal.load_account(db, address).is_ok()
2145                    && let Some(previous) = ecx.sload(address, key)
2146                {
2147                    previous_value = previous.data;
2148                }
2149
2150                let access = crate::Vm::StorageAccess {
2151                    account: address,
2152                    slot: key.into(),
2153                    isWrite: true,
2154                    previousValue: previous_value.into(),
2155                    newValue: value.into(),
2156                    reverted: false,
2157                };
2158                let curr_depth = ecx
2159                    .journaled_state
2160                    .depth()
2161                    .try_into()
2162                    .expect("journaled state depth exceeds u64");
2163                append_storage_access(last, access, curr_depth);
2164            }
2165
2166            // Record account accesses via the EXT family of opcodes
2167            op::EXTCODECOPY | op::EXTCODESIZE | op::EXTCODEHASH | op::BALANCE => {
2168                let kind = match interpreter.bytecode.opcode() {
2169                    op::EXTCODECOPY => crate::Vm::AccountAccessKind::Extcodecopy,
2170                    op::EXTCODESIZE => crate::Vm::AccountAccessKind::Extcodesize,
2171                    op::EXTCODEHASH => crate::Vm::AccountAccessKind::Extcodehash,
2172                    op::BALANCE => crate::Vm::AccountAccessKind::Balance,
2173                    _ => unreachable!(),
2174                };
2175                let address =
2176                    Address::from_word(B256::from(try_or_return!(interpreter.stack.peek(0))));
2177                let initialized;
2178                let balance;
2179                let nonce;
2180                let (db, journal, _) = ecx.as_db_env_and_journal();
2181                if let Ok(acc) = journal.load_account(db, address) {
2182                    initialized = acc.info.exists();
2183                    balance = acc.info.balance;
2184                    nonce = acc.info.nonce;
2185                } else {
2186                    initialized = false;
2187                    balance = U256::ZERO;
2188                    nonce = 0;
2189                }
2190                let curr_depth = ecx
2191                    .journaled_state
2192                    .depth()
2193                    .try_into()
2194                    .expect("journaled state depth exceeds u64");
2195                let account_access = crate::Vm::AccountAccess {
2196                    chainInfo: crate::Vm::ChainInfo {
2197                        forkId: ecx.journaled_state.database.active_fork_id().unwrap_or_default(),
2198                        chainId: U256::from(ecx.cfg.chain_id),
2199                    },
2200                    accessor: interpreter.input.target_address,
2201                    account: address,
2202                    kind,
2203                    initialized,
2204                    oldBalance: balance,
2205                    newBalance: balance,
2206                    oldNonce: nonce,
2207                    newNonce: nonce, // EXT* operations don't change nonce
2208                    value: U256::ZERO,
2209                    data: Bytes::new(),
2210                    reverted: false,
2211                    deployedCode: Bytes::new(),
2212                    storageAccesses: vec![],
2213                    depth: curr_depth,
2214                };
2215                // Record the EXT* call as an account access at the current depth
2216                // (future storage accesses will be recorded in a new "Resume" context)
2217                if let Some(last) = account_accesses.last_mut() {
2218                    last.push(account_access);
2219                } else {
2220                    account_accesses.push(vec![account_access]);
2221                }
2222            }
2223            _ => {}
2224        }
2225    }
2226
2227    /// Checks to see if the current opcode can either mutate directly or expand memory.
2228    ///
2229    /// If the opcode at the current program counter is a match, check if the modified memory lies
2230    /// within the allowed ranges. If not, revert and fail the test.
2231    #[cold]
2232    fn check_mem_opcodes(&self, interpreter: &mut Interpreter, depth: u64) {
2233        let Some(ranges) = self.allowed_mem_writes.get(&depth) else {
2234            return;
2235        };
2236
2237        // The `mem_opcode_match` macro is used to match the current opcode against a list of
2238        // opcodes that can mutate memory (either directly or expansion via reading). If the
2239        // opcode is a match, the memory offsets that are being written to are checked to be
2240        // within the allowed ranges. If not, the test is failed and the transaction is
2241        // reverted. For all opcodes that can mutate memory aside from MSTORE,
2242        // MSTORE8, and MLOAD, the size and destination offset are on the stack, and
2243        // the macro expands all of these cases. For MSTORE, MSTORE8, and MLOAD, the
2244        // size of the memory write is implicit, so these cases are hard-coded.
2245        macro_rules! mem_opcode_match {
2246            ($(($opcode:ident, $offset_depth:expr, $size_depth:expr, $writes:expr)),* $(,)?) => {
2247                match interpreter.bytecode.opcode() {
2248                    ////////////////////////////////////////////////////////////////
2249                    //    OPERATIONS THAT CAN EXPAND/MUTATE MEMORY BY WRITING     //
2250                    ////////////////////////////////////////////////////////////////
2251
2252                    op::MSTORE => {
2253                        // The offset of the mstore operation is at the top of the stack.
2254                        let offset = try_or_return!(interpreter.stack.peek(0)).saturating_to::<u64>();
2255
2256                        // If none of the allowed ranges contain [offset, offset + 32), memory has been
2257                        // unexpectedly mutated.
2258                        if !ranges.iter().any(|range| {
2259                            range.contains(&offset) && range.contains(&(offset + 31))
2260                        }) {
2261                            // SPECIAL CASE: When the compiler attempts to store the selector for
2262                            // `stopExpectSafeMemory`, this is allowed. It will do so at the current free memory
2263                            // pointer, which could have been updated to the exclusive upper bound during
2264                            // execution.
2265                            let value = try_or_return!(interpreter.stack.peek(1)).to_be_bytes::<32>();
2266                            if value[..SELECTOR_LEN] == stopExpectSafeMemoryCall::SELECTOR {
2267                                return
2268                            }
2269
2270                            disallowed_mem_write(offset, 32, interpreter, ranges);
2271                            return
2272                        }
2273                    }
2274                    op::MSTORE8 => {
2275                        // The offset of the mstore8 operation is at the top of the stack.
2276                        let offset = try_or_return!(interpreter.stack.peek(0)).saturating_to::<u64>();
2277
2278                        // If none of the allowed ranges contain the offset, memory has been
2279                        // unexpectedly mutated.
2280                        if !ranges.iter().any(|range| range.contains(&offset)) {
2281                            disallowed_mem_write(offset, 1, interpreter, ranges);
2282                            return
2283                        }
2284                    }
2285
2286                    ////////////////////////////////////////////////////////////////
2287                    //        OPERATIONS THAT CAN EXPAND MEMORY BY READING        //
2288                    ////////////////////////////////////////////////////////////////
2289
2290                    op::MLOAD => {
2291                        // The offset of the mload operation is at the top of the stack
2292                        let offset = try_or_return!(interpreter.stack.peek(0)).saturating_to::<u64>();
2293
2294                        // If the offset being loaded is >= than the memory size, the
2295                        // memory is being expanded. If none of the allowed ranges contain
2296                        // [offset, offset + 32), memory has been unexpectedly mutated.
2297                        if offset >= interpreter.memory.size() as u64 && !ranges.iter().any(|range| {
2298                            range.contains(&offset) && range.contains(&(offset + 31))
2299                        }) {
2300                            disallowed_mem_write(offset, 32, interpreter, ranges);
2301                            return
2302                        }
2303                    }
2304
2305                    ////////////////////////////////////////////////////////////////
2306                    //          OPERATIONS WITH OFFSET AND SIZE ON STACK          //
2307                    ////////////////////////////////////////////////////////////////
2308
2309                    op::CALL => {
2310                        // The destination offset of the operation is the fifth element on the stack.
2311                        let dest_offset = try_or_return!(interpreter.stack.peek(5)).saturating_to::<u64>();
2312
2313                        // The size of the data that will be copied is the sixth element on the stack.
2314                        let size = try_or_return!(interpreter.stack.peek(6)).saturating_to::<u64>();
2315
2316                        // If none of the allowed ranges contain [dest_offset, dest_offset + size),
2317                        // memory outside of the expected ranges has been touched. If the opcode
2318                        // only reads from memory, this is okay as long as the memory is not expanded.
2319                        let fail_cond = !ranges.iter().any(|range| {
2320                            range.contains(&dest_offset) &&
2321                                range.contains(&(dest_offset + size.saturating_sub(1)))
2322                        });
2323
2324                        // If the failure condition is met, set the output buffer to a revert string
2325                        // that gives information about the allowed ranges and revert.
2326                        if fail_cond {
2327                            // SPECIAL CASE: When a call to `stopExpectSafeMemory` is performed, this is allowed.
2328                            // It allocated calldata at the current free memory pointer, and will attempt to read
2329                            // from this memory region to perform the call.
2330                            let to = Address::from_word(try_or_return!(interpreter.stack.peek(1)).to_be_bytes::<32>().into());
2331                            if to == CHEATCODE_ADDRESS {
2332                                let args_offset = try_or_return!(interpreter.stack.peek(3)).saturating_to::<usize>();
2333                                let args_size = try_or_return!(interpreter.stack.peek(4)).saturating_to::<usize>();
2334                                let memory_word = interpreter.memory.slice_len(args_offset, args_size);
2335                                if memory_word[..SELECTOR_LEN] == stopExpectSafeMemoryCall::SELECTOR {
2336                                    return
2337                                }
2338                            }
2339
2340                            disallowed_mem_write(dest_offset, size, interpreter, ranges);
2341                            return
2342                        }
2343                    }
2344
2345                    $(op::$opcode => {
2346                        // The destination offset of the operation.
2347                        let dest_offset = try_or_return!(interpreter.stack.peek($offset_depth)).saturating_to::<u64>();
2348
2349                        // The size of the data that will be copied.
2350                        let size = try_or_return!(interpreter.stack.peek($size_depth)).saturating_to::<u64>();
2351
2352                        // If none of the allowed ranges contain [dest_offset, dest_offset + size),
2353                        // memory outside of the expected ranges has been touched. If the opcode
2354                        // only reads from memory, this is okay as long as the memory is not expanded.
2355                        let fail_cond = !ranges.iter().any(|range| {
2356                                range.contains(&dest_offset) &&
2357                                    range.contains(&(dest_offset + size.saturating_sub(1)))
2358                            }) && ($writes ||
2359                                [dest_offset, (dest_offset + size).saturating_sub(1)].into_iter().any(|offset| {
2360                                    offset >= interpreter.memory.size() as u64
2361                                })
2362                            );
2363
2364                        // If the failure condition is met, set the output buffer to a revert string
2365                        // that gives information about the allowed ranges and revert.
2366                        if fail_cond {
2367                            disallowed_mem_write(dest_offset, size, interpreter, ranges);
2368                            return
2369                        }
2370                    })*
2371
2372                    _ => {}
2373                }
2374            }
2375        }
2376
2377        // Check if the current opcode can write to memory, and if so, check if the memory
2378        // being written to is registered as safe to modify.
2379        mem_opcode_match!(
2380            (CALLDATACOPY, 0, 2, true),
2381            (CODECOPY, 0, 2, true),
2382            (RETURNDATACOPY, 0, 2, true),
2383            (EXTCODECOPY, 1, 3, true),
2384            (CALLCODE, 5, 6, true),
2385            (STATICCALL, 4, 5, true),
2386            (DELEGATECALL, 4, 5, true),
2387            (KECCAK256, 0, 1, false),
2388            (LOG0, 0, 1, false),
2389            (LOG1, 0, 1, false),
2390            (LOG2, 0, 1, false),
2391            (LOG3, 0, 1, false),
2392            (LOG4, 0, 1, false),
2393            (CREATE, 1, 2, false),
2394            (CREATE2, 1, 2, false),
2395            (RETURN, 0, 1, false),
2396            (REVERT, 0, 1, false),
2397        );
2398    }
2399
2400    #[cold]
2401    fn set_gas_limit_type(&mut self, interpreter: &mut Interpreter) {
2402        match interpreter.bytecode.opcode() {
2403            op::CREATE2 => self.dynamic_gas_limit = true,
2404            op::CALL => {
2405                // If first element of the stack is close to current remaining gas then assume
2406                // dynamic gas limit.
2407                self.dynamic_gas_limit =
2408                    try_or_return!(interpreter.stack.peek(0)) >= interpreter.gas.remaining() - 100
2409            }
2410            _ => self.dynamic_gas_limit = false,
2411        }
2412    }
2413}
2414
2415/// Helper that expands memory, stores a revert string pertaining to a disallowed memory write,
2416/// and sets the return range to the revert string's location in memory.
2417///
2418/// This will set the interpreter's next action to a return with the revert string as the output.
2419/// And trigger a revert.
2420fn disallowed_mem_write(
2421    dest_offset: u64,
2422    size: u64,
2423    interpreter: &mut Interpreter,
2424    ranges: &[Range<u64>],
2425) {
2426    let revert_string = format!(
2427        "memory write at offset 0x{:02X} of size 0x{:02X} not allowed; safe range: {}",
2428        dest_offset,
2429        size,
2430        ranges.iter().map(|r| format!("(0x{:02X}, 0x{:02X}]", r.start, r.end)).join(" U ")
2431    );
2432
2433    interpreter.bytecode.set_action(InterpreterAction::new_return(
2434        InstructionResult::Revert,
2435        Bytes::from(revert_string.into_bytes()),
2436        interpreter.gas,
2437    ));
2438}
2439
2440/// Returns true if the kind of account access is a call.
2441fn access_is_call(kind: crate::Vm::AccountAccessKind) -> bool {
2442    matches!(
2443        kind,
2444        crate::Vm::AccountAccessKind::Call
2445            | crate::Vm::AccountAccessKind::StaticCall
2446            | crate::Vm::AccountAccessKind::CallCode
2447            | crate::Vm::AccountAccessKind::DelegateCall
2448    )
2449}
2450
2451/// Records a log into the recorded logs vector, if it exists.
2452fn record_logs(recorded_logs: &mut Option<Vec<Vm::Log>>, log: &Log) {
2453    if let Some(storage_recorded_logs) = recorded_logs {
2454        storage_recorded_logs.push(Vm::Log {
2455            topics: log.data.topics().to_vec(),
2456            data: log.data.data.clone(),
2457            emitter: log.address,
2458        });
2459    }
2460}
2461
2462/// Appends an AccountAccess that resumes the recording of the current context.
2463fn append_storage_access(
2464    last: &mut Vec<AccountAccess>,
2465    storage_access: crate::Vm::StorageAccess,
2466    storage_depth: u64,
2467) {
2468    // Assert that there's an existing record for the current context.
2469    if !last.is_empty() && last.first().unwrap().depth < storage_depth {
2470        // Three cases to consider:
2471        // 1. If there hasn't been a context switch since the start of this context, then add the
2472        //    storage access to the current context record.
2473        // 2. If there's an existing Resume record, then add the storage access to it.
2474        // 3. Otherwise, create a new Resume record based on the current context.
2475        if last.len() == 1 {
2476            last.first_mut().unwrap().storageAccesses.push(storage_access);
2477        } else {
2478            let last_record = last.last_mut().unwrap();
2479            if last_record.kind as u8 == crate::Vm::AccountAccessKind::Resume as u8 {
2480                last_record.storageAccesses.push(storage_access);
2481            } else {
2482                let entry = last.first().unwrap();
2483                let resume_record = crate::Vm::AccountAccess {
2484                    chainInfo: crate::Vm::ChainInfo {
2485                        forkId: entry.chainInfo.forkId,
2486                        chainId: entry.chainInfo.chainId,
2487                    },
2488                    accessor: entry.accessor,
2489                    account: entry.account,
2490                    kind: crate::Vm::AccountAccessKind::Resume,
2491                    initialized: entry.initialized,
2492                    storageAccesses: vec![storage_access],
2493                    reverted: entry.reverted,
2494                    // The remaining fields are defaults
2495                    oldBalance: U256::ZERO,
2496                    newBalance: U256::ZERO,
2497                    oldNonce: 0,
2498                    newNonce: 0,
2499                    value: U256::ZERO,
2500                    data: Bytes::new(),
2501                    deployedCode: Bytes::new(),
2502                    depth: entry.depth,
2503                };
2504                last.push(resume_record);
2505            }
2506        }
2507    }
2508}
2509
2510/// Dispatches the cheatcode call to the appropriate function.
2511fn apply_dispatch(
2512    calls: &Vm::VmCalls,
2513    ccx: &mut CheatsCtxt,
2514    executor: &mut dyn CheatcodesExecutor,
2515) -> Result {
2516    let cheat = calls_as_dyn_cheatcode(calls);
2517
2518    let _guard = debug_span!(target: "cheatcodes", "apply", id = %cheat.id()).entered();
2519    trace!(target: "cheatcodes", ?cheat, "applying");
2520
2521    if let spec::Status::Deprecated(replacement) = *cheat.status() {
2522        ccx.state.deprecated.insert(cheat.signature(), replacement);
2523    }
2524
2525    // Apply the cheatcode.
2526    let mut result = cheat.dyn_apply(ccx, executor);
2527
2528    // Format the error message to include the cheatcode name.
2529    if let Err(e) = &mut result
2530        && e.is_str()
2531    {
2532        let name = cheat.name();
2533        // Skip showing the cheatcode name for:
2534        // - assertions: too verbose, and can already be inferred from the error message
2535        // - `rpcUrl`: forge-std relies on it in `getChainWithUpdatedRpcUrl`
2536        if !name.contains("assert") && name != "rpcUrl" {
2537            *e = fmt_err!("vm.{name}: {e}");
2538        }
2539    }
2540
2541    trace!(
2542        target: "cheatcodes",
2543        return = %match &result {
2544            Ok(b) => hex::encode(b),
2545            Err(e) => e.to_string(),
2546        }
2547    );
2548
2549    result
2550}
2551
2552fn calls_as_dyn_cheatcode(calls: &Vm::VmCalls) -> &dyn DynCheatcode {
2553    macro_rules! as_dyn {
2554        ($($variant:ident),*) => {
2555            match calls {
2556                $(Vm::VmCalls::$variant(cheat) => cheat,)*
2557            }
2558        };
2559    }
2560    vm_calls!(as_dyn)
2561}
2562
2563/// Helper function to check if frame execution will exit.
2564fn will_exit(action: &InterpreterAction) -> bool {
2565    match action {
2566        InterpreterAction::Return(result) => {
2567            result.result.is_ok_or_revert() || result.result.is_error()
2568        }
2569        _ => false,
2570    }
2571}