forge_script/
broadcast.rs

1use std::{cmp::Ordering, sync::Arc, time::Duration};
2
3use alloy_chains::{Chain, NamedChain};
4use alloy_consensus::TxEnvelope;
5use alloy_eips::{BlockId, eip2718::Encodable2718};
6use alloy_network::{AnyNetwork, EthereumWallet, TransactionBuilder};
7use alloy_primitives::{
8    Address, TxHash,
9    map::{AddressHashMap, AddressHashSet},
10    utils::format_units,
11};
12use alloy_provider::{Provider, utils::Eip1559Estimation};
13use alloy_rpc_types::TransactionRequest;
14use alloy_serde::WithOtherFields;
15use eyre::{Context, Result, bail};
16use forge_verify::provider::VerificationProviderType;
17use foundry_cheatcodes::Wallets;
18use foundry_cli::utils::{has_batch_support, has_different_gas_calc};
19use foundry_common::{
20    TransactionMaybeSigned,
21    provider::{RetryProvider, get_http_provider, try_get_http_provider},
22    shell,
23};
24use foundry_config::Config;
25use futures::{FutureExt, StreamExt, future::join_all, stream::FuturesUnordered};
26use itertools::Itertools;
27
28use crate::{
29    ScriptArgs, ScriptConfig, build::LinkedBuildData, progress::ScriptProgress,
30    sequence::ScriptSequenceKind, verify::BroadcastedState,
31};
32
33pub async fn estimate_gas<P: Provider<AnyNetwork>>(
34    tx: &mut WithOtherFields<TransactionRequest>,
35    provider: &P,
36    estimate_multiplier: u64,
37) -> Result<()> {
38    // if already set, some RPC endpoints might simply return the gas value that is already
39    // set in the request and omit the estimate altogether, so we remove it here
40    tx.gas = None;
41
42    tx.set_gas_limit(
43        provider.estimate_gas(tx.clone()).await.wrap_err("Failed to estimate gas for tx")?
44            * estimate_multiplier
45            / 100,
46    );
47    Ok(())
48}
49
50pub async fn next_nonce(
51    caller: Address,
52    provider_url: &str,
53    block_number: Option<u64>,
54) -> eyre::Result<u64> {
55    let provider = try_get_http_provider(provider_url)
56        .wrap_err_with(|| format!("bad fork_url provider: {provider_url}"))?;
57
58    let block_id = block_number.map_or(BlockId::latest(), BlockId::number);
59    Ok(provider.get_transaction_count(caller).block_id(block_id).await?)
60}
61
62/// Represents how to send a single transaction.
63#[derive(Clone)]
64pub enum SendTransactionKind<'a> {
65    Unlocked(WithOtherFields<TransactionRequest>),
66    Raw(WithOtherFields<TransactionRequest>, &'a EthereumWallet),
67    Signed(TxEnvelope),
68}
69
70impl<'a> SendTransactionKind<'a> {
71    /// Prepares the transaction for broadcasting by synchronizing nonce and estimating gas.
72    ///
73    /// This method performs two key operations:
74    /// 1. Nonce synchronization: Waits for the provider's nonce to catch up to the expected
75    ///    transaction nonce when doing sequential broadcast
76    /// 2. Gas estimation: Re-estimates gas right before broadcasting for chains that require it
77    pub async fn prepare(
78        &mut self,
79        provider: &RetryProvider,
80        sequential_broadcast: bool,
81        is_fixed_gas_limit: bool,
82        estimate_via_rpc: bool,
83        estimate_multiplier: u64,
84    ) -> Result<()> {
85        if let Self::Raw(tx, _) | Self::Unlocked(tx) = self {
86            if sequential_broadcast {
87                let from = tx.from.expect("no sender");
88
89                let tx_nonce = tx.nonce.expect("no nonce");
90                for attempt in 0..5 {
91                    let nonce = provider.get_transaction_count(from).await?;
92                    match nonce.cmp(&tx_nonce) {
93                        Ordering::Greater => {
94                            bail!(
95                                "EOA nonce changed unexpectedly while sending transactions. Expected {tx_nonce} got {nonce} from provider."
96                            )
97                        }
98                        Ordering::Less => {
99                            if attempt == 4 {
100                                bail!(
101                                    "After 5 attempts, provider nonce ({nonce}) is still behind expected nonce ({tx_nonce})."
102                                )
103                            }
104                            warn!(
105                                "Expected nonce ({tx_nonce}) is ahead of provider nonce ({nonce}). Retrying in 1 second..."
106                            );
107                            tokio::time::sleep(std::time::Duration::from_millis(1000)).await;
108                        }
109                        Ordering::Equal => {
110                            // Nonces are equal, we can proceed.
111                            break;
112                        }
113                    }
114                }
115            }
116
117            // Chains which use `eth_estimateGas` are being sent sequentially and require their
118            // gas to be re-estimated right before broadcasting.
119            if !is_fixed_gas_limit && estimate_via_rpc {
120                estimate_gas(tx, provider, estimate_multiplier).await?;
121            }
122        }
123
124        Ok(())
125    }
126
127    /// Sends the transaction to the network.
128    ///
129    /// Depending on the transaction kind, this will either:
130    /// - Submit via `eth_sendTransaction` for unlocked accounts
131    /// - Sign and submit via `eth_sendRawTransaction` for raw transactions
132    /// - Submit pre-signed transaction via `eth_sendRawTransaction`
133    pub async fn send(self, provider: Arc<RetryProvider>) -> Result<TxHash> {
134        let pending = match self {
135            Self::Unlocked(tx) => {
136                debug!("sending transaction from unlocked account {:?}", tx);
137
138                // Submit the transaction
139                provider.send_transaction(tx).await?
140            }
141            Self::Raw(tx, signer) => {
142                debug!("sending transaction: {:?}", tx);
143                let signed = tx.build(signer).await?;
144
145                // Submit the raw transaction
146                provider.send_raw_transaction(signed.encoded_2718().as_ref()).await?
147            }
148            Self::Signed(tx) => {
149                debug!("sending transaction: {:?}", tx);
150                provider.send_raw_transaction(tx.encoded_2718().as_ref()).await?
151            }
152        };
153
154        Ok(*pending.tx_hash())
155    }
156
157    /// Prepares and sends the transaction in one operation.
158    ///
159    /// This is a convenience method that combines [`prepare`](Self::prepare) and
160    /// [`send`](Self::send) into a single call.
161    pub async fn prepare_and_send(
162        mut self,
163        provider: Arc<RetryProvider>,
164        sequential_broadcast: bool,
165        is_fixed_gas_limit: bool,
166        estimate_via_rpc: bool,
167        estimate_multiplier: u64,
168    ) -> Result<TxHash> {
169        self.prepare(
170            &provider,
171            sequential_broadcast,
172            is_fixed_gas_limit,
173            estimate_via_rpc,
174            estimate_multiplier,
175        )
176        .await?;
177
178        self.send(provider).await
179    }
180}
181
182/// Represents how to send _all_ transactions
183pub enum SendTransactionsKind {
184    /// Send via `eth_sendTransaction` and rely on the  `from` address being unlocked.
185    Unlocked(AddressHashSet),
186    /// Send a signed transaction via `eth_sendRawTransaction`
187    Raw(AddressHashMap<EthereumWallet>),
188}
189
190impl SendTransactionsKind {
191    /// Returns the [`SendTransactionKind`] for the given address
192    ///
193    /// Returns an error if no matching signer is found or the address is not unlocked
194    pub fn for_sender(
195        &self,
196        addr: &Address,
197        tx: WithOtherFields<TransactionRequest>,
198    ) -> Result<SendTransactionKind<'_>> {
199        match self {
200            Self::Unlocked(unlocked) => {
201                if !unlocked.contains(addr) {
202                    bail!("Sender address {:?} is not unlocked", addr)
203                }
204                Ok(SendTransactionKind::Unlocked(tx))
205            }
206            Self::Raw(wallets) => {
207                if let Some(wallet) = wallets.get(addr) {
208                    Ok(SendTransactionKind::Raw(tx, wallet))
209                } else {
210                    bail!("No matching signer for {:?} found", addr)
211                }
212            }
213        }
214    }
215}
216
217/// State after we have bundled all
218/// [`TransactionWithMetadata`](forge_script_sequence::TransactionWithMetadata) objects into a
219/// single [`ScriptSequenceKind`] object containing one or more script sequences.
220pub struct BundledState {
221    pub args: ScriptArgs,
222    pub script_config: ScriptConfig,
223    pub script_wallets: Wallets,
224    pub build_data: LinkedBuildData,
225    pub sequence: ScriptSequenceKind,
226}
227
228impl BundledState {
229    pub async fn wait_for_pending(mut self) -> Result<Self> {
230        let progress = ScriptProgress::default();
231        let progress_ref = &progress;
232        let futs = self
233            .sequence
234            .sequences_mut()
235            .iter_mut()
236            .enumerate()
237            .map(|(sequence_idx, sequence)| async move {
238                let rpc_url = sequence.rpc_url();
239                let provider = Arc::new(get_http_provider(rpc_url));
240                progress_ref
241                    .wait_for_pending(
242                        sequence_idx,
243                        sequence,
244                        &provider,
245                        self.script_config.config.transaction_timeout,
246                    )
247                    .await
248            })
249            .collect::<Vec<_>>();
250
251        let errors = join_all(futs).await.into_iter().filter_map(Result::err).collect::<Vec<_>>();
252
253        self.sequence.save(true, false)?;
254
255        if !errors.is_empty() {
256            return Err(eyre::eyre!("{}", errors.iter().format("\n")));
257        }
258
259        Ok(self)
260    }
261
262    /// Broadcasts transactions from all sequences.
263    pub async fn broadcast(mut self) -> Result<BroadcastedState> {
264        let required_addresses = self
265            .sequence
266            .sequences()
267            .iter()
268            .flat_map(|sequence| {
269                sequence
270                    .transactions()
271                    .filter(|tx| tx.is_unsigned())
272                    .map(|tx| tx.from().expect("missing from"))
273            })
274            .collect::<AddressHashSet>();
275
276        if required_addresses.contains(&Config::DEFAULT_SENDER) {
277            eyre::bail!(
278                "You seem to be using Foundry's default sender. Be sure to set your own --sender."
279            );
280        }
281
282        let send_kind = if self.args.unlocked {
283            SendTransactionsKind::Unlocked(required_addresses.clone())
284        } else {
285            let signers = self.script_wallets.into_multi_wallet().into_signers()?;
286            let mut missing_addresses = Vec::new();
287
288            for addr in &required_addresses {
289                if !signers.contains_key(addr) {
290                    missing_addresses.push(addr);
291                }
292            }
293
294            if !missing_addresses.is_empty() {
295                eyre::bail!(
296                    "No associated wallet for addresses: {:?}. Unlocked wallets: {:?}",
297                    missing_addresses,
298                    signers.keys().collect::<Vec<_>>()
299                );
300            }
301
302            let signers = signers
303                .into_iter()
304                .map(|(addr, signer)| (addr, EthereumWallet::new(signer)))
305                .collect();
306
307            SendTransactionsKind::Raw(signers)
308        };
309
310        let progress = ScriptProgress::default();
311
312        for i in 0..self.sequence.sequences().len() {
313            let mut sequence = self.sequence.sequences_mut().get_mut(i).unwrap();
314
315            let provider = Arc::new(try_get_http_provider(sequence.rpc_url())?);
316            let already_broadcasted = sequence.receipts.len();
317
318            let seq_progress = progress.get_sequence_progress(i, sequence);
319
320            if already_broadcasted < sequence.transactions.len() {
321                let is_legacy = Chain::from(sequence.chain).is_legacy() || self.args.legacy;
322                // Make a one-time gas price estimation
323                let (gas_price, eip1559_fees) = match (
324                    is_legacy,
325                    self.args.with_gas_price,
326                    self.args.priority_gas_price,
327                ) {
328                    (true, Some(gas_price), _) => (Some(gas_price.to()), None),
329                    (true, None, _) => (Some(provider.get_gas_price().await?), None),
330                    (false, Some(max_fee_per_gas), Some(max_priority_fee_per_gas)) => (
331                        None,
332                        Some(Eip1559Estimation {
333                            max_fee_per_gas: max_fee_per_gas.to(),
334                            max_priority_fee_per_gas: max_priority_fee_per_gas.to(),
335                        }),
336                    ),
337                    (false, _, _) => {
338                        let mut fees = provider.estimate_eip1559_fees().await.wrap_err("Failed to estimate EIP1559 fees. This chain might not support EIP1559, try adding --legacy to your command.")?;
339
340                        if let Some(gas_price) = self.args.with_gas_price {
341                            fees.max_fee_per_gas = gas_price.to();
342                        }
343
344                        if let Some(priority_gas_price) = self.args.priority_gas_price {
345                            fees.max_priority_fee_per_gas = priority_gas_price.to();
346                        }
347
348                        (None, Some(fees))
349                    }
350                };
351
352                // Iterate through transactions, matching the `from` field with the associated
353                // wallet. Then send the transaction. Panics if we find a unknown `from`
354                let transactions = sequence
355                    .transactions
356                    .iter()
357                    .skip(already_broadcasted)
358                    .map(|tx_with_metadata| {
359                        let is_fixed_gas_limit = tx_with_metadata.is_fixed_gas_limit;
360
361                        let kind = match tx_with_metadata.tx().clone() {
362                            TransactionMaybeSigned::Signed { tx, .. } => {
363                                SendTransactionKind::Signed(tx)
364                            }
365                            TransactionMaybeSigned::Unsigned(mut tx) => {
366                                let from = tx.from.expect("No sender for onchain transaction!");
367
368                                tx.set_chain_id(sequence.chain);
369
370                                // Set TxKind::Create explicitly to satisfy `check_reqd_fields` in
371                                // alloy
372                                if tx.to.is_none() {
373                                    tx.set_create();
374                                }
375
376                                if let Some(gas_price) = gas_price {
377                                    tx.set_gas_price(gas_price);
378                                } else {
379                                    let eip1559_fees = eip1559_fees.expect("was set above");
380                                    tx.set_max_priority_fee_per_gas(
381                                        eip1559_fees.max_priority_fee_per_gas,
382                                    );
383                                    tx.set_max_fee_per_gas(eip1559_fees.max_fee_per_gas);
384                                }
385
386                                send_kind.for_sender(&from, tx)?
387                            }
388                        };
389
390                        Ok((kind, is_fixed_gas_limit))
391                    })
392                    .collect::<Result<Vec<_>>>()?;
393
394                let estimate_via_rpc =
395                    has_different_gas_calc(sequence.chain) || self.args.skip_simulation;
396
397                // We only wait for a transaction receipt before sending the next transaction, if
398                // there is more than one signer. There would be no way of assuring
399                // their order otherwise.
400                // Or if the chain does not support batched transactions (eg. Arbitrum).
401                // Or if we need to invoke eth_estimateGas before sending transactions.
402                let sequential_broadcast = estimate_via_rpc
403                    || self.args.slow
404                    || required_addresses.len() != 1
405                    || !has_batch_support(sequence.chain);
406
407                // We send transactions and wait for receipts in batches.
408                let batch_size = if sequential_broadcast { 1 } else { self.args.batch_size };
409                let mut index = already_broadcasted;
410
411                for (batch_number, batch) in transactions.chunks(batch_size).enumerate() {
412                    seq_progress.inner.write().set_status(&format!(
413                        "Sending transactions [{} - {}]",
414                        batch_number * batch_size,
415                        batch_number * batch_size + std::cmp::min(batch_size, batch.len()) - 1
416                    ));
417
418                    if !batch.is_empty() {
419                        let pending_transactions =
420                            batch.iter().map(|(kind, is_fixed_gas_limit)| {
421                                let provider = provider.clone();
422                                async move {
423                                    let res = kind
424                                        .clone()
425                                        .prepare_and_send(
426                                            provider,
427                                            sequential_broadcast,
428                                            *is_fixed_gas_limit,
429                                            estimate_via_rpc,
430                                            self.args.gas_estimate_multiplier,
431                                        )
432                                        .await;
433                                    (res, kind, 0, None)
434                                }
435                                .boxed()
436                            });
437
438                        let mut buffer = pending_transactions.collect::<FuturesUnordered<_>>();
439
440                        'send: while let Some((res, kind, attempt, original_res)) =
441                            buffer.next().await
442                        {
443                            if res.is_err() && attempt <= 3 {
444                                // Try to resubmit the transaction
445                                let provider = provider.clone();
446                                let progress = seq_progress.inner.clone();
447                                buffer.push(Box::pin(async move {
448                                    debug!(err=?res, ?attempt, "retrying transaction ");
449                                    let attempt = attempt + 1;
450                                    progress.write().set_status(&format!(
451                                        "retrying transaction {res:?} (attempt {attempt})"
452                                    ));
453                                    tokio::time::sleep(Duration::from_millis(1000 * attempt)).await;
454                                    let r = kind.clone().send(provider).await;
455                                    (r, kind, attempt, original_res.or(Some(res)))
456                                }));
457
458                                continue 'send;
459                            }
460
461                            // Preserve the original error if any
462                            let tx_hash = res.wrap_err_with(|| {
463                                if let Some(original_res) = original_res {
464                                    format!(
465                                        "Failed to send transaction after {attempt} attempts {original_res:?}"
466                                    )
467                                } else {
468                                    "Failed to send transaction".to_string()
469                                }
470                            })?;
471                            sequence.add_pending(index, tx_hash);
472
473                            // Checkpoint save
474                            self.sequence.save(true, false)?;
475                            sequence = self.sequence.sequences_mut().get_mut(i).unwrap();
476
477                            seq_progress.inner.write().tx_sent(tx_hash);
478                            index += 1;
479                        }
480
481                        // Checkpoint save
482                        self.sequence.save(true, false)?;
483                        sequence = self.sequence.sequences_mut().get_mut(i).unwrap();
484
485                        progress
486                            .wait_for_pending(
487                                i,
488                                sequence,
489                                &provider,
490                                self.script_config.config.transaction_timeout,
491                            )
492                            .await?
493                    }
494                    // Checkpoint save
495                    self.sequence.save(true, false)?;
496                    sequence = self.sequence.sequences_mut().get_mut(i).unwrap();
497                }
498            }
499
500            let (total_gas, total_gas_price, total_paid) =
501                sequence.receipts.iter().fold((0, 0, 0), |acc, receipt| {
502                    let gas_used = receipt.gas_used;
503                    let gas_price = receipt.effective_gas_price as u64;
504                    (acc.0 + gas_used, acc.1 + gas_price, acc.2 + gas_used * gas_price)
505                });
506            let paid = format_units(total_paid, 18).unwrap_or_else(|_| "N/A".to_string());
507            let avg_gas_price = format_units(total_gas_price / sequence.receipts.len() as u64, 9)
508                .unwrap_or_else(|_| "N/A".to_string());
509
510            let token_symbol = NamedChain::try_from(sequence.chain)
511                .unwrap_or_default()
512                .native_currency_symbol()
513                .unwrap_or("ETH");
514            seq_progress.inner.write().set_status(&format!(
515                "Total Paid: {} {} ({} gas * avg {} gwei)\n",
516                paid.trim_end_matches('0'),
517                token_symbol,
518                total_gas,
519                avg_gas_price.trim_end_matches('0').trim_end_matches('.')
520            ));
521            seq_progress.inner.write().finish();
522        }
523
524        if !shell::is_json() {
525            sh_println!("\n\n==========================")?;
526            sh_println!("\nONCHAIN EXECUTION COMPLETE & SUCCESSFUL.")?;
527        }
528
529        Ok(BroadcastedState {
530            args: self.args,
531            script_config: self.script_config,
532            build_data: self.build_data,
533            sequence: self.sequence,
534        })
535    }
536
537    pub fn verify_preflight_check(&self) -> Result<()> {
538        for sequence in self.sequence.sequences() {
539            if self.args.verifier.verifier == VerificationProviderType::Etherscan
540                && self
541                    .script_config
542                    .config
543                    .get_etherscan_api_key(Some(sequence.chain.into()))
544                    .is_none()
545            {
546                eyre::bail!("Missing etherscan key for chain {}", sequence.chain);
547            }
548        }
549
550        Ok(())
551    }
552}