forge_script/
broadcast.rs

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