zebra_consensus/
transaction.rs

1//! Asynchronous verification of transactions.
2
3use std::{
4    collections::{HashMap, HashSet},
5    future::Future,
6    pin::Pin,
7    sync::Arc,
8    task::{Context, Poll},
9    time::Duration,
10};
11
12use chrono::{DateTime, Utc};
13use futures::{
14    stream::{FuturesUnordered, StreamExt},
15    FutureExt,
16};
17use tokio::sync::oneshot;
18use tower::{
19    buffer::Buffer,
20    timeout::{error::Elapsed, Timeout},
21    util::BoxService,
22    Service, ServiceExt,
23};
24use tracing::Instrument;
25
26use zebra_chain::{
27    amount::{Amount, NonNegative},
28    block, orchard,
29    parameters::{Network, NetworkUpgrade},
30    primitives::Groth16Proof,
31    sapling,
32    serialization::DateTime32,
33    transaction::{
34        self, HashType, SigHash, Transaction, UnminedTx, UnminedTxId, VerifiedUnminedTx,
35    },
36    transparent,
37};
38
39use zebra_node_services::mempool;
40use zebra_script::CachedFfiTransaction;
41use zebra_state as zs;
42
43use crate::{error::TransactionError, groth16::DescriptionWrapper, primitives, script, BoxError};
44
45pub mod check;
46#[cfg(test)]
47mod tests;
48
49/// A timeout applied to UTXO lookup requests.
50///
51/// The exact value is non-essential, but this should be long enough to allow
52/// out-of-order verification of blocks (UTXOs are not required to be ready
53/// immediately) while being short enough to:
54///   * prune blocks that are too far in the future to be worth keeping in the
55///     queue,
56///   * fail blocks that reference invalid UTXOs, and
57///   * fail blocks that reference UTXOs from blocks that have temporarily failed
58///     to download, because a peer sent Zebra a bad list of block hashes. (The
59///     UTXO verification failure will restart the sync, and re-download the
60///     chain in the correct order.)
61const UTXO_LOOKUP_TIMEOUT: std::time::Duration = std::time::Duration::from_secs(6 * 60);
62
63/// A timeout applied to output lookup requests sent to the mempool. This is shorter than the
64/// timeout for the state UTXO lookups because a block is likely to be mined every 75 seconds
65/// after Blossom is active, changing the best chain tip and requiring re-verification of transactions
66/// in the mempool.
67///
68/// This is how long Zebra will wait for an output to be added to the mempool before verification
69/// of the transaction that spends it will fail.
70const MEMPOOL_OUTPUT_LOOKUP_TIMEOUT: std::time::Duration = std::time::Duration::from_secs(60);
71
72/// How long to wait after responding to a mempool request with a transaction that creates new
73/// transparent outputs before polling the mempool service so that it will try adding the verified
74/// transaction and responding to any potential `AwaitOutput` requests.
75///
76/// This should be long enough for the mempool service's `Downloads` to finish processing the
77/// response from the transaction verifier.
78const POLL_MEMPOOL_DELAY: std::time::Duration = Duration::from_millis(50);
79
80/// Asynchronous transaction verification.
81///
82/// # Correctness
83///
84/// Transaction verification requests should be wrapped in a timeout, so that
85/// out-of-order and invalid requests do not hang indefinitely. See the [`router`](`crate::router`)
86/// module documentation for details.
87pub struct Verifier<ZS, Mempool> {
88    network: Network,
89    state: Timeout<ZS>,
90    // TODO: Use an enum so that this can either be Pending(oneshot::Receiver) or Initialized(MempoolService)
91    mempool: Option<Timeout<Mempool>>,
92    script_verifier: script::Verifier,
93    mempool_setup_rx: oneshot::Receiver<Mempool>,
94}
95
96impl<ZS, Mempool> Verifier<ZS, Mempool>
97where
98    ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
99    ZS::Future: Send + 'static,
100    Mempool: Service<mempool::Request, Response = mempool::Response, Error = BoxError>
101        + Send
102        + Clone
103        + 'static,
104    Mempool::Future: Send + 'static,
105{
106    /// Create a new transaction verifier.
107    pub fn new(network: &Network, state: ZS, mempool_setup_rx: oneshot::Receiver<Mempool>) -> Self {
108        Self {
109            network: network.clone(),
110            state: Timeout::new(state, UTXO_LOOKUP_TIMEOUT),
111            mempool: None,
112            script_verifier: script::Verifier,
113            mempool_setup_rx,
114        }
115    }
116}
117
118impl<ZS>
119    Verifier<
120        ZS,
121        Buffer<BoxService<mempool::Request, mempool::Response, BoxError>, mempool::Request>,
122    >
123where
124    ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
125    ZS::Future: Send + 'static,
126{
127    /// Create a new transaction verifier with a closed channel receiver for mempool setup for tests.
128    #[cfg(test)]
129    pub fn new_for_tests(network: &Network, state: ZS) -> Self {
130        Self {
131            network: network.clone(),
132            state: Timeout::new(state, UTXO_LOOKUP_TIMEOUT),
133            mempool: None,
134            script_verifier: script::Verifier,
135            mempool_setup_rx: oneshot::channel().1,
136        }
137    }
138}
139
140/// Specifies whether a transaction should be verified as part of a block or as
141/// part of the mempool.
142///
143/// Transaction verification has slightly different consensus rules, depending on
144/// whether the transaction is to be included in a block on in the mempool.
145#[derive(Clone, Debug, Eq, PartialEq)]
146pub enum Request {
147    /// Verify the supplied transaction as part of a block.
148    Block {
149        /// The transaction hash.
150        transaction_hash: transaction::Hash,
151        /// The transaction itself.
152        transaction: Arc<Transaction>,
153        /// Set of transaction hashes that create new transparent outputs.
154        known_outpoint_hashes: Arc<HashSet<transaction::Hash>>,
155        /// Additional UTXOs which are known at the time of verification.
156        known_utxos: Arc<HashMap<transparent::OutPoint, transparent::OrderedUtxo>>,
157        /// The height of the block containing this transaction.
158        height: block::Height,
159        /// The time that the block was mined.
160        time: DateTime<Utc>,
161    },
162    /// Verify the supplied transaction as part of the mempool.
163    ///
164    /// Mempool transactions do not have any additional UTXOs.
165    ///
166    /// Note: coinbase transactions are invalid in the mempool
167    Mempool {
168        /// The transaction itself.
169        transaction: UnminedTx,
170        /// The height of the next block.
171        ///
172        /// The next block is the first block that could possibly contain a
173        /// mempool transaction.
174        height: block::Height,
175    },
176}
177
178/// The response type for the transaction verifier service.
179/// Responses identify the transaction that was verified.
180#[derive(Clone, Debug, PartialEq)]
181pub enum Response {
182    /// A response to a block transaction verification request.
183    Block {
184        /// The witnessed transaction ID for this transaction.
185        ///
186        /// [`Response::Block`] responses can be uniquely identified by
187        /// [`UnminedTxId::mined_id`], because the block's authorizing data root
188        /// will be checked during contextual validation.
189        tx_id: UnminedTxId,
190
191        /// The miner fee for this transaction.
192        ///
193        /// `None` for coinbase transactions.
194        ///
195        /// # Consensus
196        ///
197        /// > The remaining value in the transparent transaction value pool
198        /// > of a coinbase transaction is destroyed.
199        ///
200        /// <https://zips.z.cash/protocol/protocol.pdf#transactions>
201        miner_fee: Option<Amount<NonNegative>>,
202
203        /// The number of legacy signature operations in this transaction's
204        /// transparent inputs and outputs.
205        legacy_sigop_count: u64,
206    },
207
208    /// A response to a mempool transaction verification request.
209    Mempool {
210        /// The full content of the verified mempool transaction.
211        /// Also contains the transaction fee and other associated fields.
212        ///
213        /// Mempool transactions always have a transaction fee,
214        /// because coinbase transactions are rejected from the mempool.
215        ///
216        /// [`Response::Mempool`] responses are uniquely identified by the
217        /// [`UnminedTxId`] variant for their transaction version.
218        transaction: VerifiedUnminedTx,
219
220        /// A list of spent [`transparent::OutPoint`]s that were found in
221        /// the mempool's list of `created_outputs`.
222        ///
223        /// Used by the mempool to determine dependencies between transactions
224        /// in the mempool and to avoid adding transactions with missing spends
225        /// to its verified set.
226        spent_mempool_outpoints: Vec<transparent::OutPoint>,
227    },
228}
229
230#[cfg(any(test, feature = "proptest-impl"))]
231impl From<VerifiedUnminedTx> for Response {
232    fn from(transaction: VerifiedUnminedTx) -> Self {
233        Response::Mempool {
234            transaction,
235            spent_mempool_outpoints: Vec::new(),
236        }
237    }
238}
239
240impl Request {
241    /// The transaction to verify that's in this request.
242    pub fn transaction(&self) -> Arc<Transaction> {
243        match self {
244            Request::Block { transaction, .. } => transaction.clone(),
245            Request::Mempool { transaction, .. } => transaction.transaction.clone(),
246        }
247    }
248
249    /// The unverified mempool transaction, if this is a mempool request.
250    pub fn mempool_transaction(&self) -> Option<UnminedTx> {
251        match self {
252            Request::Block { .. } => None,
253            Request::Mempool { transaction, .. } => Some(transaction.clone()),
254        }
255    }
256
257    /// The unmined transaction ID for the transaction in this request.
258    pub fn tx_id(&self) -> UnminedTxId {
259        match self {
260            // TODO: get the precalculated ID from the block verifier
261            Request::Block { transaction, .. } => transaction.unmined_id(),
262            Request::Mempool { transaction, .. } => transaction.id,
263        }
264    }
265
266    /// The mined transaction ID for the transaction in this request.
267    pub fn tx_mined_id(&self) -> transaction::Hash {
268        match self {
269            Request::Block {
270                transaction_hash, ..
271            } => *transaction_hash,
272            Request::Mempool { transaction, .. } => transaction.id.mined_id(),
273        }
274    }
275
276    /// The set of additional known unspent transaction outputs that's in this request.
277    pub fn known_utxos(&self) -> Arc<HashMap<transparent::OutPoint, transparent::OrderedUtxo>> {
278        match self {
279            Request::Block { known_utxos, .. } => known_utxos.clone(),
280            Request::Mempool { .. } => HashMap::new().into(),
281        }
282    }
283
284    /// The set of additional known [`transparent::OutPoint`]s of unspent transaction outputs that's in this request.
285    pub fn known_outpoint_hashes(&self) -> Arc<HashSet<transaction::Hash>> {
286        match self {
287            Request::Block {
288                known_outpoint_hashes,
289                ..
290            } => known_outpoint_hashes.clone(),
291            Request::Mempool { .. } => HashSet::new().into(),
292        }
293    }
294
295    /// The height used to select the consensus rules for verifying this transaction.
296    pub fn height(&self) -> block::Height {
297        match self {
298            Request::Block { height, .. } | Request::Mempool { height, .. } => *height,
299        }
300    }
301
302    /// The block time used for lock time consensus rules validation.
303    pub fn block_time(&self) -> Option<DateTime<Utc>> {
304        match self {
305            Request::Block { time, .. } => Some(*time),
306            Request::Mempool { .. } => None,
307        }
308    }
309
310    /// The network upgrade to consider for the verification.
311    ///
312    /// This is based on the block height from the request, and the supplied `network`.
313    pub fn upgrade(&self, network: &Network) -> NetworkUpgrade {
314        NetworkUpgrade::current(network, self.height())
315    }
316
317    /// Returns true if the request is a mempool request.
318    pub fn is_mempool(&self) -> bool {
319        matches!(self, Request::Mempool { .. })
320    }
321}
322
323impl Response {
324    /// The unmined transaction ID for the transaction in this response.
325    pub fn tx_id(&self) -> UnminedTxId {
326        match self {
327            Response::Block { tx_id, .. } => *tx_id,
328            Response::Mempool { transaction, .. } => transaction.transaction.id,
329        }
330    }
331
332    /// The miner fee for the transaction in this response.
333    ///
334    /// Coinbase transactions do not have a miner fee,
335    /// and they don't need UTXOs to calculate their value balance,
336    /// because they don't spend any inputs.
337    pub fn miner_fee(&self) -> Option<Amount<NonNegative>> {
338        match self {
339            Response::Block { miner_fee, .. } => *miner_fee,
340            Response::Mempool { transaction, .. } => Some(transaction.miner_fee),
341        }
342    }
343
344    /// The number of legacy transparent signature operations in this transaction's
345    /// inputs and outputs.
346    pub fn legacy_sigop_count(&self) -> u64 {
347        match self {
348            Response::Block {
349                legacy_sigop_count, ..
350            } => *legacy_sigop_count,
351            Response::Mempool { transaction, .. } => transaction.legacy_sigop_count,
352        }
353    }
354
355    /// Returns true if the request is a mempool request.
356    pub fn is_mempool(&self) -> bool {
357        match self {
358            Response::Block { .. } => false,
359            Response::Mempool { .. } => true,
360        }
361    }
362}
363
364impl<ZS, Mempool> Service<Request> for Verifier<ZS, Mempool>
365where
366    ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
367    ZS::Future: Send + 'static,
368    Mempool: Service<mempool::Request, Response = mempool::Response, Error = BoxError>
369        + Send
370        + Clone
371        + 'static,
372    Mempool::Future: Send + 'static,
373{
374    type Response = Response;
375    type Error = TransactionError;
376    type Future =
377        Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send + 'static>>;
378
379    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
380        // Note: The block verifier expects the transaction verifier to always be ready.
381
382        if self.mempool.is_none() {
383            if let Ok(mempool) = self.mempool_setup_rx.try_recv() {
384                self.mempool = Some(Timeout::new(mempool, MEMPOOL_OUTPUT_LOOKUP_TIMEOUT));
385            }
386        }
387
388        Poll::Ready(Ok(()))
389    }
390
391    // TODO: break up each chunk into its own method
392    fn call(&mut self, req: Request) -> Self::Future {
393        let script_verifier = self.script_verifier;
394        let network = self.network.clone();
395        let state = self.state.clone();
396        let mempool = self.mempool.clone();
397
398        let tx = req.transaction();
399        let tx_id = req.tx_id();
400        let span = tracing::debug_span!("tx", ?tx_id);
401
402        async move {
403            tracing::trace!(?tx_id, ?req, "got tx verify request");
404
405            if let Some(result) = Self::find_verified_unmined_tx(&req, mempool.clone(), state.clone()).await {
406                let verified_tx = result?;
407
408                return Ok(Response::Block {
409                    tx_id,
410                    miner_fee: Some(verified_tx.miner_fee),
411                    legacy_sigop_count: verified_tx.legacy_sigop_count
412                });
413            }
414
415            // Do quick checks first
416            check::has_inputs_and_outputs(&tx)?;
417            check::has_enough_orchard_flags(&tx)?;
418            check::consensus_branch_id(&tx, req.height(), &network)?;
419
420            // Validate the coinbase input consensus rules
421            if req.is_mempool() && tx.is_coinbase() {
422                return Err(TransactionError::CoinbaseInMempool);
423            }
424
425            if tx.is_coinbase() {
426                check::coinbase_tx_no_prevout_joinsplit_spend(&tx)?;
427            } else if !tx.is_valid_non_coinbase() {
428                return Err(TransactionError::NonCoinbaseHasCoinbaseInput);
429            }
430
431            // Validate `nExpiryHeight` consensus rules
432            if tx.is_coinbase() {
433                check::coinbase_expiry_height(&req.height(), &tx, &network)?;
434            } else {
435                check::non_coinbase_expiry_height(&req.height(), &tx)?;
436            }
437
438            // Consensus rule:
439            //
440            // > Either v_{pub}^{old} or v_{pub}^{new} MUST be zero.
441            //
442            // https://zips.z.cash/protocol/protocol.pdf#joinsplitdesc
443            check::joinsplit_has_vpub_zero(&tx)?;
444
445            // [Canopy onward]: `vpub_old` MUST be zero.
446            // https://zips.z.cash/protocol/protocol.pdf#joinsplitdesc
447            check::disabled_add_to_sprout_pool(&tx, req.height(), &network)?;
448
449            check::spend_conflicts(&tx)?;
450
451            tracing::trace!(?tx_id, "passed quick checks");
452
453            if let Some(block_time) = req.block_time() {
454                check::lock_time_has_passed(&tx, req.height(), block_time)?;
455            } else {
456                // Skip the state query if we don't need the time for this check.
457                let next_median_time_past = if tx.lock_time_is_time() {
458                    // This state query is much faster than loading UTXOs from the database,
459                    // so it doesn't need to be executed in parallel
460                    let state = state.clone();
461                    Some(Self::mempool_best_chain_next_median_time_past(state).await?.to_chrono())
462                } else {
463                    None
464                };
465
466                // This consensus check makes sure Zebra produces valid block templates.
467                check::lock_time_has_passed(&tx, req.height(), next_median_time_past)?;
468            }
469
470            // "The consensus rules applied to valueBalance, vShieldedOutput, and bindingSig
471            // in non-coinbase transactions MUST also be applied to coinbase transactions."
472            //
473            // This rule is implicitly implemented during Sapling and Orchard verification,
474            // because they do not distinguish between coinbase and non-coinbase transactions.
475            //
476            // Note: this rule originally applied to Sapling, but we assume it also applies to Orchard.
477            //
478            // https://zips.z.cash/zip-0213#specification
479
480            // Load spent UTXOs from state.
481            // The UTXOs are required for almost all the async checks.
482            let load_spent_utxos_fut =
483                Self::spent_utxos(tx.clone(), req.clone(), state.clone(), mempool.clone(),);
484            let (spent_utxos, spent_outputs, spent_mempool_outpoints) = load_spent_utxos_fut.await?;
485
486            // WONTFIX: Return an error for Request::Block as well to replace this check in
487            //       the state once #2336 has been implemented?
488            if req.is_mempool() {
489                Self::check_maturity_height(&network, &req, &spent_utxos)?;
490            }
491
492            let nu = req.upgrade(&network);
493            let cached_ffi_transaction =
494                Arc::new(CachedFfiTransaction::new(tx.clone(), Arc::new(spent_outputs), nu).map_err(|_| TransactionError::UnsupportedByNetworkUpgrade(tx.version(), nu))?);
495
496            tracing::trace!(?tx_id, "got state UTXOs");
497
498            let mut async_checks = match tx.as_ref() {
499                Transaction::V1 { .. } | Transaction::V2 { .. } | Transaction::V3 { .. } => {
500                    tracing::debug!(?tx, "got transaction with wrong version");
501                    return Err(TransactionError::WrongVersion);
502                }
503                Transaction::V4 {
504                    joinsplit_data,
505                    sapling_shielded_data,
506                    ..
507                } => Self::verify_v4_transaction(
508                    &req,
509                    &network,
510                    script_verifier,
511                    cached_ffi_transaction.clone(),
512                    joinsplit_data,
513                    sapling_shielded_data,
514                )?,
515                Transaction::V5 {
516                    sapling_shielded_data,
517                    orchard_shielded_data,
518                    ..
519                } => Self::verify_v5_transaction(
520                    &req,
521                    &network,
522                    script_verifier,
523                    cached_ffi_transaction.clone(),
524                    sapling_shielded_data,
525                    orchard_shielded_data,
526                )?,
527                #[cfg(feature="tx_v6")]
528                Transaction::V6 {
529                    sapling_shielded_data,
530                    orchard_shielded_data,
531                    ..
532                } => Self::verify_v6_transaction(
533                    &req,
534                    &network,
535                    script_verifier,
536                    cached_ffi_transaction.clone(),
537                    sapling_shielded_data,
538                    orchard_shielded_data,
539                )?,
540            };
541
542            if let Some(unmined_tx) = req.mempool_transaction() {
543                let check_anchors_and_revealed_nullifiers_query = state
544                    .clone()
545                    .oneshot(zs::Request::CheckBestChainTipNullifiersAndAnchors(
546                        unmined_tx,
547                    ))
548                    .map(|res| {
549                        assert!(
550                            res? == zs::Response::ValidBestChainTipNullifiersAndAnchors,
551                            "unexpected response to CheckBestChainTipNullifiersAndAnchors request"
552                        );
553                        Ok(())
554                    }
555                );
556
557                async_checks.push(check_anchors_and_revealed_nullifiers_query);
558            }
559
560            tracing::trace!(?tx_id, "awaiting async checks...");
561
562            // If the Groth16 parameter download hangs,
563            // Zebra will timeout here, waiting for the async checks.
564            async_checks.check().await?;
565
566            tracing::trace!(?tx_id, "finished async checks");
567
568            // Get the `value_balance` to calculate the transaction fee.
569            let value_balance = tx.value_balance(&spent_utxos);
570
571            // Calculate the fee only for non-coinbase transactions.
572            let mut miner_fee = None;
573            if !tx.is_coinbase() {
574                // TODO: deduplicate this code with remaining_transaction_value()?
575                miner_fee = match value_balance {
576                    Ok(vb) => match vb.remaining_transaction_value() {
577                        Ok(tx_rtv) => Some(tx_rtv),
578                        Err(_) => return Err(TransactionError::IncorrectFee),
579                    },
580                    Err(_) => return Err(TransactionError::IncorrectFee),
581                };
582            }
583
584            let legacy_sigop_count = zebra_script::legacy_sigop_count(&tx)?;
585
586            let rsp = match req {
587                Request::Block { .. } => Response::Block {
588                    tx_id,
589                    miner_fee,
590                    legacy_sigop_count,
591                },
592                Request::Mempool { transaction: tx, .. } => {
593                    let transaction = VerifiedUnminedTx::new(
594                        tx,
595                        miner_fee.expect("fee should have been checked earlier"),
596                        legacy_sigop_count,
597                    )?;
598
599                    if let Some(mut mempool) = mempool {
600                        tokio::spawn(async move {
601                            // Best-effort poll of the mempool to provide a timely response to
602                            // `sendrawtransaction` RPC calls or `AwaitOutput` mempool calls.
603                            tokio::time::sleep(POLL_MEMPOOL_DELAY).await;
604                            let _ = mempool
605                                .ready()
606                                .await
607                                .expect("mempool poll_ready() method should not return an error")
608                                .call(mempool::Request::CheckForVerifiedTransactions)
609                                .await;
610                        });
611                    }
612
613                    Response::Mempool { transaction, spent_mempool_outpoints }
614                },
615            };
616
617            Ok(rsp)
618        }
619        .inspect(move |result| {
620            // Hide the transaction data to avoid filling the logs
621            tracing::trace!(?tx_id, result = ?result.as_ref().map(|_tx| ()), "got tx verify result");
622        })
623        .instrument(span)
624        .boxed()
625    }
626}
627
628impl<ZS, Mempool> Verifier<ZS, Mempool>
629where
630    ZS: Service<zs::Request, Response = zs::Response, Error = BoxError> + Send + Clone + 'static,
631    ZS::Future: Send + 'static,
632    Mempool: Service<mempool::Request, Response = mempool::Response, Error = BoxError>
633        + Send
634        + Clone
635        + 'static,
636    Mempool::Future: Send + 'static,
637{
638    /// Fetches the median-time-past of the *next* block after the best state tip.
639    ///
640    /// This is used to verify that the lock times of mempool transactions
641    /// can be included in any valid next block.
642    async fn mempool_best_chain_next_median_time_past(
643        state: Timeout<ZS>,
644    ) -> Result<DateTime32, TransactionError> {
645        let query = state
646            .clone()
647            .oneshot(zs::Request::BestChainNextMedianTimePast);
648
649        if let zebra_state::Response::BestChainNextMedianTimePast(median_time_past) = query
650            .await
651            .map_err(|e| TransactionError::ValidateMempoolLockTimeError(e.to_string()))?
652        {
653            Ok(median_time_past)
654        } else {
655            unreachable!("Request::BestChainNextMedianTimePast always responds with BestChainNextMedianTimePast")
656        }
657    }
658
659    /// Attempts to find a transaction in the mempool by its transaction hash and checks
660    /// that all of its dependencies are available in the block or in the state.  Waits
661    /// for UTXOs being spent by the given transaction to arrive in the state if they're
662    /// not found elsewhere.
663    ///
664    /// Returns [`Some(Ok(VerifiedUnminedTx))`](VerifiedUnminedTx) if successful,
665    /// None if the transaction id was not found in the mempool,
666    /// or `Some(Err(TransparentInputNotFound))` if the transaction was found, but some of its
667    /// dependencies were not found in the block or state after a timeout.
668    async fn find_verified_unmined_tx(
669        req: &Request,
670        mempool: Option<Timeout<Mempool>>,
671        state: Timeout<ZS>,
672    ) -> Option<Result<VerifiedUnminedTx, TransactionError>> {
673        let tx = req.transaction();
674
675        if req.is_mempool() || tx.is_coinbase() {
676            return None;
677        }
678
679        let mempool = mempool?;
680        let known_outpoint_hashes = req.known_outpoint_hashes();
681        let tx_id = req.tx_mined_id();
682
683        let mempool::Response::TransactionWithDeps {
684            transaction: verified_tx,
685            dependencies,
686        } = mempool
687            .oneshot(mempool::Request::TransactionWithDepsByMinedId(tx_id))
688            .await
689            .ok()?
690        else {
691            panic!("unexpected response to TransactionWithDepsByMinedId request");
692        };
693
694        // Note: This does not verify that the spends are in order, the spend order
695        //       should be verified during contextual validation in zebra-state.
696        let missing_deps: HashSet<_> = dependencies
697            .into_iter()
698            .filter(|dependency_id| !known_outpoint_hashes.contains(dependency_id))
699            .collect();
700
701        if missing_deps.is_empty() {
702            return Some(Ok(verified_tx));
703        }
704
705        let missing_outpoints = tx.inputs().iter().filter_map(|input| {
706            if let transparent::Input::PrevOut { outpoint, .. } = input {
707                missing_deps.contains(&outpoint.hash).then_some(outpoint)
708            } else {
709                None
710            }
711        });
712
713        for missing_outpoint in missing_outpoints {
714            let query = state
715                .clone()
716                .oneshot(zebra_state::Request::AwaitUtxo(*missing_outpoint));
717            match query.await {
718                Ok(zebra_state::Response::Utxo(_)) => {}
719                Err(_) => return Some(Err(TransactionError::TransparentInputNotFound)),
720                _ => unreachable!("AwaitUtxo always responds with Utxo"),
721            };
722        }
723
724        Some(Ok(verified_tx))
725    }
726
727    /// Wait for the UTXOs that are being spent by the given transaction.
728    ///
729    /// Looks up UTXOs that are being spent by the given transaction in the state or waits
730    /// for them to be added to the mempool for [`Mempool`](Request::Mempool) requests.
731    ///
732    /// Returns a triple containing:
733    /// - `OutPoint` -> `Utxo` map,
734    /// - vec of `Output`s in the same order as the matching inputs in the `tx`,
735    /// - vec of `Outpoint`s spent by a mempool `tx` that were not found in the best chain's utxo set.
736    async fn spent_utxos(
737        tx: Arc<Transaction>,
738        req: Request,
739        state: Timeout<ZS>,
740        mempool: Option<Timeout<Mempool>>,
741    ) -> Result<
742        (
743            HashMap<transparent::OutPoint, transparent::Utxo>,
744            Vec<transparent::Output>,
745            Vec<transparent::OutPoint>,
746        ),
747        TransactionError,
748    > {
749        let is_mempool = req.is_mempool();
750        // Additional UTXOs known at the time of validation,
751        // i.e., from previous transactions in the block.
752        let known_utxos = req.known_utxos();
753
754        let inputs = tx.inputs();
755        let mut spent_utxos = HashMap::new();
756        let mut spent_outputs = Vec::new();
757        let mut spent_mempool_outpoints = Vec::new();
758
759        for input in inputs {
760            if let transparent::Input::PrevOut { outpoint, .. } = input {
761                tracing::trace!("awaiting outpoint lookup");
762                let utxo = if let Some(output) = known_utxos.get(outpoint) {
763                    tracing::trace!("UXTO in known_utxos, discarding query");
764                    output.utxo.clone()
765                } else if is_mempool {
766                    let query = state
767                        .clone()
768                        .oneshot(zs::Request::UnspentBestChainUtxo(*outpoint));
769
770                    let zebra_state::Response::UnspentBestChainUtxo(utxo) = query
771                        .await
772                        .map_err(|_| TransactionError::TransparentInputNotFound)?
773                    else {
774                        unreachable!("UnspentBestChainUtxo always responds with Option<Utxo>")
775                    };
776
777                    let Some(utxo) = utxo else {
778                        spent_mempool_outpoints.push(*outpoint);
779                        continue;
780                    };
781
782                    utxo
783                } else {
784                    let query = state
785                        .clone()
786                        .oneshot(zebra_state::Request::AwaitUtxo(*outpoint));
787                    if let zebra_state::Response::Utxo(utxo) = query.await? {
788                        utxo
789                    } else {
790                        unreachable!("AwaitUtxo always responds with Utxo")
791                    }
792                };
793                tracing::trace!(?utxo, "got UTXO");
794                spent_outputs.push(utxo.output.clone());
795                spent_utxos.insert(*outpoint, utxo);
796            } else {
797                continue;
798            }
799        }
800
801        if let Some(mempool) = mempool {
802            for &spent_mempool_outpoint in &spent_mempool_outpoints {
803                let query = mempool
804                    .clone()
805                    .oneshot(mempool::Request::AwaitOutput(spent_mempool_outpoint));
806
807                let output = match query.await {
808                    Ok(mempool::Response::UnspentOutput(output)) => output,
809                    Ok(_) => unreachable!("UnspentOutput always responds with UnspentOutput"),
810                    Err(err) => {
811                        return match err.downcast::<Elapsed>() {
812                            Ok(_) => Err(TransactionError::TransparentInputNotFound),
813                            Err(err) => Err(err.into()),
814                        };
815                    }
816                };
817
818                spent_outputs.push(output.clone());
819                spent_utxos.insert(
820                    spent_mempool_outpoint,
821                    // Assume the Utxo height will be next height after the best chain tip height
822                    //
823                    // # Correctness
824                    //
825                    // If the tip height changes while an umined transaction is being verified,
826                    // the transaction must be re-verified before being added to the mempool.
827                    transparent::Utxo::new(output, req.height(), false),
828                );
829            }
830        } else if !spent_mempool_outpoints.is_empty() {
831            return Err(TransactionError::TransparentInputNotFound);
832        }
833
834        Ok((spent_utxos, spent_outputs, spent_mempool_outpoints))
835    }
836
837    /// Accepts `request`, a transaction verifier [`&Request`](Request),
838    /// and `spent_utxos`, a HashMap of UTXOs in the chain that are spent by this transaction.
839    ///
840    /// Gets the `transaction`, `height`, and `known_utxos` for the request and checks calls
841    /// [`check::tx_transparent_coinbase_spends_maturity`] to verify that every transparent
842    /// coinbase output spent by the transaction will have matured by `height`.
843    ///
844    /// Returns `Ok(())` if every transparent coinbase output spent by the transaction is
845    /// mature and valid for the request height, or a [`TransactionError`] if the transaction
846    /// spends transparent coinbase outputs that are immature and invalid for the request height.
847    pub fn check_maturity_height(
848        network: &Network,
849        request: &Request,
850        spent_utxos: &HashMap<transparent::OutPoint, transparent::Utxo>,
851    ) -> Result<(), TransactionError> {
852        check::tx_transparent_coinbase_spends_maturity(
853            network,
854            request.transaction(),
855            request.height(),
856            request.known_utxos(),
857            spent_utxos,
858        )
859    }
860
861    /// Verify a V4 transaction.
862    ///
863    /// Returns a set of asynchronous checks that must all succeed for the transaction to be
864    /// considered valid. These checks include:
865    ///
866    /// - transparent transfers
867    /// - sprout shielded data
868    /// - sapling shielded data
869    ///
870    /// The parameters of this method are:
871    ///
872    /// - the `request` to verify (that contains the transaction and other metadata, see [`Request`]
873    ///   for more information)
874    /// - the `network` to consider when verifying
875    /// - the `script_verifier` to use for verifying the transparent transfers
876    /// - the prepared `cached_ffi_transaction` used by the script verifier
877    /// - the Sprout `joinsplit_data` shielded data in the transaction
878    /// - the `sapling_shielded_data` in the transaction
879    #[allow(clippy::unwrap_in_result)]
880    fn verify_v4_transaction(
881        request: &Request,
882        network: &Network,
883        script_verifier: script::Verifier,
884        cached_ffi_transaction: Arc<CachedFfiTransaction>,
885        joinsplit_data: &Option<transaction::JoinSplitData<Groth16Proof>>,
886        sapling_shielded_data: &Option<sapling::ShieldedData<sapling::PerSpendAnchor>>,
887    ) -> Result<AsyncChecks, TransactionError> {
888        let tx = request.transaction();
889        let nu = request.upgrade(network);
890
891        Self::verify_v4_transaction_network_upgrade(&tx, nu)?;
892
893        let shielded_sighash = cached_ffi_transaction
894            .sighasher()
895            .sighash(HashType::ALL, None);
896
897        Ok(Self::verify_transparent_inputs_and_outputs(
898            request,
899            script_verifier,
900            cached_ffi_transaction,
901        )?
902        .and(Self::verify_sprout_shielded_data(
903            joinsplit_data,
904            &shielded_sighash,
905        )?)
906        .and(Self::verify_sapling_shielded_data(
907            sapling_shielded_data,
908            &shielded_sighash,
909        )?))
910    }
911
912    /// Verifies if a V4 `transaction` is supported by `network_upgrade`.
913    fn verify_v4_transaction_network_upgrade(
914        transaction: &Transaction,
915        network_upgrade: NetworkUpgrade,
916    ) -> Result<(), TransactionError> {
917        match network_upgrade {
918            // Supports V4 transactions
919            //
920            // # Consensus
921            //
922            // > [Sapling to Canopy inclusive, pre-NU5] The transaction version number MUST be 4,
923            // > and the version group ID MUST be 0x892F2085.
924            //
925            // > [NU5 onward] The transaction version number MUST be 4 or 5.
926            // > If the transaction version number is 4 then the version group ID MUST be 0x892F2085.
927            // > If the transaction version number is 5 then the version group ID MUST be 0x26A7270A.
928            //
929            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
930            //
931            // Note: Here we verify the transaction version number of the above two rules, the group
932            // id is checked in zebra-chain crate, in the transaction serialize.
933            NetworkUpgrade::Sapling
934            | NetworkUpgrade::Blossom
935            | NetworkUpgrade::Heartwood
936            | NetworkUpgrade::Canopy
937            | NetworkUpgrade::Nu5
938            | NetworkUpgrade::Nu6
939            | NetworkUpgrade::Nu6_1
940            | NetworkUpgrade::Nu7 => Ok(()),
941
942            // Does not support V4 transactions
943            NetworkUpgrade::Genesis
944            | NetworkUpgrade::BeforeOverwinter
945            | NetworkUpgrade::Overwinter => Err(TransactionError::UnsupportedByNetworkUpgrade(
946                transaction.version(),
947                network_upgrade,
948            )),
949        }
950    }
951
952    /// Verify a V5 transaction.
953    ///
954    /// Returns a set of asynchronous checks that must all succeed for the transaction to be
955    /// considered valid. These checks include:
956    ///
957    /// - transaction support by the considered network upgrade (see [`Request::upgrade`])
958    /// - transparent transfers
959    /// - sapling shielded data (TODO)
960    /// - orchard shielded data (TODO)
961    ///
962    /// The parameters of this method are:
963    ///
964    /// - the `request` to verify (that contains the transaction and other metadata, see [`Request`]
965    ///   for more information)
966    /// - the `network` to consider when verifying
967    /// - the `script_verifier` to use for verifying the transparent transfers
968    /// - the prepared `cached_ffi_transaction` used by the script verifier
969    /// - the sapling shielded data of the transaction, if any
970    /// - the orchard shielded data of the transaction, if any
971    #[allow(clippy::unwrap_in_result)]
972    fn verify_v5_transaction(
973        request: &Request,
974        network: &Network,
975        script_verifier: script::Verifier,
976        cached_ffi_transaction: Arc<CachedFfiTransaction>,
977        sapling_shielded_data: &Option<sapling::ShieldedData<sapling::SharedAnchor>>,
978        orchard_shielded_data: &Option<orchard::ShieldedData>,
979    ) -> Result<AsyncChecks, TransactionError> {
980        let transaction = request.transaction();
981        let nu = request.upgrade(network);
982
983        Self::verify_v5_transaction_network_upgrade(&transaction, nu)?;
984
985        let shielded_sighash = cached_ffi_transaction
986            .sighasher()
987            .sighash(HashType::ALL, None);
988
989        Ok(Self::verify_transparent_inputs_and_outputs(
990            request,
991            script_verifier,
992            cached_ffi_transaction,
993        )?
994        .and(Self::verify_sapling_shielded_data(
995            sapling_shielded_data,
996            &shielded_sighash,
997        )?)
998        .and(Self::verify_orchard_shielded_data(
999            orchard_shielded_data,
1000            &shielded_sighash,
1001        )?))
1002    }
1003
1004    /// Verifies if a V5 `transaction` is supported by `network_upgrade`.
1005    fn verify_v5_transaction_network_upgrade(
1006        transaction: &Transaction,
1007        network_upgrade: NetworkUpgrade,
1008    ) -> Result<(), TransactionError> {
1009        match network_upgrade {
1010            // Supports V5 transactions
1011            //
1012            // # Consensus
1013            //
1014            // > [NU5 onward] The transaction version number MUST be 4 or 5.
1015            // > If the transaction version number is 4 then the version group ID MUST be 0x892F2085.
1016            // > If the transaction version number is 5 then the version group ID MUST be 0x26A7270A.
1017            //
1018            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
1019            //
1020            // Note: Here we verify the transaction version number of the above rule, the group
1021            // id is checked in zebra-chain crate, in the transaction serialize.
1022            NetworkUpgrade::Nu5
1023            | NetworkUpgrade::Nu6
1024            | NetworkUpgrade::Nu6_1
1025            | NetworkUpgrade::Nu7 => Ok(()),
1026
1027            // Does not support V5 transactions
1028            NetworkUpgrade::Genesis
1029            | NetworkUpgrade::BeforeOverwinter
1030            | NetworkUpgrade::Overwinter
1031            | NetworkUpgrade::Sapling
1032            | NetworkUpgrade::Blossom
1033            | NetworkUpgrade::Heartwood
1034            | NetworkUpgrade::Canopy => Err(TransactionError::UnsupportedByNetworkUpgrade(
1035                transaction.version(),
1036                network_upgrade,
1037            )),
1038        }
1039    }
1040
1041    /// Passthrough to verify_v5_transaction, but for V6 transactions.
1042    #[cfg(feature = "tx_v6")]
1043    fn verify_v6_transaction(
1044        request: &Request,
1045        network: &Network,
1046        script_verifier: script::Verifier,
1047        cached_ffi_transaction: Arc<CachedFfiTransaction>,
1048        sapling_shielded_data: &Option<sapling::ShieldedData<sapling::SharedAnchor>>,
1049        orchard_shielded_data: &Option<orchard::ShieldedData>,
1050    ) -> Result<AsyncChecks, TransactionError> {
1051        Self::verify_v5_transaction(
1052            request,
1053            network,
1054            script_verifier,
1055            cached_ffi_transaction,
1056            sapling_shielded_data,
1057            orchard_shielded_data,
1058        )
1059    }
1060
1061    /// Verifies if a transaction's transparent inputs are valid using the provided
1062    /// `script_verifier` and `cached_ffi_transaction`.
1063    ///
1064    /// Returns script verification responses via the `utxo_sender`.
1065    fn verify_transparent_inputs_and_outputs(
1066        request: &Request,
1067        script_verifier: script::Verifier,
1068        cached_ffi_transaction: Arc<CachedFfiTransaction>,
1069    ) -> Result<AsyncChecks, TransactionError> {
1070        let transaction = request.transaction();
1071
1072        if transaction.is_coinbase() {
1073            // The script verifier only verifies PrevOut inputs and their corresponding UTXOs.
1074            // Coinbase transactions don't have any PrevOut inputs.
1075            Ok(AsyncChecks::new())
1076        } else {
1077            // feed all of the inputs to the script verifier
1078            let inputs = transaction.inputs();
1079
1080            let script_checks = (0..inputs.len())
1081                .map(move |input_index| {
1082                    let request = script::Request {
1083                        cached_ffi_transaction: cached_ffi_transaction.clone(),
1084                        input_index,
1085                    };
1086
1087                    script_verifier.oneshot(request)
1088                })
1089                .collect();
1090
1091            Ok(script_checks)
1092        }
1093    }
1094
1095    /// Verifies a transaction's Sprout shielded join split data.
1096    fn verify_sprout_shielded_data(
1097        joinsplit_data: &Option<transaction::JoinSplitData<Groth16Proof>>,
1098        shielded_sighash: &SigHash,
1099    ) -> Result<AsyncChecks, TransactionError> {
1100        let mut checks = AsyncChecks::new();
1101
1102        if let Some(joinsplit_data) = joinsplit_data {
1103            for joinsplit in joinsplit_data.joinsplits() {
1104                // # Consensus
1105                //
1106                // > The proof π_ZKJoinSplit MUST be valid given a
1107                // > primary input formed from the relevant other fields and h_{Sig}
1108                //
1109                // https://zips.z.cash/protocol/protocol.pdf#joinsplitdesc
1110                //
1111                // Queue the verification of the Groth16 spend proof
1112                // for each JoinSplit description while adding the
1113                // resulting future to our collection of async
1114                // checks that (at a minimum) must pass for the
1115                // transaction to verify.
1116                checks.push(primitives::groth16::JOINSPLIT_VERIFIER.oneshot(
1117                    DescriptionWrapper(&(joinsplit, &joinsplit_data.pub_key)).try_into()?,
1118                ));
1119            }
1120
1121            // # Consensus
1122            //
1123            // > If effectiveVersion ≥ 2 and nJoinSplit > 0, then:
1124            // > - joinSplitPubKey MUST be a valid encoding of an Ed25519 validating key
1125            // > - joinSplitSig MUST represent a valid signature under
1126            //     joinSplitPubKey of dataToBeSigned, as defined in § 4.11
1127            //
1128            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
1129            //
1130            // The `if` part is indirectly enforced, since the `joinsplit_data`
1131            // is only parsed if those conditions apply in
1132            // [`Transaction::zcash_deserialize`].
1133            //
1134            // The valid encoding is defined in
1135            //
1136            // > A valid Ed25519 validating key is defined as a sequence of 32
1137            // > bytes encoding a point on the Ed25519 curve
1138            //
1139            // https://zips.z.cash/protocol/protocol.pdf#concreteed25519
1140            //
1141            // which is enforced during signature verification, in both batched
1142            // and single verification, when decompressing the encoded point.
1143            //
1144            // Queue the validation of the JoinSplit signature while
1145            // adding the resulting future to our collection of
1146            // async checks that (at a minimum) must pass for the
1147            // transaction to verify.
1148            //
1149            // https://zips.z.cash/protocol/protocol.pdf#sproutnonmalleability
1150            // https://zips.z.cash/protocol/protocol.pdf#txnencodingandconsensus
1151            let ed25519_verifier = primitives::ed25519::VERIFIER.clone();
1152            let ed25519_item =
1153                (joinsplit_data.pub_key, joinsplit_data.sig, shielded_sighash).into();
1154
1155            checks.push(ed25519_verifier.oneshot(ed25519_item));
1156        }
1157
1158        Ok(checks)
1159    }
1160
1161    /// Verifies a transaction's Sapling shielded data.
1162    fn verify_sapling_shielded_data<A>(
1163        sapling_shielded_data: &Option<sapling::ShieldedData<A>>,
1164        shielded_sighash: &SigHash,
1165    ) -> Result<AsyncChecks, TransactionError>
1166    where
1167        A: sapling::AnchorVariant + Clone,
1168        sapling::Spend<sapling::PerSpendAnchor>: From<(sapling::Spend<A>, A::Shared)>,
1169    {
1170        let mut async_checks = AsyncChecks::new();
1171
1172        if let Some(sapling_shielded_data) = sapling_shielded_data {
1173            for spend in sapling_shielded_data.spends_per_anchor() {
1174                // # Consensus
1175                //
1176                // > The proof π_ZKSpend MUST be valid
1177                // > given a primary input formed from the other
1178                // > fields except spendAuthSig.
1179                //
1180                // https://zips.z.cash/protocol/protocol.pdf#spenddesc
1181                //
1182                // Queue the verification of the Groth16 spend proof
1183                // for each Spend description while adding the
1184                // resulting future to our collection of async
1185                // checks that (at a minimum) must pass for the
1186                // transaction to verify.
1187                async_checks.push(
1188                    primitives::groth16::SPEND_VERIFIER
1189                        .clone()
1190                        .oneshot(DescriptionWrapper(&spend).try_into()?),
1191                );
1192
1193                // # Consensus
1194                //
1195                // > The spend authorization signature
1196                // > MUST be a valid SpendAuthSig signature over
1197                // > SigHash using rk as the validating key.
1198                //
1199                // This is validated by the verifier.
1200                //
1201                // > [NU5 onward] As specified in § 5.4.7 ‘RedDSA, RedJubjub,
1202                // > and RedPallas’ on p. 88, the validation of the 𝑅
1203                // > component of the signature changes to prohibit non-canonical encodings.
1204                //
1205                // This is validated by the verifier, inside the `redjubjub` crate.
1206                // It calls [`jubjub::AffinePoint::from_bytes`] to parse R and
1207                // that enforces the canonical encoding.
1208                //
1209                // https://zips.z.cash/protocol/protocol.pdf#spenddesc
1210                //
1211                // Queue the validation of the RedJubjub spend
1212                // authorization signature for each Spend
1213                // description while adding the resulting future to
1214                // our collection of async checks that (at a
1215                // minimum) must pass for the transaction to verify.
1216                async_checks.push(
1217                    primitives::redjubjub::VERIFIER
1218                        .clone()
1219                        .oneshot((spend.rk.into(), spend.spend_auth_sig, shielded_sighash).into()),
1220                );
1221            }
1222
1223            for output in sapling_shielded_data.outputs() {
1224                // # Consensus
1225                //
1226                // > The proof π_ZKOutput MUST be
1227                // > valid given a primary input formed from the other
1228                // > fields except C^enc and C^out.
1229                //
1230                // https://zips.z.cash/protocol/protocol.pdf#outputdesc
1231                //
1232                // Queue the verification of the Groth16 output
1233                // proof for each Output description while adding
1234                // the resulting future to our collection of async
1235                // checks that (at a minimum) must pass for the
1236                // transaction to verify.
1237                async_checks.push(
1238                    primitives::groth16::OUTPUT_VERIFIER
1239                        .clone()
1240                        .oneshot(DescriptionWrapper(output).try_into()?),
1241                );
1242            }
1243
1244            // # Consensus
1245            //
1246            // > The Spend transfers and Action transfers of a transaction MUST be
1247            // > consistent with its vbalanceSapling value as specified in § 4.13
1248            // > ‘Balance and Binding Signature (Sapling)’.
1249            //
1250            // https://zips.z.cash/protocol/protocol.pdf#spendsandoutputs
1251            //
1252            // > [Sapling onward] If effectiveVersion ≥ 4 and
1253            // > nSpendsSapling + nOutputsSapling > 0, then:
1254            // > – let bvk^{Sapling} and SigHash be as defined in § 4.13;
1255            // > – bindingSigSapling MUST represent a valid signature under the
1256            // >   transaction binding validating key bvk Sapling of SigHash —
1257            // >   i.e. BindingSig^{Sapling}.Validate_{bvk^{Sapling}}(SigHash, bindingSigSapling ) = 1.
1258            //
1259            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
1260            //
1261            // This is validated by the verifier. The `if` part is indirectly
1262            // enforced, since the `sapling_shielded_data` is only parsed if those
1263            // conditions apply in [`Transaction::zcash_deserialize`].
1264            //
1265            // >   [NU5 onward] As specified in § 5.4.7, the validation of the 𝑅 component
1266            // >   of the signature changes to prohibit non-canonical encodings.
1267            //
1268            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
1269            //
1270            // This is validated by the verifier, inside the `redjubjub` crate.
1271            // It calls [`jubjub::AffinePoint::from_bytes`] to parse R and
1272            // that enforces the canonical encoding.
1273
1274            let bvk = sapling_shielded_data.binding_verification_key();
1275
1276            async_checks.push(
1277                primitives::redjubjub::VERIFIER
1278                    .clone()
1279                    .oneshot((bvk, sapling_shielded_data.binding_sig, &shielded_sighash).into()),
1280            );
1281        }
1282
1283        Ok(async_checks)
1284    }
1285
1286    /// Verifies a transaction's Orchard shielded data.
1287    fn verify_orchard_shielded_data(
1288        orchard_shielded_data: &Option<orchard::ShieldedData>,
1289        shielded_sighash: &SigHash,
1290    ) -> Result<AsyncChecks, TransactionError> {
1291        let mut async_checks = AsyncChecks::new();
1292
1293        if let Some(orchard_shielded_data) = orchard_shielded_data {
1294            // # Consensus
1295            //
1296            // > The proof 𝜋 MUST be valid given a primary input (cv, rt^{Orchard},
1297            // > nf, rk, cm_x, enableSpends, enableOutputs)
1298            //
1299            // https://zips.z.cash/protocol/protocol.pdf#actiondesc
1300            //
1301            // Unlike Sapling, Orchard shielded transactions have a single
1302            // aggregated Halo2 proof per transaction, even with multiple
1303            // Actions in one transaction. So we queue it for verification
1304            // only once instead of queuing it up for every Action description.
1305            async_checks.push(
1306                primitives::halo2::VERIFIER
1307                    .clone()
1308                    .oneshot(primitives::halo2::Item::from(orchard_shielded_data)),
1309            );
1310
1311            for authorized_action in orchard_shielded_data.actions.iter().cloned() {
1312                let (action, spend_auth_sig) = authorized_action.into_parts();
1313
1314                // # Consensus
1315                //
1316                // > - Let SigHash be the SIGHASH transaction hash of this transaction, not
1317                // >   associated with an input, as defined in § 4.10 using SIGHASH_ALL.
1318                // > - The spend authorization signature MUST be a valid SpendAuthSig^{Orchard}
1319                // >   signature over SigHash using rk as the validating key — i.e.
1320                // >   SpendAuthSig^{Orchard}.Validate_{rk}(SigHash, spendAuthSig) = 1.
1321                // >   As specified in § 5.4.7, validation of the 𝑅 component of the
1322                // >   signature prohibits non-canonical encodings.
1323                //
1324                // https://zips.z.cash/protocol/protocol.pdf#actiondesc
1325                //
1326                // This is validated by the verifier, inside the [`reddsa`] crate.
1327                // It calls [`pallas::Affine::from_bytes`] to parse R and
1328                // that enforces the canonical encoding.
1329                //
1330                // Queue the validation of the RedPallas spend
1331                // authorization signature for each Action
1332                // description while adding the resulting future to
1333                // our collection of async checks that (at a
1334                // minimum) must pass for the transaction to verify.
1335                async_checks.push(primitives::redpallas::VERIFIER.clone().oneshot(
1336                    primitives::redpallas::Item::from_spendauth(
1337                        action.rk,
1338                        spend_auth_sig,
1339                        &shielded_sighash,
1340                    ),
1341                ));
1342            }
1343
1344            let bvk = orchard_shielded_data.binding_verification_key();
1345
1346            // # Consensus
1347            //
1348            // > The Action transfers of a transaction MUST be consistent with
1349            // > its v balanceOrchard value as specified in § 4.14.
1350            //
1351            // https://zips.z.cash/protocol/protocol.pdf#actions
1352            //
1353            // > [NU5 onward] If effectiveVersion ≥ 5 and nActionsOrchard > 0, then:
1354            // > – let bvk^{Orchard} and SigHash be as defined in § 4.14;
1355            // > – bindingSigOrchard MUST represent a valid signature under the
1356            // >   transaction binding validating key bvk^{Orchard} of SigHash —
1357            // >   i.e. BindingSig^{Orchard}.Validate_{bvk^{Orchard}}(SigHash, bindingSigOrchard) = 1.
1358            //
1359            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
1360            //
1361            // This is validated by the verifier. The `if` part is indirectly
1362            // enforced, since the `orchard_shielded_data` is only parsed if those
1363            // conditions apply in [`Transaction::zcash_deserialize`].
1364            //
1365            // >   As specified in § 5.4.7, validation of the 𝑅 component of the signature
1366            // >   prohibits non-canonical encodings.
1367            //
1368            // https://zips.z.cash/protocol/protocol.pdf#txnconsensus
1369            //
1370            // This is validated by the verifier, inside the `reddsa` crate.
1371            // It calls [`pallas::Affine::from_bytes`] to parse R and
1372            // that enforces the canonical encoding.
1373
1374            async_checks.push(primitives::redpallas::VERIFIER.clone().oneshot(
1375                primitives::redpallas::Item::from_binding(
1376                    bvk,
1377                    orchard_shielded_data.binding_sig,
1378                    &shielded_sighash,
1379                ),
1380            ));
1381        }
1382
1383        Ok(async_checks)
1384    }
1385}
1386
1387/// A set of unordered asynchronous checks that should succeed.
1388///
1389/// A wrapper around [`FuturesUnordered`] with some auxiliary methods.
1390struct AsyncChecks(FuturesUnordered<Pin<Box<dyn Future<Output = Result<(), BoxError>> + Send>>>);
1391
1392impl AsyncChecks {
1393    /// Create an empty set of unordered asynchronous checks.
1394    pub fn new() -> Self {
1395        AsyncChecks(FuturesUnordered::new())
1396    }
1397
1398    /// Push a check into the set.
1399    pub fn push(&mut self, check: impl Future<Output = Result<(), BoxError>> + Send + 'static) {
1400        self.0.push(check.boxed());
1401    }
1402
1403    /// Push a set of checks into the set.
1404    ///
1405    /// This method can be daisy-chained.
1406    pub fn and(mut self, checks: AsyncChecks) -> Self {
1407        self.0.extend(checks.0);
1408        self
1409    }
1410
1411    /// Wait until all checks in the set finish.
1412    ///
1413    /// If any of the checks fail, this method immediately returns the error and cancels all other
1414    /// checks by dropping them.
1415    async fn check(mut self) -> Result<(), BoxError> {
1416        // Wait for all asynchronous checks to complete
1417        // successfully, or fail verification if they error.
1418        while let Some(check) = self.0.next().await {
1419            tracing::trace!(?check, remaining = self.0.len());
1420            check?;
1421        }
1422
1423        Ok(())
1424    }
1425}
1426
1427impl<F> FromIterator<F> for AsyncChecks
1428where
1429    F: Future<Output = Result<(), BoxError>> + Send + 'static,
1430{
1431    fn from_iter<I>(iterator: I) -> Self
1432    where
1433        I: IntoIterator<Item = F>,
1434    {
1435        AsyncChecks(iterator.into_iter().map(FutureExt::boxed).collect())
1436    }
1437}