zebra_consensus/primitives/
halo2.rs

1//! Async Halo2 batch verifier service
2
3use std::{
4    fmt,
5    future::Future,
6    mem,
7    pin::Pin,
8    task::{Context, Poll},
9};
10
11use futures::{future::BoxFuture, FutureExt};
12use once_cell::sync::Lazy;
13use orchard::circuit::VerifyingKey;
14use rand::{thread_rng, CryptoRng, RngCore};
15
16use thiserror::Error;
17use tokio::sync::watch;
18use tower::{util::ServiceFn, Service};
19use tower_batch_control::{Batch, BatchControl};
20use tower_fallback::Fallback;
21
22use crate::BoxError;
23
24use super::{spawn_fifo, spawn_fifo_and_convert};
25
26#[cfg(test)]
27mod tests;
28
29/// Adjusted batch size for halo2 batches.
30///
31/// Unlike other batch verifiers, halo2 has aggregate proofs.
32/// This means that there can be hundreds of actions verified by some proofs,
33/// but just one action in others.
34///
35/// To compensate for larger proofs, we decrease the batch size.
36///
37/// We also decrease the batch size for these reasons:
38/// - the default number of actions in `zcashd` is 2,
39/// - halo2 proofs take longer to verify than Sapling proofs, and
40/// - transactions with many actions generate very large proofs.
41///
42/// # TODO
43///
44/// Count each halo2 action as a batch item.
45/// We could increase the batch item count by the action count each time a batch request
46/// is received, which would reduce batch size, but keep the batch queue size larger.
47const HALO2_MAX_BATCH_SIZE: usize = 2;
48
49/* TODO: implement batch verification
50
51/// The type of the batch verifier.
52type BatchVerifier = plonk::BatchVerifier<vesta::Affine>;
53 */
54
55/// The type of verification results.
56type VerifyResult = Result<(), Halo2Error>;
57
58/// The type of the batch sender channel.
59type Sender = watch::Sender<Option<VerifyResult>>;
60
61/* TODO: implement batch verification
62
63/// The type of a raw verifying key.
64/// This is the key used to verify batches.
65pub type BatchVerifyingKey = VerifyingKey<vesta::Affine>;
66 */
67/// Temporary substitute type for fake batch verification.
68///
69/// TODO: implement batch verification
70pub type BatchVerifyingKey = ItemVerifyingKey;
71
72/// The type of a prepared verifying key.
73/// This is the key used to verify individual items.
74pub type ItemVerifyingKey = VerifyingKey;
75
76lazy_static::lazy_static! {
77    /// The halo2 proof verifying key.
78    pub static ref VERIFYING_KEY: ItemVerifyingKey = ItemVerifyingKey::build();
79}
80
81// === TEMPORARY BATCH HALO2 SUBSTITUTE ===
82//
83// These types are meant to be API compatible with the batch verification APIs
84// in bellman::groth16::batch, reddsa::batch, redjubjub::batch, and
85// ed25519-zebra::batch. Once Halo2 batch proof verification math and
86// implementation is available, this code can be replaced with that.
87
88/// A Halo2 verification item, used as the request type of the service.
89#[derive(Clone, Debug)]
90pub struct Item {
91    instances: Vec<orchard::circuit::Instance>,
92    proof: orchard::circuit::Proof,
93}
94
95impl Item {
96    /// Perform non-batched verification of this `Item`.
97    ///
98    /// This is useful (in combination with `Item::clone`) for implementing
99    /// fallback logic when batch verification fails.
100    pub fn verify_single(&self, vk: &ItemVerifyingKey) -> Result<(), halo2::plonk::Error> {
101        self.proof.verify(vk, &self.instances[..])
102    }
103}
104
105/// A fake batch verifier that queues and verifies halo2 proofs.
106#[derive(Default)]
107pub struct BatchVerifier {
108    queue: Vec<Item>,
109}
110
111impl BatchVerifier {
112    /// Queues an item for fake batch verification.
113    pub fn queue(&mut self, item: Item) {
114        self.queue.push(item);
115    }
116
117    /// Verifies the current fake batch.
118    pub fn verify<R: RngCore + CryptoRng>(
119        self,
120        _rng: R,
121        vk: &ItemVerifyingKey,
122    ) -> Result<(), halo2::plonk::Error> {
123        for item in self.queue {
124            item.verify_single(vk)?;
125        }
126
127        Ok(())
128    }
129}
130
131// === END TEMPORARY BATCH HALO2 SUBSTITUTE ===
132
133impl From<&zebra_chain::orchard::ShieldedData> for Item {
134    fn from(shielded_data: &zebra_chain::orchard::ShieldedData) -> Item {
135        use orchard::{circuit, note, primitives::redpallas, tree, value};
136
137        let anchor = tree::Anchor::from_bytes(shielded_data.shared_anchor.into()).unwrap();
138
139        let enable_spend = shielded_data
140            .flags
141            .contains(zebra_chain::orchard::Flags::ENABLE_SPENDS);
142        let enable_output = shielded_data
143            .flags
144            .contains(zebra_chain::orchard::Flags::ENABLE_OUTPUTS);
145
146        let instances = shielded_data
147            .actions()
148            .map(|action| {
149                circuit::Instance::from_parts(
150                    anchor,
151                    value::ValueCommitment::from_bytes(&action.cv.into()).unwrap(),
152                    note::Nullifier::from_bytes(&action.nullifier.into()).unwrap(),
153                    redpallas::VerificationKey::<redpallas::SpendAuth>::try_from(<[u8; 32]>::from(
154                        action.rk,
155                    ))
156                    .expect("should be a valid redpallas spendauth verification key"),
157                    note::ExtractedNoteCommitment::from_bytes(&action.cm_x.into()).unwrap(),
158                    enable_spend,
159                    enable_output,
160                )
161            })
162            .collect();
163
164        Item {
165            instances,
166            proof: orchard::circuit::Proof::new(shielded_data.proof.0.clone()),
167        }
168    }
169}
170
171/// An error that may occur when verifying [Halo2 proofs of Zcash Orchard Action
172/// descriptions][actions].
173///
174/// [actions]: https://zips.z.cash/protocol/protocol.pdf#actiondesc
175// TODO: if halo2::plonk::Error gets the std::error::Error trait derived on it,
176// remove this and just wrap `halo2::plonk::Error` as an enum variant of
177// `crate::transaction::Error`, which does the trait derivation via `thiserror`
178#[derive(Clone, Debug, Error, Eq, PartialEq)]
179#[allow(missing_docs)]
180pub enum Halo2Error {
181    #[error("the constraint system is not satisfied")]
182    ConstraintSystemFailure,
183    #[error("unknown Halo2 error")]
184    Other,
185}
186
187impl From<halo2::plonk::Error> for Halo2Error {
188    fn from(err: halo2::plonk::Error) -> Halo2Error {
189        match err {
190            halo2::plonk::Error::ConstraintSystemFailure => Halo2Error::ConstraintSystemFailure,
191            _ => Halo2Error::Other,
192        }
193    }
194}
195
196/// Global batch verification context for Halo2 proofs of Action statements.
197///
198/// This service transparently batches contemporaneous proof verifications,
199/// handling batch failures by falling back to individual verification.
200///
201/// Note that making a `Service` call requires mutable access to the service, so
202/// you should call `.clone()` on the global handle to create a local, mutable
203/// handle.
204pub static VERIFIER: Lazy<
205    Fallback<
206        Batch<Verifier, Item>,
207        ServiceFn<fn(Item) -> BoxFuture<'static, Result<(), BoxError>>>,
208    >,
209> = Lazy::new(|| {
210    Fallback::new(
211        Batch::new(
212            Verifier::new(&VERIFYING_KEY),
213            HALO2_MAX_BATCH_SIZE,
214            None,
215            super::MAX_BATCH_LATENCY,
216        ),
217        // We want to fallback to individual verification if batch verification fails,
218        // so we need a Service to use.
219        //
220        // Because we have to specify the type of a static, we need to be able to
221        // write the type of the closure and its return value. But both closures and
222        // async blocks have unnameable types. So instead we cast the closure to a function
223        // (which is possible because it doesn't capture any state), and use a BoxFuture
224        // to erase the result type.
225        // (We can't use BoxCloneService to erase the service type, because it is !Sync.)
226        tower::service_fn(
227            (|item: Item| Verifier::verify_single_spawning(item, &VERIFYING_KEY).boxed())
228                as fn(_) -> _,
229        ),
230    )
231});
232
233/// Halo2 proof verifier implementation
234///
235/// This is the core implementation for the batch verification logic of the
236/// Halo2 verifier. It handles batching incoming requests, driving batches to
237/// completion, and reporting results.
238pub struct Verifier {
239    /// The synchronous Halo2 batch verifier.
240    batch: BatchVerifier,
241
242    /// The halo2 proof verification key.
243    ///
244    /// Making this 'static makes managing lifetimes much easier.
245    vk: &'static ItemVerifyingKey,
246
247    /// A channel for broadcasting the result of a batch to the futures for each batch item.
248    ///
249    /// Each batch gets a newly created channel, so there is only ever one result sent per channel.
250    /// Tokio doesn't have a oneshot multi-consumer channel, so we use a watch channel.
251    tx: Sender,
252}
253
254impl Verifier {
255    fn new(vk: &'static ItemVerifyingKey) -> Self {
256        let batch = BatchVerifier::default();
257        let (tx, _) = watch::channel(None);
258        Self { batch, vk, tx }
259    }
260
261    /// Returns the batch verifier and channel sender from `self`,
262    /// replacing them with a new empty batch.
263    fn take(&mut self) -> (BatchVerifier, &'static BatchVerifyingKey, Sender) {
264        // Use a new verifier and channel for each batch.
265        let batch = mem::take(&mut self.batch);
266
267        let (tx, _) = watch::channel(None);
268        let tx = mem::replace(&mut self.tx, tx);
269
270        (batch, self.vk, tx)
271    }
272
273    /// Synchronously process the batch, and send the result using the channel sender.
274    /// This function blocks until the batch is completed.
275    fn verify(batch: BatchVerifier, vk: &'static BatchVerifyingKey, tx: Sender) {
276        let result = batch.verify(thread_rng(), vk).map_err(Halo2Error::from);
277        let _ = tx.send(Some(result));
278    }
279
280    /// Flush the batch using a thread pool, and return the result via the channel.
281    /// This returns immediately, usually before the batch is completed.
282    fn flush_blocking(&mut self) {
283        let (batch, vk, tx) = self.take();
284
285        // Correctness: Do CPU-intensive work on a dedicated thread, to avoid blocking other futures.
286        //
287        // We don't care about execution order here, because this method is only called on drop.
288        tokio::task::block_in_place(|| rayon::spawn_fifo(|| Self::verify(batch, vk, tx)));
289    }
290
291    /// Flush the batch using a thread pool, and return the result via the channel.
292    /// This function returns a future that becomes ready when the batch is completed.
293    async fn flush_spawning(batch: BatchVerifier, vk: &'static BatchVerifyingKey, tx: Sender) {
294        // Correctness: Do CPU-intensive work on a dedicated thread, to avoid blocking other futures.
295        let _ = tx.send(
296            spawn_fifo(move || batch.verify(thread_rng(), vk).map_err(Halo2Error::from))
297                .await
298                .ok(),
299        );
300    }
301
302    /// Verify a single item using a thread pool, and return the result.
303    async fn verify_single_spawning(
304        item: Item,
305        pvk: &'static ItemVerifyingKey,
306    ) -> Result<(), BoxError> {
307        // Correctness: Do CPU-intensive work on a dedicated thread, to avoid blocking other futures.
308        spawn_fifo_and_convert(move || item.verify_single(pvk).map_err(Halo2Error::from)).await
309    }
310}
311
312impl fmt::Debug for Verifier {
313    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
314        let name = "Verifier";
315        f.debug_struct(name)
316            .field("batch", &"..")
317            .field("vk", &"..")
318            .field("tx", &self.tx)
319            .finish()
320    }
321}
322
323impl Service<BatchControl<Item>> for Verifier {
324    type Response = ();
325    type Error = BoxError;
326    type Future = Pin<Box<dyn Future<Output = Result<(), BoxError>> + Send + 'static>>;
327
328    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
329        Poll::Ready(Ok(()))
330    }
331
332    fn call(&mut self, req: BatchControl<Item>) -> Self::Future {
333        match req {
334            BatchControl::Item(item) => {
335                tracing::trace!("got item");
336                self.batch.queue(item);
337                let mut rx = self.tx.subscribe();
338                Box::pin(async move {
339                    match rx.changed().await {
340                        Ok(()) => {
341                            // We use a new channel for each batch,
342                            // so we always get the correct batch result here.
343                            let result = rx
344                                .borrow()
345                                .as_ref()
346                                .ok_or("threadpool unexpectedly dropped response channel sender. Is Zebra shutting down?")?
347                                .clone();
348
349                            if result.is_ok() {
350                                tracing::trace!(?result, "verified halo2 proof");
351                                metrics::counter!("proofs.halo2.verified").increment(1);
352                            } else {
353                                tracing::trace!(?result, "invalid halo2 proof");
354                                metrics::counter!("proofs.halo2.invalid").increment(1);
355                            }
356
357                            result.map_err(BoxError::from)
358                        }
359                        Err(_recv_error) => panic!("verifier was dropped without flushing"),
360                    }
361                })
362            }
363
364            BatchControl::Flush => {
365                tracing::trace!("got halo2 flush command");
366
367                let (batch, vk, tx) = self.take();
368
369                Box::pin(Self::flush_spawning(batch, vk, tx).map(Ok))
370            }
371        }
372    }
373}
374
375impl Drop for Verifier {
376    fn drop(&mut self) {
377        // We need to flush the current batch in case there are still any pending futures.
378        // This returns immediately, usually before the batch is completed.
379        self.flush_blocking()
380    }
381}