Hash stored accounts in bg (#16157)

* lazy calculate account hash

* push to bg thread

* remove deadlock

* logs

* format

* some cleanup on aisle 9

* format, fix up some metrics

* fix test, remove legacy function only there for tests

* cleanup

* remove unused store_hasher

* Switch to crossbeam

* clippy

* format

* use iter()

* rework from feedback

* hash_slot -> slot

* hash(cluster_type)

Co-authored-by: Carl Lin <carl@solana.com>
This commit is contained in:
Jeff Washington (jwash) 2021-03-31 15:39:34 -05:00 committed by GitHub
parent 6f3926b643
commit f374b35944
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 233 additions and 87 deletions

View File

@ -2,6 +2,7 @@ use dashmap::DashMap;
use solana_sdk::{ use solana_sdk::{
account::{AccountSharedData, ReadableAccount}, account::{AccountSharedData, ReadableAccount},
clock::Slot, clock::Slot,
genesis_config::ClusterType,
hash::Hash, hash::Hash,
pubkey::Pubkey, pubkey::Pubkey,
}; };
@ -47,7 +48,13 @@ impl SlotCacheInner {
); );
} }
pub fn insert(&self, pubkey: &Pubkey, account: AccountSharedData, hash: Hash) { pub fn insert(
&self,
pubkey: &Pubkey,
account: AccountSharedData,
hash: Option<Hash>,
slot: Slot,
) -> CachedAccount {
if self.cache.contains_key(pubkey) { if self.cache.contains_key(pubkey) {
self.same_account_writes.fetch_add(1, Ordering::Relaxed); self.same_account_writes.fetch_add(1, Ordering::Relaxed);
self.same_account_writes_size self.same_account_writes_size
@ -56,7 +63,14 @@ impl SlotCacheInner {
self.unique_account_writes_size self.unique_account_writes_size
.fetch_add(account.data().len() as u64, Ordering::Relaxed); .fetch_add(account.data().len() as u64, Ordering::Relaxed);
} }
self.cache.insert(*pubkey, CachedAccount { account, hash }); let item = Arc::new(CachedAccountInner {
account,
hash: RwLock::new(hash),
slot,
pubkey: *pubkey,
});
self.cache.insert(*pubkey, item.clone());
item
} }
pub fn get_cloned(&self, pubkey: &Pubkey) -> Option<CachedAccount> { pub fn get_cloned(&self, pubkey: &Pubkey) -> Option<CachedAccount> {
@ -89,10 +103,34 @@ impl Deref for SlotCacheInner {
} }
} }
#[derive(Debug, Clone)] pub type CachedAccount = Arc<CachedAccountInner>;
pub struct CachedAccount {
#[derive(Debug)]
pub struct CachedAccountInner {
pub account: AccountSharedData, pub account: AccountSharedData,
pub hash: Hash, hash: RwLock<Option<Hash>>,
slot: Slot,
pubkey: Pubkey,
}
impl CachedAccountInner {
pub fn hash(&self, cluster_type: ClusterType) -> Hash {
let hash = self.hash.read().unwrap();
match *hash {
Some(hash) => hash,
None => {
drop(hash);
let hash = crate::accounts_db::AccountsDb::hash_account(
self.slot,
&self.account,
&self.pubkey,
&cluster_type,
);
*self.hash.write().unwrap() = Some(hash);
hash
}
}
}
} }
#[derive(Debug, Default)] #[derive(Debug, Default)]
@ -128,7 +166,13 @@ impl AccountsCache {
); );
} }
pub fn store(&self, slot: Slot, pubkey: &Pubkey, account: AccountSharedData, hash: Hash) { pub fn store(
&self,
slot: Slot,
pubkey: &Pubkey,
account: AccountSharedData,
hash: Option<Hash>,
) -> CachedAccount {
let slot_cache = self.slot_cache(slot).unwrap_or_else(|| let slot_cache = self.slot_cache(slot).unwrap_or_else(||
// DashMap entry.or_insert() returns a RefMut, essentially a write lock, // DashMap entry.or_insert() returns a RefMut, essentially a write lock,
// which is dropped after this block ends, minimizing time held by the lock. // which is dropped after this block ends, minimizing time held by the lock.
@ -140,7 +184,7 @@ impl AccountsCache {
.or_insert(Arc::new(SlotCacheInner::default())) .or_insert(Arc::new(SlotCacheInner::default()))
.clone()); .clone());
slot_cache.insert(pubkey, account, hash); slot_cache.insert(pubkey, account, hash, slot)
} }
pub fn load(&self, slot: Slot, pubkey: &Pubkey) -> Option<CachedAccount> { pub fn load(&self, slot: Slot, pubkey: &Pubkey) -> Option<CachedAccount> {
@ -241,7 +285,7 @@ pub mod tests {
inserted_slot, inserted_slot,
&Pubkey::new_unique(), &Pubkey::new_unique(),
AccountSharedData::new(1, 0, &Pubkey::default()), AccountSharedData::new(1, 0, &Pubkey::default()),
Hash::default(), Some(Hash::default()),
); );
// If the cache is told the size limit is 0, it should return the one slot // If the cache is told the size limit is 0, it should return the one slot
let removed = cache.remove_slots_le(0); let removed = cache.remove_slots_le(0);
@ -259,7 +303,7 @@ pub mod tests {
inserted_slot, inserted_slot,
&Pubkey::new_unique(), &Pubkey::new_unique(),
AccountSharedData::new(1, 0, &Pubkey::default()), AccountSharedData::new(1, 0, &Pubkey::default()),
Hash::default(), Some(Hash::default()),
); );
// If the cache is told the size limit is 0, it should return nothing because there's only // If the cache is told the size limit is 0, it should return nothing because there's only

View File

@ -29,6 +29,7 @@ use crate::{
contains::Contains, contains::Contains,
}; };
use blake3::traits::digest::Digest; use blake3::traits::digest::Digest;
use crossbeam_channel::{unbounded, Receiver, Sender};
use dashmap::{ use dashmap::{
mapref::entry::Entry::{Occupied, Vacant}, mapref::entry::Entry::{Occupied, Vacant},
DashMap, DashSet, DashMap, DashSet,
@ -58,6 +59,7 @@ use std::{
path::{Path, PathBuf}, path::{Path, PathBuf},
sync::atomic::{AtomicBool, AtomicU64, AtomicUsize, Ordering}, sync::atomic::{AtomicBool, AtomicU64, AtomicUsize, Ordering},
sync::{Arc, Mutex, MutexGuard, RwLock}, sync::{Arc, Mutex, MutexGuard, RwLock},
thread::Builder,
time::Instant, time::Instant,
}; };
use tempfile::TempDir; use tempfile::TempDir;
@ -230,10 +232,10 @@ impl<'a> LoadedAccount<'a> {
} }
} }
pub fn loaded_hash(&self) -> &Hash { pub fn loaded_hash(&self, cluster_type: ClusterType) -> Hash {
match self { match self {
LoadedAccount::Stored(stored_account_meta) => &stored_account_meta.hash, LoadedAccount::Stored(stored_account_meta) => *stored_account_meta.hash,
LoadedAccount::Cached((_, cached_account)) => &cached_account.hash, LoadedAccount::Cached((_, cached_account)) => cached_account.hash(cluster_type),
} }
} }
@ -280,7 +282,7 @@ impl<'a> LoadedAccount<'a> {
match self { match self {
LoadedAccount::Stored(stored_account_meta) => stored_account_meta.clone_account(), LoadedAccount::Stored(stored_account_meta) => stored_account_meta.clone_account(),
LoadedAccount::Cached((_, cached_account)) => match cached_account { LoadedAccount::Cached((_, cached_account)) => match cached_account {
Cow::Owned(cached_account) => cached_account.account, Cow::Owned(cached_account) => cached_account.account.clone(),
Cow::Borrowed(cached_account) => cached_account.account.clone(), Cow::Borrowed(cached_account) => cached_account.account.clone(),
}, },
} }
@ -691,6 +693,8 @@ pub struct AccountsDb {
pub accounts_cache: AccountsCache, pub accounts_cache: AccountsCache,
sender_bg_hasher: Option<Sender<CachedAccount>>,
recycle_stores: RwLock<RecycleStores>, recycle_stores: RwLock<RecycleStores>,
/// distribute the accounts across storage lists /// distribute the accounts across storage lists
@ -1068,6 +1072,7 @@ impl Default for AccountsDb {
accounts_index: AccountsIndex::default(), accounts_index: AccountsIndex::default(),
storage: AccountStorage::default(), storage: AccountStorage::default(),
accounts_cache: AccountsCache::default(), accounts_cache: AccountsCache::default(),
sender_bg_hasher: None,
recycle_stores: RwLock::new(RecycleStores::default()), recycle_stores: RwLock::new(RecycleStores::default()),
uncleaned_pubkeys: DashMap::new(), uncleaned_pubkeys: DashMap::new(),
next_id: AtomicUsize::new(0), next_id: AtomicUsize::new(0),
@ -1109,7 +1114,7 @@ impl AccountsDb {
account_indexes: HashSet<AccountIndex>, account_indexes: HashSet<AccountIndex>,
caching_enabled: bool, caching_enabled: bool,
) -> Self { ) -> Self {
let new = if !paths.is_empty() { let mut new = if !paths.is_empty() {
Self { Self {
paths, paths,
temp_paths: None, temp_paths: None,
@ -1131,6 +1136,8 @@ impl AccountsDb {
..Self::default() ..Self::default()
} }
}; };
new.start_background_hasher();
{ {
for path in new.paths.iter() { for path in new.paths.iter() {
std::fs::create_dir_all(path).expect("Create directory failed."); std::fs::create_dir_all(path).expect("Create directory failed.");
@ -1298,6 +1305,36 @@ impl AccountsDb {
} }
} }
fn background_hasher(receiver: Receiver<CachedAccount>, cluster_type: ClusterType) {
loop {
let result = receiver.recv();
match result {
Ok(account) => {
// if we hold the only ref, then this account doesn't need to be hashed, we ignore this account and it will disappear
if Arc::strong_count(&account) > 1 {
// this will cause the hash to be calculated and store inside account if it needs to be calculated
let _ = (*account).hash(cluster_type);
};
}
Err(_) => {
break;
}
}
}
}
fn start_background_hasher(&mut self) {
let (sender, receiver) = unbounded();
let cluster_type = self.expected_cluster_type();
Builder::new()
.name("solana-accounts-db-store-hasher".to_string())
.spawn(move || {
Self::background_hasher(receiver, cluster_type);
})
.unwrap();
self.sender_bg_hasher = Some(sender);
}
fn purge_keys_exact<'a, C: 'a>( fn purge_keys_exact<'a, C: 'a>(
&'a self, &'a self,
pubkey_to_slot_set: &'a [(Pubkey, C)], pubkey_to_slot_set: &'a [(Pubkey, C)],
@ -1860,7 +1897,7 @@ impl AccountsDb {
store_accounts_timing = self.store_accounts_frozen( store_accounts_timing = self.store_accounts_frozen(
slot, slot,
&accounts, &accounts,
&hashes, Some(&hashes),
Some(Box::new(move |_, _| shrunken_store.clone())), Some(Box::new(move |_, _| shrunken_store.clone())),
Some(Box::new(write_versions.into_iter())), Some(Box::new(write_versions.into_iter())),
); );
@ -2259,7 +2296,7 @@ impl AccountsDb {
self.get_account_accessor_from_cache_or_storage(slot, pubkey, store_id, offset) self.get_account_accessor_from_cache_or_storage(slot, pubkey, store_id, offset)
.get_loaded_account() .get_loaded_account()
.map(|loaded_account| *loaded_account.loaded_hash()) .map(|loaded_account| loaded_account.loaded_hash(self.expected_cluster_type()))
.unwrap() .unwrap()
} }
@ -3280,7 +3317,7 @@ impl AccountsDb {
.map(|should_flush_f| should_flush_f(key, account)) .map(|should_flush_f| should_flush_f(key, account))
.unwrap_or(true); .unwrap_or(true);
if should_flush { if should_flush {
let hash = iter_item.value().hash; let hash = iter_item.value().hash(self.expected_cluster_type());
total_size += (account.data().len() + STORE_META_OVERHEAD) as u64; total_size += (account.data().len() + STORE_META_OVERHEAD) as u64;
num_flushed += 1; num_flushed += 1;
Some(((key, account), hash)) Some(((key, account), hash))
@ -3315,7 +3352,7 @@ impl AccountsDb {
self.store_accounts_frozen( self.store_accounts_frozen(
slot, slot,
&accounts, &accounts,
&hashes, Some(&hashes),
Some(Box::new(move |_, _| flushed_store.clone())), Some(Box::new(move |_, _| flushed_store.clone())),
None, None,
); );
@ -3351,16 +3388,31 @@ impl AccountsDb {
fn write_accounts_to_cache( fn write_accounts_to_cache(
&self, &self,
slot: Slot, slot: Slot,
hashes: &[Hash], hashes: Option<&[Hash]>,
accounts_and_meta_to_store: &[(StoredMeta, &AccountSharedData)], accounts_and_meta_to_store: &[(StoredMeta, &AccountSharedData)],
) -> Vec<AccountInfo> { ) -> Vec<AccountInfo> {
assert_eq!(hashes.len(), accounts_and_meta_to_store.len()); let len = accounts_and_meta_to_store.len();
let hashes = hashes.map(|hashes| {
assert_eq!(hashes.len(), len);
hashes
});
accounts_and_meta_to_store accounts_and_meta_to_store
.iter() .iter()
.zip(hashes) .enumerate()
.map(|((meta, account), hash)| { .map(|(i, (meta, account))| {
self.accounts_cache let hash = hashes.map(|hashes| hashes[i]);
.store(slot, &meta.pubkey, (**account).clone(), *hash); let cached_account =
self.accounts_cache
.store(slot, &meta.pubkey, (*account).clone(), hash);
// hash this account in the bg
match &self.sender_bg_hasher {
Some(ref sender) => {
let _ = sender.send(cached_account);
}
None => (),
};
AccountInfo { AccountInfo {
store_id: CACHE_VIRTUAL_STORAGE_ID, store_id: CACHE_VIRTUAL_STORAGE_ID,
offset: CACHE_VIRTUAL_OFFSET, offset: CACHE_VIRTUAL_OFFSET,
@ -3378,7 +3430,7 @@ impl AccountsDb {
&self, &self,
slot: Slot, slot: Slot,
accounts: &[(&Pubkey, &AccountSharedData)], accounts: &[(&Pubkey, &AccountSharedData)],
hashes: &[Hash], hashes: Option<&[Hash]>,
storage_finder: F, storage_finder: F,
mut write_version_producer: P, mut write_version_producer: P,
is_cached_store: bool, is_cached_store: bool,
@ -3405,12 +3457,42 @@ impl AccountsDb {
if self.caching_enabled && is_cached_store { if self.caching_enabled && is_cached_store {
self.write_accounts_to_cache(slot, hashes, &accounts_and_meta_to_store) self.write_accounts_to_cache(slot, hashes, &accounts_and_meta_to_store)
} else { } else {
self.write_accounts_to_storage( match hashes {
slot, Some(hashes) => self.write_accounts_to_storage(
hashes, slot,
storage_finder, hashes,
&accounts_and_meta_to_store, storage_finder,
) &accounts_and_meta_to_store,
),
None => {
// hash any accounts where we were lazy in calculating the hash
let mut hash_time = Measure::start("hash_accounts");
let mut stats = BankHashStats::default();
let len = accounts_and_meta_to_store.len();
let mut hashes = Vec::with_capacity(len);
for account in accounts {
stats.update(account.1);
let hash = Self::hash_account(
slot,
account.1,
account.0,
&self.expected_cluster_type(),
);
hashes.push(hash);
}
hash_time.stop();
self.stats
.store_hash_accounts
.fetch_add(hash_time.as_us(), Ordering::Relaxed);
self.write_accounts_to_storage(
slot,
&hashes,
storage_finder,
&accounts_and_meta_to_store,
)
}
}
} }
} }
@ -3539,7 +3621,8 @@ impl AccountsDb {
.get_loaded_account() .get_loaded_account()
.and_then( .and_then(
|loaded_account| { |loaded_account| {
let loaded_hash = loaded_account.loaded_hash(); let loaded_hash = loaded_account
.loaded_hash(self.expected_cluster_type());
let balance = let balance =
Self::account_balance_for_capitalization( Self::account_balance_for_capitalization(
account_info.lamports, account_info.lamports,
@ -3554,7 +3637,7 @@ impl AccountsDb {
&self.expected_cluster_type(), &self.expected_cluster_type(),
pubkey, pubkey,
); );
if computed_hash != *loaded_hash { if computed_hash != loaded_hash {
mismatch_found mismatch_found
.fetch_add(1, Ordering::Relaxed); .fetch_add(1, Ordering::Relaxed);
return None; return None;
@ -3562,7 +3645,7 @@ impl AccountsDb {
} }
sum += balance as u128; sum += balance as u128;
Some(*loaded_hash) Some(loaded_hash)
}, },
) )
} else { } else {
@ -3668,6 +3751,7 @@ impl AccountsDb {
Self::calculate_accounts_hash_without_index( Self::calculate_accounts_hash_without_index(
&combined_maps, &combined_maps,
Some(&self.thread_pool_clean), Some(&self.thread_pool_clean),
self.expected_cluster_type(),
) )
} else { } else {
self.calculate_accounts_hash(slot, ancestors, false) self.calculate_accounts_hash(slot, ancestors, false)
@ -3706,6 +3790,7 @@ impl AccountsDb {
mut stats: &mut crate::accounts_hash::HashStats, mut stats: &mut crate::accounts_hash::HashStats,
bins: usize, bins: usize,
bin_range: &Range<usize>, bin_range: &Range<usize>,
cluster_type: ClusterType,
) -> Vec<Vec<Vec<CalculateHashIntermediate>>> { ) -> Vec<Vec<Vec<CalculateHashIntermediate>>> {
let max_plus_1 = std::u8::MAX as usize + 1; let max_plus_1 = std::u8::MAX as usize + 1;
assert!(bins <= max_plus_1 && bins > 0); assert!(bins <= max_plus_1 && bins > 0);
@ -3739,7 +3824,7 @@ impl AccountsDb {
let source_item = CalculateHashIntermediate::new( let source_item = CalculateHashIntermediate::new(
version, version,
*loaded_account.loaded_hash(), loaded_account.loaded_hash(cluster_type),
balance, balance,
slot, slot,
pubkey, pubkey,
@ -3761,6 +3846,7 @@ impl AccountsDb {
pub fn calculate_accounts_hash_without_index( pub fn calculate_accounts_hash_without_index(
storages: &[SnapshotStorage], storages: &[SnapshotStorage],
thread_pool: Option<&ThreadPool>, thread_pool: Option<&ThreadPool>,
cluster_type: ClusterType,
) -> (Hash, u64) { ) -> (Hash, u64) {
let scan_and_hash = || { let scan_and_hash = || {
let mut stats = HashStats::default(); let mut stats = HashStats::default();
@ -3793,6 +3879,7 @@ impl AccountsDb {
&mut stats, &mut stats,
PUBKEY_BINS_FOR_CALCULATING_HASHES, PUBKEY_BINS_FOR_CALCULATING_HASHES,
&bounds, &bounds,
cluster_type,
); );
let (hash, lamports, for_next_pass) = AccountsHash::rest_of_hash_calculation( let (hash, lamports, for_next_pass) = AccountsHash::rest_of_hash_calculation(
@ -3856,11 +3943,14 @@ impl AccountsDb {
slot, slot,
|loaded_account: LoadedAccount| { |loaded_account: LoadedAccount| {
// Cache only has one version per key, don't need to worry about versioning // Cache only has one version per key, don't need to worry about versioning
Some((*loaded_account.pubkey(), *loaded_account.loaded_hash())) Some((
*loaded_account.pubkey(),
loaded_account.loaded_hash(self.expected_cluster_type()),
))
}, },
|accum: &DashMap<Pubkey, (u64, Hash)>, loaded_account: LoadedAccount| { |accum: &DashMap<Pubkey, (u64, Hash)>, loaded_account: LoadedAccount| {
let loaded_write_version = loaded_account.write_version(); let loaded_write_version = loaded_account.write_version();
let loaded_hash = *loaded_account.loaded_hash(); let loaded_hash = loaded_account.loaded_hash(self.expected_cluster_type());
let should_insert = let should_insert =
if let Some(existing_entry) = accum.get(loaded_account.pubkey()) { if let Some(existing_entry) = accum.get(loaded_account.pubkey()) {
loaded_write_version > existing_entry.value().version() loaded_write_version > existing_entry.value().version()
@ -4098,36 +4188,6 @@ impl AccountsDb {
inc_new_counter_info!("clean_stored_dead_slots-ms", measure.as_ms() as usize); inc_new_counter_info!("clean_stored_dead_slots-ms", measure.as_ms() as usize);
} }
fn hash_accounts(
&self,
slot: Slot,
accounts: &[(&Pubkey, &AccountSharedData)],
cluster_type: &ClusterType,
) -> Vec<Hash> {
let mut stats = BankHashStats::default();
let mut total_data = 0;
let hashes: Vec<_> = accounts
.iter()
.map(|(pubkey, account)| {
total_data += account.data().len();
stats.update(account);
Self::hash_account(slot, account, pubkey, cluster_type)
})
.collect();
self.stats
.store_total_data
.fetch_add(total_data as u64, Ordering::Relaxed);
let mut bank_hashes = self.bank_hashes.write().unwrap();
let slot_info = bank_hashes
.entry(slot)
.or_insert_with(BankHashInfo::default);
slot_info.stats.merge(&stats);
hashes
}
pub(crate) fn freeze_accounts(&mut self, ancestors: &Ancestors, account_pubkeys: &[Pubkey]) { pub(crate) fn freeze_accounts(&mut self, ancestors: &Ancestors, account_pubkeys: &[Pubkey]) {
for account_pubkey in account_pubkeys { for account_pubkey in account_pubkeys {
if let Some((account, _slot)) = self.load_slow(ancestors, &account_pubkey) { if let Some((account, _slot)) = self.load_slow(ancestors, &account_pubkey) {
@ -4193,13 +4253,26 @@ impl AccountsDb {
return; return;
} }
self.assert_frozen_accounts(accounts); self.assert_frozen_accounts(accounts);
let mut hash_time = Measure::start("hash_accounts");
let hashes = self.hash_accounts(slot, accounts, &self.expected_cluster_type()); let mut stats = BankHashStats::default();
hash_time.stop(); let mut total_data = 0;
accounts.iter().for_each(|(_pubkey, account)| {
total_data += account.data().len();
stats.update(account);
});
self.stats self.stats
.store_hash_accounts .store_total_data
.fetch_add(hash_time.as_us(), Ordering::Relaxed); .fetch_add(total_data as u64, Ordering::Relaxed);
self.store_accounts_unfrozen(slot, accounts, &hashes, is_cached_store);
let mut bank_hashes = self.bank_hashes.write().unwrap();
let slot_info = bank_hashes
.entry(slot)
.or_insert_with(BankHashInfo::default);
slot_info.stats.merge(&stats);
// we use default hashes for now since the same account may be stored to the cache multiple times
self.store_accounts_unfrozen(slot, accounts, None, is_cached_store);
self.report_store_timings(); self.report_store_timings();
} }
@ -4305,7 +4378,7 @@ impl AccountsDb {
&self, &self,
slot: Slot, slot: Slot,
accounts: &[(&Pubkey, &AccountSharedData)], accounts: &[(&Pubkey, &AccountSharedData)],
hashes: &[Hash], hashes: Option<&[Hash]>,
is_cached_store: bool, is_cached_store: bool,
) { ) {
// This path comes from a store to a non-frozen slot. // This path comes from a store to a non-frozen slot.
@ -4331,7 +4404,7 @@ impl AccountsDb {
&'a self, &'a self,
slot: Slot, slot: Slot,
accounts: &[(&Pubkey, &AccountSharedData)], accounts: &[(&Pubkey, &AccountSharedData)],
hashes: &[Hash], hashes: Option<&[Hash]>,
storage_finder: Option<StorageFinder<'a>>, storage_finder: Option<StorageFinder<'a>>,
write_version_producer: Option<Box<dyn Iterator<Item = u64>>>, write_version_producer: Option<Box<dyn Iterator<Item = u64>>>,
) -> StoreAccountsTiming { ) -> StoreAccountsTiming {
@ -4355,7 +4428,7 @@ impl AccountsDb {
&'a self, &'a self,
slot: Slot, slot: Slot,
accounts: &[(&Pubkey, &AccountSharedData)], accounts: &[(&Pubkey, &AccountSharedData)],
hashes: &[Hash], hashes: Option<&[Hash]>,
storage_finder: Option<StorageFinder<'a>>, storage_finder: Option<StorageFinder<'a>>,
write_version_producer: Option<Box<dyn Iterator<Item = u64>>>, write_version_producer: Option<Box<dyn Iterator<Item = u64>>>,
is_cached_store: bool, is_cached_store: bool,
@ -4842,7 +4915,7 @@ impl AccountsDb {
store_accounts_timing = self.store_accounts_frozen( store_accounts_timing = self.store_accounts_frozen(
slot, slot,
&accounts, &accounts,
&hashes, Some(&hashes),
Some(Box::new(move |_, _| shrunken_store.clone())), Some(Box::new(move |_, _| shrunken_store.clone())),
Some(Box::new(write_versions.into_iter())), Some(Box::new(write_versions.into_iter())),
); );
@ -5055,7 +5128,7 @@ pub mod tests {
let mut stats = HashStats::default(); let mut stats = HashStats::default();
let bounds = Range { start: 0, end: 0 }; let bounds = Range { start: 0, end: 0 };
AccountsDb::scan_snapshot_stores(&[], &mut stats, 257, &bounds); AccountsDb::scan_snapshot_stores(&[], &mut stats, 257, &bounds, ClusterType::Development);
} }
#[test] #[test]
#[should_panic(expected = "assertion failed: bins <= max_plus_1 && bins > 0")] #[should_panic(expected = "assertion failed: bins <= max_plus_1 && bins > 0")]
@ -5063,7 +5136,7 @@ pub mod tests {
let mut stats = HashStats::default(); let mut stats = HashStats::default();
let bounds = Range { start: 0, end: 0 }; let bounds = Range { start: 0, end: 0 };
AccountsDb::scan_snapshot_stores(&[], &mut stats, 0, &bounds); AccountsDb::scan_snapshot_stores(&[], &mut stats, 0, &bounds, ClusterType::Development);
} }
#[test] #[test]
@ -5074,7 +5147,7 @@ pub mod tests {
let mut stats = HashStats::default(); let mut stats = HashStats::default();
let bounds = Range { start: 2, end: 2 }; let bounds = Range { start: 2, end: 2 };
AccountsDb::scan_snapshot_stores(&[], &mut stats, 2, &bounds); AccountsDb::scan_snapshot_stores(&[], &mut stats, 2, &bounds, ClusterType::Development);
} }
#[test] #[test]
#[should_panic( #[should_panic(
@ -5084,7 +5157,7 @@ pub mod tests {
let mut stats = HashStats::default(); let mut stats = HashStats::default();
let bounds = Range { start: 1, end: 3 }; let bounds = Range { start: 1, end: 3 };
AccountsDb::scan_snapshot_stores(&[], &mut stats, 2, &bounds); AccountsDb::scan_snapshot_stores(&[], &mut stats, 2, &bounds, ClusterType::Development);
} }
#[test] #[test]
@ -5095,7 +5168,7 @@ pub mod tests {
let mut stats = HashStats::default(); let mut stats = HashStats::default();
let bounds = Range { start: 1, end: 0 }; let bounds = Range { start: 1, end: 0 };
AccountsDb::scan_snapshot_stores(&[], &mut stats, 2, &bounds); AccountsDb::scan_snapshot_stores(&[], &mut stats, 2, &bounds, ClusterType::Development);
} }
fn sample_storages_and_accounts() -> (SnapshotStorages, Vec<CalculateHashIntermediate>) { fn sample_storages_and_accounts() -> (SnapshotStorages, Vec<CalculateHashIntermediate>) {
@ -5191,6 +5264,7 @@ pub mod tests {
start: 0, start: 0,
end: bins, end: bins,
}, },
ClusterType::Development,
); );
assert_eq!(result, vec![vec![raw_expected.clone()]]); assert_eq!(result, vec![vec![raw_expected.clone()]]);
@ -5203,6 +5277,7 @@ pub mod tests {
start: 0, start: 0,
end: bins, end: bins,
}, },
ClusterType::Development,
); );
let mut expected = vec![Vec::new(); bins]; let mut expected = vec![Vec::new(); bins];
expected[0].push(raw_expected[0].clone()); expected[0].push(raw_expected[0].clone());
@ -5220,6 +5295,7 @@ pub mod tests {
start: 0, start: 0,
end: bins, end: bins,
}, },
ClusterType::Development,
); );
let mut expected = vec![Vec::new(); bins]; let mut expected = vec![Vec::new(); bins];
expected[0].push(raw_expected[0].clone()); expected[0].push(raw_expected[0].clone());
@ -5237,6 +5313,7 @@ pub mod tests {
start: 0, start: 0,
end: bins, end: bins,
}, },
ClusterType::Development,
); );
let mut expected = vec![Vec::new(); bins]; let mut expected = vec![Vec::new(); bins];
expected[0].push(raw_expected[0].clone()); expected[0].push(raw_expected[0].clone());
@ -5266,6 +5343,7 @@ pub mod tests {
start: 0, start: 0,
end: bins, end: bins,
}, },
ClusterType::Development,
); );
assert_eq!(result.len(), 2); // 2 chunks assert_eq!(result.len(), 2); // 2 chunks
assert_eq!(result[0].len(), 0); // nothing found in first slots assert_eq!(result[0].len(), 0); // nothing found in first slots
@ -5288,6 +5366,7 @@ pub mod tests {
start: 0, start: 0,
end: bins / 2, end: bins / 2,
}, },
ClusterType::Development,
); );
let mut expected = vec![Vec::new(); bins]; let mut expected = vec![Vec::new(); bins];
expected[0].push(raw_expected[0].clone()); expected[0].push(raw_expected[0].clone());
@ -5303,6 +5382,7 @@ pub mod tests {
start: 1, start: 1,
end: bins, end: bins,
}, },
ClusterType::Development,
); );
let mut expected = vec![Vec::new(); bins]; let mut expected = vec![Vec::new(); bins];
@ -5321,6 +5401,7 @@ pub mod tests {
start: bin, start: bin,
end: bin + 1, end: bin + 1,
}, },
ClusterType::Development,
); );
let mut expected = vec![Vec::new(); bins]; let mut expected = vec![Vec::new(); bins];
expected[bin].push(raw_expected[bin].clone()); expected[bin].push(raw_expected[bin].clone());
@ -5338,6 +5419,7 @@ pub mod tests {
start: bin, start: bin,
end: bin + 1, end: bin + 1,
}, },
ClusterType::Development,
); );
let mut expected = vec![]; let mut expected = vec![];
if let Some(index) = bin_locations.iter().position(|&r| r == bin) { if let Some(index) = bin_locations.iter().position(|&r| r == bin) {
@ -5369,6 +5451,7 @@ pub mod tests {
start: 127, start: 127,
end: 128, end: 128,
}, },
ClusterType::Development,
); );
assert_eq!(result.len(), 2); // 2 chunks assert_eq!(result.len(), 2); // 2 chunks
assert_eq!(result[0].len(), 0); // nothing found in first slots assert_eq!(result[0].len(), 0); // nothing found in first slots
@ -5383,7 +5466,11 @@ pub mod tests {
solana_logger::setup(); solana_logger::setup();
let (storages, _size, _slot_expected) = sample_storage(); let (storages, _size, _slot_expected) = sample_storage();
let result = AccountsDb::calculate_accounts_hash_without_index(&storages, None); let result = AccountsDb::calculate_accounts_hash_without_index(
&storages,
None,
ClusterType::Development,
);
let expected_hash = Hash::from_str("GKot5hBsd81kMupNCXHaqbhv3huEbxAFMLnpcX2hniwn").unwrap(); let expected_hash = Hash::from_str("GKot5hBsd81kMupNCXHaqbhv3huEbxAFMLnpcX2hniwn").unwrap();
assert_eq!(result, (expected_hash, 0)); assert_eq!(result, (expected_hash, 0));
} }
@ -5398,7 +5485,11 @@ pub mod tests {
item.hash item.hash
}); });
let sum = raw_expected.iter().map(|item| item.lamports).sum(); let sum = raw_expected.iter().map(|item| item.lamports).sum();
let result = AccountsDb::calculate_accounts_hash_without_index(&storages, None); let result = AccountsDb::calculate_accounts_hash_without_index(
&storages,
None,
ClusterType::Development,
);
assert_eq!(result, (expected_hash, sum)); assert_eq!(result, (expected_hash, sum));
} }
@ -7252,11 +7343,16 @@ pub mod tests {
let ancestors = vec![(some_slot, 0)].into_iter().collect(); let ancestors = vec![(some_slot, 0)].into_iter().collect();
let accounts = &[(&key, &account)]; let accounts = &[(&key, &account)];
// update AccountsDb's bank hash but discard real account hashes // update AccountsDb's bank hash
db.hash_accounts(some_slot, accounts, &ClusterType::Development); {
let mut bank_hashes = db.bank_hashes.write().unwrap();
bank_hashes
.entry(some_slot)
.or_insert_with(BankHashInfo::default);
}
// provide bogus account hashes // provide bogus account hashes
let some_hash = Hash::new(&[0xca; HASH_BYTES]); let some_hash = Hash::new(&[0xca; HASH_BYTES]);
db.store_accounts_unfrozen(some_slot, accounts, &[some_hash], false); db.store_accounts_unfrozen(some_slot, accounts, Some(&[some_hash]), false);
db.add_root(some_slot); db.add_root(some_slot);
assert_matches!( assert_matches!(
db.verify_bank_hash_and_lamports(some_slot, &ancestors, 1), db.verify_bank_hash_and_lamports(some_slot, &ancestors, 1),

View File

@ -2,6 +2,7 @@ use crate::bank_forks::ArchiveFormat;
use crate::snapshot_utils::SnapshotVersion; use crate::snapshot_utils::SnapshotVersion;
use crate::{accounts_db::SnapshotStorages, bank::BankSlotDelta}; use crate::{accounts_db::SnapshotStorages, bank::BankSlotDelta};
use solana_sdk::clock::Slot; use solana_sdk::clock::Slot;
use solana_sdk::genesis_config::ClusterType;
use solana_sdk::hash::Hash; use solana_sdk::hash::Hash;
use std::{ use std::{
path::PathBuf, path::PathBuf,
@ -26,6 +27,7 @@ pub struct AccountsPackagePre {
pub snapshot_output_dir: PathBuf, pub snapshot_output_dir: PathBuf,
pub expected_capitalization: u64, pub expected_capitalization: u64,
pub hash_for_testing: Option<Hash>, pub hash_for_testing: Option<Hash>,
pub cluster_type: ClusterType,
} }
impl AccountsPackagePre { impl AccountsPackagePre {
@ -42,6 +44,7 @@ impl AccountsPackagePre {
snapshot_output_dir: PathBuf, snapshot_output_dir: PathBuf,
expected_capitalization: u64, expected_capitalization: u64,
hash_for_testing: Option<Hash>, hash_for_testing: Option<Hash>,
cluster_type: ClusterType,
) -> Self { ) -> Self {
Self { Self {
slot, slot,
@ -55,6 +58,7 @@ impl AccountsPackagePre {
snapshot_output_dir, snapshot_output_dir,
expected_capitalization, expected_capitalization,
hash_for_testing, hash_for_testing,
cluster_type,
} }
} }
} }

View File

@ -187,6 +187,7 @@ pub fn package_snapshot<P: AsRef<Path>, Q: AsRef<Path>>(
snapshot_package_output_path.as_ref().to_path_buf(), snapshot_package_output_path.as_ref().to_path_buf(),
bank.capitalization(), bank.capitalization(),
hash_for_testing, hash_for_testing,
bank.cluster_type(),
); );
Ok(package) Ok(package)
@ -978,6 +979,7 @@ pub fn process_accounts_package_pre(
let (hash, lamports) = AccountsDb::calculate_accounts_hash_without_index( let (hash, lamports) = AccountsDb::calculate_accounts_hash_without_index(
&accounts_package.storages, &accounts_package.storages,
thread_pool, thread_pool,
accounts_package.cluster_type,
); );
assert_eq!(accounts_package.expected_capitalization, lamports); assert_eq!(accounts_package.expected_capitalization, lamports);