anvil/
cmd.rs

1use crate::{
2    AccountGenerator, CHAIN_ID, EthereumHardfork, NodeConfig,
3    config::{DEFAULT_MNEMONIC, ForkChoice},
4    eth::{EthApi, backend::db::SerializableState, pool::transactions::TransactionOrder},
5};
6use alloy_genesis::Genesis;
7use alloy_op_hardforks::OpHardfork;
8use alloy_primitives::{B256, U256, utils::Unit};
9use alloy_signer_local::coins_bip39::{English, Mnemonic};
10use anvil_server::ServerConfig;
11use clap::Parser;
12use core::fmt;
13use foundry_common::shell;
14use foundry_config::{Chain, Config, FigmentProviders};
15use futures::FutureExt;
16use rand_08::{SeedableRng, rngs::StdRng};
17use std::{
18    net::IpAddr,
19    path::{Path, PathBuf},
20    pin::Pin,
21    str::FromStr,
22    sync::{
23        Arc,
24        atomic::{AtomicUsize, Ordering},
25    },
26    task::{Context, Poll},
27    time::Duration,
28};
29use tokio::time::{Instant, Interval};
30
31#[derive(Clone, Debug, Parser)]
32pub struct NodeArgs {
33    /// Port number to listen on.
34    #[arg(long, short, default_value = "8545", value_name = "NUM")]
35    pub port: u16,
36
37    /// Number of dev accounts to generate and configure.
38    #[arg(long, short, default_value = "10", value_name = "NUM")]
39    pub accounts: u64,
40
41    /// The balance of every dev account in Ether.
42    #[arg(long, default_value = "10000", value_name = "NUM")]
43    pub balance: u64,
44
45    /// The timestamp of the genesis block.
46    #[arg(long, value_name = "NUM")]
47    pub timestamp: Option<u64>,
48
49    /// The number of the genesis block.
50    #[arg(long, value_name = "NUM")]
51    pub number: Option<u64>,
52
53    /// BIP39 mnemonic phrase used for generating accounts.
54    /// Cannot be used if `mnemonic_random` or `mnemonic_seed` are used.
55    #[arg(long, short, conflicts_with_all = &["mnemonic_seed", "mnemonic_random"])]
56    pub mnemonic: Option<String>,
57
58    /// Automatically generates a BIP39 mnemonic phrase, and derives accounts from it.
59    /// Cannot be used with other `mnemonic` options.
60    /// You can specify the number of words you want in the mnemonic.
61    /// [default: 12]
62    #[arg(long, conflicts_with_all = &["mnemonic", "mnemonic_seed"], default_missing_value = "12", num_args(0..=1))]
63    pub mnemonic_random: Option<usize>,
64
65    /// Generates a BIP39 mnemonic phrase from a given seed
66    /// Cannot be used with other `mnemonic` options.
67    ///
68    /// CAREFUL: This is NOT SAFE and should only be used for testing.
69    /// Never use the private keys generated in production.
70    #[arg(long = "mnemonic-seed-unsafe", conflicts_with_all = &["mnemonic", "mnemonic_random"])]
71    pub mnemonic_seed: Option<u64>,
72
73    /// Sets the derivation path of the child key to be derived.
74    ///
75    /// [default: m/44'/60'/0'/0/]
76    #[arg(long)]
77    pub derivation_path: Option<String>,
78
79    /// The EVM hardfork to use.
80    ///
81    /// Choose the hardfork by name, e.g. `prague`, `cancun`, `shanghai`, `paris`, `london`, etc...
82    /// [default: latest]
83    #[arg(long)]
84    pub hardfork: Option<String>,
85
86    /// Block time in seconds for interval mining.
87    #[arg(short, long, visible_alias = "blockTime", value_name = "SECONDS", value_parser = duration_from_secs_f64)]
88    pub block_time: Option<Duration>,
89
90    /// Slots in an epoch
91    #[arg(long, value_name = "SLOTS_IN_AN_EPOCH", default_value_t = 32)]
92    pub slots_in_an_epoch: u64,
93
94    /// Writes output of `anvil` as json to user-specified file.
95    #[arg(long, value_name = "FILE", value_hint = clap::ValueHint::FilePath)]
96    pub config_out: Option<PathBuf>,
97
98    /// Disable auto and interval mining, and mine on demand instead.
99    #[arg(long, visible_alias = "no-mine", conflicts_with = "block_time")]
100    pub no_mining: bool,
101
102    #[arg(long, visible_alias = "mixed-mining", requires = "block_time")]
103    pub mixed_mining: bool,
104
105    /// The hosts the server will listen on.
106    #[arg(
107        long,
108        value_name = "IP_ADDR",
109        env = "ANVIL_IP_ADDR",
110        default_value = "127.0.0.1",
111        help_heading = "Server options",
112        value_delimiter = ','
113    )]
114    pub host: Vec<IpAddr>,
115
116    /// How transactions are sorted in the mempool.
117    #[arg(long, default_value = "fees")]
118    pub order: TransactionOrder,
119
120    /// Initialize the genesis block with the given `genesis.json` file.
121    #[arg(long, value_name = "PATH", value_parser= read_genesis_file)]
122    pub init: Option<Genesis>,
123
124    /// This is an alias for both --load-state and --dump-state.
125    ///
126    /// It initializes the chain with the state and block environment stored at the file, if it
127    /// exists, and dumps the chain's state on exit.
128    #[arg(
129        long,
130        value_name = "PATH",
131        value_parser = StateFile::parse,
132        conflicts_with_all = &[
133            "init",
134            "dump_state",
135            "load_state"
136        ]
137    )]
138    pub state: Option<StateFile>,
139
140    /// Interval in seconds at which the state and block environment is to be dumped to disk.
141    ///
142    /// See --state and --dump-state
143    #[arg(short, long, value_name = "SECONDS")]
144    pub state_interval: Option<u64>,
145
146    /// Dump the state and block environment of chain on exit to the given file.
147    ///
148    /// If the value is a directory, the state will be written to `<VALUE>/state.json`.
149    #[arg(long, value_name = "PATH", conflicts_with = "init")]
150    pub dump_state: Option<PathBuf>,
151
152    /// Preserve historical state snapshots when dumping the state.
153    ///
154    /// This will save the in-memory states of the chain at particular block hashes.
155    ///
156    /// These historical states will be loaded into the memory when `--load-state` / `--state`, and
157    /// aids in RPC calls beyond the block at which state was dumped.
158    #[arg(long, conflicts_with = "init", default_value = "false")]
159    pub preserve_historical_states: bool,
160
161    /// Initialize the chain from a previously saved state snapshot.
162    #[arg(
163        long,
164        value_name = "PATH",
165        value_parser = SerializableState::parse,
166        conflicts_with = "init"
167    )]
168    pub load_state: Option<SerializableState>,
169
170    #[arg(long, help = IPC_HELP, value_name = "PATH", visible_alias = "ipcpath")]
171    pub ipc: Option<Option<String>>,
172
173    /// Don't keep full chain history.
174    /// If a number argument is specified, at most this number of states is kept in memory.
175    ///
176    /// If enabled, no state will be persisted on disk, so `max_persisted_states` will be 0.
177    #[arg(long)]
178    pub prune_history: Option<Option<usize>>,
179
180    /// Max number of states to persist on disk.
181    ///
182    /// Note that `prune_history` will overwrite `max_persisted_states` to 0.
183    #[arg(long, conflicts_with = "prune_history")]
184    pub max_persisted_states: Option<usize>,
185
186    /// Number of blocks with transactions to keep in memory.
187    #[arg(long)]
188    pub transaction_block_keeper: Option<usize>,
189
190    #[command(flatten)]
191    pub evm: AnvilEvmArgs,
192
193    #[command(flatten)]
194    pub server_config: ServerConfig,
195
196    /// Path to the cache directory where states are stored.    
197    #[arg(long, value_name = "PATH")]
198    pub cache_path: Option<PathBuf>,
199}
200
201#[cfg(windows)]
202const IPC_HELP: &str =
203    "Launch an ipc server at the given path or default path = `\\.\\pipe\\anvil.ipc`";
204
205/// The default IPC endpoint
206#[cfg(not(windows))]
207const IPC_HELP: &str = "Launch an ipc server at the given path or default path = `/tmp/anvil.ipc`";
208
209/// Default interval for periodically dumping the state.
210const DEFAULT_DUMP_INTERVAL: Duration = Duration::from_secs(60);
211
212impl NodeArgs {
213    pub fn into_node_config(self) -> eyre::Result<NodeConfig> {
214        let genesis_balance = Unit::ETHER.wei().saturating_mul(U256::from(self.balance));
215        let compute_units_per_second =
216            if self.evm.no_rate_limit { Some(u64::MAX) } else { self.evm.compute_units_per_second };
217
218        let hardfork = match &self.hardfork {
219            Some(hf) => {
220                if self.evm.optimism {
221                    Some(OpHardfork::from_str(hf)?.into())
222                } else {
223                    Some(EthereumHardfork::from_str(hf)?.into())
224                }
225            }
226            None => None,
227        };
228
229        Ok(NodeConfig::default()
230            .with_gas_limit(self.evm.gas_limit)
231            .disable_block_gas_limit(self.evm.disable_block_gas_limit)
232            .with_gas_price(self.evm.gas_price)
233            .with_hardfork(hardfork)
234            .with_blocktime(self.block_time)
235            .with_no_mining(self.no_mining)
236            .with_mixed_mining(self.mixed_mining, self.block_time)
237            .with_account_generator(self.account_generator())?
238            .with_genesis_balance(genesis_balance)
239            .with_genesis_timestamp(self.timestamp)
240            .with_genesis_block_number(self.number)
241            .with_port(self.port)
242            .with_fork_choice(match (self.evm.fork_block_number, self.evm.fork_transaction_hash) {
243                (Some(block), None) => Some(ForkChoice::Block(block)),
244                (None, Some(hash)) => Some(ForkChoice::Transaction(hash)),
245                _ => self
246                    .evm
247                    .fork_url
248                    .as_ref()
249                    .and_then(|f| f.block)
250                    .map(|num| ForkChoice::Block(num as i128)),
251            })
252            .with_fork_headers(self.evm.fork_headers)
253            .with_fork_chain_id(self.evm.fork_chain_id.map(u64::from).map(U256::from))
254            .fork_request_timeout(self.evm.fork_request_timeout.map(Duration::from_millis))
255            .fork_request_retries(self.evm.fork_request_retries)
256            .fork_retry_backoff(self.evm.fork_retry_backoff.map(Duration::from_millis))
257            .fork_compute_units_per_second(compute_units_per_second)
258            .with_eth_rpc_url(self.evm.fork_url.map(|fork| fork.url))
259            .with_base_fee(self.evm.block_base_fee_per_gas)
260            .disable_min_priority_fee(self.evm.disable_min_priority_fee)
261            .with_storage_caching(self.evm.no_storage_caching)
262            .with_server_config(self.server_config)
263            .with_host(self.host)
264            .set_silent(shell::is_quiet())
265            .set_config_out(self.config_out)
266            .with_chain_id(self.evm.chain_id)
267            .with_transaction_order(self.order)
268            .with_genesis(self.init)
269            .with_steps_tracing(self.evm.steps_tracing)
270            .with_print_logs(!self.evm.disable_console_log)
271            .with_print_traces(self.evm.print_traces)
272            .with_auto_impersonate(self.evm.auto_impersonate)
273            .with_ipc(self.ipc)
274            .with_code_size_limit(self.evm.code_size_limit)
275            .disable_code_size_limit(self.evm.disable_code_size_limit)
276            .set_pruned_history(self.prune_history)
277            .with_init_state(self.load_state.or_else(|| self.state.and_then(|s| s.state)))
278            .with_transaction_block_keeper(self.transaction_block_keeper)
279            .with_max_persisted_states(self.max_persisted_states)
280            .with_optimism(self.evm.optimism)
281            .with_odyssey(self.evm.odyssey)
282            .with_celo(self.evm.celo)
283            .with_disable_default_create2_deployer(self.evm.disable_default_create2_deployer)
284            .with_disable_pool_balance_checks(self.evm.disable_pool_balance_checks)
285            .with_slots_in_an_epoch(self.slots_in_an_epoch)
286            .with_memory_limit(self.evm.memory_limit)
287            .with_cache_path(self.cache_path))
288    }
289
290    fn account_generator(&self) -> AccountGenerator {
291        let mut generator = AccountGenerator::new(self.accounts as usize)
292            .phrase(DEFAULT_MNEMONIC)
293            .chain_id(self.evm.chain_id.unwrap_or(CHAIN_ID.into()));
294        if let Some(ref mnemonic) = self.mnemonic {
295            generator = generator.phrase(mnemonic);
296        } else if let Some(count) = self.mnemonic_random {
297            let mut rng = rand_08::thread_rng();
298            let mnemonic = match Mnemonic::<English>::new_with_count(&mut rng, count) {
299                Ok(mnemonic) => mnemonic.to_phrase(),
300                Err(_) => DEFAULT_MNEMONIC.to_string(),
301            };
302            generator = generator.phrase(mnemonic);
303        } else if let Some(seed) = self.mnemonic_seed {
304            let mut seed = StdRng::seed_from_u64(seed);
305            let mnemonic = Mnemonic::<English>::new(&mut seed).to_phrase();
306            generator = generator.phrase(mnemonic);
307        }
308        if let Some(ref derivation) = self.derivation_path {
309            generator = generator.derivation_path(derivation);
310        }
311        generator
312    }
313
314    /// Returns the location where to dump the state to.
315    fn dump_state_path(&self) -> Option<PathBuf> {
316        self.dump_state.as_ref().or_else(|| self.state.as_ref().map(|s| &s.path)).cloned()
317    }
318
319    /// Starts the node
320    ///
321    /// See also [crate::spawn()]
322    pub async fn run(self) -> eyre::Result<()> {
323        let dump_state = self.dump_state_path();
324        let dump_interval =
325            self.state_interval.map(Duration::from_secs).unwrap_or(DEFAULT_DUMP_INTERVAL);
326        let preserve_historical_states = self.preserve_historical_states;
327
328        let (api, mut handle) = crate::try_spawn(self.into_node_config()?).await?;
329
330        // sets the signal handler to gracefully shutdown.
331        let mut fork = api.get_fork();
332        let running = Arc::new(AtomicUsize::new(0));
333
334        // handle for the currently running rt, this must be obtained before setting the crtlc
335        // handler, See [Handle::current]
336        let mut signal = handle.shutdown_signal_mut().take();
337
338        let task_manager = handle.task_manager();
339        let mut on_shutdown = task_manager.on_shutdown();
340
341        let mut state_dumper =
342            PeriodicStateDumper::new(api, dump_state, dump_interval, preserve_historical_states);
343
344        task_manager.spawn(async move {
345            // wait for the SIGTERM signal on unix systems
346            #[cfg(unix)]
347            let mut sigterm = Box::pin(async {
348                if let Ok(mut stream) =
349                    tokio::signal::unix::signal(tokio::signal::unix::SignalKind::terminate())
350                {
351                    stream.recv().await;
352                } else {
353                    futures::future::pending::<()>().await;
354                }
355            });
356
357            // On windows, this will never fire.
358            #[cfg(not(unix))]
359            let mut sigterm = Box::pin(futures::future::pending::<()>());
360
361            // await shutdown signal but also periodically flush state
362            tokio::select! {
363                 _ = &mut sigterm => {
364                    trace!("received sigterm signal, shutting down");
365                }
366                _ = &mut on_shutdown => {}
367                _ = &mut state_dumper => {}
368            }
369
370            // shutdown received
371            state_dumper.dump().await;
372
373            // cleaning up and shutting down
374            // this will make sure that the fork RPC cache is flushed if caching is configured
375            if let Some(fork) = fork.take() {
376                trace!("flushing cache on shutdown");
377                fork.database
378                    .read()
379                    .await
380                    .maybe_flush_cache()
381                    .expect("Could not flush cache on fork DB");
382                // cleaning up and shutting down
383                // this will make sure that the fork RPC cache is flushed if caching is configured
384            }
385            std::process::exit(0);
386        });
387
388        ctrlc::set_handler(move || {
389            let prev = running.fetch_add(1, Ordering::SeqCst);
390            if prev == 0 {
391                trace!("received shutdown signal, shutting down");
392                let _ = signal.take();
393            }
394        })
395        .expect("Error setting Ctrl-C handler");
396
397        Ok(handle.await??)
398    }
399}
400
401/// Anvil's EVM related arguments.
402#[derive(Clone, Debug, Parser)]
403#[command(next_help_heading = "EVM options")]
404pub struct AnvilEvmArgs {
405    /// Fetch state over a remote endpoint instead of starting from an empty state.
406    ///
407    /// If you want to fetch state from a specific block number, add a block number like `http://localhost:8545@1400000` or use the `--fork-block-number` argument.
408    #[arg(
409        long,
410        short,
411        visible_alias = "rpc-url",
412        value_name = "URL",
413        help_heading = "Fork config"
414    )]
415    pub fork_url: Option<ForkUrl>,
416
417    /// Headers to use for the rpc client, e.g. "User-Agent: test-agent"
418    ///
419    /// See --fork-url.
420    #[arg(
421        long = "fork-header",
422        value_name = "HEADERS",
423        help_heading = "Fork config",
424        requires = "fork_url"
425    )]
426    pub fork_headers: Vec<String>,
427
428    /// Timeout in ms for requests sent to remote JSON-RPC server in forking mode.
429    ///
430    /// Default value 45000
431    #[arg(id = "timeout", long = "timeout", help_heading = "Fork config", requires = "fork_url")]
432    pub fork_request_timeout: Option<u64>,
433
434    /// Number of retry requests for spurious networks (timed out requests)
435    ///
436    /// Default value 5
437    #[arg(id = "retries", long = "retries", help_heading = "Fork config", requires = "fork_url")]
438    pub fork_request_retries: Option<u32>,
439
440    /// Fetch state from a specific block number over a remote endpoint.
441    ///
442    /// If negative, the given value is subtracted from the `latest` block number.
443    ///
444    /// See --fork-url.
445    #[arg(
446        long,
447        requires = "fork_url",
448        value_name = "BLOCK",
449        help_heading = "Fork config",
450        allow_hyphen_values = true
451    )]
452    pub fork_block_number: Option<i128>,
453
454    /// Fetch state from a specific transaction hash over a remote endpoint.
455    ///
456    /// See --fork-url.
457    #[arg(
458        long,
459        requires = "fork_url",
460        value_name = "TRANSACTION",
461        help_heading = "Fork config",
462        conflicts_with = "fork_block_number"
463    )]
464    pub fork_transaction_hash: Option<B256>,
465
466    /// Initial retry backoff on encountering errors.
467    ///
468    /// See --fork-url.
469    #[arg(long, requires = "fork_url", value_name = "BACKOFF", help_heading = "Fork config")]
470    pub fork_retry_backoff: Option<u64>,
471
472    /// Specify chain id to skip fetching it from remote endpoint. This enables offline-start mode.
473    ///
474    /// You still must pass both `--fork-url` and `--fork-block-number`, and already have your
475    /// required state cached on disk, anything missing locally would be fetched from the
476    /// remote.
477    #[arg(
478        long,
479        help_heading = "Fork config",
480        value_name = "CHAIN",
481        requires = "fork_block_number"
482    )]
483    pub fork_chain_id: Option<Chain>,
484
485    /// Sets the number of assumed available compute units per second for this provider
486    ///
487    /// default value: 330
488    ///
489    /// See also --fork-url and <https://docs.alchemy.com/reference/compute-units#what-are-cups-compute-units-per-second>
490    #[arg(
491        long,
492        requires = "fork_url",
493        alias = "cups",
494        value_name = "CUPS",
495        help_heading = "Fork config"
496    )]
497    pub compute_units_per_second: Option<u64>,
498
499    /// Disables rate limiting for this node's provider.
500    ///
501    /// default value: false
502    ///
503    /// See also --fork-url and <https://docs.alchemy.com/reference/compute-units#what-are-cups-compute-units-per-second>
504    #[arg(
505        long,
506        requires = "fork_url",
507        value_name = "NO_RATE_LIMITS",
508        help_heading = "Fork config",
509        visible_alias = "no-rpc-rate-limit"
510    )]
511    pub no_rate_limit: bool,
512
513    /// Explicitly disables the use of RPC caching.
514    ///
515    /// All storage slots are read entirely from the endpoint.
516    ///
517    /// This flag overrides the project's configuration file.
518    ///
519    /// See --fork-url.
520    #[arg(long, requires = "fork_url", help_heading = "Fork config")]
521    pub no_storage_caching: bool,
522
523    /// The block gas limit.
524    #[arg(long, alias = "block-gas-limit", help_heading = "Environment config")]
525    pub gas_limit: Option<u64>,
526
527    /// Disable the `call.gas_limit <= block.gas_limit` constraint.
528    #[arg(
529        long,
530        value_name = "DISABLE_GAS_LIMIT",
531        help_heading = "Environment config",
532        alias = "disable-gas-limit",
533        conflicts_with = "gas_limit"
534    )]
535    pub disable_block_gas_limit: bool,
536
537    /// EIP-170: Contract code size limit in bytes. Useful to increase this because of tests. To
538    /// disable entirely, use `--disable-code-size-limit`. By default, it is 0x6000 (~25kb).
539    #[arg(long, value_name = "CODE_SIZE", help_heading = "Environment config")]
540    pub code_size_limit: Option<usize>,
541
542    /// Disable EIP-170: Contract code size limit.
543    #[arg(
544        long,
545        value_name = "DISABLE_CODE_SIZE_LIMIT",
546        conflicts_with = "code_size_limit",
547        help_heading = "Environment config"
548    )]
549    pub disable_code_size_limit: bool,
550
551    /// The gas price.
552    #[arg(long, help_heading = "Environment config")]
553    pub gas_price: Option<u128>,
554
555    /// The base fee in a block.
556    #[arg(
557        long,
558        visible_alias = "base-fee",
559        value_name = "FEE",
560        help_heading = "Environment config"
561    )]
562    pub block_base_fee_per_gas: Option<u64>,
563
564    /// Disable the enforcement of a minimum suggested priority fee.
565    #[arg(long, visible_alias = "no-priority-fee", help_heading = "Environment config")]
566    pub disable_min_priority_fee: bool,
567
568    /// The chain ID.
569    #[arg(long, alias = "chain", help_heading = "Environment config")]
570    pub chain_id: Option<Chain>,
571
572    /// Enable steps tracing used for debug calls returning geth-style traces
573    #[arg(long, visible_alias = "tracing")]
574    pub steps_tracing: bool,
575
576    /// Disable printing of `console.log` invocations to stdout.
577    #[arg(long, visible_alias = "no-console-log")]
578    pub disable_console_log: bool,
579
580    /// Enable printing of traces for executed transactions and `eth_call` to stdout.
581    #[arg(long, visible_alias = "enable-trace-printing")]
582    pub print_traces: bool,
583
584    /// Enables automatic impersonation on startup. This allows any transaction sender to be
585    /// simulated as different accounts, which is useful for testing contract behavior.
586    #[arg(long, visible_alias = "auto-unlock")]
587    pub auto_impersonate: bool,
588
589    /// Run an Optimism chain
590    #[arg(long, visible_alias = "optimism")]
591    pub optimism: bool,
592
593    /// Disable the default create2 deployer
594    #[arg(long, visible_alias = "no-create2")]
595    pub disable_default_create2_deployer: bool,
596
597    /// Disable pool balance checks
598    #[arg(long)]
599    pub disable_pool_balance_checks: bool,
600
601    /// The memory limit per EVM execution in bytes.
602    #[arg(long)]
603    pub memory_limit: Option<u64>,
604
605    /// Enable Odyssey features
606    #[arg(long, alias = "alphanet")]
607    pub odyssey: bool,
608
609    /// Run a Celo chain
610    #[arg(long)]
611    pub celo: bool,
612}
613
614/// Resolves an alias passed as fork-url to the matching url defined in the rpc_endpoints section
615/// of the project configuration file.
616/// Does nothing if the fork-url is not a configured alias.
617impl AnvilEvmArgs {
618    pub fn resolve_rpc_alias(&mut self) {
619        if let Some(fork_url) = &self.fork_url
620            && let Ok(config) = Config::load_with_providers(FigmentProviders::Anvil)
621            && let Some(Ok(url)) = config.get_rpc_url_with_alias(&fork_url.url)
622        {
623            self.fork_url = Some(ForkUrl { url: url.to_string(), block: fork_url.block });
624        }
625    }
626}
627
628/// Helper type to periodically dump the state of the chain to disk
629struct PeriodicStateDumper {
630    in_progress_dump: Option<Pin<Box<dyn Future<Output = ()> + Send + Sync + 'static>>>,
631    api: EthApi,
632    dump_state: Option<PathBuf>,
633    preserve_historical_states: bool,
634    interval: Interval,
635}
636
637impl PeriodicStateDumper {
638    fn new(
639        api: EthApi,
640        dump_state: Option<PathBuf>,
641        interval: Duration,
642        preserve_historical_states: bool,
643    ) -> Self {
644        let dump_state = dump_state.map(|mut dump_state| {
645            if dump_state.is_dir() {
646                dump_state = dump_state.join("state.json");
647            }
648            dump_state
649        });
650
651        // periodically flush the state
652        let interval = tokio::time::interval_at(Instant::now() + interval, interval);
653        Self { in_progress_dump: None, api, dump_state, preserve_historical_states, interval }
654    }
655
656    async fn dump(&self) {
657        if let Some(state) = self.dump_state.clone() {
658            Self::dump_state(self.api.clone(), state, self.preserve_historical_states).await
659        }
660    }
661
662    /// Infallible state dump
663    async fn dump_state(api: EthApi, dump_state: PathBuf, preserve_historical_states: bool) {
664        trace!(path=?dump_state, "Dumping state on shutdown");
665        match api.serialized_state(preserve_historical_states).await {
666            Ok(state) => {
667                if let Err(err) = foundry_common::fs::write_json_file(&dump_state, &state) {
668                    error!(?err, "Failed to dump state");
669                } else {
670                    trace!(path=?dump_state, "Dumped state on shutdown");
671                }
672            }
673            Err(err) => {
674                error!(?err, "Failed to extract state");
675            }
676        }
677    }
678}
679
680// An endless future that periodically dumps the state to disk if configured.
681impl Future for PeriodicStateDumper {
682    type Output = ();
683
684    fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
685        let this = self.get_mut();
686        if this.dump_state.is_none() {
687            return Poll::Pending;
688        }
689
690        loop {
691            if let Some(mut flush) = this.in_progress_dump.take() {
692                match flush.poll_unpin(cx) {
693                    Poll::Ready(_) => {
694                        this.interval.reset();
695                    }
696                    Poll::Pending => {
697                        this.in_progress_dump = Some(flush);
698                        return Poll::Pending;
699                    }
700                }
701            }
702
703            if this.interval.poll_tick(cx).is_ready() {
704                let api = this.api.clone();
705                let path = this.dump_state.clone().expect("exists; see above");
706                this.in_progress_dump =
707                    Some(Box::pin(Self::dump_state(api, path, this.preserve_historical_states)));
708            } else {
709                break;
710            }
711        }
712
713        Poll::Pending
714    }
715}
716
717/// Represents the --state flag and where to load from, or dump the state to
718#[derive(Clone, Debug)]
719pub struct StateFile {
720    pub path: PathBuf,
721    pub state: Option<SerializableState>,
722}
723
724impl StateFile {
725    /// This is used as the clap `value_parser` implementation to parse from file but only if it
726    /// exists
727    fn parse(path: &str) -> Result<Self, String> {
728        Self::parse_path(path)
729    }
730
731    /// Parse from file but only if it exists
732    pub fn parse_path(path: impl AsRef<Path>) -> Result<Self, String> {
733        let mut path = path.as_ref().to_path_buf();
734        if path.is_dir() {
735            path = path.join("state.json");
736        }
737        let mut state = Self { path, state: None };
738        if !state.path.exists() {
739            return Ok(state);
740        }
741
742        state.state = Some(SerializableState::load(&state.path).map_err(|err| err.to_string())?);
743
744        Ok(state)
745    }
746}
747
748/// Represents the input URL for a fork with an optional trailing block number:
749/// `http://localhost:8545@1000000`
750#[derive(Clone, Debug, PartialEq, Eq)]
751pub struct ForkUrl {
752    /// The endpoint url
753    pub url: String,
754    /// Optional trailing block
755    pub block: Option<u64>,
756}
757
758impl fmt::Display for ForkUrl {
759    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
760        self.url.fmt(f)?;
761        if let Some(block) = self.block {
762            write!(f, "@{block}")?;
763        }
764        Ok(())
765    }
766}
767
768impl FromStr for ForkUrl {
769    type Err = String;
770
771    fn from_str(s: &str) -> Result<Self, Self::Err> {
772        if let Some((url, block)) = s.rsplit_once('@') {
773            if block == "latest" {
774                return Ok(Self { url: url.to_string(), block: None });
775            }
776            // this will prevent false positives for auths `user:password@example.com`
777            if !block.is_empty() && !block.contains(':') && !block.contains('.') {
778                let block: u64 = block
779                    .parse()
780                    .map_err(|_| format!("Failed to parse block number: `{block}`"))?;
781                return Ok(Self { url: url.to_string(), block: Some(block) });
782            }
783        }
784        Ok(Self { url: s.to_string(), block: None })
785    }
786}
787
788/// Clap's value parser for genesis. Loads a genesis.json file.
789fn read_genesis_file(path: &str) -> Result<Genesis, String> {
790    foundry_common::fs::read_json_file(path.as_ref()).map_err(|err| err.to_string())
791}
792
793fn duration_from_secs_f64(s: &str) -> Result<Duration, String> {
794    let s = s.parse::<f64>().map_err(|e| e.to_string())?;
795    if s == 0.0 {
796        return Err("Duration must be greater than 0".to_string());
797    }
798    Duration::try_from_secs_f64(s).map_err(|e| e.to_string())
799}
800
801#[cfg(test)]
802mod tests {
803    use super::*;
804    use std::{env, net::Ipv4Addr};
805
806    #[test]
807    fn test_parse_fork_url() {
808        let fork: ForkUrl = "http://localhost:8545@1000000".parse().unwrap();
809        assert_eq!(
810            fork,
811            ForkUrl { url: "http://localhost:8545".to_string(), block: Some(1000000) }
812        );
813
814        let fork: ForkUrl = "http://localhost:8545".parse().unwrap();
815        assert_eq!(fork, ForkUrl { url: "http://localhost:8545".to_string(), block: None });
816
817        let fork: ForkUrl = "wss://user:password@example.com/".parse().unwrap();
818        assert_eq!(
819            fork,
820            ForkUrl { url: "wss://user:password@example.com/".to_string(), block: None }
821        );
822
823        let fork: ForkUrl = "wss://user:password@example.com/@latest".parse().unwrap();
824        assert_eq!(
825            fork,
826            ForkUrl { url: "wss://user:password@example.com/".to_string(), block: None }
827        );
828
829        let fork: ForkUrl = "wss://user:password@example.com/@100000".parse().unwrap();
830        assert_eq!(
831            fork,
832            ForkUrl { url: "wss://user:password@example.com/".to_string(), block: Some(100000) }
833        );
834    }
835
836    #[test]
837    fn can_parse_ethereum_hardfork() {
838        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--hardfork", "berlin"]);
839        let config = args.into_node_config().unwrap();
840        assert_eq!(config.hardfork, Some(EthereumHardfork::Berlin.into()));
841    }
842
843    #[test]
844    fn can_parse_optimism_hardfork() {
845        let args: NodeArgs =
846            NodeArgs::parse_from(["anvil", "--optimism", "--hardfork", "Regolith"]);
847        let config = args.into_node_config().unwrap();
848        assert_eq!(config.hardfork, Some(OpHardfork::Regolith.into()));
849    }
850
851    #[test]
852    fn cant_parse_invalid_hardfork() {
853        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--hardfork", "Regolith"]);
854        let config = args.into_node_config();
855        assert!(config.is_err());
856    }
857
858    #[test]
859    fn can_parse_fork_headers() {
860        let args: NodeArgs = NodeArgs::parse_from([
861            "anvil",
862            "--fork-url",
863            "http,://localhost:8545",
864            "--fork-header",
865            "User-Agent: test-agent",
866            "--fork-header",
867            "Referrer: example.com",
868        ]);
869        assert_eq!(args.evm.fork_headers, vec!["User-Agent: test-agent", "Referrer: example.com"]);
870    }
871
872    #[test]
873    fn can_parse_prune_config() {
874        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--prune-history"]);
875        assert!(args.prune_history.is_some());
876
877        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--prune-history", "100"]);
878        assert_eq!(args.prune_history, Some(Some(100)));
879    }
880
881    #[test]
882    fn can_parse_max_persisted_states_config() {
883        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--max-persisted-states", "500"]);
884        assert_eq!(args.max_persisted_states, (Some(500)));
885    }
886
887    #[test]
888    fn can_parse_disable_block_gas_limit() {
889        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--disable-block-gas-limit"]);
890        assert!(args.evm.disable_block_gas_limit);
891
892        let args =
893            NodeArgs::try_parse_from(["anvil", "--disable-block-gas-limit", "--gas-limit", "100"]);
894        assert!(args.is_err());
895    }
896
897    #[test]
898    fn can_parse_disable_code_size_limit() {
899        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--disable-code-size-limit"]);
900        assert!(args.evm.disable_code_size_limit);
901
902        let args = NodeArgs::try_parse_from([
903            "anvil",
904            "--disable-code-size-limit",
905            "--code-size-limit",
906            "100",
907        ]);
908        // can't be used together
909        assert!(args.is_err());
910    }
911
912    #[test]
913    fn can_parse_host() {
914        let args = NodeArgs::parse_from(["anvil"]);
915        assert_eq!(args.host, vec![IpAddr::V4(Ipv4Addr::LOCALHOST)]);
916
917        let args = NodeArgs::parse_from([
918            "anvil", "--host", "::1", "--host", "1.1.1.1", "--host", "2.2.2.2",
919        ]);
920        assert_eq!(
921            args.host,
922            ["::1", "1.1.1.1", "2.2.2.2"].map(|ip| ip.parse::<IpAddr>().unwrap()).to_vec()
923        );
924
925        let args = NodeArgs::parse_from(["anvil", "--host", "::1,1.1.1.1,2.2.2.2"]);
926        assert_eq!(
927            args.host,
928            ["::1", "1.1.1.1", "2.2.2.2"].map(|ip| ip.parse::<IpAddr>().unwrap()).to_vec()
929        );
930
931        unsafe { env::set_var("ANVIL_IP_ADDR", "1.1.1.1") };
932        let args = NodeArgs::parse_from(["anvil"]);
933        assert_eq!(args.host, vec!["1.1.1.1".parse::<IpAddr>().unwrap()]);
934
935        unsafe { env::set_var("ANVIL_IP_ADDR", "::1,1.1.1.1,2.2.2.2") };
936        let args = NodeArgs::parse_from(["anvil"]);
937        assert_eq!(
938            args.host,
939            ["::1", "1.1.1.1", "2.2.2.2"].map(|ip| ip.parse::<IpAddr>().unwrap()).to_vec()
940        );
941    }
942}