Cache Sapling and Orchard bundle validation

This adds two new CuckooCaches in validation, each caching whether all
of a transaction bundle's proofs and signatures were valid.

Bundles which match the validation cache never have proofs or signatures
added to the batch validators. For blocks where all transactions have
been previously observed in the mempool, the final validation of the
batches should be a no-op.

Part of zcash/zcash#6049.
This commit is contained in:
Jack Grigg 2022-07-13 04:13:05 +00:00
parent 882935e35e
commit 087c85ec03
15 changed files with 375 additions and 36 deletions

View File

@ -47,16 +47,19 @@ endif
# TODO: Figure out how to avoid an explicit file list.
CXXBRIDGE_RS = \
rust/src/blake2b.rs \
rust/src/bundlecache.rs \
rust/src/equihash.rs \
rust/src/orchard_bundle.rs \
rust/src/sapling.rs
CXXBRIDGE_H = \
rust/gen/include/rust/blake2b.h \
rust/gen/include/rust/bundlecache.h \
rust/gen/include/rust/equihash.h \
rust/gen/include/rust/orchard_bundle.h \
rust/gen/include/rust/sapling.h
CXXBRIDGE_CPP = \
rust/gen/src/blake2b.cpp \
rust/gen/src/bundlecache.cpp \
rust/gen/src/equihash.cpp \
rust/gen/src/orchard_bundle.cpp \
rust/gen/src/sapling.cpp
@ -174,6 +177,7 @@ LIBZCASH_H = \
zcash/address/sprout.hpp \
zcash/address/unified.h \
zcash/address/zip32.h \
zcash/cache.h \
zcash/History.hpp \
zcash/JoinSplit.hpp \
zcash/Note.hpp \
@ -613,6 +617,7 @@ libzcash_a_SOURCES = \
zcash/address/sprout.cpp \
zcash/address/unified.cpp \
zcash/address/zip32.cpp \
zcash/cache.cpp \
zcash/History.cpp \
zcash/JoinSplit.cpp \
zcash/Note.cpp \

View File

@ -11,6 +11,8 @@
#include <sodium.h>
#include <tracing.h>
#include <rust/bundlecache.h>
#include <boost/filesystem.hpp>
const std::function<std::string(const char*)> G_TRANSLATION_FUN = nullptr;
@ -66,7 +68,8 @@ public:
int main(int argc, char **argv) {
assert(sodium_init() != -1);
ECC_Start();
InitSignatureCache();
InitSignatureCache(DEFAULT_MAX_SIG_CACHE_SIZE * ((size_t) 1 << 20));
bundlecache::init(DEFAULT_MAX_SIG_CACHE_SIZE * ((size_t) 1 << 20));
// Log all errors to a common test file.
fs::path tmpPath = fs::temp_directory_path();

View File

@ -1325,7 +1325,7 @@ TEST(ChecktransactionTests, HeartwoodEnforcesSaplingRulesOnShieldedCoinbase) {
// Coinbase transaction should pass contextual checks.
EXPECT_TRUE(ContextualCheckTransaction(tx, state, chainparams, 10, 57));
std::optional<rust::Box<sapling::BatchValidator>> saplingAuth = sapling::init_batch_validator();
std::optional<rust::Box<sapling::BatchValidator>> saplingAuth = sapling::init_batch_validator(false);
auto orchardAuth = orchard::AuthValidator::Disabled();
auto heartwoodBranchId = NetworkUpgradeInfo[Consensus::UPGRADE_HEARTWOOD].nBranchId;

View File

@ -66,6 +66,7 @@
#include "zmq/zmqnotificationinterface.h"
#endif
#include <rust/bundlecache.h>
#include <rust/init.h>
#include <rust/metrics.h>
@ -465,7 +466,7 @@ std::string HelpMessage(HelpMessageMode mode)
{
strUsage += HelpMessageOpt("-limitfreerelay=<n>", strprintf("Continuously rate-limit free transactions to <n>*1000 bytes per minute (default: %u)", DEFAULT_LIMITFREERELAY));
strUsage += HelpMessageOpt("-relaypriority", strprintf("Require high priority for relaying free or low-fee transactions (default: %u)", DEFAULT_RELAYPRIORITY));
strUsage += HelpMessageOpt("-maxsigcachesize=<n>", strprintf("Limit size of signature cache to <n> MiB (default: %u)", DEFAULT_MAX_SIG_CACHE_SIZE));
strUsage += HelpMessageOpt("-maxsigcachesize=<n>", strprintf("Limit total size of signature and bundle caches to <n> MiB (default: %u)", DEFAULT_MAX_SIG_CACHE_SIZE));
strUsage += HelpMessageOpt("-maxtipage=<n>", strprintf("Maximum tip age in seconds to consider node in initial block download (default: %u)", DEFAULT_MAX_TIP_AGE));
}
strUsage += HelpMessageOpt("-minrelaytxfee=<amt>", strprintf(_("Fees (in %s/kB) smaller than this are considered zero fee for relaying, mining and transaction creation (default: %s)"),
@ -1410,7 +1411,18 @@ bool AppInit2(boost::thread_group& threadGroup, CScheduler& scheduler)
LogPrintf("Using at most %i connections (%i file descriptors available)\n", nMaxConnections, nFD);
std::ostringstream strErrors;
InitSignatureCache();
// Initialize the validity caches. We currently have three:
// - Transparent signature validity.
// - Sapling bundle validity.
// - Orchard bundle validity.
// Assign half of the cap to transparent signatures, and split the rest
// between Sapling and Orchard bundles.
size_t nMaxCacheSize = GetArg("-maxsigcachesize", DEFAULT_MAX_SIG_CACHE_SIZE) * ((size_t) 1 << 20);
if (nMaxCacheSize <= 0) {
return InitError(strprintf(_("-maxsigcachesize must be at least 1")));
}
InitSignatureCache(nMaxCacheSize / 2);
bundlecache::init(nMaxCacheSize / 4);
LogPrintf("Using %u threads for script verification\n", nScriptCheckThreads);
if (nScriptCheckThreads) {

View File

@ -1994,11 +1994,11 @@ bool AcceptToMemoryPool(
// This will be a single-transaction batch, which will be more efficient
// than unbatched if the transaction contains at least one Sapling Spend
// or at least two Sapling Outputs.
std::optional<rust::Box<sapling::BatchValidator>> saplingAuth = sapling::init_batch_validator();
std::optional<rust::Box<sapling::BatchValidator>> saplingAuth = sapling::init_batch_validator(true);
// This will be a single-transaction batch, which is still more efficient as every
// Orchard bundle contains at least two signatures.
std::optional<orchard::AuthValidator> orchardAuth = orchard::AuthValidator::Batch();
std::optional<orchard::AuthValidator> orchardAuth = orchard::AuthValidator::Batch(true);
// Check shielded input signatures.
if (!ContextualCheckShieldedInputs(
@ -3094,14 +3094,17 @@ bool ConnectBlock(const CBlock& block, CValidationState& state, CBlockIndex* pin
fExpensiveChecks = false;
}
// Don't cache results if we're actually connecting blocks (still consult the cache, though).
bool fCacheResults = fJustCheck;
// proof verification is expensive, disable if possible
auto verifier = fExpensiveChecks ? ProofVerifier::Strict() : ProofVerifier::Disabled();
// Disable Sapling and Orchard batch validation if possible.
std::optional<rust::Box<sapling::BatchValidator>> saplingAuth = fExpensiveChecks ?
std::optional(sapling::init_batch_validator()) : std::nullopt;
std::optional(sapling::init_batch_validator(fCacheResults)) : std::nullopt;
std::optional<orchard::AuthValidator> orchardAuth = fExpensiveChecks ?
orchard::AuthValidator::Batch() : orchard::AuthValidator::Disabled();
orchard::AuthValidator::Batch(fCacheResults) : orchard::AuthValidator::Disabled();
// If in initial block download, and this block is an ancestor of a checkpoint,
// and -ibdskiptxverification is set, disable all transaction checks.
@ -3329,7 +3332,6 @@ bool ConnectBlock(const CBlock& block, CValidationState& state, CBlockIndex* pin
nFees += view.GetValueIn(tx)-tx.GetValueOut();
std::vector<CScriptCheck> vChecks;
bool fCacheResults = fJustCheck; /* Don't cache results if we're actually connecting blocks (still consult the cache, though) */
if (!ContextualCheckInputs(tx, state, view, fExpensiveChecks, flags, fCacheResults, txdata.back(), chainparams.GetConsensus(), consensusBranchId, nScriptCheckThreads ? &vChecks : NULL))
return error("ConnectBlock(): CheckInputs on %s failed with %s",
tx.GetHash().ToString(), FormatStateMessage(state));

View File

@ -80,7 +80,7 @@ bool orchard_bundle_anchor(
///
/// Please free this with `orchard_batch_validation_free` when you are done with
/// it.
OrchardBatchValidatorPtr* orchard_batch_validation_init();
OrchardBatchValidatorPtr* orchard_batch_validation_init(bool cache_store);
/// Frees a batch validator returned from `orchard_batch_validation_init`.
void orchard_batch_validation_free(OrchardBatchValidatorPtr* batch);
@ -153,9 +153,9 @@ public:
/// Creates a validation context that batch-validates Orchard proofs and
/// signatures.
static AuthValidator Batch() {
static AuthValidator Batch(bool cacheResult) {
auto batch = AuthValidator();
batch.inner.reset(orchard_batch_validation_init());
batch.inner.reset(orchard_batch_validation_init(cacheResult));
return batch;
}

158
src/rust/src/bundlecache.rs Normal file
View File

@ -0,0 +1,158 @@
use std::{
convert::TryInto,
sync::{Once, RwLock, RwLockReadGuard, RwLockWriteGuard},
};
use rand_core::{OsRng, RngCore};
#[cxx::bridge]
mod ffi {
#[namespace = "libzcash"]
unsafe extern "C++" {
include!("zcash/cache.h");
type BundleValidityCache;
fn NewBundleValidityCache(bytes: usize) -> UniquePtr<BundleValidityCache>;
fn insert(self: Pin<&mut BundleValidityCache>, entry: [u8; 32]);
fn contains(&self, entry: &[u8; 32], erase: bool) -> bool;
}
#[namespace = "bundlecache"]
extern "Rust" {
fn init(cache_bytes: usize);
}
}
pub(crate) struct CacheEntry([u8; 32]);
pub(crate) enum CacheEntries {
Storing(Vec<CacheEntry>),
NotStoring,
}
impl CacheEntries {
pub(crate) fn new(cache_store: bool) -> Self {
if cache_store {
CacheEntries::Storing(vec![])
} else {
CacheEntries::NotStoring
}
}
}
pub(crate) struct BundleValidityCache {
hasher: blake2b_simd::State,
cache: cxx::UniquePtr<ffi::BundleValidityCache>,
}
impl BundleValidityCache {
fn new(personalization: &[u8; 16], cache_bytes: usize) -> Self {
// Use BLAKE2b to produce entries from bundles. It has a block size of 128 bytes,
// into which we put:
// - 32 byte nonce
// - 64 bytes of bundle commitments
// - 32 byte sighash
let mut hasher = blake2b_simd::Params::new()
.hash_length(32)
.personal(personalization)
.to_state();
// Pre-load the hasher with a per-instance nonce. This ensures that cache entries
// are deterministic but also unique per node.
let mut nonce = [0; 32];
OsRng.fill_bytes(&mut nonce);
hasher.update(&nonce);
Self {
hasher,
cache: ffi::NewBundleValidityCache(cache_bytes),
}
}
pub(crate) fn compute_entry(
&self,
bundle_commitment: &[u8; 32],
bundle_authorizing_commitment: &[u8; 32],
sighash: &[u8; 32],
) -> CacheEntry {
self.hasher
.clone()
.update(bundle_commitment)
.update(bundle_authorizing_commitment)
.update(sighash)
.finalize()
.as_bytes()
.try_into()
.map(CacheEntry)
.unwrap()
}
pub(crate) fn insert(&mut self, queued_entries: CacheEntries) {
if let CacheEntries::Storing(cache_entries) = queued_entries {
for cache_entry in cache_entries {
self.cache.pin_mut().insert(cache_entry.0);
}
}
}
pub(crate) fn contains(&self, entry: CacheEntry, queued_entries: &mut CacheEntries) -> bool {
if self
.cache
.contains(&entry.0, matches!(queued_entries, CacheEntries::NotStoring))
{
true
} else {
if let CacheEntries::Storing(cache_entries) = queued_entries {
cache_entries.push(entry);
}
false
}
}
}
static BUNDLE_CACHES_LOADED: Once = Once::new();
static mut SAPLING_BUNDLE_VALIDITY_CACHE: Option<RwLock<BundleValidityCache>> = None;
static mut ORCHARD_BUNDLE_VALIDITY_CACHE: Option<RwLock<BundleValidityCache>> = None;
fn init(cache_bytes: usize) {
BUNDLE_CACHES_LOADED.call_once(|| unsafe {
SAPLING_BUNDLE_VALIDITY_CACHE = Some(RwLock::new(BundleValidityCache::new(
b"SaplingVeriCache",
cache_bytes,
)));
ORCHARD_BUNDLE_VALIDITY_CACHE = Some(RwLock::new(BundleValidityCache::new(
b"OrchardVeriCache",
cache_bytes,
)));
});
}
pub(crate) fn sapling_bundle_validity_cache() -> RwLockReadGuard<'static, BundleValidityCache> {
unsafe { SAPLING_BUNDLE_VALIDITY_CACHE.as_ref() }
.expect("bundlecache::init() should have been called")
.read()
.unwrap()
}
pub(crate) fn sapling_bundle_validity_cache_mut() -> RwLockWriteGuard<'static, BundleValidityCache>
{
unsafe { SAPLING_BUNDLE_VALIDITY_CACHE.as_mut() }
.expect("bundlecache::init() should have been called")
.write()
.unwrap()
}
pub(crate) fn orchard_bundle_validity_cache() -> RwLockReadGuard<'static, BundleValidityCache> {
unsafe { ORCHARD_BUNDLE_VALIDITY_CACHE.as_ref() }
.expect("bundlecache::init() should have been called")
.read()
.unwrap()
}
pub(crate) fn orchard_bundle_validity_cache_mut() -> RwLockWriteGuard<'static, BundleValidityCache>
{
unsafe { ORCHARD_BUNDLE_VALIDITY_CACHE.as_mut() }
.expect("bundlecache::init() should have been called")
.write()
.unwrap()
}

View File

@ -1,19 +1,19 @@
use std::{mem, ptr};
use std::{convert::TryInto, mem, ptr};
use libc::size_t;
use memuse::DynamicUsage;
use orchard::{
bundle::{Authorized, BatchValidator},
keys::OutgoingViewingKey,
note_encryption::OrchardDomain,
Bundle,
bundle::Authorized, keys::OutgoingViewingKey, note_encryption::OrchardDomain, Bundle,
};
use rand_core::OsRng;
use tracing::{debug, error};
use zcash_note_encryption::try_output_recovery_with_ovk;
use zcash_primitives::transaction::components::{orchard as orchard_serialization, Amount};
use crate::streams_ffi::{CppStreamReader, CppStreamWriter, ReadCb, StreamObj, WriteCb};
use crate::{
bundlecache::{orchard_bundle_validity_cache, orchard_bundle_validity_cache_mut, CacheEntries},
streams_ffi::{CppStreamReader, CppStreamWriter, ReadCb, StreamObj, WriteCb},
};
#[no_mangle]
pub extern "C" fn orchard_bundle_clone(
@ -145,12 +145,20 @@ pub extern "C" fn orchard_bundle_anchor(
}
}
pub struct BatchValidator {
validator: orchard::bundle::BatchValidator,
queued_entries: CacheEntries,
}
/// Creates an Orchard bundle batch validation context.
///
/// Please free this when you're done.
#[no_mangle]
pub extern "C" fn orchard_batch_validation_init() -> *mut BatchValidator {
let ctx = Box::new(BatchValidator::new());
pub extern "C" fn orchard_batch_validation_init(cache_store: bool) -> *mut BatchValidator {
let ctx = Box::new(BatchValidator {
validator: orchard::bundle::BatchValidator::new(),
queued_entries: CacheEntries::new(cache_store),
});
Box::into_raw(ctx)
}
@ -175,7 +183,31 @@ pub extern "C" fn orchard_batch_add_bundle(
let sighash = unsafe { sighash.as_ref() };
match (batch, bundle, sighash) {
(Some(batch), Some(bundle), Some(sighash)) => batch.add_bundle(bundle, *sighash),
(Some(batch), Some(bundle), Some(sighash)) => {
let cache = orchard_bundle_validity_cache();
// Compute the cache entry for this bundle.
let cache_entry = {
let bundle_commitment = bundle.commitment();
let bundle_authorizing_commitment = bundle.authorizing_commitment();
cache.compute_entry(
bundle_commitment.0.as_bytes().try_into().unwrap(),
bundle_authorizing_commitment
.0
.as_bytes()
.try_into()
.unwrap(),
sighash,
)
};
// Check if this bundle's validation result exists in the cache.
if !cache.contains(cache_entry, &mut batch.queued_entries) {
// The bundle has been added to `inner.queued_entries` because it was not
// in the cache. We now add its authorization to the validation batch.
batch.validator.add_bundle(bundle, *sighash);
}
}
(_, _, None) => error!("orchard_batch_add_bundle() called without sighash!"),
(Some(_), None, Some(_)) => debug!("Tx has no Orchard component"),
(None, Some(_), _) => debug!("Orchard BatchValidator not provided, assuming disabled."),
@ -205,7 +237,15 @@ pub extern "C" fn orchard_batch_validate(batch: *mut BatchValidator) -> bool {
let batch = unsafe { Box::from_raw(batch) };
let vk =
unsafe { crate::ORCHARD_VK.as_ref() }.expect("ORCHARD_VK should have been initialized");
batch.validate(vk, OsRng)
if batch.validator.validate(vk, OsRng) {
// `Self::validate()` is only called if every `Self::check_bundle()`
// returned `true`, so at this point every bundle that was added to
// `inner.queued_entries` has valid authorization.
orchard_bundle_validity_cache_mut().insert(batch.queued_entries);
true
} else {
false
}
} else {
// The orchard::BatchValidator C++ class uses null to represent a disabled batch
// validator.

View File

@ -69,6 +69,7 @@ mod zcashd_orchard;
mod address_ffi;
mod builder_ffi;
mod bundlecache;
mod history_ffi;
mod incremental_merkle_tree;
mod incremental_merkle_tree_ffi;

View File

@ -7,6 +7,8 @@
// on the entire module.
#![allow(clippy::too_many_arguments)]
use std::convert::TryInto;
use bellman::groth16::{prepare_verifying_key, Proof};
use group::GroupEncoding;
@ -17,12 +19,19 @@ use zcash_primitives::{
redjubjub::{self, Signature},
Nullifier,
},
transaction::components::{sapling, Amount},
transaction::{
components::{sapling, Amount},
txid::{BlockTxCommitmentDigester, TxIdDigester},
Authorized, TransactionDigest,
},
};
use zcash_proofs::sapling::{self as sapling_proofs, SaplingVerificationContext};
use super::GROTH_PROOF_SIZE;
use super::{de_ct, SAPLING_OUTPUT_VK, SAPLING_SPEND_VK};
use crate::bundlecache::{
sapling_bundle_validity_cache, sapling_bundle_validity_cache_mut, CacheEntries,
};
#[cxx::bridge(namespace = "sapling")]
mod ffi {
@ -82,7 +91,7 @@ mod ffi {
) -> bool;
type BatchValidator;
fn init_batch_validator() -> Box<BatchValidator>;
fn init_batch_validator(cache_store: bool) -> Box<BatchValidator>;
fn check_bundle(self: &mut BatchValidator, bundle: Box<Bundle>, sighash: [u8; 32]) -> bool;
fn validate(self: &mut BatchValidator) -> bool;
}
@ -90,6 +99,12 @@ mod ffi {
struct Bundle(sapling::Bundle<sapling::Authorized>);
impl Bundle {
fn commitment<D: TransactionDigest<Authorized>>(&self, digester: D) -> D::SaplingDigest {
digester.digest_sapling(Some(&self.0))
}
}
struct BundleAssembler {
shielded_spends: Vec<sapling::SpendDescription<sapling::Authorized>>,
shielded_outputs: Vec<sapling::OutputDescription<[u8; 192]>>, // GROTH_PROOF_SIZE
@ -325,10 +340,18 @@ impl Verifier {
}
}
struct BatchValidator(Option<sapling_proofs::BatchValidator>);
struct BatchValidatorInner {
validator: sapling_proofs::BatchValidator,
queued_entries: CacheEntries,
}
fn init_batch_validator() -> Box<BatchValidator> {
Box::new(BatchValidator(Some(sapling_proofs::BatchValidator::new())))
struct BatchValidator(Option<BatchValidatorInner>);
fn init_batch_validator(cache_store: bool) -> Box<BatchValidator> {
Box::new(BatchValidator(Some(BatchValidatorInner {
validator: sapling_proofs::BatchValidator::new(),
queued_entries: CacheEntries::new(cache_store),
})))
}
impl BatchValidator {
@ -343,8 +366,29 @@ impl BatchValidator {
/// `sighash` must be for the transaction this bundle is within.
#[allow(clippy::boxed_local)]
fn check_bundle(&mut self, bundle: Box<Bundle>, sighash: [u8; 32]) -> bool {
if let Some(validator) = &mut self.0 {
validator.check_bundle(bundle.0, sighash)
if let Some(inner) = &mut self.0 {
let cache = sapling_bundle_validity_cache();
// Compute the cache entry for this bundle.
let cache_entry = {
let bundle_commitment = bundle.commitment(TxIdDigester).unwrap();
let bundle_authorizing_commitment = bundle.commitment(BlockTxCommitmentDigester);
cache.compute_entry(
bundle_commitment.as_bytes().try_into().unwrap(),
bundle_authorizing_commitment.as_bytes().try_into().unwrap(),
&sighash,
)
};
// Check if this bundle's validation result exists in the cache.
if cache.contains(cache_entry, &mut inner.queued_entries) {
true
} else {
// The bundle has been added to `inner.queued_entries` because it was not
// in the cache. We now check the bundle against the Sapling-specific
// consensus rules, and add its authorization to the validation batch.
inner.validator.check_bundle(bundle.0, sighash)
}
} else {
tracing::error!("sapling::BatchValidator has already been used");
false
@ -352,12 +396,21 @@ impl BatchValidator {
}
fn validate(&mut self) -> bool {
if let Some(validator) = self.0.take() {
validator.validate(
if let Some(inner) = self.0.take() {
if inner.validator.validate(
unsafe { SAPLING_SPEND_VK.as_ref() }.unwrap(),
unsafe { SAPLING_OUTPUT_VK.as_ref() }.unwrap(),
OsRng,
)
) {
// `Self::validate()` is only called if every `Self::check_bundle()`
// returned `true`, so at this point every bundle that was added to
// `inner.queued_entries` has valid authorization and satisfies the
// Sapling-specific consensus rules.
sapling_bundle_validity_cache_mut().insert(inner.queued_entries);
true
} else {
false
}
} else {
tracing::error!("sapling::BatchValidator has already been used");
false

View File

@ -70,9 +70,8 @@ static CSignatureCache signatureCache;
}
// To be called once in AppInit2/TestingSetup to initialize the signatureCache
void InitSignatureCache()
void InitSignatureCache(size_t nMaxCacheSize)
{
size_t nMaxCacheSize = GetArg("-maxsigcachesize", DEFAULT_MAX_SIG_CACHE_SIZE) * ((size_t) 1 << 20);
if (nMaxCacheSize <= 0) return;
size_t nElems = signatureCache.setup_bytes(nMaxCacheSize);
LogPrintf("Using %zu MiB out of %zu requested for signature cache, able to store %zu elements\n",

View File

@ -50,6 +50,6 @@ public:
bool VerifySignature(const std::vector<unsigned char>& vchSig, const CPubKey& vchPubKey, const uint256& sighash) const;
};
void InitSignatureCache();
void InitSignatureCache(size_t nMaxCacheSize);
#endif // BITCOIN_SCRIPT_SIGCACHE_H

View File

@ -33,6 +33,8 @@
#include "librustzcash.h"
#include <rust/bundlecache.h>
const std::function<std::string(const char*)> G_TRANSLATION_FUN = nullptr;
CClientUIInterface uiInterface; // Declared but not defined in ui_interface.h
@ -50,7 +52,8 @@ BasicTestingSetup::BasicTestingSetup(const std::string& chainName)
ECC_Start();
SetupEnvironment();
SetupNetworking();
InitSignatureCache();
InitSignatureCache(DEFAULT_MAX_SIG_CACHE_SIZE * ((size_t) 1 << 20));
bundlecache::init(DEFAULT_MAX_SIG_CACHE_SIZE * ((size_t) 1 << 20));
// Uncomment this to log all errors to stdout so we see them in test output.
// We don't enable this by default because several tests intentionally cause

18
src/zcash/cache.cpp Normal file
View File

@ -0,0 +1,18 @@
// Copyright (c) 2022 The Zcash developers
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#include "zcash/cache.h"
#include "util/system.h"
namespace libzcash
{
std::unique_ptr<BundleValidityCache> NewBundleValidityCache(size_t nMaxCacheSize)
{
auto cache = std::unique_ptr<BundleValidityCache>(new BundleValidityCache());
size_t nElems = cache->setup_bytes(nMaxCacheSize);
LogPrintf("Using %zu MiB out of %zu requested for bundle cache, able to store %zu elements\n",
(nElems * sizeof(BundleCacheEntry)) >> 20, nMaxCacheSize >> 20, nElems);
return cache;
}
} // namespace libzcash

45
src/zcash/cache.h Normal file
View File

@ -0,0 +1,45 @@
// Copyright (c) 2022 The Zcash developers
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#ifndef ZCASH_ZCASH_CACHE_H
#define ZCASH_ZCASH_CACHE_H
#include "cuckoocache.h"
#include <array>
namespace libzcash
{
typedef std::array<uint8_t, 32> BundleCacheEntry;
/**
* We're hashing a nonce into the entries themselves, so we don't need extra
* blinding in the set hash computation.
*
* This may exhibit platform endian dependent behavior but because these are
* nonced hashes (random) and this state is only ever used locally it is safe.
* All that matters is local consistency.
*/
class BundleCacheHasher
{
public:
template <uint8_t hash_select>
uint32_t operator()(const BundleCacheEntry& key) const
{
static_assert(hash_select < 8, "BundleCacheHasher only has 8 hashes available.");
uint32_t u;
std::memcpy(&u, key.begin() + 4 * hash_select, 4);
return u;
}
};
typedef CuckooCache::cache<BundleCacheEntry, BundleCacheHasher> BundleValidityCache;
std::unique_ptr<BundleValidityCache> NewBundleValidityCache(size_t nMaxCacheSize);
} // namespace libzcash
template void CuckooCache::cache<libzcash::BundleCacheEntry, libzcash::BundleCacheHasher>::insert(libzcash::BundleCacheEntry e);
template bool CuckooCache::cache<libzcash::BundleCacheEntry, libzcash::BundleCacheHasher>::contains(const libzcash::BundleCacheEntry& e, const bool erase) const;
#endif // ZCASH_ZCASH_CACHE_H