foundry_cheatcodes/
inspector.rs

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