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