solana_entry/
entry.rs

1//! The `entry` module is a fundamental building block of Proof of History. It contains a
2//! unique ID that is the hash of the Entry before it, plus the hash of the
3//! transactions within it. Entries cannot be reordered, and its field `num_hashes`
4//! represents an approximate amount of time since the last Entry was created.
5use {
6    crate::poh::Poh,
7    crossbeam_channel::{Receiver, Sender},
8    dlopen2::symbor::{Container, SymBorApi, Symbol},
9    log::*,
10    rand::{thread_rng, Rng},
11    rayon::{prelude::*, ThreadPool},
12    serde::{Deserialize, Serialize},
13    solana_hash::Hash,
14    solana_measure::measure::Measure,
15    solana_merkle_tree::MerkleTree,
16    solana_metrics::*,
17    solana_packet::Meta,
18    solana_perf::{
19        cuda_runtime::PinnedVec,
20        packet::{Packet, PacketBatch, PacketBatchRecycler, PinnedPacketBatch, PACKETS_PER_BATCH},
21        perf_libs,
22        recycler::Recycler,
23        sigverify,
24    },
25    solana_runtime_transaction::transaction_with_meta::TransactionWithMeta,
26    solana_transaction::{
27        versioned::VersionedTransaction, Transaction, TransactionVerificationMode,
28    },
29    solana_transaction_error::{TransactionError, TransactionResult as Result},
30    std::{
31        cmp,
32        ffi::OsStr,
33        iter::repeat_with,
34        sync::{Arc, Mutex, Once, OnceLock},
35        thread::{self, JoinHandle},
36        time::Instant,
37    },
38    wincode::{containers::Pod, SchemaRead, SchemaWrite},
39};
40
41pub type EntrySender = Sender<Vec<Entry>>;
42pub type EntryReceiver = Receiver<Vec<Entry>>;
43
44static API: OnceLock<Container<Api>> = OnceLock::new();
45
46pub fn init_poh() {
47    init(OsStr::new("libpoh-simd.so"));
48}
49
50fn init(name: &OsStr) {
51    static INIT_HOOK: Once = Once::new();
52
53    info!("Loading {name:?}");
54    INIT_HOOK.call_once(|| {
55        let path;
56        let lib_name = if let Some(perf_libs_path) = solana_perf::perf_libs::locate_perf_libs() {
57            solana_perf::perf_libs::append_to_ld_library_path(
58                perf_libs_path.to_str().unwrap_or("").to_string(),
59            );
60            path = perf_libs_path.join(name);
61            path.as_os_str()
62        } else {
63            name
64        };
65
66        match unsafe { Container::load(lib_name) } {
67            Ok(api) => _ = API.set(api),
68            Err(err) => error!("Unable to load {lib_name:?}: {err}"),
69        }
70    })
71}
72
73pub fn api() -> Option<&'static Container<Api<'static>>> {
74    {
75        static INIT_HOOK: Once = Once::new();
76        INIT_HOOK.call_once(|| {
77            if std::env::var("TEST_PERF_LIBS").is_ok() {
78                init_poh()
79            }
80        });
81    }
82
83    API.get()
84}
85
86#[derive(SymBorApi)]
87pub struct Api<'a> {
88    pub poh_verify_many_simd_avx512skx:
89        Symbol<'a, unsafe extern "C" fn(hashes: *mut u8, num_hashes: *const u64)>,
90    pub poh_verify_many_simd_avx2:
91        Symbol<'a, unsafe extern "C" fn(hashes: *mut u8, num_hashes: *const u64)>,
92}
93
94/// Each Entry contains three pieces of data. The `num_hashes` field is the number
95/// of hashes performed since the previous entry.  The `hash` field is the result
96/// of hashing `hash` from the previous entry `num_hashes` times.  The `transactions`
97/// field points to Transactions that took place shortly before `hash` was generated.
98///
99/// If you multiply `num_hashes` by the amount of time it takes to generate a new hash, you
100/// get a duration estimate since the last `Entry`. Since processing power increases
101/// over time, one should expect the duration `num_hashes` represents to decrease proportionally.
102/// An upper bound on Duration can be estimated by assuming each hash was generated by the
103/// world's fastest processor at the time the entry was recorded. Or said another way, it
104/// is physically not possible for a shorter duration to have occurred if one assumes the
105/// hash was computed by the world's fastest processor at that time. The hash chain is both
106/// a Verifiable Delay Function (VDF) and a Proof of Work (not to be confused with Proof of
107/// Work consensus!)
108///
109/// The solana core protocol currently requires an `Entry` to contain `transactions` that are
110/// executable in parallel. Implemented in:
111///
112/// * For TPU: `solana_core::banking_stage::BankingStage::process_and_record_transactions()`
113/// * For TVU: `solana_core::replay_stage::ReplayStage::replay_blockstore_into_bank()`
114///
115/// Until SIMD83 is activated:
116/// All transactions in the `transactions` field have to follow the read/write locking restrictions
117/// with regard to the accounts they reference. A single account can be either written by a single
118/// transaction, or read by one or more transactions, but not both.
119/// This enforcement is done via a call to `solana_runtime::accounts::Accounts::lock_accounts()`
120/// with the `txs` argument holding all the `transactions` in the `Entry`.
121#[derive(Serialize, Deserialize, Debug, Default, PartialEq, Eq, Clone, SchemaWrite, SchemaRead)]
122pub struct Entry {
123    /// The number of hashes since the previous Entry ID.
124    pub num_hashes: u64,
125
126    /// The SHA-256 hash `num_hashes` after the previous Entry ID.
127    #[wincode(with = "Pod<Hash>")]
128    pub hash: Hash,
129
130    /// An unordered list of transactions that were observed before the Entry ID was
131    /// generated. They may have been observed before a previous Entry ID but were
132    /// pushed back into this list to ensure deterministic interpretation of the ledger.
133    #[wincode(with = "Vec<crate::wincode::VersionedTransaction>")]
134    pub transactions: Vec<VersionedTransaction>,
135}
136
137pub struct EntrySummary {
138    pub num_hashes: u64,
139    pub hash: Hash,
140    pub num_transactions: u64,
141}
142
143impl From<&Entry> for EntrySummary {
144    fn from(entry: &Entry) -> Self {
145        Self {
146            num_hashes: entry.num_hashes,
147            hash: entry.hash,
148            num_transactions: entry.transactions.len() as u64,
149        }
150    }
151}
152
153/// Typed entry to distinguish between transaction and tick entries
154pub enum EntryType<Tx: TransactionWithMeta> {
155    Transactions(Vec<Tx>),
156    Tick(Hash),
157}
158
159impl Entry {
160    /// Creates the next Entry `num_hashes` after `start_hash`.
161    pub fn new(prev_hash: &Hash, mut num_hashes: u64, transactions: Vec<Transaction>) -> Self {
162        // If you passed in transactions, but passed in num_hashes == 0, then
163        // next_hash will generate the next hash and set num_hashes == 1
164        if num_hashes == 0 && !transactions.is_empty() {
165            num_hashes = 1;
166        }
167
168        let transactions = transactions.into_iter().map(Into::into).collect::<Vec<_>>();
169        let hash = next_hash(prev_hash, num_hashes, &transactions);
170        Entry {
171            num_hashes,
172            hash,
173            transactions,
174        }
175    }
176
177    pub fn new_mut(
178        start_hash: &mut Hash,
179        num_hashes: &mut u64,
180        transactions: Vec<Transaction>,
181    ) -> Self {
182        let entry = Self::new(start_hash, *num_hashes, transactions);
183        *start_hash = entry.hash;
184        *num_hashes = 0;
185
186        entry
187    }
188
189    #[cfg(test)]
190    pub fn new_tick(num_hashes: u64, hash: &Hash) -> Self {
191        Entry {
192            num_hashes,
193            hash: *hash,
194            transactions: vec![],
195        }
196    }
197
198    /// Verifies self.hash is the result of hashing a `start_hash` `self.num_hashes` times.
199    /// If the transaction is not a Tick, then hash that as well.
200    pub fn verify(&self, start_hash: &Hash) -> bool {
201        let ref_hash = next_hash(start_hash, self.num_hashes, &self.transactions);
202        if self.hash != ref_hash {
203            warn!(
204                "next_hash is invalid expected: {:?} actual: {:?}",
205                self.hash, ref_hash
206            );
207            return false;
208        }
209        true
210    }
211
212    pub fn is_tick(&self) -> bool {
213        self.transactions.is_empty()
214    }
215}
216
217pub fn hash_transactions(transactions: &[VersionedTransaction]) -> Hash {
218    // a hash of a slice of transactions only needs to hash the signatures
219    let signatures: Vec<_> = transactions
220        .iter()
221        .flat_map(|tx| tx.signatures.iter())
222        .collect();
223    let merkle_tree = MerkleTree::new(&signatures);
224    if let Some(root_hash) = merkle_tree.get_root() {
225        *root_hash
226    } else {
227        Hash::default()
228    }
229}
230
231/// Creates the hash `num_hashes` after `start_hash`. If the transaction contains
232/// a signature, the final hash will be a hash of both the previous ID and
233/// the signature.  If num_hashes is zero and there's no transaction data,
234///  start_hash is returned.
235pub fn next_hash(
236    start_hash: &Hash,
237    num_hashes: u64,
238    transactions: &[VersionedTransaction],
239) -> Hash {
240    if num_hashes == 0 && transactions.is_empty() {
241        return *start_hash;
242    }
243
244    let mut poh = Poh::new(*start_hash, None);
245    poh.hash(num_hashes.saturating_sub(1));
246    if transactions.is_empty() {
247        poh.tick().unwrap().hash
248    } else {
249        poh.record(hash_transactions(transactions)).unwrap().hash
250    }
251}
252
253/// Last action required to verify an entry
254enum VerifyAction {
255    /// Mixin a hash before computing the last hash for a transaction entry
256    Mixin(Hash),
257    /// Compute one last hash for a tick entry
258    Tick,
259    /// No action needed (tick entry with no hashes)
260    None,
261}
262
263pub struct GpuVerificationData {
264    thread_h: Option<JoinHandle<u64>>,
265    hashes: Option<Arc<Mutex<PinnedVec<Hash>>>>,
266    verifications: Option<Vec<(VerifyAction, Hash)>>,
267}
268
269pub enum DeviceVerificationData {
270    Cpu(),
271    Gpu(GpuVerificationData),
272}
273
274pub struct EntryVerificationState {
275    verification_status: EntryVerificationStatus,
276    poh_duration_us: u64,
277    device_verification_data: DeviceVerificationData,
278}
279
280pub struct GpuSigVerificationData {
281    thread_h: Option<JoinHandle<(bool, u64)>>,
282}
283
284pub enum DeviceSigVerificationData {
285    Cpu(),
286    Gpu(GpuSigVerificationData),
287}
288
289pub struct EntrySigVerificationState<Tx: TransactionWithMeta> {
290    verification_status: EntryVerificationStatus,
291    entries: Option<Vec<EntryType<Tx>>>,
292    device_verification_data: DeviceSigVerificationData,
293    gpu_verify_duration_us: u64,
294}
295
296impl<Tx: TransactionWithMeta> EntrySigVerificationState<Tx> {
297    pub fn entries(&mut self) -> Option<Vec<EntryType<Tx>>> {
298        self.entries.take()
299    }
300    pub fn finish_verify(&mut self) -> bool {
301        match &mut self.device_verification_data {
302            DeviceSigVerificationData::Gpu(verification_state) => {
303                let (verified, gpu_time_us) =
304                    verification_state.thread_h.take().unwrap().join().unwrap();
305                self.gpu_verify_duration_us = gpu_time_us;
306                self.verification_status = if verified {
307                    EntryVerificationStatus::Success
308                } else {
309                    EntryVerificationStatus::Failure
310                };
311                verified
312            }
313            DeviceSigVerificationData::Cpu() => {
314                self.verification_status == EntryVerificationStatus::Success
315            }
316        }
317    }
318    pub fn status(&self) -> EntryVerificationStatus {
319        self.verification_status
320    }
321    pub fn gpu_verify_duration(&self) -> u64 {
322        self.gpu_verify_duration_us
323    }
324}
325
326#[derive(Default, Clone)]
327pub struct VerifyRecyclers {
328    hash_recycler: Recycler<PinnedVec<Hash>>,
329    tick_count_recycler: Recycler<PinnedVec<u64>>,
330    packet_recycler: PacketBatchRecycler,
331    out_recycler: Recycler<PinnedVec<u8>>,
332    tx_offset_recycler: Recycler<sigverify::TxOffset>,
333}
334
335#[derive(PartialEq, Eq, Clone, Copy, Debug)]
336pub enum EntryVerificationStatus {
337    Failure,
338    Success,
339    Pending,
340}
341
342impl EntryVerificationState {
343    pub fn status(&self) -> EntryVerificationStatus {
344        self.verification_status
345    }
346
347    pub fn poh_duration_us(&self) -> u64 {
348        self.poh_duration_us
349    }
350
351    pub fn finish_verify(&mut self, thread_pool: &ThreadPool) -> bool {
352        match &mut self.device_verification_data {
353            DeviceVerificationData::Gpu(verification_state) => {
354                let gpu_time_us = verification_state.thread_h.take().unwrap().join().unwrap();
355
356                let mut verify_check_time = Measure::start("verify_check");
357                let hashes = verification_state.hashes.take().unwrap();
358                let hashes = Arc::try_unwrap(hashes)
359                    .expect("unwrap Arc")
360                    .into_inner()
361                    .expect("into_inner");
362                let res = thread_pool.install(|| {
363                    hashes
364                        .into_par_iter()
365                        .cloned()
366                        .zip(verification_state.verifications.take().unwrap())
367                        .all(|(hash, (action, expected))| {
368                            let actual = match action {
369                                VerifyAction::Mixin(mixin) => {
370                                    Poh::new(hash, None).record(mixin).unwrap().hash
371                                }
372                                VerifyAction::Tick => Poh::new(hash, None).tick().unwrap().hash,
373                                VerifyAction::None => hash,
374                            };
375                            actual == expected
376                        })
377                });
378                verify_check_time.stop();
379                self.poh_duration_us += gpu_time_us + verify_check_time.as_us();
380
381                self.verification_status = if res {
382                    EntryVerificationStatus::Success
383                } else {
384                    EntryVerificationStatus::Failure
385                };
386                res
387            }
388            DeviceVerificationData::Cpu() => {
389                self.verification_status == EntryVerificationStatus::Success
390            }
391        }
392    }
393}
394
395pub fn verify_transactions<Tx: TransactionWithMeta + Send + Sync>(
396    entries: Vec<Entry>,
397    thread_pool: &ThreadPool,
398    verify: Arc<dyn Fn(VersionedTransaction) -> Result<Tx> + Send + Sync>,
399) -> Result<Vec<EntryType<Tx>>> {
400    thread_pool.install(|| {
401        entries
402            .into_par_iter()
403            .map(|entry| {
404                if entry.transactions.is_empty() {
405                    Ok(EntryType::Tick(entry.hash))
406                } else {
407                    Ok(EntryType::Transactions(
408                        entry
409                            .transactions
410                            .into_par_iter()
411                            .map(verify.as_ref())
412                            .collect::<Result<Vec<_>>>()?,
413                    ))
414                }
415            })
416            .collect()
417    })
418}
419
420pub fn start_verify_transactions<Tx: TransactionWithMeta + Send + Sync + 'static>(
421    entries: Vec<Entry>,
422    skip_verification: bool,
423    thread_pool: &ThreadPool,
424    verify_recyclers: VerifyRecyclers,
425    verify: Arc<
426        dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<Tx> + Send + Sync,
427    >,
428) -> Result<EntrySigVerificationState<Tx>> {
429    let api = perf_libs::api();
430
431    // Use the CPU if we have too few transactions for GPU signature verification to be worth it.
432    // We will also use the CPU if no acceleration API is used or if we're skipping
433    // the signature verification as we'd have nothing to do on the GPU in that case.
434    // TODO: make the CPU-to GPU crossover point dynamic, perhaps based on similar future
435    // heuristics to what might be used in sigverify::ed25519_verify when a dynamic crossover
436    // is introduced for that function (see TODO in sigverify::ed25519_verify)
437    let use_cpu = skip_verification
438        || api.is_none()
439        || entries
440            .iter()
441            .try_fold(0, |accum: usize, entry: &Entry| -> Option<usize> {
442                if accum.saturating_add(entry.transactions.len()) < 512 {
443                    Some(accum.saturating_add(entry.transactions.len()))
444                } else {
445                    None
446                }
447            })
448            .is_some();
449
450    if use_cpu {
451        start_verify_transactions_cpu(entries, skip_verification, thread_pool, verify)
452    } else {
453        start_verify_transactions_gpu(entries, verify_recyclers, thread_pool, verify)
454    }
455}
456
457fn start_verify_transactions_cpu<Tx: TransactionWithMeta + Send + Sync + 'static>(
458    entries: Vec<Entry>,
459    skip_verification: bool,
460    thread_pool: &ThreadPool,
461    verify: Arc<
462        dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<Tx> + Send + Sync,
463    >,
464) -> Result<EntrySigVerificationState<Tx>> {
465    let verify_func = {
466        let mode = if skip_verification {
467            TransactionVerificationMode::HashOnly
468        } else {
469            TransactionVerificationMode::FullVerification
470        };
471
472        move |versioned_tx| verify(versioned_tx, mode)
473    };
474
475    let entries = verify_transactions(entries, thread_pool, Arc::new(verify_func))?;
476
477    Ok(EntrySigVerificationState {
478        verification_status: EntryVerificationStatus::Success,
479        entries: Some(entries),
480        device_verification_data: DeviceSigVerificationData::Cpu(),
481        gpu_verify_duration_us: 0,
482    })
483}
484
485fn start_verify_transactions_gpu<Tx: TransactionWithMeta + Send + Sync + 'static>(
486    entries: Vec<Entry>,
487    verify_recyclers: VerifyRecyclers,
488    thread_pool: &ThreadPool,
489    verify: Arc<
490        dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<Tx> + Send + Sync,
491    >,
492) -> Result<EntrySigVerificationState<Tx>> {
493    let verify_func = {
494        move |versioned_tx: VersionedTransaction| -> Result<Tx> {
495            verify(versioned_tx, TransactionVerificationMode::HashOnly)
496        }
497    };
498
499    let entries = verify_transactions(entries, thread_pool, Arc::new(verify_func))?;
500
501    let transactions = entries
502        .iter()
503        .filter_map(|entry_type| match entry_type {
504            EntryType::Tick(_) => None,
505            EntryType::Transactions(transactions) => Some(transactions),
506        })
507        .flatten()
508        .collect::<Vec<_>>();
509
510    if transactions.is_empty() {
511        return Ok(EntrySigVerificationState {
512            verification_status: EntryVerificationStatus::Success,
513            entries: Some(entries),
514            device_verification_data: DeviceSigVerificationData::Cpu(),
515            gpu_verify_duration_us: 0,
516        });
517    }
518
519    let packet_batches = thread_pool.install(|| {
520        transactions
521            .par_chunks(PACKETS_PER_BATCH)
522            .map(|transaction_chunk| {
523                let num_transactions = transaction_chunk.len();
524                let mut packet_batch = PinnedPacketBatch::new_with_recycler(
525                    &verify_recyclers.packet_recycler,
526                    num_transactions,
527                    "entry-sig-verify",
528                );
529                // We use set_len here instead of resize(num_txs, Packet::default()), to save
530                // memory bandwidth and avoid writing a large amount of data that will be overwritten
531                // soon afterwards. As well, Packet::default() actually leaves the packet data
532                // uninitialized, so the initialization would simply write junk into
533                // the vector anyway.
534                unsafe {
535                    packet_batch.set_len(num_transactions);
536                }
537                let transaction_iter = transaction_chunk
538                    .iter()
539                    .map(|tx| tx.to_versioned_transaction());
540
541                let res = packet_batch
542                    .iter_mut()
543                    .zip(transaction_iter)
544                    .all(|(packet, tx)| {
545                        *packet.meta_mut() = Meta::default();
546                        Packet::populate_packet(packet, None, &tx).is_ok()
547                    });
548                if res {
549                    Ok(PacketBatch::from(packet_batch))
550                } else {
551                    Err(TransactionError::SanitizeFailure)
552                }
553            })
554            .collect::<Result<Vec<_>>>()
555    });
556    let mut packet_batches = packet_batches?;
557
558    let tx_offset_recycler = verify_recyclers.tx_offset_recycler;
559    let out_recycler = verify_recyclers.out_recycler;
560    let num_packets = transactions.len();
561    let gpu_verify_thread = thread::Builder::new()
562        .name("solGpuSigVerify".into())
563        .spawn(move || {
564            let mut verify_time = Measure::start("sigverify");
565            sigverify::ed25519_verify(
566                &mut packet_batches,
567                &tx_offset_recycler,
568                &out_recycler,
569                false,
570                num_packets,
571            );
572            let verified = packet_batches
573                .iter()
574                .all(|batch| batch.iter().all(|p| !p.meta().discard()));
575            verify_time.stop();
576            (verified, verify_time.as_us())
577        })
578        .unwrap();
579
580    Ok(EntrySigVerificationState {
581        verification_status: EntryVerificationStatus::Pending,
582        entries: Some(entries),
583        device_verification_data: DeviceSigVerificationData::Gpu(GpuSigVerificationData {
584            thread_h: Some(gpu_verify_thread),
585        }),
586        gpu_verify_duration_us: 0,
587    })
588}
589
590fn compare_hashes(computed_hash: Hash, ref_entry: &Entry) -> bool {
591    let actual = if !ref_entry.transactions.is_empty() {
592        let tx_hash = hash_transactions(&ref_entry.transactions);
593        let mut poh = Poh::new(computed_hash, None);
594        poh.record(tx_hash).unwrap().hash
595    } else if ref_entry.num_hashes > 0 {
596        let mut poh = Poh::new(computed_hash, None);
597        poh.tick().unwrap().hash
598    } else {
599        computed_hash
600    };
601    actual == ref_entry.hash
602}
603
604// an EntrySlice is a slice of Entries
605pub trait EntrySlice {
606    /// Verifies the hashes and counts of a slice of transactions are all consistent.
607    fn verify_cpu(&self, start_hash: &Hash, thread_pool: &ThreadPool) -> EntryVerificationState;
608    fn verify_cpu_generic(
609        &self,
610        start_hash: &Hash,
611        thread_pool: &ThreadPool,
612    ) -> EntryVerificationState;
613    fn verify_cpu_x86_simd(
614        &self,
615        start_hash: &Hash,
616        simd_len: usize,
617        thread_pool: &ThreadPool,
618    ) -> EntryVerificationState;
619    fn start_verify(
620        &self,
621        start_hash: &Hash,
622        thread_pool: &ThreadPool,
623        recyclers: VerifyRecyclers,
624    ) -> EntryVerificationState;
625    fn verify(&self, start_hash: &Hash, thread_pool: &ThreadPool) -> bool;
626    /// Checks that each entry tick has the correct number of hashes. Entry slices do not
627    /// necessarily end in a tick, so `tick_hash_count` is used to carry over the hash count
628    /// for the next entry slice.
629    fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool;
630    /// Counts tick entries
631    fn tick_count(&self) -> u64;
632}
633
634impl EntrySlice for [Entry] {
635    fn verify(&self, start_hash: &Hash, thread_pool: &ThreadPool) -> bool {
636        self.start_verify(start_hash, thread_pool, VerifyRecyclers::default())
637            .finish_verify(thread_pool)
638    }
639
640    fn verify_cpu_generic(
641        &self,
642        start_hash: &Hash,
643        thread_pool: &ThreadPool,
644    ) -> EntryVerificationState {
645        let now = Instant::now();
646        let genesis = [Entry {
647            num_hashes: 0,
648            hash: *start_hash,
649            transactions: vec![],
650        }];
651        let entry_pairs = genesis.par_iter().chain(self).zip(self);
652        let res = thread_pool.install(|| {
653            entry_pairs.all(|(x0, x1)| {
654                let r = x1.verify(&x0.hash);
655                if !r {
656                    warn!(
657                        "entry invalid!: x0: {:?}, x1: {:?} num txs: {}",
658                        x0.hash,
659                        x1.hash,
660                        x1.transactions.len()
661                    );
662                }
663                r
664            })
665        });
666        let poh_duration_us = now.elapsed().as_micros() as u64;
667        EntryVerificationState {
668            verification_status: if res {
669                EntryVerificationStatus::Success
670            } else {
671                EntryVerificationStatus::Failure
672            },
673            poh_duration_us,
674            device_verification_data: DeviceVerificationData::Cpu(),
675        }
676    }
677
678    fn verify_cpu_x86_simd(
679        &self,
680        start_hash: &Hash,
681        simd_len: usize,
682        thread_pool: &ThreadPool,
683    ) -> EntryVerificationState {
684        use solana_hash::HASH_BYTES;
685        let now = Instant::now();
686        let genesis = [Entry {
687            num_hashes: 0,
688            hash: *start_hash,
689            transactions: vec![],
690        }];
691
692        let aligned_len = self.len().div_ceil(simd_len) * simd_len;
693        let mut hashes_bytes = vec![0u8; HASH_BYTES * aligned_len];
694        genesis
695            .iter()
696            .chain(self)
697            .enumerate()
698            .for_each(|(i, entry)| {
699                if i < self.len() {
700                    let start = i * HASH_BYTES;
701                    let end = start + HASH_BYTES;
702                    hashes_bytes[start..end].copy_from_slice(&entry.hash.to_bytes());
703                }
704            });
705        let mut hashes_chunked: Vec<_> = hashes_bytes.chunks_mut(simd_len * HASH_BYTES).collect();
706
707        let mut num_hashes: Vec<u64> = self
708            .iter()
709            .map(|entry| entry.num_hashes.saturating_sub(1))
710            .collect();
711        num_hashes.resize(aligned_len, 0);
712        let num_hashes: Vec<_> = num_hashes.chunks(simd_len).collect();
713
714        let res = thread_pool.install(|| {
715            hashes_chunked
716                .par_iter_mut()
717                .zip(num_hashes)
718                .enumerate()
719                .all(|(i, (chunk, num_hashes))| {
720                    match simd_len {
721                        8 => unsafe {
722                            (api().unwrap().poh_verify_many_simd_avx2)(
723                                chunk.as_mut_ptr(),
724                                num_hashes.as_ptr(),
725                            );
726                        },
727                        16 => unsafe {
728                            (api().unwrap().poh_verify_many_simd_avx512skx)(
729                                chunk.as_mut_ptr(),
730                                num_hashes.as_ptr(),
731                            );
732                        },
733                        _ => {
734                            panic!("unsupported simd len: {simd_len}");
735                        }
736                    }
737                    let entry_start = i * simd_len;
738                    // The last chunk may produce indexes larger than what we have in the reference entries
739                    // because it is aligned to simd_len.
740                    let entry_end = std::cmp::min(entry_start + simd_len, self.len());
741                    self[entry_start..entry_end]
742                        .iter()
743                        .enumerate()
744                        .all(|(j, ref_entry)| {
745                            let start = j * HASH_BYTES;
746                            let end = start + HASH_BYTES;
747                            let hash = <[u8; HASH_BYTES]>::try_from(&chunk[start..end])
748                                .map(Hash::new_from_array)
749                                .unwrap();
750                            compare_hashes(hash, ref_entry)
751                        })
752                })
753        });
754        let poh_duration_us = now.elapsed().as_micros() as u64;
755        EntryVerificationState {
756            verification_status: if res {
757                EntryVerificationStatus::Success
758            } else {
759                EntryVerificationStatus::Failure
760            },
761            poh_duration_us,
762            device_verification_data: DeviceVerificationData::Cpu(),
763        }
764    }
765
766    fn verify_cpu(&self, start_hash: &Hash, thread_pool: &ThreadPool) -> EntryVerificationState {
767        #[cfg(any(target_arch = "x86", target_arch = "x86_64"))]
768        let (has_avx2, has_avx512) = (
769            is_x86_feature_detected!("avx2"),
770            is_x86_feature_detected!("avx512f"),
771        );
772        #[cfg(not(any(target_arch = "x86", target_arch = "x86_64")))]
773        let (has_avx2, has_avx512) = (false, false);
774
775        if api().is_some() {
776            if has_avx512 && self.len() >= 128 {
777                self.verify_cpu_x86_simd(start_hash, 16, thread_pool)
778            } else if has_avx2 && self.len() >= 48 {
779                self.verify_cpu_x86_simd(start_hash, 8, thread_pool)
780            } else {
781                self.verify_cpu_generic(start_hash, thread_pool)
782            }
783        } else {
784            self.verify_cpu_generic(start_hash, thread_pool)
785        }
786    }
787
788    fn start_verify(
789        &self,
790        start_hash: &Hash,
791        thread_pool: &ThreadPool,
792        recyclers: VerifyRecyclers,
793    ) -> EntryVerificationState {
794        let start = Instant::now();
795        let Some(api) = perf_libs::api() else {
796            return self.verify_cpu(start_hash, thread_pool);
797        };
798        inc_new_counter_info!("entry_verify-num_entries", self.len());
799
800        let genesis = [Entry {
801            num_hashes: 0,
802            hash: *start_hash,
803            transactions: vec![],
804        }];
805
806        let hashes: Vec<Hash> = genesis
807            .iter()
808            .chain(self)
809            .map(|entry| entry.hash)
810            .take(self.len())
811            .collect();
812
813        let mut hashes_pinned = recyclers.hash_recycler.allocate("poh_verify_hash");
814        hashes_pinned.set_pinnable();
815        hashes_pinned.resize(hashes.len(), Hash::default());
816        hashes_pinned.copy_from_slice(&hashes);
817
818        let mut num_hashes_vec = recyclers
819            .tick_count_recycler
820            .allocate("poh_verify_num_hashes");
821        num_hashes_vec.reserve_and_pin(cmp::max(1, self.len()));
822        for entry in self {
823            num_hashes_vec.push(entry.num_hashes.saturating_sub(1));
824        }
825
826        let length = self.len();
827        let hashes = Arc::new(Mutex::new(hashes_pinned));
828        let hashes_clone = hashes.clone();
829
830        let gpu_verify_thread = thread::Builder::new()
831            .name("solGpuPohVerify".into())
832            .spawn(move || {
833                let mut hashes = hashes_clone.lock().unwrap();
834                let gpu_wait = Instant::now();
835                let res;
836                unsafe {
837                    res = (api.poh_verify_many)(
838                        hashes.as_mut_ptr() as *mut u8,
839                        num_hashes_vec.as_ptr(),
840                        length,
841                        1,
842                    );
843                }
844                assert!(res == 0, "GPU PoH verify many failed");
845                inc_new_counter_info!(
846                    "entry_verify-gpu_thread",
847                    gpu_wait.elapsed().as_micros() as usize
848                );
849                gpu_wait.elapsed().as_micros() as u64
850            })
851            .unwrap();
852
853        let verifications = thread_pool.install(|| {
854            self.into_par_iter()
855                .map(|entry| {
856                    let answer = entry.hash;
857                    let action = if entry.transactions.is_empty() {
858                        if entry.num_hashes == 0 {
859                            VerifyAction::None
860                        } else {
861                            VerifyAction::Tick
862                        }
863                    } else {
864                        VerifyAction::Mixin(hash_transactions(&entry.transactions))
865                    };
866                    (action, answer)
867                })
868                .collect()
869        });
870        let device_verification_data = DeviceVerificationData::Gpu(GpuVerificationData {
871            thread_h: Some(gpu_verify_thread),
872            verifications: Some(verifications),
873            hashes: Some(hashes),
874        });
875        EntryVerificationState {
876            verification_status: EntryVerificationStatus::Pending,
877            poh_duration_us: start.elapsed().as_micros() as u64,
878            device_verification_data,
879        }
880    }
881
882    fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool {
883        // When hashes_per_tick is 0, hashing is disabled.
884        if hashes_per_tick == 0 {
885            return true;
886        }
887
888        for entry in self {
889            *tick_hash_count = tick_hash_count.saturating_add(entry.num_hashes);
890            if entry.is_tick() {
891                if *tick_hash_count != hashes_per_tick {
892                    warn!(
893                        "invalid tick hash count!: entry: {entry:#?}, tick_hash_count: \
894                         {tick_hash_count}, hashes_per_tick: {hashes_per_tick}"
895                    );
896                    return false;
897                }
898                *tick_hash_count = 0;
899            }
900        }
901        *tick_hash_count < hashes_per_tick
902    }
903
904    fn tick_count(&self) -> u64 {
905        self.iter().filter(|e| e.is_tick()).count() as u64
906    }
907}
908
909pub fn next_entry_mut(start: &mut Hash, num_hashes: u64, transactions: Vec<Transaction>) -> Entry {
910    let entry = Entry::new(start, num_hashes, transactions);
911    *start = entry.hash;
912    entry
913}
914
915pub fn create_ticks(num_ticks: u64, hashes_per_tick: u64, mut hash: Hash) -> Vec<Entry> {
916    repeat_with(|| next_entry_mut(&mut hash, hashes_per_tick, vec![]))
917        .take(num_ticks as usize)
918        .collect()
919}
920
921pub fn create_random_ticks(num_ticks: u64, max_hashes_per_tick: u64, mut hash: Hash) -> Vec<Entry> {
922    repeat_with(|| {
923        let hashes_per_tick = thread_rng().gen_range(1..max_hashes_per_tick);
924        next_entry_mut(&mut hash, hashes_per_tick, vec![])
925    })
926    .take(num_ticks as usize)
927    .collect()
928}
929
930/// Creates the next Tick or Transaction Entry `num_hashes` after `start_hash`.
931pub fn next_entry(prev_hash: &Hash, num_hashes: u64, transactions: Vec<Transaction>) -> Entry {
932    let transactions = transactions.into_iter().map(Into::into).collect::<Vec<_>>();
933    next_versioned_entry(prev_hash, num_hashes, transactions)
934}
935
936/// Creates the next Tick or Transaction Entry `num_hashes` after `start_hash`.
937pub fn next_versioned_entry(
938    prev_hash: &Hash,
939    num_hashes: u64,
940    transactions: Vec<VersionedTransaction>,
941) -> Entry {
942    assert!(num_hashes > 0 || transactions.is_empty());
943    Entry {
944        num_hashes,
945        hash: next_hash(prev_hash, num_hashes, &transactions),
946        transactions,
947    }
948}
949
950pub fn thread_pool_for_tests() -> ThreadPool {
951    // Allocate fewer threads for unit tests
952    // Unit tests typically aren't creating massive blocks to verify, and
953    // multiple tests could be running in parallel so any further parallelism
954    // will do more harm than good
955    rayon::ThreadPoolBuilder::new()
956        .num_threads(4)
957        .thread_name(|i| format!("solEntryTest{i:02}"))
958        .build()
959        .expect("new rayon threadpool")
960}
961
962#[cfg(feature = "dev-context-only-utils")]
963pub fn thread_pool_for_benches() -> ThreadPool {
964    rayon::ThreadPoolBuilder::new()
965        .num_threads(num_cpus::get())
966        .thread_name(|i| format!("solEntryBnch{i:02}"))
967        .build()
968        .expect("new rayon threadpool")
969}
970
971#[cfg(test)]
972mod tests {
973    use {
974        super::*,
975        agave_reserved_account_keys::ReservedAccountKeys,
976        solana_hash::Hash,
977        solana_keypair::Keypair,
978        solana_message::SimpleAddressLoader,
979        solana_perf::test_tx::{test_invalid_tx, test_tx},
980        solana_pubkey::Pubkey,
981        solana_runtime_transaction::runtime_transaction::RuntimeTransaction,
982        solana_sha256_hasher::hash,
983        solana_signer::Signer,
984        solana_system_transaction as system_transaction,
985        solana_transaction::{
986            sanitized::{MessageHash, SanitizedTransaction},
987            versioned::VersionedTransaction,
988        },
989        solana_transaction_error::TransactionResult as Result,
990    };
991
992    #[test]
993    fn test_entry_verify() {
994        let zero = Hash::default();
995        let one = hash(zero.as_ref());
996        assert!(Entry::new_tick(0, &zero).verify(&zero)); // base case, never used
997        assert!(!Entry::new_tick(0, &zero).verify(&one)); // base case, bad
998        assert!(next_entry(&zero, 1, vec![]).verify(&zero)); // inductive step
999        assert!(!next_entry(&zero, 1, vec![]).verify(&one)); // inductive step, bad
1000    }
1001
1002    fn test_verify_transactions<Tx: TransactionWithMeta + Send + Sync + 'static>(
1003        entries: Vec<Entry>,
1004        skip_verification: bool,
1005        verify_recyclers: VerifyRecyclers,
1006        thread_pool: &ThreadPool,
1007        verify: Arc<
1008            dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<Tx> + Send + Sync,
1009        >,
1010    ) -> bool {
1011        let verify_func = {
1012            let verify = verify.clone();
1013            let verification_mode = if skip_verification {
1014                TransactionVerificationMode::HashOnly
1015            } else {
1016                TransactionVerificationMode::FullVerification
1017            };
1018            move |versioned_tx: VersionedTransaction| -> Result<Tx> {
1019                verify(versioned_tx, verification_mode)
1020            }
1021        };
1022
1023        let cpu_verify_result =
1024            verify_transactions(entries.clone(), thread_pool, Arc::new(verify_func));
1025        let mut gpu_verify_result: EntrySigVerificationState<Tx> = {
1026            let verify_result = start_verify_transactions(
1027                entries,
1028                skip_verification,
1029                thread_pool,
1030                verify_recyclers,
1031                verify,
1032            );
1033            match verify_result {
1034                Ok(res) => res,
1035                _ => EntrySigVerificationState {
1036                    verification_status: EntryVerificationStatus::Failure,
1037                    entries: None,
1038                    device_verification_data: DeviceSigVerificationData::Cpu(),
1039                    gpu_verify_duration_us: 0,
1040                },
1041            }
1042        };
1043
1044        match cpu_verify_result {
1045            Ok(_) => {
1046                assert!(gpu_verify_result.verification_status != EntryVerificationStatus::Failure);
1047                assert!(gpu_verify_result.finish_verify());
1048                true
1049            }
1050            _ => {
1051                assert!(
1052                    gpu_verify_result.verification_status == EntryVerificationStatus::Failure
1053                        || !gpu_verify_result.finish_verify()
1054                );
1055                false
1056            }
1057        }
1058    }
1059
1060    #[test]
1061    fn test_entry_gpu_verify() {
1062        let thread_pool = thread_pool_for_tests();
1063
1064        let verify_transaction = {
1065            move |versioned_tx: VersionedTransaction,
1066                  verification_mode: TransactionVerificationMode|
1067                  -> Result<RuntimeTransaction<SanitizedTransaction>> {
1068                let sanitized_tx = {
1069                    let message_hash =
1070                        if verification_mode == TransactionVerificationMode::FullVerification {
1071                            versioned_tx.verify_and_hash_message()?
1072                        } else {
1073                            versioned_tx.message.hash()
1074                        };
1075
1076                    RuntimeTransaction::try_create(
1077                        versioned_tx,
1078                        MessageHash::Precomputed(message_hash),
1079                        None,
1080                        SimpleAddressLoader::Disabled,
1081                        &ReservedAccountKeys::empty_key_set(),
1082                        true,
1083                    )
1084                }?;
1085
1086                Ok(sanitized_tx)
1087            }
1088        };
1089
1090        let recycler = VerifyRecyclers::default();
1091
1092        // Make sure we test with a number of transactions that's not a multiple of PACKETS_PER_BATCH
1093        let entries_invalid = (0..1025)
1094            .map(|_| {
1095                let transaction = test_invalid_tx();
1096                next_entry_mut(&mut Hash::default(), 0, vec![transaction])
1097            })
1098            .collect::<Vec<_>>();
1099
1100        let entries_valid = (0..1025)
1101            .map(|_| {
1102                let transaction = test_tx();
1103                next_entry_mut(&mut Hash::default(), 0, vec![transaction])
1104            })
1105            .collect::<Vec<_>>();
1106
1107        assert!(!test_verify_transactions(
1108            entries_invalid,
1109            false,
1110            recycler.clone(),
1111            &thread_pool,
1112            Arc::new(verify_transaction)
1113        ));
1114        assert!(test_verify_transactions(
1115            entries_valid,
1116            false,
1117            recycler,
1118            &thread_pool,
1119            Arc::new(verify_transaction)
1120        ));
1121    }
1122
1123    #[test]
1124    fn test_transaction_reorder_attack() {
1125        let zero = Hash::default();
1126
1127        // First, verify entries
1128        let keypair = Keypair::new();
1129        let tx0 = system_transaction::transfer(&keypair, &keypair.pubkey(), 0, zero);
1130        let tx1 = system_transaction::transfer(&keypair, &keypair.pubkey(), 1, zero);
1131        let mut e0 = Entry::new(&zero, 0, vec![tx0.clone(), tx1.clone()]);
1132        assert!(e0.verify(&zero));
1133
1134        // Next, swap two transactions and ensure verification fails.
1135        e0.transactions[0] = tx1.into(); // <-- attack
1136        e0.transactions[1] = tx0.into();
1137        assert!(!e0.verify(&zero));
1138    }
1139
1140    #[test]
1141    fn test_transaction_signing() {
1142        let thread_pool = thread_pool_for_tests();
1143
1144        use solana_signature::Signature;
1145        let zero = Hash::default();
1146
1147        let keypair = Keypair::new();
1148        let tx0 = system_transaction::transfer(&keypair, &keypair.pubkey(), 0, zero);
1149        let tx1 = system_transaction::transfer(&keypair, &keypair.pubkey(), 1, zero);
1150
1151        // Verify entry with 2 transactions
1152        let mut e0 = [Entry::new(&zero, 0, vec![tx0, tx1])];
1153        assert!(e0.verify(&zero, &thread_pool));
1154
1155        // Clear signature of the first transaction, see that it does not verify
1156        let orig_sig = e0[0].transactions[0].signatures[0];
1157        e0[0].transactions[0].signatures[0] = Signature::default();
1158        assert!(!e0.verify(&zero, &thread_pool));
1159
1160        // restore original signature
1161        e0[0].transactions[0].signatures[0] = orig_sig;
1162        assert!(e0.verify(&zero, &thread_pool));
1163
1164        // Resize signatures and see verification fails.
1165        let len = e0[0].transactions[0].signatures.len();
1166        e0[0].transactions[0]
1167            .signatures
1168            .resize(len - 1, Signature::default());
1169        assert!(!e0.verify(&zero, &thread_pool));
1170
1171        // Pass an entry with no transactions
1172        let e0 = [Entry::new(&zero, 0, vec![])];
1173        assert!(e0.verify(&zero, &thread_pool));
1174    }
1175
1176    #[test]
1177    fn test_next_entry() {
1178        let zero = Hash::default();
1179        let tick = next_entry(&zero, 1, vec![]);
1180        assert_eq!(tick.num_hashes, 1);
1181        assert_ne!(tick.hash, zero);
1182
1183        let tick = next_entry(&zero, 0, vec![]);
1184        assert_eq!(tick.num_hashes, 0);
1185        assert_eq!(tick.hash, zero);
1186
1187        let keypair = Keypair::new();
1188        let tx0 = system_transaction::transfer(&keypair, &Pubkey::new_unique(), 42, zero);
1189        let entry0 = next_entry(&zero, 1, vec![tx0.clone()]);
1190        assert_eq!(entry0.num_hashes, 1);
1191        assert_eq!(entry0.hash, next_hash(&zero, 1, &[tx0.into()]));
1192    }
1193
1194    #[test]
1195    #[should_panic]
1196    fn test_next_entry_panic() {
1197        let zero = Hash::default();
1198        let keypair = Keypair::new();
1199        let tx = system_transaction::transfer(&keypair, &keypair.pubkey(), 0, zero);
1200        next_entry(&zero, 0, vec![tx]);
1201    }
1202
1203    #[test]
1204    fn test_verify_slice1() {
1205        agave_logger::setup();
1206        let thread_pool = thread_pool_for_tests();
1207
1208        let zero = Hash::default();
1209        let one = hash(zero.as_ref());
1210        // base case
1211        assert!(vec![][..].verify(&zero, &thread_pool));
1212        // singleton case 1
1213        assert!(vec![Entry::new_tick(0, &zero)][..].verify(&zero, &thread_pool));
1214        // singleton case 2, bad
1215        assert!(!vec![Entry::new_tick(0, &zero)][..].verify(&one, &thread_pool));
1216        // inductive step
1217        assert!(vec![next_entry(&zero, 0, vec![]); 2][..].verify(&zero, &thread_pool));
1218
1219        let mut bad_ticks = vec![next_entry(&zero, 0, vec![]); 2];
1220        bad_ticks[1].hash = one;
1221        // inductive step, bad
1222        assert!(!bad_ticks.verify(&zero, &thread_pool));
1223    }
1224
1225    #[test]
1226    fn test_verify_slice_with_hashes1() {
1227        agave_logger::setup();
1228        let thread_pool = thread_pool_for_tests();
1229
1230        let zero = Hash::default();
1231        let one = hash(zero.as_ref());
1232        let two = hash(one.as_ref());
1233        // base case
1234        assert!(vec![][..].verify(&one, &thread_pool));
1235        // singleton case 1
1236        assert!(vec![Entry::new_tick(1, &two)][..].verify(&one, &thread_pool));
1237        // singleton case 2, bad
1238        assert!(!vec![Entry::new_tick(1, &two)][..].verify(&two, &thread_pool));
1239
1240        let mut ticks = vec![next_entry(&one, 1, vec![])];
1241        ticks.push(next_entry(&ticks.last().unwrap().hash, 1, vec![]));
1242        // inductive step
1243        assert!(ticks.verify(&one, &thread_pool));
1244
1245        let mut bad_ticks = vec![next_entry(&one, 1, vec![])];
1246        bad_ticks.push(next_entry(&bad_ticks.last().unwrap().hash, 1, vec![]));
1247        bad_ticks[1].hash = one;
1248        // inductive step, bad
1249        assert!(!bad_ticks.verify(&one, &thread_pool));
1250    }
1251
1252    #[test]
1253    fn test_verify_slice_with_hashes_and_transactions() {
1254        agave_logger::setup();
1255        let thread_pool = thread_pool_for_tests();
1256
1257        let zero = Hash::default();
1258        let one = hash(zero.as_ref());
1259        let two = hash(one.as_ref());
1260        let alice_keypair = Keypair::new();
1261        let bob_keypair = Keypair::new();
1262        let tx0 = system_transaction::transfer(&alice_keypair, &bob_keypair.pubkey(), 1, one);
1263        let tx1 = system_transaction::transfer(&bob_keypair, &alice_keypair.pubkey(), 1, one);
1264        // base case
1265        assert!(vec![][..].verify(&one, &thread_pool));
1266        // singleton case 1
1267        assert!(vec![next_entry(&one, 1, vec![tx0.clone()])][..].verify(&one, &thread_pool));
1268        // singleton case 2, bad
1269        assert!(!vec![next_entry(&one, 1, vec![tx0.clone()])][..].verify(&two, &thread_pool));
1270
1271        let mut ticks = vec![next_entry(&one, 1, vec![tx0.clone()])];
1272        ticks.push(next_entry(
1273            &ticks.last().unwrap().hash,
1274            1,
1275            vec![tx1.clone()],
1276        ));
1277
1278        // inductive step
1279        assert!(ticks.verify(&one, &thread_pool));
1280
1281        let mut bad_ticks = vec![next_entry(&one, 1, vec![tx0])];
1282        bad_ticks.push(next_entry(&bad_ticks.last().unwrap().hash, 1, vec![tx1]));
1283        bad_ticks[1].hash = one;
1284        // inductive step, bad
1285        assert!(!bad_ticks.verify(&one, &thread_pool));
1286    }
1287
1288    #[test]
1289    fn test_verify_tick_hash_count() {
1290        let hashes_per_tick = 10;
1291        let tx = VersionedTransaction::default();
1292
1293        let no_hash_tx_entry = Entry {
1294            transactions: vec![tx.clone()],
1295            ..Entry::default()
1296        };
1297        let single_hash_tx_entry = Entry {
1298            transactions: vec![tx.clone()],
1299            num_hashes: 1,
1300            ..Entry::default()
1301        };
1302        let partial_tx_entry = Entry {
1303            num_hashes: hashes_per_tick - 1,
1304            transactions: vec![tx.clone()],
1305            ..Entry::default()
1306        };
1307        let full_tx_entry = Entry {
1308            num_hashes: hashes_per_tick,
1309            transactions: vec![tx.clone()],
1310            ..Entry::default()
1311        };
1312        let max_hash_tx_entry = Entry {
1313            transactions: vec![tx],
1314            num_hashes: u64::MAX,
1315            ..Entry::default()
1316        };
1317
1318        let no_hash_tick_entry = Entry::new_tick(0, &Hash::default());
1319        let single_hash_tick_entry = Entry::new_tick(1, &Hash::default());
1320        let partial_tick_entry = Entry::new_tick(hashes_per_tick - 1, &Hash::default());
1321        let full_tick_entry = Entry::new_tick(hashes_per_tick, &Hash::default());
1322        let max_hash_tick_entry = Entry::new_tick(u64::MAX, &Hash::default());
1323
1324        // empty batch should succeed if hashes_per_tick hasn't been reached
1325        let mut tick_hash_count = 0;
1326        let mut entries = vec![];
1327        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1328        assert_eq!(tick_hash_count, 0);
1329
1330        // empty batch should fail if hashes_per_tick has been reached
1331        tick_hash_count = hashes_per_tick;
1332        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1333        assert_eq!(tick_hash_count, hashes_per_tick);
1334        tick_hash_count = 0;
1335
1336        // validation is disabled when hashes_per_tick == 0
1337        entries = vec![max_hash_tx_entry.clone()];
1338        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, 0));
1339        assert_eq!(tick_hash_count, 0);
1340
1341        // partial tick should fail
1342        entries = vec![partial_tick_entry.clone()];
1343        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1344        assert_eq!(tick_hash_count, hashes_per_tick - 1);
1345        tick_hash_count = 0;
1346
1347        // full tick entry should succeed
1348        entries = vec![no_hash_tx_entry, full_tick_entry.clone()];
1349        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1350        assert_eq!(tick_hash_count, 0);
1351
1352        // oversized tick entry should fail
1353        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick - 1));
1354        assert_eq!(tick_hash_count, hashes_per_tick);
1355        tick_hash_count = 0;
1356
1357        // partial tx entry without tick entry should succeed
1358        entries = vec![partial_tx_entry];
1359        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1360        assert_eq!(tick_hash_count, hashes_per_tick - 1);
1361        tick_hash_count = 0;
1362
1363        // full tx entry with tick entry should succeed
1364        entries = vec![full_tx_entry.clone(), no_hash_tick_entry];
1365        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1366        assert_eq!(tick_hash_count, 0);
1367
1368        // full tx entry with oversized tick entry should fail
1369        entries = vec![full_tx_entry.clone(), single_hash_tick_entry.clone()];
1370        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1371        assert_eq!(tick_hash_count, hashes_per_tick + 1);
1372        tick_hash_count = 0;
1373
1374        // full tx entry without tick entry should fail
1375        entries = vec![full_tx_entry];
1376        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1377        assert_eq!(tick_hash_count, hashes_per_tick);
1378        tick_hash_count = 0;
1379
1380        // tx entry and a tick should succeed
1381        entries = vec![single_hash_tx_entry.clone(), partial_tick_entry];
1382        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1383        assert_eq!(tick_hash_count, 0);
1384
1385        // many tx entries and a tick should succeed
1386        let tx_entries: Vec<Entry> = (0..hashes_per_tick - 1)
1387            .map(|_| single_hash_tx_entry.clone())
1388            .collect();
1389        entries = [tx_entries, vec![single_hash_tick_entry]].concat();
1390        assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1391        assert_eq!(tick_hash_count, 0);
1392
1393        // check overflow saturation should fail
1394        entries = vec![full_tick_entry.clone(), max_hash_tick_entry];
1395        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1396        assert_eq!(tick_hash_count, u64::MAX);
1397        tick_hash_count = 0;
1398
1399        // check overflow saturation should fail
1400        entries = vec![max_hash_tx_entry, full_tick_entry];
1401        assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
1402        assert_eq!(tick_hash_count, u64::MAX);
1403    }
1404
1405    #[test]
1406    fn test_poh_verify_fuzz() {
1407        agave_logger::setup();
1408        for _ in 0..100 {
1409            let mut time = Measure::start("ticks");
1410            let num_ticks = thread_rng().gen_range(1..100);
1411            info!("create {num_ticks} ticks:");
1412            let mut entries = create_random_ticks(num_ticks, 100, Hash::default());
1413            time.stop();
1414
1415            let mut modified = false;
1416            if thread_rng().gen_ratio(1, 2) {
1417                modified = true;
1418                let modify_idx = thread_rng().gen_range(0..num_ticks) as usize;
1419                entries[modify_idx].hash = hash(&[1, 2, 3]);
1420            }
1421
1422            info!("done.. {time}");
1423            let mut time = Measure::start("poh");
1424            let res = entries.verify(&Hash::default(), &thread_pool_for_tests());
1425            assert_eq!(res, !modified);
1426            time.stop();
1427            info!("{time} {res}");
1428        }
1429    }
1430
1431    #[test]
1432    fn test_hash_transactions() {
1433        let mut transactions: Vec<_> = [test_tx(), test_tx(), test_tx()]
1434            .into_iter()
1435            .map(VersionedTransaction::from)
1436            .collect();
1437
1438        // Test different permutations of the transactions have different final hashes.
1439        // i.e. that **order** of transactions is included in the hash.
1440        let hash1 = hash_transactions(&transactions);
1441        transactions.swap(0, 1);
1442        let hash2 = hash_transactions(&transactions);
1443        assert_ne!(hash1, hash2);
1444    }
1445}