reth_rpc_eth_api/helpers/
transaction.rs

1//! Database access for `eth_` transaction RPC methods. Loads transaction and receipt data w.r.t.
2//! network.
3
4use alloy_consensus::{BlockHeader, Transaction, Typed2718};
5use alloy_dyn_abi::TypedData;
6use alloy_eips::{eip2718::Encodable2718, BlockId};
7use alloy_network::TransactionBuilder;
8use alloy_primitives::{Address, Bytes, TxHash, B256};
9use alloy_rpc_types_eth::{transaction::TransactionRequest, BlockNumberOrTag, TransactionInfo};
10use futures::Future;
11use reth_node_api::BlockBody;
12use reth_primitives::{
13    transaction::SignedTransactionIntoRecoveredExt, SealedBlockWithSenders, TransactionMeta,
14};
15use reth_primitives_traits::SignedTransaction;
16use reth_provider::{
17    BlockNumReader, BlockReaderIdExt, ProviderBlock, ProviderReceipt, ProviderTx, ReceiptProvider,
18    TransactionsProvider,
19};
20use reth_rpc_eth_types::{utils::binary_search, EthApiError, SignError, TransactionSource};
21use reth_rpc_types_compat::transaction::{from_recovered, from_recovered_with_block_context};
22use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool};
23use std::sync::Arc;
24
25use super::{EthApiSpec, EthSigner, LoadBlock, LoadReceipt, LoadState, SpawnBlocking};
26use crate::{
27    helpers::estimate::EstimateCall, FromEthApiError, FullEthApiTypes, IntoEthApiError,
28    RpcNodeCore, RpcNodeCoreExt, RpcReceipt, RpcTransaction,
29};
30
31/// Transaction related functions for the [`EthApiServer`](crate::EthApiServer) trait in
32/// the `eth_` namespace.
33///
34/// This includes utilities for transaction tracing, transacting and inspection.
35///
36/// Async functions that are spawned onto the
37/// [`BlockingTaskPool`](reth_tasks::pool::BlockingTaskPool) begin with `spawn_`
38///
39/// ## Calls
40///
41/// There are subtle differences between when transacting [`TransactionRequest`]:
42///
43/// The endpoints `eth_call` and `eth_estimateGas` and `eth_createAccessList` should always
44/// __disable__ the base fee check in the
45/// [`EnvWithHandlerCfg`](revm_primitives::CfgEnvWithHandlerCfg).
46///
47/// The behaviour for tracing endpoints is not consistent across clients.
48/// Geth also disables the basefee check for tracing: <https://github.com/ethereum/go-ethereum/blob/bc0b87ca196f92e5af49bd33cc190ef0ec32b197/eth/tracers/api.go#L955-L955>
49/// Erigon does not: <https://github.com/ledgerwatch/erigon/blob/aefb97b07d1c4fd32a66097a24eddd8f6ccacae0/turbo/transactions/tracing.go#L209-L209>
50///
51/// See also <https://github.com/paradigmxyz/reth/issues/6240>
52///
53/// This implementation follows the behaviour of Geth and disables the basefee check for tracing.
54pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
55    /// Returns a handle for signing data.
56    ///
57    /// Singer access in default (L1) trait method implementations.
58    #[expect(clippy::type_complexity)]
59    fn signers(&self) -> &parking_lot::RwLock<Vec<Box<dyn EthSigner<ProviderTx<Self::Provider>>>>>;
60
61    /// Decodes and recovers the transaction and submits it to the pool.
62    ///
63    /// Returns the hash of the transaction.
64    fn send_raw_transaction(
65        &self,
66        tx: Bytes,
67    ) -> impl Future<Output = Result<B256, Self::Error>> + Send;
68
69    /// Decodes and recovers the transaction and submits it to the pool.
70    ///
71    /// Returns the hash of the transaction.
72    fn send_typed_data_transaction(
73        &self,
74        tx: alloy_eips::eip712::TypedDataRequest,
75    ) -> impl Future<Output = Result<B256, Self::Error>> + Send;
76
77    /// Returns the transaction by hash.
78    ///
79    /// Checks the pool and state.
80    ///
81    /// Returns `Ok(None)` if no matching transaction was found.
82    #[expect(clippy::complexity)]
83    fn transaction_by_hash(
84        &self,
85        hash: B256,
86    ) -> impl Future<
87        Output = Result<Option<TransactionSource<ProviderTx<Self::Provider>>>, Self::Error>,
88    > + Send {
89        LoadTransaction::transaction_by_hash(self, hash)
90    }
91
92    /// Get all transactions in the block with the given hash.
93    ///
94    /// Returns `None` if block does not exist.
95    #[expect(clippy::type_complexity)]
96    fn transactions_by_block(
97        &self,
98        block: B256,
99    ) -> impl Future<Output = Result<Option<Vec<ProviderTx<Self::Provider>>>, Self::Error>> + Send
100    {
101        async move {
102            self.cache()
103                .get_sealed_block_with_senders(block)
104                .await
105                .map(|b| b.map(|b| b.body.transactions().to_vec()))
106                .map_err(Self::Error::from_eth_err)
107        }
108    }
109
110    /// Returns the EIP-2718 encoded transaction by hash.
111    ///
112    /// If this is a pooled EIP-4844 transaction, the blob sidecar is included.
113    ///
114    /// Checks the pool and state.
115    ///
116    /// Returns `Ok(None)` if no matching transaction was found.
117    fn raw_transaction_by_hash(
118        &self,
119        hash: B256,
120    ) -> impl Future<Output = Result<Option<Bytes>, Self::Error>> + Send {
121        async move {
122            // Note: this is mostly used to fetch pooled transactions so we check the pool first
123            if let Some(tx) =
124                self.pool().get_pooled_transaction_element(hash).map(|tx| tx.encoded_2718().into())
125            {
126                return Ok(Some(tx))
127            }
128
129            self.spawn_blocking_io(move |ref this| {
130                Ok(this
131                    .provider()
132                    .transaction_by_hash(hash)
133                    .map_err(Self::Error::from_eth_err)?
134                    .map(|tx| tx.encoded_2718().into()))
135            })
136            .await
137        }
138    }
139
140    /// Returns the _historical_ transaction and the block it was mined in
141    #[expect(clippy::type_complexity)]
142    fn historical_transaction_by_hash_at(
143        &self,
144        hash: B256,
145    ) -> impl Future<
146        Output = Result<Option<(TransactionSource<ProviderTx<Self::Provider>>, B256)>, Self::Error>,
147    > + Send {
148        async move {
149            match self.transaction_by_hash_at(hash).await? {
150                None => Ok(None),
151                Some((tx, at)) => Ok(at.as_block_hash().map(|hash| (tx, hash))),
152            }
153        }
154    }
155
156    /// Returns the transaction receipt for the given hash.
157    ///
158    /// Returns None if the transaction does not exist or is pending
159    /// Note: The tx receipt is not available for pending transactions.
160    fn transaction_receipt(
161        &self,
162        hash: B256,
163    ) -> impl Future<Output = Result<Option<RpcReceipt<Self::NetworkTypes>>, Self::Error>> + Send
164    where
165        Self: LoadReceipt + 'static,
166    {
167        async move {
168            match self.load_transaction_and_receipt(hash).await? {
169                Some((tx, meta, receipt)) => {
170                    self.build_transaction_receipt(tx, meta, receipt).await.map(Some)
171                }
172                None => Ok(None),
173            }
174        }
175    }
176
177    /// Helper method that loads a transaction and its receipt.
178    #[expect(clippy::complexity)]
179    fn load_transaction_and_receipt(
180        &self,
181        hash: TxHash,
182    ) -> impl Future<
183        Output = Result<
184            Option<(ProviderTx<Self::Provider>, TransactionMeta, ProviderReceipt<Self::Provider>)>,
185            Self::Error,
186        >,
187    > + Send
188    where
189        Self: 'static,
190    {
191        let provider = self.provider().clone();
192        self.spawn_blocking_io(move |_| {
193            let (tx, meta) = match provider
194                .transaction_by_hash_with_meta(hash)
195                .map_err(Self::Error::from_eth_err)?
196            {
197                Some((tx, meta)) => (tx, meta),
198                None => return Ok(None),
199            };
200
201            let receipt = match provider.receipt_by_hash(hash).map_err(Self::Error::from_eth_err)? {
202                Some(recpt) => recpt,
203                None => return Ok(None),
204            };
205
206            Ok(Some((tx, meta, receipt)))
207        })
208    }
209
210    /// Get transaction by [`BlockId`] and index of transaction within that block.
211    ///
212    /// Returns `Ok(None)` if the block does not exist, or index is out of range.
213    fn transaction_by_block_and_tx_index(
214        &self,
215        block_id: BlockId,
216        index: usize,
217    ) -> impl Future<Output = Result<Option<RpcTransaction<Self::NetworkTypes>>, Self::Error>> + Send
218    where
219        Self: LoadBlock,
220    {
221        async move {
222            if let Some(block) = self.block_with_senders(block_id).await? {
223                let block_hash = block.hash();
224                let block_number = block.number();
225                let base_fee_per_gas = block.base_fee_per_gas();
226                if let Some((signer, tx)) = block.transactions_with_sender().nth(index) {
227                    let tx_type = Some(tx.ty() as isize);
228                    let tx_info = TransactionInfo {
229                        hash: Some(*tx.tx_hash()),
230                        block_hash: Some(block_hash),
231                        block_number: Some(block_number),
232                        base_fee: base_fee_per_gas.map(u128::from),
233                        index: Some(index as u64),
234                        tx_type,
235                    };
236
237                    return Ok(Some(from_recovered_with_block_context(
238                        tx.clone().with_signer(*signer),
239                        tx_info,
240                        self.tx_resp_builder(),
241                    )?))
242                }
243            }
244
245            Ok(None)
246        }
247    }
248
249    /// Find a transaction by sender's address and nonce.
250    fn get_transaction_by_sender_and_nonce(
251        &self,
252        sender: Address,
253        nonce: u64,
254        include_pending: bool,
255    ) -> impl Future<Output = Result<Option<RpcTransaction<Self::NetworkTypes>>, Self::Error>> + Send
256    where
257        Self: LoadBlock + LoadState,
258    {
259        async move {
260            // Check the pool first
261            if include_pending {
262                if let Some(tx) =
263                    RpcNodeCore::pool(self).get_transaction_by_sender_and_nonce(sender, nonce)
264                {
265                    let transaction = tx.transaction.clone_into_consensus();
266                    return Ok(Some(from_recovered(transaction, self.tx_resp_builder())?));
267                }
268            }
269
270            // Check if the sender is a contract
271            if self.get_code(sender, None).await?.len() > 0 {
272                return Ok(None);
273            }
274
275            let highest = self.transaction_count(sender, None).await?.saturating_to::<u64>();
276
277            // If the nonce is higher or equal to the highest nonce, the transaction is pending or
278            // not exists.
279            if nonce >= highest {
280                return Ok(None);
281            }
282
283            let Ok(high) = self.provider().best_block_number() else {
284                return Err(EthApiError::HeaderNotFound(BlockNumberOrTag::Latest.into()).into());
285            };
286
287            // Perform a binary search over the block range to find the block in which the sender's
288            // nonce reached the requested nonce.
289            let num = binary_search::<_, _, Self::Error>(1, high, |mid| async move {
290                let mid_nonce =
291                    self.transaction_count(sender, Some(mid.into())).await?.saturating_to::<u64>();
292
293                Ok(mid_nonce > nonce)
294            })
295            .await?;
296
297            let block_id = num.into();
298            self.block_with_senders(block_id)
299                .await?
300                .and_then(|block| {
301                    let block_hash = block.hash();
302                    let block_number = block.number();
303                    let base_fee_per_gas = block.base_fee_per_gas();
304
305                    block
306                        .transactions_with_sender()
307                        .enumerate()
308                        .find(|(_, (signer, tx))| **signer == sender && (*tx).nonce() == nonce)
309                        .map(|(index, (signer, tx))| {
310                            let tx_info = TransactionInfo {
311                                hash: Some(*tx.tx_hash()),
312                                block_hash: Some(block_hash),
313                                block_number: Some(block_number),
314                                base_fee: base_fee_per_gas.map(u128::from),
315                                index: Some(index as u64),
316                                tx_type: Some(tx.ty() as isize),
317                            };
318                            from_recovered_with_block_context(
319                                tx.clone().with_signer(*signer),
320                                tx_info,
321                                self.tx_resp_builder(),
322                            )
323                        })
324                })
325                .ok_or(EthApiError::HeaderNotFound(block_id))?
326                .map(Some)
327        }
328    }
329
330    /// Get transaction, as raw bytes, by [`BlockId`] and index of transaction within that block.
331    ///
332    /// Returns `Ok(None)` if the block does not exist, or index is out of range.
333    fn raw_transaction_by_block_and_tx_index(
334        &self,
335        block_id: BlockId,
336        index: usize,
337    ) -> impl Future<Output = Result<Option<Bytes>, Self::Error>> + Send
338    where
339        Self: LoadBlock,
340    {
341        async move {
342            if let Some(block) = self.block_with_senders(block_id).await? {
343                if let Some(tx) = block.transactions().get(index) {
344                    return Ok(Some(tx.encoded_2718().into()))
345                }
346            }
347
348            Ok(None)
349        }
350    }
351
352    /// Signs transaction with a matching signer, if any and submits the transaction to the pool.
353    /// Returns the hash of the signed transaction.
354    fn send_transaction(
355        &self,
356        mut request: TransactionRequest,
357    ) -> impl Future<Output = Result<B256, Self::Error>> + Send
358    where
359        Self: EthApiSpec + LoadBlock + EstimateCall,
360    {
361        async move {
362            let from = match request.from {
363                Some(from) => from,
364                None => return Err(SignError::NoAccount.into_eth_err()),
365            };
366
367            if self.find_signer(&from).is_err() {
368                return Err(SignError::NoAccount.into_eth_err())
369            }
370
371            // set nonce if not already set before
372            if request.nonce.is_none() {
373                let nonce = self.next_available_nonce(from).await?;
374                request.nonce = Some(nonce);
375            }
376
377            let chain_id = self.chain_id();
378            request.chain_id = Some(chain_id.to());
379
380            let estimated_gas =
381                self.estimate_gas_at(request.clone(), BlockId::pending(), None).await?;
382            let gas_limit = estimated_gas;
383            request.set_gas_limit(gas_limit.to());
384
385            let transaction = self.sign_request(&from, request).await?.with_signer(from);
386
387            let pool_transaction =
388                <<Self as RpcNodeCore>::Pool as TransactionPool>::Transaction::try_from_consensus(
389                    transaction,
390                )
391                .map_err(|_| EthApiError::TransactionConversionError)?;
392
393            // submit the transaction to the pool with a `Local` origin
394            let hash = self
395                .pool()
396                .add_transaction(TransactionOrigin::Local, pool_transaction)
397                .await
398                .map_err(Self::Error::from_eth_err)?;
399
400            Ok(hash)
401        }
402    }
403
404    /// Signs a transaction, with configured signers.
405    fn sign_request(
406        &self,
407        from: &Address,
408        txn: TransactionRequest,
409    ) -> impl Future<Output = Result<ProviderTx<Self::Provider>, Self::Error>> + Send {
410        async move {
411            self.find_signer(from)?
412                .sign_transaction(txn, from)
413                .await
414                .map_err(Self::Error::from_eth_err)
415        }
416    }
417
418    /// Signs given message. Returns the signature.
419    fn sign(
420        &self,
421        account: Address,
422        message: Bytes,
423    ) -> impl Future<Output = Result<Bytes, Self::Error>> + Send {
424        async move {
425            Ok(self
426                .find_signer(&account)?
427                .sign(account, &message)
428                .await
429                .map_err(Self::Error::from_eth_err)?
430                .as_bytes()
431                .into())
432        }
433    }
434
435    /// Signs a transaction request using the given account in request
436    /// Returns the EIP-2718 encoded signed transaction.
437    fn sign_transaction(
438        &self,
439        request: TransactionRequest,
440    ) -> impl Future<Output = Result<Bytes, Self::Error>> + Send {
441        async move {
442            let from = match request.from {
443                Some(from) => from,
444                None => return Err(SignError::NoAccount.into_eth_err()),
445            };
446
447            Ok(self.sign_request(&from, request).await?.encoded_2718().into())
448        }
449    }
450
451    /// Encodes and signs the typed data according EIP-712. Payload must implement Eip712 trait.
452    fn sign_typed_data(&self, data: &TypedData, account: Address) -> Result<Bytes, Self::Error> {
453        Ok(self
454            .find_signer(&account)?
455            .sign_typed_data(account, data)
456            .map_err(Self::Error::from_eth_err)?
457            .as_bytes()
458            .into())
459    }
460
461    /// Returns the signer for the given account, if found in configured signers.
462    #[expect(clippy::type_complexity)]
463    fn find_signer(
464        &self,
465        account: &Address,
466    ) -> Result<Box<(dyn EthSigner<ProviderTx<Self::Provider>> + 'static)>, Self::Error> {
467        self.signers()
468            .read()
469            .iter()
470            .find(|signer| signer.is_signer_for(account))
471            .map(|signer| dyn_clone::clone_box(&**signer))
472            .ok_or_else(|| SignError::NoAccount.into_eth_err())
473    }
474}
475
476/// Loads a transaction from database.
477///
478/// Behaviour shared by several `eth_` RPC methods, not exclusive to `eth_` transactions RPC
479/// methods.
480pub trait LoadTransaction: SpawnBlocking + FullEthApiTypes + RpcNodeCoreExt {
481    /// Returns the transaction by hash.
482    ///
483    /// Checks the pool and state.
484    ///
485    /// Returns `Ok(None)` if no matching transaction was found.
486    #[expect(clippy::complexity)]
487    fn transaction_by_hash(
488        &self,
489        hash: B256,
490    ) -> impl Future<
491        Output = Result<Option<TransactionSource<ProviderTx<Self::Provider>>>, Self::Error>,
492    > + Send {
493        async move {
494            // Try to find the transaction on disk
495            let mut resp = self
496                .spawn_blocking_io(move |this| {
497                    match this
498                        .provider()
499                        .transaction_by_hash_with_meta(hash)
500                        .map_err(Self::Error::from_eth_err)?
501                    {
502                        None => Ok(None),
503                        Some((tx, meta)) => {
504                            // Note: we assume this transaction is valid, because it's mined (or
505                            // part of pending block) and already. We don't need to
506                            // check for pre EIP-2 because this transaction could be pre-EIP-2.
507                            let transaction = tx
508                                .into_ecrecovered_unchecked()
509                                .ok_or(EthApiError::InvalidTransactionSignature)?;
510
511                            let tx = TransactionSource::Block {
512                                transaction,
513                                index: meta.index,
514                                block_hash: meta.block_hash,
515                                block_number: meta.block_number,
516                                base_fee: meta.base_fee,
517                            };
518                            Ok(Some(tx))
519                        }
520                    }
521                })
522                .await?;
523
524            if resp.is_none() {
525                // tx not found on disk, check pool
526                if let Some(tx) =
527                    self.pool().get(&hash).map(|tx| tx.transaction.clone().into_consensus())
528                {
529                    resp = Some(TransactionSource::Pool(tx.into()));
530                }
531            }
532
533            Ok(resp)
534        }
535    }
536
537    /// Returns the transaction by including its corresponding [`BlockId`].
538    ///
539    /// Note: this supports pending transactions
540    #[expect(clippy::type_complexity)]
541    fn transaction_by_hash_at(
542        &self,
543        transaction_hash: B256,
544    ) -> impl Future<
545        Output = Result<
546            Option<(TransactionSource<ProviderTx<Self::Provider>>, BlockId)>,
547            Self::Error,
548        >,
549    > + Send {
550        async move {
551            Ok(self.transaction_by_hash(transaction_hash).await?.map(|tx| match tx {
552                tx @ TransactionSource::Pool(_) => (tx, BlockId::pending()),
553                tx @ TransactionSource::Block { block_hash, .. } => {
554                    (tx, BlockId::Hash(block_hash.into()))
555                }
556            }))
557        }
558    }
559
560    /// Fetches the transaction and the transaction's block
561    #[expect(clippy::type_complexity)]
562    fn transaction_and_block(
563        &self,
564        hash: B256,
565    ) -> impl Future<
566        Output = Result<
567            Option<(
568                TransactionSource<ProviderTx<Self::Provider>>,
569                Arc<SealedBlockWithSenders<ProviderBlock<Self::Provider>>>,
570            )>,
571            Self::Error,
572        >,
573    > + Send {
574        async move {
575            let (transaction, at) = match self.transaction_by_hash_at(hash).await? {
576                None => return Ok(None),
577                Some(res) => res,
578            };
579
580            // Note: this is always either hash or pending
581            let block_hash = match at {
582                BlockId::Hash(hash) => hash.block_hash,
583                _ => return Ok(None),
584            };
585            let block = self
586                .cache()
587                .get_sealed_block_with_senders(block_hash)
588                .await
589                .map_err(Self::Error::from_eth_err)?;
590            Ok(block.map(|block| (transaction, block)))
591        }
592    }
593}