solana/entry/src/entry.rs

1367 lines
49 KiB
Rust

//! The `entry` module is a fundamental building block of Proof of History. It contains a
//! unique ID that is the hash of the Entry before it, plus the hash of the
//! transactions within it. Entries cannot be reordered, and its field `num_hashes`
//! represents an approximate amount of time since the last Entry was created.
use {
crate::poh::Poh,
crossbeam_channel::{Receiver, Sender},
dlopen::symbor::{Container, SymBorApi, Symbol},
dlopen_derive::SymBorApi,
lazy_static::lazy_static,
log::*,
rand::{thread_rng, Rng},
rayon::{prelude::*, ThreadPool},
serde::{Deserialize, Serialize},
solana_measure::measure::Measure,
solana_merkle_tree::MerkleTree,
solana_metrics::*,
solana_perf::{
cuda_runtime::PinnedVec,
packet::{Packet, PacketBatch, PacketBatchRecycler, PACKETS_PER_BATCH},
perf_libs,
recycler::Recycler,
sigverify,
},
solana_rayon_threadlimit::get_max_thread_count,
solana_sdk::{
hash::Hash,
packet::Meta,
timing,
transaction::{
Result, SanitizedTransaction, Transaction, TransactionError,
TransactionVerificationMode, VersionedTransaction,
},
},
std::{
cmp,
ffi::OsStr,
iter::repeat_with,
sync::{Arc, Mutex, Once},
thread::{self, JoinHandle},
time::Instant,
},
};
// get_max_thread_count to match number of threads in the old code.
// see: https://github.com/solana-labs/solana/pull/24853
lazy_static! {
static ref PAR_THREAD_POOL: ThreadPool = rayon::ThreadPoolBuilder::new()
.num_threads(get_max_thread_count())
.thread_name(|i| format!("solEntry{i:02}"))
.build()
.unwrap();
}
pub type EntrySender = Sender<Vec<Entry>>;
pub type EntryReceiver = Receiver<Vec<Entry>>;
static mut API: Option<Container<Api>> = None;
pub fn init_poh() {
init(OsStr::new("libpoh-simd.so"));
}
fn init(name: &OsStr) {
static INIT_HOOK: Once = Once::new();
info!("Loading {:?}", name);
unsafe {
INIT_HOOK.call_once(|| {
let path;
let lib_name = if let Some(perf_libs_path) = solana_perf::perf_libs::locate_perf_libs()
{
solana_perf::perf_libs::append_to_ld_library_path(
perf_libs_path.to_str().unwrap_or("").to_string(),
);
path = perf_libs_path.join(name);
path.as_os_str()
} else {
name
};
API = Container::load(lib_name).ok();
})
}
}
pub fn api() -> Option<&'static Container<Api<'static>>> {
{
static INIT_HOOK: Once = Once::new();
INIT_HOOK.call_once(|| {
if std::env::var("TEST_PERF_LIBS").is_ok() {
init_poh()
}
})
}
unsafe { API.as_ref() }
}
#[derive(SymBorApi)]
pub struct Api<'a> {
pub poh_verify_many_simd_avx512skx:
Symbol<'a, unsafe extern "C" fn(hashes: *mut u8, num_hashes: *const u64)>,
pub poh_verify_many_simd_avx2:
Symbol<'a, unsafe extern "C" fn(hashes: *mut u8, num_hashes: *const u64)>,
}
/// Each Entry contains three pieces of data. The `num_hashes` field is the number
/// of hashes performed since the previous entry. The `hash` field is the result
/// of hashing `hash` from the previous entry `num_hashes` times. The `transactions`
/// field points to Transactions that took place shortly before `hash` was generated.
///
/// If you multiply `num_hashes` by the amount of time it takes to generate a new hash, you
/// get a duration estimate since the last `Entry`. Since processing power increases
/// over time, one should expect the duration `num_hashes` represents to decrease proportionally.
/// An upper bound on Duration can be estimated by assuming each hash was generated by the
/// world's fastest processor at the time the entry was recorded. Or said another way, it
/// is physically not possible for a shorter duration to have occurred if one assumes the
/// hash was computed by the world's fastest processor at that time. The hash chain is both
/// a Verifiable Delay Function (VDF) and a Proof of Work (not to be confused with Proof of
/// Work consensus!)
///
/// The solana core protocol currently requires an `Entry` to contain `transactions` that are
/// executable in parallel. Implemented in:
///
/// * For TPU: `solana_core::banking_stage::BankingStage::process_and_record_transactions()`
/// * For TVU: `solana_core::replay_stage::ReplayStage::replay_blockstore_into_bank()`
///
/// All transactions in the `transactions` field have to follow the read/write locking restrictions
/// with regard to the accounts they reference. A single account can be either written by a single
/// transaction, or read by one or more transactions, but not both.
///
/// This enforcement is done via a call to `solana_runtime::accounts::Accounts::lock_accounts()`
/// with the `txs` argument holding all the `transactions` in the `Entry`.
#[derive(Serialize, Deserialize, Debug, Default, PartialEq, Eq, Clone)]
pub struct Entry {
/// The number of hashes since the previous Entry ID.
pub num_hashes: u64,
/// The SHA-256 hash `num_hashes` after the previous Entry ID.
pub hash: Hash,
/// An unordered list of transactions that were observed before the Entry ID was
/// generated. They may have been observed before a previous Entry ID but were
/// pushed back into this list to ensure deterministic interpretation of the ledger.
pub transactions: Vec<VersionedTransaction>,
}
pub struct EntrySummary {
pub num_hashes: u64,
pub hash: Hash,
pub num_transactions: u64,
}
impl From<&Entry> for EntrySummary {
fn from(entry: &Entry) -> Self {
Self {
num_hashes: entry.num_hashes,
hash: entry.hash,
num_transactions: entry.transactions.len() as u64,
}
}
}
/// Typed entry to distinguish between transaction and tick entries
pub enum EntryType {
Transactions(Vec<SanitizedTransaction>),
Tick(Hash),
}
impl Entry {
/// Creates the next Entry `num_hashes` after `start_hash`.
pub fn new(prev_hash: &Hash, mut num_hashes: u64, transactions: Vec<Transaction>) -> Self {
// If you passed in transactions, but passed in num_hashes == 0, then
// next_hash will generate the next hash and set num_hashes == 1
if num_hashes == 0 && !transactions.is_empty() {
num_hashes = 1;
}
let transactions = transactions.into_iter().map(Into::into).collect::<Vec<_>>();
let hash = next_hash(prev_hash, num_hashes, &transactions);
Entry {
num_hashes,
hash,
transactions,
}
}
pub fn new_mut(
start_hash: &mut Hash,
num_hashes: &mut u64,
transactions: Vec<Transaction>,
) -> Self {
let entry = Self::new(start_hash, *num_hashes, transactions);
*start_hash = entry.hash;
*num_hashes = 0;
entry
}
#[cfg(test)]
pub fn new_tick(num_hashes: u64, hash: &Hash) -> Self {
Entry {
num_hashes,
hash: *hash,
transactions: vec![],
}
}
/// Verifies self.hash is the result of hashing a `start_hash` `self.num_hashes` times.
/// If the transaction is not a Tick, then hash that as well.
pub fn verify(&self, start_hash: &Hash) -> bool {
let ref_hash = next_hash(start_hash, self.num_hashes, &self.transactions);
if self.hash != ref_hash {
warn!(
"next_hash is invalid expected: {:?} actual: {:?}",
self.hash, ref_hash
);
return false;
}
true
}
pub fn is_tick(&self) -> bool {
self.transactions.is_empty()
}
}
pub fn hash_transactions(transactions: &[VersionedTransaction]) -> Hash {
// a hash of a slice of transactions only needs to hash the signatures
let signatures: Vec<_> = transactions
.iter()
.flat_map(|tx| tx.signatures.iter())
.collect();
let merkle_tree = MerkleTree::new(&signatures);
if let Some(root_hash) = merkle_tree.get_root() {
*root_hash
} else {
Hash::default()
}
}
/// Creates the hash `num_hashes` after `start_hash`. If the transaction contains
/// a signature, the final hash will be a hash of both the previous ID and
/// the signature. If num_hashes is zero and there's no transaction data,
/// start_hash is returned.
pub fn next_hash(
start_hash: &Hash,
num_hashes: u64,
transactions: &[VersionedTransaction],
) -> Hash {
if num_hashes == 0 && transactions.is_empty() {
return *start_hash;
}
let mut poh = Poh::new(*start_hash, None);
poh.hash(num_hashes.saturating_sub(1));
if transactions.is_empty() {
poh.tick().unwrap().hash
} else {
poh.record(hash_transactions(transactions)).unwrap().hash
}
}
/// Last action required to verify an entry
enum VerifyAction {
/// Mixin a hash before computing the last hash for a transaction entry
Mixin(Hash),
/// Compute one last hash for a tick entry
Tick,
/// No action needed (tick entry with no hashes)
None,
}
pub struct GpuVerificationData {
thread_h: Option<JoinHandle<u64>>,
hashes: Option<Arc<Mutex<PinnedVec<Hash>>>>,
verifications: Option<Vec<(VerifyAction, Hash)>>,
}
pub enum DeviceVerificationData {
Cpu(),
Gpu(GpuVerificationData),
}
pub struct EntryVerificationState {
verification_status: EntryVerificationStatus,
poh_duration_us: u64,
device_verification_data: DeviceVerificationData,
}
pub struct GpuSigVerificationData {
thread_h: Option<JoinHandle<(bool, u64)>>,
}
pub enum DeviceSigVerificationData {
Cpu(),
Gpu(GpuSigVerificationData),
}
pub struct EntrySigVerificationState {
verification_status: EntryVerificationStatus,
entries: Option<Vec<EntryType>>,
device_verification_data: DeviceSigVerificationData,
gpu_verify_duration_us: u64,
}
impl EntrySigVerificationState {
pub fn entries(&mut self) -> Option<Vec<EntryType>> {
self.entries.take()
}
pub fn finish_verify(&mut self) -> bool {
match &mut self.device_verification_data {
DeviceSigVerificationData::Gpu(verification_state) => {
let (verified, gpu_time_us) =
verification_state.thread_h.take().unwrap().join().unwrap();
self.gpu_verify_duration_us = gpu_time_us;
self.verification_status = if verified {
EntryVerificationStatus::Success
} else {
EntryVerificationStatus::Failure
};
verified
}
DeviceSigVerificationData::Cpu() => {
self.verification_status == EntryVerificationStatus::Success
}
}
}
pub fn status(&self) -> EntryVerificationStatus {
self.verification_status
}
pub fn gpu_verify_duration(&self) -> u64 {
self.gpu_verify_duration_us
}
}
#[derive(Default, Clone)]
pub struct VerifyRecyclers {
hash_recycler: Recycler<PinnedVec<Hash>>,
tick_count_recycler: Recycler<PinnedVec<u64>>,
packet_recycler: PacketBatchRecycler,
out_recycler: Recycler<PinnedVec<u8>>,
tx_offset_recycler: Recycler<sigverify::TxOffset>,
}
#[derive(PartialEq, Eq, Clone, Copy, Debug)]
pub enum EntryVerificationStatus {
Failure,
Success,
Pending,
}
impl EntryVerificationState {
pub fn status(&self) -> EntryVerificationStatus {
self.verification_status
}
pub fn poh_duration_us(&self) -> u64 {
self.poh_duration_us
}
pub fn finish_verify(&mut self) -> bool {
match &mut self.device_verification_data {
DeviceVerificationData::Gpu(verification_state) => {
let gpu_time_us = verification_state.thread_h.take().unwrap().join().unwrap();
let mut verify_check_time = Measure::start("verify_check");
let hashes = verification_state.hashes.take().unwrap();
let hashes = Arc::try_unwrap(hashes)
.expect("unwrap Arc")
.into_inner()
.expect("into_inner");
let res = PAR_THREAD_POOL.install(|| {
hashes
.into_par_iter()
.cloned()
.zip(verification_state.verifications.take().unwrap())
.all(|(hash, (action, expected))| {
let actual = match action {
VerifyAction::Mixin(mixin) => {
Poh::new(hash, None).record(mixin).unwrap().hash
}
VerifyAction::Tick => Poh::new(hash, None).tick().unwrap().hash,
VerifyAction::None => hash,
};
actual == expected
})
});
verify_check_time.stop();
self.poh_duration_us += gpu_time_us + verify_check_time.as_us();
self.verification_status = if res {
EntryVerificationStatus::Success
} else {
EntryVerificationStatus::Failure
};
res
}
DeviceVerificationData::Cpu() => {
self.verification_status == EntryVerificationStatus::Success
}
}
}
}
pub fn verify_transactions(
entries: Vec<Entry>,
verify: Arc<dyn Fn(VersionedTransaction) -> Result<SanitizedTransaction> + Send + Sync>,
) -> Result<Vec<EntryType>> {
PAR_THREAD_POOL.install(|| {
entries
.into_par_iter()
.map(|entry| {
if entry.transactions.is_empty() {
Ok(EntryType::Tick(entry.hash))
} else {
Ok(EntryType::Transactions(
entry
.transactions
.into_par_iter()
.map(verify.as_ref())
.collect::<Result<Vec<_>>>()?,
))
}
})
.collect()
})
}
pub fn start_verify_transactions(
entries: Vec<Entry>,
skip_verification: bool,
verify_recyclers: VerifyRecyclers,
verify: Arc<
dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<SanitizedTransaction>
+ Send
+ Sync,
>,
) -> Result<EntrySigVerificationState> {
let api = perf_libs::api();
// Use the CPU if we have too few transactions for GPU signature verification to be worth it.
// We will also use the CPU if no acceleration API is used or if we're skipping
// the signature verification as we'd have nothing to do on the GPU in that case.
// TODO: make the CPU-to GPU crossover point dynamic, perhaps based on similar future
// heuristics to what might be used in sigverify::ed25519_verify when a dynamic crossover
// is introduced for that function (see TODO in sigverify::ed25519_verify)
let use_cpu = skip_verification
|| api.is_none()
|| entries
.iter()
.try_fold(0, |accum: usize, entry: &Entry| -> Option<usize> {
if accum.saturating_add(entry.transactions.len()) < 512 {
Some(accum.saturating_add(entry.transactions.len()))
} else {
None
}
})
.is_some();
if use_cpu {
start_verify_transactions_cpu(entries, skip_verification, verify)
} else {
start_verify_transactions_gpu(entries, verify_recyclers, verify)
}
}
fn start_verify_transactions_cpu(
entries: Vec<Entry>,
skip_verification: bool,
verify: Arc<
dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<SanitizedTransaction>
+ Send
+ Sync,
>,
) -> Result<EntrySigVerificationState> {
let verify_func = {
let mode = if skip_verification {
TransactionVerificationMode::HashOnly
} else {
TransactionVerificationMode::FullVerification
};
move |versioned_tx| verify(versioned_tx, mode)
};
let entries = verify_transactions(entries, Arc::new(verify_func))?;
Ok(EntrySigVerificationState {
verification_status: EntryVerificationStatus::Success,
entries: Some(entries),
device_verification_data: DeviceSigVerificationData::Cpu(),
gpu_verify_duration_us: 0,
})
}
fn start_verify_transactions_gpu(
entries: Vec<Entry>,
verify_recyclers: VerifyRecyclers,
verify: Arc<
dyn Fn(VersionedTransaction, TransactionVerificationMode) -> Result<SanitizedTransaction>
+ Send
+ Sync,
>,
) -> Result<EntrySigVerificationState> {
let verify_func = {
move |versioned_tx: VersionedTransaction| -> Result<SanitizedTransaction> {
verify(
versioned_tx,
TransactionVerificationMode::HashAndVerifyPrecompiles,
)
}
};
let entries = verify_transactions(entries, Arc::new(verify_func))?;
let entry_txs: Vec<&SanitizedTransaction> = entries
.iter()
.filter_map(|entry_type| match entry_type {
EntryType::Tick(_) => None,
EntryType::Transactions(transactions) => Some(transactions),
})
.flatten()
.collect::<Vec<_>>();
if entry_txs.is_empty() {
return Ok(EntrySigVerificationState {
verification_status: EntryVerificationStatus::Success,
entries: Some(entries),
device_verification_data: DeviceSigVerificationData::Cpu(),
gpu_verify_duration_us: 0,
});
}
let mut packet_batches = entry_txs
.par_iter()
.chunks(PACKETS_PER_BATCH)
.map(|slice| {
let vec_size = slice.len();
let mut packet_batch = PacketBatch::new_with_recycler(
verify_recyclers.packet_recycler.clone(),
vec_size,
"entry-sig-verify",
);
// We use set_len here instead of resize(vec_size, Packet::default()), to save
// memory bandwidth and avoid writing a large amount of data that will be overwritten
// soon afterwards. As well, Packet::default() actually leaves the packet data
// uninitialized, so the initialization would simply write junk into
// the vector anyway.
unsafe {
packet_batch.set_len(vec_size);
}
let entry_tx_iter = slice
.into_par_iter()
.map(|tx| tx.to_versioned_transaction());
let res = packet_batch
.par_iter_mut()
.zip(entry_tx_iter)
.all(|(packet, tx)| {
*packet.meta_mut() = Meta::default();
Packet::populate_packet(packet, None, &tx).is_ok()
});
if res {
Ok(packet_batch)
} else {
Err(TransactionError::SanitizeFailure)
}
})
.collect::<Result<Vec<_>>>()?;
let tx_offset_recycler = verify_recyclers.tx_offset_recycler;
let out_recycler = verify_recyclers.out_recycler;
let num_packets = entry_txs.len();
let gpu_verify_thread = thread::Builder::new()
.name("solGpuSigVerify".into())
.spawn(move || {
let mut verify_time = Measure::start("sigverify");
sigverify::ed25519_verify(
&mut packet_batches,
&tx_offset_recycler,
&out_recycler,
false,
num_packets,
);
let verified = packet_batches
.iter()
.all(|batch| batch.iter().all(|p| !p.meta().discard()));
verify_time.stop();
(verified, verify_time.as_us())
})
.unwrap();
Ok(EntrySigVerificationState {
verification_status: EntryVerificationStatus::Pending,
entries: Some(entries),
device_verification_data: DeviceSigVerificationData::Gpu(GpuSigVerificationData {
thread_h: Some(gpu_verify_thread),
}),
gpu_verify_duration_us: 0,
})
}
fn compare_hashes(computed_hash: Hash, ref_entry: &Entry) -> bool {
let actual = if !ref_entry.transactions.is_empty() {
let tx_hash = hash_transactions(&ref_entry.transactions);
let mut poh = Poh::new(computed_hash, None);
poh.record(tx_hash).unwrap().hash
} else if ref_entry.num_hashes > 0 {
let mut poh = Poh::new(computed_hash, None);
poh.tick().unwrap().hash
} else {
computed_hash
};
actual == ref_entry.hash
}
// an EntrySlice is a slice of Entries
pub trait EntrySlice {
/// Verifies the hashes and counts of a slice of transactions are all consistent.
fn verify_cpu(&self, start_hash: &Hash) -> EntryVerificationState;
fn verify_cpu_generic(&self, start_hash: &Hash) -> EntryVerificationState;
fn verify_cpu_x86_simd(&self, start_hash: &Hash, simd_len: usize) -> EntryVerificationState;
fn start_verify(&self, start_hash: &Hash, recyclers: VerifyRecyclers)
-> EntryVerificationState;
fn verify(&self, start_hash: &Hash) -> bool;
/// Checks that each entry tick has the correct number of hashes. Entry slices do not
/// necessarily end in a tick, so `tick_hash_count` is used to carry over the hash count
/// for the next entry slice.
fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool;
/// Counts tick entries
fn tick_count(&self) -> u64;
}
impl EntrySlice for [Entry] {
fn verify(&self, start_hash: &Hash) -> bool {
self.start_verify(start_hash, VerifyRecyclers::default())
.finish_verify()
}
fn verify_cpu_generic(&self, start_hash: &Hash) -> EntryVerificationState {
let now = Instant::now();
let genesis = [Entry {
num_hashes: 0,
hash: *start_hash,
transactions: vec![],
}];
let entry_pairs = genesis.par_iter().chain(self).zip(self);
let res = PAR_THREAD_POOL.install(|| {
entry_pairs.all(|(x0, x1)| {
let r = x1.verify(&x0.hash);
if !r {
warn!(
"entry invalid!: x0: {:?}, x1: {:?} num txs: {}",
x0.hash,
x1.hash,
x1.transactions.len()
);
}
r
})
});
let poh_duration_us = timing::duration_as_us(&now.elapsed());
EntryVerificationState {
verification_status: if res {
EntryVerificationStatus::Success
} else {
EntryVerificationStatus::Failure
},
poh_duration_us,
device_verification_data: DeviceVerificationData::Cpu(),
}
}
fn verify_cpu_x86_simd(&self, start_hash: &Hash, simd_len: usize) -> EntryVerificationState {
use solana_sdk::hash::HASH_BYTES;
let now = Instant::now();
let genesis = [Entry {
num_hashes: 0,
hash: *start_hash,
transactions: vec![],
}];
let aligned_len = ((self.len() + simd_len - 1) / simd_len) * simd_len;
let mut hashes_bytes = vec![0u8; HASH_BYTES * aligned_len];
genesis
.iter()
.chain(self)
.enumerate()
.for_each(|(i, entry)| {
if i < self.len() {
let start = i * HASH_BYTES;
let end = start + HASH_BYTES;
hashes_bytes[start..end].copy_from_slice(&entry.hash.to_bytes());
}
});
let mut hashes_chunked: Vec<_> = hashes_bytes.chunks_mut(simd_len * HASH_BYTES).collect();
let mut num_hashes: Vec<u64> = self
.iter()
.map(|entry| entry.num_hashes.saturating_sub(1))
.collect();
num_hashes.resize(aligned_len, 0);
let num_hashes: Vec<_> = num_hashes.chunks(simd_len).collect();
let res = PAR_THREAD_POOL.install(|| {
hashes_chunked
.par_iter_mut()
.zip(num_hashes)
.enumerate()
.all(|(i, (chunk, num_hashes))| {
match simd_len {
8 => unsafe {
(api().unwrap().poh_verify_many_simd_avx2)(
chunk.as_mut_ptr(),
num_hashes.as_ptr(),
);
},
16 => unsafe {
(api().unwrap().poh_verify_many_simd_avx512skx)(
chunk.as_mut_ptr(),
num_hashes.as_ptr(),
);
},
_ => {
panic!("unsupported simd len: {simd_len}");
}
}
let entry_start = i * simd_len;
// The last chunk may produce indexes larger than what we have in the reference entries
// because it is aligned to simd_len.
let entry_end = std::cmp::min(entry_start + simd_len, self.len());
self[entry_start..entry_end]
.iter()
.enumerate()
.all(|(j, ref_entry)| {
let start = j * HASH_BYTES;
let end = start + HASH_BYTES;
let hash = Hash::new(&chunk[start..end]);
compare_hashes(hash, ref_entry)
})
})
});
let poh_duration_us = timing::duration_as_us(&now.elapsed());
EntryVerificationState {
verification_status: if res {
EntryVerificationStatus::Success
} else {
EntryVerificationStatus::Failure
},
poh_duration_us,
device_verification_data: DeviceVerificationData::Cpu(),
}
}
fn verify_cpu(&self, start_hash: &Hash) -> EntryVerificationState {
#[cfg(any(target_arch = "x86", target_arch = "x86_64"))]
let (has_avx2, has_avx512) = (
is_x86_feature_detected!("avx2"),
is_x86_feature_detected!("avx512f"),
);
#[cfg(not(any(target_arch = "x86", target_arch = "x86_64")))]
let (has_avx2, has_avx512) = (false, false);
if api().is_some() {
if has_avx512 && self.len() >= 128 {
self.verify_cpu_x86_simd(start_hash, 16)
} else if has_avx2 && self.len() >= 48 {
self.verify_cpu_x86_simd(start_hash, 8)
} else {
self.verify_cpu_generic(start_hash)
}
} else {
self.verify_cpu_generic(start_hash)
}
}
fn start_verify(
&self,
start_hash: &Hash,
recyclers: VerifyRecyclers,
) -> EntryVerificationState {
let start = Instant::now();
let api = match perf_libs::api() {
None => return self.verify_cpu(start_hash),
Some(api) => api,
};
inc_new_counter_info!("entry_verify-num_entries", self.len());
let genesis = [Entry {
num_hashes: 0,
hash: *start_hash,
transactions: vec![],
}];
let hashes: Vec<Hash> = genesis
.iter()
.chain(self)
.map(|entry| entry.hash)
.take(self.len())
.collect();
let mut hashes_pinned = recyclers.hash_recycler.allocate("poh_verify_hash");
hashes_pinned.set_pinnable();
hashes_pinned.resize(hashes.len(), Hash::default());
hashes_pinned.copy_from_slice(&hashes);
let mut num_hashes_vec = recyclers
.tick_count_recycler
.allocate("poh_verify_num_hashes");
num_hashes_vec.reserve_and_pin(cmp::max(1, self.len()));
for entry in self {
num_hashes_vec.push(entry.num_hashes.saturating_sub(1));
}
let length = self.len();
let hashes = Arc::new(Mutex::new(hashes_pinned));
let hashes_clone = hashes.clone();
let gpu_verify_thread = thread::Builder::new()
.name("solGpuPohVerify".into())
.spawn(move || {
let mut hashes = hashes_clone.lock().unwrap();
let gpu_wait = Instant::now();
let res;
unsafe {
res = (api.poh_verify_many)(
hashes.as_mut_ptr() as *mut u8,
num_hashes_vec.as_ptr(),
length,
1,
);
}
assert!(res == 0, "GPU PoH verify many failed");
inc_new_counter_info!(
"entry_verify-gpu_thread",
timing::duration_as_us(&gpu_wait.elapsed()) as usize
);
timing::duration_as_us(&gpu_wait.elapsed())
})
.unwrap();
let verifications = PAR_THREAD_POOL.install(|| {
self.into_par_iter()
.map(|entry| {
let answer = entry.hash;
let action = if entry.transactions.is_empty() {
if entry.num_hashes == 0 {
VerifyAction::None
} else {
VerifyAction::Tick
}
} else {
VerifyAction::Mixin(hash_transactions(&entry.transactions))
};
(action, answer)
})
.collect()
});
let device_verification_data = DeviceVerificationData::Gpu(GpuVerificationData {
thread_h: Some(gpu_verify_thread),
verifications: Some(verifications),
hashes: Some(hashes),
});
EntryVerificationState {
verification_status: EntryVerificationStatus::Pending,
poh_duration_us: timing::duration_as_us(&start.elapsed()),
device_verification_data,
}
}
fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool {
// When hashes_per_tick is 0, hashing is disabled.
if hashes_per_tick == 0 {
return true;
}
for entry in self {
*tick_hash_count = tick_hash_count.saturating_add(entry.num_hashes);
if entry.is_tick() {
if *tick_hash_count != hashes_per_tick {
warn!(
"invalid tick hash count!: entry: {:#?}, tick_hash_count: {}, hashes_per_tick: {}",
entry,
tick_hash_count,
hashes_per_tick
);
return false;
}
*tick_hash_count = 0;
}
}
*tick_hash_count < hashes_per_tick
}
fn tick_count(&self) -> u64 {
self.iter().filter(|e| e.is_tick()).count() as u64
}
}
pub fn next_entry_mut(start: &mut Hash, num_hashes: u64, transactions: Vec<Transaction>) -> Entry {
let entry = Entry::new(start, num_hashes, transactions);
*start = entry.hash;
entry
}
pub fn create_ticks(num_ticks: u64, hashes_per_tick: u64, mut hash: Hash) -> Vec<Entry> {
repeat_with(|| next_entry_mut(&mut hash, hashes_per_tick, vec![]))
.take(num_ticks as usize)
.collect()
}
pub fn create_random_ticks(num_ticks: u64, max_hashes_per_tick: u64, mut hash: Hash) -> Vec<Entry> {
repeat_with(|| {
let hashes_per_tick = thread_rng().gen_range(1, max_hashes_per_tick);
next_entry_mut(&mut hash, hashes_per_tick, vec![])
})
.take(num_ticks as usize)
.collect()
}
/// Creates the next Tick or Transaction Entry `num_hashes` after `start_hash`.
pub fn next_entry(prev_hash: &Hash, num_hashes: u64, transactions: Vec<Transaction>) -> Entry {
let transactions = transactions.into_iter().map(Into::into).collect::<Vec<_>>();
next_versioned_entry(prev_hash, num_hashes, transactions)
}
/// Creates the next Tick or Transaction Entry `num_hashes` after `start_hash`.
pub fn next_versioned_entry(
prev_hash: &Hash,
num_hashes: u64,
transactions: Vec<VersionedTransaction>,
) -> Entry {
assert!(num_hashes > 0 || transactions.is_empty());
Entry {
num_hashes,
hash: next_hash(prev_hash, num_hashes, &transactions),
transactions,
}
}
#[cfg(test)]
mod tests {
use {
super::*,
solana_perf::test_tx::{test_invalid_tx, test_tx},
solana_sdk::{
hash::{hash, Hash},
pubkey::Pubkey,
signature::{Keypair, Signer},
system_transaction,
transaction::{
Result, SanitizedTransaction, SimpleAddressLoader, VersionedTransaction,
},
},
};
#[test]
fn test_entry_verify() {
let zero = Hash::default();
let one = hash(zero.as_ref());
assert!(Entry::new_tick(0, &zero).verify(&zero)); // base case, never used
assert!(!Entry::new_tick(0, &zero).verify(&one)); // base case, bad
assert!(next_entry(&zero, 1, vec![]).verify(&zero)); // inductive step
assert!(!next_entry(&zero, 1, vec![]).verify(&one)); // inductive step, bad
}
fn test_verify_transactions(
entries: Vec<Entry>,
skip_verification: bool,
verify_recyclers: VerifyRecyclers,
verify: Arc<
dyn Fn(
VersionedTransaction,
TransactionVerificationMode,
) -> Result<SanitizedTransaction>
+ Send
+ Sync,
>,
) -> bool {
let verify_func = {
let verify = verify.clone();
let verification_mode = if skip_verification {
TransactionVerificationMode::HashOnly
} else {
TransactionVerificationMode::FullVerification
};
move |versioned_tx: VersionedTransaction| -> Result<SanitizedTransaction> {
verify(versioned_tx, verification_mode)
}
};
let cpu_verify_result = verify_transactions(entries.clone(), Arc::new(verify_func));
let mut gpu_verify_result: EntrySigVerificationState = {
let verify_result =
start_verify_transactions(entries, skip_verification, verify_recyclers, verify);
match verify_result {
Ok(res) => res,
_ => EntrySigVerificationState {
verification_status: EntryVerificationStatus::Failure,
entries: None,
device_verification_data: DeviceSigVerificationData::Cpu(),
gpu_verify_duration_us: 0,
},
}
};
match cpu_verify_result {
Ok(_) => {
assert!(gpu_verify_result.verification_status != EntryVerificationStatus::Failure);
assert!(gpu_verify_result.finish_verify());
true
}
_ => {
assert!(
gpu_verify_result.verification_status == EntryVerificationStatus::Failure
|| !gpu_verify_result.finish_verify()
);
false
}
}
}
#[test]
fn test_entry_gpu_verify() {
let verify_transaction = {
move |versioned_tx: VersionedTransaction,
verification_mode: TransactionVerificationMode|
-> Result<SanitizedTransaction> {
let sanitized_tx = {
let message_hash =
if verification_mode == TransactionVerificationMode::FullVerification {
versioned_tx.verify_and_hash_message()?
} else {
versioned_tx.message.hash()
};
SanitizedTransaction::try_create(
versioned_tx,
message_hash,
None,
SimpleAddressLoader::Disabled,
true, // require_static_program_ids
)
}?;
Ok(sanitized_tx)
}
};
let recycler = VerifyRecyclers::default();
// Make sure we test with a number of transactions that's not a multiple of PACKETS_PER_BATCH
let entries_invalid = (0..1025)
.map(|_| {
let transaction = test_invalid_tx();
next_entry_mut(&mut Hash::default(), 0, vec![transaction])
})
.collect::<Vec<_>>();
let entries_valid = (0..1025)
.map(|_| {
let transaction = test_tx();
next_entry_mut(&mut Hash::default(), 0, vec![transaction])
})
.collect::<Vec<_>>();
assert!(!test_verify_transactions(
entries_invalid,
false,
recycler.clone(),
Arc::new(verify_transaction)
));
assert!(test_verify_transactions(
entries_valid,
false,
recycler,
Arc::new(verify_transaction)
));
}
#[test]
fn test_transaction_reorder_attack() {
let zero = Hash::default();
// First, verify entries
let keypair = Keypair::new();
let tx0 = system_transaction::transfer(&keypair, &keypair.pubkey(), 0, zero);
let tx1 = system_transaction::transfer(&keypair, &keypair.pubkey(), 1, zero);
let mut e0 = Entry::new(&zero, 0, vec![tx0.clone(), tx1.clone()]);
assert!(e0.verify(&zero));
// Next, swap two transactions and ensure verification fails.
e0.transactions[0] = tx1.into(); // <-- attack
e0.transactions[1] = tx0.into();
assert!(!e0.verify(&zero));
}
#[test]
fn test_transaction_signing() {
use solana_sdk::signature::Signature;
let zero = Hash::default();
let keypair = Keypair::new();
let tx0 = system_transaction::transfer(&keypair, &keypair.pubkey(), 0, zero);
let tx1 = system_transaction::transfer(&keypair, &keypair.pubkey(), 1, zero);
// Verify entry with 2 transactions
let mut e0 = vec![Entry::new(&zero, 0, vec![tx0, tx1])];
assert!(e0.verify(&zero));
// Clear signature of the first transaction, see that it does not verify
let orig_sig = e0[0].transactions[0].signatures[0];
e0[0].transactions[0].signatures[0] = Signature::default();
assert!(!e0.verify(&zero));
// restore original signature
e0[0].transactions[0].signatures[0] = orig_sig;
assert!(e0.verify(&zero));
// Resize signatures and see verification fails.
let len = e0[0].transactions[0].signatures.len();
e0[0].transactions[0]
.signatures
.resize(len - 1, Signature::default());
assert!(!e0.verify(&zero));
// Pass an entry with no transactions
let e0 = vec![Entry::new(&zero, 0, vec![])];
assert!(e0.verify(&zero));
}
#[test]
fn test_next_entry() {
let zero = Hash::default();
let tick = next_entry(&zero, 1, vec![]);
assert_eq!(tick.num_hashes, 1);
assert_ne!(tick.hash, zero);
let tick = next_entry(&zero, 0, vec![]);
assert_eq!(tick.num_hashes, 0);
assert_eq!(tick.hash, zero);
let keypair = Keypair::new();
let tx0 = system_transaction::transfer(&keypair, &Pubkey::new_unique(), 42, zero);
let entry0 = next_entry(&zero, 1, vec![tx0.clone()]);
assert_eq!(entry0.num_hashes, 1);
assert_eq!(entry0.hash, next_hash(&zero, 1, &[tx0.into()]));
}
#[test]
#[should_panic]
fn test_next_entry_panic() {
let zero = Hash::default();
let keypair = Keypair::new();
let tx = system_transaction::transfer(&keypair, &keypair.pubkey(), 0, zero);
next_entry(&zero, 0, vec![tx]);
}
#[test]
fn test_verify_slice1() {
solana_logger::setup();
let zero = Hash::default();
let one = hash(zero.as_ref());
assert!(vec![][..].verify(&zero)); // base case
assert!(vec![Entry::new_tick(0, &zero)][..].verify(&zero)); // singleton case 1
assert!(!vec![Entry::new_tick(0, &zero)][..].verify(&one)); // singleton case 2, bad
assert!(vec![next_entry(&zero, 0, vec![]); 2][..].verify(&zero)); // inductive step
let mut bad_ticks = vec![next_entry(&zero, 0, vec![]); 2];
bad_ticks[1].hash = one;
assert!(!bad_ticks.verify(&zero)); // inductive step, bad
}
#[test]
fn test_verify_slice_with_hashes1() {
solana_logger::setup();
let zero = Hash::default();
let one = hash(zero.as_ref());
let two = hash(one.as_ref());
assert!(vec![][..].verify(&one)); // base case
assert!(vec![Entry::new_tick(1, &two)][..].verify(&one)); // singleton case 1
assert!(!vec![Entry::new_tick(1, &two)][..].verify(&two)); // singleton case 2, bad
let mut ticks = vec![next_entry(&one, 1, vec![])];
ticks.push(next_entry(&ticks.last().unwrap().hash, 1, vec![]));
assert!(ticks.verify(&one)); // inductive step
let mut bad_ticks = vec![next_entry(&one, 1, vec![])];
bad_ticks.push(next_entry(&bad_ticks.last().unwrap().hash, 1, vec![]));
bad_ticks[1].hash = one;
assert!(!bad_ticks.verify(&one)); // inductive step, bad
}
#[test]
fn test_verify_slice_with_hashes_and_transactions() {
solana_logger::setup();
let zero = Hash::default();
let one = hash(zero.as_ref());
let two = hash(one.as_ref());
let alice_keypair = Keypair::new();
let bob_keypair = Keypair::new();
let tx0 = system_transaction::transfer(&alice_keypair, &bob_keypair.pubkey(), 1, one);
let tx1 = system_transaction::transfer(&bob_keypair, &alice_keypair.pubkey(), 1, one);
assert!(vec![][..].verify(&one)); // base case
assert!(vec![next_entry(&one, 1, vec![tx0.clone()])][..].verify(&one)); // singleton case 1
assert!(!vec![next_entry(&one, 1, vec![tx0.clone()])][..].verify(&two)); // singleton case 2, bad
let mut ticks = vec![next_entry(&one, 1, vec![tx0.clone()])];
ticks.push(next_entry(
&ticks.last().unwrap().hash,
1,
vec![tx1.clone()],
));
assert!(ticks.verify(&one)); // inductive step
let mut bad_ticks = vec![next_entry(&one, 1, vec![tx0])];
bad_ticks.push(next_entry(&bad_ticks.last().unwrap().hash, 1, vec![tx1]));
bad_ticks[1].hash = one;
assert!(!bad_ticks.verify(&one)); // inductive step, bad
}
#[test]
fn test_verify_tick_hash_count() {
let hashes_per_tick = 10;
let tx = VersionedTransaction::default();
let no_hash_tx_entry = Entry {
transactions: vec![tx.clone()],
..Entry::default()
};
let single_hash_tx_entry = Entry {
transactions: vec![tx.clone()],
num_hashes: 1,
..Entry::default()
};
let partial_tx_entry = Entry {
num_hashes: hashes_per_tick - 1,
transactions: vec![tx.clone()],
..Entry::default()
};
let full_tx_entry = Entry {
num_hashes: hashes_per_tick,
transactions: vec![tx.clone()],
..Entry::default()
};
let max_hash_tx_entry = Entry {
transactions: vec![tx],
num_hashes: u64::MAX,
..Entry::default()
};
let no_hash_tick_entry = Entry::new_tick(0, &Hash::default());
let single_hash_tick_entry = Entry::new_tick(1, &Hash::default());
let partial_tick_entry = Entry::new_tick(hashes_per_tick - 1, &Hash::default());
let full_tick_entry = Entry::new_tick(hashes_per_tick, &Hash::default());
let max_hash_tick_entry = Entry::new_tick(u64::MAX, &Hash::default());
// empty batch should succeed if hashes_per_tick hasn't been reached
let mut tick_hash_count = 0;
let mut entries = vec![];
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, 0);
// empty batch should fail if hashes_per_tick has been reached
tick_hash_count = hashes_per_tick;
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, hashes_per_tick);
tick_hash_count = 0;
// validation is disabled when hashes_per_tick == 0
entries = vec![max_hash_tx_entry.clone()];
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, 0));
assert_eq!(tick_hash_count, 0);
// partial tick should fail
entries = vec![partial_tick_entry.clone()];
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, hashes_per_tick - 1);
tick_hash_count = 0;
// full tick entry should succeed
entries = vec![no_hash_tx_entry, full_tick_entry.clone()];
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, 0);
// oversized tick entry should fail
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick - 1));
assert_eq!(tick_hash_count, hashes_per_tick);
tick_hash_count = 0;
// partial tx entry without tick entry should succeed
entries = vec![partial_tx_entry];
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, hashes_per_tick - 1);
tick_hash_count = 0;
// full tx entry with tick entry should succeed
entries = vec![full_tx_entry.clone(), no_hash_tick_entry];
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, 0);
// full tx entry with oversized tick entry should fail
entries = vec![full_tx_entry.clone(), single_hash_tick_entry.clone()];
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, hashes_per_tick + 1);
tick_hash_count = 0;
// full tx entry without tick entry should fail
entries = vec![full_tx_entry];
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, hashes_per_tick);
tick_hash_count = 0;
// tx entry and a tick should succeed
entries = vec![single_hash_tx_entry.clone(), partial_tick_entry];
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, 0);
// many tx entries and a tick should succeed
let tx_entries: Vec<Entry> = (0..hashes_per_tick - 1)
.map(|_| single_hash_tx_entry.clone())
.collect();
entries = [tx_entries, vec![single_hash_tick_entry]].concat();
assert!(entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, 0);
// check overflow saturation should fail
entries = vec![full_tick_entry.clone(), max_hash_tick_entry];
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, u64::MAX);
tick_hash_count = 0;
// check overflow saturation should fail
entries = vec![max_hash_tx_entry, full_tick_entry];
assert!(!entries.verify_tick_hash_count(&mut tick_hash_count, hashes_per_tick));
assert_eq!(tick_hash_count, u64::MAX);
}
#[test]
fn test_poh_verify_fuzz() {
solana_logger::setup();
for _ in 0..100 {
let mut time = Measure::start("ticks");
let num_ticks = thread_rng().gen_range(1, 100);
info!("create {} ticks:", num_ticks);
let mut entries = create_random_ticks(num_ticks, 100, Hash::default());
time.stop();
let mut modified = false;
if thread_rng().gen_ratio(1, 2) {
modified = true;
let modify_idx = thread_rng().gen_range(0, num_ticks) as usize;
entries[modify_idx].hash = hash(&[1, 2, 3]);
}
info!("done.. {}", time);
let mut time = Measure::start("poh");
let res = entries.verify(&Hash::default());
assert_eq!(res, !modified);
time.stop();
info!("{} {}", time, res);
}
}
}