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_disable_default_create2_deployer(self.evm.disable_default_create2_deployer)
283            .with_slots_in_an_epoch(self.slots_in_an_epoch)
284            .with_memory_limit(self.evm.memory_limit)
285            .with_cache_path(self.cache_path))
286    }
287
288    fn account_generator(&self) -> AccountGenerator {
289        let mut generator = AccountGenerator::new(self.accounts as usize)
290            .phrase(DEFAULT_MNEMONIC)
291            .chain_id(self.evm.chain_id.unwrap_or(CHAIN_ID.into()));
292        if let Some(ref mnemonic) = self.mnemonic {
293            generator = generator.phrase(mnemonic);
294        } else if let Some(count) = self.mnemonic_random {
295            let mut rng = rand_08::thread_rng();
296            let mnemonic = match Mnemonic::<English>::new_with_count(&mut rng, count) {
297                Ok(mnemonic) => mnemonic.to_phrase(),
298                Err(_) => DEFAULT_MNEMONIC.to_string(),
299            };
300            generator = generator.phrase(mnemonic);
301        } else if let Some(seed) = self.mnemonic_seed {
302            let mut seed = StdRng::seed_from_u64(seed);
303            let mnemonic = Mnemonic::<English>::new(&mut seed).to_phrase();
304            generator = generator.phrase(mnemonic);
305        }
306        if let Some(ref derivation) = self.derivation_path {
307            generator = generator.derivation_path(derivation);
308        }
309        generator
310    }
311
312    /// Returns the location where to dump the state to.
313    fn dump_state_path(&self) -> Option<PathBuf> {
314        self.dump_state.as_ref().or_else(|| self.state.as_ref().map(|s| &s.path)).cloned()
315    }
316
317    /// Starts the node
318    ///
319    /// See also [crate::spawn()]
320    pub async fn run(self) -> eyre::Result<()> {
321        let dump_state = self.dump_state_path();
322        let dump_interval =
323            self.state_interval.map(Duration::from_secs).unwrap_or(DEFAULT_DUMP_INTERVAL);
324        let preserve_historical_states = self.preserve_historical_states;
325
326        let (api, mut handle) = crate::try_spawn(self.into_node_config()?).await?;
327
328        // sets the signal handler to gracefully shutdown.
329        let mut fork = api.get_fork();
330        let running = Arc::new(AtomicUsize::new(0));
331
332        // handle for the currently running rt, this must be obtained before setting the crtlc
333        // handler, See [Handle::current]
334        let mut signal = handle.shutdown_signal_mut().take();
335
336        let task_manager = handle.task_manager();
337        let mut on_shutdown = task_manager.on_shutdown();
338
339        let mut state_dumper =
340            PeriodicStateDumper::new(api, dump_state, dump_interval, preserve_historical_states);
341
342        task_manager.spawn(async move {
343            // wait for the SIGTERM signal on unix systems
344            #[cfg(unix)]
345            let mut sigterm = Box::pin(async {
346                if let Ok(mut stream) =
347                    tokio::signal::unix::signal(tokio::signal::unix::SignalKind::terminate())
348                {
349                    stream.recv().await;
350                } else {
351                    futures::future::pending::<()>().await;
352                }
353            });
354
355            // On windows, this will never fire.
356            #[cfg(not(unix))]
357            let mut sigterm = Box::pin(futures::future::pending::<()>());
358
359            // await shutdown signal but also periodically flush state
360            tokio::select! {
361                 _ = &mut sigterm => {
362                    trace!("received sigterm signal, shutting down");
363                }
364                _ = &mut on_shutdown => {}
365                _ = &mut state_dumper => {}
366            }
367
368            // shutdown received
369            state_dumper.dump().await;
370
371            // cleaning up and shutting down
372            // this will make sure that the fork RPC cache is flushed if caching is configured
373            if let Some(fork) = fork.take() {
374                trace!("flushing cache on shutdown");
375                fork.database
376                    .read()
377                    .await
378                    .maybe_flush_cache()
379                    .expect("Could not flush cache on fork DB");
380                // cleaning up and shutting down
381                // this will make sure that the fork RPC cache is flushed if caching is configured
382            }
383            std::process::exit(0);
384        });
385
386        ctrlc::set_handler(move || {
387            let prev = running.fetch_add(1, Ordering::SeqCst);
388            if prev == 0 {
389                trace!("received shutdown signal, shutting down");
390                let _ = signal.take();
391            }
392        })
393        .expect("Error setting Ctrl-C handler");
394
395        Ok(handle.await??)
396    }
397}
398
399/// Anvil's EVM related arguments.
400#[derive(Clone, Debug, Parser)]
401#[command(next_help_heading = "EVM options")]
402pub struct AnvilEvmArgs {
403    /// Fetch state over a remote endpoint instead of starting from an empty state.
404    ///
405    /// 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.
406    #[arg(
407        long,
408        short,
409        visible_alias = "rpc-url",
410        value_name = "URL",
411        help_heading = "Fork config"
412    )]
413    pub fork_url: Option<ForkUrl>,
414
415    /// Headers to use for the rpc client, e.g. "User-Agent: test-agent"
416    ///
417    /// See --fork-url.
418    #[arg(
419        long = "fork-header",
420        value_name = "HEADERS",
421        help_heading = "Fork config",
422        requires = "fork_url"
423    )]
424    pub fork_headers: Vec<String>,
425
426    /// Timeout in ms for requests sent to remote JSON-RPC server in forking mode.
427    ///
428    /// Default value 45000
429    #[arg(id = "timeout", long = "timeout", help_heading = "Fork config", requires = "fork_url")]
430    pub fork_request_timeout: Option<u64>,
431
432    /// Number of retry requests for spurious networks (timed out requests)
433    ///
434    /// Default value 5
435    #[arg(id = "retries", long = "retries", help_heading = "Fork config", requires = "fork_url")]
436    pub fork_request_retries: Option<u32>,
437
438    /// Fetch state from a specific block number over a remote endpoint.
439    ///
440    /// If negative, the given value is subtracted from the `latest` block number.
441    ///
442    /// See --fork-url.
443    #[arg(
444        long,
445        requires = "fork_url",
446        value_name = "BLOCK",
447        help_heading = "Fork config",
448        allow_hyphen_values = true
449    )]
450    pub fork_block_number: Option<i128>,
451
452    /// Fetch state from a specific transaction hash over a remote endpoint.
453    ///
454    /// See --fork-url.
455    #[arg(
456        long,
457        requires = "fork_url",
458        value_name = "TRANSACTION",
459        help_heading = "Fork config",
460        conflicts_with = "fork_block_number"
461    )]
462    pub fork_transaction_hash: Option<B256>,
463
464    /// Initial retry backoff on encountering errors.
465    ///
466    /// See --fork-url.
467    #[arg(long, requires = "fork_url", value_name = "BACKOFF", help_heading = "Fork config")]
468    pub fork_retry_backoff: Option<u64>,
469
470    /// Specify chain id to skip fetching it from remote endpoint. This enables offline-start mode.
471    ///
472    /// You still must pass both `--fork-url` and `--fork-block-number`, and already have your
473    /// required state cached on disk, anything missing locally would be fetched from the
474    /// remote.
475    #[arg(
476        long,
477        help_heading = "Fork config",
478        value_name = "CHAIN",
479        requires = "fork_block_number"
480    )]
481    pub fork_chain_id: Option<Chain>,
482
483    /// Sets the number of assumed available compute units per second for this provider
484    ///
485    /// default value: 330
486    ///
487    /// See also --fork-url and <https://docs.alchemy.com/reference/compute-units#what-are-cups-compute-units-per-second>
488    #[arg(
489        long,
490        requires = "fork_url",
491        alias = "cups",
492        value_name = "CUPS",
493        help_heading = "Fork config"
494    )]
495    pub compute_units_per_second: Option<u64>,
496
497    /// Disables rate limiting for this node's provider.
498    ///
499    /// default value: false
500    ///
501    /// See also --fork-url and <https://docs.alchemy.com/reference/compute-units#what-are-cups-compute-units-per-second>
502    #[arg(
503        long,
504        requires = "fork_url",
505        value_name = "NO_RATE_LIMITS",
506        help_heading = "Fork config",
507        visible_alias = "no-rpc-rate-limit"
508    )]
509    pub no_rate_limit: bool,
510
511    /// Explicitly disables the use of RPC caching.
512    ///
513    /// All storage slots are read entirely from the endpoint.
514    ///
515    /// This flag overrides the project's configuration file.
516    ///
517    /// See --fork-url.
518    #[arg(long, requires = "fork_url", help_heading = "Fork config")]
519    pub no_storage_caching: bool,
520
521    /// The block gas limit.
522    #[arg(long, alias = "block-gas-limit", help_heading = "Environment config")]
523    pub gas_limit: Option<u64>,
524
525    /// Disable the `call.gas_limit <= block.gas_limit` constraint.
526    #[arg(
527        long,
528        value_name = "DISABLE_GAS_LIMIT",
529        help_heading = "Environment config",
530        alias = "disable-gas-limit",
531        conflicts_with = "gas_limit"
532    )]
533    pub disable_block_gas_limit: bool,
534
535    /// EIP-170: Contract code size limit in bytes. Useful to increase this because of tests. To
536    /// disable entirely, use `--disable-code-size-limit`. By default, it is 0x6000 (~25kb).
537    #[arg(long, value_name = "CODE_SIZE", help_heading = "Environment config")]
538    pub code_size_limit: Option<usize>,
539
540    /// Disable EIP-170: Contract code size limit.
541    #[arg(
542        long,
543        value_name = "DISABLE_CODE_SIZE_LIMIT",
544        conflicts_with = "code_size_limit",
545        help_heading = "Environment config"
546    )]
547    pub disable_code_size_limit: bool,
548
549    /// The gas price.
550    #[arg(long, help_heading = "Environment config")]
551    pub gas_price: Option<u128>,
552
553    /// The base fee in a block.
554    #[arg(
555        long,
556        visible_alias = "base-fee",
557        value_name = "FEE",
558        help_heading = "Environment config"
559    )]
560    pub block_base_fee_per_gas: Option<u64>,
561
562    /// Disable the enforcement of a minimum suggested priority fee.
563    #[arg(long, visible_alias = "no-priority-fee", help_heading = "Environment config")]
564    pub disable_min_priority_fee: bool,
565
566    /// The chain ID.
567    #[arg(long, alias = "chain", help_heading = "Environment config")]
568    pub chain_id: Option<Chain>,
569
570    /// Enable steps tracing used for debug calls returning geth-style traces
571    #[arg(long, visible_alias = "tracing")]
572    pub steps_tracing: bool,
573
574    /// Disable printing of `console.log` invocations to stdout.
575    #[arg(long, visible_alias = "no-console-log")]
576    pub disable_console_log: bool,
577
578    /// Enable printing of traces for executed transactions and `eth_call` to stdout.
579    #[arg(long, visible_alias = "enable-trace-printing")]
580    pub print_traces: bool,
581
582    /// Enables automatic impersonation on startup. This allows any transaction sender to be
583    /// simulated as different accounts, which is useful for testing contract behavior.
584    #[arg(long, visible_alias = "auto-unlock")]
585    pub auto_impersonate: bool,
586
587    /// Run an Optimism chain
588    #[arg(long, visible_alias = "optimism")]
589    pub optimism: bool,
590
591    /// Disable the default create2 deployer
592    #[arg(long, visible_alias = "no-create2")]
593    pub disable_default_create2_deployer: bool,
594
595    /// The memory limit per EVM execution in bytes.
596    #[arg(long)]
597    pub memory_limit: Option<u64>,
598
599    /// Enable Odyssey features
600    #[arg(long, alias = "alphanet")]
601    pub odyssey: bool,
602}
603
604/// Resolves an alias passed as fork-url to the matching url defined in the rpc_endpoints section
605/// of the project configuration file.
606/// Does nothing if the fork-url is not a configured alias.
607impl AnvilEvmArgs {
608    pub fn resolve_rpc_alias(&mut self) {
609        if let Some(fork_url) = &self.fork_url
610            && let Ok(config) = Config::load_with_providers(FigmentProviders::Anvil)
611            && let Some(Ok(url)) = config.get_rpc_url_with_alias(&fork_url.url)
612        {
613            self.fork_url = Some(ForkUrl { url: url.to_string(), block: fork_url.block });
614        }
615    }
616}
617
618/// Helper type to periodically dump the state of the chain to disk
619struct PeriodicStateDumper {
620    in_progress_dump: Option<Pin<Box<dyn Future<Output = ()> + Send + Sync + 'static>>>,
621    api: EthApi,
622    dump_state: Option<PathBuf>,
623    preserve_historical_states: bool,
624    interval: Interval,
625}
626
627impl PeriodicStateDumper {
628    fn new(
629        api: EthApi,
630        dump_state: Option<PathBuf>,
631        interval: Duration,
632        preserve_historical_states: bool,
633    ) -> Self {
634        let dump_state = dump_state.map(|mut dump_state| {
635            if dump_state.is_dir() {
636                dump_state = dump_state.join("state.json");
637            }
638            dump_state
639        });
640
641        // periodically flush the state
642        let interval = tokio::time::interval_at(Instant::now() + interval, interval);
643        Self { in_progress_dump: None, api, dump_state, preserve_historical_states, interval }
644    }
645
646    async fn dump(&self) {
647        if let Some(state) = self.dump_state.clone() {
648            Self::dump_state(self.api.clone(), state, self.preserve_historical_states).await
649        }
650    }
651
652    /// Infallible state dump
653    async fn dump_state(api: EthApi, dump_state: PathBuf, preserve_historical_states: bool) {
654        trace!(path=?dump_state, "Dumping state on shutdown");
655        match api.serialized_state(preserve_historical_states).await {
656            Ok(state) => {
657                if let Err(err) = foundry_common::fs::write_json_file(&dump_state, &state) {
658                    error!(?err, "Failed to dump state");
659                } else {
660                    trace!(path=?dump_state, "Dumped state on shutdown");
661                }
662            }
663            Err(err) => {
664                error!(?err, "Failed to extract state");
665            }
666        }
667    }
668}
669
670// An endless future that periodically dumps the state to disk if configured.
671impl Future for PeriodicStateDumper {
672    type Output = ();
673
674    fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
675        let this = self.get_mut();
676        if this.dump_state.is_none() {
677            return Poll::Pending;
678        }
679
680        loop {
681            if let Some(mut flush) = this.in_progress_dump.take() {
682                match flush.poll_unpin(cx) {
683                    Poll::Ready(_) => {
684                        this.interval.reset();
685                    }
686                    Poll::Pending => {
687                        this.in_progress_dump = Some(flush);
688                        return Poll::Pending;
689                    }
690                }
691            }
692
693            if this.interval.poll_tick(cx).is_ready() {
694                let api = this.api.clone();
695                let path = this.dump_state.clone().expect("exists; see above");
696                this.in_progress_dump =
697                    Some(Box::pin(Self::dump_state(api, path, this.preserve_historical_states)));
698            } else {
699                break;
700            }
701        }
702
703        Poll::Pending
704    }
705}
706
707/// Represents the --state flag and where to load from, or dump the state to
708#[derive(Clone, Debug)]
709pub struct StateFile {
710    pub path: PathBuf,
711    pub state: Option<SerializableState>,
712}
713
714impl StateFile {
715    /// This is used as the clap `value_parser` implementation to parse from file but only if it
716    /// exists
717    fn parse(path: &str) -> Result<Self, String> {
718        Self::parse_path(path)
719    }
720
721    /// Parse from file but only if it exists
722    pub fn parse_path(path: impl AsRef<Path>) -> Result<Self, String> {
723        let mut path = path.as_ref().to_path_buf();
724        if path.is_dir() {
725            path = path.join("state.json");
726        }
727        let mut state = Self { path, state: None };
728        if !state.path.exists() {
729            return Ok(state);
730        }
731
732        state.state = Some(SerializableState::load(&state.path).map_err(|err| err.to_string())?);
733
734        Ok(state)
735    }
736}
737
738/// Represents the input URL for a fork with an optional trailing block number:
739/// `http://localhost:8545@1000000`
740#[derive(Clone, Debug, PartialEq, Eq)]
741pub struct ForkUrl {
742    /// The endpoint url
743    pub url: String,
744    /// Optional trailing block
745    pub block: Option<u64>,
746}
747
748impl fmt::Display for ForkUrl {
749    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
750        self.url.fmt(f)?;
751        if let Some(block) = self.block {
752            write!(f, "@{block}")?;
753        }
754        Ok(())
755    }
756}
757
758impl FromStr for ForkUrl {
759    type Err = String;
760
761    fn from_str(s: &str) -> Result<Self, Self::Err> {
762        if let Some((url, block)) = s.rsplit_once('@') {
763            if block == "latest" {
764                return Ok(Self { url: url.to_string(), block: None });
765            }
766            // this will prevent false positives for auths `user:password@example.com`
767            if !block.is_empty() && !block.contains(':') && !block.contains('.') {
768                let block: u64 = block
769                    .parse()
770                    .map_err(|_| format!("Failed to parse block number: `{block}`"))?;
771                return Ok(Self { url: url.to_string(), block: Some(block) });
772            }
773        }
774        Ok(Self { url: s.to_string(), block: None })
775    }
776}
777
778/// Clap's value parser for genesis. Loads a genesis.json file.
779fn read_genesis_file(path: &str) -> Result<Genesis, String> {
780    foundry_common::fs::read_json_file(path.as_ref()).map_err(|err| err.to_string())
781}
782
783fn duration_from_secs_f64(s: &str) -> Result<Duration, String> {
784    let s = s.parse::<f64>().map_err(|e| e.to_string())?;
785    if s == 0.0 {
786        return Err("Duration must be greater than 0".to_string());
787    }
788    Duration::try_from_secs_f64(s).map_err(|e| e.to_string())
789}
790
791#[cfg(test)]
792mod tests {
793    use super::*;
794    use std::{env, net::Ipv4Addr};
795
796    #[test]
797    fn test_parse_fork_url() {
798        let fork: ForkUrl = "http://localhost:8545@1000000".parse().unwrap();
799        assert_eq!(
800            fork,
801            ForkUrl { url: "http://localhost:8545".to_string(), block: Some(1000000) }
802        );
803
804        let fork: ForkUrl = "http://localhost:8545".parse().unwrap();
805        assert_eq!(fork, ForkUrl { url: "http://localhost:8545".to_string(), block: None });
806
807        let fork: ForkUrl = "wss://user:password@example.com/".parse().unwrap();
808        assert_eq!(
809            fork,
810            ForkUrl { url: "wss://user:password@example.com/".to_string(), block: None }
811        );
812
813        let fork: ForkUrl = "wss://user:password@example.com/@latest".parse().unwrap();
814        assert_eq!(
815            fork,
816            ForkUrl { url: "wss://user:password@example.com/".to_string(), block: None }
817        );
818
819        let fork: ForkUrl = "wss://user:password@example.com/@100000".parse().unwrap();
820        assert_eq!(
821            fork,
822            ForkUrl { url: "wss://user:password@example.com/".to_string(), block: Some(100000) }
823        );
824    }
825
826    #[test]
827    fn can_parse_ethereum_hardfork() {
828        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--hardfork", "berlin"]);
829        let config = args.into_node_config().unwrap();
830        assert_eq!(config.hardfork, Some(EthereumHardfork::Berlin.into()));
831    }
832
833    #[test]
834    fn can_parse_optimism_hardfork() {
835        let args: NodeArgs =
836            NodeArgs::parse_from(["anvil", "--optimism", "--hardfork", "Regolith"]);
837        let config = args.into_node_config().unwrap();
838        assert_eq!(config.hardfork, Some(OpHardfork::Regolith.into()));
839    }
840
841    #[test]
842    fn cant_parse_invalid_hardfork() {
843        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--hardfork", "Regolith"]);
844        let config = args.into_node_config();
845        assert!(config.is_err());
846    }
847
848    #[test]
849    fn can_parse_fork_headers() {
850        let args: NodeArgs = NodeArgs::parse_from([
851            "anvil",
852            "--fork-url",
853            "http,://localhost:8545",
854            "--fork-header",
855            "User-Agent: test-agent",
856            "--fork-header",
857            "Referrer: example.com",
858        ]);
859        assert_eq!(args.evm.fork_headers, vec!["User-Agent: test-agent", "Referrer: example.com"]);
860    }
861
862    #[test]
863    fn can_parse_prune_config() {
864        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--prune-history"]);
865        assert!(args.prune_history.is_some());
866
867        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--prune-history", "100"]);
868        assert_eq!(args.prune_history, Some(Some(100)));
869    }
870
871    #[test]
872    fn can_parse_max_persisted_states_config() {
873        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--max-persisted-states", "500"]);
874        assert_eq!(args.max_persisted_states, (Some(500)));
875    }
876
877    #[test]
878    fn can_parse_disable_block_gas_limit() {
879        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--disable-block-gas-limit"]);
880        assert!(args.evm.disable_block_gas_limit);
881
882        let args =
883            NodeArgs::try_parse_from(["anvil", "--disable-block-gas-limit", "--gas-limit", "100"]);
884        assert!(args.is_err());
885    }
886
887    #[test]
888    fn can_parse_disable_code_size_limit() {
889        let args: NodeArgs = NodeArgs::parse_from(["anvil", "--disable-code-size-limit"]);
890        assert!(args.evm.disable_code_size_limit);
891
892        let args = NodeArgs::try_parse_from([
893            "anvil",
894            "--disable-code-size-limit",
895            "--code-size-limit",
896            "100",
897        ]);
898        // can't be used together
899        assert!(args.is_err());
900    }
901
902    #[test]
903    fn can_parse_host() {
904        let args = NodeArgs::parse_from(["anvil"]);
905        assert_eq!(args.host, vec![IpAddr::V4(Ipv4Addr::LOCALHOST)]);
906
907        let args = NodeArgs::parse_from([
908            "anvil", "--host", "::1", "--host", "1.1.1.1", "--host", "2.2.2.2",
909        ]);
910        assert_eq!(
911            args.host,
912            ["::1", "1.1.1.1", "2.2.2.2"].map(|ip| ip.parse::<IpAddr>().unwrap()).to_vec()
913        );
914
915        let args = NodeArgs::parse_from(["anvil", "--host", "::1,1.1.1.1,2.2.2.2"]);
916        assert_eq!(
917            args.host,
918            ["::1", "1.1.1.1", "2.2.2.2"].map(|ip| ip.parse::<IpAddr>().unwrap()).to_vec()
919        );
920
921        unsafe { env::set_var("ANVIL_IP_ADDR", "1.1.1.1") };
922        let args = NodeArgs::parse_from(["anvil"]);
923        assert_eq!(args.host, vec!["1.1.1.1".parse::<IpAddr>().unwrap()]);
924
925        unsafe { env::set_var("ANVIL_IP_ADDR", "::1,1.1.1.1,2.2.2.2") };
926        let args = NodeArgs::parse_from(["anvil"]);
927        assert_eq!(
928            args.host,
929            ["::1", "1.1.1.1", "2.2.2.2"].map(|ip| ip.parse::<IpAddr>().unwrap()).to_vec()
930        );
931    }
932}