solana_runtime/
accounts_db.rs

1//! Persistent accounts are stored at this path location:
2//!  `<path>/<pid>/data/`
3//!
4//! The persistent store would allow for this mode of operation:
5//!  - Concurrent single thread append with many concurrent readers.
6//!
7//! The underlying memory is memory mapped to a file. The accounts would be
8//! stored across multiple files and the mappings of file and offset of a
9//! particular account would be stored in a shared index. This will allow for
10//! concurrent commits without blocking reads, which will sequentially write
11//! to memory, ssd or disk, and should be as fast as the hardware allow for.
12//! The only required in memory data structure with a write lock is the index,
13//! which should be fast to update.
14//!
15//! [`AppendVec`]'s only store accounts for single slots.  To bootstrap the
16//! index from a persistent store of [`AppendVec`]'s, the entries include
17//! a "write_version".  A single global atomic `AccountsDb::write_version`
18//! tracks the number of commits to the entire data store. So the latest
19//! commit for each slot entry would be indexed.
20
21use {
22    crate::{
23        account_info::{AccountInfo, Offset, StorageLocation, StoredSize},
24        account_storage::{AccountStorage, AccountStorageStatus, ShrinkInProgress},
25        accounts_background_service::{DroppedSlotsSender, SendDroppedBankCallback},
26        accounts_cache::{AccountsCache, CachedAccount, SlotCache},
27        accounts_file::AccountsFile,
28        accounts_hash::{
29            AccountsDeltaHash, AccountsHash, AccountsHasher, CalcAccountsHashConfig,
30            CalculateHashIntermediate, HashStats, ZeroLamportAccounts,
31        },
32        accounts_index::{
33            AccountIndexGetResult, AccountSecondaryIndexes, AccountsIndex, AccountsIndexConfig,
34            AccountsIndexRootsStats, AccountsIndexScanResult, IndexKey, IndexValue, IsCached,
35            RefCount, ScanConfig, ScanResult, SlotList, UpsertReclaim, ZeroLamport,
36            ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS, ACCOUNTS_INDEX_CONFIG_FOR_TESTING,
37        },
38        accounts_index_storage::Startup,
39        accounts_update_notifier_interface::AccountsUpdateNotifier,
40        active_stats::{ActiveStatItem, ActiveStats},
41        ancestors::Ancestors,
42        ancient_append_vecs::{
43            get_ancient_append_vec_capacity, is_ancient, AccountsToStore, StorageSelector,
44        },
45        append_vec::{
46            aligned_stored_size, AppendVec, MatchAccountOwnerError,
47            StorableAccountsWithHashesAndWriteVersions, StoredAccountMeta, StoredMetaWriteVersion,
48            APPEND_VEC_MMAPPED_FILES_OPEN, STORE_META_OVERHEAD,
49        },
50        cache_hash_data::{CacheHashData, CacheHashDataFile},
51        contains::Contains,
52        epoch_accounts_hash::EpochAccountsHashManager,
53        pubkey_bins::PubkeyBinCalculator24,
54        read_only_accounts_cache::ReadOnlyAccountsCache,
55        rent_collector::RentCollector,
56        rent_paying_accounts_by_partition::RentPayingAccountsByPartition,
57        snapshot_utils::create_accounts_run_and_snapshot_dirs,
58        sorted_storages::SortedStorages,
59        storable_accounts::StorableAccounts,
60        verify_accounts_hash_in_background::VerifyAccountsHashInBackground,
61    },
62    blake3::traits::digest::Digest,
63    crossbeam_channel::{unbounded, Receiver, Sender},
64    dashmap::{
65        mapref::entry::Entry::{Occupied, Vacant},
66        DashMap, DashSet,
67    },
68    log::*,
69    rand::{thread_rng, Rng},
70    rayon::{prelude::*, ThreadPool},
71    serde::{Deserialize, Serialize},
72    solana_measure::{measure::Measure, measure_us},
73    solana_rayon_threadlimit::get_thread_count,
74    solana_sdk::{
75        account::{Account, AccountSharedData, ReadableAccount, WritableAccount},
76        clock::{BankId, Epoch, Slot},
77        epoch_schedule::EpochSchedule,
78        genesis_config::{ClusterType, GenesisConfig},
79        hash::Hash,
80        pubkey::Pubkey,
81        rent::Rent,
82        saturating_add_assign,
83        timing::AtomicInterval,
84        transaction::SanitizedTransaction,
85    },
86    std::{
87        borrow::{Borrow, Cow},
88        boxed::Box,
89        collections::{BTreeSet, HashMap, HashSet},
90        hash::{Hash as StdHash, Hasher as StdHasher},
91        io::{Error as IoError, Result as IoResult},
92        ops::{Range, RangeBounds},
93        path::{Path, PathBuf},
94        str::FromStr,
95        sync::{
96            atomic::{AtomicBool, AtomicU32, AtomicU64, AtomicUsize, Ordering},
97            Arc, Condvar, Mutex, RwLock,
98        },
99        thread::{sleep, Builder},
100        time::{Duration, Instant},
101    },
102    tempfile::TempDir,
103};
104
105const PAGE_SIZE: u64 = 4 * 1024;
106const MAX_RECYCLE_STORES: usize = 1000;
107// when the accounts write cache exceeds this many bytes, we will flush it
108// this can be specified on the command line, too (--accounts-db-cache-limit-mb)
109const WRITE_CACHE_LIMIT_BYTES_DEFAULT: u64 = 15_000_000_000;
110const SCAN_SLOT_PAR_ITER_THRESHOLD: usize = 4000;
111
112const UNREF_ACCOUNTS_BATCH_SIZE: usize = 10_000;
113
114pub const DEFAULT_FILE_SIZE: u64 = PAGE_SIZE * 1024;
115pub const DEFAULT_NUM_THREADS: u32 = 8;
116pub const DEFAULT_NUM_DIRS: u32 = 4;
117
118// When calculating hashes, it is helpful to break the pubkeys found into bins based on the pubkey value.
119// More bins means smaller vectors to sort, copy, etc.
120pub const PUBKEY_BINS_FOR_CALCULATING_HASHES: usize = 65536;
121
122// Without chunks, we end up with 1 output vec for each outer snapshot storage.
123// This results in too many vectors to be efficient.
124// Chunks when scanning storages to calculate hashes.
125// If this is too big, we don't get enough parallelism of scanning storages.
126// If this is too small, then we produce too many output vectors to iterate.
127// Metrics indicate a sweet spot in the 2.5k-5k range for mnb.
128const MAX_ITEMS_PER_CHUNK: Slot = 2_500;
129
130// A specially reserved offset (represents an offset into an AppendVec)
131// for entries in the cache, so that  operations that take a storage entry can maintain
132// a common interface when interacting with cached accounts. This version is "virtual" in
133// that it doesn't actually map to an entry in an AppendVec.
134pub(crate) const CACHE_VIRTUAL_OFFSET: Offset = 0;
135const CACHE_VIRTUAL_STORED_SIZE: StoredSize = 0;
136
137// When getting accounts for shrinking from the index, this is the # of accounts to lookup per thread.
138// This allows us to split up accounts index accesses across multiple threads.
139const SHRINK_COLLECT_CHUNK_SIZE: usize = 50;
140
141/// temporary enum during feature activation of
142/// ignore slot when calculating an account hash #28420
143#[derive(Debug, Clone, Copy)]
144pub enum IncludeSlotInHash {
145    /// this is the status quo, prior to feature activation
146    /// INCLUDE the slot in the account hash calculation
147    IncludeSlot,
148    /// this is the value once feature activation occurs
149    /// do NOT include the slot in the account hash calculation
150    RemoveSlot,
151    /// this option should not be used.
152    /// If it is, this is a panic worthy event.
153    /// There are code paths where the feature activation status isn't known, but this value should not possibly be used.
154    IrrelevantAssertOnUse,
155}
156
157#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
158pub enum CreateAncientStorage {
159    /// ancient storages are created by appending
160    #[default]
161    Append,
162    /// ancient storages are created by 1-shot write to pack multiple accounts together more efficiently with new formats
163    Pack,
164}
165
166#[derive(Debug)]
167enum StoreTo<'a> {
168    /// write to cache
169    Cache,
170    /// write to storage
171    Storage(&'a Arc<AccountStorageEntry>),
172}
173
174impl<'a> StoreTo<'a> {
175    fn is_cached(&self) -> bool {
176        matches!(self, StoreTo::Cache)
177    }
178}
179
180#[derive(Default, Debug)]
181/// hold alive accounts
182/// alive means in the accounts index
183pub(crate) struct AliveAccounts<'a> {
184    /// slot the accounts are currently stored in
185    pub(crate) slot: Slot,
186    pub(crate) accounts: Vec<&'a StoredAccountMeta<'a>>,
187    pub(crate) bytes: usize,
188}
189
190/// separate pubkeys into those with a single refcount and those with > 1 refcount
191#[derive(Debug)]
192pub(crate) struct ShrinkCollectAliveSeparatedByRefs<'a> {
193    pub(crate) one_ref: AliveAccounts<'a>,
194    pub(crate) many_refs: AliveAccounts<'a>,
195}
196
197/// Configuration Parameters for running bank hash and total lamports verification
198#[derive(Debug, Clone)]
199pub struct BankHashLamportsVerifyConfig<'a> {
200    /// bank ancestors
201    pub ancestors: &'a Ancestors,
202    /// true to verify hash calculation
203    pub test_hash_calculation: bool,
204    /// epoch_schedule
205    pub epoch_schedule: &'a EpochSchedule,
206    /// rent_collector
207    pub rent_collector: &'a RentCollector,
208    /// true to ignore mismatches
209    pub ignore_mismatch: bool,
210    /// true to dump debug log if mismatch happens
211    pub store_detailed_debug_info: bool,
212    /// true to use dedicated background thread pool for verification
213    pub use_bg_thread_pool: bool,
214}
215
216pub(crate) trait ShrinkCollectRefs<'a>: Sync + Send {
217    fn with_capacity(capacity: usize, slot: Slot) -> Self;
218    fn collect(&mut self, other: Self);
219    fn add(&mut self, ref_count: u64, account: &'a StoredAccountMeta<'a>);
220    fn len(&self) -> usize;
221    fn alive_bytes(&self) -> usize;
222    fn alive_accounts(&self) -> &Vec<&'a StoredAccountMeta<'a>>;
223}
224
225impl<'a> ShrinkCollectRefs<'a> for AliveAccounts<'a> {
226    fn collect(&mut self, mut other: Self) {
227        self.bytes = self.bytes.saturating_add(other.bytes);
228        self.accounts.append(&mut other.accounts);
229    }
230    fn with_capacity(capacity: usize, slot: Slot) -> Self {
231        Self {
232            accounts: Vec::with_capacity(capacity),
233            bytes: 0,
234            slot,
235        }
236    }
237    fn add(&mut self, _ref_count: u64, account: &'a StoredAccountMeta<'a>) {
238        self.accounts.push(account);
239        self.bytes = self.bytes.saturating_add(account.stored_size);
240    }
241    fn len(&self) -> usize {
242        self.accounts.len()
243    }
244    fn alive_bytes(&self) -> usize {
245        self.bytes
246    }
247    fn alive_accounts(&self) -> &Vec<&'a StoredAccountMeta<'a>> {
248        &self.accounts
249    }
250}
251
252impl<'a> ShrinkCollectRefs<'a> for ShrinkCollectAliveSeparatedByRefs<'a> {
253    fn collect(&mut self, other: Self) {
254        self.one_ref.collect(other.one_ref);
255        self.many_refs.collect(other.many_refs);
256    }
257    fn with_capacity(capacity: usize, slot: Slot) -> Self {
258        Self {
259            one_ref: AliveAccounts::with_capacity(capacity, slot),
260            many_refs: AliveAccounts::with_capacity(capacity, slot),
261        }
262    }
263    fn add(&mut self, ref_count: u64, account: &'a StoredAccountMeta<'a>) {
264        let other = if ref_count == 1 {
265            &mut self.one_ref
266        } else {
267            &mut self.many_refs
268        };
269        other.add(ref_count, account);
270    }
271    fn len(&self) -> usize {
272        self.one_ref.len().saturating_add(self.many_refs.len())
273    }
274    fn alive_bytes(&self) -> usize {
275        self.one_ref
276            .alive_bytes()
277            .saturating_add(self.many_refs.alive_bytes())
278    }
279    fn alive_accounts(&self) -> &Vec<&'a StoredAccountMeta<'a>> {
280        unimplemented!("illegal use");
281    }
282}
283
284/// used by tests for 'include_slot_in_hash' parameter
285/// Tests just need to be self-consistent, so any value should work here.
286pub const INCLUDE_SLOT_IN_HASH_TESTS: IncludeSlotInHash = IncludeSlotInHash::IncludeSlot;
287
288// This value is irrelevant because we are reading from append vecs and the hash is already computed and saved.
289// The hash will just be loaded from the append vec as opposed to being calculated initially.
290// A shrink-type operation involves reading from an append vec and writing a subset of the read accounts to a new append vec.
291// So, by definition, we will just read hashes and write hashes. The hash will not be calculated.
292// The 'store' apis are shared, such that the initial store from a bank (where we need to know whether to include the slot)
293// must include a feature-based value for 'include_slot_in_hash'. Other uses, specifically shrink, do NOT need to pass this
294// parameter, but the shared api requires a value.
295pub const INCLUDE_SLOT_IN_HASH_IRRELEVANT_APPEND_VEC_OPERATION: IncludeSlotInHash =
296    IncludeSlotInHash::IrrelevantAssertOnUse;
297
298// This value is irrelevant because the the debug-only check_hash debug option is not possible to enable at the moment.
299// This has been true for some time now, due to fallout from disabling rewrites.
300// The check_hash debug option can be re-enabled once this feature and the 'rent_epoch' features are enabled.
301pub const INCLUDE_SLOT_IN_HASH_IRRELEVANT_CHECK_HASH: IncludeSlotInHash =
302    IncludeSlotInHash::IrrelevantAssertOnUse;
303
304pub enum StoreReclaims {
305    /// normal reclaim mode
306    Default,
307    /// do not return reclaims from accounts index upsert
308    Ignore,
309}
310
311/// while combining into ancient append vecs, we need to keep track of the current one that is receiving new data
312/// The pattern for callers is:
313/// 1. this is a mut local
314/// 2. do some version of create/new
315/// 3. use it (slot, append_vec, etc.)
316/// 4. re-create it sometimes
317/// 5. goto 3
318/// If a caller uses it before initializing it, it will be a runtime unwrap() error, similar to an assert.
319/// That condition is an illegal use pattern and is justifiably an assertable condition.
320#[derive(Default)]
321struct CurrentAncientAppendVec {
322    slot_and_append_vec: Option<(Slot, Arc<AccountStorageEntry>)>,
323}
324
325impl CurrentAncientAppendVec {
326    fn new(slot: Slot, append_vec: Arc<AccountStorageEntry>) -> CurrentAncientAppendVec {
327        Self {
328            slot_and_append_vec: Some((slot, append_vec)),
329        }
330    }
331
332    #[must_use]
333    fn create_ancient_append_vec<'a>(
334        &mut self,
335        slot: Slot,
336        db: &'a AccountsDb,
337    ) -> ShrinkInProgress<'a> {
338        let shrink_in_progress = db.get_store_for_shrink(slot, get_ancient_append_vec_capacity());
339        *self = Self::new(slot, Arc::clone(shrink_in_progress.new_storage()));
340        shrink_in_progress
341    }
342    #[must_use]
343    fn create_if_necessary<'a>(
344        &mut self,
345        slot: Slot,
346        db: &'a AccountsDb,
347    ) -> Option<ShrinkInProgress<'a>> {
348        if self.slot_and_append_vec.is_none() {
349            Some(self.create_ancient_append_vec(slot, db))
350        } else {
351            None
352        }
353    }
354
355    /// note this requires that 'slot_and_append_vec' is Some
356    fn slot(&self) -> Slot {
357        self.slot_and_append_vec.as_ref().unwrap().0
358    }
359
360    /// note this requires that 'slot_and_append_vec' is Some
361    fn append_vec(&self) -> &Arc<AccountStorageEntry> {
362        &self.slot_and_append_vec.as_ref().unwrap().1
363    }
364
365    /// helper function to cleanup call to 'store_accounts_frozen'
366    fn store_ancient_accounts(
367        &self,
368        db: &AccountsDb,
369        accounts_to_store: &AccountsToStore,
370        storage_selector: StorageSelector,
371    ) -> StoreAccountsTiming {
372        let accounts = accounts_to_store.get(storage_selector);
373
374        db.store_accounts_frozen(
375            (
376                self.slot(),
377                accounts,
378                INCLUDE_SLOT_IN_HASH_IRRELEVANT_APPEND_VEC_OPERATION,
379                accounts_to_store.slot,
380            ),
381            None::<Vec<Hash>>,
382            self.append_vec(),
383            None,
384            StoreReclaims::Ignore,
385        )
386    }
387}
388
389/// specifies how to return zero lamport accounts from a load
390#[derive(Clone, Copy)]
391enum LoadZeroLamports {
392    /// return None if loaded account has zero lamports
393    None,
394    /// return Some(account with zero lamports) if loaded account has zero lamports
395    /// This used to be the only behavior.
396    /// Note that this is non-deterministic if clean is running asynchronously.
397    /// If a zero lamport account exists in the index, then Some is returned.
398    /// Once it is cleaned from the index, None is returned.
399    #[cfg(test)]
400    SomeWithZeroLamportAccountForTests,
401}
402
403#[derive(Debug)]
404struct AncientSlotPubkeysInner {
405    pubkeys: HashSet<Pubkey>,
406    slot: Slot,
407}
408
409#[derive(Debug, Default)]
410struct AncientSlotPubkeys {
411    inner: Option<AncientSlotPubkeysInner>,
412}
413
414impl AncientSlotPubkeys {
415    /// All accounts in 'slot' will be moved to 'current_ancient'
416    /// If 'slot' is different than the 'current_ancient'.slot, then an account in 'slot' may ALREADY be in the current ancient append vec.
417    /// In that case, we need to unref the pubkey because it will now only be referenced from 'current_ancient'.slot and no longer from 'slot'.
418    /// 'self' is also changed to accumulate the pubkeys that now exist in 'current_ancient'
419    /// When 'slot' differs from the previous inner slot, then we have moved to a new ancient append vec, and inner.pubkeys gets reset to the
420    ///  pubkeys in the new 'current_ancient'.append_vec
421    fn maybe_unref_accounts_already_in_ancient(
422        &mut self,
423        slot: Slot,
424        db: &AccountsDb,
425        current_ancient: &CurrentAncientAppendVec,
426        to_store: &AccountsToStore,
427    ) {
428        if slot != current_ancient.slot() {
429            // we are taking accounts from 'slot' and putting them into 'current_ancient.slot()'
430            // StorageSelector::Primary here because only the accounts that are moving from 'slot' to 'current_ancient.slot()'
431            // Any overflow accounts will get written into a new append vec AT 'slot', so they don't need to be unrefed
432            let accounts = to_store.get(StorageSelector::Primary);
433            if Some(current_ancient.slot()) != self.inner.as_ref().map(|ap| ap.slot) {
434                let pubkeys = current_ancient
435                    .append_vec()
436                    .accounts
437                    .account_iter()
438                    .map(|account| *account.pubkey())
439                    .collect::<HashSet<_>>();
440                self.inner = Some(AncientSlotPubkeysInner {
441                    pubkeys,
442                    slot: current_ancient.slot(),
443                });
444            }
445            // accounts in 'slot' but ALSO already in the ancient append vec at a different slot need to be unref'd since 'slot' is going away
446            // unwrap cannot fail because the code above will cause us to set it to Some(...) if it is None
447            db.unref_accounts_already_in_storage(
448                accounts,
449                self.inner.as_mut().map(|p| &mut p.pubkeys).unwrap(),
450            );
451        }
452    }
453}
454
455#[derive(Debug)]
456pub(crate) struct ShrinkCollect<'a, T: ShrinkCollectRefs<'a>> {
457    pub(crate) slot: Slot,
458    pub(crate) capacity: u64,
459    pub(crate) aligned_total_bytes: u64,
460    pub(crate) unrefed_pubkeys: Vec<&'a Pubkey>,
461    pub(crate) alive_accounts: T,
462    /// total size in storage of all alive accounts
463    pub(crate) alive_total_bytes: usize,
464    pub(crate) total_starting_accounts: usize,
465    /// true if all alive accounts are zero lamports
466    pub(crate) all_are_zero_lamports: bool,
467}
468
469pub const ACCOUNTS_DB_CONFIG_FOR_TESTING: AccountsDbConfig = AccountsDbConfig {
470    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_TESTING),
471    accounts_hash_cache_path: None,
472    filler_accounts_config: FillerAccountsConfig::const_default(),
473    write_cache_limit_bytes: None,
474    ancient_append_vec_offset: None,
475    skip_initial_hash_calc: false,
476    exhaustively_verify_refcounts: false,
477    assert_stakes_cache_consistency: true,
478    create_ancient_storage: CreateAncientStorage::Append,
479};
480pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig {
481    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS),
482    accounts_hash_cache_path: None,
483    filler_accounts_config: FillerAccountsConfig::const_default(),
484    write_cache_limit_bytes: None,
485    ancient_append_vec_offset: None,
486    skip_initial_hash_calc: false,
487    exhaustively_verify_refcounts: false,
488    assert_stakes_cache_consistency: false,
489    create_ancient_storage: CreateAncientStorage::Append,
490};
491
492pub type BinnedHashData = Vec<Vec<CalculateHashIntermediate>>;
493
494struct LoadAccountsIndexForShrink<'a, T: ShrinkCollectRefs<'a>> {
495    /// all alive accounts
496    alive_accounts: T,
497    /// pubkeys that were unref'd in the accounts index because they were dead
498    unrefed_pubkeys: Vec<&'a Pubkey>,
499    /// true if all alive accounts are zero lamport accounts
500    all_are_zero_lamports: bool,
501}
502
503pub struct GetUniqueAccountsResult<'a> {
504    pub stored_accounts: Vec<StoredAccountMeta<'a>>,
505    pub capacity: u64,
506}
507
508pub struct AccountsAddRootTiming {
509    pub index_us: u64,
510    pub cache_us: u64,
511    pub store_us: u64,
512}
513
514#[derive(Debug, Clone, Copy)]
515pub struct FillerAccountsConfig {
516    /// Number of filler accounts
517    pub count: usize,
518    /// Data size per account, in bytes
519    pub size: usize,
520}
521
522impl FillerAccountsConfig {
523    pub const fn const_default() -> Self {
524        Self { count: 0, size: 0 }
525    }
526}
527
528impl Default for FillerAccountsConfig {
529    fn default() -> Self {
530        Self::const_default()
531    }
532}
533
534const ANCIENT_APPEND_VEC_DEFAULT_OFFSET: Option<i64> = Some(-10_000);
535
536#[derive(Debug, Default, Clone)]
537pub struct AccountsDbConfig {
538    pub index: Option<AccountsIndexConfig>,
539    pub accounts_hash_cache_path: Option<PathBuf>,
540    pub filler_accounts_config: FillerAccountsConfig,
541    pub write_cache_limit_bytes: Option<u64>,
542    /// if None, ancient append vecs are set to ANCIENT_APPEND_VEC_DEFAULT_OFFSET
543    /// Some(offset) means include slots up to (max_slot - (slots_per_epoch - 'offset'))
544    pub ancient_append_vec_offset: Option<i64>,
545    pub skip_initial_hash_calc: bool,
546    pub exhaustively_verify_refcounts: bool,
547    /// when stakes cache consistency check occurs, assert that cached accounts match accounts db
548    pub assert_stakes_cache_consistency: bool,
549    /// how to create ancient storages
550    pub create_ancient_storage: CreateAncientStorage,
551}
552
553#[cfg(not(test))]
554const ABSURD_CONSECUTIVE_FAILED_ITERATIONS: usize = 100;
555
556#[derive(Debug, Clone, Copy)]
557pub enum AccountShrinkThreshold {
558    /// Measure the total space sparseness across all candidates
559    /// And select the candidates by using the top sparse account storage entries to shrink.
560    /// The value is the overall shrink threshold measured as ratio of the total live bytes
561    /// over the total bytes.
562    TotalSpace { shrink_ratio: f64 },
563    /// Use the following option to shrink all stores whose alive ratio is below
564    /// the specified threshold.
565    IndividualStore { shrink_ratio: f64 },
566}
567pub const DEFAULT_ACCOUNTS_SHRINK_OPTIMIZE_TOTAL_SPACE: bool = true;
568pub const DEFAULT_ACCOUNTS_SHRINK_RATIO: f64 = 0.80;
569// The default extra account space in percentage from the ideal target
570const DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION: AccountShrinkThreshold =
571    AccountShrinkThreshold::TotalSpace {
572        shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_RATIO,
573    };
574
575impl Default for AccountShrinkThreshold {
576    fn default() -> AccountShrinkThreshold {
577        DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION
578    }
579}
580
581pub enum ScanStorageResult<R, B> {
582    Cached(Vec<R>),
583    Stored(B),
584}
585
586#[derive(Debug, Default)]
587pub struct IndexGenerationInfo {
588    pub accounts_data_len: u64,
589    pub rent_paying_accounts_by_partition: RentPayingAccountsByPartition,
590}
591
592#[derive(Debug, Default)]
593struct SlotIndexGenerationInfo {
594    insert_time_us: u64,
595    num_accounts: u64,
596    num_accounts_rent_paying: usize,
597    accounts_data_len: u64,
598    amount_to_top_off_rent: u64,
599    rent_paying_accounts_by_partition: Vec<Pubkey>,
600}
601
602#[derive(Default, Debug)]
603struct GenerateIndexTimings {
604    pub index_time: u64,
605    pub scan_time: u64,
606    pub insertion_time_us: u64,
607    pub min_bin_size: usize,
608    pub max_bin_size: usize,
609    pub total_items: usize,
610    pub storage_size_accounts_map_us: u64,
611    pub storage_size_storages_us: u64,
612    pub storage_size_accounts_map_flatten_us: u64,
613    pub index_flush_us: u64,
614    pub rent_paying: AtomicUsize,
615    pub amount_to_top_off_rent: AtomicU64,
616    pub total_duplicates: u64,
617    pub accounts_data_len_dedup_time_us: u64,
618}
619
620#[derive(Default, Debug, PartialEq, Eq)]
621struct StorageSizeAndCount {
622    pub stored_size: usize,
623    pub count: usize,
624}
625type StorageSizeAndCountMap = DashMap<AppendVecId, StorageSizeAndCount>;
626
627impl GenerateIndexTimings {
628    pub fn report(&self) {
629        datapoint_info!(
630            "generate_index",
631            // we cannot accurately measure index insertion time because of many threads and lock contention
632            ("total_us", self.index_time, i64),
633            ("scan_stores_us", self.scan_time, i64),
634            ("insertion_time_us", self.insertion_time_us, i64),
635            ("min_bin_size", self.min_bin_size as i64, i64),
636            ("max_bin_size", self.max_bin_size as i64, i64),
637            (
638                "storage_size_accounts_map_us",
639                self.storage_size_accounts_map_us as i64,
640                i64
641            ),
642            (
643                "storage_size_storages_us",
644                self.storage_size_storages_us as i64,
645                i64
646            ),
647            (
648                "storage_size_accounts_map_flatten_us",
649                self.storage_size_accounts_map_flatten_us as i64,
650                i64
651            ),
652            ("index_flush_us", self.index_flush_us as i64, i64),
653            (
654                "total_rent_paying",
655                self.rent_paying.load(Ordering::Relaxed) as i64,
656                i64
657            ),
658            (
659                "amount_to_top_off_rent",
660                self.amount_to_top_off_rent.load(Ordering::Relaxed) as i64,
661                i64
662            ),
663            (
664                "total_items_with_duplicates",
665                self.total_duplicates as i64,
666                i64
667            ),
668            ("total_items", self.total_items as i64, i64),
669            (
670                "accounts_data_len_dedup_time_us",
671                self.accounts_data_len_dedup_time_us as i64,
672                i64
673            ),
674        );
675    }
676}
677
678impl IndexValue for AccountInfo {}
679
680impl ZeroLamport for AccountSharedData {
681    fn is_zero_lamport(&self) -> bool {
682        self.lamports() == 0
683    }
684}
685
686impl ZeroLamport for Account {
687    fn is_zero_lamport(&self) -> bool {
688        self.lamports() == 0
689    }
690}
691
692struct MultiThreadProgress<'a> {
693    last_update: Instant,
694    my_last_report_count: u64,
695    total_count: &'a AtomicU64,
696    report_delay_secs: u64,
697    first_caller: bool,
698    ultimate_count: u64,
699}
700
701impl<'a> MultiThreadProgress<'a> {
702    fn new(total_count: &'a AtomicU64, report_delay_secs: u64, ultimate_count: u64) -> Self {
703        Self {
704            last_update: Instant::now(),
705            my_last_report_count: 0,
706            total_count,
707            report_delay_secs,
708            first_caller: false,
709            ultimate_count,
710        }
711    }
712    fn report(&mut self, my_current_count: u64) {
713        let now = Instant::now();
714        if now.duration_since(self.last_update).as_secs() >= self.report_delay_secs {
715            let my_total_newly_processed_slots_since_last_report =
716                my_current_count - self.my_last_report_count;
717
718            self.my_last_report_count = my_current_count;
719            let previous_total_processed_slots_across_all_threads = self.total_count.fetch_add(
720                my_total_newly_processed_slots_since_last_report,
721                Ordering::Relaxed,
722            );
723            self.first_caller =
724                self.first_caller || 0 == previous_total_processed_slots_across_all_threads;
725            if self.first_caller {
726                info!(
727                    "generating index: {}/{} slots...",
728                    previous_total_processed_slots_across_all_threads
729                        + my_total_newly_processed_slots_since_last_report,
730                    self.ultimate_count
731                );
732            }
733            self.last_update = now;
734        }
735    }
736}
737
738/// An offset into the AccountsDb::storage vector
739pub type AtomicAppendVecId = AtomicU32;
740pub type AppendVecId = u32;
741
742type AccountSlots = HashMap<Pubkey, HashSet<Slot>>;
743type AppendVecOffsets = HashMap<AppendVecId, HashSet<usize>>;
744type ReclaimResult = (AccountSlots, AppendVecOffsets);
745type PubkeysRemovedFromAccountsIndex = HashSet<Pubkey>;
746type ShrinkCandidates = HashMap<Slot, Arc<AccountStorageEntry>>;
747
748trait Versioned {
749    fn version(&self) -> u64;
750}
751
752impl Versioned for (u64, Hash) {
753    fn version(&self) -> u64 {
754        self.0
755    }
756}
757
758impl Versioned for (u64, AccountInfo) {
759    fn version(&self) -> u64 {
760        self.0
761    }
762}
763
764// Some hints for applicability of additional sanity checks for the do_load fast-path;
765// Slower fallback code path will be taken if the fast path has failed over the retry
766// threshold, regardless of these hints. Also, load cannot fail not-deterministically
767// even under very rare circumstances, unlike previously did allow.
768#[derive(Clone, Copy, Debug, PartialEq, Eq)]
769pub enum LoadHint {
770    // Caller hints that it's loading transactions for a block which is
771    // descended from the current root, and at the tip of its fork.
772    // Thereby, further this assumes AccountIndex::max_root should not increase
773    // during this load, meaning there should be no squash.
774    // Overall, this enables us to assert!() strictly while running the fast-path for
775    // account loading, while maintaining the determinism of account loading and resultant
776    // transaction execution thereof.
777    FixedMaxRoot,
778    // Caller can't hint the above safety assumption. Generally RPC and miscellaneous
779    // other call-site falls into this category. The likelihood of slower path is slightly
780    // increased as well.
781    Unspecified,
782}
783
784#[derive(Debug)]
785pub enum LoadedAccountAccessor<'a> {
786    // StoredAccountMeta can't be held directly here due to its lifetime dependency to
787    // AccountStorageEntry
788    Stored(Option<(Arc<AccountStorageEntry>, usize)>),
789    // None value in Cached variant means the cache was flushed
790    Cached(Option<Cow<'a, CachedAccount>>),
791}
792
793mod geyser_plugin_utils;
794
795impl<'a> LoadedAccountAccessor<'a> {
796    fn check_and_get_loaded_account(&mut self) -> LoadedAccount {
797        // all of these following .expect() and .unwrap() are like serious logic errors,
798        // ideal for representing this as rust type system....
799
800        match self {
801            LoadedAccountAccessor::Cached(None) | LoadedAccountAccessor::Stored(None) => {
802                panic!("Should have already been taken care of when creating this LoadedAccountAccessor");
803            }
804            LoadedAccountAccessor::Cached(Some(_cached_account)) => {
805                // Cached(Some(x)) variant always produces `Some` for get_loaded_account() since
806                // it just returns the inner `x` without additional fetches
807                self.get_loaded_account().unwrap()
808            }
809            LoadedAccountAccessor::Stored(Some(_maybe_storage_entry)) => {
810                // If we do find the storage entry, we can guarantee that the storage entry is
811                // safe to read from because we grabbed a reference to the storage entry while it
812                // was still in the storage map. This means even if the storage entry is removed
813                // from the storage map after we grabbed the storage entry, the recycler should not
814                // reset the storage entry until we drop the reference to the storage entry.
815                self.get_loaded_account()
816                    .expect("If a storage entry was found in the storage map, it must not have been reset yet")
817            }
818        }
819    }
820
821    fn get_loaded_account(&mut self) -> Option<LoadedAccount> {
822        match self {
823            LoadedAccountAccessor::Cached(cached_account) => {
824                let cached_account: Cow<'a, CachedAccount> = cached_account.take().expect(
825                    "Cache flushed/purged should be handled before trying to fetch account",
826                );
827                Some(LoadedAccount::Cached(cached_account))
828            }
829            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
830                // storage entry may not be present if slot was cleaned up in
831                // between reading the accounts index and calling this function to
832                // get account meta from the storage entry here
833                maybe_storage_entry
834                    .as_ref()
835                    .and_then(|(storage_entry, offset)| {
836                        storage_entry
837                            .get_stored_account_meta(*offset)
838                            .map(LoadedAccount::Stored)
839                    })
840            }
841        }
842    }
843
844    fn account_matches_owners(&self, owners: &[&Pubkey]) -> Result<usize, MatchAccountOwnerError> {
845        match self {
846            LoadedAccountAccessor::Cached(cached_account) => cached_account
847                .as_ref()
848                .and_then(|cached_account| {
849                    if cached_account.account.is_zero_lamport() {
850                        None
851                    } else {
852                        owners
853                            .iter()
854                            .position(|entry| &cached_account.account.owner() == entry)
855                    }
856                })
857                .ok_or(MatchAccountOwnerError::NoMatch),
858            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
859                // storage entry may not be present if slot was cleaned up in
860                // between reading the accounts index and calling this function to
861                // get account meta from the storage entry here
862                maybe_storage_entry
863                    .as_ref()
864                    .map(|(storage_entry, offset)| {
865                        storage_entry
866                            .accounts
867                            .account_matches_owners(*offset, owners)
868                    })
869                    .unwrap_or(Err(MatchAccountOwnerError::UnableToLoad))
870            }
871        }
872    }
873}
874
875pub enum LoadedAccount<'a> {
876    Stored(StoredAccountMeta<'a>),
877    Cached(Cow<'a, CachedAccount>),
878}
879
880impl<'a> LoadedAccount<'a> {
881    pub fn loaded_hash(&self) -> Hash {
882        match self {
883            LoadedAccount::Stored(stored_account_meta) => *stored_account_meta.hash,
884            LoadedAccount::Cached(cached_account) => cached_account.hash(),
885        }
886    }
887
888    pub fn pubkey(&self) -> &Pubkey {
889        match self {
890            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.pubkey(),
891            LoadedAccount::Cached(cached_account) => cached_account.pubkey(),
892        }
893    }
894
895    pub fn compute_hash(
896        &self,
897        slot: Slot,
898        pubkey: &Pubkey,
899        include_slot: IncludeSlotInHash,
900    ) -> Hash {
901        match self {
902            LoadedAccount::Stored(stored_account_meta) => AccountsDb::hash_account(
903                slot,
904                stored_account_meta,
905                stored_account_meta.pubkey(),
906                include_slot,
907            ),
908            LoadedAccount::Cached(cached_account) => {
909                AccountsDb::hash_account(slot, &cached_account.account, pubkey, include_slot)
910            }
911        }
912    }
913
914    pub fn take_account(self) -> AccountSharedData {
915        match self {
916            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.clone_account(),
917            LoadedAccount::Cached(cached_account) => match cached_account {
918                Cow::Owned(cached_account) => cached_account.account.clone(),
919                Cow::Borrowed(cached_account) => cached_account.account.clone(),
920            },
921        }
922    }
923
924    pub fn is_cached(&self) -> bool {
925        match self {
926            LoadedAccount::Stored(_) => false,
927            LoadedAccount::Cached(_) => true,
928        }
929    }
930}
931
932impl<'a> ReadableAccount for LoadedAccount<'a> {
933    fn lamports(&self) -> u64 {
934        match self {
935            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.account_meta.lamports,
936            LoadedAccount::Cached(cached_account) => cached_account.account.lamports(),
937        }
938    }
939
940    fn data(&self) -> &[u8] {
941        match self {
942            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.data,
943            LoadedAccount::Cached(cached_account) => cached_account.account.data(),
944        }
945    }
946    fn owner(&self) -> &Pubkey {
947        match self {
948            LoadedAccount::Stored(stored_account_meta) => &stored_account_meta.account_meta.owner,
949            LoadedAccount::Cached(cached_account) => cached_account.account.owner(),
950        }
951    }
952    fn executable(&self) -> bool {
953        match self {
954            LoadedAccount::Stored(stored_account_meta) => {
955                stored_account_meta.account_meta.executable
956            }
957            LoadedAccount::Cached(cached_account) => cached_account.account.executable(),
958        }
959    }
960    fn rent_epoch(&self) -> Epoch {
961        match self {
962            LoadedAccount::Stored(stored_account_meta) => {
963                stored_account_meta.account_meta.rent_epoch
964            }
965            LoadedAccount::Cached(cached_account) => cached_account.account.rent_epoch(),
966        }
967    }
968    fn to_account_shared_data(&self) -> AccountSharedData {
969        match self {
970            LoadedAccount::Stored(_stored_account_meta) => AccountSharedData::create(
971                self.lamports(),
972                self.data().to_vec(),
973                *self.owner(),
974                self.executable(),
975                self.rent_epoch(),
976            ),
977            // clone here to prevent data copy
978            LoadedAccount::Cached(cached_account) => cached_account.account.clone(),
979        }
980    }
981}
982
983#[derive(Debug)]
984pub enum BankHashVerificationError {
985    MismatchedAccountHash,
986    MismatchedBankHash,
987    MissingBankHash,
988    MismatchedTotalLamports(u64, u64),
989}
990
991#[derive(Default)]
992struct CleanKeyTimings {
993    collect_delta_keys_us: u64,
994    delta_insert_us: u64,
995    hashset_to_vec_us: u64,
996    dirty_store_processing_us: u64,
997    delta_key_count: u64,
998    dirty_pubkeys_count: u64,
999    oldest_dirty_slot: Slot,
1000    /// number of ancient append vecs that were scanned because they were dirty when clean started
1001    dirty_ancient_stores: usize,
1002}
1003
1004/// Persistent storage structure holding the accounts
1005#[derive(Debug)]
1006pub struct AccountStorageEntry {
1007    pub(crate) id: AtomicAppendVecId,
1008
1009    pub(crate) slot: AtomicU64,
1010
1011    /// storage holding the accounts
1012    pub(crate) accounts: AccountsFile,
1013
1014    /// Keeps track of the number of accounts stored in a specific AppendVec.
1015    ///  This is periodically checked to reuse the stores that do not have
1016    ///  any accounts in it
1017    /// status corresponding to the storage, lets us know that
1018    ///  the append_vec, once maxed out, then emptied, can be reclaimed
1019    count_and_status: RwLock<(usize, AccountStorageStatus)>,
1020
1021    /// This is the total number of accounts stored ever since initialized to keep
1022    /// track of lifetime count of all store operations. And this differs from
1023    /// count_and_status in that this field won't be decremented.
1024    ///
1025    /// This is used as a rough estimate for slot shrinking. As such a relaxed
1026    /// use case, this value ARE NOT strictly synchronized with count_and_status!
1027    approx_store_count: AtomicUsize,
1028
1029    alive_bytes: AtomicUsize,
1030}
1031
1032impl AccountStorageEntry {
1033    pub fn new(path: &Path, slot: Slot, id: AppendVecId, file_size: u64) -> Self {
1034        let tail = AccountsFile::file_name(slot, id);
1035        let path = Path::new(path).join(tail);
1036        let accounts = AccountsFile::AppendVec(AppendVec::new(&path, true, file_size as usize));
1037
1038        Self {
1039            id: AtomicAppendVecId::new(id),
1040            slot: AtomicU64::new(slot),
1041            accounts,
1042            count_and_status: RwLock::new((0, AccountStorageStatus::Available)),
1043            approx_store_count: AtomicUsize::new(0),
1044            alive_bytes: AtomicUsize::new(0),
1045        }
1046    }
1047
1048    pub(crate) fn new_existing(
1049        slot: Slot,
1050        id: AppendVecId,
1051        accounts: AccountsFile,
1052        num_accounts: usize,
1053    ) -> Self {
1054        Self {
1055            id: AtomicAppendVecId::new(id),
1056            slot: AtomicU64::new(slot),
1057            accounts,
1058            count_and_status: RwLock::new((0, AccountStorageStatus::Available)),
1059            approx_store_count: AtomicUsize::new(num_accounts),
1060            alive_bytes: AtomicUsize::new(0),
1061        }
1062    }
1063
1064    pub fn set_status(&self, mut status: AccountStorageStatus) {
1065        let mut count_and_status = self.count_and_status.write().unwrap();
1066
1067        let count = count_and_status.0;
1068
1069        if status == AccountStorageStatus::Full && count == 0 {
1070            // this case arises when the append_vec is full (store_ptrs fails),
1071            //  but all accounts have already been removed from the storage
1072            //
1073            // the only time it's safe to call reset() on an append_vec is when
1074            //  every account has been removed
1075            //          **and**
1076            //  the append_vec has previously been completely full
1077            //
1078            self.accounts.reset();
1079            status = AccountStorageStatus::Available;
1080        }
1081
1082        *count_and_status = (count, status);
1083    }
1084
1085    pub fn recycle(&self, slot: Slot, id: AppendVecId) {
1086        let mut count_and_status = self.count_and_status.write().unwrap();
1087        self.accounts.reset();
1088        *count_and_status = (0, AccountStorageStatus::Available);
1089        self.slot.store(slot, Ordering::Release);
1090        self.id.store(id, Ordering::Release);
1091        self.approx_store_count.store(0, Ordering::Relaxed);
1092        self.alive_bytes.store(0, Ordering::Release);
1093    }
1094
1095    pub fn status(&self) -> AccountStorageStatus {
1096        self.count_and_status.read().unwrap().1
1097    }
1098
1099    pub fn count(&self) -> usize {
1100        self.count_and_status.read().unwrap().0
1101    }
1102
1103    pub fn approx_stored_count(&self) -> usize {
1104        self.approx_store_count.load(Ordering::Relaxed)
1105    }
1106
1107    pub fn alive_bytes(&self) -> usize {
1108        self.alive_bytes.load(Ordering::SeqCst)
1109    }
1110
1111    pub fn written_bytes(&self) -> u64 {
1112        self.accounts.len() as u64
1113    }
1114
1115    pub fn capacity(&self) -> u64 {
1116        self.accounts.capacity()
1117    }
1118
1119    pub fn has_accounts(&self) -> bool {
1120        self.count() > 0
1121    }
1122
1123    pub fn slot(&self) -> Slot {
1124        self.slot.load(Ordering::Acquire)
1125    }
1126
1127    pub fn append_vec_id(&self) -> AppendVecId {
1128        self.id.load(Ordering::Acquire)
1129    }
1130
1131    pub fn flush(&self) -> Result<(), IoError> {
1132        self.accounts.flush()
1133    }
1134
1135    fn get_stored_account_meta(&self, offset: usize) -> Option<StoredAccountMeta> {
1136        Some(self.accounts.get_account(offset)?.0)
1137    }
1138
1139    fn add_account(&self, num_bytes: usize) {
1140        let mut count_and_status = self.count_and_status.write().unwrap();
1141        *count_and_status = (count_and_status.0 + 1, count_and_status.1);
1142        self.approx_store_count.fetch_add(1, Ordering::Relaxed);
1143        self.alive_bytes.fetch_add(num_bytes, Ordering::SeqCst);
1144    }
1145
1146    fn try_available(&self) -> bool {
1147        let mut count_and_status = self.count_and_status.write().unwrap();
1148        let (count, status) = *count_and_status;
1149
1150        if status == AccountStorageStatus::Available {
1151            *count_and_status = (count, AccountStorageStatus::Candidate);
1152            true
1153        } else {
1154            false
1155        }
1156    }
1157
1158    pub fn all_accounts(&self) -> Vec<StoredAccountMeta> {
1159        self.accounts.accounts(0)
1160    }
1161
1162    fn remove_account(&self, num_bytes: usize, reset_accounts: bool) -> usize {
1163        let mut count_and_status = self.count_and_status.write().unwrap();
1164        let (mut count, mut status) = *count_and_status;
1165
1166        if count == 1 && status == AccountStorageStatus::Full && reset_accounts {
1167            // this case arises when we remove the last account from the
1168            //  storage, but we've learned from previous write attempts that
1169            //  the storage is full
1170            //
1171            // the only time it's safe to call reset() on an append_vec is when
1172            //  every account has been removed
1173            //          **and**
1174            //  the append_vec has previously been completely full
1175            //
1176            // otherwise, the storage may be in flight with a store()
1177            //   call
1178            self.accounts.reset();
1179            status = AccountStorageStatus::Available;
1180        }
1181
1182        // Some code path is removing accounts too many; this may result in an
1183        // unintended reveal of old state for unrelated accounts.
1184        assert!(
1185            count > 0,
1186            "double remove of account in slot: {}/store: {}!!",
1187            self.slot(),
1188            self.append_vec_id(),
1189        );
1190
1191        self.alive_bytes.fetch_sub(num_bytes, Ordering::SeqCst);
1192        count -= 1;
1193        *count_and_status = (count, status);
1194        count
1195    }
1196
1197    pub fn get_path(&self) -> PathBuf {
1198        self.accounts.get_path()
1199    }
1200}
1201
1202pub fn get_temp_accounts_paths(count: u32) -> IoResult<(Vec<TempDir>, Vec<PathBuf>)> {
1203    let temp_dirs: IoResult<Vec<TempDir>> = (0..count).map(|_| TempDir::new()).collect();
1204    let temp_dirs = temp_dirs?;
1205
1206    let paths: IoResult<Vec<_>> = temp_dirs
1207        .iter()
1208        .map(|temp_dir| {
1209            create_accounts_run_and_snapshot_dirs(temp_dir).map(|(run_dir, _snapshot_dir)| run_dir)
1210        })
1211        .collect();
1212    let paths = paths?;
1213    Ok((temp_dirs, paths))
1214}
1215
1216#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, Eq, AbiExample)]
1217pub struct BankHashStats {
1218    pub num_updated_accounts: u64,
1219    pub num_removed_accounts: u64,
1220    pub num_lamports_stored: u64,
1221    pub total_data_len: u64,
1222    pub num_executable_accounts: u64,
1223}
1224
1225impl BankHashStats {
1226    pub fn update<T: ReadableAccount + ZeroLamport>(&mut self, account: &T) {
1227        if account.is_zero_lamport() {
1228            self.num_removed_accounts += 1;
1229        } else {
1230            self.num_updated_accounts += 1;
1231        }
1232        self.total_data_len = self
1233            .total_data_len
1234            .wrapping_add(account.data().len() as u64);
1235        if account.executable() {
1236            self.num_executable_accounts += 1;
1237        }
1238        self.num_lamports_stored = self.num_lamports_stored.wrapping_add(account.lamports());
1239    }
1240
1241    pub fn accumulate(&mut self, other: &BankHashStats) {
1242        self.num_updated_accounts += other.num_updated_accounts;
1243        self.num_removed_accounts += other.num_removed_accounts;
1244        self.total_data_len = self.total_data_len.wrapping_add(other.total_data_len);
1245        self.num_lamports_stored = self
1246            .num_lamports_stored
1247            .wrapping_add(other.num_lamports_stored);
1248        self.num_executable_accounts += other.num_executable_accounts;
1249    }
1250}
1251
1252#[derive(Default, Debug)]
1253pub struct StoreAccountsTiming {
1254    store_accounts_elapsed: u64,
1255    update_index_elapsed: u64,
1256    handle_reclaims_elapsed: u64,
1257}
1258
1259impl StoreAccountsTiming {
1260    fn accumulate(&mut self, other: &Self) {
1261        self.store_accounts_elapsed += other.store_accounts_elapsed;
1262        self.update_index_elapsed += other.update_index_elapsed;
1263        self.handle_reclaims_elapsed += other.handle_reclaims_elapsed;
1264    }
1265}
1266
1267#[derive(Debug, Default)]
1268struct RecycleStores {
1269    entries: Vec<(Instant, Arc<AccountStorageEntry>)>,
1270    total_bytes: u64,
1271}
1272
1273// 30 min should be enough to be certain there won't be any prospective recycle uses for given
1274// store entry
1275// That's because it already processed ~2500 slots and ~25 passes of AccountsBackgroundService
1276pub const EXPIRATION_TTL_SECONDS: u64 = 1800;
1277
1278impl RecycleStores {
1279    fn add_entry(&mut self, new_entry: Arc<AccountStorageEntry>) {
1280        self.total_bytes += new_entry.capacity();
1281        self.entries.push((Instant::now(), new_entry))
1282    }
1283
1284    fn iter(&self) -> std::slice::Iter<(Instant, Arc<AccountStorageEntry>)> {
1285        self.entries.iter()
1286    }
1287
1288    fn add_entries(&mut self, new_entries: Vec<Arc<AccountStorageEntry>>) {
1289        let now = Instant::now();
1290        for new_entry in new_entries {
1291            self.total_bytes += new_entry.capacity();
1292            self.entries.push((now, new_entry));
1293        }
1294    }
1295
1296    fn expire_old_entries(&mut self) -> Vec<Arc<AccountStorageEntry>> {
1297        let mut expired = vec![];
1298        let now = Instant::now();
1299        let mut expired_bytes = 0;
1300        self.entries.retain(|(recycled_time, entry)| {
1301            if now.duration_since(*recycled_time).as_secs() > EXPIRATION_TTL_SECONDS {
1302                if Arc::strong_count(entry) >= 2 {
1303                    warn!(
1304                        "Expiring still in-use recycled StorageEntry anyway...: id: {} slot: {}",
1305                        entry.append_vec_id(),
1306                        entry.slot(),
1307                    );
1308                }
1309                expired_bytes += entry.capacity();
1310                expired.push(entry.clone());
1311                false
1312            } else {
1313                true
1314            }
1315        });
1316
1317        self.total_bytes -= expired_bytes;
1318
1319        expired
1320    }
1321
1322    fn remove_entry(&mut self, index: usize) -> Arc<AccountStorageEntry> {
1323        let (_added_time, removed_entry) = self.entries.swap_remove(index);
1324        self.total_bytes -= removed_entry.capacity();
1325        removed_entry
1326    }
1327
1328    fn entry_count(&self) -> usize {
1329        self.entries.len()
1330    }
1331
1332    fn total_bytes(&self) -> u64 {
1333        self.total_bytes
1334    }
1335}
1336
1337/// Removing unrooted slots in Accounts Background Service needs to be synchronized with flushing
1338/// slots from the Accounts Cache.  This keeps track of those slots and the Mutex + Condvar for
1339/// synchronization.
1340#[derive(Debug, Default)]
1341struct RemoveUnrootedSlotsSynchronization {
1342    // slots being flushed from the cache or being purged
1343    slots_under_contention: Mutex<HashSet<Slot>>,
1344    signal: Condvar,
1345}
1346
1347type AccountInfoAccountsIndex = AccountsIndex<AccountInfo>;
1348
1349// This structure handles the load/store of the accounts
1350#[derive(Debug)]
1351pub struct AccountsDb {
1352    /// Keeps tracks of index into AppendVec on a per slot basis
1353    pub accounts_index: AccountInfoAccountsIndex,
1354
1355    /// slot that is one epoch older than the highest slot where accounts hash calculation has completed
1356    pub accounts_hash_complete_one_epoch_old: RwLock<Slot>,
1357
1358    /// Some(offset) iff we want to squash old append vecs together into 'ancient append vecs'
1359    /// Some(offset) means for slots up to (max_slot - (slots_per_epoch - 'offset')), put them in ancient append vecs
1360    pub ancient_append_vec_offset: Option<i64>,
1361
1362    /// true iff we want to skip the initial hash calculation on startup
1363    pub skip_initial_hash_calc: bool,
1364
1365    pub(crate) storage: AccountStorage,
1366
1367    /// from AccountsDbConfig
1368    pub(crate) assert_stakes_cache_consistency: bool,
1369
1370    #[allow(dead_code)]
1371    /// from AccountsDbConfig
1372    create_ancient_storage: CreateAncientStorage,
1373
1374    pub accounts_cache: AccountsCache,
1375
1376    write_cache_limit_bytes: Option<u64>,
1377
1378    sender_bg_hasher: Option<Sender<CachedAccount>>,
1379    read_only_accounts_cache: ReadOnlyAccountsCache,
1380
1381    recycle_stores: RwLock<RecycleStores>,
1382
1383    /// distribute the accounts across storage lists
1384    pub next_id: AtomicAppendVecId,
1385
1386    /// Set of shrinkable stores organized by map of slot to append_vec_id
1387    pub shrink_candidate_slots: Mutex<ShrinkCandidates>,
1388
1389    pub(crate) write_version: AtomicU64,
1390
1391    /// Set of storage paths to pick from
1392    pub(crate) paths: Vec<PathBuf>,
1393
1394    full_accounts_hash_cache_path: PathBuf,
1395    incremental_accounts_hash_cache_path: PathBuf,
1396
1397    // used by tests
1398    // holds this until we are dropped
1399    #[allow(dead_code)]
1400    temp_accounts_hash_cache_path: Option<TempDir>,
1401
1402    pub shrink_paths: RwLock<Option<Vec<PathBuf>>>,
1403
1404    /// Directory of paths this accounts_db needs to hold/remove
1405    #[allow(dead_code)]
1406    pub(crate) temp_paths: Option<Vec<TempDir>>,
1407
1408    /// Starting file size of appendvecs
1409    file_size: u64,
1410
1411    /// Thread pool used for par_iter
1412    pub thread_pool: ThreadPool,
1413
1414    pub thread_pool_clean: ThreadPool,
1415
1416    accounts_delta_hashes: Mutex<HashMap<Slot, AccountsDeltaHash>>,
1417    accounts_hashes: Mutex<HashMap<Slot, AccountsHash>>,
1418    bank_hash_stats: Mutex<HashMap<Slot, BankHashStats>>,
1419
1420    pub stats: AccountsStats,
1421
1422    clean_accounts_stats: CleanAccountsStats,
1423
1424    // Stats for purges called outside of clean_accounts()
1425    external_purge_slots_stats: PurgeStats,
1426
1427    pub(crate) shrink_stats: ShrinkStats,
1428
1429    pub(crate) shrink_ancient_stats: ShrinkAncientStats,
1430
1431    pub cluster_type: Option<ClusterType>,
1432
1433    pub account_indexes: AccountSecondaryIndexes,
1434
1435    /// Set of unique keys per slot which is used
1436    /// to drive clean_accounts
1437    /// Generated by calculate_accounts_delta_hash
1438    uncleaned_pubkeys: DashMap<Slot, Vec<Pubkey>>,
1439
1440    #[cfg(test)]
1441    load_delay: u64,
1442
1443    #[cfg(test)]
1444    load_limit: AtomicU64,
1445
1446    /// true if drop_callback is attached to the bank.
1447    is_bank_drop_callback_enabled: AtomicBool,
1448
1449    /// Set of slots currently being flushed by `flush_slot_cache()` or removed
1450    /// by `remove_unrooted_slot()`. Used to ensure `remove_unrooted_slots(slots)`
1451    /// can safely clear the set of unrooted slots `slots`.
1452    remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization,
1453
1454    shrink_ratio: AccountShrinkThreshold,
1455
1456    /// Set of stores which are recently rooted or had accounts removed
1457    /// such that potentially a 0-lamport account update could be present which
1458    /// means we can remove the account from the index entirely.
1459    dirty_stores: DashMap<(Slot, AppendVecId), Arc<AccountStorageEntry>>,
1460
1461    /// Zero-lamport accounts that are *not* purged during clean because they need to stay alive
1462    /// for incremental snapshot support.
1463    zero_lamport_accounts_to_purge_after_full_snapshot: DashSet<(Slot, Pubkey)>,
1464
1465    /// GeyserPlugin accounts update notifier
1466    accounts_update_notifier: Option<AccountsUpdateNotifier>,
1467
1468    filler_accounts_config: FillerAccountsConfig,
1469    pub filler_account_suffix: Option<Pubkey>,
1470
1471    pub(crate) active_stats: ActiveStats,
1472
1473    /// number of filler accounts to add for each slot
1474    pub filler_accounts_per_slot: AtomicU64,
1475
1476    /// number of slots remaining where filler accounts should be added
1477    pub filler_account_slots_remaining: AtomicU64,
1478
1479    pub(crate) verify_accounts_hash_in_bg: VerifyAccountsHashInBackground,
1480
1481    /// Used to disable logging dead slots during removal.
1482    /// allow disabling noisy log
1483    pub(crate) log_dead_slots: AtomicBool,
1484
1485    /// debug feature to scan every append vec and verify refcounts are equal
1486    exhaustively_verify_refcounts: bool,
1487
1488    /// the full accounts hash calculation as of a predetermined block height 'N'
1489    /// to be included in the bank hash at a predetermined block height 'M'
1490    /// The cadence is once per epoch, all nodes calculate a full accounts hash as of a known slot calculated using 'N'
1491    /// Some time later (to allow for slow calculation time), the bank hash at a slot calculated using 'M' includes the full accounts hash.
1492    /// Thus, the state of all accounts on a validator is known to be correct at least once per epoch.
1493    pub epoch_accounts_hash_manager: EpochAccountsHashManager,
1494}
1495
1496#[derive(Debug, Default)]
1497pub struct AccountsStats {
1498    delta_hash_scan_time_total_us: AtomicU64,
1499    delta_hash_accumulate_time_total_us: AtomicU64,
1500    delta_hash_num: AtomicU64,
1501
1502    last_store_report: AtomicInterval,
1503    store_hash_accounts: AtomicU64,
1504    calc_stored_meta: AtomicU64,
1505    store_accounts: AtomicU64,
1506    store_update_index: AtomicU64,
1507    store_handle_reclaims: AtomicU64,
1508    store_append_accounts: AtomicU64,
1509    pub stakes_cache_check_and_store_us: AtomicU64,
1510    store_num_accounts: AtomicU64,
1511    store_total_data: AtomicU64,
1512    recycle_store_count: AtomicU64,
1513    create_store_count: AtomicU64,
1514    store_get_slot_store: AtomicU64,
1515    store_find_existing: AtomicU64,
1516    dropped_stores: AtomicU64,
1517    store_uncleaned_update: AtomicU64,
1518}
1519
1520#[derive(Debug, Default)]
1521pub(crate) struct PurgeStats {
1522    last_report: AtomicInterval,
1523    safety_checks_elapsed: AtomicU64,
1524    remove_cache_elapsed: AtomicU64,
1525    remove_storage_entries_elapsed: AtomicU64,
1526    drop_storage_entries_elapsed: AtomicU64,
1527    num_cached_slots_removed: AtomicUsize,
1528    num_stored_slots_removed: AtomicUsize,
1529    total_removed_storage_entries: AtomicUsize,
1530    total_removed_cached_bytes: AtomicU64,
1531    total_removed_stored_bytes: AtomicU64,
1532    recycle_stores_write_elapsed: AtomicU64,
1533    scan_storages_elapsed: AtomicU64,
1534    purge_accounts_index_elapsed: AtomicU64,
1535    handle_reclaims_elapsed: AtomicU64,
1536}
1537
1538impl PurgeStats {
1539    fn report(&self, metric_name: &'static str, report_interval_ms: Option<u64>) {
1540        let should_report = report_interval_ms
1541            .map(|report_interval_ms| self.last_report.should_update(report_interval_ms))
1542            .unwrap_or(true);
1543
1544        if should_report {
1545            datapoint_info!(
1546                metric_name,
1547                (
1548                    "safety_checks_elapsed",
1549                    self.safety_checks_elapsed.swap(0, Ordering::Relaxed) as i64,
1550                    i64
1551                ),
1552                (
1553                    "remove_cache_elapsed",
1554                    self.remove_cache_elapsed.swap(0, Ordering::Relaxed) as i64,
1555                    i64
1556                ),
1557                (
1558                    "remove_storage_entries_elapsed",
1559                    self.remove_storage_entries_elapsed
1560                        .swap(0, Ordering::Relaxed) as i64,
1561                    i64
1562                ),
1563                (
1564                    "drop_storage_entries_elapsed",
1565                    self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
1566                    i64
1567                ),
1568                (
1569                    "num_cached_slots_removed",
1570                    self.num_cached_slots_removed.swap(0, Ordering::Relaxed) as i64,
1571                    i64
1572                ),
1573                (
1574                    "num_stored_slots_removed",
1575                    self.num_stored_slots_removed.swap(0, Ordering::Relaxed) as i64,
1576                    i64
1577                ),
1578                (
1579                    "total_removed_storage_entries",
1580                    self.total_removed_storage_entries
1581                        .swap(0, Ordering::Relaxed) as i64,
1582                    i64
1583                ),
1584                (
1585                    "total_removed_cached_bytes",
1586                    self.total_removed_cached_bytes.swap(0, Ordering::Relaxed) as i64,
1587                    i64
1588                ),
1589                (
1590                    "total_removed_stored_bytes",
1591                    self.total_removed_stored_bytes.swap(0, Ordering::Relaxed) as i64,
1592                    i64
1593                ),
1594                (
1595                    "recycle_stores_write_elapsed",
1596                    self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
1597                    i64
1598                ),
1599                (
1600                    "scan_storages_elapsed",
1601                    self.scan_storages_elapsed.swap(0, Ordering::Relaxed) as i64,
1602                    i64
1603                ),
1604                (
1605                    "purge_accounts_index_elapsed",
1606                    self.purge_accounts_index_elapsed.swap(0, Ordering::Relaxed) as i64,
1607                    i64
1608                ),
1609                (
1610                    "handle_reclaims_elapsed",
1611                    self.handle_reclaims_elapsed.swap(0, Ordering::Relaxed) as i64,
1612                    i64
1613                ),
1614            );
1615        }
1616    }
1617}
1618
1619/// results from 'split_storages_ancient'
1620#[derive(Debug, Default, PartialEq)]
1621struct SplitAncientStorages {
1622    /// # ancient slots
1623    ancient_slot_count: usize,
1624    /// the specific ancient slots
1625    ancient_slots: Vec<Slot>,
1626    /// lowest slot that is not an ancient append vec
1627    first_non_ancient_slot: Slot,
1628    /// slot # of beginning of first aligned chunk starting from the first non ancient slot
1629    first_chunk_start: Slot,
1630    /// # non-ancient slots to scan
1631    non_ancient_slot_count: usize,
1632    /// # chunks to use to iterate the storages
1633    /// all ancient chunks, the special 0 and last chunks for non-full chunks, and all the 'full' chunks of normal slots
1634    chunk_count: usize,
1635    /// start and end(exclusive) of normal (non-ancient) slots to be scanned
1636    normal_slot_range: Range<Slot>,
1637}
1638
1639impl SplitAncientStorages {
1640    /// When calculating accounts hash, we break the slots/storages into chunks that remain the same during an entire epoch.
1641    /// a slot is in this chunk of slots:
1642    /// start:         (slot / MAX_ITEMS_PER_CHUNK) * MAX_ITEMS_PER_CHUNK
1643    /// end_exclusive: start + MAX_ITEMS_PER_CHUNK
1644    /// So a slot remains in the same chunk whenever it is included in the accounts hash.
1645    /// When the slot gets deleted or gets consumed in an ancient append vec, it will no longer be in its chunk.
1646    /// The results of scanning a chunk of appendvecs can be cached to avoid scanning large amounts of data over and over.
1647    fn new(one_epoch_old_slot: Slot, snapshot_storages: &SortedStorages) -> Self {
1648        let range = snapshot_storages.range();
1649
1650        // any ancient append vecs should definitely be cached
1651        // We need to break the ranges into:
1652        // 1. individual ancient append vecs (may be empty)
1653        // 2. first unevenly divided chunk starting at 1 epoch old slot (may be empty)
1654        // 3. evenly divided full chunks in the middle
1655        // 4. unevenly divided chunk of most recent slots (may be empty)
1656        let ancient_slots = Self::get_ancient_slots(one_epoch_old_slot, snapshot_storages);
1657
1658        let first_non_ancient_slot = ancient_slots
1659            .last()
1660            .map(|last_ancient_slot| last_ancient_slot.saturating_add(1))
1661            .unwrap_or(range.start);
1662        Self::new_with_ancient_info(range, ancient_slots, first_non_ancient_slot)
1663    }
1664
1665    /// return all ancient append vec slots from the early slots referenced by 'snapshot_storages'
1666    fn get_ancient_slots(
1667        one_epoch_old_slot: Slot,
1668        snapshot_storages: &SortedStorages,
1669    ) -> Vec<Slot> {
1670        let range = snapshot_storages.range();
1671        let mut ancient_slots = Vec::default();
1672        for (slot, storage) in snapshot_storages.iter_range(&(range.start..one_epoch_old_slot)) {
1673            if let Some(storage) = storage {
1674                if is_ancient(&storage.accounts) {
1675                    ancient_slots.push(slot);
1676                    continue; // was ancient, keep looking
1677                }
1678                // we found a slot with a non-ancient append vec
1679                break;
1680            }
1681        }
1682        ancient_slots
1683    }
1684
1685    /// create once ancient slots have been identified
1686    /// This is easier to test, removing SortedStorages as a type to deal with here.
1687    fn new_with_ancient_info(
1688        range: &Range<Slot>,
1689        ancient_slots: Vec<Slot>,
1690        first_non_ancient_slot: Slot,
1691    ) -> Self {
1692        if range.is_empty() {
1693            // Corner case mainly for tests, but gives us a consistent base case. Makes more sense to return default here than anything else.
1694            // caller is asking to split for empty set of slots
1695            return SplitAncientStorages::default();
1696        }
1697
1698        let max_slot_inclusive = range.end.saturating_sub(1);
1699        let ancient_slot_count = ancient_slots.len();
1700        let first_chunk_start = ((first_non_ancient_slot + MAX_ITEMS_PER_CHUNK)
1701            / MAX_ITEMS_PER_CHUNK)
1702            * MAX_ITEMS_PER_CHUNK;
1703
1704        let non_ancient_slot_count = (max_slot_inclusive - first_non_ancient_slot + 1) as usize;
1705
1706        let normal_slot_range = Range {
1707            start: first_non_ancient_slot,
1708            end: range.end,
1709        };
1710
1711        // 2 is for 2 special chunks - unaligned slots at the beginning and end
1712        let chunk_count =
1713            ancient_slot_count + 2 + non_ancient_slot_count / (MAX_ITEMS_PER_CHUNK as usize);
1714
1715        SplitAncientStorages {
1716            ancient_slot_count,
1717            ancient_slots,
1718            first_non_ancient_slot,
1719            first_chunk_start,
1720            non_ancient_slot_count,
1721            chunk_count,
1722            normal_slot_range,
1723        }
1724    }
1725
1726    /// given 'normal_chunk', return the starting slot of that chunk in the normal/non-ancient range
1727    /// a normal_chunk is 0<=normal_chunk<=non_ancient_chunk_count
1728    /// non_ancient_chunk_count is chunk_count-ancient_slot_count
1729    fn get_starting_slot_from_normal_chunk(&self, normal_chunk: usize) -> Slot {
1730        if normal_chunk == 0 {
1731            self.normal_slot_range.start
1732        } else {
1733            assert!(
1734                normal_chunk.saturating_add(self.ancient_slot_count) < self.chunk_count,
1735                "out of bounds: {}, {}",
1736                normal_chunk,
1737                self.chunk_count
1738            );
1739
1740            let normal_chunk = normal_chunk.saturating_sub(1);
1741            (self.first_chunk_start + MAX_ITEMS_PER_CHUNK * (normal_chunk as Slot))
1742                .max(self.normal_slot_range.start)
1743        }
1744    }
1745
1746    /// ancient slots are the first chunks
1747    fn is_chunk_ancient(&self, chunk: usize) -> bool {
1748        chunk < self.ancient_slot_count
1749    }
1750
1751    /// given chunk in 0<=chunk<self.chunk_count
1752    /// return the range of slots in that chunk
1753    /// None indicates the range is empty for that chunk.
1754    fn get_slot_range(&self, chunk: usize) -> Option<Range<Slot>> {
1755        let range = if self.is_chunk_ancient(chunk) {
1756            // ancient append vecs are handled individually
1757            let slot = self.ancient_slots[chunk];
1758            Range {
1759                start: slot,
1760                end: slot + 1,
1761            }
1762        } else {
1763            // normal chunks are after ancient chunks
1764            let normal_chunk = chunk - self.ancient_slot_count;
1765            if normal_chunk == 0 {
1766                // first slot
1767                Range {
1768                    start: self.normal_slot_range.start,
1769                    end: self.first_chunk_start.min(self.normal_slot_range.end),
1770                }
1771            } else {
1772                // normal full chunk or the last chunk
1773                let first_slot = self.get_starting_slot_from_normal_chunk(normal_chunk);
1774                Range {
1775                    start: first_slot,
1776                    end: (first_slot + MAX_ITEMS_PER_CHUNK).min(self.normal_slot_range.end),
1777                }
1778            }
1779        };
1780        // return empty range as None
1781        (!range.is_empty()).then_some(range)
1782    }
1783}
1784
1785#[derive(Debug, Default)]
1786struct FlushStats {
1787    num_flushed: usize,
1788    num_purged: usize,
1789    total_size: u64,
1790}
1791
1792#[derive(Debug, Default)]
1793struct LatestAccountsIndexRootsStats {
1794    roots_len: AtomicUsize,
1795    historical_roots_len: AtomicUsize,
1796    uncleaned_roots_len: AtomicUsize,
1797    previous_uncleaned_roots_len: AtomicUsize,
1798    roots_range: AtomicU64,
1799    rooted_cleaned_count: AtomicUsize,
1800    unrooted_cleaned_count: AtomicUsize,
1801    clean_unref_from_storage_us: AtomicU64,
1802    clean_dead_slot_us: AtomicU64,
1803}
1804
1805impl LatestAccountsIndexRootsStats {
1806    fn update(&self, accounts_index_roots_stats: &AccountsIndexRootsStats) {
1807        if let Some(value) = accounts_index_roots_stats.roots_len {
1808            self.roots_len.store(value, Ordering::Relaxed);
1809        }
1810        if let Some(value) = accounts_index_roots_stats.uncleaned_roots_len {
1811            self.uncleaned_roots_len.store(value, Ordering::Relaxed);
1812        }
1813        if let Some(value) = accounts_index_roots_stats.previous_uncleaned_roots_len {
1814            self.previous_uncleaned_roots_len
1815                .store(value, Ordering::Relaxed);
1816        }
1817        if let Some(value) = accounts_index_roots_stats.historical_roots_len {
1818            self.historical_roots_len.store(value, Ordering::Relaxed);
1819        }
1820        if let Some(value) = accounts_index_roots_stats.roots_range {
1821            self.roots_range.store(value, Ordering::Relaxed);
1822        }
1823        self.rooted_cleaned_count.fetch_add(
1824            accounts_index_roots_stats.rooted_cleaned_count,
1825            Ordering::Relaxed,
1826        );
1827        self.unrooted_cleaned_count.fetch_add(
1828            accounts_index_roots_stats.unrooted_cleaned_count,
1829            Ordering::Relaxed,
1830        );
1831        self.clean_unref_from_storage_us.fetch_add(
1832            accounts_index_roots_stats.clean_unref_from_storage_us,
1833            Ordering::Relaxed,
1834        );
1835        self.clean_dead_slot_us.fetch_add(
1836            accounts_index_roots_stats.clean_dead_slot_us,
1837            Ordering::Relaxed,
1838        );
1839    }
1840
1841    fn report(&self) {
1842        datapoint_info!(
1843            "accounts_index_roots_len",
1844            (
1845                "roots_len",
1846                self.roots_len.load(Ordering::Relaxed) as i64,
1847                i64
1848            ),
1849            (
1850                "historical_roots_len",
1851                self.historical_roots_len.load(Ordering::Relaxed) as i64,
1852                i64
1853            ),
1854            (
1855                "uncleaned_roots_len",
1856                self.uncleaned_roots_len.load(Ordering::Relaxed) as i64,
1857                i64
1858            ),
1859            (
1860                "previous_uncleaned_roots_len",
1861                self.previous_uncleaned_roots_len.load(Ordering::Relaxed) as i64,
1862                i64
1863            ),
1864            (
1865                "roots_range_width",
1866                self.roots_range.load(Ordering::Relaxed) as i64,
1867                i64
1868            ),
1869            (
1870                "unrooted_cleaned_count",
1871                self.unrooted_cleaned_count.swap(0, Ordering::Relaxed) as i64,
1872                i64
1873            ),
1874            (
1875                "rooted_cleaned_count",
1876                self.rooted_cleaned_count.swap(0, Ordering::Relaxed) as i64,
1877                i64
1878            ),
1879            (
1880                "clean_unref_from_storage_us",
1881                self.clean_unref_from_storage_us.swap(0, Ordering::Relaxed) as i64,
1882                i64
1883            ),
1884            (
1885                "clean_dead_slot_us",
1886                self.clean_dead_slot_us.swap(0, Ordering::Relaxed) as i64,
1887                i64
1888            ),
1889            (
1890                "append_vecs_open",
1891                APPEND_VEC_MMAPPED_FILES_OPEN.load(Ordering::Relaxed) as i64,
1892                i64
1893            )
1894        );
1895
1896        // Don't need to reset since this tracks the latest updates, not a cumulative total
1897    }
1898}
1899
1900#[derive(Debug, Default)]
1901struct CleanAccountsStats {
1902    purge_stats: PurgeStats,
1903    latest_accounts_index_roots_stats: LatestAccountsIndexRootsStats,
1904
1905    // stats held here and reported by clean_accounts
1906    clean_old_root_us: AtomicU64,
1907    clean_old_root_reclaim_us: AtomicU64,
1908    reset_uncleaned_roots_us: AtomicU64,
1909    remove_dead_accounts_remove_us: AtomicU64,
1910    remove_dead_accounts_shrink_us: AtomicU64,
1911    clean_stored_dead_slots_us: AtomicU64,
1912}
1913
1914impl CleanAccountsStats {
1915    fn report(&self) {
1916        self.purge_stats.report("clean_purge_slots_stats", None);
1917        self.latest_accounts_index_roots_stats.report();
1918    }
1919}
1920
1921#[derive(Debug, Default)]
1922pub(crate) struct ShrinkAncientStats {
1923    pub(crate) shrink_stats: ShrinkStats,
1924    pub(crate) ancient_append_vecs_shrunk: AtomicU64,
1925    pub(crate) total_us: AtomicU64,
1926    pub(crate) random_shrink: AtomicU64,
1927    pub(crate) slots_considered: AtomicU64,
1928    pub(crate) ancient_scanned: AtomicU64,
1929}
1930
1931#[derive(Debug, Default)]
1932pub(crate) struct ShrinkStatsSub {
1933    pub(crate) store_accounts_timing: StoreAccountsTiming,
1934    pub(crate) rewrite_elapsed_us: u64,
1935    pub(crate) create_and_insert_store_elapsed_us: u64,
1936}
1937
1938impl ShrinkStatsSub {
1939    #[allow(dead_code)]
1940    pub(crate) fn accumulate(&mut self, other: &Self) {
1941        self.store_accounts_timing
1942            .accumulate(&other.store_accounts_timing);
1943        saturating_add_assign!(self.rewrite_elapsed_us, other.rewrite_elapsed_us);
1944        saturating_add_assign!(
1945            self.create_and_insert_store_elapsed_us,
1946            other.create_and_insert_store_elapsed_us
1947        );
1948    }
1949}
1950
1951#[derive(Debug, Default)]
1952pub(crate) struct ShrinkStats {
1953    last_report: AtomicInterval,
1954    num_slots_shrunk: AtomicUsize,
1955    storage_read_elapsed: AtomicU64,
1956    index_read_elapsed: AtomicU64,
1957    create_and_insert_store_elapsed: AtomicU64,
1958    store_accounts_elapsed: AtomicU64,
1959    update_index_elapsed: AtomicU64,
1960    handle_reclaims_elapsed: AtomicU64,
1961    remove_old_stores_shrink_us: AtomicU64,
1962    rewrite_elapsed: AtomicU64,
1963    drop_storage_entries_elapsed: AtomicU64,
1964    recycle_stores_write_elapsed: AtomicU64,
1965    accounts_removed: AtomicUsize,
1966    bytes_removed: AtomicU64,
1967    bytes_written: AtomicU64,
1968    skipped_shrink: AtomicU64,
1969    dead_accounts: AtomicU64,
1970    alive_accounts: AtomicU64,
1971    accounts_loaded: AtomicU64,
1972}
1973
1974impl ShrinkStats {
1975    fn report(&self) {
1976        if self.last_report.should_update(1000) {
1977            datapoint_info!(
1978                "shrink_stats",
1979                (
1980                    "num_slots_shrunk",
1981                    self.num_slots_shrunk.swap(0, Ordering::Relaxed) as i64,
1982                    i64
1983                ),
1984                (
1985                    "storage_read_elapsed",
1986                    self.storage_read_elapsed.swap(0, Ordering::Relaxed) as i64,
1987                    i64
1988                ),
1989                (
1990                    "index_read_elapsed",
1991                    self.index_read_elapsed.swap(0, Ordering::Relaxed) as i64,
1992                    i64
1993                ),
1994                (
1995                    "create_and_insert_store_elapsed",
1996                    self.create_and_insert_store_elapsed
1997                        .swap(0, Ordering::Relaxed) as i64,
1998                    i64
1999                ),
2000                (
2001                    "store_accounts_elapsed",
2002                    self.store_accounts_elapsed.swap(0, Ordering::Relaxed) as i64,
2003                    i64
2004                ),
2005                (
2006                    "update_index_elapsed",
2007                    self.update_index_elapsed.swap(0, Ordering::Relaxed) as i64,
2008                    i64
2009                ),
2010                (
2011                    "handle_reclaims_elapsed",
2012                    self.handle_reclaims_elapsed.swap(0, Ordering::Relaxed) as i64,
2013                    i64
2014                ),
2015                (
2016                    "remove_old_stores_shrink_us",
2017                    self.remove_old_stores_shrink_us.swap(0, Ordering::Relaxed) as i64,
2018                    i64
2019                ),
2020                (
2021                    "rewrite_elapsed",
2022                    self.rewrite_elapsed.swap(0, Ordering::Relaxed) as i64,
2023                    i64
2024                ),
2025                (
2026                    "drop_storage_entries_elapsed",
2027                    self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
2028                    i64
2029                ),
2030                (
2031                    "recycle_stores_write_time",
2032                    self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
2033                    i64
2034                ),
2035                (
2036                    "accounts_removed",
2037                    self.accounts_removed.swap(0, Ordering::Relaxed) as i64,
2038                    i64
2039                ),
2040                (
2041                    "bytes_removed",
2042                    self.bytes_removed.swap(0, Ordering::Relaxed) as i64,
2043                    i64
2044                ),
2045                (
2046                    "bytes_written",
2047                    self.bytes_written.swap(0, Ordering::Relaxed) as i64,
2048                    i64
2049                ),
2050                (
2051                    "skipped_shrink",
2052                    self.skipped_shrink.swap(0, Ordering::Relaxed) as i64,
2053                    i64
2054                ),
2055                (
2056                    "alive_accounts",
2057                    self.alive_accounts.swap(0, Ordering::Relaxed) as i64,
2058                    i64
2059                ),
2060                (
2061                    "dead_accounts",
2062                    self.dead_accounts.swap(0, Ordering::Relaxed) as i64,
2063                    i64
2064                ),
2065                (
2066                    "accounts_loaded",
2067                    self.accounts_loaded.swap(0, Ordering::Relaxed) as i64,
2068                    i64
2069                ),
2070            );
2071        }
2072    }
2073}
2074
2075impl ShrinkAncientStats {
2076    pub(crate) fn report(&self) {
2077        if self.shrink_stats.last_report.should_update(1000) {
2078            datapoint_info!(
2079                "shrink_ancient_stats",
2080                (
2081                    "num_slots_shrunk",
2082                    self.shrink_stats
2083                        .num_slots_shrunk
2084                        .swap(0, Ordering::Relaxed) as i64,
2085                    i64
2086                ),
2087                (
2088                    "storage_read_elapsed",
2089                    self.shrink_stats
2090                        .storage_read_elapsed
2091                        .swap(0, Ordering::Relaxed) as i64,
2092                    i64
2093                ),
2094                (
2095                    "index_read_elapsed",
2096                    self.shrink_stats
2097                        .index_read_elapsed
2098                        .swap(0, Ordering::Relaxed) as i64,
2099                    i64
2100                ),
2101                (
2102                    "create_and_insert_store_elapsed",
2103                    self.shrink_stats
2104                        .create_and_insert_store_elapsed
2105                        .swap(0, Ordering::Relaxed) as i64,
2106                    i64
2107                ),
2108                (
2109                    "store_accounts_elapsed",
2110                    self.shrink_stats
2111                        .store_accounts_elapsed
2112                        .swap(0, Ordering::Relaxed) as i64,
2113                    i64
2114                ),
2115                (
2116                    "update_index_elapsed",
2117                    self.shrink_stats
2118                        .update_index_elapsed
2119                        .swap(0, Ordering::Relaxed) as i64,
2120                    i64
2121                ),
2122                (
2123                    "handle_reclaims_elapsed",
2124                    self.shrink_stats
2125                        .handle_reclaims_elapsed
2126                        .swap(0, Ordering::Relaxed) as i64,
2127                    i64
2128                ),
2129                (
2130                    "remove_old_stores_shrink_us",
2131                    self.shrink_stats
2132                        .remove_old_stores_shrink_us
2133                        .swap(0, Ordering::Relaxed) as i64,
2134                    i64
2135                ),
2136                (
2137                    "rewrite_elapsed",
2138                    self.shrink_stats.rewrite_elapsed.swap(0, Ordering::Relaxed) as i64,
2139                    i64
2140                ),
2141                (
2142                    "drop_storage_entries_elapsed",
2143                    self.shrink_stats
2144                        .drop_storage_entries_elapsed
2145                        .swap(0, Ordering::Relaxed) as i64,
2146                    i64
2147                ),
2148                (
2149                    "recycle_stores_write_time",
2150                    self.shrink_stats
2151                        .recycle_stores_write_elapsed
2152                        .swap(0, Ordering::Relaxed) as i64,
2153                    i64
2154                ),
2155                (
2156                    "accounts_removed",
2157                    self.shrink_stats
2158                        .accounts_removed
2159                        .swap(0, Ordering::Relaxed) as i64,
2160                    i64
2161                ),
2162                (
2163                    "bytes_removed",
2164                    self.shrink_stats.bytes_removed.swap(0, Ordering::Relaxed) as i64,
2165                    i64
2166                ),
2167                (
2168                    "bytes_written",
2169                    self.shrink_stats.bytes_written.swap(0, Ordering::Relaxed) as i64,
2170                    i64
2171                ),
2172                (
2173                    "alive_accounts",
2174                    self.shrink_stats.alive_accounts.swap(0, Ordering::Relaxed) as i64,
2175                    i64
2176                ),
2177                (
2178                    "dead_accounts",
2179                    self.shrink_stats.dead_accounts.swap(0, Ordering::Relaxed) as i64,
2180                    i64
2181                ),
2182                (
2183                    "accounts_loaded",
2184                    self.shrink_stats.accounts_loaded.swap(0, Ordering::Relaxed) as i64,
2185                    i64
2186                ),
2187                (
2188                    "ancient_append_vecs_shrunk",
2189                    self.ancient_append_vecs_shrunk.swap(0, Ordering::Relaxed) as i64,
2190                    i64
2191                ),
2192                (
2193                    "random",
2194                    self.random_shrink.swap(0, Ordering::Relaxed) as i64,
2195                    i64
2196                ),
2197                (
2198                    "slots_considered",
2199                    self.slots_considered.swap(0, Ordering::Relaxed) as i64,
2200                    i64
2201                ),
2202                (
2203                    "ancient_scanned",
2204                    self.ancient_scanned.swap(0, Ordering::Relaxed) as i64,
2205                    i64
2206                ),
2207                (
2208                    "total_us",
2209                    self.total_us.swap(0, Ordering::Relaxed) as i64,
2210                    i64
2211                ),
2212            );
2213        }
2214    }
2215}
2216
2217pub fn quarter_thread_count() -> usize {
2218    std::cmp::max(2, num_cpus::get() / 4)
2219}
2220
2221pub fn make_min_priority_thread_pool() -> ThreadPool {
2222    // Use lower thread count to reduce priority.
2223    let num_threads = quarter_thread_count();
2224    rayon::ThreadPoolBuilder::new()
2225        .thread_name(|i| format!("solAccountsLo{i:02}"))
2226        .num_threads(num_threads)
2227        .build()
2228        .unwrap()
2229}
2230
2231#[cfg(all(test, RUSTC_WITH_SPECIALIZATION))]
2232impl solana_frozen_abi::abi_example::AbiExample for AccountsDb {
2233    fn example() -> Self {
2234        let accounts_db = AccountsDb::new_single_for_tests();
2235        let key = Pubkey::default();
2236        let some_data_len = 5;
2237        let some_slot: Slot = 0;
2238        let account = AccountSharedData::new(1, some_data_len, &key);
2239        accounts_db.store_uncached(some_slot, &[(&key, &account)]);
2240        accounts_db.add_root(0);
2241
2242        accounts_db
2243    }
2244}
2245
2246impl<'a> ZeroLamport for StoredAccountMeta<'a> {
2247    fn is_zero_lamport(&self) -> bool {
2248        self.lamports() == 0
2249    }
2250}
2251
2252impl<'a> ReadableAccount for StoredAccountMeta<'a> {
2253    fn lamports(&self) -> u64 {
2254        self.account_meta.lamports
2255    }
2256    fn data(&self) -> &[u8] {
2257        self.data
2258    }
2259    fn owner(&self) -> &Pubkey {
2260        &self.account_meta.owner
2261    }
2262    fn executable(&self) -> bool {
2263        self.account_meta.executable
2264    }
2265    fn rent_epoch(&self) -> Epoch {
2266        self.account_meta.rent_epoch
2267    }
2268}
2269
2270struct IndexAccountMapEntry<'a> {
2271    pub write_version: StoredMetaWriteVersion,
2272    pub store_id: AppendVecId,
2273    pub stored_account: StoredAccountMeta<'a>,
2274}
2275
2276type GenerateIndexAccountsMap<'a> = HashMap<Pubkey, IndexAccountMapEntry<'a>>;
2277
2278/// called on a struct while scanning append vecs
2279trait AppendVecScan: Send + Sync + Clone {
2280    /// return true if this pubkey should be included
2281    fn filter(&mut self, pubkey: &Pubkey) -> bool;
2282    /// set current slot of the scan
2283    fn set_slot(&mut self, slot: Slot);
2284    /// found `account` in the append vec
2285    fn found_account(&mut self, account: &LoadedAccount);
2286    /// scanning is done
2287    fn scanning_complete(self) -> BinnedHashData;
2288    /// initialize accumulator
2289    fn init_accum(&mut self, count: usize);
2290    fn get_accum(&mut self) -> BinnedHashData;
2291    fn set_accum(&mut self, accum: BinnedHashData);
2292}
2293
2294#[derive(Clone)]
2295/// state to keep while scanning append vec accounts for hash calculation
2296/// These would have been captured in a fn from within the scan function.
2297/// Some of these are constant across all pubkeys, some are constant across a slot.
2298/// Some could be unique per pubkey.
2299struct ScanState<'a> {
2300    /// slot we're currently scanning
2301    current_slot: Slot,
2302    /// accumulated results
2303    accum: BinnedHashData,
2304    bin_calculator: &'a PubkeyBinCalculator24,
2305    bin_range: &'a Range<usize>,
2306    config: &'a CalcAccountsHashConfig<'a>,
2307    mismatch_found: Arc<AtomicU64>,
2308    filler_account_suffix: Option<&'a Pubkey>,
2309    range: usize,
2310    sort_time: Arc<AtomicU64>,
2311    pubkey_to_bin_index: usize,
2312}
2313
2314impl<'a> AppendVecScan for ScanState<'a> {
2315    fn set_slot(&mut self, slot: Slot) {
2316        self.current_slot = slot;
2317    }
2318    fn filter(&mut self, pubkey: &Pubkey) -> bool {
2319        self.pubkey_to_bin_index = self.bin_calculator.bin_from_pubkey(pubkey);
2320        self.bin_range.contains(&self.pubkey_to_bin_index)
2321    }
2322    fn init_accum(&mut self, count: usize) {
2323        if self.accum.is_empty() {
2324            self.accum.append(&mut vec![Vec::new(); count]);
2325        }
2326    }
2327    fn found_account(&mut self, loaded_account: &LoadedAccount) {
2328        let pubkey = loaded_account.pubkey();
2329        assert!(self.bin_range.contains(&self.pubkey_to_bin_index)); // get rid of this once we have confidence
2330
2331        // when we are scanning with bin ranges, we don't need to use exact bin numbers. Subtract to make first bin we care about at index 0.
2332        self.pubkey_to_bin_index -= self.bin_range.start;
2333
2334        let balance = loaded_account.lamports();
2335        let loaded_hash = loaded_account.loaded_hash();
2336        let source_item = CalculateHashIntermediate::new(loaded_hash, balance, *pubkey);
2337
2338        if self.config.check_hash
2339            && !AccountsDb::is_filler_account_helper(pubkey, self.filler_account_suffix)
2340        {
2341            // this will not be supported anymore
2342            let computed_hash = loaded_account.compute_hash(
2343                self.current_slot,
2344                pubkey,
2345                INCLUDE_SLOT_IN_HASH_IRRELEVANT_CHECK_HASH,
2346            );
2347            if computed_hash != source_item.hash {
2348                info!(
2349                    "hash mismatch found: computed: {}, loaded: {}, pubkey: {}",
2350                    computed_hash, source_item.hash, pubkey
2351                );
2352                self.mismatch_found.fetch_add(1, Ordering::Relaxed);
2353            }
2354        }
2355        self.init_accum(self.range);
2356        self.accum[self.pubkey_to_bin_index].push(source_item);
2357    }
2358    fn scanning_complete(self) -> BinnedHashData {
2359        let (result, timing) = AccountsDb::sort_slot_storage_scan(self.accum);
2360        self.sort_time.fetch_add(timing, Ordering::Relaxed);
2361        result
2362    }
2363    fn get_accum(&mut self) -> BinnedHashData {
2364        std::mem::take(&mut self.accum)
2365    }
2366    fn set_accum(&mut self, accum: BinnedHashData) {
2367        self.accum = accum;
2368    }
2369}
2370
2371impl AccountsDb {
2372    pub const ACCOUNTS_HASH_CACHE_DIR: &str = "accounts_hash_cache";
2373
2374    pub fn default_for_tests() -> Self {
2375        Self::default_with_accounts_index(AccountInfoAccountsIndex::default_for_tests(), None)
2376    }
2377
2378    fn default_with_accounts_index(
2379        accounts_index: AccountInfoAccountsIndex,
2380        accounts_hash_cache_path: Option<PathBuf>,
2381    ) -> Self {
2382        let num_threads = get_thread_count();
2383        const MAX_READ_ONLY_CACHE_DATA_SIZE: usize = 400_000_000; // 400M bytes
2384
2385        let mut temp_accounts_hash_cache_path = None;
2386        let accounts_hash_cache_path = accounts_hash_cache_path.unwrap_or_else(|| {
2387            temp_accounts_hash_cache_path = Some(TempDir::new().unwrap());
2388            temp_accounts_hash_cache_path
2389                .as_ref()
2390                .unwrap()
2391                .path()
2392                .to_path_buf()
2393        });
2394
2395        let mut bank_hash_stats = HashMap::new();
2396        bank_hash_stats.insert(0, BankHashStats::default());
2397
2398        // Increase the stack for accounts threads
2399        // rayon needs a lot of stack
2400        const ACCOUNTS_STACK_SIZE: usize = 8 * 1024 * 1024;
2401
2402        AccountsDb {
2403            assert_stakes_cache_consistency: false,
2404            create_ancient_storage: CreateAncientStorage::Append,
2405            verify_accounts_hash_in_bg: VerifyAccountsHashInBackground::default(),
2406            filler_accounts_per_slot: AtomicU64::default(),
2407            filler_account_slots_remaining: AtomicU64::default(),
2408            active_stats: ActiveStats::default(),
2409            accounts_hash_complete_one_epoch_old: RwLock::default(),
2410            skip_initial_hash_calc: false,
2411            ancient_append_vec_offset: None,
2412            accounts_index,
2413            storage: AccountStorage::default(),
2414            accounts_cache: AccountsCache::default(),
2415            sender_bg_hasher: None,
2416            read_only_accounts_cache: ReadOnlyAccountsCache::new(MAX_READ_ONLY_CACHE_DATA_SIZE),
2417            recycle_stores: RwLock::new(RecycleStores::default()),
2418            uncleaned_pubkeys: DashMap::new(),
2419            next_id: AtomicAppendVecId::new(0),
2420            shrink_candidate_slots: Mutex::new(HashMap::new()),
2421            write_cache_limit_bytes: None,
2422            write_version: AtomicU64::new(0),
2423            paths: vec![],
2424            full_accounts_hash_cache_path: accounts_hash_cache_path.join("full"),
2425            incremental_accounts_hash_cache_path: accounts_hash_cache_path.join("incremental"),
2426            temp_accounts_hash_cache_path,
2427            shrink_paths: RwLock::new(None),
2428            temp_paths: None,
2429            file_size: DEFAULT_FILE_SIZE,
2430            thread_pool: rayon::ThreadPoolBuilder::new()
2431                .num_threads(num_threads)
2432                .thread_name(|i| format!("solAccounts{i:02}"))
2433                .stack_size(ACCOUNTS_STACK_SIZE)
2434                .build()
2435                .unwrap(),
2436            thread_pool_clean: make_min_priority_thread_pool(),
2437            accounts_delta_hashes: Mutex::new(HashMap::new()),
2438            accounts_hashes: Mutex::new(HashMap::new()),
2439            bank_hash_stats: Mutex::new(bank_hash_stats),
2440            external_purge_slots_stats: PurgeStats::default(),
2441            clean_accounts_stats: CleanAccountsStats::default(),
2442            shrink_stats: ShrinkStats::default(),
2443            shrink_ancient_stats: ShrinkAncientStats::default(),
2444            stats: AccountsStats::default(),
2445            cluster_type: None,
2446            account_indexes: AccountSecondaryIndexes::default(),
2447            #[cfg(test)]
2448            load_delay: u64::default(),
2449            #[cfg(test)]
2450            load_limit: AtomicU64::default(),
2451            is_bank_drop_callback_enabled: AtomicBool::default(),
2452            remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization::default(),
2453            shrink_ratio: AccountShrinkThreshold::default(),
2454            dirty_stores: DashMap::default(),
2455            zero_lamport_accounts_to_purge_after_full_snapshot: DashSet::default(),
2456            accounts_update_notifier: None,
2457            filler_accounts_config: FillerAccountsConfig::default(),
2458            filler_account_suffix: None,
2459            log_dead_slots: AtomicBool::new(true),
2460            exhaustively_verify_refcounts: false,
2461            epoch_accounts_hash_manager: EpochAccountsHashManager::new_invalid(),
2462        }
2463    }
2464
2465    pub fn new_for_tests(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
2466        AccountsDb::new_with_config(
2467            paths,
2468            cluster_type,
2469            AccountSecondaryIndexes::default(),
2470            AccountShrinkThreshold::default(),
2471            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
2472            None,
2473            &Arc::default(),
2474        )
2475    }
2476
2477    pub fn new_for_tests_with_caching(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
2478        AccountsDb::new_with_config(
2479            paths,
2480            cluster_type,
2481            AccountSecondaryIndexes::default(),
2482            AccountShrinkThreshold::default(),
2483            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
2484            None,
2485            &Arc::default(),
2486        )
2487    }
2488
2489    pub fn new_with_config(
2490        paths: Vec<PathBuf>,
2491        cluster_type: &ClusterType,
2492        account_indexes: AccountSecondaryIndexes,
2493        shrink_ratio: AccountShrinkThreshold,
2494        mut accounts_db_config: Option<AccountsDbConfig>,
2495        accounts_update_notifier: Option<AccountsUpdateNotifier>,
2496        exit: &Arc<AtomicBool>,
2497    ) -> Self {
2498        let accounts_index = AccountsIndex::new(
2499            accounts_db_config.as_mut().and_then(|x| x.index.take()),
2500            exit,
2501        );
2502        let accounts_hash_cache_path = accounts_db_config
2503            .as_ref()
2504            .and_then(|x| x.accounts_hash_cache_path.clone());
2505
2506        let filler_accounts_config = accounts_db_config
2507            .as_ref()
2508            .map(|config| config.filler_accounts_config)
2509            .unwrap_or_default();
2510        let skip_initial_hash_calc = accounts_db_config
2511            .as_ref()
2512            .map(|config| config.skip_initial_hash_calc)
2513            .unwrap_or_default();
2514
2515        let ancient_append_vec_offset = accounts_db_config
2516            .as_ref()
2517            .map(|config| config.ancient_append_vec_offset)
2518            .unwrap_or(ANCIENT_APPEND_VEC_DEFAULT_OFFSET);
2519
2520        let exhaustively_verify_refcounts = accounts_db_config
2521            .as_ref()
2522            .map(|config| config.exhaustively_verify_refcounts)
2523            .unwrap_or_default();
2524
2525        let assert_stakes_cache_consistency = accounts_db_config
2526            .as_ref()
2527            .map(|config| config.assert_stakes_cache_consistency)
2528            .unwrap_or_default();
2529
2530        let create_ancient_storage = accounts_db_config
2531            .as_ref()
2532            .map(|config| config.create_ancient_storage)
2533            .unwrap_or(CreateAncientStorage::Append);
2534
2535        let filler_account_suffix = if filler_accounts_config.count > 0 {
2536            Some(solana_sdk::pubkey::new_rand())
2537        } else {
2538            None
2539        };
2540        let paths_is_empty = paths.is_empty();
2541        let mut new = Self {
2542            paths,
2543            skip_initial_hash_calc,
2544            ancient_append_vec_offset,
2545            cluster_type: Some(*cluster_type),
2546            account_indexes,
2547            shrink_ratio,
2548            accounts_update_notifier,
2549            filler_accounts_config,
2550            filler_account_suffix,
2551            assert_stakes_cache_consistency,
2552            create_ancient_storage,
2553            write_cache_limit_bytes: accounts_db_config
2554                .as_ref()
2555                .and_then(|x| x.write_cache_limit_bytes),
2556            exhaustively_verify_refcounts,
2557            ..Self::default_with_accounts_index(accounts_index, accounts_hash_cache_path)
2558        };
2559        if paths_is_empty {
2560            // Create a temporary set of accounts directories, used primarily
2561            // for testing
2562            let (temp_dirs, paths) = get_temp_accounts_paths(DEFAULT_NUM_DIRS).unwrap();
2563            new.accounts_update_notifier = None;
2564            new.paths = paths;
2565            new.temp_paths = Some(temp_dirs);
2566        };
2567
2568        new.start_background_hasher();
2569        {
2570            for path in new.paths.iter() {
2571                std::fs::create_dir_all(path).expect("Create directory failed.");
2572            }
2573        }
2574        new
2575    }
2576
2577    /// Gradual means filler accounts will be added over the course of an epoch, during cache flush.
2578    /// This is in contrast to adding all the filler accounts immediately before the validator starts.
2579    fn init_gradual_filler_accounts(&self, slots_per_epoch: Slot) {
2580        let count = self.filler_accounts_config.count;
2581        if count > 0 {
2582            // filler accounts are a debug only feature. integer division is fine here
2583            let accounts_per_slot = (count as u64) / slots_per_epoch;
2584            self.filler_accounts_per_slot
2585                .store(accounts_per_slot, Ordering::Release);
2586            self.filler_account_slots_remaining
2587                .store(slots_per_epoch, Ordering::Release);
2588        }
2589    }
2590
2591    pub fn set_shrink_paths(&self, paths: Vec<PathBuf>) {
2592        assert!(!paths.is_empty());
2593        let mut shrink_paths = self.shrink_paths.write().unwrap();
2594        for path in &paths {
2595            std::fs::create_dir_all(path).expect("Create directory failed.");
2596        }
2597        *shrink_paths = Some(paths);
2598    }
2599
2600    pub fn file_size(&self) -> u64 {
2601        self.file_size
2602    }
2603
2604    pub fn new_single_for_tests() -> Self {
2605        AccountsDb::new_for_tests(Vec::new(), &ClusterType::Development)
2606    }
2607
2608    pub fn new_single_for_tests_with_caching() -> Self {
2609        AccountsDb::new_for_tests_with_caching(Vec::new(), &ClusterType::Development)
2610    }
2611
2612    pub fn new_single_for_tests_with_secondary_indexes(
2613        secondary_indexes: AccountSecondaryIndexes,
2614    ) -> Self {
2615        let mut accounts_db = AccountsDb::new_single_for_tests();
2616        accounts_db.account_indexes = secondary_indexes;
2617        accounts_db
2618    }
2619
2620    fn next_id(&self) -> AppendVecId {
2621        let next_id = self.next_id.fetch_add(1, Ordering::AcqRel);
2622        assert!(next_id != AppendVecId::MAX, "We've run out of storage ids!");
2623        next_id
2624    }
2625
2626    fn new_storage_entry(&self, slot: Slot, path: &Path, size: u64) -> AccountStorageEntry {
2627        AccountStorageEntry::new(path, slot, self.next_id(), size)
2628    }
2629
2630    pub fn expected_cluster_type(&self) -> ClusterType {
2631        self.cluster_type
2632            .expect("Cluster type must be set at initialization")
2633    }
2634
2635    /// Reclaim older states of accounts older than max_clean_root_inclusive for AccountsDb bloat mitigation.
2636    /// Any accounts which are removed from the accounts index are returned in PubkeysRemovedFromAccountsIndex.
2637    /// These should NOT be unref'd later from the accounts index.
2638    fn clean_accounts_older_than_root(
2639        &self,
2640        purges: Vec<Pubkey>,
2641        max_clean_root_inclusive: Option<Slot>,
2642        ancient_account_cleans: &AtomicU64,
2643    ) -> (ReclaimResult, PubkeysRemovedFromAccountsIndex) {
2644        let pubkeys_removed_from_accounts_index = HashSet::default();
2645        if purges.is_empty() {
2646            return (
2647                ReclaimResult::default(),
2648                pubkeys_removed_from_accounts_index,
2649            );
2650        }
2651        // This number isn't carefully chosen; just guessed randomly such that
2652        // the hot loop will be the order of ~Xms.
2653        const INDEX_CLEAN_BULK_COUNT: usize = 4096;
2654
2655        let one_epoch_old = self.get_accounts_hash_complete_one_epoch_old();
2656        let pubkeys_removed_from_accounts_index = Mutex::new(pubkeys_removed_from_accounts_index);
2657
2658        let mut clean_rooted = Measure::start("clean_old_root-ms");
2659        let reclaim_vecs = purges
2660            .par_chunks(INDEX_CLEAN_BULK_COUNT)
2661            .filter_map(|pubkeys: &[Pubkey]| {
2662                let mut reclaims = Vec::new();
2663                for pubkey in pubkeys {
2664                    let removed_from_index = self.accounts_index.clean_rooted_entries(
2665                        pubkey,
2666                        &mut reclaims,
2667                        max_clean_root_inclusive,
2668                    );
2669                    if removed_from_index {
2670                        pubkeys_removed_from_accounts_index
2671                            .lock()
2672                            .unwrap()
2673                            .insert(*pubkey);
2674                    }
2675                }
2676
2677                (!reclaims.is_empty()).then(|| {
2678                    // figure out how many ancient accounts have been reclaimed
2679                    let old_reclaims = reclaims
2680                        .iter()
2681                        .filter_map(|(slot, _)| (slot < &one_epoch_old).then_some(1))
2682                        .sum();
2683                    ancient_account_cleans.fetch_add(old_reclaims, Ordering::Relaxed);
2684                    reclaims
2685                })
2686            })
2687            .collect::<Vec<_>>();
2688        clean_rooted.stop();
2689        let pubkeys_removed_from_accounts_index =
2690            pubkeys_removed_from_accounts_index.into_inner().unwrap();
2691        self.clean_accounts_stats
2692            .clean_old_root_us
2693            .fetch_add(clean_rooted.as_us(), Ordering::Relaxed);
2694
2695        let mut measure = Measure::start("clean_old_root_reclaims");
2696
2697        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
2698        // and those stores may be used for background hashing.
2699        let reset_accounts = false;
2700
2701        let mut reclaim_result = ReclaimResult::default();
2702        self.handle_reclaims(
2703            (!reclaim_vecs.is_empty()).then(|| reclaim_vecs.iter().flatten()),
2704            None,
2705            Some((&self.clean_accounts_stats.purge_stats, &mut reclaim_result)),
2706            reset_accounts,
2707            &pubkeys_removed_from_accounts_index,
2708        );
2709        measure.stop();
2710        debug!("{} {}", clean_rooted, measure);
2711        self.clean_accounts_stats
2712            .clean_old_root_reclaim_us
2713            .fetch_add(measure.as_us(), Ordering::Relaxed);
2714        (reclaim_result, pubkeys_removed_from_accounts_index)
2715    }
2716
2717    fn do_reset_uncleaned_roots(&self, max_clean_root: Option<Slot>) {
2718        let mut measure = Measure::start("reset");
2719        self.accounts_index.reset_uncleaned_roots(max_clean_root);
2720        measure.stop();
2721        self.clean_accounts_stats
2722            .reset_uncleaned_roots_us
2723            .fetch_add(measure.as_us(), Ordering::Relaxed);
2724    }
2725
2726    /// increment store_counts to non-zero for all stores that can not be deleted.
2727    /// a store cannot be deleted if:
2728    /// 1. one of the pubkeys in the store has account info to a store whose store count is not going to zero
2729    /// 2. a pubkey we were planning to remove is not removing all stores that contain the account
2730    fn calc_delete_dependencies(
2731        purges: &HashMap<Pubkey, (SlotList<AccountInfo>, RefCount)>,
2732        store_counts: &mut HashMap<AppendVecId, (usize, HashSet<Pubkey>)>,
2733        min_store_id: Option<AppendVecId>,
2734    ) {
2735        // Another pass to check if there are some filtered accounts which
2736        // do not match the criteria of deleting all appendvecs which contain them
2737        // then increment their storage count.
2738        let mut already_counted = HashSet::new();
2739        for (pubkey, (account_infos, ref_count_from_storage)) in purges.iter() {
2740            let mut failed_store_id = None;
2741            let all_stores_being_deleted =
2742                account_infos.len() as RefCount == *ref_count_from_storage;
2743            if all_stores_being_deleted {
2744                let mut delete = true;
2745                for (_slot, account_info) in account_infos {
2746                    let store_id = account_info.store_id();
2747                    if let Some(count) = store_counts.get(&store_id).map(|s| s.0) {
2748                        debug!(
2749                            "calc_delete_dependencies()
2750                            storage id: {},
2751                            count len: {}",
2752                            store_id, count,
2753                        );
2754                        if count == 0 {
2755                            // this store CAN be removed
2756                            continue;
2757                        }
2758                    }
2759                    // One of the pubkeys in the store has account info to a store whose store count is not going to zero.
2760                    // If the store cannot be found, that also means store isn't being deleted.
2761                    failed_store_id = Some(store_id);
2762                    delete = false;
2763                    break;
2764                }
2765                if delete {
2766                    // this pubkey can be deleted from all stores it is in
2767                    continue;
2768                }
2769            } else {
2770                // a pubkey we were planning to remove is not removing all stores that contain the account
2771                debug!(
2772                    "calc_delete_dependencies(),
2773                    pubkey: {},
2774                    account_infos: {:?},
2775                    account_infos_len: {},
2776                    ref_count_from_storage: {}",
2777                    pubkey,
2778                    account_infos,
2779                    account_infos.len(),
2780                    ref_count_from_storage,
2781                );
2782            }
2783
2784            // increment store_counts to non-zero for all stores that can not be deleted.
2785            let mut pending_store_ids = HashSet::new();
2786            for (_slot, account_info) in account_infos {
2787                if !already_counted.contains(&account_info.store_id()) {
2788                    pending_store_ids.insert(account_info.store_id());
2789                }
2790            }
2791            while !pending_store_ids.is_empty() {
2792                let id = pending_store_ids.iter().next().cloned().unwrap();
2793                if Some(id) == min_store_id {
2794                    if let Some(failed_store_id) = failed_store_id.take() {
2795                        info!("calc_delete_dependencies, oldest store is not able to be deleted because of {pubkey} in store {failed_store_id}");
2796                    } else {
2797                        info!("calc_delete_dependencies, oldest store is not able to be deleted because of {pubkey}, account infos len: {}, ref count: {ref_count_from_storage}", account_infos.len());
2798                    }
2799                }
2800
2801                pending_store_ids.remove(&id);
2802                if !already_counted.insert(id) {
2803                    continue;
2804                }
2805                // the point of all this code: remove the store count for all stores we cannot remove
2806                if let Some(store_count) = store_counts.remove(&id) {
2807                    // all pubkeys in this store also cannot be removed from all stores they are in
2808                    let affected_pubkeys = &store_count.1;
2809                    for key in affected_pubkeys {
2810                        for (_slot, account_info) in &purges.get(key).unwrap().0 {
2811                            if !already_counted.contains(&account_info.store_id()) {
2812                                pending_store_ids.insert(account_info.store_id());
2813                            }
2814                        }
2815                    }
2816                }
2817            }
2818        }
2819    }
2820
2821    fn background_hasher(receiver: Receiver<CachedAccount>) {
2822        loop {
2823            let result = receiver.recv();
2824            match result {
2825                Ok(account) => {
2826                    // if we hold the only ref, then this account doesn't need to be hashed, we ignore this account and it will disappear
2827                    if Arc::strong_count(&account) > 1 {
2828                        // this will cause the hash to be calculated and store inside account if it needs to be calculated
2829                        let _ = (*account).hash();
2830                    };
2831                }
2832                Err(_) => {
2833                    break;
2834                }
2835            }
2836        }
2837    }
2838
2839    fn start_background_hasher(&mut self) {
2840        let (sender, receiver) = unbounded();
2841        Builder::new()
2842            .name("solDbStoreHashr".to_string())
2843            .spawn(move || {
2844                Self::background_hasher(receiver);
2845            })
2846            .unwrap();
2847        self.sender_bg_hasher = Some(sender);
2848    }
2849
2850    #[must_use]
2851    pub(crate) fn purge_keys_exact<'a, C: 'a>(
2852        &'a self,
2853        pubkey_to_slot_set: impl Iterator<Item = &'a (Pubkey, C)>,
2854    ) -> (Vec<(Slot, AccountInfo)>, PubkeysRemovedFromAccountsIndex)
2855    where
2856        C: Contains<'a, Slot>,
2857    {
2858        let mut reclaims = Vec::new();
2859        let mut dead_keys = Vec::new();
2860
2861        for (pubkey, slots_set) in pubkey_to_slot_set {
2862            let is_empty = self
2863                .accounts_index
2864                .purge_exact(pubkey, slots_set, &mut reclaims);
2865            if is_empty {
2866                dead_keys.push(pubkey);
2867            }
2868        }
2869
2870        let pubkeys_removed_from_accounts_index = self
2871            .accounts_index
2872            .handle_dead_keys(&dead_keys, &self.account_indexes);
2873        (reclaims, pubkeys_removed_from_accounts_index)
2874    }
2875
2876    fn max_clean_root(&self, proposed_clean_root: Option<Slot>) -> Option<Slot> {
2877        match (
2878            self.accounts_index.min_ongoing_scan_root(),
2879            proposed_clean_root,
2880        ) {
2881            (None, None) => None,
2882            (Some(min_scan_root), None) => Some(min_scan_root),
2883            (None, Some(proposed_clean_root)) => Some(proposed_clean_root),
2884            (Some(min_scan_root), Some(proposed_clean_root)) => {
2885                Some(std::cmp::min(min_scan_root, proposed_clean_root))
2886            }
2887        }
2888    }
2889
2890    /// return 'slot' - slots_in_epoch
2891    fn get_slot_one_epoch_prior(slot: Slot, epoch_schedule: &EpochSchedule) -> Slot {
2892        // would like to use:
2893        // slot.saturating_sub(epoch_schedule.get_slots_in_epoch(epoch_schedule.get_epoch(slot)))
2894        // but there are problems with warmup and such on tests and probably test clusters.
2895        // So, just use the maximum below (epoch_schedule.slots_per_epoch)
2896        slot.saturating_sub(epoch_schedule.slots_per_epoch)
2897    }
2898
2899    /// hash calc is completed as of 'slot'
2900    /// so, any process that wants to take action on really old slots can now proceed up to 'completed_slot'-slots per epoch
2901    pub fn notify_accounts_hash_calculated_complete(
2902        &self,
2903        completed_slot: Slot,
2904        epoch_schedule: &EpochSchedule,
2905    ) {
2906        let one_epoch_old_slot = Self::get_slot_one_epoch_prior(completed_slot, epoch_schedule);
2907        let mut accounts_hash_complete_one_epoch_old =
2908            self.accounts_hash_complete_one_epoch_old.write().unwrap();
2909        *accounts_hash_complete_one_epoch_old =
2910            std::cmp::max(*accounts_hash_complete_one_epoch_old, one_epoch_old_slot);
2911        let accounts_hash_complete_one_epoch_old = *accounts_hash_complete_one_epoch_old;
2912
2913        // now that accounts hash calculation is complete, we can remove old historical roots
2914        self.remove_old_historical_roots(accounts_hash_complete_one_epoch_old);
2915    }
2916
2917    /// get the slot that is one epoch older than the highest slot that has been used for hash calculation
2918    fn get_accounts_hash_complete_one_epoch_old(&self) -> Slot {
2919        *self.accounts_hash_complete_one_epoch_old.read().unwrap()
2920    }
2921
2922    /// Collect all the uncleaned slots, up to a max slot
2923    ///
2924    /// Search through the uncleaned Pubkeys and return all the slots, up to a maximum slot.
2925    fn collect_uncleaned_slots_up_to_slot(&self, max_slot_inclusive: Slot) -> Vec<Slot> {
2926        self.uncleaned_pubkeys
2927            .iter()
2928            .filter_map(|entry| {
2929                let slot = *entry.key();
2930                (slot <= max_slot_inclusive).then_some(slot)
2931            })
2932            .collect()
2933    }
2934
2935    /// Remove `slots` from `uncleaned_pubkeys` and collect all pubkeys
2936    ///
2937    /// For each slot in the list of uncleaned slots, remove it from the `uncleaned_pubkeys` Map
2938    /// and collect all the pubkeys to return.
2939    fn remove_uncleaned_slots_and_collect_pubkeys(
2940        &self,
2941        uncleaned_slots: Vec<Slot>,
2942    ) -> Vec<Vec<Pubkey>> {
2943        uncleaned_slots
2944            .into_iter()
2945            .filter_map(|uncleaned_slot| {
2946                self.uncleaned_pubkeys
2947                    .remove(&uncleaned_slot)
2948                    .map(|(_removed_slot, removed_pubkeys)| removed_pubkeys)
2949            })
2950            .collect()
2951    }
2952
2953    /// Remove uncleaned slots, up to a maximum slot, and return the collected pubkeys
2954    ///
2955    fn remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(
2956        &self,
2957        max_slot_inclusive: Slot,
2958    ) -> Vec<Vec<Pubkey>> {
2959        let uncleaned_slots = self.collect_uncleaned_slots_up_to_slot(max_slot_inclusive);
2960        self.remove_uncleaned_slots_and_collect_pubkeys(uncleaned_slots)
2961    }
2962
2963    // Construct a vec of pubkeys for cleaning from:
2964    //   uncleaned_pubkeys - the delta set of updated pubkeys in rooted slots from the last clean
2965    //   dirty_stores - set of stores which had accounts removed or recently rooted
2966    fn construct_candidate_clean_keys(
2967        &self,
2968        max_clean_root_inclusive: Option<Slot>,
2969        is_startup: bool,
2970        last_full_snapshot_slot: Option<Slot>,
2971        timings: &mut CleanKeyTimings,
2972    ) -> (Vec<Pubkey>, Option<AppendVecId>) {
2973        let mut dirty_store_processing_time = Measure::start("dirty_store_processing");
2974        let max_slot_inclusive =
2975            max_clean_root_inclusive.unwrap_or_else(|| self.accounts_index.max_root_inclusive());
2976        let mut dirty_stores = Vec::with_capacity(self.dirty_stores.len());
2977        // find the oldest append vec older than one epoch old
2978        // we'll add logging if that append vec cannot be marked dead
2979        let mut min_dirty_slot = self.get_accounts_hash_complete_one_epoch_old();
2980        let mut min_dirty_store_id = None;
2981        self.dirty_stores.retain(|(slot, store_id), store| {
2982            if *slot > max_slot_inclusive {
2983                true
2984            } else {
2985                if *slot < min_dirty_slot {
2986                    min_dirty_slot = *slot;
2987                    min_dirty_store_id = Some(*store_id);
2988                }
2989                dirty_stores.push((*slot, store.clone()));
2990                false
2991            }
2992        });
2993        let dirty_stores_len = dirty_stores.len();
2994        let pubkeys = DashSet::new();
2995        let dirty_ancient_stores = AtomicUsize::default();
2996        let mut dirty_store_routine = || {
2997            let chunk_size = 1.max(dirty_stores_len.saturating_div(rayon::current_num_threads()));
2998            let oldest_dirty_slots: Vec<u64> = dirty_stores
2999                .par_chunks(chunk_size)
3000                .map(|dirty_store_chunk| {
3001                    let mut oldest_dirty_slot = max_slot_inclusive.saturating_add(1);
3002                    dirty_store_chunk.iter().for_each(|(slot, store)| {
3003                        if is_ancient(&store.accounts) {
3004                            dirty_ancient_stores.fetch_add(1, Ordering::Relaxed);
3005                        }
3006                        oldest_dirty_slot = oldest_dirty_slot.min(*slot);
3007                        store.accounts.account_iter().for_each(|account| {
3008                            pubkeys.insert(*account.pubkey());
3009                        });
3010                    });
3011                    oldest_dirty_slot
3012                })
3013                .collect();
3014            timings.oldest_dirty_slot = *oldest_dirty_slots
3015                .iter()
3016                .min()
3017                .unwrap_or(&max_slot_inclusive.saturating_add(1));
3018        };
3019
3020        if is_startup {
3021            // Free to consume all the cores during startup
3022            dirty_store_routine();
3023        } else {
3024            self.thread_pool_clean.install(|| {
3025                dirty_store_routine();
3026            });
3027        }
3028        trace!(
3029            "dirty_stores.len: {} pubkeys.len: {}",
3030            dirty_stores_len,
3031            pubkeys.len()
3032        );
3033        timings.dirty_pubkeys_count = pubkeys.len() as u64;
3034        dirty_store_processing_time.stop();
3035        timings.dirty_store_processing_us += dirty_store_processing_time.as_us();
3036        timings.dirty_ancient_stores = dirty_ancient_stores.load(Ordering::Relaxed);
3037
3038        let mut collect_delta_keys = Measure::start("key_create");
3039        let delta_keys =
3040            self.remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(max_slot_inclusive);
3041        collect_delta_keys.stop();
3042        timings.collect_delta_keys_us += collect_delta_keys.as_us();
3043
3044        let mut delta_insert = Measure::start("delta_insert");
3045        self.thread_pool_clean.install(|| {
3046            delta_keys.par_iter().for_each(|keys| {
3047                for key in keys {
3048                    pubkeys.insert(*key);
3049                }
3050            });
3051        });
3052        delta_insert.stop();
3053        timings.delta_insert_us += delta_insert.as_us();
3054
3055        timings.delta_key_count = pubkeys.len() as u64;
3056
3057        let mut hashset_to_vec = Measure::start("flat_map");
3058        let mut pubkeys: Vec<Pubkey> = pubkeys.into_iter().collect();
3059        hashset_to_vec.stop();
3060        timings.hashset_to_vec_us += hashset_to_vec.as_us();
3061
3062        // Check if we should purge any of the zero_lamport_accounts_to_purge_later, based on the
3063        // last_full_snapshot_slot.
3064        assert!(
3065            last_full_snapshot_slot.is_some() || self.zero_lamport_accounts_to_purge_after_full_snapshot.is_empty(),
3066            "if snapshots are disabled, then zero_lamport_accounts_to_purge_later should always be empty"
3067        );
3068        if let Some(last_full_snapshot_slot) = last_full_snapshot_slot {
3069            self.zero_lamport_accounts_to_purge_after_full_snapshot
3070                .retain(|(slot, pubkey)| {
3071                    let is_candidate_for_clean =
3072                        max_slot_inclusive >= *slot && last_full_snapshot_slot >= *slot;
3073                    if is_candidate_for_clean {
3074                        pubkeys.push(*pubkey);
3075                    }
3076                    !is_candidate_for_clean
3077                });
3078        }
3079
3080        (pubkeys, min_dirty_store_id)
3081    }
3082
3083    /// Call clean_accounts() with the common parameters that tests/benches use.
3084    pub fn clean_accounts_for_tests(&self) {
3085        self.clean_accounts(None, false, None)
3086    }
3087
3088    /// called with cli argument to verify refcounts are correct on all accounts
3089    /// this is very slow
3090    fn exhaustively_verify_refcounts(&self, max_slot_inclusive: Option<Slot>) {
3091        let max_slot_inclusive =
3092            max_slot_inclusive.unwrap_or_else(|| self.accounts_index.max_root_inclusive());
3093        info!("exhaustively verifying refcounts as of slot: {max_slot_inclusive}");
3094        let pubkey_refcount = DashMap::<Pubkey, Vec<Slot>>::default();
3095        let slots = self.storage.all_slots();
3096        // populate
3097        slots.into_par_iter().for_each(|slot| {
3098            if slot > max_slot_inclusive {
3099                return;
3100            }
3101            if let Some(storage) = self.storage.get_slot_storage_entry(slot) {
3102                storage.all_accounts().iter().for_each(|account| {
3103                    let pk = account.pubkey();
3104                    match pubkey_refcount.entry(*pk) {
3105                        dashmap::mapref::entry::Entry::Occupied(mut occupied_entry) => {
3106                            if !occupied_entry.get().iter().any(|s| s == &slot) {
3107                                occupied_entry.get_mut().push(slot);
3108                            }
3109                        }
3110                        dashmap::mapref::entry::Entry::Vacant(vacant_entry) => {
3111                            vacant_entry.insert(vec![slot]);
3112                        }
3113                    }
3114                });
3115            }
3116        });
3117        let total = pubkey_refcount.len();
3118        let failed = AtomicBool::default();
3119        let threads = quarter_thread_count();
3120        let per_batch = total / threads;
3121        (0..=threads).into_par_iter().for_each(|attempt| {
3122                pubkey_refcount.iter().skip(attempt * per_batch).take(per_batch).for_each(|entry| {
3123                    if failed.load(Ordering::Relaxed) {
3124                        return;
3125                    }
3126                    if let Some(idx) = self.accounts_index.get_account_read_entry(entry.key()) {
3127                        match (idx.ref_count() as usize).cmp(&entry.value().len()) {
3128                            std::cmp::Ordering::Greater => {
3129                            let list = idx.slot_list();
3130                            let too_new = list.iter().filter_map(|(slot, _)| (slot > &max_slot_inclusive).then_some(())).count();
3131
3132                            if ((idx.ref_count() as usize) - too_new) > entry.value().len() {
3133                                failed.store(true, Ordering::Relaxed);
3134                                error!("exhaustively_verify_refcounts: {} refcount too large: {}, should be: {}, {:?}, {:?}, original: {:?}, too_new: {too_new}", entry.key(), idx.ref_count(), entry.value().len(), *entry.value(), list, idx.slot_list());
3135                            }
3136                        }
3137                        std::cmp::Ordering::Less => {
3138                            error!("exhaustively_verify_refcounts: {} refcount too small: {}, should be: {}, {:?}, {:?}", entry.key(), idx.ref_count(), entry.value().len(), *entry.value(), idx.slot_list());
3139                        }
3140                        _ => {}
3141                    }
3142                    }
3143                });
3144            });
3145        if failed.load(Ordering::Relaxed) {
3146            panic!("exhaustively_verify_refcounts failed");
3147        }
3148    }
3149
3150    // Purge zero lamport accounts and older rooted account states as garbage
3151    // collection
3152    // Only remove those accounts where the entire rooted history of the account
3153    // can be purged because there are no live append vecs in the ancestors
3154    pub fn clean_accounts(
3155        &self,
3156        max_clean_root_inclusive: Option<Slot>,
3157        is_startup: bool,
3158        last_full_snapshot_slot: Option<Slot>,
3159    ) {
3160        if self.exhaustively_verify_refcounts {
3161            self.exhaustively_verify_refcounts(max_clean_root_inclusive);
3162        }
3163
3164        let _guard = self.active_stats.activate(ActiveStatItem::Clean);
3165
3166        let ancient_account_cleans = AtomicU64::default();
3167
3168        let mut measure_all = Measure::start("clean_accounts");
3169        let max_clean_root_inclusive = self.max_clean_root(max_clean_root_inclusive);
3170
3171        self.report_store_stats();
3172
3173        let mut key_timings = CleanKeyTimings::default();
3174        let (mut pubkeys, min_dirty_store_id) = self.construct_candidate_clean_keys(
3175            max_clean_root_inclusive,
3176            is_startup,
3177            last_full_snapshot_slot,
3178            &mut key_timings,
3179        );
3180
3181        let mut sort = Measure::start("sort");
3182        if is_startup {
3183            pubkeys.par_sort_unstable();
3184        } else {
3185            self.thread_pool_clean
3186                .install(|| pubkeys.par_sort_unstable());
3187        }
3188        sort.stop();
3189
3190        let total_keys_count = pubkeys.len();
3191        let mut accounts_scan = Measure::start("accounts_scan");
3192        let uncleaned_roots = self.accounts_index.clone_uncleaned_roots();
3193        let found_not_zero_accum = AtomicU64::new(0);
3194        let not_found_on_fork_accum = AtomicU64::new(0);
3195        let missing_accum = AtomicU64::new(0);
3196        let useful_accum = AtomicU64::new(0);
3197
3198        // parallel scan the index.
3199        let (mut purges_zero_lamports, purges_old_accounts) = {
3200            let do_clean_scan = || {
3201                pubkeys
3202                    .par_chunks(4096)
3203                    .map(|pubkeys: &[Pubkey]| {
3204                        let mut purges_zero_lamports = HashMap::new();
3205                        let mut purges_old_accounts = Vec::new();
3206                        let mut found_not_zero = 0;
3207                        let mut not_found_on_fork = 0;
3208                        let mut missing = 0;
3209                        let mut useful = 0;
3210                        self.accounts_index.scan(
3211                            pubkeys.iter(),
3212                            |pubkey, slots_refs| {
3213                                let mut useless = true;
3214                                if let Some((slot_list, ref_count)) = slots_refs {
3215                                    let index_in_slot_list = self.accounts_index.latest_slot(
3216                                        None,
3217                                        slot_list,
3218                                        max_clean_root_inclusive,
3219                                    );
3220
3221                                    match index_in_slot_list {
3222                                        Some(index_in_slot_list) => {
3223                                            // found info relative to max_clean_root
3224                                            let (slot, account_info) =
3225                                                &slot_list[index_in_slot_list];
3226                                            if account_info.is_zero_lamport() {
3227                                                useless = false;
3228                                                purges_zero_lamports.insert(
3229                                                    *pubkey,
3230                                                    (
3231                                                        self.accounts_index.get_rooted_entries(
3232                                                            slot_list,
3233                                                            max_clean_root_inclusive,
3234                                                        ),
3235                                                        ref_count,
3236                                                    ),
3237                                                );
3238                                            } else {
3239                                                found_not_zero += 1;
3240                                            }
3241                                            if uncleaned_roots.contains(slot) {
3242                                                // Assertion enforced by `accounts_index.get()`, the latest slot
3243                                                // will not be greater than the given `max_clean_root`
3244                                                if let Some(max_clean_root_inclusive) =
3245                                                    max_clean_root_inclusive
3246                                                {
3247                                                    assert!(slot <= &max_clean_root_inclusive);
3248                                                }
3249                                                purges_old_accounts.push(*pubkey);
3250                                                useless = false;
3251                                            }
3252                                        }
3253                                        None => {
3254                                            // This pubkey is in the index but not in a root slot, so clean
3255                                            // it up by adding it to the to-be-purged list.
3256                                            //
3257                                            // Also, this pubkey must have been touched by some slot since
3258                                            // it was in the dirty list, so we assume that the slot it was
3259                                            // touched in must be unrooted.
3260                                            not_found_on_fork += 1;
3261                                            useless = false;
3262                                            purges_old_accounts.push(*pubkey);
3263                                        }
3264                                    }
3265                                } else {
3266                                    missing += 1;
3267                                }
3268                                if !useless {
3269                                    useful += 1;
3270                                }
3271                                if useless {
3272                                    AccountsIndexScanResult::None
3273                                } else {
3274                                    AccountsIndexScanResult::KeepInMemory
3275                                }
3276                            },
3277                            None,
3278                        );
3279                        found_not_zero_accum.fetch_add(found_not_zero, Ordering::Relaxed);
3280                        not_found_on_fork_accum.fetch_add(not_found_on_fork, Ordering::Relaxed);
3281                        missing_accum.fetch_add(missing, Ordering::Relaxed);
3282                        useful_accum.fetch_add(useful, Ordering::Relaxed);
3283                        (purges_zero_lamports, purges_old_accounts)
3284                    })
3285                    .reduce(
3286                        || (HashMap::new(), Vec::new()),
3287                        |mut m1, m2| {
3288                            // Collapse down the hashmaps/vecs into one.
3289                            m1.0.extend(m2.0);
3290                            m1.1.extend(m2.1);
3291                            m1
3292                        },
3293                    )
3294            };
3295            if is_startup {
3296                do_clean_scan()
3297            } else {
3298                self.thread_pool_clean.install(do_clean_scan)
3299            }
3300        };
3301        accounts_scan.stop();
3302
3303        let mut clean_old_rooted = Measure::start("clean_old_roots");
3304        let ((purged_account_slots, removed_accounts), mut pubkeys_removed_from_accounts_index) =
3305            self.clean_accounts_older_than_root(
3306                purges_old_accounts,
3307                max_clean_root_inclusive,
3308                &ancient_account_cleans,
3309            );
3310
3311        self.do_reset_uncleaned_roots(max_clean_root_inclusive);
3312        clean_old_rooted.stop();
3313
3314        let mut store_counts_time = Measure::start("store_counts");
3315
3316        // Calculate store counts as if everything was purged
3317        // Then purge if we can
3318        let mut store_counts: HashMap<AppendVecId, (usize, HashSet<Pubkey>)> = HashMap::new();
3319        for (key, (account_infos, ref_count)) in purges_zero_lamports.iter_mut() {
3320            if purged_account_slots.contains_key(key) {
3321                *ref_count = self.accounts_index.ref_count_from_storage(key);
3322            }
3323            account_infos.retain(|(slot, account_info)| {
3324                let was_slot_purged = purged_account_slots
3325                    .get(key)
3326                    .map(|slots_removed| slots_removed.contains(slot))
3327                    .unwrap_or(false);
3328                if was_slot_purged {
3329                    // No need to look up the slot storage below if the entire
3330                    // slot was purged
3331                    return false;
3332                }
3333                // Check if this update in `slot` to the account with `key` was reclaimed earlier by
3334                // `clean_accounts_older_than_root()`
3335                let was_reclaimed = removed_accounts
3336                    .get(&account_info.store_id())
3337                    .map(|store_removed| store_removed.contains(&account_info.offset()))
3338                    .unwrap_or(false);
3339                if was_reclaimed {
3340                    return false;
3341                }
3342                if let Some(store_count) = store_counts.get_mut(&account_info.store_id()) {
3343                    store_count.0 -= 1;
3344                    store_count.1.insert(*key);
3345                } else {
3346                    let mut key_set = HashSet::new();
3347                    key_set.insert(*key);
3348                    assert!(
3349                        !account_info.is_cached(),
3350                        "The Accounts Cache must be flushed first for this account info. pubkey: {}, slot: {}",
3351                        *key,
3352                        *slot
3353                    );
3354                    let count = self
3355                        .storage
3356                        .get_account_storage_entry(*slot, account_info.store_id())
3357                        .map(|store| store.count())
3358                        .unwrap()
3359                        - 1;
3360                    debug!(
3361                        "store_counts, inserting slot: {}, store id: {}, count: {}",
3362                        slot, account_info.store_id(), count
3363                    );
3364                    store_counts.insert(account_info.store_id(), (count, key_set));
3365                }
3366                true
3367            });
3368        }
3369        store_counts_time.stop();
3370
3371        let mut calc_deps_time = Measure::start("calc_deps");
3372        Self::calc_delete_dependencies(
3373            &purges_zero_lamports,
3374            &mut store_counts,
3375            min_dirty_store_id,
3376        );
3377        calc_deps_time.stop();
3378
3379        let mut purge_filter = Measure::start("purge_filter");
3380        self.filter_zero_lamport_clean_for_incremental_snapshots(
3381            max_clean_root_inclusive,
3382            last_full_snapshot_slot,
3383            &store_counts,
3384            &mut purges_zero_lamports,
3385        );
3386        purge_filter.stop();
3387
3388        let mut reclaims_time = Measure::start("reclaims");
3389        // Recalculate reclaims with new purge set
3390        let pubkey_to_slot_set: Vec<_> = purges_zero_lamports
3391            .into_iter()
3392            .map(|(key, (slots_list, _ref_count))| {
3393                (
3394                    key,
3395                    slots_list
3396                        .into_iter()
3397                        .map(|(slot, _)| slot)
3398                        .collect::<HashSet<Slot>>(),
3399                )
3400            })
3401            .collect();
3402
3403        let (reclaims, pubkeys_removed_from_accounts_index2) =
3404            self.purge_keys_exact(pubkey_to_slot_set.iter());
3405        pubkeys_removed_from_accounts_index
3406            .extend(pubkeys_removed_from_accounts_index2.into_iter());
3407
3408        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
3409        // and those stores may be used for background hashing.
3410        let reset_accounts = false;
3411        let mut reclaim_result = ReclaimResult::default();
3412        self.handle_reclaims(
3413            (!reclaims.is_empty()).then(|| reclaims.iter()),
3414            None,
3415            Some((&self.clean_accounts_stats.purge_stats, &mut reclaim_result)),
3416            reset_accounts,
3417            &pubkeys_removed_from_accounts_index,
3418        );
3419
3420        reclaims_time.stop();
3421        measure_all.stop();
3422
3423        self.clean_accounts_stats.report();
3424        datapoint_info!(
3425            "clean_accounts",
3426            ("total_us", measure_all.as_us(), i64),
3427            (
3428                "collect_delta_keys_us",
3429                key_timings.collect_delta_keys_us,
3430                i64
3431            ),
3432            ("oldest_dirty_slot", key_timings.oldest_dirty_slot, i64),
3433            (
3434                "pubkeys_removed_from_accounts_index",
3435                pubkeys_removed_from_accounts_index.len(),
3436                i64
3437            ),
3438            (
3439                "dirty_ancient_stores",
3440                key_timings.dirty_ancient_stores,
3441                i64
3442            ),
3443            (
3444                "dirty_store_processing_us",
3445                key_timings.dirty_store_processing_us,
3446                i64
3447            ),
3448            ("accounts_scan", accounts_scan.as_us() as i64, i64),
3449            ("clean_old_rooted", clean_old_rooted.as_us() as i64, i64),
3450            ("store_counts", store_counts_time.as_us() as i64, i64),
3451            ("purge_filter", purge_filter.as_us() as i64, i64),
3452            ("calc_deps", calc_deps_time.as_us() as i64, i64),
3453            ("reclaims", reclaims_time.as_us() as i64, i64),
3454            ("delta_insert_us", key_timings.delta_insert_us, i64),
3455            ("delta_key_count", key_timings.delta_key_count, i64),
3456            ("dirty_pubkeys_count", key_timings.dirty_pubkeys_count, i64),
3457            ("sort_us", sort.as_us(), i64),
3458            ("useful_keys", useful_accum.load(Ordering::Relaxed), i64),
3459            ("total_keys_count", total_keys_count, i64),
3460            (
3461                "scan_found_not_zero",
3462                found_not_zero_accum.load(Ordering::Relaxed),
3463                i64
3464            ),
3465            (
3466                "scan_not_found_on_fork",
3467                not_found_on_fork_accum.load(Ordering::Relaxed),
3468                i64
3469            ),
3470            ("scan_missing", missing_accum.load(Ordering::Relaxed), i64),
3471            ("uncleaned_roots_len", uncleaned_roots.len(), i64),
3472            (
3473                "clean_old_root_us",
3474                self.clean_accounts_stats
3475                    .clean_old_root_us
3476                    .swap(0, Ordering::Relaxed),
3477                i64
3478            ),
3479            (
3480                "clean_old_root_reclaim_us",
3481                self.clean_accounts_stats
3482                    .clean_old_root_reclaim_us
3483                    .swap(0, Ordering::Relaxed),
3484                i64
3485            ),
3486            (
3487                "reset_uncleaned_roots_us",
3488                self.clean_accounts_stats
3489                    .reset_uncleaned_roots_us
3490                    .swap(0, Ordering::Relaxed),
3491                i64
3492            ),
3493            (
3494                "remove_dead_accounts_remove_us",
3495                self.clean_accounts_stats
3496                    .remove_dead_accounts_remove_us
3497                    .swap(0, Ordering::Relaxed),
3498                i64
3499            ),
3500            (
3501                "remove_dead_accounts_shrink_us",
3502                self.clean_accounts_stats
3503                    .remove_dead_accounts_shrink_us
3504                    .swap(0, Ordering::Relaxed),
3505                i64
3506            ),
3507            (
3508                "clean_stored_dead_slots_us",
3509                self.clean_accounts_stats
3510                    .clean_stored_dead_slots_us
3511                    .swap(0, Ordering::Relaxed),
3512                i64
3513            ),
3514            (
3515                "roots_added",
3516                self.accounts_index.roots_added.swap(0, Ordering::Relaxed) as i64,
3517                i64
3518            ),
3519            (
3520                "roots_removed",
3521                self.accounts_index.roots_removed.swap(0, Ordering::Relaxed) as i64,
3522                i64
3523            ),
3524            (
3525                "active_scans",
3526                self.accounts_index.active_scans.load(Ordering::Relaxed) as i64,
3527                i64
3528            ),
3529            (
3530                "max_distance_to_min_scan_slot",
3531                self.accounts_index
3532                    .max_distance_to_min_scan_slot
3533                    .swap(0, Ordering::Relaxed),
3534                i64
3535            ),
3536            (
3537                "ancient_account_cleans",
3538                ancient_account_cleans.load(Ordering::Relaxed),
3539                i64
3540            ),
3541            ("next_store_id", self.next_id.load(Ordering::Relaxed), i64),
3542        );
3543    }
3544
3545    /// Removes the accounts in the input `reclaims` from the tracked "count" of
3546    /// their corresponding  storage entries. Note this does not actually free
3547    /// the memory from the storage entries until all the storage entries for
3548    /// a given slot `S` are empty, at which point `process_dead_slots` will
3549    /// remove all the storage entries for `S`.
3550    ///
3551    /// # Arguments
3552    /// * `reclaims` - The accounts to remove from storage entries' "count". Note here
3553    ///    that we should not remove cache entries, only entries for accounts actually
3554    ///    stored in a storage entry.
3555    ///
3556    /// * `expected_single_dead_slot` - A correctness assertion. If this is equal to `Some(S)`,
3557    ///    then the function will check that the only slot being cleaned up in `reclaims`
3558    ///    is the slot == `S`. This is true for instance when `handle_reclaims` is called
3559    ///    from store or slot shrinking, as those should only touch the slot they are
3560    ///    currently storing to or shrinking.
3561    ///
3562    /// * `purge_stats_and_reclaim_result` - Option containing `purge_stats` and `reclaim_result`.
3563    ///    `purge_stats`. `purge_stats` are stats used to track performance of purging dead slots.
3564    ///    `reclaim_result` contains information about accounts that were removed from storage,
3565    ///    does not include accounts that were removed from the cache.
3566    ///    If `purge_stats_and_reclaim_result.is_none()`, this implies there can be no dead slots
3567    ///    that happen as a result of this call, and the function will check that no slots are
3568    ///    cleaned up/removed via `process_dead_slots`. For instance, on store, no slots should
3569    ///    be cleaned up, but during the background clean accounts purges accounts from old rooted
3570    ///    slots, so outdated slots may be removed.
3571    ///
3572    /// * `reset_accounts` - Reset the append_vec store when the store is dead (count==0)
3573    ///    From the clean and shrink paths it should be false since there may be an in-progress
3574    ///    hash operation and the stores may hold accounts that need to be unref'ed.
3575    /// * `pubkeys_removed_from_accounts_index` - These keys have already been removed from the accounts index
3576    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
3577    fn handle_reclaims<'a, I>(
3578        &'a self,
3579        reclaims: Option<I>,
3580        expected_single_dead_slot: Option<Slot>,
3581        purge_stats_and_reclaim_result: Option<(&PurgeStats, &mut ReclaimResult)>,
3582        reset_accounts: bool,
3583        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3584    ) where
3585        I: Iterator<Item = &'a (Slot, AccountInfo)>,
3586    {
3587        if let Some(reclaims) = reclaims {
3588            let (purge_stats, purged_account_slots, reclaimed_offsets) = if let Some((
3589                purge_stats,
3590                (ref mut purged_account_slots, ref mut reclaimed_offsets),
3591            )) =
3592                purge_stats_and_reclaim_result
3593            {
3594                (
3595                    Some(purge_stats),
3596                    Some(purged_account_slots),
3597                    Some(reclaimed_offsets),
3598                )
3599            } else {
3600                (None, None, None)
3601            };
3602
3603            let dead_slots = self.remove_dead_accounts(
3604                reclaims,
3605                expected_single_dead_slot,
3606                reclaimed_offsets,
3607                reset_accounts,
3608            );
3609
3610            if let Some(purge_stats) = purge_stats {
3611                if let Some(expected_single_dead_slot) = expected_single_dead_slot {
3612                    assert!(dead_slots.len() <= 1);
3613                    if dead_slots.len() == 1 {
3614                        assert!(dead_slots.contains(&expected_single_dead_slot));
3615                    }
3616                }
3617
3618                self.process_dead_slots(
3619                    &dead_slots,
3620                    purged_account_slots,
3621                    purge_stats,
3622                    pubkeys_removed_from_accounts_index,
3623                );
3624            } else {
3625                assert!(dead_slots.is_empty());
3626            }
3627        }
3628    }
3629
3630    /// During clean, some zero-lamport accounts that are marked for purge should *not* actually
3631    /// get purged.  Filter out those accounts here by removing them from 'purges_zero_lamports'
3632    ///
3633    /// When using incremental snapshots, do not purge zero-lamport accounts if the slot is higher
3634    /// than the last full snapshot slot.  This is to protect against the following scenario:
3635    ///
3636    ///   ```text
3637    ///   A full snapshot is taken, including account 'alpha' with a non-zero balance.  In a later slot,
3638    ///   alpha's lamports go to zero.  Eventually, cleaning runs.  Without this change,
3639    ///   alpha would be cleaned up and removed completely. Finally, an incremental snapshot is taken.
3640    ///
3641    ///   Later, the incremental and full snapshots are used to rebuild the bank and accounts
3642    ///   database (e.x. if the node restarts).  The full snapshot _does_ contain alpha
3643    ///   and its balance is non-zero.  However, since alpha was cleaned up in a slot after the full
3644    ///   snapshot slot (due to having zero lamports), the incremental snapshot would not contain alpha.
3645    ///   Thus, the accounts database will contain the old, incorrect info for alpha with a non-zero
3646    ///   balance.  Very bad!
3647    ///   ```
3648    ///
3649    /// This filtering step can be skipped if there is no `last_full_snapshot_slot`, or if the
3650    /// `max_clean_root_inclusive` is less-than-or-equal-to the `last_full_snapshot_slot`.
3651    fn filter_zero_lamport_clean_for_incremental_snapshots(
3652        &self,
3653        max_clean_root_inclusive: Option<Slot>,
3654        last_full_snapshot_slot: Option<Slot>,
3655        store_counts: &HashMap<AppendVecId, (usize, HashSet<Pubkey>)>,
3656        purges_zero_lamports: &mut HashMap<Pubkey, (SlotList<AccountInfo>, RefCount)>,
3657    ) {
3658        let should_filter_for_incremental_snapshots = max_clean_root_inclusive.unwrap_or(Slot::MAX)
3659            > last_full_snapshot_slot.unwrap_or(Slot::MAX);
3660        assert!(
3661            last_full_snapshot_slot.is_some() || !should_filter_for_incremental_snapshots,
3662            "if filtering for incremental snapshots, then snapshots should be enabled",
3663        );
3664
3665        purges_zero_lamports.retain(|pubkey, (slot_account_infos, _ref_count)| {
3666            // Only keep purges_zero_lamports where the entire history of the account in the root set
3667            // can be purged. All AppendVecs for those updates are dead.
3668            for (_slot, account_info) in slot_account_infos.iter() {
3669                if let Some(store_count) = store_counts.get(&account_info.store_id()) {
3670                    if store_count.0 != 0 {
3671                        // one store this pubkey is in is not being removed, so this pubkey cannot be removed at all
3672                        return false;
3673                    }
3674                } else {
3675                    // store is not being removed, so this pubkey cannot be removed at all
3676                    return false;
3677                }
3678            }
3679
3680            // Exit early if not filtering more for incremental snapshots
3681            if !should_filter_for_incremental_snapshots {
3682                return true;
3683            }
3684
3685            let slot_account_info_at_highest_slot = slot_account_infos
3686                .iter()
3687                .max_by_key(|(slot, _account_info)| slot);
3688
3689            slot_account_info_at_highest_slot.map_or(true, |(slot, account_info)| {
3690                // Do *not* purge zero-lamport accounts if the slot is greater than the last full
3691                // snapshot slot.  Since we're `retain`ing the accounts-to-purge, I felt creating
3692                // the `cannot_purge` variable made this easier to understand.  Accounts that do
3693                // not get purged here are added to a list so they be considered for purging later
3694                // (i.e. after the next full snapshot).
3695                assert!(account_info.is_zero_lamport());
3696                let cannot_purge = *slot > last_full_snapshot_slot.unwrap();
3697                if cannot_purge {
3698                    self.zero_lamport_accounts_to_purge_after_full_snapshot
3699                        .insert((*slot, *pubkey));
3700                }
3701                !cannot_purge
3702            })
3703        });
3704    }
3705
3706    // Must be kept private!, does sensitive cleanup that should only be called from
3707    // supported pipelines in AccountsDb
3708    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
3709    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
3710    fn process_dead_slots(
3711        &self,
3712        dead_slots: &HashSet<Slot>,
3713        purged_account_slots: Option<&mut AccountSlots>,
3714        purge_stats: &PurgeStats,
3715        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3716    ) {
3717        if dead_slots.is_empty() {
3718            return;
3719        }
3720        let mut clean_dead_slots = Measure::start("reclaims::clean_dead_slots");
3721        self.clean_stored_dead_slots(
3722            dead_slots,
3723            purged_account_slots,
3724            pubkeys_removed_from_accounts_index,
3725        );
3726        clean_dead_slots.stop();
3727
3728        let mut purge_removed_slots = Measure::start("reclaims::purge_removed_slots");
3729        self.purge_dead_slots_from_storage(dead_slots.iter(), purge_stats);
3730        purge_removed_slots.stop();
3731
3732        // If the slot is dead, remove the need to shrink the storages as
3733        // the storage entries will be purged.
3734        {
3735            let mut list = self.shrink_candidate_slots.lock().unwrap();
3736            for slot in dead_slots {
3737                list.remove(slot);
3738            }
3739        }
3740
3741        debug!(
3742            "process_dead_slots({}): {} {} {:?}",
3743            dead_slots.len(),
3744            clean_dead_slots,
3745            purge_removed_slots,
3746            dead_slots,
3747        );
3748    }
3749
3750    /// load the account index entry for the first `count` items in `accounts`
3751    /// store a reference to all alive accounts in `alive_accounts`
3752    /// unref and optionally store a reference to all pubkeys that are in the index, but dead in `unrefed_pubkeys`
3753    /// return sum of account size for all alive accounts
3754    fn load_accounts_index_for_shrink<'a, T: ShrinkCollectRefs<'a>>(
3755        &self,
3756        accounts: &'a [StoredAccountMeta<'a>],
3757        stats: &ShrinkStats,
3758        slot_to_shrink: Slot,
3759    ) -> LoadAccountsIndexForShrink<'a, T> {
3760        let count = accounts.len();
3761        let mut alive_accounts = T::with_capacity(count, slot_to_shrink);
3762        let mut unrefed_pubkeys = Vec::with_capacity(count);
3763
3764        let mut alive = 0;
3765        let mut dead = 0;
3766        let mut index = 0;
3767        let mut all_are_zero_lamports = true;
3768        self.accounts_index.scan(
3769            accounts.iter().map(|account| account.pubkey()),
3770            |pubkey, slots_refs| {
3771                let mut result = AccountsIndexScanResult::None;
3772                if let Some((slot_list, ref_count)) = slots_refs {
3773                    let stored_account = &accounts[index];
3774                    let is_alive = slot_list.iter().any(|(slot, _acct_info)| {
3775                        // if the accounts index contains an entry at this slot, then the append vec we're asking about contains this item and thus, it is alive at this slot
3776                        *slot == slot_to_shrink
3777                    });
3778                    if !is_alive {
3779                        // This pubkey was found in the storage, but no longer exists in the index.
3780                        // It would have had a ref to the storage from the initial store, but it will
3781                        // not exist in the re-written slot. Unref it to keep the index consistent with
3782                        // rewriting the storage entries.
3783                        unrefed_pubkeys.push(pubkey);
3784                        result = AccountsIndexScanResult::Unref;
3785                        dead += 1;
3786                    } else {
3787                        all_are_zero_lamports &= stored_account.lamports() == 0;
3788                        alive_accounts.add(ref_count, stored_account);
3789                        alive += 1;
3790                    }
3791                }
3792                index += 1;
3793                result
3794            },
3795            None,
3796        );
3797        assert_eq!(index, std::cmp::min(accounts.len(), count));
3798        stats.alive_accounts.fetch_add(alive, Ordering::Relaxed);
3799        stats.dead_accounts.fetch_add(dead, Ordering::Relaxed);
3800
3801        LoadAccountsIndexForShrink {
3802            alive_accounts,
3803            unrefed_pubkeys,
3804            all_are_zero_lamports,
3805        }
3806    }
3807
3808    /// get all accounts in all the storages passed in
3809    /// for duplicate pubkeys, the account with the highest write_value is returned
3810    pub(crate) fn get_unique_accounts_from_storage<'a>(
3811        &self,
3812        store: &'a Arc<AccountStorageEntry>,
3813    ) -> GetUniqueAccountsResult<'a> {
3814        let mut stored_accounts: HashMap<Pubkey, StoredAccountMeta> = HashMap::new();
3815        let capacity = store.capacity();
3816        store.accounts.account_iter().for_each(|account| {
3817            stored_accounts.insert(*account.pubkey(), account);
3818        });
3819
3820        // sort by pubkey to keep account index lookups close
3821        let mut stored_accounts = stored_accounts.drain().map(|(_k, v)| v).collect::<Vec<_>>();
3822        stored_accounts.sort_unstable_by(|a, b| a.pubkey().cmp(b.pubkey()));
3823
3824        GetUniqueAccountsResult {
3825            stored_accounts,
3826            capacity,
3827        }
3828    }
3829
3830    pub(crate) fn get_unique_accounts_from_storage_for_shrink<'a>(
3831        &self,
3832        store: &'a Arc<AccountStorageEntry>,
3833        stats: &ShrinkStats,
3834    ) -> GetUniqueAccountsResult<'a> {
3835        let (result, storage_read_elapsed_us) =
3836            measure_us!(self.get_unique_accounts_from_storage(store));
3837        stats
3838            .storage_read_elapsed
3839            .fetch_add(storage_read_elapsed_us, Ordering::Relaxed);
3840        result
3841    }
3842
3843    /// shared code for shrinking normal slots and combining into ancient append vecs
3844    /// note 'unique_accounts' is passed by ref so we can return references to data within it, avoiding self-references
3845    pub(crate) fn shrink_collect<'a: 'b, 'b, T: ShrinkCollectRefs<'b>>(
3846        &self,
3847        store: &'a Arc<AccountStorageEntry>,
3848        unique_accounts: &'b GetUniqueAccountsResult<'b>,
3849        stats: &ShrinkStats,
3850    ) -> ShrinkCollect<'b, T> {
3851        let slot = store.slot();
3852
3853        let GetUniqueAccountsResult {
3854            stored_accounts,
3855            capacity,
3856        } = unique_accounts;
3857
3858        let mut index_read_elapsed = Measure::start("index_read_elapsed");
3859
3860        let len = stored_accounts.len();
3861        let alive_accounts_collect = Mutex::new(T::with_capacity(len, slot));
3862        let unrefed_pubkeys_collect = Mutex::new(Vec::with_capacity(len));
3863        stats
3864            .accounts_loaded
3865            .fetch_add(len as u64, Ordering::Relaxed);
3866        let all_are_zero_lamports_collect = Mutex::new(true);
3867        self.thread_pool_clean.install(|| {
3868            stored_accounts
3869                .par_chunks(SHRINK_COLLECT_CHUNK_SIZE)
3870                .for_each(|stored_accounts| {
3871                    let LoadAccountsIndexForShrink {
3872                        alive_accounts,
3873                        mut unrefed_pubkeys,
3874                        all_are_zero_lamports,
3875                    } = self.load_accounts_index_for_shrink(stored_accounts, stats, slot);
3876
3877                    // collect
3878                    alive_accounts_collect
3879                        .lock()
3880                        .unwrap()
3881                        .collect(alive_accounts);
3882                    unrefed_pubkeys_collect
3883                        .lock()
3884                        .unwrap()
3885                        .append(&mut unrefed_pubkeys);
3886                    if !all_are_zero_lamports {
3887                        *all_are_zero_lamports_collect.lock().unwrap() = false;
3888                    }
3889                });
3890        });
3891
3892        let alive_accounts = alive_accounts_collect.into_inner().unwrap();
3893        let unrefed_pubkeys = unrefed_pubkeys_collect.into_inner().unwrap();
3894
3895        index_read_elapsed.stop();
3896        stats
3897            .index_read_elapsed
3898            .fetch_add(index_read_elapsed.as_us(), Ordering::Relaxed);
3899
3900        let alive_total_bytes = alive_accounts.alive_bytes();
3901
3902        let aligned_total_bytes: u64 = Self::page_align(alive_total_bytes as u64);
3903
3904        stats
3905            .accounts_removed
3906            .fetch_add(len - alive_accounts.len(), Ordering::Relaxed);
3907        stats.bytes_removed.fetch_add(
3908            capacity.saturating_sub(aligned_total_bytes),
3909            Ordering::Relaxed,
3910        );
3911        stats
3912            .bytes_written
3913            .fetch_add(aligned_total_bytes, Ordering::Relaxed);
3914
3915        ShrinkCollect {
3916            slot,
3917            capacity: *capacity,
3918            aligned_total_bytes,
3919            unrefed_pubkeys,
3920            alive_accounts,
3921            alive_total_bytes,
3922            total_starting_accounts: len,
3923            all_are_zero_lamports: all_are_zero_lamports_collect.into_inner().unwrap(),
3924        }
3925    }
3926
3927    /// common code from shrink and combine_ancient_slots
3928    /// get rid of all original store_ids in the slot
3929    pub(crate) fn remove_old_stores_shrink<'a, T: ShrinkCollectRefs<'a>>(
3930        &self,
3931        shrink_collect: &ShrinkCollect<'a, T>,
3932        stats: &ShrinkStats,
3933        shrink_in_progress: Option<ShrinkInProgress>,
3934        shrink_can_be_active: bool,
3935    ) {
3936        let mut time = Measure::start("remove_old_stores_shrink");
3937        // Purge old, overwritten storage entries
3938        let dead_storages = self.mark_dirty_dead_stores(
3939            shrink_collect.slot,
3940            // If all accounts are zero lamports, then we want to mark the entire OLD append vec as dirty.
3941            // otherwise, we'll call 'add_uncleaned_pubkeys_after_shrink' just on the unref'd keys below.
3942            shrink_collect.all_are_zero_lamports,
3943            shrink_in_progress,
3944            shrink_can_be_active,
3945        );
3946
3947        if !shrink_collect.all_are_zero_lamports {
3948            self.add_uncleaned_pubkeys_after_shrink(
3949                shrink_collect.slot,
3950                shrink_collect.unrefed_pubkeys.iter().cloned().cloned(),
3951            );
3952        }
3953
3954        self.drop_or_recycle_stores(dead_storages, stats);
3955        time.stop();
3956
3957        stats
3958            .remove_old_stores_shrink_us
3959            .fetch_add(time.as_us(), Ordering::Relaxed);
3960    }
3961
3962    fn do_shrink_slot_store(&self, slot: Slot, store: &Arc<AccountStorageEntry>) {
3963        if self.accounts_cache.contains(slot) {
3964            // It is not correct to shrink a slot while it is in the write cache until flush is complete and the slot is removed from the write cache.
3965            // There can exist a window after a slot is made a root and before the write cache flushing for that slot begins and then completes.
3966            // There can also exist a window after a slot is being flushed from the write cache until the index is updated and the slot is removed from the write cache.
3967            // During the second window, once an append vec has been created for the slot, it could be possible to try to shrink that slot.
3968            // Shrink no-ops before this function if there is no store for the slot (notice this function requires 'store' to be passed).
3969            // So, if we enter this function but the slot is still in the write cache, reasonable behavior is to skip shrinking this slot.
3970            // Flush will ONLY write alive accounts to the append vec, which is what shrink does anyway.
3971            // Flush then adds the slot to 'uncleaned_roots', which causes clean to take a look at the slot.
3972            // Clean causes us to mark accounts as dead, which causes shrink to later take a look at the slot.
3973            // This could be an assert, but it could lead to intermittency in tests.
3974            // It is 'correct' to ignore calls to shrink when a slot is still in the write cache.
3975            return;
3976        }
3977        let unique_accounts =
3978            self.get_unique_accounts_from_storage_for_shrink(store, &self.shrink_stats);
3979        debug!("do_shrink_slot_store: slot: {}", slot);
3980        let shrink_collect =
3981            self.shrink_collect::<AliveAccounts<'_>>(store, &unique_accounts, &self.shrink_stats);
3982
3983        // This shouldn't happen if alive_bytes/approx_stored_count are accurate
3984        if Self::should_not_shrink(shrink_collect.aligned_total_bytes, shrink_collect.capacity) {
3985            self.shrink_stats
3986                .skipped_shrink
3987                .fetch_add(1, Ordering::Relaxed);
3988            for pubkey in shrink_collect.unrefed_pubkeys {
3989                if let Some(locked_entry) = self.accounts_index.get_account_read_entry(pubkey) {
3990                    locked_entry.addref();
3991                }
3992            }
3993            return;
3994        }
3995
3996        let total_accounts_after_shrink = shrink_collect.alive_accounts.len();
3997        debug!(
3998            "shrinking: slot: {}, accounts: ({} => {}) bytes: ({} ; aligned to: {}) original: {}",
3999            slot,
4000            shrink_collect.total_starting_accounts,
4001            total_accounts_after_shrink,
4002            shrink_collect.alive_total_bytes,
4003            shrink_collect.aligned_total_bytes,
4004            shrink_collect.capacity,
4005        );
4006
4007        let mut stats_sub = ShrinkStatsSub::default();
4008        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
4009        if shrink_collect.aligned_total_bytes > 0 {
4010            let (shrink_in_progress, time_us) =
4011                measure_us!(self.get_store_for_shrink(slot, shrink_collect.aligned_total_bytes));
4012            stats_sub.create_and_insert_store_elapsed_us = time_us;
4013
4014            // here, we're writing back alive_accounts. That should be an atomic operation
4015            // without use of rather wide locks in this whole function, because we're
4016            // mutating rooted slots; There should be no writers to them.
4017            stats_sub.store_accounts_timing = self.store_accounts_frozen(
4018                (
4019                    slot,
4020                    &shrink_collect.alive_accounts.alive_accounts()[..],
4021                    INCLUDE_SLOT_IN_HASH_IRRELEVANT_APPEND_VEC_OPERATION,
4022                ),
4023                None::<Vec<&Hash>>,
4024                shrink_in_progress.new_storage(),
4025                None,
4026                StoreReclaims::Ignore,
4027            );
4028
4029            rewrite_elapsed.stop();
4030            stats_sub.rewrite_elapsed_us = rewrite_elapsed.as_us();
4031
4032            // `store_accounts_frozen()` above may have purged accounts from some
4033            // other storage entries (the ones that were just overwritten by this
4034            // new storage entry). This means some of those stores might have caused
4035            // this slot to be read to `self.shrink_candidate_slots`, so delete
4036            // those here
4037            self.shrink_candidate_slots.lock().unwrap().remove(&slot);
4038
4039            self.remove_old_stores_shrink(
4040                &shrink_collect,
4041                &self.shrink_stats,
4042                Some(shrink_in_progress),
4043                false,
4044            );
4045        }
4046
4047        Self::update_shrink_stats(&self.shrink_stats, stats_sub);
4048        self.shrink_stats.report();
4049    }
4050
4051    pub(crate) fn update_shrink_stats(shrink_stats: &ShrinkStats, stats_sub: ShrinkStatsSub) {
4052        shrink_stats
4053            .num_slots_shrunk
4054            .fetch_add(1, Ordering::Relaxed);
4055        shrink_stats.create_and_insert_store_elapsed.fetch_add(
4056            stats_sub.create_and_insert_store_elapsed_us,
4057            Ordering::Relaxed,
4058        );
4059        shrink_stats.store_accounts_elapsed.fetch_add(
4060            stats_sub.store_accounts_timing.store_accounts_elapsed,
4061            Ordering::Relaxed,
4062        );
4063        shrink_stats.update_index_elapsed.fetch_add(
4064            stats_sub.store_accounts_timing.update_index_elapsed,
4065            Ordering::Relaxed,
4066        );
4067        shrink_stats.handle_reclaims_elapsed.fetch_add(
4068            stats_sub.store_accounts_timing.handle_reclaims_elapsed,
4069            Ordering::Relaxed,
4070        );
4071        shrink_stats
4072            .rewrite_elapsed
4073            .fetch_add(stats_sub.rewrite_elapsed_us, Ordering::Relaxed);
4074    }
4075
4076    /// get stores for 'slot'
4077    /// Drop 'shrink_in_progress', which will cause the old store to be removed from the storage map.
4078    /// For 'shrink_in_progress'.'old_storage' which is not retained, insert in 'dead_storages' and optionally 'dirty_stores'
4079    /// This is the end of the life cycle of `shrink_in_progress`.
4080    pub(crate) fn mark_dirty_dead_stores(
4081        &self,
4082        slot: Slot,
4083        add_dirty_stores: bool,
4084        shrink_in_progress: Option<ShrinkInProgress>,
4085        shrink_can_be_active: bool,
4086    ) -> Vec<Arc<AccountStorageEntry>> {
4087        let mut dead_storages = Vec::default();
4088
4089        let mut not_retaining_store = |store: &Arc<AccountStorageEntry>| {
4090            if add_dirty_stores {
4091                self.dirty_stores
4092                    .insert((slot, store.append_vec_id()), store.clone());
4093            }
4094            dead_storages.push(store.clone());
4095        };
4096
4097        if let Some(shrink_in_progress) = shrink_in_progress {
4098            // shrink is in progress, so 1 new append vec to keep, 1 old one to throw away
4099            not_retaining_store(shrink_in_progress.old_storage());
4100            // dropping 'shrink_in_progress' removes the old append vec that was being shrunk from db's storage
4101        } else if let Some(store) = self.storage.remove(&slot, shrink_can_be_active) {
4102            // no shrink in progress, so all append vecs in this slot are dead
4103            not_retaining_store(&store);
4104        }
4105
4106        dead_storages
4107    }
4108
4109    pub(crate) fn drop_or_recycle_stores(
4110        &self,
4111        dead_storages: Vec<Arc<AccountStorageEntry>>,
4112        stats: &ShrinkStats,
4113    ) {
4114        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
4115        let mut recycle_stores = self.recycle_stores.write().unwrap();
4116        recycle_stores_write_elapsed.stop();
4117
4118        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
4119        if recycle_stores.entry_count() < MAX_RECYCLE_STORES {
4120            recycle_stores.add_entries(dead_storages);
4121            drop(recycle_stores);
4122        } else {
4123            self.stats
4124                .dropped_stores
4125                .fetch_add(dead_storages.len() as u64, Ordering::Relaxed);
4126            drop(recycle_stores);
4127            drop(dead_storages);
4128        }
4129        drop_storage_entries_elapsed.stop();
4130        stats
4131            .drop_storage_entries_elapsed
4132            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
4133        stats
4134            .recycle_stores_write_elapsed
4135            .fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
4136    }
4137
4138    /// return a store that can contain 'aligned_total' bytes
4139    pub(crate) fn get_store_for_shrink(
4140        &self,
4141        slot: Slot,
4142        aligned_total: u64,
4143    ) -> ShrinkInProgress<'_> {
4144        let shrunken_store = self
4145            .try_recycle_store(slot, aligned_total, aligned_total + 1024)
4146            .unwrap_or_else(|| {
4147                let maybe_shrink_paths = self.shrink_paths.read().unwrap();
4148                let (shrink_paths, from) = maybe_shrink_paths
4149                    .as_ref()
4150                    .map(|paths| (paths, "shrink-w-path"))
4151                    .unwrap_or_else(|| (&self.paths, "shrink"));
4152                self.create_store(slot, aligned_total, from, shrink_paths)
4153            });
4154        self.storage.shrinking_in_progress(slot, shrunken_store)
4155    }
4156
4157    // Reads all accounts in given slot's AppendVecs and filter only to alive,
4158    // then create a minimum AppendVec filled with the alive.
4159    fn shrink_slot_forced(&self, slot: Slot) {
4160        debug!("shrink_slot_forced: slot: {}", slot);
4161
4162        if let Some(store) = self
4163            .storage
4164            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
4165        {
4166            if !Self::is_shrinking_productive(slot, &store) {
4167                return;
4168            }
4169            self.do_shrink_slot_store(slot, &store)
4170        }
4171    }
4172
4173    fn all_slots_in_storage(&self) -> Vec<Slot> {
4174        self.storage.all_slots()
4175    }
4176
4177    /// Given the input `ShrinkCandidates`, this function sorts the stores by their alive ratio
4178    /// in increasing order with the most sparse entries in the front. It will then simulate the
4179    /// shrinking by working on the most sparse entries first and if the overall alive ratio is
4180    /// achieved, it will stop and return the filtered-down candidates and the candidates which
4181    /// are skipped in this round and might be eligible for the future shrink.
4182    fn select_candidates_by_total_usage(
4183        shrink_slots: &ShrinkCandidates,
4184        shrink_ratio: f64,
4185        newest_ancient_slot: Option<Slot>,
4186    ) -> (ShrinkCandidates, ShrinkCandidates) {
4187        struct StoreUsageInfo {
4188            slot: Slot,
4189            alive_ratio: f64,
4190            store: Arc<AccountStorageEntry>,
4191        }
4192        let mut measure = Measure::start("select_top_sparse_storage_entries-ms");
4193        let mut store_usage: Vec<StoreUsageInfo> = Vec::with_capacity(shrink_slots.len());
4194        let mut total_alive_bytes: u64 = 0;
4195        let mut candidates_count: usize = 0;
4196        let mut total_bytes: u64 = 0;
4197        let mut total_candidate_stores: usize = 0;
4198        for (slot, store) in shrink_slots {
4199            if newest_ancient_slot
4200                .map(|newest_ancient_slot| slot <= &newest_ancient_slot)
4201                .unwrap_or_default()
4202            {
4203                // this slot will be 'shrunk' by ancient code
4204                continue;
4205            }
4206            candidates_count += 1;
4207            total_alive_bytes += Self::page_align(store.alive_bytes() as u64);
4208            total_bytes += store.capacity();
4209            let alive_ratio =
4210                Self::page_align(store.alive_bytes() as u64) as f64 / store.capacity() as f64;
4211            store_usage.push(StoreUsageInfo {
4212                slot: *slot,
4213                alive_ratio,
4214                store: store.clone(),
4215            });
4216            total_candidate_stores += 1;
4217        }
4218        store_usage.sort_by(|a, b| {
4219            a.alive_ratio
4220                .partial_cmp(&b.alive_ratio)
4221                .unwrap_or(std::cmp::Ordering::Equal)
4222        });
4223
4224        // Working from the beginning of store_usage which are the most sparse and see when we can stop
4225        // shrinking while still achieving the overall goals.
4226        let mut shrink_slots: ShrinkCandidates = HashMap::new();
4227        let mut shrink_slots_next_batch: ShrinkCandidates = HashMap::new();
4228        for usage in &store_usage {
4229            let store = &usage.store;
4230            let alive_ratio = (total_alive_bytes as f64) / (total_bytes as f64);
4231            debug!("alive_ratio: {:?} store_id: {:?}, store_ratio: {:?} requirment: {:?}, total_bytes: {:?} total_alive_bytes: {:?}",
4232                alive_ratio, usage.store.append_vec_id(), usage.alive_ratio, shrink_ratio, total_bytes, total_alive_bytes);
4233            if alive_ratio > shrink_ratio {
4234                // we have reached our goal, stop
4235                debug!(
4236                    "Shrinking goal can be achieved at slot {:?}, total_alive_bytes: {:?} \
4237                    total_bytes: {:?}, alive_ratio: {:}, shrink_ratio: {:?}",
4238                    usage.slot, total_alive_bytes, total_bytes, alive_ratio, shrink_ratio
4239                );
4240                if usage.alive_ratio < shrink_ratio {
4241                    shrink_slots_next_batch.insert(usage.slot, store.clone());
4242                } else {
4243                    break;
4244                }
4245            } else {
4246                let current_store_size = store.capacity();
4247                let after_shrink_size = Self::page_align(store.alive_bytes() as u64);
4248                let bytes_saved = current_store_size.saturating_sub(after_shrink_size);
4249                total_bytes -= bytes_saved;
4250                shrink_slots.insert(usage.slot, store.clone());
4251            }
4252        }
4253        measure.stop();
4254        inc_new_counter_debug!(
4255            "shrink_select_top_sparse_storage_entries-ms",
4256            measure.as_ms() as usize
4257        );
4258        inc_new_counter_debug!(
4259            "shrink_select_top_sparse_storage_entries-seeds",
4260            candidates_count
4261        );
4262        inc_new_counter_debug!(
4263            "shrink_total_preliminary_candidate_stores",
4264            total_candidate_stores
4265        );
4266
4267        (shrink_slots, shrink_slots_next_batch)
4268    }
4269
4270    fn get_roots_less_than(&self, slot: Slot) -> Vec<Slot> {
4271        self.accounts_index
4272            .roots_tracker
4273            .read()
4274            .unwrap()
4275            .alive_roots
4276            .get_all_less_than(slot)
4277    }
4278
4279    fn get_prior_root(&self, slot: Slot) -> Option<Slot> {
4280        self.accounts_index
4281            .roots_tracker
4282            .read()
4283            .unwrap()
4284            .alive_roots
4285            .get_prior(slot)
4286    }
4287
4288    /// return highest slot that should be treated as ancient
4289    fn get_newest_ancient_slot(&self) -> Slot {
4290        let mut reference_slot = self.get_accounts_hash_complete_one_epoch_old();
4291        if let Some(offset) = self.ancient_append_vec_offset {
4292            reference_slot = Self::apply_offset_to_slot(reference_slot, offset);
4293        }
4294        reference_slot
4295    }
4296
4297    /// return all slots that are more than one epoch old and thus could already be an ancient append vec
4298    /// or which could need to be combined into a new or existing ancient append vec
4299    /// offset is used to combine newer slots than we normally would. This is designed to be used for testing.
4300    fn get_sorted_potential_ancient_slots(&self) -> Vec<Slot> {
4301        let mut old_slots = self.get_roots_less_than(self.get_newest_ancient_slot());
4302        old_slots.sort_unstable();
4303        old_slots
4304    }
4305
4306    /// get a sorted list of slots older than an epoch
4307    /// squash those slots into ancient append vecs
4308    fn shrink_ancient_slots(&self) {
4309        if self.ancient_append_vec_offset.is_none() {
4310            return;
4311        }
4312
4313        let can_randomly_shrink = true;
4314        if self.create_ancient_storage == CreateAncientStorage::Append {
4315            self.combine_ancient_slots(
4316                self.get_sorted_potential_ancient_slots(),
4317                can_randomly_shrink,
4318            );
4319        } else {
4320            self.combine_ancient_slots_packed(
4321                self.get_sorted_potential_ancient_slots(),
4322                can_randomly_shrink,
4323            );
4324        }
4325    }
4326
4327    #[cfg(test)]
4328    pub(crate) fn sizes_of_accounts_in_storage_for_tests(&self, slot: Slot) -> Vec<usize> {
4329        self.storage
4330            .get_slot_storage_entry(slot)
4331            .map(|storage| {
4332                storage
4333                    .accounts
4334                    .account_iter()
4335                    .map(|account| account.stored_size)
4336                    .collect()
4337            })
4338            .unwrap_or_default()
4339    }
4340
4341    /// 'accounts' that exist in the current slot we are combining into a different ancient slot
4342    /// 'existing_ancient_pubkeys': pubkeys that exist currently in the ancient append vec slot
4343    /// returns the pubkeys that are in 'accounts' that are already in 'existing_ancient_pubkeys'
4344    /// Also updated 'existing_ancient_pubkeys' to include all pubkeys in 'accounts' since they will soon be written into the ancient slot.
4345    fn get_keys_to_unref_ancient<'a>(
4346        accounts: &'a [&StoredAccountMeta<'_>],
4347        existing_ancient_pubkeys: &mut HashSet<Pubkey>,
4348    ) -> HashSet<&'a Pubkey> {
4349        let mut unref = HashSet::<&Pubkey>::default();
4350        // for each key that we're about to add that already exists in this storage, we need to unref. The account was in a different storage.
4351        // Now it is being put into an ancient storage again, but it is already there, so maintain max of 1 ref per storage in the accounts index.
4352        // The slot that currently references the account is going away, so unref to maintain # slots that reference the pubkey = refcount.
4353        accounts.iter().for_each(|account| {
4354            let key = account.pubkey();
4355            if !existing_ancient_pubkeys.insert(*key) {
4356                // this key exists BOTH in 'accounts' and already in the ancient append vec, so we need to unref it
4357                unref.insert(key);
4358            }
4359        });
4360        unref
4361    }
4362
4363    /// 'accounts' are about to be appended to an ancient append vec. That ancient append vec may already have some accounts.
4364    /// Unref each account in 'accounts' that already exists in 'existing_ancient_pubkeys'.
4365    /// As a side effect, on exit, 'existing_ancient_pubkeys' will now contain all pubkeys in 'accounts'.
4366    fn unref_accounts_already_in_storage(
4367        &self,
4368        accounts: &[&StoredAccountMeta<'_>],
4369        existing_ancient_pubkeys: &mut HashSet<Pubkey>,
4370    ) {
4371        let unref = Self::get_keys_to_unref_ancient(accounts, existing_ancient_pubkeys);
4372
4373        self.unref_pubkeys(
4374            unref.iter().cloned(),
4375            unref.len(),
4376            &PubkeysRemovedFromAccountsIndex::default(),
4377        );
4378    }
4379
4380    /// get the storage from 'slot' to squash
4381    /// or None if this slot should be skipped
4382    /// side effect could be updating 'current_ancient'
4383    fn get_storage_to_move_to_ancient_append_vec(
4384        &self,
4385        slot: Slot,
4386        current_ancient: &mut CurrentAncientAppendVec,
4387        can_randomly_shrink: bool,
4388    ) -> Option<Arc<AccountStorageEntry>> {
4389        self.storage
4390            .get_slot_storage_entry(slot)
4391            .and_then(|storage| {
4392                self.should_move_to_ancient_append_vec(
4393                    &storage,
4394                    current_ancient,
4395                    slot,
4396                    can_randomly_shrink,
4397                )
4398                .then_some(storage)
4399            })
4400    }
4401
4402    /// return true if the accounts in this slot should be moved to an ancient append vec
4403    /// otherwise, return false and the caller can skip this slot
4404    /// side effect could be updating 'current_ancient'
4405    /// can_randomly_shrink: true if ancient append vecs that otherwise don't qualify to be shrunk can be randomly shrunk
4406    ///  this is convenient for a running system
4407    ///  this is not useful for testing
4408    fn should_move_to_ancient_append_vec(
4409        &self,
4410        storage: &Arc<AccountStorageEntry>,
4411        current_ancient: &mut CurrentAncientAppendVec,
4412        slot: Slot,
4413        can_randomly_shrink: bool,
4414    ) -> bool {
4415        let accounts = &storage.accounts;
4416
4417        self.shrink_ancient_stats
4418            .slots_considered
4419            .fetch_add(1, Ordering::Relaxed);
4420
4421        if is_ancient(accounts) {
4422            self.shrink_ancient_stats
4423                .ancient_scanned
4424                .fetch_add(1, Ordering::Relaxed);
4425
4426            // randomly shrink ancient slots
4427            // this exercises the ancient shrink code more often
4428            let written_bytes = storage.written_bytes();
4429            let mut alive_ratio = 0;
4430            let is_candidate = if written_bytes > 0 {
4431                alive_ratio = (storage.alive_bytes() as u64) * 100 / written_bytes;
4432                alive_ratio < 90
4433            } else {
4434                false
4435            };
4436            if is_candidate || (can_randomly_shrink && thread_rng().gen_range(0, 10000) == 0) {
4437                // we are a candidate for shrink, so either append us to the previous append vec
4438                // or recreate us as a new append vec and eliminate the dead accounts
4439                info!(
4440                    "ancient_append_vec: shrinking full ancient: {}, random: {}, alive_ratio: {}",
4441                    slot, !is_candidate, alive_ratio
4442                );
4443                if !is_candidate {
4444                    self.shrink_ancient_stats
4445                        .random_shrink
4446                        .fetch_add(1, Ordering::Relaxed);
4447                }
4448                self.shrink_ancient_stats
4449                    .ancient_append_vecs_shrunk
4450                    .fetch_add(1, Ordering::Relaxed);
4451                return true;
4452            }
4453            // this slot is ancient and can become the 'current' ancient for other slots to be squashed into
4454            *current_ancient = CurrentAncientAppendVec::new(slot, Arc::clone(storage));
4455            return false; // we're done with this slot - this slot IS the ancient append vec
4456        }
4457
4458        // otherwise, yes, squash this slot into the current ancient append vec or create one at this slot
4459        true
4460    }
4461
4462    /// Combine all account data from storages in 'sorted_slots' into ancient append vecs.
4463    /// This keeps us from accumulating append vecs for each slot older than an epoch.
4464    fn combine_ancient_slots(&self, sorted_slots: Vec<Slot>, can_randomly_shrink: bool) {
4465        let mut total = Measure::start("combine_ancient_slots");
4466        if sorted_slots.is_empty() {
4467            return;
4468        }
4469        let mut guard = None;
4470
4471        // the ancient append vec currently being written to
4472        let mut current_ancient = CurrentAncientAppendVec::default();
4473        let mut dropped_roots = vec![];
4474
4475        // we have to keep track of what pubkeys exist in the current ancient append vec so we can unref correctly
4476        let mut ancient_slot_pubkeys = AncientSlotPubkeys::default();
4477
4478        let len = sorted_slots.len();
4479        for slot in sorted_slots {
4480            let old_storage = match self.get_storage_to_move_to_ancient_append_vec(
4481                slot,
4482                &mut current_ancient,
4483                can_randomly_shrink,
4484            ) {
4485                Some(old_storages) => old_storages,
4486                None => {
4487                    // nothing to squash for this slot
4488                    continue;
4489                }
4490            };
4491
4492            if guard.is_none() {
4493                // we are now doing interesting work in squashing ancient
4494                guard = Some(self.active_stats.activate(ActiveStatItem::SquashAncient));
4495                info!(
4496                    "ancient_append_vec: combine_ancient_slots first slot: {}, num_roots: {}",
4497                    slot, len
4498                );
4499            }
4500
4501            self.combine_one_store_into_ancient(
4502                slot,
4503                &old_storage,
4504                &mut current_ancient,
4505                &mut ancient_slot_pubkeys,
4506                &mut dropped_roots,
4507            );
4508        }
4509
4510        self.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
4511
4512        total.stop();
4513        self.shrink_ancient_stats
4514            .total_us
4515            .fetch_add(total.as_us(), Ordering::Relaxed);
4516
4517        // only log when we moved some accounts to ancient append vecs or we've exceeded 100ms
4518        // results will continue to accumulate otherwise
4519        if guard.is_some() || self.shrink_ancient_stats.total_us.load(Ordering::Relaxed) > 100_000 {
4520            self.shrink_ancient_stats.report();
4521        }
4522    }
4523
4524    /// put entire alive contents of 'old_storage' into the current ancient append vec or a newly created ancient append vec
4525    fn combine_one_store_into_ancient(
4526        &self,
4527        slot: Slot,
4528        old_storage: &Arc<AccountStorageEntry>,
4529        current_ancient: &mut CurrentAncientAppendVec,
4530        ancient_slot_pubkeys: &mut AncientSlotPubkeys,
4531        dropped_roots: &mut Vec<Slot>,
4532    ) {
4533        let unique_accounts = self.get_unique_accounts_from_storage_for_shrink(
4534            old_storage,
4535            &self.shrink_ancient_stats.shrink_stats,
4536        );
4537        let shrink_collect = self.shrink_collect::<AliveAccounts<'_>>(
4538            old_storage,
4539            &unique_accounts,
4540            &self.shrink_ancient_stats.shrink_stats,
4541        );
4542
4543        // could follow what shrink does more closely
4544        if shrink_collect.total_starting_accounts == 0 || shrink_collect.alive_total_bytes == 0 {
4545            return; // skipping slot with no useful accounts to write
4546        }
4547
4548        let mut stats_sub = ShrinkStatsSub::default();
4549        let (mut shrink_in_progress, create_and_insert_store_elapsed_us) =
4550            measure_us!(current_ancient.create_if_necessary(slot, self));
4551        stats_sub.create_and_insert_store_elapsed_us = create_and_insert_store_elapsed_us;
4552        let available_bytes = current_ancient.append_vec().accounts.remaining_bytes();
4553        // split accounts in 'slot' into:
4554        // 'Primary', which can fit in 'current_ancient'
4555        // 'Overflow', which will have to go into a new ancient append vec at 'slot'
4556        let to_store = AccountsToStore::new(
4557            available_bytes,
4558            shrink_collect.alive_accounts.alive_accounts(),
4559            shrink_collect.alive_total_bytes,
4560            slot,
4561        );
4562
4563        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
4564            slot,
4565            self,
4566            current_ancient,
4567            &to_store,
4568        );
4569
4570        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
4571        // write what we can to the current ancient storage
4572        stats_sub.store_accounts_timing =
4573            current_ancient.store_ancient_accounts(self, &to_store, StorageSelector::Primary);
4574
4575        // handle accounts from 'slot' which did not fit into the current ancient append vec
4576        if to_store.has_overflow() {
4577            // We need a new ancient append vec at this slot.
4578            // Assert: it cannot be the case that we already had an ancient append vec at this slot and
4579            // yet that ancient append vec does not have room for the accounts stored at this slot currently
4580            assert_ne!(slot, current_ancient.slot());
4581            let (shrink_in_progress_overflow, time_us) =
4582                measure_us!(current_ancient.create_ancient_append_vec(slot, self));
4583            stats_sub.create_and_insert_store_elapsed_us += time_us;
4584            // We cannot possibly be shrinking the original slot that created an ancient append vec
4585            // AND not have enough room in the ancient append vec at that slot
4586            // to hold all the contents of that slot.
4587            // We need this new 'shrink_in_progress' to be used in 'remove_old_stores_shrink' below.
4588            // All non-overflow accounts were put in a prior slot's ancient append vec. All overflow accounts
4589            // are essentially being shrunk into a new ancient append vec in 'slot'.
4590            assert!(shrink_in_progress.is_none());
4591            shrink_in_progress = Some(shrink_in_progress_overflow);
4592
4593            // write the overflow accounts to the next ancient storage
4594            let timing =
4595                current_ancient.store_ancient_accounts(self, &to_store, StorageSelector::Overflow);
4596            stats_sub.store_accounts_timing.accumulate(&timing);
4597        }
4598        rewrite_elapsed.stop();
4599        stats_sub.rewrite_elapsed_us = rewrite_elapsed.as_us();
4600
4601        if slot != current_ancient.slot() {
4602            // all append vecs in this slot have been combined into an ancient append vec
4603            dropped_roots.push(slot);
4604        }
4605
4606        self.remove_old_stores_shrink(
4607            &shrink_collect,
4608            &self.shrink_ancient_stats.shrink_stats,
4609            shrink_in_progress,
4610            false,
4611        );
4612
4613        // we should not try to shrink any of the stores from this slot anymore. All shrinking for this slot is now handled by ancient append vec code.
4614        self.shrink_candidate_slots.lock().unwrap().remove(&slot);
4615
4616        Self::update_shrink_stats(&self.shrink_ancient_stats.shrink_stats, stats_sub);
4617    }
4618
4619    /// each slot in 'dropped_roots' has been combined into an ancient append vec.
4620    /// We are done with the slot now forever.
4621    pub(crate) fn handle_dropped_roots_for_ancient(
4622        &self,
4623        dropped_roots: impl Iterator<Item = Slot>,
4624    ) {
4625        dropped_roots.for_each(|slot| {
4626            self.accounts_index
4627                .clean_dead_slot(slot, &mut AccountsIndexRootsStats::default());
4628            self.remove_bank_hash_info(&slot);
4629            // the storage has been removed from this slot and recycled or dropped
4630            assert!(self.storage.remove(&slot, false).is_none());
4631        });
4632    }
4633
4634    /// add all 'pubkeys' into the set of pubkeys that are 'uncleaned', associated with 'slot'
4635    /// clean will visit these pubkeys next time it runs
4636    fn add_uncleaned_pubkeys_after_shrink(
4637        &self,
4638        slot: Slot,
4639        pubkeys: impl Iterator<Item = Pubkey>,
4640    ) {
4641        /*
4642        This is only called during 'shrink'-type operations.
4643        Original accounts were separated into 'accounts' and 'unrefed_pubkeys'.
4644        These sets correspond to 'alive' and 'dead'.
4645        'alive' means this account in this slot is in the accounts index.
4646        'dead' means this account in this slot is NOT in the accounts index.
4647        If dead, nobody will care if this version of this account is not written into the newly shrunk append vec for this slot.
4648        For all dead accounts, they were already unrefed and are now absent in the new append vec.
4649        This means that another version of this pubkey could possibly now be cleaned since this one is now gone.
4650        For example, a zero lamport account in a later slot can be removed if we just removed the only non-zero lamport account for that pubkey in this slot.
4651        So, for all unrefed accounts, send them to clean to be revisited next time clean runs.
4652        If an account is alive, then its status has not changed. It was previously alive in this slot. It is still alive in this slot.
4653        Clean doesn't care about alive accounts that remain alive.
4654        Except... A slightly different case is if ALL the alive accounts in this slot are zero lamport accounts, then it is possible that
4655        this slot can be marked dead. So, if all alive accounts are zero lamports, we send the entire OLD/pre-shrunk append vec
4656        to clean so that all the pubkeys are visited.
4657        It is a performance optimization to not send the ENTIRE old/pre-shrunk append vec to clean in the normal case.
4658        */
4659
4660        let mut uncleaned_pubkeys = self
4661            .uncleaned_pubkeys
4662            .entry(slot)
4663            .or_insert_with(Vec::default);
4664        uncleaned_pubkeys.extend(pubkeys);
4665    }
4666
4667    pub fn shrink_candidate_slots(&self) -> usize {
4668        if !self.shrink_candidate_slots.lock().unwrap().is_empty() {
4669            // this can affect 'shrink_candidate_slots', so don't 'take' it until after this completes
4670            self.shrink_ancient_slots();
4671        }
4672
4673        let shrink_candidates_slots =
4674            std::mem::take(&mut *self.shrink_candidate_slots.lock().unwrap());
4675
4676        let (shrink_slots, shrink_slots_next_batch) = {
4677            if let AccountShrinkThreshold::TotalSpace { shrink_ratio } = self.shrink_ratio {
4678                let (shrink_slots, shrink_slots_next_batch) =
4679                    Self::select_candidates_by_total_usage(
4680                        &shrink_candidates_slots,
4681                        shrink_ratio,
4682                        self.ancient_append_vec_offset
4683                            .map(|_| self.get_newest_ancient_slot()),
4684                    );
4685                (shrink_slots, Some(shrink_slots_next_batch))
4686            } else {
4687                (shrink_candidates_slots, None)
4688            }
4689        };
4690
4691        if shrink_slots.is_empty()
4692            && shrink_slots_next_batch
4693                .as_ref()
4694                .map(|s| s.is_empty())
4695                .unwrap_or(true)
4696        {
4697            return 0;
4698        }
4699
4700        let _guard = self.active_stats.activate(ActiveStatItem::Shrink);
4701
4702        let mut measure_shrink_all_candidates = Measure::start("shrink_all_candidate_slots-ms");
4703        let num_candidates = shrink_slots.len();
4704        let shrink_candidates_count = shrink_slots.len();
4705        self.thread_pool_clean.install(|| {
4706            shrink_slots
4707                .into_par_iter()
4708                .for_each(|(slot, slot_shrink_candidate)| {
4709                    let mut measure = Measure::start("shrink_candidate_slots-ms");
4710                    self.do_shrink_slot_store(slot, &slot_shrink_candidate);
4711                    measure.stop();
4712                    inc_new_counter_info!("shrink_candidate_slots-ms", measure.as_ms() as usize);
4713                });
4714        });
4715        measure_shrink_all_candidates.stop();
4716        inc_new_counter_info!(
4717            "shrink_all_candidate_slots-ms",
4718            measure_shrink_all_candidates.as_ms() as usize
4719        );
4720        inc_new_counter_info!("shrink_all_candidate_slots-count", shrink_candidates_count);
4721        let mut pended_counts: usize = 0;
4722        if let Some(shrink_slots_next_batch) = shrink_slots_next_batch {
4723            let mut shrink_slots = self.shrink_candidate_slots.lock().unwrap();
4724            pended_counts += shrink_slots_next_batch.len();
4725            for (slot, store) in shrink_slots_next_batch {
4726                shrink_slots.insert(slot, store);
4727            }
4728        }
4729        inc_new_counter_info!("shrink_pended_stores-count", pended_counts);
4730
4731        num_candidates
4732    }
4733
4734    pub fn shrink_all_slots(&self, is_startup: bool, last_full_snapshot_slot: Option<Slot>) {
4735        let _guard = self.active_stats.activate(ActiveStatItem::Shrink);
4736        const DIRTY_STORES_CLEANING_THRESHOLD: usize = 10_000;
4737        const OUTER_CHUNK_SIZE: usize = 2000;
4738        if is_startup {
4739            let slots = self.all_slots_in_storage();
4740            let threads = num_cpus::get();
4741            let inner_chunk_size = std::cmp::max(OUTER_CHUNK_SIZE / threads, 1);
4742            slots.chunks(OUTER_CHUNK_SIZE).for_each(|chunk| {
4743                chunk.par_chunks(inner_chunk_size).for_each(|slots| {
4744                    for slot in slots {
4745                        self.shrink_slot_forced(*slot);
4746                    }
4747                });
4748                if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
4749                    self.clean_accounts(None, is_startup, last_full_snapshot_slot);
4750                }
4751            });
4752        } else {
4753            for slot in self.all_slots_in_storage() {
4754                self.shrink_slot_forced(slot);
4755                if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
4756                    self.clean_accounts(None, is_startup, last_full_snapshot_slot);
4757                }
4758            }
4759        }
4760    }
4761
4762    pub fn scan_accounts<F>(
4763        &self,
4764        ancestors: &Ancestors,
4765        bank_id: BankId,
4766        mut scan_func: F,
4767        config: &ScanConfig,
4768    ) -> ScanResult<()>
4769    where
4770        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4771    {
4772        // This can error out if the slots being scanned over are aborted
4773        self.accounts_index.scan_accounts(
4774            ancestors,
4775            bank_id,
4776            |pubkey, (account_info, slot)| {
4777                let account_slot = self
4778                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4779                    .get_loaded_account()
4780                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot));
4781                scan_func(account_slot)
4782            },
4783            config,
4784        )?;
4785
4786        Ok(())
4787    }
4788
4789    pub fn unchecked_scan_accounts<F>(
4790        &self,
4791        metric_name: &'static str,
4792        ancestors: &Ancestors,
4793        mut scan_func: F,
4794        config: &ScanConfig,
4795    ) where
4796        F: FnMut(&Pubkey, LoadedAccount, Slot),
4797    {
4798        self.accounts_index.unchecked_scan_accounts(
4799            metric_name,
4800            ancestors,
4801            |pubkey, (account_info, slot)| {
4802                if let Some(loaded_account) = self
4803                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4804                    .get_loaded_account()
4805                {
4806                    scan_func(pubkey, loaded_account, slot);
4807                }
4808            },
4809            config,
4810        );
4811    }
4812
4813    /// Only guaranteed to be safe when called from rent collection
4814    pub fn range_scan_accounts<F, R>(
4815        &self,
4816        metric_name: &'static str,
4817        ancestors: &Ancestors,
4818        range: R,
4819        config: &ScanConfig,
4820        mut scan_func: F,
4821    ) where
4822        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4823        R: RangeBounds<Pubkey> + std::fmt::Debug,
4824    {
4825        self.accounts_index.range_scan_accounts(
4826            metric_name,
4827            ancestors,
4828            range,
4829            config,
4830            |pubkey, (account_info, slot)| {
4831                // unlike other scan fns, this is called from Bank::collect_rent_eagerly(),
4832                // which is on-consensus processing in the banking/replaying stage.
4833                // This requires infallible and consistent account loading.
4834                // So, we unwrap Option<LoadedAccount> from get_loaded_account() here.
4835                // This is safe because this closure is invoked with the account_info,
4836                // while we lock the index entry at AccountsIndex::do_scan_accounts() ultimately,
4837                // meaning no other subsystems can invalidate the account_info before making their
4838                // changes to the index entry.
4839                // For details, see the comment in retry_to_get_account_accessor()
4840                if let Some(account_slot) = self
4841                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4842                    .get_loaded_account()
4843                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot))
4844                {
4845                    scan_func(Some(account_slot))
4846                }
4847            },
4848        );
4849    }
4850
4851    pub fn index_scan_accounts<F>(
4852        &self,
4853        ancestors: &Ancestors,
4854        bank_id: BankId,
4855        index_key: IndexKey,
4856        mut scan_func: F,
4857        config: &ScanConfig,
4858    ) -> ScanResult<bool>
4859    where
4860        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4861    {
4862        let key = match &index_key {
4863            IndexKey::ProgramId(key) => key,
4864            IndexKey::SplTokenMint(key) => key,
4865            IndexKey::SplTokenOwner(key) => key,
4866        };
4867        if !self.account_indexes.include_key(key) {
4868            // the requested key was not indexed in the secondary index, so do a normal scan
4869            let used_index = false;
4870            self.scan_accounts(ancestors, bank_id, scan_func, config)?;
4871            return Ok(used_index);
4872        }
4873
4874        self.accounts_index.index_scan_accounts(
4875            ancestors,
4876            bank_id,
4877            index_key,
4878            |pubkey, (account_info, slot)| {
4879                let account_slot = self
4880                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4881                    .get_loaded_account()
4882                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot));
4883                scan_func(account_slot)
4884            },
4885            config,
4886        )?;
4887        let used_index = true;
4888        Ok(used_index)
4889    }
4890
4891    /// Scan a specific slot through all the account storage
4892    pub fn scan_account_storage<R, B>(
4893        &self,
4894        slot: Slot,
4895        cache_map_func: impl Fn(LoadedAccount) -> Option<R> + Sync,
4896        storage_scan_func: impl Fn(&B, LoadedAccount) + Sync,
4897    ) -> ScanStorageResult<R, B>
4898    where
4899        R: Send,
4900        B: Send + Default + Sync,
4901    {
4902        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
4903            // If we see the slot in the cache, then all the account information
4904            // is in this cached slot
4905            if slot_cache.len() > SCAN_SLOT_PAR_ITER_THRESHOLD {
4906                ScanStorageResult::Cached(self.thread_pool.install(|| {
4907                    slot_cache
4908                        .par_iter()
4909                        .filter_map(|cached_account| {
4910                            cache_map_func(LoadedAccount::Cached(Cow::Borrowed(
4911                                cached_account.value(),
4912                            )))
4913                        })
4914                        .collect()
4915                }))
4916            } else {
4917                ScanStorageResult::Cached(
4918                    slot_cache
4919                        .iter()
4920                        .filter_map(|cached_account| {
4921                            cache_map_func(LoadedAccount::Cached(Cow::Borrowed(
4922                                cached_account.value(),
4923                            )))
4924                        })
4925                        .collect(),
4926                )
4927            }
4928        } else {
4929            let retval = B::default();
4930            // If the slot is not in the cache, then all the account information must have
4931            // been flushed. This is guaranteed because we only remove the rooted slot from
4932            // the cache *after* we've finished flushing in `flush_slot_cache`.
4933            if let Some(storage) = self.storage.get_slot_storage_entry(slot) {
4934                storage
4935                    .accounts
4936                    .account_iter()
4937                    .for_each(|account| storage_scan_func(&retval, LoadedAccount::Stored(account)));
4938            }
4939
4940            ScanStorageResult::Stored(retval)
4941        }
4942    }
4943
4944    /// Insert a default bank hash stats for `slot`
4945    ///
4946    /// This fn is called when creating a new bank from parent.
4947    pub fn insert_default_bank_hash_stats(&self, slot: Slot, parent_slot: Slot) {
4948        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
4949        if bank_hash_stats.get(&slot).is_some() {
4950            error!( "set_hash: already exists; multiple forks with shared slot {slot} as child (parent: {parent_slot})!?");
4951            return;
4952        }
4953        bank_hash_stats.insert(slot, BankHashStats::default());
4954    }
4955
4956    pub fn load(
4957        &self,
4958        ancestors: &Ancestors,
4959        pubkey: &Pubkey,
4960        load_hint: LoadHint,
4961    ) -> Option<(AccountSharedData, Slot)> {
4962        self.do_load(ancestors, pubkey, None, load_hint, LoadZeroLamports::None)
4963    }
4964
4965    /// Return Ok(index_of_matching_owner) if the account owner at `offset` is one of the pubkeys in `owners`.
4966    /// Return Err(MatchAccountOwnerError::NoMatch) if the account has 0 lamports or the owner is not one of
4967    /// the pubkeys in `owners`.
4968    /// Return Err(MatchAccountOwnerError::UnableToLoad) if the account could not be accessed.
4969    pub fn account_matches_owners(
4970        &self,
4971        ancestors: &Ancestors,
4972        account: &Pubkey,
4973        owners: &[&Pubkey],
4974    ) -> Result<usize, MatchAccountOwnerError> {
4975        let (slot, storage_location, _maybe_account_accesor) = self
4976            .read_index_for_accessor_or_load_slow(ancestors, account, None, false)
4977            .ok_or(MatchAccountOwnerError::UnableToLoad)?;
4978
4979        if !storage_location.is_cached() {
4980            let result = self.read_only_accounts_cache.load(*account, slot);
4981            if let Some(account) = result {
4982                return if account.is_zero_lamport() {
4983                    Err(MatchAccountOwnerError::NoMatch)
4984                } else {
4985                    owners
4986                        .iter()
4987                        .position(|entry| &account.owner() == entry)
4988                        .ok_or(MatchAccountOwnerError::NoMatch)
4989                };
4990            }
4991        }
4992
4993        let (account_accessor, _slot) = self
4994            .retry_to_get_account_accessor(
4995                slot,
4996                storage_location,
4997                ancestors,
4998                account,
4999                None,
5000                LoadHint::Unspecified,
5001            )
5002            .ok_or(MatchAccountOwnerError::UnableToLoad)?;
5003        account_accessor.account_matches_owners(owners)
5004    }
5005
5006    pub fn load_account_into_read_cache(&self, ancestors: &Ancestors, pubkey: &Pubkey) {
5007        self.do_load_with_populate_read_cache(
5008            ancestors,
5009            pubkey,
5010            None,
5011            LoadHint::Unspecified,
5012            true,
5013            // no return from this function, so irrelevant
5014            LoadZeroLamports::None,
5015        );
5016    }
5017
5018    /// note this returns None for accounts with zero lamports
5019    pub fn load_with_fixed_root(
5020        &self,
5021        ancestors: &Ancestors,
5022        pubkey: &Pubkey,
5023    ) -> Option<(AccountSharedData, Slot)> {
5024        self.load(ancestors, pubkey, LoadHint::FixedMaxRoot)
5025    }
5026
5027    fn read_index_for_accessor_or_load_slow<'a>(
5028        &'a self,
5029        ancestors: &Ancestors,
5030        pubkey: &'a Pubkey,
5031        max_root: Option<Slot>,
5032        clone_in_lock: bool,
5033    ) -> Option<(Slot, StorageLocation, Option<LoadedAccountAccessor<'a>>)> {
5034        let (lock, index) = match self.accounts_index.get(pubkey, Some(ancestors), max_root) {
5035            AccountIndexGetResult::Found(lock, index) => (lock, index),
5036            // we bail out pretty early for missing.
5037            AccountIndexGetResult::NotFound => {
5038                return None;
5039            }
5040        };
5041
5042        let slot_list = lock.slot_list();
5043        let (slot, info) = slot_list[index];
5044        let storage_location = info.storage_location();
5045        let some_from_slow_path = if clone_in_lock {
5046            // the fast path must have failed.... so take the slower approach
5047            // of copying potentially large Account::data inside the lock.
5048
5049            // calling check_and_get_loaded_account is safe as long as we're guaranteed to hold
5050            // the lock during the time and there should be no purge thanks to alive ancestors
5051            // held by our caller.
5052            Some(self.get_account_accessor(slot, pubkey, &storage_location))
5053        } else {
5054            None
5055        };
5056
5057        Some((slot, storage_location, some_from_slow_path))
5058        // `lock` is dropped here rather pretty quickly with clone_in_lock = false,
5059        // so the entry could be raced for mutation by other subsystems,
5060        // before we actually provision an account data for caller's use from now on.
5061        // This is traded for less contention and resultant performance, introducing fair amount of
5062        // delicate handling in retry_to_get_account_accessor() below ;)
5063        // you're warned!
5064    }
5065
5066    fn retry_to_get_account_accessor<'a>(
5067        &'a self,
5068        mut slot: Slot,
5069        mut storage_location: StorageLocation,
5070        ancestors: &'a Ancestors,
5071        pubkey: &'a Pubkey,
5072        max_root: Option<Slot>,
5073        load_hint: LoadHint,
5074    ) -> Option<(LoadedAccountAccessor<'a>, Slot)> {
5075        // Happy drawing time! :)
5076        //
5077        // Reader                               | Accessed data source for cached/stored
5078        // -------------------------------------+----------------------------------
5079        // R1 read_index_for_accessor_or_load_slow()| cached/stored: index
5080        //          |                           |
5081        //        <(store_id, offset, ..)>      |
5082        //          V                           |
5083        // R2 retry_to_get_account_accessor()/  | cached: map of caches & entry for (slot, pubkey)
5084        //        get_account_accessor()        | stored: map of stores
5085        //          |                           |
5086        //        <Accessor>                    |
5087        //          V                           |
5088        // R3 check_and_get_loaded_account()/   | cached: N/A (note: basically noop unwrap)
5089        //        get_loaded_account()          | stored: store's entry for slot
5090        //          |                           |
5091        //        <LoadedAccount>               |
5092        //          V                           |
5093        // R4 take_account()                    | cached/stored: entry of cache/storage for (slot, pubkey)
5094        //          |                           |
5095        //        <AccountSharedData>           |
5096        //          V                           |
5097        //    Account!!                         V
5098        //
5099        // Flusher                              | Accessed data source for cached/stored
5100        // -------------------------------------+----------------------------------
5101        // F1 flush_slot_cache()                | N/A
5102        //          |                           |
5103        //          V                           |
5104        // F2 store_accounts_frozen()/          | map of stores (creates new entry)
5105        //        write_accounts_to_storage()   |
5106        //          |                           |
5107        //          V                           |
5108        // F3 store_accounts_frozen()/          | index
5109        //        update_index()                | (replaces existing store_id, offset in caches)
5110        //          |                           |
5111        //          V                           |
5112        // F4 accounts_cache.remove_slot()      | map of caches (removes old entry)
5113        //                                      V
5114        //
5115        // Remarks for flusher: So, for any reading operations, it's a race condition where F4 happens
5116        // between R1 and R2. In that case, retrying from R1 is safu because F3 should have
5117        // been occurred.
5118        //
5119        // Shrinker                             | Accessed data source for stored
5120        // -------------------------------------+----------------------------------
5121        // S1 do_shrink_slot_store()            | N/A
5122        //          |                           |
5123        //          V                           |
5124        // S2 store_accounts_frozen()/          | map of stores (creates new entry)
5125        //        write_accounts_to_storage()   |
5126        //          |                           |
5127        //          V                           |
5128        // S3 store_accounts_frozen()/          | index
5129        //        update_index()                | (replaces existing store_id, offset in stores)
5130        //          |                           |
5131        //          V                           |
5132        // S4 do_shrink_slot_store()/           | map of stores (removes old entry)
5133        //        dead_storages
5134        //
5135        // Remarks for shrinker: So, for any reading operations, it's a race condition
5136        // where S4 happens between R1 and R2. In that case, retrying from R1 is safu because S3 should have
5137        // been occurred, and S3 atomically replaced the index accordingly.
5138        //
5139        // Cleaner                              | Accessed data source for stored
5140        // -------------------------------------+----------------------------------
5141        // C1 clean_accounts()                  | N/A
5142        //          |                           |
5143        //          V                           |
5144        // C2 clean_accounts()/                 | index
5145        //        purge_keys_exact()            | (removes existing store_id, offset for stores)
5146        //          |                           |
5147        //          V                           |
5148        // C3 clean_accounts()/                 | map of stores (removes old entry)
5149        //        handle_reclaims()             |
5150        //
5151        // Remarks for cleaner: So, for any reading operations, it's a race condition
5152        // where C3 happens between R1 and R2. In that case, retrying from R1 is safu.
5153        // In that case, None would be returned while bailing out at R1.
5154        //
5155        // Purger                                 | Accessed data source for cached/stored
5156        // ---------------------------------------+----------------------------------
5157        // P1 purge_slot()                        | N/A
5158        //          |                             |
5159        //          V                             |
5160        // P2 purge_slots_from_cache_and_store()  | map of caches/stores (removes old entry)
5161        //          |                             |
5162        //          V                             |
5163        // P3 purge_slots_from_cache_and_store()/ | index
5164        //       purge_slot_cache()/              |
5165        //          purge_slot_cache_pubkeys()    | (removes existing store_id, offset for cache)
5166        //       purge_slot_storage()/            |
5167        //          purge_keys_exact()            | (removes accounts index entries)
5168        //          handle_reclaims()             | (removes storage entries)
5169        //      OR                                |
5170        //    clean_accounts()/                   |
5171        //        clean_accounts_older_than_root()| (removes existing store_id, offset for stores)
5172        //                                        V
5173        //
5174        // Remarks for purger: So, for any reading operations, it's a race condition
5175        // where P2 happens between R1 and R2. In that case, retrying from R1 is safu.
5176        // In that case, we may bail at index read retry when P3 hasn't been run
5177
5178        #[cfg(test)]
5179        {
5180            // Give some time for cache flushing to occur here for unit tests
5181            sleep(Duration::from_millis(self.load_delay));
5182        }
5183
5184        // Failsafe for potential race conditions with other subsystems
5185        let mut num_acceptable_failed_iterations = 0;
5186        loop {
5187            let account_accessor = self.get_account_accessor(slot, pubkey, &storage_location);
5188            match account_accessor {
5189                LoadedAccountAccessor::Cached(Some(_)) | LoadedAccountAccessor::Stored(Some(_)) => {
5190                    // Great! There was no race, just return :) This is the most usual situation
5191                    return Some((account_accessor, slot));
5192                }
5193                LoadedAccountAccessor::Cached(None) => {
5194                    num_acceptable_failed_iterations += 1;
5195                    // Cache was flushed in between checking the index and retrieving from the cache,
5196                    // so retry. This works because in accounts cache flush, an account is written to
5197                    // storage *before* it is removed from the cache
5198                    match load_hint {
5199                        LoadHint::FixedMaxRoot => {
5200                            // it's impossible for this to fail for transaction loads from
5201                            // replaying/banking more than once.
5202                            // This is because:
5203                            // 1) For a slot `X` that's being replayed, there is only one
5204                            // latest ancestor containing the latest update for the account, and this
5205                            // ancestor can only be flushed once.
5206                            // 2) The root cannot move while replaying, so the index cannot continually
5207                            // find more up to date entries than the current `slot`
5208                            assert!(num_acceptable_failed_iterations <= 1);
5209                        }
5210                        LoadHint::Unspecified => {
5211                            // Because newer root can be added to the index (= not fixed),
5212                            // multiple flush race conditions can be observed under very rare
5213                            // condition, at least theoretically
5214                        }
5215                    }
5216                }
5217                LoadedAccountAccessor::Stored(None) => {
5218                    match load_hint {
5219                        LoadHint::FixedMaxRoot => {
5220                            // When running replay on the validator, or banking stage on the leader,
5221                            // it should be very rare that the storage entry doesn't exist if the
5222                            // entry in the accounts index is the latest version of this account.
5223                            //
5224                            // There are only a few places where the storage entry may not exist
5225                            // after reading the index:
5226                            // 1) Shrink has removed the old storage entry and rewritten to
5227                            // a newer storage entry
5228                            // 2) The `pubkey` asked for in this function is a zero-lamport account,
5229                            // and the storage entry holding this account qualified for zero-lamport clean.
5230                            //
5231                            // In both these cases, it should be safe to retry and recheck the accounts
5232                            // index indefinitely, without incrementing num_acceptable_failed_iterations.
5233                            // That's because if the root is fixed, there should be a bounded number
5234                            // of pending cleans/shrinks (depends how far behind the AccountsBackgroundService
5235                            // is), termination to the desired condition is guaranteed.
5236                            //
5237                            // Also note that in both cases, if we do find the storage entry,
5238                            // we can guarantee that the storage entry is safe to read from because
5239                            // we grabbed a reference to the storage entry while it was still in the
5240                            // storage map. This means even if the storage entry is removed from the storage
5241                            // map after we grabbed the storage entry, the recycler should not reset the
5242                            // storage entry until we drop the reference to the storage entry.
5243                            //
5244                            // eh, no code in this arm? yes!
5245                        }
5246                        LoadHint::Unspecified => {
5247                            // RPC get_account() may have fetched an old root from the index that was
5248                            // either:
5249                            // 1) Cleaned up by clean_accounts(), so the accounts index has been updated
5250                            // and the storage entries have been removed.
5251                            // 2) Dropped by purge_slots() because the slot was on a minor fork, which
5252                            // removes the slots' storage entries but doesn't purge from the accounts index
5253                            // (account index cleanup is left to clean for stored slots). Note that
5254                            // this generally is impossible to occur in the wild because the RPC
5255                            // should hold the slot's bank, preventing it from being purged() to
5256                            // begin with.
5257                            num_acceptable_failed_iterations += 1;
5258                        }
5259                    }
5260                }
5261            }
5262            #[cfg(not(test))]
5263            let load_limit = ABSURD_CONSECUTIVE_FAILED_ITERATIONS;
5264
5265            #[cfg(test)]
5266            let load_limit = self.load_limit.load(Ordering::Relaxed);
5267
5268            let fallback_to_slow_path = if num_acceptable_failed_iterations >= load_limit {
5269                // The latest version of the account existed in the index, but could not be
5270                // fetched from storage. This means a race occurred between this function and clean
5271                // accounts/purge_slots
5272                let message = format!(
5273                    "do_load() failed to get key: {pubkey} from storage, latest attempt was for \
5274                     slot: {slot}, storage_location: {storage_location:?}, load_hint: {load_hint:?}",
5275                );
5276                datapoint_warn!("accounts_db-do_load_warn", ("warn", message, String));
5277                true
5278            } else {
5279                false
5280            };
5281
5282            // Because reading from the cache/storage failed, retry from the index read
5283            let (new_slot, new_storage_location, maybe_account_accessor) = self
5284                .read_index_for_accessor_or_load_slow(
5285                    ancestors,
5286                    pubkey,
5287                    max_root,
5288                    fallback_to_slow_path,
5289                )?;
5290            // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5291
5292            if new_slot == slot && new_storage_location.is_store_id_equal(&storage_location) {
5293                inc_new_counter_info!("retry_to_get_account_accessor-panic", 1);
5294                let message = format!(
5295                    "Bad index entry detected ({}, {}, {:?}, {:?}, {:?}, {:?})",
5296                    pubkey,
5297                    slot,
5298                    storage_location,
5299                    load_hint,
5300                    new_storage_location,
5301                    self.accounts_index.get_account_read_entry(pubkey)
5302                );
5303                // Considering that we've failed to get accessor above and further that
5304                // the index still returned the same (slot, store_id) tuple, offset must be same
5305                // too.
5306                assert!(
5307                    new_storage_location.is_offset_equal(&storage_location),
5308                    "{message}"
5309                );
5310
5311                // If the entry was missing from the cache, that means it must have been flushed,
5312                // and the accounts index is always updated before cache flush, so store_id must
5313                // not indicate being cached at this point.
5314                assert!(!new_storage_location.is_cached(), "{message}");
5315
5316                // If this is not a cache entry, then this was a minor fork slot
5317                // that had its storage entries cleaned up by purge_slots() but hasn't been
5318                // cleaned yet. That means this must be rpc access and not replay/banking at the
5319                // very least. Note that purge shouldn't occur even for RPC as caller must hold all
5320                // of ancestor slots..
5321                assert_eq!(load_hint, LoadHint::Unspecified, "{message}");
5322
5323                // Everything being assert!()-ed, let's panic!() here as it's an error condition
5324                // after all....
5325                // That reasoning is based on the fact all of code-path reaching this fn
5326                // retry_to_get_account_accessor() must outlive the Arc<Bank> (and its all
5327                // ancestors) over this fn invocation, guaranteeing the prevention of being purged,
5328                // first of all.
5329                // For details, see the comment in AccountIndex::do_checked_scan_accounts(),
5330                // which is referring back here.
5331                panic!("{message}");
5332            } else if fallback_to_slow_path {
5333                // the above bad-index-entry check must had been checked first to retain the same
5334                // behavior
5335                return Some((
5336                    maybe_account_accessor.expect("must be some if clone_in_lock=true"),
5337                    new_slot,
5338                ));
5339            }
5340
5341            slot = new_slot;
5342            storage_location = new_storage_location;
5343        }
5344    }
5345
5346    fn do_load(
5347        &self,
5348        ancestors: &Ancestors,
5349        pubkey: &Pubkey,
5350        max_root: Option<Slot>,
5351        load_hint: LoadHint,
5352        load_zero_lamports: LoadZeroLamports,
5353    ) -> Option<(AccountSharedData, Slot)> {
5354        self.do_load_with_populate_read_cache(
5355            ancestors,
5356            pubkey,
5357            max_root,
5358            load_hint,
5359            false,
5360            load_zero_lamports,
5361        )
5362    }
5363
5364    /// remove all entries from the read only accounts cache
5365    /// useful for benches/tests
5366    pub fn flush_read_only_cache_for_tests(&self) {
5367        self.read_only_accounts_cache.reset_for_tests();
5368    }
5369
5370    /// if 'load_into_read_cache_only', then return value is meaningless.
5371    ///   The goal is to get the account into the read-only cache.
5372    fn do_load_with_populate_read_cache(
5373        &self,
5374        ancestors: &Ancestors,
5375        pubkey: &Pubkey,
5376        max_root: Option<Slot>,
5377        load_hint: LoadHint,
5378        load_into_read_cache_only: bool,
5379        load_zero_lamports: LoadZeroLamports,
5380    ) -> Option<(AccountSharedData, Slot)> {
5381        #[cfg(not(test))]
5382        assert!(max_root.is_none());
5383
5384        let (slot, storage_location, _maybe_account_accesor) =
5385            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
5386        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5387
5388        let in_write_cache = storage_location.is_cached();
5389        if !load_into_read_cache_only {
5390            if !in_write_cache {
5391                let result = self.read_only_accounts_cache.load(*pubkey, slot);
5392                if let Some(account) = result {
5393                    if matches!(load_zero_lamports, LoadZeroLamports::None)
5394                        && account.is_zero_lamport()
5395                    {
5396                        return None;
5397                    }
5398                    return Some((account, slot));
5399                }
5400            }
5401        } else {
5402            // goal is to load into read cache
5403            if in_write_cache {
5404                // no reason to load in read cache. already in write cache
5405                return None;
5406            }
5407            if self.read_only_accounts_cache.in_cache(pubkey, slot) {
5408                // already in read cache
5409                return None;
5410            }
5411        }
5412
5413        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
5414            slot,
5415            storage_location,
5416            ancestors,
5417            pubkey,
5418            max_root,
5419            load_hint,
5420        )?;
5421        let loaded_account = account_accessor.check_and_get_loaded_account();
5422        let is_cached = loaded_account.is_cached();
5423        let account = loaded_account.take_account();
5424        if matches!(load_zero_lamports, LoadZeroLamports::None) && account.is_zero_lamport() {
5425            return None;
5426        }
5427
5428        if !is_cached {
5429            /*
5430            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
5431            safe/reflect 'A''s latest state on this fork.
5432            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
5433            not the read-only cache, after it's been updated in replay of slot 'S'.
5434            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
5435            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
5436            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
5437            which means '(S', A)' does not exist in the write cache yet.
5438            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
5439            must exist in the write cache, which is a contradiction.
5440            */
5441            self.read_only_accounts_cache
5442                .store(*pubkey, slot, account.clone());
5443        }
5444        Some((account, slot))
5445    }
5446
5447    pub fn load_account_hash(
5448        &self,
5449        ancestors: &Ancestors,
5450        pubkey: &Pubkey,
5451        max_root: Option<Slot>,
5452        load_hint: LoadHint,
5453    ) -> Option<Hash> {
5454        let (slot, storage_location, _maybe_account_accesor) =
5455            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
5456        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5457
5458        let (mut account_accessor, _) = self.retry_to_get_account_accessor(
5459            slot,
5460            storage_location,
5461            ancestors,
5462            pubkey,
5463            max_root,
5464            load_hint,
5465        )?;
5466        let loaded_account = account_accessor.check_and_get_loaded_account();
5467        Some(loaded_account.loaded_hash())
5468    }
5469
5470    fn get_account_accessor<'a>(
5471        &'a self,
5472        slot: Slot,
5473        pubkey: &'a Pubkey,
5474        storage_location: &StorageLocation,
5475    ) -> LoadedAccountAccessor<'a> {
5476        match storage_location {
5477            StorageLocation::Cached => {
5478                let maybe_cached_account = self.accounts_cache.load(slot, pubkey).map(Cow::Owned);
5479                LoadedAccountAccessor::Cached(maybe_cached_account)
5480            }
5481            StorageLocation::AppendVec(store_id, offset) => {
5482                let maybe_storage_entry = self
5483                    .storage
5484                    .get_account_storage_entry(slot, *store_id)
5485                    .map(|account_storage_entry| (account_storage_entry, *offset));
5486                LoadedAccountAccessor::Stored(maybe_storage_entry)
5487            }
5488        }
5489    }
5490
5491    fn try_recycle_and_insert_store(
5492        &self,
5493        slot: Slot,
5494        min_size: u64,
5495        max_size: u64,
5496    ) -> Option<Arc<AccountStorageEntry>> {
5497        let store = self.try_recycle_store(slot, min_size, max_size)?;
5498        self.insert_store(slot, store.clone());
5499        Some(store)
5500    }
5501
5502    fn try_recycle_store(
5503        &self,
5504        slot: Slot,
5505        min_size: u64,
5506        max_size: u64,
5507    ) -> Option<Arc<AccountStorageEntry>> {
5508        let mut max = 0;
5509        let mut min = std::u64::MAX;
5510        let mut avail = 0;
5511        let mut recycle_stores = self.recycle_stores.write().unwrap();
5512        for (i, (_recycled_time, store)) in recycle_stores.iter().enumerate() {
5513            if Arc::strong_count(store) == 1 {
5514                max = std::cmp::max(store.accounts.capacity(), max);
5515                min = std::cmp::min(store.accounts.capacity(), min);
5516                avail += 1;
5517
5518                if store.accounts.capacity() >= min_size && store.accounts.capacity() < max_size {
5519                    let ret = recycle_stores.remove_entry(i);
5520                    drop(recycle_stores);
5521                    let old_id = ret.append_vec_id();
5522                    ret.recycle(slot, self.next_id());
5523                    // This info show the appendvec history change history.  It helps debugging
5524                    // the appendvec data corrupution issues related to recycling.
5525                    debug!(
5526                        "recycling store: old slot {}, old_id: {}, new slot {}, new id{}, path {:?} ",
5527                        slot,
5528                        old_id,
5529                        ret.slot(),
5530                        ret.append_vec_id(),
5531                        ret.get_path(),
5532                    );
5533                    self.stats
5534                        .recycle_store_count
5535                        .fetch_add(1, Ordering::Relaxed);
5536                    return Some(ret);
5537                }
5538            }
5539        }
5540        debug!(
5541            "no recycle stores max: {} min: {} len: {} looking: {}, {} avail: {}",
5542            max,
5543            min,
5544            recycle_stores.entry_count(),
5545            min_size,
5546            max_size,
5547            avail,
5548        );
5549        None
5550    }
5551
5552    fn find_storage_candidate(&self, slot: Slot, size: usize) -> Arc<AccountStorageEntry> {
5553        let mut get_slot_stores = Measure::start("get_slot_stores");
5554        let store = self.storage.get_slot_storage_entry(slot);
5555        get_slot_stores.stop();
5556        self.stats
5557            .store_get_slot_store
5558            .fetch_add(get_slot_stores.as_us(), Ordering::Relaxed);
5559        let mut find_existing = Measure::start("find_existing");
5560        if let Some(store) = store {
5561            if store.try_available() {
5562                let ret = store.clone();
5563                drop(store);
5564                find_existing.stop();
5565                self.stats
5566                    .store_find_existing
5567                    .fetch_add(find_existing.as_us(), Ordering::Relaxed);
5568                return ret;
5569            }
5570        }
5571        find_existing.stop();
5572        self.stats
5573            .store_find_existing
5574            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
5575
5576        let store = if let Some(store) = self.try_recycle_store(slot, size as u64, std::u64::MAX) {
5577            store
5578        } else {
5579            self.create_store(slot, self.file_size, "store", &self.paths)
5580        };
5581
5582        // try_available is like taking a lock on the store,
5583        // preventing other threads from using it.
5584        // It must succeed here and happen before insert,
5585        // otherwise another thread could also grab it from the index.
5586        assert!(store.try_available());
5587        self.insert_store(slot, store.clone());
5588        store
5589    }
5590
5591    pub(crate) fn page_align(size: u64) -> u64 {
5592        (size + (PAGE_SIZE - 1)) & !(PAGE_SIZE - 1)
5593    }
5594
5595    fn has_space_available(&self, slot: Slot, size: u64) -> bool {
5596        let store = self.storage.get_slot_storage_entry(slot).unwrap();
5597        if store.status() == AccountStorageStatus::Available
5598            && (store.accounts.capacity() - store.accounts.len() as u64) > size
5599        {
5600            return true;
5601        }
5602        false
5603    }
5604
5605    fn create_store(
5606        &self,
5607        slot: Slot,
5608        size: u64,
5609        from: &str,
5610        paths: &[PathBuf],
5611    ) -> Arc<AccountStorageEntry> {
5612        self.stats
5613            .create_store_count
5614            .fetch_add(1, Ordering::Relaxed);
5615        let path_index = thread_rng().gen_range(0, paths.len());
5616        let store = Arc::new(self.new_storage_entry(
5617            slot,
5618            Path::new(&paths[path_index]),
5619            Self::page_align(size),
5620        ));
5621
5622        debug!(
5623            "creating store: {} slot: {} len: {} size: {} from: {} path: {:?}",
5624            store.append_vec_id(),
5625            slot,
5626            store.accounts.len(),
5627            store.accounts.capacity(),
5628            from,
5629            store.accounts.get_path()
5630        );
5631
5632        store
5633    }
5634
5635    fn create_and_insert_store(
5636        &self,
5637        slot: Slot,
5638        size: u64,
5639        from: &str,
5640    ) -> Arc<AccountStorageEntry> {
5641        self.create_and_insert_store_with_paths(slot, size, from, &self.paths)
5642    }
5643
5644    fn create_and_insert_store_with_paths(
5645        &self,
5646        slot: Slot,
5647        size: u64,
5648        from: &str,
5649        paths: &[PathBuf],
5650    ) -> Arc<AccountStorageEntry> {
5651        let store = self.create_store(slot, size, from, paths);
5652        let store_for_index = store.clone();
5653
5654        self.insert_store(slot, store_for_index);
5655        store
5656    }
5657
5658    fn insert_store(&self, slot: Slot, store: Arc<AccountStorageEntry>) {
5659        self.storage.insert(slot, store)
5660    }
5661
5662    pub fn create_drop_bank_callback(
5663        &self,
5664        pruned_banks_sender: DroppedSlotsSender,
5665    ) -> SendDroppedBankCallback {
5666        self.is_bank_drop_callback_enabled
5667            .store(true, Ordering::Release);
5668        SendDroppedBankCallback::new(pruned_banks_sender)
5669    }
5670
5671    /// This should only be called after the `Bank::drop()` runs in bank.rs, See BANK_DROP_SAFETY
5672    /// comment below for more explanation.
5673    ///   * `is_serialized_with_abs` - indicates whehter this call runs sequentially with all other
5674    ///        accounts_db relevant calls, such as shrinking, purging etc., in account background
5675    ///        service.
5676    pub fn purge_slot(&self, slot: Slot, bank_id: BankId, is_serialized_with_abs: bool) {
5677        if self.is_bank_drop_callback_enabled.load(Ordering::Acquire) && !is_serialized_with_abs {
5678            panic!(
5679                "bad drop callpath detected; Bank::drop() must run serially with other logic in
5680                ABS like clean_accounts()"
5681            )
5682        }
5683
5684        // BANK_DROP_SAFETY: Because this function only runs once the bank is dropped,
5685        // we know that there are no longer any ongoing scans on this bank, because scans require
5686        // and hold a reference to the bank at the tip of the fork they're scanning. Hence it's
5687        // safe to remove this bank_id from the `removed_bank_ids` list at this point.
5688        if self
5689            .accounts_index
5690            .removed_bank_ids
5691            .lock()
5692            .unwrap()
5693            .remove(&bank_id)
5694        {
5695            // If this slot was already cleaned up, no need to do any further cleans
5696            return;
5697        }
5698
5699        self.purge_slots(std::iter::once(&slot));
5700    }
5701
5702    fn recycle_slot_stores(
5703        &self,
5704        total_removed_storage_entries: usize,
5705        slot_stores: &[Arc<AccountStorageEntry>],
5706    ) -> u64 {
5707        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_elapsed");
5708        let mut recycle_stores = self.recycle_stores.write().unwrap();
5709        recycle_stores_write_elapsed.stop();
5710
5711        for (recycled_count, store) in slot_stores.iter().enumerate() {
5712            if recycle_stores.entry_count() > MAX_RECYCLE_STORES {
5713                let dropped_count = total_removed_storage_entries - recycled_count;
5714                self.stats
5715                    .dropped_stores
5716                    .fetch_add(dropped_count as u64, Ordering::Relaxed);
5717                return recycle_stores_write_elapsed.as_us();
5718            }
5719            recycle_stores.add_entry(Arc::clone(store));
5720        }
5721        recycle_stores_write_elapsed.as_us()
5722    }
5723
5724    /// Purges every slot in `removed_slots` from both the cache and storage. This includes
5725    /// entries in the accounts index, cache entries, and any backing storage entries.
5726    pub(crate) fn purge_slots_from_cache_and_store<'a>(
5727        &self,
5728        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5729        purge_stats: &PurgeStats,
5730        log_accounts: bool,
5731    ) {
5732        let mut remove_cache_elapsed_across_slots = 0;
5733        let mut num_cached_slots_removed = 0;
5734        let mut total_removed_cached_bytes = 0;
5735        if log_accounts {
5736            if let Some(min) = removed_slots.clone().min() {
5737                info!(
5738                    "purge_slots_from_cache_and_store: {:?}",
5739                    self.get_pubkey_hash_for_slot(*min).0
5740                );
5741            }
5742        }
5743        for remove_slot in removed_slots {
5744            // This function is only currently safe with respect to `flush_slot_cache()` because
5745            // both functions run serially in AccountsBackgroundService.
5746            let mut remove_cache_elapsed = Measure::start("remove_cache_elapsed");
5747            // Note: we cannot remove this slot from the slot cache until we've removed its
5748            // entries from the accounts index first. This is because `scan_accounts()` relies on
5749            // holding the index lock, finding the index entry, and then looking up the entry
5750            // in the cache. If it fails to find that entry, it will panic in `get_loaded_account()`
5751            if let Some(slot_cache) = self.accounts_cache.slot_cache(*remove_slot) {
5752                // If the slot is still in the cache, remove the backing storages for
5753                // the slot and from the Accounts Index
5754                num_cached_slots_removed += 1;
5755                total_removed_cached_bytes += slot_cache.total_bytes();
5756                self.purge_slot_cache(*remove_slot, slot_cache);
5757                remove_cache_elapsed.stop();
5758                remove_cache_elapsed_across_slots += remove_cache_elapsed.as_us();
5759                // Nobody else shoud have removed the slot cache entry yet
5760                assert!(self.accounts_cache.remove_slot(*remove_slot).is_some());
5761            } else {
5762                self.purge_slot_storage(*remove_slot, purge_stats);
5763            }
5764            // It should not be possible that a slot is neither in the cache or storage. Even in
5765            // a slot with all ticks, `Bank::new_from_parent()` immediately stores some sysvars
5766            // on bank creation.
5767        }
5768
5769        purge_stats
5770            .remove_cache_elapsed
5771            .fetch_add(remove_cache_elapsed_across_slots, Ordering::Relaxed);
5772        purge_stats
5773            .num_cached_slots_removed
5774            .fetch_add(num_cached_slots_removed, Ordering::Relaxed);
5775        purge_stats
5776            .total_removed_cached_bytes
5777            .fetch_add(total_removed_cached_bytes, Ordering::Relaxed);
5778    }
5779
5780    /// Purge the backing storage entries for the given slot, does not purge from
5781    /// the cache!
5782    fn purge_dead_slots_from_storage<'a>(
5783        &'a self,
5784        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5785        purge_stats: &PurgeStats,
5786    ) {
5787        // Check all slots `removed_slots` are no longer "relevant" roots.
5788        // Note that the slots here could have been rooted slots, but if they're passed here
5789        // for removal it means:
5790        // 1) All updates in that old root have been outdated by updates in newer roots
5791        // 2) Those slots/roots should have already been purged from the accounts index root
5792        // tracking metadata via `accounts_index.clean_dead_slot()`.
5793        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5794        assert!(self
5795            .accounts_index
5796            .get_rooted_from_list(removed_slots.clone())
5797            .is_empty());
5798        safety_checks_elapsed.stop();
5799        purge_stats
5800            .safety_checks_elapsed
5801            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5802
5803        let mut total_removed_storage_entries = 0;
5804        let mut total_removed_stored_bytes = 0;
5805        let mut all_removed_slot_storages = vec![];
5806
5807        let mut remove_storage_entries_elapsed = Measure::start("remove_storage_entries_elapsed");
5808        for remove_slot in removed_slots {
5809            // Remove the storage entries and collect some metrics
5810            if let Some(store) = self.storage.remove(remove_slot, false) {
5811                {
5812                    total_removed_storage_entries += 1;
5813                    total_removed_stored_bytes += store.accounts.capacity();
5814                }
5815                all_removed_slot_storages.push(store);
5816            }
5817        }
5818        remove_storage_entries_elapsed.stop();
5819        let num_stored_slots_removed = all_removed_slot_storages.len();
5820
5821        let recycle_stores_write_elapsed =
5822            self.recycle_slot_stores(total_removed_storage_entries, &all_removed_slot_storages);
5823
5824        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
5825        // Backing mmaps for removed storages entries explicitly dropped here outside
5826        // of any locks
5827        drop(all_removed_slot_storages);
5828        drop_storage_entries_elapsed.stop();
5829        purge_stats
5830            .remove_storage_entries_elapsed
5831            .fetch_add(remove_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5832        purge_stats
5833            .drop_storage_entries_elapsed
5834            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5835        purge_stats
5836            .num_stored_slots_removed
5837            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
5838        purge_stats
5839            .total_removed_storage_entries
5840            .fetch_add(total_removed_storage_entries, Ordering::Relaxed);
5841        purge_stats
5842            .total_removed_stored_bytes
5843            .fetch_add(total_removed_stored_bytes, Ordering::Relaxed);
5844        purge_stats
5845            .recycle_stores_write_elapsed
5846            .fetch_add(recycle_stores_write_elapsed, Ordering::Relaxed);
5847    }
5848
5849    fn purge_slot_cache(&self, purged_slot: Slot, slot_cache: SlotCache) {
5850        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
5851        let pubkey_to_slot_set: Vec<(Pubkey, Slot)> = slot_cache
5852            .iter()
5853            .map(|account| {
5854                purged_slot_pubkeys.insert((purged_slot, *account.key()));
5855                (*account.key(), purged_slot)
5856            })
5857            .collect();
5858        self.purge_slot_cache_pubkeys(
5859            purged_slot,
5860            purged_slot_pubkeys,
5861            pubkey_to_slot_set,
5862            true,
5863            &HashSet::default(),
5864        );
5865    }
5866
5867    fn purge_slot_cache_pubkeys(
5868        &self,
5869        purged_slot: Slot,
5870        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
5871        pubkey_to_slot_set: Vec<(Pubkey, Slot)>,
5872        is_dead: bool,
5873        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
5874    ) {
5875        // Slot purged from cache should not exist in the backing store
5876        assert!(self.storage.get_slot_storage_entry(purged_slot).is_none());
5877        let num_purged_keys = pubkey_to_slot_set.len();
5878        let (reclaims, _) = self.purge_keys_exact(pubkey_to_slot_set.iter());
5879        assert_eq!(reclaims.len(), num_purged_keys);
5880        if is_dead {
5881            self.remove_dead_slots_metadata(
5882                std::iter::once(&purged_slot),
5883                purged_slot_pubkeys,
5884                None,
5885                pubkeys_removed_from_accounts_index,
5886            );
5887        }
5888    }
5889
5890    fn purge_slot_storage(&self, remove_slot: Slot, purge_stats: &PurgeStats) {
5891        // Because AccountsBackgroundService synchronously flushes from the accounts cache
5892        // and handles all Bank::drop() (the cleanup function that leads to this
5893        // function call), then we don't need to worry above an overlapping cache flush
5894        // with this function call. This means, if we get into this case, we can be
5895        // confident that the entire state for this slot has been flushed to the storage
5896        // already.
5897        let mut scan_storages_elasped = Measure::start("scan_storages_elasped");
5898        type ScanResult = ScanStorageResult<Pubkey, Arc<Mutex<HashSet<(Pubkey, Slot)>>>>;
5899        let scan_result: ScanResult = self.scan_account_storage(
5900            remove_slot,
5901            |loaded_account: LoadedAccount| Some(*loaded_account.pubkey()),
5902            |accum: &Arc<Mutex<HashSet<(Pubkey, Slot)>>>, loaded_account: LoadedAccount| {
5903                accum
5904                    .lock()
5905                    .unwrap()
5906                    .insert((*loaded_account.pubkey(), remove_slot));
5907            },
5908        );
5909        scan_storages_elasped.stop();
5910        purge_stats
5911            .scan_storages_elapsed
5912            .fetch_add(scan_storages_elasped.as_us(), Ordering::Relaxed);
5913
5914        let mut purge_accounts_index_elapsed = Measure::start("purge_accounts_index_elapsed");
5915        let (reclaims, pubkeys_removed_from_accounts_index) = match scan_result {
5916            ScanStorageResult::Cached(_) => {
5917                panic!("Should not see cached keys in this `else` branch, since we checked this slot did not exist in the cache above");
5918            }
5919            ScanStorageResult::Stored(stored_keys) => {
5920                // Purge this slot from the accounts index
5921                self.purge_keys_exact(stored_keys.lock().unwrap().iter())
5922            }
5923        };
5924        purge_accounts_index_elapsed.stop();
5925        purge_stats
5926            .purge_accounts_index_elapsed
5927            .fetch_add(purge_accounts_index_elapsed.as_us(), Ordering::Relaxed);
5928
5929        // `handle_reclaims()` should remove all the account index entries and
5930        // storage entries
5931        let mut handle_reclaims_elapsed = Measure::start("handle_reclaims_elapsed");
5932        // Slot should be dead after removing all its account entries
5933        let expected_dead_slot = Some(remove_slot);
5934        self.handle_reclaims(
5935            (!reclaims.is_empty()).then(|| reclaims.iter()),
5936            expected_dead_slot,
5937            Some((purge_stats, &mut ReclaimResult::default())),
5938            false,
5939            &pubkeys_removed_from_accounts_index,
5940        );
5941        handle_reclaims_elapsed.stop();
5942        purge_stats
5943            .handle_reclaims_elapsed
5944            .fetch_add(handle_reclaims_elapsed.as_us(), Ordering::Relaxed);
5945        // After handling the reclaimed entries, this slot's
5946        // storage entries should be purged from self.storage
5947        assert!(
5948            self.storage.get_slot_storage_entry(remove_slot).is_none(),
5949            "slot {remove_slot} is not none"
5950        );
5951    }
5952
5953    #[allow(clippy::needless_collect)]
5954    fn purge_slots<'a>(&self, slots: impl Iterator<Item = &'a Slot> + Clone) {
5955        // `add_root()` should be called first
5956        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5957        let non_roots = slots
5958            // Only safe to check when there are duplicate versions of a slot
5959            // because ReplayStage will not make new roots before dumping the
5960            // duplicate slots first. Thus we will not be in a case where we
5961            // root slot `S`, then try to dump some other version of slot `S`, the
5962            // dumping has to finish first
5963            //
5964            // Also note roots are never removed via `remove_unrooted_slot()`, so
5965            // it's safe to filter them out here as they won't need deletion from
5966            // self.accounts_index.removed_bank_ids in `purge_slots_from_cache_and_store()`.
5967            .filter(|slot| !self.accounts_index.is_alive_root(**slot));
5968        safety_checks_elapsed.stop();
5969        self.external_purge_slots_stats
5970            .safety_checks_elapsed
5971            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5972        self.purge_slots_from_cache_and_store(non_roots, &self.external_purge_slots_stats, false);
5973        self.external_purge_slots_stats
5974            .report("external_purge_slots_stats", Some(1000));
5975    }
5976
5977    pub fn remove_unrooted_slots(&self, remove_slots: &[(Slot, BankId)]) {
5978        let rooted_slots = self
5979            .accounts_index
5980            .get_rooted_from_list(remove_slots.iter().map(|(slot, _)| slot));
5981        assert!(
5982            rooted_slots.is_empty(),
5983            "Trying to remove accounts for rooted slots {rooted_slots:?}"
5984        );
5985
5986        let RemoveUnrootedSlotsSynchronization {
5987            slots_under_contention,
5988            signal,
5989        } = &self.remove_unrooted_slots_synchronization;
5990
5991        {
5992            // Slots that are currently being flushed by flush_slot_cache()
5993
5994            let mut currently_contended_slots = slots_under_contention.lock().unwrap();
5995
5996            // Slots that are currently being flushed by flush_slot_cache() AND
5997            // we want to remove in this function
5998            let mut remaining_contended_flush_slots: Vec<Slot> = remove_slots
5999                .iter()
6000                .filter_map(|(remove_slot, _)| {
6001                    // Reserve the slots that we want to purge that aren't currently
6002                    // being flushed to prevent cache from flushing those slots in
6003                    // the future.
6004                    //
6005                    // Note that the single replay thread has to remove a specific slot `N`
6006                    // before another version of the same slot can be replayed. This means
6007                    // multiple threads should not call `remove_unrooted_slots()` simultaneously
6008                    // with the same slot.
6009                    let is_being_flushed = !currently_contended_slots.insert(*remove_slot);
6010                    // If the cache is currently flushing this slot, add it to the list
6011                    is_being_flushed.then_some(remove_slot)
6012                })
6013                .cloned()
6014                .collect();
6015
6016            // Wait for cache flushes to finish
6017            loop {
6018                if !remaining_contended_flush_slots.is_empty() {
6019                    // Wait for the signal that the cache has finished flushing a slot
6020                    //
6021                    // Don't wait if the remaining_contended_flush_slots is empty, otherwise
6022                    // we may never get a signal since there's no cache flush thread to
6023                    // do the signaling
6024                    currently_contended_slots = signal.wait(currently_contended_slots).unwrap();
6025                } else {
6026                    // There are no slots being flushed to wait on, so it's safe to continue
6027                    // to purging the slots we want to purge!
6028                    break;
6029                }
6030
6031                // For each slot the cache flush has finished, mark that we're about to start
6032                // purging these slots by reserving it in `currently_contended_slots`.
6033                remaining_contended_flush_slots.retain(|flush_slot| {
6034                    // returns true if slot was already in set. This means slot is being flushed
6035                    !currently_contended_slots.insert(*flush_slot)
6036                });
6037            }
6038        }
6039
6040        // Mark down these slots are about to be purged so that new attempts to scan these
6041        // banks fail, and any ongoing scans over these slots will detect that they should abort
6042        // their results
6043        {
6044            let mut locked_removed_bank_ids = self.accounts_index.removed_bank_ids.lock().unwrap();
6045            for (_slot, remove_bank_id) in remove_slots.iter() {
6046                locked_removed_bank_ids.insert(*remove_bank_id);
6047            }
6048        }
6049
6050        let remove_unrooted_purge_stats = PurgeStats::default();
6051        self.purge_slots_from_cache_and_store(
6052            remove_slots.iter().map(|(slot, _)| slot),
6053            &remove_unrooted_purge_stats,
6054            true,
6055        );
6056        remove_unrooted_purge_stats.report("remove_unrooted_slots_purge_slots_stats", Some(0));
6057
6058        let mut currently_contended_slots = slots_under_contention.lock().unwrap();
6059        for (remove_slot, _) in remove_slots {
6060            assert!(currently_contended_slots.remove(remove_slot));
6061        }
6062    }
6063
6064    pub fn hash_account<T: ReadableAccount>(
6065        slot: Slot,
6066        account: &T,
6067        pubkey: &Pubkey,
6068        include_slot: IncludeSlotInHash,
6069    ) -> Hash {
6070        Self::hash_account_data(
6071            slot,
6072            account.lamports(),
6073            account.owner(),
6074            account.executable(),
6075            account.rent_epoch(),
6076            account.data(),
6077            pubkey,
6078            include_slot,
6079        )
6080    }
6081
6082    fn hash_account_data(
6083        slot: Slot,
6084        lamports: u64,
6085        owner: &Pubkey,
6086        executable: bool,
6087        rent_epoch: Epoch,
6088        data: &[u8],
6089        pubkey: &Pubkey,
6090        include_slot: IncludeSlotInHash,
6091    ) -> Hash {
6092        if lamports == 0 {
6093            return Hash::default();
6094        }
6095        let mut hasher = blake3::Hasher::new();
6096
6097        hasher.update(&lamports.to_le_bytes());
6098
6099        match include_slot {
6100            IncludeSlotInHash::IncludeSlot => {
6101                // upon feature activation, stop including slot# in the account hash
6102                hasher.update(&slot.to_le_bytes());
6103            }
6104            IncludeSlotInHash::RemoveSlot => {}
6105            IncludeSlotInHash::IrrelevantAssertOnUse => {
6106                panic!("IncludeSlotInHash is irrelevant, but we are calculating hash");
6107            }
6108        }
6109
6110        hasher.update(&rent_epoch.to_le_bytes());
6111
6112        hasher.update(data);
6113
6114        if executable {
6115            hasher.update(&[1u8; 1]);
6116        } else {
6117            hasher.update(&[0u8; 1]);
6118        }
6119
6120        hasher.update(owner.as_ref());
6121        hasher.update(pubkey.as_ref());
6122
6123        Hash::new_from_array(hasher.finalize().into())
6124    }
6125
6126    fn bulk_assign_write_version(&self, count: usize) -> StoredMetaWriteVersion {
6127        self.write_version
6128            .fetch_add(count as StoredMetaWriteVersion, Ordering::AcqRel)
6129    }
6130
6131    fn write_accounts_to_storage<
6132        'a,
6133        'b,
6134        T: ReadableAccount + Sync,
6135        U: StorableAccounts<'a, T>,
6136        V: Borrow<Hash>,
6137    >(
6138        &self,
6139        slot: Slot,
6140        storage: &AccountStorageEntry,
6141        accounts_and_meta_to_store: &StorableAccountsWithHashesAndWriteVersions<'a, 'b, T, U, V>,
6142    ) -> Vec<AccountInfo> {
6143        let mut infos: Vec<AccountInfo> = Vec::with_capacity(accounts_and_meta_to_store.len());
6144        let mut total_append_accounts_us = 0;
6145        while infos.len() < accounts_and_meta_to_store.len() {
6146            let mut append_accounts = Measure::start("append_accounts");
6147            let rvs = storage
6148                .accounts
6149                .append_accounts(accounts_and_meta_to_store, infos.len());
6150            append_accounts.stop();
6151            total_append_accounts_us += append_accounts.as_us();
6152            if rvs.is_none() {
6153                storage.set_status(AccountStorageStatus::Full);
6154
6155                // See if an account overflows the append vecs in the slot.
6156                let account = accounts_and_meta_to_store.account(infos.len());
6157                let data_len = account
6158                    .map(|account| account.data().len())
6159                    .unwrap_or_default();
6160                let data_len = (data_len + STORE_META_OVERHEAD) as u64;
6161                if !self.has_space_available(slot, data_len) {
6162                    let special_store_size = std::cmp::max(data_len * 2, self.file_size);
6163                    if self
6164                        .try_recycle_and_insert_store(slot, special_store_size, std::u64::MAX)
6165                        .is_none()
6166                    {
6167                        self.create_and_insert_store(slot, special_store_size, "large create");
6168                    }
6169                }
6170                continue;
6171            }
6172
6173            for (i, offsets) in rvs.unwrap().windows(2).enumerate() {
6174                let stored_size = offsets[1] - offsets[0];
6175                storage.add_account(stored_size);
6176
6177                infos.push(AccountInfo::new(
6178                    StorageLocation::AppendVec(storage.append_vec_id(), offsets[0]),
6179                    stored_size as StoredSize, // stored_size should never exceed StoredSize::MAX because of max data len const
6180                    accounts_and_meta_to_store
6181                        .account(i)
6182                        .map(|account| account.lamports())
6183                        .unwrap_or_default(),
6184                ));
6185            }
6186            // restore the state to available
6187            storage.set_status(AccountStorageStatus::Available);
6188        }
6189
6190        self.stats
6191            .store_append_accounts
6192            .fetch_add(total_append_accounts_us, Ordering::Relaxed);
6193
6194        infos
6195    }
6196
6197    pub fn mark_slot_frozen(&self, slot: Slot) {
6198        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
6199            slot_cache.mark_slot_frozen();
6200            slot_cache.report_slot_store_metrics();
6201        }
6202        self.accounts_cache.report_size();
6203    }
6204
6205    pub fn expire_old_recycle_stores(&self) {
6206        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
6207        let recycle_stores = self.recycle_stores.write().unwrap().expire_old_entries();
6208        recycle_stores_write_elapsed.stop();
6209
6210        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
6211        drop(recycle_stores);
6212        drop_storage_entries_elapsed.stop();
6213
6214        self.clean_accounts_stats
6215            .purge_stats
6216            .drop_storage_entries_elapsed
6217            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
6218        self.clean_accounts_stats
6219            .purge_stats
6220            .recycle_stores_write_elapsed
6221            .fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
6222    }
6223
6224    #[cfg(test)]
6225    pub(crate) fn flush_accounts_cache_slot_for_tests(&self, slot: Slot) {
6226        self.flush_slot_cache(slot);
6227    }
6228
6229    /// true if write cache is too big
6230    fn should_aggressively_flush_cache(&self) -> bool {
6231        self.write_cache_limit_bytes
6232            .unwrap_or(WRITE_CACHE_LIMIT_BYTES_DEFAULT)
6233            < self.accounts_cache.size()
6234    }
6235
6236    // `force_flush` flushes all the cached roots `<= requested_flush_root`. It also then
6237    // flushes:
6238    // 1) excess remaining roots or unrooted slots while 'should_aggressively_flush_cache' is true
6239    pub fn flush_accounts_cache(&self, force_flush: bool, requested_flush_root: Option<Slot>) {
6240        #[cfg(not(test))]
6241        assert!(requested_flush_root.is_some());
6242
6243        if !force_flush && !self.should_aggressively_flush_cache() {
6244            return;
6245        }
6246
6247        // Flush only the roots <= requested_flush_root, so that snapshotting has all
6248        // the relevant roots in storage.
6249        let mut flush_roots_elapsed = Measure::start("flush_roots_elapsed");
6250        let mut account_bytes_saved = 0;
6251        let mut num_accounts_saved = 0;
6252
6253        let _guard = self.active_stats.activate(ActiveStatItem::Flush);
6254
6255        // Note even if force_flush is false, we will still flush all roots <= the
6256        // given `requested_flush_root`, even if some of the later roots cannot be used for
6257        // cleaning due to an ongoing scan
6258        let (total_new_cleaned_roots, num_cleaned_roots_flushed) = self
6259            .flush_rooted_accounts_cache(
6260                requested_flush_root,
6261                Some((&mut account_bytes_saved, &mut num_accounts_saved)),
6262            );
6263        flush_roots_elapsed.stop();
6264
6265        // Note we don't purge unrooted slots here because there may be ongoing scans/references
6266        // for those slot, let the Bank::drop() implementation do cleanup instead on dead
6267        // banks
6268
6269        // If 'should_aggressively_flush_cache', then flush the excess ones to storage
6270        let (total_new_excess_roots, num_excess_roots_flushed) =
6271            if self.should_aggressively_flush_cache() {
6272                // Start by flushing the roots
6273                //
6274                // Cannot do any cleaning on roots past `requested_flush_root` because future
6275                // snapshots may need updates from those later slots, hence we pass `None`
6276                // for `should_clean`.
6277                self.flush_rooted_accounts_cache(None, None)
6278            } else {
6279                (0, 0)
6280            };
6281
6282        let mut excess_slot_count = 0;
6283        let mut unflushable_unrooted_slot_count = 0;
6284        let max_flushed_root = self.accounts_cache.fetch_max_flush_root();
6285        if self.should_aggressively_flush_cache() {
6286            let old_slots = self.accounts_cache.cached_frozen_slots();
6287            excess_slot_count = old_slots.len();
6288            let mut flush_stats = FlushStats::default();
6289            old_slots.into_iter().for_each(|old_slot| {
6290                // Don't flush slots that are known to be unrooted
6291                if old_slot > max_flushed_root {
6292                    if self.should_aggressively_flush_cache() {
6293                        if let Some(stats) = self.flush_slot_cache(old_slot) {
6294                            flush_stats.num_flushed += stats.num_flushed;
6295                            flush_stats.num_purged += stats.num_purged;
6296                            flush_stats.total_size += stats.total_size;
6297                        }
6298                    }
6299                } else {
6300                    unflushable_unrooted_slot_count += 1;
6301                }
6302            });
6303            datapoint_info!(
6304                "accounts_db-flush_accounts_cache_aggressively",
6305                ("num_flushed", flush_stats.num_flushed, i64),
6306                ("num_purged", flush_stats.num_purged, i64),
6307                ("total_flush_size", flush_stats.total_size, i64),
6308                ("total_cache_size", self.accounts_cache.size(), i64),
6309                ("total_frozen_slots", excess_slot_count, i64),
6310                ("total_slots", self.accounts_cache.num_slots(), i64),
6311            );
6312        }
6313
6314        datapoint_info!(
6315            "accounts_db-flush_accounts_cache",
6316            ("total_new_cleaned_roots", total_new_cleaned_roots, i64),
6317            ("num_cleaned_roots_flushed", num_cleaned_roots_flushed, i64),
6318            ("total_new_excess_roots", total_new_excess_roots, i64),
6319            ("num_excess_roots_flushed", num_excess_roots_flushed, i64),
6320            ("excess_slot_count", excess_slot_count, i64),
6321            (
6322                "unflushable_unrooted_slot_count",
6323                unflushable_unrooted_slot_count,
6324                i64
6325            ),
6326            (
6327                "flush_roots_elapsed",
6328                flush_roots_elapsed.as_us() as i64,
6329                i64
6330            ),
6331            ("account_bytes_saved", account_bytes_saved, i64),
6332            ("num_accounts_saved", num_accounts_saved, i64),
6333        );
6334    }
6335
6336    fn flush_rooted_accounts_cache(
6337        &self,
6338        requested_flush_root: Option<Slot>,
6339        should_clean: Option<(&mut usize, &mut usize)>,
6340    ) -> (usize, usize) {
6341        let max_clean_root = should_clean.as_ref().and_then(|_| {
6342            // If there is a long running scan going on, this could prevent any cleaning
6343            // based on updates from slots > `max_clean_root`.
6344            self.max_clean_root(requested_flush_root)
6345        });
6346
6347        let mut written_accounts = HashSet::new();
6348
6349        // If `should_clean` is None, then`should_flush_f` is also None, which will cause
6350        // `flush_slot_cache` to flush all accounts to storage without cleaning any accounts.
6351        let mut should_flush_f = should_clean.map(|(account_bytes_saved, num_accounts_saved)| {
6352            move |&pubkey: &Pubkey, account: &AccountSharedData| {
6353                // if not in hashset, then not flushed previously, so flush it
6354                let should_flush = written_accounts.insert(pubkey);
6355                if !should_flush {
6356                    *account_bytes_saved += account.data().len();
6357                    *num_accounts_saved += 1;
6358                    // If a later root already wrote this account, no point
6359                    // in flushing it
6360                }
6361                should_flush
6362            }
6363        });
6364
6365        // Always flush up to `requested_flush_root`, which is necessary for things like snapshotting.
6366        let cached_roots: BTreeSet<Slot> = self.accounts_cache.clear_roots(requested_flush_root);
6367
6368        // Iterate from highest to lowest so that we don't need to flush earlier
6369        // outdated updates in earlier roots
6370        let mut num_roots_flushed = 0;
6371        for &root in cached_roots.iter().rev() {
6372            if self
6373                .flush_slot_cache_with_clean(&[root], should_flush_f.as_mut(), max_clean_root)
6374                .is_some()
6375            {
6376                num_roots_flushed += 1;
6377            }
6378
6379            // Regardless of whether this slot was *just* flushed from the cache by the above
6380            // `flush_slot_cache()`, we should update the `max_flush_root`.
6381            // This is because some rooted slots may be flushed to storage *before* they are marked as root.
6382            // This can occur for instance when
6383            //  the cache is overwhelmed, we flushed some yet to be rooted frozen slots
6384            // These slots may then *later* be marked as root, so we still need to handle updating the
6385            // `max_flush_root` in the accounts cache.
6386            self.accounts_cache.set_max_flush_root(root);
6387        }
6388
6389        // Only add to the uncleaned roots set *after* we've flushed the previous roots,
6390        // so that clean will actually be able to clean the slots.
6391        let num_new_roots = cached_roots.len();
6392        self.accounts_index.add_uncleaned_roots(cached_roots);
6393        (num_new_roots, num_roots_flushed)
6394    }
6395
6396    fn do_flush_slot_cache(
6397        &self,
6398        slot: Slot,
6399        slot_cache: &SlotCache,
6400        mut should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
6401        max_clean_root: Option<Slot>,
6402    ) -> FlushStats {
6403        let mut num_purged = 0;
6404        let mut total_size = 0;
6405        let mut num_flushed = 0;
6406        let iter_items: Vec<_> = slot_cache.iter().collect();
6407        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
6408        let mut pubkey_to_slot_set: Vec<(Pubkey, Slot)> = vec![];
6409        if should_flush_f.is_some() {
6410            if let Some(max_clean_root) = max_clean_root {
6411                if slot > max_clean_root {
6412                    // Only if the root is greater than the `max_clean_root` do we
6413                    // have to prevent cleaning, otherwise, just default to `should_flush_f`
6414                    // for any slots <= `max_clean_root`
6415                    should_flush_f = None;
6416                }
6417            }
6418        }
6419
6420        let mut filler_accounts = 0;
6421        if self.filler_accounts_enabled() {
6422            let slots_remaining = self.filler_account_slots_remaining.load(Ordering::Acquire);
6423            if slots_remaining > 0 {
6424                // figure out
6425                let pr = self.get_prior_root(slot);
6426
6427                if let Some(prior_root) = pr {
6428                    let filler_account_slots =
6429                        std::cmp::min(slot.saturating_sub(prior_root), slots_remaining);
6430                    self.filler_account_slots_remaining
6431                        .fetch_sub(filler_account_slots, Ordering::Release);
6432                    let filler_accounts_per_slot =
6433                        self.filler_accounts_per_slot.load(Ordering::Acquire);
6434                    filler_accounts = filler_account_slots * filler_accounts_per_slot;
6435
6436                    // keep space for filler accounts
6437                    let addl_size = filler_accounts
6438                        * (aligned_stored_size(self.filler_accounts_config.size) as u64);
6439                    total_size += addl_size;
6440                }
6441            }
6442        }
6443
6444        let (accounts, hashes): (Vec<(&Pubkey, &AccountSharedData)>, Vec<Hash>) = iter_items
6445            .iter()
6446            .filter_map(|iter_item| {
6447                let key = iter_item.key();
6448                let account = &iter_item.value().account;
6449                let should_flush = should_flush_f
6450                    .as_mut()
6451                    .map(|should_flush_f| should_flush_f(key, account))
6452                    .unwrap_or(true);
6453                if should_flush {
6454                    let hash = iter_item.value().hash();
6455                    total_size += aligned_stored_size(account.data().len()) as u64;
6456                    num_flushed += 1;
6457                    Some(((key, account), hash))
6458                } else {
6459                    // If we don't flush, we have to remove the entry from the
6460                    // index, since it's equivalent to purging
6461                    purged_slot_pubkeys.insert((slot, *key));
6462                    pubkey_to_slot_set.push((*key, slot));
6463                    num_purged += 1;
6464                    None
6465                }
6466            })
6467            .unzip();
6468
6469        let is_dead_slot = accounts.is_empty();
6470        // Remove the account index entries from earlier roots that are outdated by later roots.
6471        // Safe because queries to the index will be reading updates from later roots.
6472        self.purge_slot_cache_pubkeys(
6473            slot,
6474            purged_slot_pubkeys,
6475            pubkey_to_slot_set,
6476            is_dead_slot,
6477            &HashSet::default(),
6478        );
6479
6480        if !is_dead_slot {
6481            let aligned_total_size = Self::page_align(total_size);
6482            // This ensures that all updates are written to an AppendVec, before any
6483            // updates to the index happen, so anybody that sees a real entry in the index,
6484            // will be able to find the account in storage
6485            let flushed_store =
6486                self.create_and_insert_store(slot, aligned_total_size, "flush_slot_cache");
6487            // irrelevant - account will already be hashed since it was used in bank hash previously
6488            let include_slot_in_hash = IncludeSlotInHash::IrrelevantAssertOnUse;
6489            self.store_accounts_frozen(
6490                (slot, &accounts[..], include_slot_in_hash),
6491                Some(hashes),
6492                &flushed_store,
6493                None,
6494                StoreReclaims::Default,
6495            );
6496
6497            if filler_accounts > 0 {
6498                // add extra filler accounts at the end of the append vec
6499                let (account, hash) = self.get_filler_account(&Rent::default());
6500                let mut accounts = Vec::with_capacity(filler_accounts as usize);
6501                let mut hashes = Vec::with_capacity(filler_accounts as usize);
6502                let pubkeys = self.get_filler_account_pubkeys(filler_accounts as usize);
6503                pubkeys.iter().for_each(|key| {
6504                    accounts.push((key, &account));
6505                    hashes.push(hash);
6506                });
6507                self.store_accounts_frozen(
6508                    (slot, &accounts[..], include_slot_in_hash),
6509                    Some(hashes),
6510                    &flushed_store,
6511                    None,
6512                    StoreReclaims::Ignore,
6513                );
6514            }
6515
6516            // If the above sizing function is correct, just one AppendVec is enough to hold
6517            // all the data for the slot
6518            assert!(self.storage.get_slot_storage_entry(slot).is_some());
6519        }
6520
6521        // Remove this slot from the cache, which will to AccountsDb's new readers should look like an
6522        // atomic switch from the cache to storage.
6523        // There is some racy condition for existing readers who just has read exactly while
6524        // flushing. That case is handled by retry_to_get_account_accessor()
6525        assert!(self.accounts_cache.remove_slot(slot).is_some());
6526        FlushStats {
6527            num_flushed,
6528            num_purged,
6529            total_size,
6530        }
6531    }
6532
6533    /// flush all accounts in this slot
6534    fn flush_slot_cache(&self, slot: Slot) -> Option<FlushStats> {
6535        self.flush_slot_cache_with_clean(&[slot], None::<&mut fn(&_, &_) -> bool>, None)
6536    }
6537
6538    /// `should_flush_f` is an optional closure that determines whether a given
6539    /// account should be flushed. Passing `None` will by default flush all
6540    /// accounts
6541    fn flush_slot_cache_with_clean(
6542        &self,
6543        slots: &[Slot],
6544        should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
6545        max_clean_root: Option<Slot>,
6546    ) -> Option<FlushStats> {
6547        assert_eq!(1, slots.len());
6548        let slot = slots[0];
6549        if self
6550            .remove_unrooted_slots_synchronization
6551            .slots_under_contention
6552            .lock()
6553            .unwrap()
6554            .insert(slot)
6555        {
6556            // We have not seen this slot, flush it.
6557            let flush_stats = self.accounts_cache.slot_cache(slot).map(|slot_cache| {
6558                #[cfg(test)]
6559                {
6560                    // Give some time for cache flushing to occur here for unit tests
6561                    sleep(Duration::from_millis(self.load_delay));
6562                }
6563                // Since we added the slot to `slots_under_contention` AND this slot
6564                // still exists in the cache, we know the slot cannot be removed
6565                // by any other threads past this point. We are now responsible for
6566                // flushing this slot.
6567                self.do_flush_slot_cache(slot, &slot_cache, should_flush_f, max_clean_root)
6568            });
6569
6570            // Nobody else should have been purging this slot, so should not have been removed
6571            // from `self.remove_unrooted_slots_synchronization`.
6572
6573            slots.iter().for_each(|slot| {
6574                assert!(self
6575                    .remove_unrooted_slots_synchronization
6576                    .slots_under_contention
6577                    .lock()
6578                    .unwrap()
6579                    .remove(slot));
6580            });
6581
6582            // Signal to any threads blocked on `remove_unrooted_slots(slot)` that we have finished
6583            // flushing
6584            self.remove_unrooted_slots_synchronization
6585                .signal
6586                .notify_all();
6587            flush_stats
6588        } else {
6589            // We have already seen this slot. It is already under flushing. Skip.
6590            None
6591        }
6592    }
6593
6594    fn write_accounts_to_cache<'a, 'b, T: ReadableAccount + Sync, P>(
6595        &self,
6596        slot: Slot,
6597        accounts_and_meta_to_store: &impl StorableAccounts<'b, T>,
6598        txn_iter: Box<dyn std::iter::Iterator<Item = &Option<&SanitizedTransaction>> + 'a>,
6599        include_slot_in_hash: IncludeSlotInHash,
6600        mut write_version_producer: P,
6601    ) -> Vec<AccountInfo>
6602    where
6603        P: Iterator<Item = u64>,
6604    {
6605        txn_iter
6606            .enumerate()
6607            .map(|(i, txn)| {
6608                let account = accounts_and_meta_to_store
6609                    .account_default_if_zero_lamport(i)
6610                    .map(|account| account.to_account_shared_data())
6611                    .unwrap_or_default();
6612                let account_info = AccountInfo::new(
6613                    StorageLocation::Cached,
6614                    CACHE_VIRTUAL_STORED_SIZE,
6615                    account.lamports(),
6616                );
6617
6618                self.notify_account_at_accounts_update(
6619                    slot,
6620                    &account,
6621                    txn,
6622                    accounts_and_meta_to_store.pubkey(i),
6623                    &mut write_version_producer,
6624                );
6625
6626                let cached_account = self.accounts_cache.store(
6627                    slot,
6628                    accounts_and_meta_to_store.pubkey(i),
6629                    account,
6630                    None::<&Hash>,
6631                    include_slot_in_hash,
6632                );
6633                // hash this account in the bg
6634                match &self.sender_bg_hasher {
6635                    Some(ref sender) => {
6636                        let _ = sender.send(cached_account);
6637                    }
6638                    None => (),
6639                };
6640                account_info
6641            })
6642            .collect()
6643    }
6644
6645    fn store_accounts_to<
6646        'a: 'c,
6647        'b,
6648        'c,
6649        P: Iterator<Item = u64>,
6650        T: ReadableAccount + Sync + ZeroLamport + 'b,
6651    >(
6652        &self,
6653        accounts: &'c impl StorableAccounts<'b, T>,
6654        hashes: Option<Vec<impl Borrow<Hash>>>,
6655        mut write_version_producer: P,
6656        store_to: &StoreTo,
6657        transactions: Option<&[Option<&'a SanitizedTransaction>]>,
6658    ) -> Vec<AccountInfo> {
6659        let mut calc_stored_meta_time = Measure::start("calc_stored_meta");
6660        let slot = accounts.target_slot();
6661        (0..accounts.len()).for_each(|index| {
6662            let pubkey = accounts.pubkey(index);
6663            self.read_only_accounts_cache.remove(*pubkey, slot);
6664        });
6665        calc_stored_meta_time.stop();
6666        self.stats
6667            .calc_stored_meta
6668            .fetch_add(calc_stored_meta_time.as_us(), Ordering::Relaxed);
6669
6670        match store_to {
6671            StoreTo::Cache => {
6672                let txn_iter: Box<dyn std::iter::Iterator<Item = &Option<&SanitizedTransaction>>> =
6673                    match transactions {
6674                        Some(transactions) => {
6675                            assert_eq!(transactions.len(), accounts.len());
6676                            Box::new(transactions.iter())
6677                        }
6678                        None => Box::new(std::iter::repeat(&None).take(accounts.len())),
6679                    };
6680
6681                self.write_accounts_to_cache(
6682                    slot,
6683                    accounts,
6684                    txn_iter,
6685                    accounts.include_slot_in_hash(),
6686                    write_version_producer,
6687                )
6688            }
6689            StoreTo::Storage(storage) => {
6690                if accounts.has_hash_and_write_version() {
6691                    self.write_accounts_to_storage(
6692                        slot,
6693                        storage,
6694                        &StorableAccountsWithHashesAndWriteVersions::<'_, '_, _, _, &Hash>::new(
6695                            accounts,
6696                        ),
6697                    )
6698                } else {
6699                    let write_versions = (0..accounts.len())
6700                        .map(|_| write_version_producer.next().unwrap())
6701                        .collect::<Vec<_>>();
6702                    match hashes {
6703                        Some(hashes) => self.write_accounts_to_storage(
6704                            slot,
6705                            storage,
6706                            &StorableAccountsWithHashesAndWriteVersions::new_with_hashes_and_write_versions(
6707                                accounts,
6708                                hashes,
6709                                write_versions,
6710                            ),
6711                        ),
6712                        None => {
6713                            // hash any accounts where we were lazy in calculating the hash
6714                            let mut hash_time = Measure::start("hash_accounts");
6715                            let len = accounts.len();
6716                            let mut hashes = Vec::with_capacity(len);
6717                            for index in 0..accounts.len() {
6718                                let (pubkey, account) = (accounts.pubkey(index), accounts.account(index));
6719                                let hash = Self::hash_account(
6720                                    slot,
6721                                    account,
6722                                    pubkey,
6723                                    accounts.include_slot_in_hash(),
6724                                );
6725                                hashes.push(hash);
6726                            }
6727                            hash_time.stop();
6728                            self.stats
6729                                .store_hash_accounts
6730                                .fetch_add(hash_time.as_us(), Ordering::Relaxed);
6731
6732                            self.write_accounts_to_storage(
6733                                    slot,
6734                                    storage,
6735                                    &StorableAccountsWithHashesAndWriteVersions::new_with_hashes_and_write_versions(accounts, hashes, write_versions),
6736                                )
6737                        }
6738                    }
6739                }
6740            }
6741        }
6742    }
6743
6744    fn report_store_stats(&self) {
6745        let mut total_count = 0;
6746        let mut newest_slot = 0;
6747        let mut oldest_slot = std::u64::MAX;
6748        let mut total_bytes = 0;
6749        let mut total_alive_bytes = 0;
6750        for (slot, store) in self.storage.iter() {
6751            total_count += 1;
6752            newest_slot = std::cmp::max(newest_slot, slot);
6753
6754            oldest_slot = std::cmp::min(oldest_slot, slot);
6755
6756            total_alive_bytes += Self::page_align(store.alive_bytes() as u64);
6757            total_bytes += store.capacity();
6758        }
6759        info!(
6760            "total_stores: {total_count}, newest_slot: {newest_slot}, oldest_slot: {oldest_slot}"
6761        );
6762
6763        let total_alive_ratio = if total_bytes > 0 {
6764            total_alive_bytes as f64 / total_bytes as f64
6765        } else {
6766            0.
6767        };
6768
6769        datapoint_info!(
6770            "accounts_db-stores",
6771            ("total_count", total_count, i64),
6772            (
6773                "recycle_count",
6774                self.recycle_stores.read().unwrap().entry_count() as u64,
6775                i64
6776            ),
6777            ("total_bytes", total_bytes, i64),
6778            ("total_alive_bytes", total_alive_bytes, i64),
6779            ("total_alive_ratio", total_alive_ratio, f64),
6780        );
6781        datapoint_info!(
6782            "accounts_db-perf-stats",
6783            (
6784                "delta_hash_num",
6785                self.stats.delta_hash_num.swap(0, Ordering::Relaxed),
6786                i64
6787            ),
6788            (
6789                "delta_hash_scan_us",
6790                self.stats
6791                    .delta_hash_scan_time_total_us
6792                    .swap(0, Ordering::Relaxed),
6793                i64
6794            ),
6795            (
6796                "delta_hash_accumulate_us",
6797                self.stats
6798                    .delta_hash_accumulate_time_total_us
6799                    .swap(0, Ordering::Relaxed),
6800                i64
6801            ),
6802        );
6803    }
6804
6805    /// find slot >= 'slot' which is a root or in 'ancestors'
6806    pub fn find_unskipped_slot(&self, slot: Slot, ancestors: Option<&Ancestors>) -> Option<Slot> {
6807        self.accounts_index.get_next_original_root(slot, ancestors)
6808    }
6809
6810    pub fn checked_iterative_sum_for_capitalization(total_cap: u64, new_cap: u64) -> u64 {
6811        let new_total = total_cap as u128 + new_cap as u128;
6812        AccountsHasher::checked_cast_for_capitalization(new_total)
6813    }
6814
6815    pub fn checked_sum_for_capitalization<T: Iterator<Item = u64>>(balances: T) -> u64 {
6816        AccountsHasher::checked_cast_for_capitalization(balances.map(|b| b as u128).sum::<u128>())
6817    }
6818
6819    pub fn calculate_accounts_hash_from_index(
6820        &self,
6821        max_slot: Slot,
6822        config: &CalcAccountsHashConfig<'_>,
6823    ) -> Result<(AccountsHash, u64), BankHashVerificationError> {
6824        use BankHashVerificationError::*;
6825        let mut collect = Measure::start("collect");
6826        let keys: Vec<_> = self
6827            .accounts_index
6828            .account_maps
6829            .iter()
6830            .flat_map(|map| {
6831                let mut keys = map.keys();
6832                keys.sort_unstable(); // hashmap is not ordered, but bins are relative to each other
6833                keys
6834            })
6835            .collect();
6836        collect.stop();
6837
6838        let mut scan = Measure::start("scan");
6839        let mismatch_found = AtomicU64::new(0);
6840        // Pick a chunk size big enough to allow us to produce output vectors that are smaller than the overall size.
6841        // We'll also accumulate the lamports within each chunk and fewer chunks results in less contention to accumulate the sum.
6842        let chunks = crate::accounts_hash::MERKLE_FANOUT.pow(4);
6843        let total_lamports = Mutex::<u64>::new(0);
6844        let stats = HashStats::default();
6845
6846        let get_hashes = || {
6847            keys.par_chunks(chunks)
6848                .map(|pubkeys| {
6849                    let mut sum = 0u128;
6850                    let result: Vec<Hash> = pubkeys
6851                        .iter()
6852                        .filter_map(|pubkey| {
6853                            if self.is_filler_account(pubkey) {
6854                                return None;
6855                            }
6856                            if let AccountIndexGetResult::Found(lock, index) =
6857                                self.accounts_index.get(pubkey, config.ancestors, Some(max_slot))
6858                            {
6859                                let (slot, account_info) = &lock.slot_list()[index];
6860                                if !account_info.is_zero_lamport() {
6861                                    // Because we're keeping the `lock' here, there is no need
6862                                    // to use retry_to_get_account_accessor()
6863                                    // In other words, flusher/shrinker/cleaner is blocked to
6864                                    // cause any Accessor(None) situation.
6865                                    // Anyway this race condition concern is currently a moot
6866                                    // point because calculate_accounts_hash() should not
6867                                    // currently race with clean/shrink because the full hash
6868                                    // is synchronous with clean/shrink in
6869                                    // AccountsBackgroundService
6870                                    self.get_account_accessor(
6871                                        *slot,
6872                                        pubkey,
6873                                        &account_info.storage_location(),
6874                                    )
6875                                    .get_loaded_account()
6876                                    .and_then(
6877                                        |loaded_account| {
6878                                            let loaded_hash = loaded_account.loaded_hash();
6879                                            let balance = loaded_account.lamports();
6880                                            if config.check_hash && !self.is_filler_account(pubkey) {  // this will not be supported anymore
6881                                                let computed_hash =
6882                                                    loaded_account.compute_hash(*slot, pubkey, INCLUDE_SLOT_IN_HASH_IRRELEVANT_CHECK_HASH);
6883                                                if computed_hash != loaded_hash {
6884                                                    info!("hash mismatch found: computed: {}, loaded: {}, pubkey: {}", computed_hash, loaded_hash, pubkey);
6885                                                    mismatch_found
6886                                                        .fetch_add(1, Ordering::Relaxed);
6887                                                    return None;
6888                                                }
6889                                            }
6890
6891                                            sum += balance as u128;
6892                                            Some(loaded_hash)
6893                                        },
6894                                    )
6895                                } else {
6896                                    None
6897                                }
6898                            } else {
6899                                None
6900                            }
6901                        })
6902                        .collect();
6903                    let mut total = total_lamports.lock().unwrap();
6904                    *total =
6905                        AccountsHasher::checked_cast_for_capitalization(*total as u128 + sum);
6906                    result
6907                }).collect()
6908        };
6909
6910        let hashes: Vec<Vec<Hash>> = if config.check_hash {
6911            get_hashes()
6912        } else {
6913            self.thread_pool_clean.install(get_hashes)
6914        };
6915        if mismatch_found.load(Ordering::Relaxed) > 0 {
6916            warn!(
6917                "{} mismatched account hash(es) found",
6918                mismatch_found.load(Ordering::Relaxed)
6919            );
6920            return Err(MismatchedAccountHash);
6921        }
6922
6923        scan.stop();
6924        let total_lamports = *total_lamports.lock().unwrap();
6925
6926        let mut hash_time = Measure::start("hash");
6927        let (accumulated_hash, hash_total) = AccountsHasher::calculate_hash(hashes);
6928        hash_time.stop();
6929        datapoint_info!(
6930            "calculate_accounts_hash_from_index",
6931            ("accounts_scan", scan.as_us(), i64),
6932            ("hash", hash_time.as_us(), i64),
6933            ("hash_total", hash_total, i64),
6934            ("collect", collect.as_us(), i64),
6935            (
6936                "rehashed_rewrites",
6937                stats.rehash_required.load(Ordering::Relaxed),
6938                i64
6939            ),
6940            (
6941                "rehashed_rewrites_unnecessary",
6942                stats.rehash_unnecessary.load(Ordering::Relaxed),
6943                i64
6944            ),
6945        );
6946        self.assert_safe_squashing_accounts_hash(max_slot, config.epoch_schedule);
6947
6948        let accounts_hash = AccountsHash(accumulated_hash);
6949        Ok((accounts_hash, total_lamports))
6950    }
6951
6952    pub fn update_accounts_hash_for_tests(
6953        &self,
6954        slot: Slot,
6955        ancestors: &Ancestors,
6956        debug_verify: bool,
6957        is_startup: bool,
6958    ) -> (AccountsHash, u64) {
6959        self.update_accounts_hash(
6960            CalcAccountsHashDataSource::IndexForTests,
6961            debug_verify,
6962            slot,
6963            ancestors,
6964            None,
6965            &EpochSchedule::default(),
6966            &RentCollector::default(),
6967            is_startup,
6968        )
6969    }
6970
6971    /// iterate over a single storage, calling scanner on each item
6972    fn scan_single_account_storage<S>(storage: &Arc<AccountStorageEntry>, scanner: &mut S)
6973    where
6974        S: AppendVecScan,
6975    {
6976        storage.accounts.account_iter().for_each(|account| {
6977            if scanner.filter(account.pubkey()) {
6978                scanner.found_account(&LoadedAccount::Stored(account))
6979            }
6980        });
6981    }
6982
6983    fn update_old_slot_stats(&self, stats: &HashStats, storage: Option<&Arc<AccountStorageEntry>>) {
6984        if let Some(storage) = storage {
6985            stats.roots_older_than_epoch.fetch_add(1, Ordering::Relaxed);
6986            let mut ancients = 0;
6987            let num_accounts = storage.count();
6988            if is_ancient(&storage.accounts) {
6989                ancients += 1;
6990            }
6991            let sizes = storage.capacity();
6992            stats
6993                .append_vec_sizes_older_than_epoch
6994                .fetch_add(sizes as usize, Ordering::Relaxed);
6995            stats
6996                .accounts_in_roots_older_than_epoch
6997                .fetch_add(num_accounts, Ordering::Relaxed);
6998            stats
6999                .ancient_append_vecs
7000                .fetch_add(ancients, Ordering::Relaxed);
7001        }
7002    }
7003
7004    /// return slot + offset, where offset can be +/-
7005    fn apply_offset_to_slot(slot: Slot, offset: i64) -> Slot {
7006        if offset > 0 {
7007            slot.saturating_add(offset as u64)
7008        } else {
7009            slot.saturating_sub(offset.unsigned_abs())
7010        }
7011    }
7012
7013    /// if ancient append vecs are enabled, return a slot 'max_slot_inclusive' - (slots_per_epoch - `self.ancient_append_vec_offset`)
7014    /// otherwise, return 0
7015    fn get_one_epoch_old_slot_for_hash_calc_scan(
7016        &self,
7017        max_slot_inclusive: Slot,
7018        config: &CalcAccountsHashConfig<'_>,
7019    ) -> Slot {
7020        if let Some(offset) = self.ancient_append_vec_offset {
7021            // we are going to use a fixed slots per epoch here.
7022            // We are mainly interested in the network at steady state.
7023            let slots_in_epoch = config.epoch_schedule.slots_per_epoch;
7024            // For performance, this is required when ancient appendvecs are enabled
7025            let slot = max_slot_inclusive.saturating_sub(slots_in_epoch);
7026            Self::apply_offset_to_slot(slot, offset)
7027        } else {
7028            // This causes the entire range to be chunked together, treating older append vecs just like new ones.
7029            // This performs well if there are many old append vecs that haven't been cleaned yet.
7030            // 0 will have the effect of causing ALL older append vecs to be chunked together, just like every other append vec.
7031            0
7032        }
7033    }
7034
7035    /// hash info about 'storage' into 'hasher'
7036    /// return true iff storage is valid for loading from cache
7037    fn hash_storage_info(
7038        hasher: &mut impl StdHasher,
7039        storage: Option<&Arc<AccountStorageEntry>>,
7040        slot: Slot,
7041    ) -> bool {
7042        if let Some(append_vec) = storage {
7043            // hash info about this storage
7044            append_vec.written_bytes().hash(hasher);
7045            let storage_file = append_vec.accounts.get_path();
7046            slot.hash(hasher);
7047            storage_file.hash(hasher);
7048            let amod = std::fs::metadata(storage_file);
7049            if amod.is_err() {
7050                return false;
7051            }
7052            let amod = amod.unwrap().modified();
7053            if amod.is_err() {
7054                return false;
7055            }
7056            let amod = amod
7057                .unwrap()
7058                .duration_since(std::time::UNIX_EPOCH)
7059                .unwrap()
7060                .as_secs();
7061            amod.hash(hasher);
7062        }
7063        // if we made it here, we have hashed info and we should try to load from the cache
7064        true
7065    }
7066
7067    /// Scan through all the account storage in parallel.
7068    /// Returns a Vec of open/mmapped files.
7069    /// Each file has serialized hash info, sorted by pubkey and then slot, from scanning the append vecs.
7070    ///   A single pubkey could be in multiple entries. The pubkey found in the latest entry is the one to use.
7071    fn scan_account_storage_no_bank<S>(
7072        &self,
7073        cache_hash_data: &CacheHashData,
7074        config: &CalcAccountsHashConfig<'_>,
7075        snapshot_storages: &SortedStorages,
7076        scanner: S,
7077        bin_range: &Range<usize>,
7078        stats: &HashStats,
7079    ) -> Vec<CacheHashDataFile>
7080    where
7081        S: AppendVecScan,
7082    {
7083        let splitter = SplitAncientStorages::new(
7084            self.get_one_epoch_old_slot_for_hash_calc_scan(
7085                snapshot_storages.max_slot_inclusive(),
7086                config,
7087            ),
7088            snapshot_storages,
7089        );
7090
7091        (0..splitter.chunk_count)
7092            .into_par_iter()
7093            .map(|chunk| {
7094                let mut scanner = scanner.clone();
7095
7096                let range_this_chunk = splitter.get_slot_range(chunk)?;
7097
7098                let slots_per_epoch = config
7099                    .rent_collector
7100                    .epoch_schedule
7101                    .get_slots_in_epoch(config.rent_collector.epoch);
7102                let one_epoch_old = snapshot_storages
7103                    .range()
7104                    .end
7105                    .saturating_sub(slots_per_epoch);
7106
7107                let file_name = {
7108                    let mut load_from_cache = true;
7109                    let mut hasher = std::collections::hash_map::DefaultHasher::new();
7110                    bin_range.start.hash(&mut hasher);
7111                    bin_range.end.hash(&mut hasher);
7112                    let is_first_scan_pass = bin_range.start == 0;
7113
7114                    // calculate hash representing all storages in this chunk
7115                    for (slot, storage) in snapshot_storages.iter_range(&range_this_chunk) {
7116                        if is_first_scan_pass && slot < one_epoch_old {
7117                            self.update_old_slot_stats(stats, storage);
7118                        }
7119                        if !Self::hash_storage_info(&mut hasher, storage, slot) {
7120                            load_from_cache = false;
7121                            break;
7122                        }
7123                    }
7124                    // we have a hash value for the storages in this chunk
7125                    // so, build a file name:
7126                    let hash = hasher.finish();
7127                    let file_name = format!(
7128                        "{}.{}.{}.{}.{}",
7129                        range_this_chunk.start,
7130                        range_this_chunk.end,
7131                        bin_range.start,
7132                        bin_range.end,
7133                        hash
7134                    );
7135                    if load_from_cache {
7136                        if let Ok(mapped_file) = cache_hash_data.load_map(&file_name) {
7137                            return Some(mapped_file);
7138                        }
7139                    }
7140
7141                    // fall through and load normally - we failed to load from a cache file
7142                    file_name
7143                };
7144
7145                let mut init_accum = true;
7146                // load from cache failed, so create the cache file for this chunk
7147                for (slot, storage) in snapshot_storages.iter_range(&range_this_chunk) {
7148                    let mut ancient = false;
7149                    let (_, scan_us) = measure_us!(if let Some(storage) = storage {
7150                        ancient = is_ancient(&storage.accounts);
7151                        if init_accum {
7152                            let range = bin_range.end - bin_range.start;
7153                            scanner.init_accum(range);
7154                            init_accum = false;
7155                        }
7156                        scanner.set_slot(slot);
7157
7158                        Self::scan_single_account_storage(storage, &mut scanner);
7159                    });
7160                    if ancient {
7161                        stats
7162                            .sum_ancient_scans_us
7163                            .fetch_add(scan_us, Ordering::Relaxed);
7164                        stats.count_ancient_scans.fetch_add(1, Ordering::Relaxed);
7165                        stats
7166                            .longest_ancient_scan_us
7167                            .fetch_max(scan_us, Ordering::Relaxed);
7168                    }
7169                }
7170                (!init_accum)
7171                    .then(|| {
7172                        let r = scanner.scanning_complete();
7173                        assert!(!file_name.is_empty());
7174                        (!r.is_empty() && r.iter().any(|b| !b.is_empty())).then(|| {
7175                            // error if we can't write this
7176                            cache_hash_data.save(&file_name, &r).unwrap();
7177                            cache_hash_data.load_map(&file_name).unwrap()
7178                        })
7179                    })
7180                    .flatten()
7181            })
7182            .filter_map(|x| x)
7183            .collect()
7184    }
7185
7186    /// storages are sorted by slot and have range info.
7187    /// add all stores older than slots_per_epoch to dirty_stores so clean visits these slots
7188    fn mark_old_slots_as_dirty(
7189        &self,
7190        storages: &SortedStorages,
7191        slots_per_epoch: Slot,
7192        mut stats: &mut crate::accounts_hash::HashStats,
7193    ) {
7194        let mut mark_time = Measure::start("mark_time");
7195        let mut num_dirty_slots: usize = 0;
7196        let max = storages.max_slot_inclusive();
7197        let acceptable_straggler_slot_count = 100; // do nothing special for these old stores which will likely get cleaned up shortly
7198        let sub = slots_per_epoch + acceptable_straggler_slot_count;
7199        let in_epoch_range_start = max.saturating_sub(sub);
7200        for (slot, storage) in storages.iter_range(&(..in_epoch_range_start)) {
7201            if let Some(storage) = storage {
7202                if !is_ancient(&storage.accounts) {
7203                    // ancient stores are managed separately - we expect them to be old and keeping accounts
7204                    // We can expect the normal processes will keep them cleaned.
7205                    // If we included them here then ALL accounts in ALL ancient append vecs will be visited by clean each time.
7206                    self.dirty_stores
7207                        .insert((slot, storage.append_vec_id()), storage.clone());
7208                    num_dirty_slots += 1;
7209                }
7210            }
7211        }
7212        mark_time.stop();
7213        stats.mark_time_us = mark_time.as_us();
7214        stats.num_dirty_slots = num_dirty_slots;
7215    }
7216
7217    pub(crate) fn calculate_accounts_hash(
7218        &self,
7219        data_source: CalcAccountsHashDataSource,
7220        slot: Slot,
7221        config: &CalcAccountsHashConfig<'_>,
7222    ) -> Result<(AccountsHash, u64), BankHashVerificationError> {
7223        match data_source {
7224            CalcAccountsHashDataSource::Storages => {
7225                if self.accounts_cache.contains_any_slots(slot) {
7226                    // this indicates a race condition
7227                    inc_new_counter_info!("accounts_hash_items_in_write_cache", 1);
7228                }
7229
7230                let mut collect_time = Measure::start("collect");
7231                let (combined_maps, slots) = self.get_snapshot_storages(..=slot, config.ancestors);
7232                collect_time.stop();
7233
7234                let mut sort_time = Measure::start("sort_storages");
7235                let min_root = self.accounts_index.min_alive_root();
7236                let storages = SortedStorages::new_with_slots(
7237                    combined_maps.iter().zip(slots.into_iter()),
7238                    min_root,
7239                    Some(slot),
7240                );
7241                sort_time.stop();
7242
7243                let mut timings = HashStats {
7244                    collect_snapshots_us: collect_time.as_us(),
7245                    storage_sort_us: sort_time.as_us(),
7246                    ..HashStats::default()
7247                };
7248                timings.calc_storage_size_quartiles(&combined_maps);
7249
7250                self.calculate_accounts_hash_from_storages(config, &storages, timings)
7251            }
7252            CalcAccountsHashDataSource::IndexForTests => {
7253                self.calculate_accounts_hash_from_index(slot, config)
7254            }
7255        }
7256    }
7257
7258    fn calculate_accounts_hash_with_verify(
7259        &self,
7260        data_source: CalcAccountsHashDataSource,
7261        debug_verify: bool,
7262        slot: Slot,
7263        config: CalcAccountsHashConfig<'_>,
7264        expected_capitalization: Option<u64>,
7265    ) -> Result<(AccountsHash, u64), BankHashVerificationError> {
7266        let (accounts_hash, total_lamports) =
7267            self.calculate_accounts_hash(data_source, slot, &config)?;
7268        if debug_verify {
7269            // calculate the other way (store or non-store) and verify results match.
7270            let data_source_other = match data_source {
7271                CalcAccountsHashDataSource::IndexForTests => CalcAccountsHashDataSource::Storages,
7272                CalcAccountsHashDataSource::Storages => CalcAccountsHashDataSource::IndexForTests,
7273            };
7274            let (accounts_hash_other, total_lamports_other) =
7275                self.calculate_accounts_hash(data_source_other, slot, &config)?;
7276
7277            let success = accounts_hash == accounts_hash_other
7278                && total_lamports == total_lamports_other
7279                && total_lamports == expected_capitalization.unwrap_or(total_lamports);
7280            assert!(success, "calculate_accounts_hash_with_verify mismatch. hashes: {}, {}; lamports: {}, {}; expected lamports: {:?}, data source: {:?}, slot: {}", accounts_hash.0, accounts_hash_other.0, total_lamports, total_lamports_other, expected_capitalization, data_source, slot);
7281        }
7282        Ok((accounts_hash, total_lamports))
7283    }
7284
7285    #[allow(clippy::too_many_arguments)]
7286    pub fn update_accounts_hash(
7287        &self,
7288        data_source: CalcAccountsHashDataSource,
7289        debug_verify: bool,
7290        slot: Slot,
7291        ancestors: &Ancestors,
7292        expected_capitalization: Option<u64>,
7293        epoch_schedule: &EpochSchedule,
7294        rent_collector: &RentCollector,
7295        is_startup: bool,
7296    ) -> (AccountsHash, u64) {
7297        let check_hash = false;
7298        let (accounts_hash, total_lamports) = self
7299            .calculate_accounts_hash_with_verify(
7300                data_source,
7301                debug_verify,
7302                slot,
7303                CalcAccountsHashConfig {
7304                    use_bg_thread_pool: !is_startup,
7305                    check_hash,
7306                    ancestors: Some(ancestors),
7307                    epoch_schedule,
7308                    rent_collector,
7309                    store_detailed_debug_info_on_failure: false,
7310                },
7311                expected_capitalization,
7312            )
7313            .unwrap(); // unwrap here will never fail since check_hash = false
7314        self.set_accounts_hash(slot, accounts_hash);
7315        (accounts_hash, total_lamports)
7316    }
7317
7318    /// Set the accounts hash for `slot` in the `accounts_hashes` map
7319    ///
7320    /// returns the previous accounts hash for `slot`
7321    fn set_accounts_hash(&self, slot: Slot, accounts_hash: AccountsHash) -> Option<AccountsHash> {
7322        self.accounts_hashes
7323            .lock()
7324            .unwrap()
7325            .insert(slot, accounts_hash)
7326    }
7327
7328    pub fn set_accounts_hash_from_snapshot(
7329        &self,
7330        slot: Slot,
7331        accounts_hash: AccountsHash,
7332    ) -> Option<AccountsHash> {
7333        self.set_accounts_hash(slot, accounts_hash)
7334    }
7335
7336    /// Get the accounts hash for `slot` in the `accounts_hashes` map
7337    pub fn get_accounts_hash(&self, slot: Slot) -> Option<AccountsHash> {
7338        self.accounts_hashes.lock().unwrap().get(&slot).cloned()
7339    }
7340
7341    /// scan 'storages', return a vec of 'CacheHashDataFile', one per pass
7342    fn scan_snapshot_stores_with_cache(
7343        &self,
7344        cache_hash_data: &CacheHashData,
7345        storages: &SortedStorages,
7346        mut stats: &mut crate::accounts_hash::HashStats,
7347        bins: usize,
7348        bin_range: &Range<usize>,
7349        config: &CalcAccountsHashConfig<'_>,
7350        filler_account_suffix: Option<&Pubkey>,
7351    ) -> Result<Vec<CacheHashDataFile>, BankHashVerificationError> {
7352        let bin_calculator = PubkeyBinCalculator24::new(bins);
7353        assert!(bin_range.start < bins && bin_range.end <= bins && bin_range.start < bin_range.end);
7354        let mut time = Measure::start("scan all accounts");
7355        stats.num_snapshot_storage = storages.storage_count();
7356        stats.num_slots = storages.slot_count();
7357        let mismatch_found = Arc::new(AtomicU64::new(0));
7358        let range = bin_range.end - bin_range.start;
7359        let sort_time = Arc::new(AtomicU64::new(0));
7360
7361        let scanner = ScanState {
7362            current_slot: Slot::default(),
7363            accum: BinnedHashData::default(),
7364            bin_calculator: &bin_calculator,
7365            config,
7366            mismatch_found: mismatch_found.clone(),
7367            filler_account_suffix,
7368            range,
7369            bin_range,
7370            sort_time: sort_time.clone(),
7371            pubkey_to_bin_index: 0,
7372        };
7373
7374        let result = self.scan_account_storage_no_bank(
7375            cache_hash_data,
7376            config,
7377            storages,
7378            scanner,
7379            bin_range,
7380            stats,
7381        );
7382
7383        stats.sort_time_total_us += sort_time.load(Ordering::Relaxed);
7384
7385        if config.check_hash && mismatch_found.load(Ordering::Relaxed) > 0 {
7386            warn!(
7387                "{} mismatched account hash(es) found",
7388                mismatch_found.load(Ordering::Relaxed)
7389            );
7390            return Err(BankHashVerificationError::MismatchedAccountHash);
7391        }
7392
7393        time.stop();
7394        stats.scan_time_total_us += time.as_us();
7395
7396        Ok(result)
7397    }
7398
7399    fn sort_slot_storage_scan(accum: BinnedHashData) -> (BinnedHashData, u64) {
7400        let time = AtomicU64::new(0);
7401        (
7402            accum
7403                .into_iter()
7404                .map(|mut items| {
7405                    let mut sort_time = Measure::start("sort");
7406                    {
7407                        // sort_by vs unstable because slot and write_version are already in order
7408                        items.sort_by(AccountsHasher::compare_two_hash_entries);
7409                    }
7410                    sort_time.stop();
7411                    time.fetch_add(sort_time.as_us(), Ordering::Relaxed);
7412                    items
7413                })
7414                .collect(),
7415            time.load(Ordering::Relaxed),
7416        )
7417    }
7418
7419    /// if we ever try to calc hash where there are squashed append vecs within the last epoch, we will fail
7420    fn assert_safe_squashing_accounts_hash(&self, slot: Slot, epoch_schedule: &EpochSchedule) {
7421        let previous = self.get_accounts_hash_complete_one_epoch_old();
7422        let current = Self::get_slot_one_epoch_prior(slot, epoch_schedule);
7423        assert!(
7424            previous <= current,
7425            "get_accounts_hash_complete_one_epoch_old: {previous}, get_slot_one_epoch_prior: {current}, slot: {slot}"
7426        );
7427    }
7428
7429    /// normal code path returns the common cache path
7430    /// when called after a failure has been detected, redirect the cache storage to a separate folder for debugging later
7431    fn get_cache_hash_data(
7432        accounts_hash_cache_path: PathBuf,
7433        config: &CalcAccountsHashConfig<'_>,
7434        slot: Slot,
7435    ) -> CacheHashData {
7436        if !config.store_detailed_debug_info_on_failure {
7437            CacheHashData::new(accounts_hash_cache_path)
7438        } else {
7439            // this path executes when we are failing with a hash mismatch
7440            let failed_dir = accounts_hash_cache_path
7441                .join("failed_calculate_accounts_hash_cache")
7442                .join(slot.to_string());
7443            let _ = std::fs::remove_dir_all(&failed_dir);
7444            CacheHashData::new(failed_dir)
7445        }
7446    }
7447
7448    // modeled after calculate_accounts_delta_hash
7449    // intended to be faster than calculate_accounts_hash
7450    pub fn calculate_accounts_hash_from_storages(
7451        &self,
7452        config: &CalcAccountsHashConfig<'_>,
7453        storages: &SortedStorages<'_>,
7454        stats: HashStats,
7455    ) -> Result<(AccountsHash, u64), BankHashVerificationError> {
7456        self._calculate_accounts_hash_from_storages(
7457            config,
7458            storages,
7459            stats,
7460            CalcAccountsHashFlavor::Full,
7461            self.full_accounts_hash_cache_path.clone(),
7462        )
7463    }
7464
7465    /// Calculate the incremental accounts hash
7466    ///
7467    /// This calculation is intended to be used by incremental snapshots, and thus differs from a
7468    /// "full" accounts hash in a few ways:
7469    /// - Zero-lamport accounts are *included* in the hash because zero-lamport accounts are also
7470    ///   included in the incremental snapshot.  This ensures reconstructing the AccountsDb is
7471    ///   still correct when using this incremental accounts hash.
7472    /// - `storages` must be *greater than* `base_slot`.  This follows the same requirements as
7473    ///   incremental snapshots.
7474    pub fn calculate_incremental_accounts_hash(
7475        &self,
7476        config: &CalcAccountsHashConfig<'_>,
7477        storages: &SortedStorages<'_>,
7478        base_slot: Slot,
7479        stats: HashStats,
7480    ) -> Result<(AccountsHash, /* capitalization */ u64), BankHashVerificationError> {
7481        assert!(storages.range().start > base_slot, "The storages for calculating an incremental accounts hash must all be higher than the base slot");
7482        self._calculate_accounts_hash_from_storages(
7483            config,
7484            storages,
7485            stats,
7486            CalcAccountsHashFlavor::Incremental,
7487            self.incremental_accounts_hash_cache_path.clone(),
7488        )
7489    }
7490
7491    fn _calculate_accounts_hash_from_storages(
7492        &self,
7493        config: &CalcAccountsHashConfig<'_>,
7494        storages: &SortedStorages<'_>,
7495        mut stats: HashStats,
7496        flavor: CalcAccountsHashFlavor,
7497        accounts_hash_cache_path: PathBuf,
7498    ) -> Result<(AccountsHash, u64), BankHashVerificationError> {
7499        let _guard = self.active_stats.activate(ActiveStatItem::Hash);
7500        stats.oldest_root = storages.range().start;
7501
7502        self.mark_old_slots_as_dirty(storages, config.epoch_schedule.slots_per_epoch, &mut stats);
7503
7504        let use_bg_thread_pool = config.use_bg_thread_pool;
7505        let scan_and_hash = || {
7506            let cache_hash_data = Self::get_cache_hash_data(
7507                accounts_hash_cache_path,
7508                config,
7509                storages.max_slot_inclusive(),
7510            );
7511
7512            let bounds = Range {
7513                start: 0,
7514                end: PUBKEY_BINS_FOR_CALCULATING_HASHES,
7515            };
7516
7517            let accounts_hasher = AccountsHasher {
7518                filler_account_suffix: if self.filler_accounts_config.count > 0 {
7519                    self.filler_account_suffix
7520                } else {
7521                    None
7522                },
7523                zero_lamport_accounts: flavor.zero_lamport_accounts(),
7524            };
7525
7526            // get raw data by scanning
7527            let cache_hash_data_files = self.scan_snapshot_stores_with_cache(
7528                &cache_hash_data,
7529                storages,
7530                &mut stats,
7531                PUBKEY_BINS_FOR_CALCULATING_HASHES,
7532                &bounds,
7533                config,
7534                accounts_hasher.filler_account_suffix.as_ref(),
7535            )?;
7536
7537            // convert mmapped cache files into slices of data
7538            let cache_hash_intermediates = cache_hash_data_files
7539                .iter()
7540                .map(|d| d.get_cache_hash_data())
7541                .collect::<Vec<_>>();
7542
7543            // rework slices of data into bins for parallel processing and to match data shape expected by 'rest_of_hash_calculation'
7544            let result = AccountsHasher::get_binned_data(
7545                &cache_hash_intermediates,
7546                PUBKEY_BINS_FOR_CALCULATING_HASHES,
7547                &bounds,
7548            );
7549
7550            // turn raw data into merkle tree hashes and sum of lamports
7551            let final_result = accounts_hasher.rest_of_hash_calculation(result, &mut stats);
7552            info!(
7553                "calculate_accounts_hash_from_storages: slot: {} {:?}",
7554                storages.max_slot_inclusive(),
7555                final_result
7556            );
7557            let final_result = (AccountsHash(final_result.0), final_result.1);
7558            Ok(final_result)
7559        };
7560
7561        let result = if use_bg_thread_pool {
7562            self.thread_pool_clean.install(scan_and_hash)
7563        } else {
7564            scan_and_hash()
7565        };
7566        self.assert_safe_squashing_accounts_hash(
7567            storages.max_slot_inclusive(),
7568            config.epoch_schedule,
7569        );
7570        stats.log();
7571        result
7572    }
7573
7574    /// return alive roots to retain, even though they are ancient
7575    fn calc_alive_ancient_historical_roots(&self, min_root: Slot) -> HashSet<Slot> {
7576        let mut ancient_alive_roots = HashSet::default();
7577        {
7578            let all_roots = self.accounts_index.roots_tracker.read().unwrap();
7579
7580            if let Some(min) = all_roots.historical_roots.min() {
7581                for slot in min..min_root {
7582                    if all_roots.alive_roots.contains(&slot) {
7583                        // there was a storage for this root, so it counts as a root
7584                        ancient_alive_roots.insert(slot);
7585                    }
7586                }
7587            }
7588        }
7589        ancient_alive_roots
7590    }
7591
7592    /// get rid of historical roots that are older than 'min_root'.
7593    /// These will be older than an epoch from a current root.
7594    fn remove_old_historical_roots(&self, min_root: Slot) {
7595        let alive_roots = self.calc_alive_ancient_historical_roots(min_root);
7596        self.accounts_index
7597            .remove_old_historical_roots(min_root, &alive_roots);
7598    }
7599
7600    /// Only called from startup or test code.
7601    pub fn verify_bank_hash_and_lamports(
7602        &self,
7603        slot: Slot,
7604        total_lamports: u64,
7605        config: BankHashLamportsVerifyConfig,
7606    ) -> Result<(), BankHashVerificationError> {
7607        use BankHashVerificationError::*;
7608
7609        let check_hash = false; // this will not be supported anymore
7610        let (calculated_accounts_hash, calculated_lamports) = self
7611            .calculate_accounts_hash_with_verify(
7612                CalcAccountsHashDataSource::Storages,
7613                config.test_hash_calculation,
7614                slot,
7615                CalcAccountsHashConfig {
7616                    use_bg_thread_pool: config.use_bg_thread_pool,
7617                    check_hash,
7618                    ancestors: Some(config.ancestors),
7619                    epoch_schedule: config.epoch_schedule,
7620                    rent_collector: config.rent_collector,
7621                    store_detailed_debug_info_on_failure: config.store_detailed_debug_info,
7622                },
7623                None,
7624            )?;
7625
7626        if calculated_lamports != total_lamports {
7627            warn!(
7628                "Mismatched total lamports: {} calculated: {}",
7629                total_lamports, calculated_lamports
7630            );
7631            return Err(MismatchedTotalLamports(calculated_lamports, total_lamports));
7632        }
7633
7634        if config.ignore_mismatch {
7635            Ok(())
7636        } else if let Some(found_accounts_hash) = self.get_accounts_hash(slot) {
7637            if calculated_accounts_hash == found_accounts_hash {
7638                Ok(())
7639            } else {
7640                warn!(
7641                    "mismatched bank hash for slot {}: {:?} (calculated) != {:?} (expected)",
7642                    slot, calculated_accounts_hash, found_accounts_hash,
7643                );
7644                Err(MismatchedBankHash)
7645            }
7646        } else {
7647            Err(MissingBankHash)
7648        }
7649    }
7650
7651    /// helper to return
7652    /// 1. pubkey, hash pairs for the slot
7653    /// 2. us spent scanning
7654    /// 3. Measure started when we began accumulating
7655    pub(crate) fn get_pubkey_hash_for_slot(
7656        &self,
7657        slot: Slot,
7658    ) -> (Vec<(Pubkey, Hash)>, u64, Measure) {
7659        let mut scan = Measure::start("scan");
7660
7661        let scan_result: ScanStorageResult<(Pubkey, Hash), DashMap<Pubkey, Hash>> = self
7662            .scan_account_storage(
7663                slot,
7664                |loaded_account: LoadedAccount| {
7665                    // Cache only has one version per key, don't need to worry about versioning
7666                    Some((*loaded_account.pubkey(), loaded_account.loaded_hash()))
7667                },
7668                |accum: &DashMap<Pubkey, Hash>, loaded_account: LoadedAccount| {
7669                    let loaded_hash = loaded_account.loaded_hash();
7670                    accum.insert(*loaded_account.pubkey(), loaded_hash);
7671                },
7672            );
7673        scan.stop();
7674
7675        let accumulate = Measure::start("accumulate");
7676        let hashes: Vec<_> = match scan_result {
7677            ScanStorageResult::Cached(cached_result) => cached_result,
7678            ScanStorageResult::Stored(stored_result) => stored_result.into_iter().collect(),
7679        };
7680        (hashes, scan.as_us(), accumulate)
7681    }
7682
7683    /// Calculate accounts delta hash for `slot`
7684    ///
7685    /// As part of calculating the accounts delta hash, get a list of accounts modified this slot
7686    /// (aka dirty pubkeys) and add them to `self.uncleaned_pubkeys` for future cleaning.
7687    pub fn calculate_accounts_delta_hash(&self, slot: Slot) -> AccountsDeltaHash {
7688        let (mut hashes, scan_us, mut accumulate) = self.get_pubkey_hash_for_slot(slot);
7689        let dirty_keys = hashes.iter().map(|(pubkey, _hash)| *pubkey).collect();
7690
7691        if self.filler_accounts_enabled() {
7692            // filler accounts must be added to 'dirty_keys' above but cannot be used to calculate hash
7693            hashes.retain(|(pubkey, _hash)| !self.is_filler_account(pubkey));
7694        }
7695
7696        let accounts_delta_hash =
7697            AccountsDeltaHash(AccountsHasher::accumulate_account_hashes(hashes));
7698        accumulate.stop();
7699        let mut uncleaned_time = Measure::start("uncleaned_index");
7700        self.uncleaned_pubkeys.insert(slot, dirty_keys);
7701        uncleaned_time.stop();
7702
7703        self.set_accounts_delta_hash(slot, accounts_delta_hash);
7704
7705        self.stats
7706            .store_uncleaned_update
7707            .fetch_add(uncleaned_time.as_us(), Ordering::Relaxed);
7708        self.stats
7709            .delta_hash_scan_time_total_us
7710            .fetch_add(scan_us, Ordering::Relaxed);
7711        self.stats
7712            .delta_hash_accumulate_time_total_us
7713            .fetch_add(accumulate.as_us(), Ordering::Relaxed);
7714        self.stats.delta_hash_num.fetch_add(1, Ordering::Relaxed);
7715        accounts_delta_hash
7716    }
7717
7718    /// Set the accounts delta hash for `slot` in the `accounts_delta_hashes` map
7719    ///
7720    /// returns the previous accounts delta hash for `slot`
7721    fn set_accounts_delta_hash(
7722        &self,
7723        slot: Slot,
7724        accounts_delta_hash: AccountsDeltaHash,
7725    ) -> Option<AccountsDeltaHash> {
7726        self.accounts_delta_hashes
7727            .lock()
7728            .unwrap()
7729            .insert(slot, accounts_delta_hash)
7730    }
7731
7732    pub fn set_accounts_delta_hash_from_snapshot(
7733        &self,
7734        slot: Slot,
7735        accounts_delta_hash: AccountsDeltaHash,
7736    ) -> Option<AccountsDeltaHash> {
7737        self.set_accounts_delta_hash(slot, accounts_delta_hash)
7738    }
7739
7740    /// Get the accounts delta hash for `slot` in the `accounts_delta_hashes` map
7741    pub fn get_accounts_delta_hash(&self, slot: Slot) -> Option<AccountsDeltaHash> {
7742        self.accounts_delta_hashes
7743            .lock()
7744            .unwrap()
7745            .get(&slot)
7746            .cloned()
7747    }
7748
7749    /// When reconstructing AccountsDb from a snapshot, insert the `bank_hash_stats` into the
7750    /// internal bank hash stats map.
7751    ///
7752    /// This fn is only called when loading from a snapshot, which means AccountsDb is new and its
7753    /// bank hash stats map is unpopulated.  Except for slot 0.
7754    ///
7755    /// Slot 0 is a special case.  When a new AccountsDb is created--like when loading from a
7756    /// snapshot--the bank hash stats map is populated with a default entry at slot 0.  Remove the
7757    /// default entry at slot 0, and then insert the new value at `slot`.
7758    pub fn update_bank_hash_stats_from_snapshot(
7759        &self,
7760        slot: Slot,
7761        stats: BankHashStats,
7762    ) -> Option<BankHashStats> {
7763        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
7764        bank_hash_stats.remove(&0);
7765        bank_hash_stats.insert(slot, stats)
7766    }
7767
7768    /// Get the bank hash stats for `slot` in the `bank_hash_stats` map
7769    pub fn get_bank_hash_stats(&self, slot: Slot) -> Option<BankHashStats> {
7770        self.bank_hash_stats.lock().unwrap().get(&slot).cloned()
7771    }
7772
7773    /// Remove "bank hash info" for `slot`
7774    ///
7775    /// This fn removes the accounts delta hash, accounts hash, and bank hash stats for `slot` from
7776    /// their respective maps.
7777    fn remove_bank_hash_info(&self, slot: &Slot) {
7778        self.remove_bank_hash_infos(std::iter::once(slot));
7779    }
7780
7781    /// Remove "bank hash info" for `slots`
7782    ///
7783    /// This fn removes the accounts delta hash, accounts hash, and bank hash stats for `slots` from
7784    /// their respective maps.
7785    fn remove_bank_hash_infos<'s>(&self, slots: impl IntoIterator<Item = &'s Slot>) {
7786        let mut accounts_delta_hashes = self.accounts_delta_hashes.lock().unwrap();
7787        let mut accounts_hashes = self.accounts_hashes.lock().unwrap();
7788        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
7789
7790        for slot in slots.into_iter() {
7791            accounts_delta_hashes.remove(slot);
7792            accounts_hashes.remove(slot);
7793            bank_hash_stats.remove(slot);
7794        }
7795    }
7796
7797    fn update_index<'a, T: ReadableAccount + Sync>(
7798        &self,
7799        infos: Vec<AccountInfo>,
7800        accounts: &impl StorableAccounts<'a, T>,
7801        reclaim: UpsertReclaim,
7802    ) -> SlotList<AccountInfo> {
7803        let target_slot = accounts.target_slot();
7804        // using a thread pool here results in deadlock panics from bank_hashes.write()
7805        // so, instead we limit how many threads will be created to the same size as the bg thread pool
7806        let len = std::cmp::min(accounts.len(), infos.len());
7807        let threshold = 1;
7808        let update = |start, end| {
7809            let mut reclaims = Vec::with_capacity((end - start) / 2);
7810
7811            (start..end).for_each(|i| {
7812                let info = infos[i];
7813                let pubkey_account = (accounts.pubkey(i), accounts.account(i));
7814                let pubkey = pubkey_account.0;
7815                let old_slot = accounts.slot(i);
7816                self.accounts_index.upsert(
7817                    target_slot,
7818                    old_slot,
7819                    pubkey,
7820                    pubkey_account.1,
7821                    &self.account_indexes,
7822                    info,
7823                    &mut reclaims,
7824                    reclaim,
7825                );
7826            });
7827            reclaims
7828        };
7829        if len > threshold {
7830            let chunk_size = std::cmp::max(1, len / quarter_thread_count()); // # pubkeys/thread
7831            let batches = 1 + len / chunk_size;
7832            (0..batches)
7833                .into_par_iter()
7834                .map(|batch| {
7835                    let start = batch * chunk_size;
7836                    let end = std::cmp::min(start + chunk_size, len);
7837                    update(start, end)
7838                })
7839                .flatten()
7840                .collect::<Vec<_>>()
7841        } else {
7842            update(0, len)
7843        }
7844    }
7845
7846    fn should_not_shrink(aligned_bytes: u64, total_bytes: u64) -> bool {
7847        aligned_bytes + PAGE_SIZE > total_bytes
7848    }
7849
7850    fn is_shrinking_productive(slot: Slot, store: &Arc<AccountStorageEntry>) -> bool {
7851        let alive_count = store.count();
7852        let stored_count = store.approx_stored_count();
7853        let alive_bytes = store.alive_bytes();
7854        let total_bytes = store.capacity();
7855
7856        let aligned_bytes = Self::page_align(alive_bytes as u64);
7857        if Self::should_not_shrink(aligned_bytes, total_bytes) {
7858            trace!(
7859                "shrink_slot_forced ({}): not able to shrink at all: alive/stored: ({} / {}) ({}b / {}b) save: {}",
7860                slot,
7861                alive_count,
7862                stored_count,
7863                aligned_bytes,
7864                total_bytes,
7865                total_bytes.saturating_sub(aligned_bytes),
7866            );
7867            return false;
7868        }
7869
7870        true
7871    }
7872
7873    fn is_candidate_for_shrink(
7874        &self,
7875        store: &Arc<AccountStorageEntry>,
7876        allow_shrink_ancient: bool,
7877    ) -> bool {
7878        let total_bytes = if is_ancient(&store.accounts) {
7879            if !allow_shrink_ancient {
7880                return false;
7881            }
7882
7883            store.written_bytes()
7884        } else {
7885            store.capacity()
7886        };
7887        match self.shrink_ratio {
7888            AccountShrinkThreshold::TotalSpace { shrink_ratio: _ } => {
7889                Self::page_align(store.alive_bytes() as u64) < total_bytes
7890            }
7891            AccountShrinkThreshold::IndividualStore { shrink_ratio } => {
7892                (Self::page_align(store.alive_bytes() as u64) as f64 / total_bytes as f64)
7893                    < shrink_ratio
7894            }
7895        }
7896    }
7897
7898    fn remove_dead_accounts<'a, I>(
7899        &'a self,
7900        reclaims: I,
7901        expected_slot: Option<Slot>,
7902        mut reclaimed_offsets: Option<&mut AppendVecOffsets>,
7903        reset_accounts: bool,
7904    ) -> HashSet<Slot>
7905    where
7906        I: Iterator<Item = &'a (Slot, AccountInfo)>,
7907    {
7908        let mut dead_slots = HashSet::new();
7909        let mut new_shrink_candidates: ShrinkCandidates = HashMap::new();
7910        let mut measure = Measure::start("remove");
7911        for (slot, account_info) in reclaims {
7912            // No cached accounts should make it here
7913            assert!(!account_info.is_cached());
7914            if let Some(ref mut reclaimed_offsets) = reclaimed_offsets {
7915                reclaimed_offsets
7916                    .entry(account_info.store_id())
7917                    .or_default()
7918                    .insert(account_info.offset());
7919            }
7920            if let Some(expected_slot) = expected_slot {
7921                assert_eq!(*slot, expected_slot);
7922            }
7923            if let Some(store) = self
7924                .storage
7925                .get_account_storage_entry(*slot, account_info.store_id())
7926            {
7927                assert_eq!(
7928                    *slot, store.slot(),
7929                    "AccountsDB::accounts_index corrupted. Storage pointed to: {}, expected: {}, should only point to one slot",
7930                    store.slot(), *slot
7931                );
7932                let account = store.accounts.get_account(account_info.offset()).unwrap();
7933                if account.1
7934                    != account_info
7935                        .offset()
7936                        .saturating_add(account_info.stored_size() as usize)
7937                {
7938                    // this should never happen. This is a metrics based assert at the moment.
7939                    inc_new_counter_info!("remove_dead_accounts-stored_size_mismatch", 1);
7940                }
7941                let count =
7942                    store.remove_account(account_info.stored_size() as usize, reset_accounts);
7943                if count == 0 {
7944                    self.dirty_stores
7945                        .insert((*slot, store.append_vec_id()), store.clone());
7946                    dead_slots.insert(*slot);
7947                } else if Self::is_shrinking_productive(*slot, &store)
7948                    && self.is_candidate_for_shrink(&store, false)
7949                {
7950                    // Checking that this single storage entry is ready for shrinking,
7951                    // should be a sufficient indication that the slot is ready to be shrunk
7952                    // because slots should only have one storage entry, namely the one that was
7953                    // created by `flush_slot_cache()`.
7954                    {
7955                        new_shrink_candidates.insert(*slot, store);
7956                    }
7957                }
7958            }
7959        }
7960        measure.stop();
7961        self.clean_accounts_stats
7962            .remove_dead_accounts_remove_us
7963            .fetch_add(measure.as_us(), Ordering::Relaxed);
7964
7965        let mut measure = Measure::start("shrink");
7966        let mut shrink_candidate_slots = self.shrink_candidate_slots.lock().unwrap();
7967        for (slot, store) in new_shrink_candidates {
7968            debug!(
7969                "adding: {} {} to shrink candidates: count: {}/{} bytes: {}/{}",
7970                store.append_vec_id(),
7971                slot,
7972                store.approx_stored_count(),
7973                store.count(),
7974                store.alive_bytes(),
7975                store.capacity()
7976            );
7977
7978            shrink_candidate_slots.insert(slot, store);
7979            measure.stop();
7980            self.clean_accounts_stats
7981                .remove_dead_accounts_shrink_us
7982                .fetch_add(measure.as_us(), Ordering::Relaxed);
7983        }
7984
7985        dead_slots.retain(|slot| {
7986            if let Some(slot_store) = self.storage.get_slot_storage_entry(*slot) {
7987                if slot_store.count() != 0 {
7988                    return false;
7989                }
7990            }
7991            true
7992        });
7993
7994        dead_slots
7995    }
7996
7997    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7998    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7999    fn remove_dead_slots_metadata<'a>(
8000        &'a self,
8001        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
8002        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
8003        // Should only be `Some` for non-cached slots
8004        purged_stored_account_slots: Option<&mut AccountSlots>,
8005        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
8006    ) {
8007        let mut measure = Measure::start("remove_dead_slots_metadata-ms");
8008        self.clean_dead_slots_from_accounts_index(
8009            dead_slots_iter.clone(),
8010            purged_slot_pubkeys,
8011            purged_stored_account_slots,
8012            pubkeys_removed_from_accounts_index,
8013        );
8014        self.remove_bank_hash_infos(dead_slots_iter);
8015        measure.stop();
8016        inc_new_counter_info!("remove_dead_slots_metadata-ms", measure.as_ms() as usize);
8017    }
8018
8019    /// lookup each pubkey in 'pubkeys' and unref it in the accounts index
8020    /// skip pubkeys that are in 'pubkeys_removed_from_accounts_index'
8021    fn unref_pubkeys<'a>(
8022        &'a self,
8023        pubkeys: impl Iterator<Item = &'a Pubkey> + Clone + Send + Sync,
8024        num_pubkeys: usize,
8025        pubkeys_removed_from_accounts_index: &'a PubkeysRemovedFromAccountsIndex,
8026    ) {
8027        let batches = 1 + (num_pubkeys / UNREF_ACCOUNTS_BATCH_SIZE);
8028        self.thread_pool_clean.install(|| {
8029            (0..batches).into_par_iter().for_each(|batch| {
8030                let skip = batch * UNREF_ACCOUNTS_BATCH_SIZE;
8031                self.accounts_index.scan(
8032                    pubkeys
8033                        .clone()
8034                        .skip(skip)
8035                        .take(UNREF_ACCOUNTS_BATCH_SIZE)
8036                        .filter(|pubkey| {
8037                            // filter out pubkeys that have already been removed from the accounts index in a previous step
8038                            let already_removed =
8039                                pubkeys_removed_from_accounts_index.contains(pubkey);
8040                            !already_removed
8041                        }),
8042                    |_pubkey, _slots_refs| {
8043                        /* unused */
8044                        AccountsIndexScanResult::Unref
8045                    },
8046                    Some(AccountsIndexScanResult::Unref),
8047                )
8048            });
8049        });
8050    }
8051
8052    /// lookup each pubkey in 'purged_slot_pubkeys' and unref it in the accounts index
8053    /// populate 'purged_stored_account_slots' by grouping 'purged_slot_pubkeys' by pubkey
8054    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
8055    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
8056    fn unref_accounts(
8057        &self,
8058        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
8059        purged_stored_account_slots: &mut AccountSlots,
8060        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
8061    ) {
8062        self.unref_pubkeys(
8063            purged_slot_pubkeys.iter().map(|(_slot, pubkey)| pubkey),
8064            purged_slot_pubkeys.len(),
8065            pubkeys_removed_from_accounts_index,
8066        );
8067        for (slot, pubkey) in purged_slot_pubkeys {
8068            purged_stored_account_slots
8069                .entry(pubkey)
8070                .or_default()
8071                .insert(slot);
8072        }
8073    }
8074
8075    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
8076    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
8077    fn clean_dead_slots_from_accounts_index<'a>(
8078        &'a self,
8079        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
8080        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
8081        // Should only be `Some` for non-cached slots
8082        purged_stored_account_slots: Option<&mut AccountSlots>,
8083        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
8084    ) {
8085        let mut accounts_index_root_stats = AccountsIndexRootsStats::default();
8086        let mut measure = Measure::start("unref_from_storage");
8087        if let Some(purged_stored_account_slots) = purged_stored_account_slots {
8088            self.unref_accounts(
8089                purged_slot_pubkeys,
8090                purged_stored_account_slots,
8091                pubkeys_removed_from_accounts_index,
8092            );
8093        }
8094        measure.stop();
8095        accounts_index_root_stats.clean_unref_from_storage_us += measure.as_us();
8096
8097        let mut measure = Measure::start("clean_dead_slot");
8098        let mut rooted_cleaned_count = 0;
8099        let mut unrooted_cleaned_count = 0;
8100        let dead_slots: Vec<_> = dead_slots_iter
8101            .map(|slot| {
8102                if self
8103                    .accounts_index
8104                    .clean_dead_slot(*slot, &mut accounts_index_root_stats)
8105                {
8106                    rooted_cleaned_count += 1;
8107                } else {
8108                    unrooted_cleaned_count += 1;
8109                }
8110                *slot
8111            })
8112            .collect();
8113        measure.stop();
8114        accounts_index_root_stats.clean_dead_slot_us += measure.as_us();
8115        if self.log_dead_slots.load(Ordering::Relaxed) {
8116            info!(
8117                "remove_dead_slots_metadata: {} dead slots",
8118                dead_slots.len()
8119            );
8120            trace!("remove_dead_slots_metadata: dead_slots: {:?}", dead_slots);
8121        }
8122
8123        accounts_index_root_stats.rooted_cleaned_count += rooted_cleaned_count;
8124        accounts_index_root_stats.unrooted_cleaned_count += unrooted_cleaned_count;
8125
8126        self.clean_accounts_stats
8127            .latest_accounts_index_roots_stats
8128            .update(&accounts_index_root_stats);
8129    }
8130
8131    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
8132    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
8133    fn clean_stored_dead_slots(
8134        &self,
8135        dead_slots: &HashSet<Slot>,
8136        purged_account_slots: Option<&mut AccountSlots>,
8137        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
8138    ) {
8139        let mut measure = Measure::start("clean_stored_dead_slots-ms");
8140        let mut stores = vec![];
8141        // get all stores in a vec so we can iterate in parallel
8142        for slot in dead_slots.iter() {
8143            if let Some(slot_storage) = self.storage.get_slot_storage_entry(*slot) {
8144                stores.push(slot_storage);
8145            }
8146        }
8147        // get all pubkeys in all dead slots
8148        let purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = {
8149            self.thread_pool_clean.install(|| {
8150                stores
8151                    .into_par_iter()
8152                    .map(|store| {
8153                        let slot = store.slot();
8154                        store
8155                            .accounts
8156                            .account_iter()
8157                            .map(|account| (slot, *account.pubkey()))
8158                            .collect::<Vec<(Slot, Pubkey)>>()
8159                    })
8160                    .flatten()
8161                    .collect::<HashSet<_>>()
8162            })
8163        };
8164        self.remove_dead_slots_metadata(
8165            dead_slots.iter(),
8166            purged_slot_pubkeys,
8167            purged_account_slots,
8168            pubkeys_removed_from_accounts_index,
8169        );
8170        measure.stop();
8171        inc_new_counter_info!("clean_stored_dead_slots-ms", measure.as_ms() as usize);
8172        self.clean_accounts_stats
8173            .clean_stored_dead_slots_us
8174            .fetch_add(measure.as_us(), Ordering::Relaxed);
8175    }
8176
8177    pub fn store_cached<'a, T: ReadableAccount + Sync + ZeroLamport + 'a>(
8178        &self,
8179        accounts: impl StorableAccounts<'a, T>,
8180        transactions: Option<&'a [Option<&'a SanitizedTransaction>]>,
8181    ) {
8182        self.store(
8183            accounts,
8184            &StoreTo::Cache,
8185            transactions,
8186            StoreReclaims::Default,
8187        );
8188    }
8189
8190    /// Store the account update.
8191    /// only called by tests
8192    pub fn store_uncached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
8193        let storage = self.find_storage_candidate(slot, 1);
8194        self.store(
8195            (slot, accounts, INCLUDE_SLOT_IN_HASH_TESTS),
8196            &StoreTo::Storage(&storage),
8197            None,
8198            StoreReclaims::Default,
8199        );
8200    }
8201
8202    fn store<'a, T: ReadableAccount + Sync + ZeroLamport + 'a>(
8203        &self,
8204        accounts: impl StorableAccounts<'a, T>,
8205        store_to: &StoreTo,
8206        transactions: Option<&'a [Option<&'a SanitizedTransaction>]>,
8207        reclaim: StoreReclaims,
8208    ) {
8209        // If all transactions in a batch are errored,
8210        // it's possible to get a store with no accounts.
8211        if accounts.is_empty() {
8212            return;
8213        }
8214
8215        let mut stats = BankHashStats::default();
8216        let mut total_data = 0;
8217        (0..accounts.len()).for_each(|index| {
8218            let account = accounts.account(index);
8219            total_data += account.data().len();
8220            stats.update(account);
8221        });
8222
8223        self.stats
8224            .store_total_data
8225            .fetch_add(total_data as u64, Ordering::Relaxed);
8226
8227        {
8228            // we need to drop the bank_hash_stats lock to prevent deadlocks
8229            self.bank_hash_stats
8230                .lock()
8231                .unwrap()
8232                .entry(accounts.target_slot())
8233                .or_insert_with(BankHashStats::default)
8234                .accumulate(&stats);
8235        }
8236
8237        // we use default hashes for now since the same account may be stored to the cache multiple times
8238        self.store_accounts_unfrozen(accounts, None::<Vec<Hash>>, store_to, transactions, reclaim);
8239        self.report_store_timings();
8240    }
8241
8242    fn report_store_timings(&self) {
8243        if self.stats.last_store_report.should_update(1000) {
8244            let (read_only_cache_hits, read_only_cache_misses, read_only_cache_evicts) =
8245                self.read_only_accounts_cache.get_and_reset_stats();
8246            datapoint_info!(
8247                "accounts_db_store_timings",
8248                (
8249                    "hash_accounts",
8250                    self.stats.store_hash_accounts.swap(0, Ordering::Relaxed),
8251                    i64
8252                ),
8253                (
8254                    "store_accounts",
8255                    self.stats.store_accounts.swap(0, Ordering::Relaxed),
8256                    i64
8257                ),
8258                (
8259                    "update_index",
8260                    self.stats.store_update_index.swap(0, Ordering::Relaxed),
8261                    i64
8262                ),
8263                (
8264                    "handle_reclaims",
8265                    self.stats.store_handle_reclaims.swap(0, Ordering::Relaxed),
8266                    i64
8267                ),
8268                (
8269                    "append_accounts",
8270                    self.stats.store_append_accounts.swap(0, Ordering::Relaxed),
8271                    i64
8272                ),
8273                (
8274                    "stakes_cache_check_and_store_us",
8275                    self.stats
8276                        .stakes_cache_check_and_store_us
8277                        .swap(0, Ordering::Relaxed),
8278                    i64
8279                ),
8280                (
8281                    "num_accounts",
8282                    self.stats.store_num_accounts.swap(0, Ordering::Relaxed),
8283                    i64
8284                ),
8285                (
8286                    "total_data",
8287                    self.stats.store_total_data.swap(0, Ordering::Relaxed),
8288                    i64
8289                ),
8290                (
8291                    "read_only_accounts_cache_entries",
8292                    self.read_only_accounts_cache.cache_len(),
8293                    i64
8294                ),
8295                (
8296                    "read_only_accounts_cache_data_size",
8297                    self.read_only_accounts_cache.data_size(),
8298                    i64
8299                ),
8300                ("read_only_accounts_cache_hits", read_only_cache_hits, i64),
8301                (
8302                    "read_only_accounts_cache_misses",
8303                    read_only_cache_misses,
8304                    i64
8305                ),
8306                (
8307                    "read_only_accounts_cache_evicts",
8308                    read_only_cache_evicts,
8309                    i64
8310                ),
8311                (
8312                    "calc_stored_meta_us",
8313                    self.stats.calc_stored_meta.swap(0, Ordering::Relaxed),
8314                    i64
8315                ),
8316            );
8317
8318            let recycle_stores = self.recycle_stores.read().unwrap();
8319            datapoint_info!(
8320                "accounts_db_store_timings2",
8321                (
8322                    "recycle_store_count",
8323                    self.stats.recycle_store_count.swap(0, Ordering::Relaxed),
8324                    i64
8325                ),
8326                (
8327                    "current_recycle_store_count",
8328                    recycle_stores.entry_count(),
8329                    i64
8330                ),
8331                (
8332                    "current_recycle_store_bytes",
8333                    recycle_stores.total_bytes(),
8334                    i64
8335                ),
8336                (
8337                    "create_store_count",
8338                    self.stats.create_store_count.swap(0, Ordering::Relaxed),
8339                    i64
8340                ),
8341                (
8342                    "store_get_slot_store",
8343                    self.stats.store_get_slot_store.swap(0, Ordering::Relaxed),
8344                    i64
8345                ),
8346                (
8347                    "store_find_existing",
8348                    self.stats.store_find_existing.swap(0, Ordering::Relaxed),
8349                    i64
8350                ),
8351                (
8352                    "dropped_stores",
8353                    self.stats.dropped_stores.swap(0, Ordering::Relaxed),
8354                    i64
8355                ),
8356            );
8357        }
8358    }
8359
8360    fn store_accounts_unfrozen<'a, T: ReadableAccount + Sync + ZeroLamport + 'a>(
8361        &self,
8362        accounts: impl StorableAccounts<'a, T>,
8363        hashes: Option<Vec<impl Borrow<Hash>>>,
8364        store_to: &StoreTo,
8365        transactions: Option<&'a [Option<&'a SanitizedTransaction>]>,
8366        reclaim: StoreReclaims,
8367    ) {
8368        // This path comes from a store to a non-frozen slot.
8369        // If a store is dead here, then a newer update for
8370        // each pubkey in the store must exist in another
8371        // store in the slot. Thus it is safe to reset the store and
8372        // re-use it for a future store op. The pubkey ref counts should still
8373        // hold just 1 ref from this slot.
8374        let reset_accounts = true;
8375
8376        self.store_accounts_custom(
8377            accounts,
8378            hashes,
8379            None::<Box<dyn Iterator<Item = u64>>>,
8380            store_to,
8381            reset_accounts,
8382            transactions,
8383            reclaim,
8384        );
8385    }
8386
8387    pub(crate) fn store_accounts_frozen<'a, T: ReadableAccount + Sync + ZeroLamport + 'a>(
8388        &self,
8389        accounts: impl StorableAccounts<'a, T>,
8390        hashes: Option<Vec<impl Borrow<Hash>>>,
8391        storage: &Arc<AccountStorageEntry>,
8392        write_version_producer: Option<Box<dyn Iterator<Item = StoredMetaWriteVersion>>>,
8393        reclaim: StoreReclaims,
8394    ) -> StoreAccountsTiming {
8395        // stores on a frozen slot should not reset
8396        // the append vec so that hashing could happen on the store
8397        // and accounts in the append_vec can be unrefed correctly
8398        let reset_accounts = false;
8399        self.store_accounts_custom(
8400            accounts,
8401            hashes,
8402            write_version_producer,
8403            &StoreTo::Storage(storage),
8404            reset_accounts,
8405            None,
8406            reclaim,
8407        )
8408    }
8409
8410    fn store_accounts_custom<'a, T: ReadableAccount + Sync + ZeroLamport + 'a>(
8411        &self,
8412        accounts: impl StorableAccounts<'a, T>,
8413        hashes: Option<Vec<impl Borrow<Hash>>>,
8414        write_version_producer: Option<Box<dyn Iterator<Item = u64>>>,
8415        store_to: &StoreTo,
8416        reset_accounts: bool,
8417        transactions: Option<&[Option<&SanitizedTransaction>]>,
8418        reclaim: StoreReclaims,
8419    ) -> StoreAccountsTiming {
8420        let write_version_producer: Box<dyn Iterator<Item = u64>> = write_version_producer
8421            .unwrap_or_else(|| {
8422                let mut current_version = self.bulk_assign_write_version(accounts.len());
8423                Box::new(std::iter::from_fn(move || {
8424                    let ret = current_version;
8425                    current_version += 1;
8426                    Some(ret)
8427                }))
8428            });
8429
8430        self.stats
8431            .store_num_accounts
8432            .fetch_add(accounts.len() as u64, Ordering::Relaxed);
8433        let mut store_accounts_time = Measure::start("store_accounts");
8434        let infos = self.store_accounts_to(
8435            &accounts,
8436            hashes,
8437            write_version_producer,
8438            store_to,
8439            transactions,
8440        );
8441        store_accounts_time.stop();
8442        self.stats
8443            .store_accounts
8444            .fetch_add(store_accounts_time.as_us(), Ordering::Relaxed);
8445        let mut update_index_time = Measure::start("update_index");
8446
8447        let reclaim = if matches!(reclaim, StoreReclaims::Ignore) {
8448            UpsertReclaim::IgnoreReclaims
8449        } else if store_to.is_cached() {
8450            UpsertReclaim::PreviousSlotEntryWasCached
8451        } else {
8452            UpsertReclaim::PopulateReclaims
8453        };
8454
8455        // if we are squashing a single slot, then we can expect a single dead slot
8456        let expected_single_dead_slot =
8457            (!accounts.contains_multiple_slots()).then(|| accounts.target_slot());
8458
8459        // If the cache was flushed, then because `update_index` occurs
8460        // after the account are stored by the above `store_accounts_to`
8461        // call and all the accounts are stored, all reads after this point
8462        // will know to not check the cache anymore
8463        let mut reclaims = self.update_index(infos, &accounts, reclaim);
8464
8465        // For each updated account, `reclaims` should only have at most one
8466        // item (if the account was previously updated in this slot).
8467        // filter out the cached reclaims as those don't actually map
8468        // to anything that needs to be cleaned in the backing storage
8469        // entries
8470        reclaims.retain(|(_, r)| !r.is_cached());
8471
8472        if store_to.is_cached() {
8473            assert!(reclaims.is_empty());
8474        }
8475
8476        update_index_time.stop();
8477        self.stats
8478            .store_update_index
8479            .fetch_add(update_index_time.as_us(), Ordering::Relaxed);
8480
8481        // A store for a single slot should:
8482        // 1) Only make "reclaims" for the same slot
8483        // 2) Should not cause any slots to be removed from the storage
8484        // database because
8485        //    a) this slot  has at least one account (the one being stored),
8486        //    b)From 1) we know no other slots are included in the "reclaims"
8487        //
8488        // From 1) and 2) we guarantee passing `no_purge_stats` == None, which is
8489        // equivalent to asserting there will be no dead slots, is safe.
8490        let mut handle_reclaims_time = Measure::start("handle_reclaims");
8491        self.handle_reclaims(
8492            (!reclaims.is_empty()).then(|| reclaims.iter()),
8493            expected_single_dead_slot,
8494            None,
8495            reset_accounts,
8496            &HashSet::default(),
8497        );
8498        handle_reclaims_time.stop();
8499        self.stats
8500            .store_handle_reclaims
8501            .fetch_add(handle_reclaims_time.as_us(), Ordering::Relaxed);
8502
8503        StoreAccountsTiming {
8504            store_accounts_elapsed: store_accounts_time.as_us(),
8505            update_index_elapsed: update_index_time.as_us(),
8506            handle_reclaims_elapsed: handle_reclaims_time.as_us(),
8507        }
8508    }
8509
8510    pub fn add_root(&self, slot: Slot) -> AccountsAddRootTiming {
8511        let mut index_time = Measure::start("index_add_root");
8512        self.accounts_index.add_root(slot);
8513        index_time.stop();
8514        let mut cache_time = Measure::start("cache_add_root");
8515        self.accounts_cache.add_root(slot);
8516        cache_time.stop();
8517        let mut store_time = Measure::start("store_add_root");
8518        // We would not expect this slot to be shrinking right now, but other slots may be.
8519        // But, even if it was, we would just mark a store id as dirty unnecessarily and that is ok.
8520        // So, allow shrinking to be in progress.
8521        if let Some(store) = self
8522            .storage
8523            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
8524        {
8525            self.dirty_stores
8526                .insert((slot, store.append_vec_id()), store);
8527        }
8528        store_time.stop();
8529
8530        AccountsAddRootTiming {
8531            index_us: index_time.as_us(),
8532            cache_us: cache_time.as_us(),
8533            store_us: store_time.as_us(),
8534        }
8535    }
8536
8537    /// Get storages to use for snapshots, for the requested slots
8538    pub fn get_snapshot_storages(
8539        &self,
8540        requested_slots: impl RangeBounds<Slot> + Sync,
8541        ancestors: Option<&Ancestors>,
8542    ) -> (Vec<Arc<AccountStorageEntry>>, Vec<Slot>) {
8543        let mut m = Measure::start("get slots");
8544        let mut slots_and_storages = self
8545            .storage
8546            .iter()
8547            .filter_map(|(slot, store)| {
8548                requested_slots
8549                    .contains(&slot)
8550                    .then_some((slot, Some(store)))
8551            })
8552            .collect::<Vec<_>>();
8553        m.stop();
8554        let mut m2 = Measure::start("filter");
8555
8556        let chunk_size = 5_000;
8557        let wide = self.thread_pool_clean.install(|| {
8558            slots_and_storages
8559                .par_chunks_mut(chunk_size)
8560                .map(|slots_and_storages| {
8561                    slots_and_storages
8562                        .iter_mut()
8563                        .filter(|(slot, _)| {
8564                            self.accounts_index.is_alive_root(*slot)
8565                                || ancestors
8566                                    .map(|ancestors| ancestors.contains_key(slot))
8567                                    .unwrap_or_default()
8568                        })
8569                        .filter_map(|(slot, store)| {
8570                            let store = std::mem::take(store).unwrap();
8571                            store.has_accounts().then_some((store, *slot))
8572                        })
8573                        .collect::<Vec<(Arc<AccountStorageEntry>, Slot)>>()
8574                })
8575                .collect::<Vec<_>>()
8576        });
8577        m2.stop();
8578        let mut m3 = Measure::start("flatten");
8579        // some slots we found above may not have been a root or met the slot # constraint.
8580        // So the resulting 'slots' vector we return will be a subset of the raw keys we got initially.
8581        let mut slots = Vec::with_capacity(slots_and_storages.len());
8582        let result = wide
8583            .into_iter()
8584            .flatten()
8585            .map(|(storage, slot)| {
8586                slots.push(slot);
8587                storage
8588            })
8589            .collect::<Vec<_>>();
8590        m3.stop();
8591
8592        debug!(
8593            "hash_total: get slots: {}, filter: {}, flatten: {}",
8594            m.as_us(),
8595            m2.as_us(),
8596            m3.as_us()
8597        );
8598        (result, slots)
8599    }
8600
8601    fn process_storage_slot<'a>(
8602        &self,
8603        storage: &'a Arc<AccountStorageEntry>,
8604    ) -> GenerateIndexAccountsMap<'a> {
8605        let num_accounts = storage.approx_stored_count();
8606        let mut accounts_map = GenerateIndexAccountsMap::with_capacity(num_accounts);
8607        storage.accounts.account_iter().for_each(|stored_account| {
8608            let this_version = stored_account.meta.write_version_obsolete;
8609            let pubkey = stored_account.pubkey();
8610            assert!(!self.is_filler_account(pubkey));
8611            accounts_map.insert(
8612                *pubkey,
8613                IndexAccountMapEntry {
8614                    write_version: this_version,
8615                    store_id: storage.append_vec_id(),
8616                    stored_account,
8617                },
8618            );
8619        });
8620        accounts_map
8621    }
8622
8623    /// return Some(lamports_to_top_off) if 'account' would collect rent
8624    fn stats_for_rent_payers<T: ReadableAccount>(
8625        pubkey: &Pubkey,
8626        account: &T,
8627        rent_collector: &RentCollector,
8628    ) -> Option<u64> {
8629        if account.lamports() == 0 {
8630            return None;
8631        }
8632        (rent_collector.should_collect_rent(pubkey, account)
8633            && !rent_collector.get_rent_due(account).is_exempt())
8634        .then(|| {
8635            let min_balance = rent_collector.rent.minimum_balance(account.data().len());
8636            // return lamports required to top off this account to make it rent exempt
8637            min_balance.saturating_sub(account.lamports())
8638        })
8639    }
8640
8641    fn generate_index_for_slot(
8642        &self,
8643        accounts_map: GenerateIndexAccountsMap<'_>,
8644        slot: &Slot,
8645        rent_collector: &RentCollector,
8646    ) -> SlotIndexGenerationInfo {
8647        if accounts_map.is_empty() {
8648            return SlotIndexGenerationInfo::default();
8649        }
8650
8651        let secondary = !self.account_indexes.is_empty();
8652
8653        let mut rent_paying_accounts_by_partition = Vec::default();
8654        let mut accounts_data_len = 0;
8655        let mut num_accounts_rent_paying = 0;
8656        let num_accounts = accounts_map.len();
8657        let mut amount_to_top_off_rent = 0;
8658        let items = accounts_map.into_iter().map(
8659            |(
8660                pubkey,
8661                IndexAccountMapEntry {
8662                    write_version: _write_version,
8663                    store_id,
8664                    stored_account,
8665                },
8666            )| {
8667                if secondary {
8668                    self.accounts_index.update_secondary_indexes(
8669                        &pubkey,
8670                        &stored_account,
8671                        &self.account_indexes,
8672                    );
8673                }
8674                if !stored_account.is_zero_lamport() {
8675                    accounts_data_len += stored_account.data().len() as u64;
8676                }
8677
8678                if let Some(amount_to_top_off_rent_this_account) =
8679                    Self::stats_for_rent_payers(&pubkey, &stored_account, rent_collector)
8680                {
8681                    amount_to_top_off_rent += amount_to_top_off_rent_this_account;
8682                    num_accounts_rent_paying += 1;
8683                    // remember this rent-paying account pubkey
8684                    rent_paying_accounts_by_partition.push(pubkey);
8685                }
8686
8687                (
8688                    pubkey,
8689                    AccountInfo::new(
8690                        StorageLocation::AppendVec(store_id, stored_account.offset), // will never be cached
8691                        stored_account.stored_size as StoredSize, // stored_size should never exceed StoredSize::MAX because of max data len const
8692                        stored_account.account_meta.lamports,
8693                    ),
8694                )
8695            },
8696        );
8697
8698        let (dirty_pubkeys, insert_time_us) = self
8699            .accounts_index
8700            .insert_new_if_missing_into_primary_index(*slot, num_accounts, items);
8701
8702        // dirty_pubkeys will contain a pubkey if an item has multiple rooted entries for
8703        // a given pubkey. If there is just a single item, there is no cleaning to
8704        // be done on that pubkey. Use only those pubkeys with multiple updates.
8705        if !dirty_pubkeys.is_empty() {
8706            self.uncleaned_pubkeys.insert(*slot, dirty_pubkeys);
8707        }
8708        SlotIndexGenerationInfo {
8709            insert_time_us,
8710            num_accounts: num_accounts as u64,
8711            num_accounts_rent_paying,
8712            accounts_data_len,
8713            amount_to_top_off_rent,
8714            rent_paying_accounts_by_partition,
8715        }
8716    }
8717
8718    fn filler_unique_id_bytes() -> usize {
8719        std::mem::size_of::<u32>()
8720    }
8721
8722    fn filler_rent_partition_prefix_bytes() -> usize {
8723        std::mem::size_of::<u64>()
8724    }
8725
8726    fn filler_prefix_bytes() -> usize {
8727        Self::filler_unique_id_bytes() + Self::filler_rent_partition_prefix_bytes()
8728    }
8729
8730    pub fn is_filler_account_helper(
8731        pubkey: &Pubkey,
8732        filler_account_suffix: Option<&Pubkey>,
8733    ) -> bool {
8734        let offset = Self::filler_prefix_bytes();
8735        filler_account_suffix
8736            .as_ref()
8737            .map(|filler_account_suffix| {
8738                pubkey.as_ref()[offset..] == filler_account_suffix.as_ref()[offset..]
8739            })
8740            .unwrap_or_default()
8741    }
8742
8743    /// true if 'pubkey' is a filler account
8744    pub fn is_filler_account(&self, pubkey: &Pubkey) -> bool {
8745        Self::is_filler_account_helper(pubkey, self.filler_account_suffix.as_ref())
8746    }
8747
8748    /// true if it is possible that there are filler accounts present
8749    pub fn filler_accounts_enabled(&self) -> bool {
8750        self.filler_account_suffix.is_some()
8751    }
8752
8753    /// return 'AccountSharedData' and a hash for a filler account
8754    fn get_filler_account(&self, rent: &Rent) -> (AccountSharedData, Hash) {
8755        let string = "FiLLERACCoUNTooooooooooooooooooooooooooooooo";
8756        let hash = Hash::from_str(string).unwrap();
8757        let owner = Pubkey::from_str(string).unwrap();
8758        let space = self.filler_accounts_config.size;
8759        let rent_exempt_reserve = rent.minimum_balance(space);
8760        let lamports = rent_exempt_reserve;
8761        let mut account = AccountSharedData::new(lamports, space, &owner);
8762        // just non-zero rent epoch. filler accounts are rent-exempt
8763        let dummy_rent_epoch = 2;
8764        account.set_rent_epoch(dummy_rent_epoch);
8765        (account, hash)
8766    }
8767
8768    fn get_filler_account_pubkeys(&self, count: usize) -> Vec<Pubkey> {
8769        (0..count)
8770            .map(|_| {
8771                let subrange = solana_sdk::pubkey::new_rand();
8772                self.get_filler_account_pubkey(&subrange)
8773            })
8774            .collect()
8775    }
8776
8777    fn get_filler_account_pubkey(&self, subrange: &Pubkey) -> Pubkey {
8778        // pubkey begins life as entire filler 'suffix' pubkey
8779        let mut key = self.filler_account_suffix.unwrap();
8780        let rent_prefix_bytes = Self::filler_rent_partition_prefix_bytes();
8781        // first bytes are replaced with rent partition range: filler_rent_partition_prefix_bytes
8782        key.as_mut()[0..rent_prefix_bytes]
8783            .copy_from_slice(&subrange.as_ref()[0..rent_prefix_bytes]);
8784        key
8785    }
8786
8787    /// filler accounts are space-holding accounts which are ignored by hash calculations and rent.
8788    /// They are designed to allow a validator to run against a network successfully while simulating having many more accounts present.
8789    /// All filler accounts share a common pubkey suffix. The suffix is randomly generated per validator on startup.
8790    /// The filler accounts are added to each slot in the snapshot after index generation.
8791    /// The accounts added in a slot are setup to have pubkeys such that rent will be collected from them before (or when?) their slot becomes an epoch old.
8792    /// Thus, the filler accounts are rewritten by rent and the old slot can be thrown away successfully.
8793    pub fn maybe_add_filler_accounts(&self, epoch_schedule: &EpochSchedule, slot: Slot) {
8794        if self.filler_accounts_config.count == 0 {
8795            return;
8796        }
8797
8798        self.init_gradual_filler_accounts(
8799            epoch_schedule.get_slots_in_epoch(epoch_schedule.get_epoch(slot)),
8800        );
8801    }
8802
8803    #[allow(clippy::needless_collect)]
8804    pub fn generate_index(
8805        &self,
8806        limit_load_slot_count_from_snapshot: Option<usize>,
8807        verify: bool,
8808        genesis_config: &GenesisConfig,
8809    ) -> IndexGenerationInfo {
8810        let mut slots = self.storage.all_slots();
8811        #[allow(clippy::stable_sort_primitive)]
8812        slots.sort();
8813        if let Some(limit) = limit_load_slot_count_from_snapshot {
8814            slots.truncate(limit); // get rid of the newer slots and keep just the older
8815        }
8816        let max_slot = slots.last().cloned().unwrap_or_default();
8817        let schedule = genesis_config.epoch_schedule;
8818        let rent_collector = RentCollector::new(
8819            schedule.get_epoch(max_slot),
8820            schedule,
8821            genesis_config.slots_per_year(),
8822            genesis_config.rent,
8823        );
8824        let accounts_data_len = AtomicU64::new(0);
8825
8826        let rent_paying_accounts_by_partition =
8827            Mutex::new(RentPayingAccountsByPartition::new(&schedule));
8828
8829        // pass == 0 always runs and generates the index
8830        // pass == 1 only runs if verify == true.
8831        // verify checks that all the expected items are in the accounts index and measures how long it takes to look them all up
8832        let passes = if verify { 2 } else { 1 };
8833        for pass in 0..passes {
8834            if pass == 0 {
8835                self.accounts_index
8836                    .set_startup(Startup::StartupWithExtraThreads);
8837            }
8838            let storage_info = StorageSizeAndCountMap::default();
8839            let total_processed_slots_across_all_threads = AtomicU64::new(0);
8840            let outer_slots_len = slots.len();
8841            let threads = if self.accounts_index.is_disk_index_enabled() {
8842                // these write directly to disk, so the more threads, the better
8843                num_cpus::get()
8844            } else {
8845                // seems to be a good hueristic given varying # cpus for in-mem disk index
8846                8
8847            };
8848            let chunk_size = (outer_slots_len / (std::cmp::max(1, threads.saturating_sub(1)))) + 1; // approximately 400k slots in a snapshot
8849            let mut index_time = Measure::start("index");
8850            let insertion_time_us = AtomicU64::new(0);
8851            let rent_paying = AtomicUsize::new(0);
8852            let amount_to_top_off_rent = AtomicU64::new(0);
8853            let total_duplicates = AtomicU64::new(0);
8854            let storage_info_timings = Mutex::new(GenerateIndexTimings::default());
8855            let scan_time: u64 = slots
8856                .par_chunks(chunk_size)
8857                .map(|slots| {
8858                    let mut log_status = MultiThreadProgress::new(
8859                        &total_processed_slots_across_all_threads,
8860                        2,
8861                        outer_slots_len as u64,
8862                    );
8863                    let mut scan_time_sum = 0;
8864                    for (index, slot) in slots.iter().enumerate() {
8865                        let mut scan_time = Measure::start("scan");
8866                        log_status.report(index as u64);
8867                        let storage = self.storage.get_slot_storage_entry(*slot);
8868                        let accounts_map = storage
8869                            .as_ref()
8870                            .map(|storage| self.process_storage_slot(storage))
8871                            .unwrap_or_default();
8872
8873                        scan_time.stop();
8874                        scan_time_sum += scan_time.as_us();
8875                        Self::update_storage_info(
8876                            &storage_info,
8877                            &accounts_map,
8878                            &storage_info_timings,
8879                        );
8880
8881                        let insert_us = if pass == 0 {
8882                            // generate index
8883                            self.maybe_throttle_index_generation();
8884                            let SlotIndexGenerationInfo {
8885                                insert_time_us: insert_us,
8886                                num_accounts: total_this_slot,
8887                                num_accounts_rent_paying: rent_paying_this_slot,
8888                                accounts_data_len: accounts_data_len_this_slot,
8889                                amount_to_top_off_rent: amount_to_top_off_rent_this_slot,
8890                                rent_paying_accounts_by_partition:
8891                                    rent_paying_accounts_by_partition_this_slot,
8892                            } = self.generate_index_for_slot(accounts_map, slot, &rent_collector);
8893                            rent_paying.fetch_add(rent_paying_this_slot, Ordering::Relaxed);
8894                            amount_to_top_off_rent
8895                                .fetch_add(amount_to_top_off_rent_this_slot, Ordering::Relaxed);
8896                            total_duplicates.fetch_add(total_this_slot, Ordering::Relaxed);
8897                            accounts_data_len
8898                                .fetch_add(accounts_data_len_this_slot, Ordering::Relaxed);
8899                            let mut rent_paying_accounts_by_partition =
8900                                rent_paying_accounts_by_partition.lock().unwrap();
8901                            rent_paying_accounts_by_partition_this_slot
8902                                .iter()
8903                                .for_each(|k| {
8904                                    rent_paying_accounts_by_partition.add_account(k);
8905                                });
8906
8907                            insert_us
8908                        } else {
8909                            // verify index matches expected and measure the time to get all items
8910                            assert!(verify);
8911                            let mut lookup_time = Measure::start("lookup_time");
8912                            for account in accounts_map.into_iter() {
8913                                let (key, account_info) = account;
8914                                let lock = self.accounts_index.get_bin(&key);
8915                                let x = lock.get(&key).unwrap();
8916                                let sl = x.slot_list.read().unwrap();
8917                                let mut count = 0;
8918                                for (slot2, account_info2) in sl.iter() {
8919                                    if slot2 == slot {
8920                                        count += 1;
8921                                        let ai = AccountInfo::new(
8922                                            StorageLocation::AppendVec(
8923                                                account_info.store_id,
8924                                                account_info.stored_account.offset,
8925                                            ), // will never be cached
8926                                            account_info.stored_account.stored_size as StoredSize, // stored_size should never exceed StoredSize::MAX because of max data len const
8927                                            account_info.stored_account.account_meta.lamports,
8928                                        );
8929                                        assert_eq!(&ai, account_info2);
8930                                    }
8931                                }
8932                                assert_eq!(1, count);
8933                            }
8934                            lookup_time.stop();
8935                            lookup_time.as_us()
8936                        };
8937                        insertion_time_us.fetch_add(insert_us, Ordering::Relaxed);
8938                    }
8939                    scan_time_sum
8940                })
8941                .sum();
8942            index_time.stop();
8943
8944            info!("rent_collector: {:?}", rent_collector);
8945            let mut min_bin_size = usize::MAX;
8946            let mut max_bin_size = usize::MIN;
8947            let total_items = self
8948                .accounts_index
8949                .account_maps
8950                .iter()
8951                .map(|map_bin| {
8952                    let len = map_bin.len_for_stats();
8953                    min_bin_size = std::cmp::min(min_bin_size, len);
8954                    max_bin_size = std::cmp::max(max_bin_size, len);
8955                    len
8956                })
8957                .sum();
8958
8959            let mut index_flush_us = 0;
8960            if pass == 0 {
8961                // tell accounts index we are done adding the initial accounts at startup
8962                let mut m = Measure::start("accounts_index_idle_us");
8963                self.accounts_index.set_startup(Startup::Normal);
8964                m.stop();
8965                index_flush_us = m.as_us();
8966
8967                // this has to happen before visit_duplicate_pubkeys_during_startup below
8968                // get duplicate keys from acct idx. We have to wait until we've finished flushing.
8969                for (slot, key) in self
8970                    .accounts_index
8971                    .retrieve_duplicate_keys_from_startup()
8972                    .into_iter()
8973                    .flatten()
8974                {
8975                    match self.uncleaned_pubkeys.entry(slot) {
8976                        Occupied(mut occupied) => occupied.get_mut().push(key),
8977                        Vacant(vacant) => {
8978                            vacant.insert(vec![key]);
8979                        }
8980                    }
8981                }
8982            }
8983
8984            let storage_info_timings = storage_info_timings.into_inner().unwrap();
8985            let mut timings = GenerateIndexTimings {
8986                index_flush_us,
8987                scan_time,
8988                index_time: index_time.as_us(),
8989                insertion_time_us: insertion_time_us.load(Ordering::Relaxed),
8990                min_bin_size,
8991                max_bin_size,
8992                total_items,
8993                rent_paying,
8994                amount_to_top_off_rent,
8995                total_duplicates: total_duplicates.load(Ordering::Relaxed),
8996                storage_size_accounts_map_us: storage_info_timings.storage_size_accounts_map_us,
8997                storage_size_accounts_map_flatten_us: storage_info_timings
8998                    .storage_size_accounts_map_flatten_us,
8999                ..GenerateIndexTimings::default()
9000            };
9001
9002            // subtract data.len() from accounts_data_len for all old accounts that are in the index twice
9003            let mut accounts_data_len_dedup_timer =
9004                Measure::start("handle accounts data len duplicates");
9005            let uncleaned_roots = Mutex::new(HashSet::<Slot>::default());
9006            if pass == 0 {
9007                let mut unique_pubkeys = HashSet::<Pubkey>::default();
9008                self.uncleaned_pubkeys.iter().for_each(|entry| {
9009                    entry.value().iter().for_each(|pubkey| {
9010                        unique_pubkeys.insert(*pubkey);
9011                    })
9012                });
9013                let accounts_data_len_from_duplicates = unique_pubkeys
9014                    .into_iter()
9015                    .collect::<Vec<_>>()
9016                    .par_chunks(4096)
9017                    .map(|pubkeys| {
9018                        let (count, uncleaned_roots_this_group) = self
9019                            .visit_duplicate_pubkeys_during_startup(
9020                                pubkeys,
9021                                &rent_collector,
9022                                &timings,
9023                            );
9024                        let mut uncleaned_roots = uncleaned_roots.lock().unwrap();
9025                        uncleaned_roots_this_group.into_iter().for_each(|slot| {
9026                            uncleaned_roots.insert(slot);
9027                        });
9028                        count
9029                    })
9030                    .sum();
9031                accounts_data_len.fetch_sub(accounts_data_len_from_duplicates, Ordering::Relaxed);
9032                info!(
9033                    "accounts data len: {}",
9034                    accounts_data_len.load(Ordering::Relaxed)
9035                );
9036            }
9037            accounts_data_len_dedup_timer.stop();
9038            timings.accounts_data_len_dedup_time_us = accounts_data_len_dedup_timer.as_us();
9039
9040            if pass == 0 {
9041                let uncleaned_roots = uncleaned_roots.into_inner().unwrap();
9042                // Need to add these last, otherwise older updates will be cleaned
9043                for root in &slots {
9044                    self.accounts_index.add_root(*root);
9045                }
9046                self.accounts_index
9047                    .add_uncleaned_roots(uncleaned_roots.into_iter());
9048
9049                self.set_storage_count_and_alive_bytes(storage_info, &mut timings);
9050            }
9051            timings.report();
9052        }
9053
9054        self.accounts_index.log_secondary_indexes();
9055
9056        IndexGenerationInfo {
9057            accounts_data_len: accounts_data_len.load(Ordering::Relaxed),
9058            rent_paying_accounts_by_partition: rent_paying_accounts_by_partition
9059                .into_inner()
9060                .unwrap(),
9061        }
9062    }
9063
9064    /// Startup processes can consume large amounts of memory while inserting accounts into the index as fast as possible.
9065    /// Calling this can slow down the insertion process to allow flushing to disk to keep pace.
9066    fn maybe_throttle_index_generation(&self) {
9067        // This number is chosen to keep the initial ram usage sufficiently small
9068        // The process of generating the index is goverened entirely by how fast the disk index can be populated.
9069        // 10M accounts is sufficiently small that it will never have memory usage. It seems sufficiently large that it will provide sufficient performance.
9070        // Performance is measured by total time to generate the index.
9071        // Just estimating - 150M accounts can easily be held in memory in the accounts index on a 256G machine. 2-300M are also likely 'fine' during startup.
9072        // 550M was straining a 384G machine at startup.
9073        // This is a tunable parameter that just needs to be small enough to keep the generation threads from overwhelming RAM and oom at startup.
9074        const LIMIT: usize = 10_000_000;
9075        while self
9076            .accounts_index
9077            .get_startup_remaining_items_to_flush_estimate()
9078            > LIMIT
9079        {
9080            // 10 ms is long enough to allow some flushing to occur before insertion is resumed.
9081            // callers of this are typically run in parallel, so many threads will be sleeping at different starting intervals, waiting to resume insertion.
9082            sleep(Duration::from_millis(10));
9083        }
9084    }
9085
9086    /// Used during generate_index() to:
9087    /// 1. get the _duplicate_ accounts data len from the given pubkeys
9088    /// 2. get the slots that contained duplicate pubkeys
9089    /// 3. update rent stats
9090    /// Note this should only be used when ALL entries in the accounts index are roots.
9091    /// returns (data len sum of all older duplicates, slots that contained duplicate pubkeys)
9092    fn visit_duplicate_pubkeys_during_startup(
9093        &self,
9094        pubkeys: &[Pubkey],
9095        rent_collector: &RentCollector,
9096        timings: &GenerateIndexTimings,
9097    ) -> (u64, HashSet<Slot>) {
9098        let mut accounts_data_len_from_duplicates = 0;
9099        let mut uncleaned_slots = HashSet::<Slot>::default();
9100        let mut removed_rent_paying = 0;
9101        let mut removed_top_off = 0;
9102        pubkeys.iter().for_each(|pubkey| {
9103            if let Some(entry) = self.accounts_index.get_account_read_entry(pubkey) {
9104                let slot_list = entry.slot_list();
9105                if slot_list.len() < 2 {
9106                    return;
9107                }
9108                // Only the account data len in the highest slot should be used, and the rest are
9109                // duplicates.  So find the max slot to keep.
9110                // Then sum up the remaining data len, which are the duplicates.
9111                // All of the slots need to go in the 'uncleaned_slots' list. For clean to work properly,
9112                // the slot where duplicate accounts are found in the index need to be in 'uncleaned_slots' list, too.
9113                let max = slot_list.iter().map(|(slot, _)| slot).max().unwrap();
9114                slot_list.iter().for_each(|(slot, account_info)| {
9115                    uncleaned_slots.insert(*slot);
9116                    if slot == max {
9117                        // the info in 'max' is the most recent, current info for this pubkey
9118                        return;
9119                    }
9120                    let maybe_storage_entry = self
9121                        .storage
9122                        .get_account_storage_entry(*slot, account_info.store_id());
9123                    let mut accessor = LoadedAccountAccessor::Stored(
9124                        maybe_storage_entry.map(|entry| (entry, account_info.offset())),
9125                    );
9126                    let loaded_account = accessor.check_and_get_loaded_account();
9127                    accounts_data_len_from_duplicates += loaded_account.data().len();
9128                    if let Some(lamports_to_top_off) =
9129                        Self::stats_for_rent_payers(pubkey, &loaded_account, rent_collector)
9130                    {
9131                        removed_rent_paying += 1;
9132                        removed_top_off += lamports_to_top_off;
9133                    }
9134                });
9135            }
9136        });
9137        timings
9138            .rent_paying
9139            .fetch_sub(removed_rent_paying, Ordering::Relaxed);
9140        timings
9141            .amount_to_top_off_rent
9142            .fetch_sub(removed_top_off, Ordering::Relaxed);
9143        (accounts_data_len_from_duplicates as u64, uncleaned_slots)
9144    }
9145
9146    fn update_storage_info(
9147        storage_info: &StorageSizeAndCountMap,
9148        accounts_map: &GenerateIndexAccountsMap<'_>,
9149        timings: &Mutex<GenerateIndexTimings>,
9150    ) {
9151        let mut storage_size_accounts_map_time = Measure::start("storage_size_accounts_map");
9152
9153        let mut storage_info_local = HashMap::<AppendVecId, StorageSizeAndCount>::default();
9154        // first collect into a local HashMap with no lock contention
9155        for (_, v) in accounts_map.iter() {
9156            let mut info = storage_info_local
9157                .entry(v.store_id)
9158                .or_insert_with(StorageSizeAndCount::default);
9159            info.stored_size += v.stored_account.stored_size;
9160            info.count += 1;
9161        }
9162        storage_size_accounts_map_time.stop();
9163        // second, collect into the shared DashMap once we've figured out all the info per store_id
9164        let mut storage_size_accounts_map_flatten_time =
9165            Measure::start("storage_size_accounts_map_flatten_time");
9166        for (store_id, v) in storage_info_local.into_iter() {
9167            let mut info = storage_info
9168                .entry(store_id)
9169                .or_insert_with(StorageSizeAndCount::default);
9170            info.stored_size += v.stored_size;
9171            info.count += v.count;
9172        }
9173        storage_size_accounts_map_flatten_time.stop();
9174
9175        let mut timings = timings.lock().unwrap();
9176        timings.storage_size_accounts_map_us += storage_size_accounts_map_time.as_us();
9177        timings.storage_size_accounts_map_flatten_us +=
9178            storage_size_accounts_map_flatten_time.as_us();
9179    }
9180    fn set_storage_count_and_alive_bytes(
9181        &self,
9182        stored_sizes_and_counts: StorageSizeAndCountMap,
9183        timings: &mut GenerateIndexTimings,
9184    ) {
9185        // store count and size for each storage
9186        let mut storage_size_storages_time = Measure::start("storage_size_storages");
9187        for (_slot, store) in self.storage.iter() {
9188            let id = store.append_vec_id();
9189            // Should be default at this point
9190            assert_eq!(store.alive_bytes(), 0);
9191            if let Some(entry) = stored_sizes_and_counts.get(&id) {
9192                trace!(
9193                    "id: {} setting count: {} cur: {}",
9194                    id,
9195                    entry.count,
9196                    store.count(),
9197                );
9198                store.count_and_status.write().unwrap().0 = entry.count;
9199                store.alive_bytes.store(entry.stored_size, Ordering::SeqCst);
9200            } else {
9201                trace!("id: {} clearing count", id);
9202                store.count_and_status.write().unwrap().0 = 0;
9203            }
9204        }
9205        storage_size_storages_time.stop();
9206        timings.storage_size_storages_us = storage_size_storages_time.as_us();
9207    }
9208
9209    pub(crate) fn print_accounts_stats(&self, label: &str) {
9210        self.print_index(label);
9211        self.print_count_and_status(label);
9212        info!("recycle_stores:");
9213        let recycle_stores = self.recycle_stores.read().unwrap();
9214        for (recycled_time, entry) in recycle_stores.iter() {
9215            info!(
9216                "  slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {} (recycled: {:?})",
9217                entry.slot(),
9218                entry.append_vec_id(),
9219                *entry.count_and_status.read().unwrap(),
9220                entry.approx_store_count.load(Ordering::Relaxed),
9221                entry.accounts.len(),
9222                entry.accounts.capacity(),
9223                recycled_time,
9224            );
9225        }
9226    }
9227
9228    fn print_index(&self, label: &str) {
9229        let mut alive_roots: Vec<_> = self.accounts_index.all_alive_roots();
9230        #[allow(clippy::stable_sort_primitive)]
9231        alive_roots.sort();
9232        info!("{}: accounts_index alive_roots: {:?}", label, alive_roots,);
9233        let full_pubkey_range = Pubkey::from([0; 32])..=Pubkey::from([0xff; 32]);
9234
9235        self.accounts_index.account_maps.iter().for_each(|map| {
9236            for (pubkey, account_entry) in map.items(&full_pubkey_range) {
9237                info!("  key: {} ref_count: {}", pubkey, account_entry.ref_count(),);
9238                info!(
9239                    "      slots: {:?}",
9240                    *account_entry.slot_list.read().unwrap()
9241                );
9242            }
9243        });
9244    }
9245
9246    fn print_count_and_status(&self, label: &str) {
9247        let mut slots: Vec<_> = self.storage.all_slots();
9248        #[allow(clippy::stable_sort_primitive)]
9249        slots.sort();
9250        info!("{}: count_and status for {} slots:", label, slots.len());
9251        for slot in &slots {
9252            let entry = self.storage.get_slot_storage_entry(*slot).unwrap();
9253            info!(
9254                "  slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {}",
9255                slot,
9256                entry.append_vec_id(),
9257                *entry.count_and_status.read().unwrap(),
9258                entry.approx_store_count.load(Ordering::Relaxed),
9259                entry.accounts.len(),
9260                entry.accounts.capacity(),
9261            );
9262        }
9263    }
9264}
9265
9266/// Specify the source of the accounts data when calculating the accounts hash
9267///
9268/// Using the Index is meant for testing the hash calculation itself and debugging;
9269/// not intended during normal validator operation.
9270#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9271pub enum CalcAccountsHashDataSource {
9272    IndexForTests,
9273    Storages,
9274}
9275
9276/// Which accounts hash calculation is being performed?
9277#[derive(Debug)]
9278enum CalcAccountsHashFlavor {
9279    Full,
9280    Incremental,
9281}
9282
9283impl CalcAccountsHashFlavor {
9284    /// How should zero-lamport accounts be handled by this accounts hash calculation?
9285    fn zero_lamport_accounts(&self) -> ZeroLamportAccounts {
9286        match self {
9287            CalcAccountsHashFlavor::Full => ZeroLamportAccounts::Excluded,
9288            CalcAccountsHashFlavor::Incremental => ZeroLamportAccounts::Included,
9289        }
9290    }
9291}
9292
9293#[cfg(test)]
9294impl AccountsDb {
9295    pub fn new(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
9296        Self::new_for_tests(paths, cluster_type)
9297    }
9298
9299    pub fn new_with_config_for_tests(
9300        paths: Vec<PathBuf>,
9301        cluster_type: &ClusterType,
9302        account_indexes: AccountSecondaryIndexes,
9303        shrink_ratio: AccountShrinkThreshold,
9304    ) -> Self {
9305        Self::new_with_config(
9306            paths,
9307            cluster_type,
9308            account_indexes,
9309            shrink_ratio,
9310            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
9311            None,
9312            &Arc::default(),
9313        )
9314    }
9315
9316    pub fn new_sized(paths: Vec<PathBuf>, file_size: u64) -> Self {
9317        AccountsDb {
9318            file_size,
9319            ..AccountsDb::new(paths, &ClusterType::Development)
9320        }
9321    }
9322
9323    pub fn new_sized_caching(paths: Vec<PathBuf>, file_size: u64) -> Self {
9324        AccountsDb {
9325            file_size,
9326            ..AccountsDb::new(paths, &ClusterType::Development)
9327        }
9328    }
9329
9330    pub fn new_sized_no_extra_stores(paths: Vec<PathBuf>, file_size: u64) -> Self {
9331        AccountsDb {
9332            file_size,
9333            ..AccountsDb::new(paths, &ClusterType::Development)
9334        }
9335    }
9336
9337    pub fn get_append_vec_id(&self, pubkey: &Pubkey, slot: Slot) -> Option<AppendVecId> {
9338        let ancestors = vec![(slot, 1)].into_iter().collect();
9339        let result = self.accounts_index.get(pubkey, Some(&ancestors), None);
9340        result.map(|(list, index)| list.slot_list()[index].1.store_id())
9341    }
9342
9343    pub fn alive_account_count_in_slot(&self, slot: Slot) -> usize {
9344        self.storage
9345            .get_slot_storage_entry(slot)
9346            .map(|storage| storage.count())
9347            .unwrap_or(0)
9348            .saturating_add(
9349                self.accounts_cache
9350                    .slot_cache(slot)
9351                    .map(|slot_cache| slot_cache.len())
9352                    .unwrap_or_default(),
9353            )
9354    }
9355}
9356
9357/// A set of utility functions used for testing and benchmarking
9358pub mod test_utils {
9359    use {
9360        super::*,
9361        crate::{accounts::Accounts, append_vec::aligned_stored_size},
9362    };
9363
9364    pub fn create_test_accounts(
9365        accounts: &Accounts,
9366        pubkeys: &mut Vec<Pubkey>,
9367        num: usize,
9368        slot: Slot,
9369    ) {
9370        let data_size = 0;
9371        if accounts
9372            .accounts_db
9373            .storage
9374            .get_slot_storage_entry(slot)
9375            .is_none()
9376        {
9377            let bytes_required = num * aligned_stored_size(data_size);
9378            // allocate an append vec for this slot that can hold all the test accounts. This prevents us from creating more than 1 append vec for this slot.
9379            _ = accounts.accounts_db.create_and_insert_store(
9380                slot,
9381                bytes_required as u64,
9382                "create_test_accounts",
9383            );
9384        }
9385
9386        for t in 0..num {
9387            let pubkey = solana_sdk::pubkey::new_rand();
9388            let account = AccountSharedData::new(
9389                (t + 1) as u64,
9390                data_size,
9391                AccountSharedData::default().owner(),
9392            );
9393            accounts.store_slow_uncached(slot, &pubkey, &account);
9394            pubkeys.push(pubkey);
9395        }
9396    }
9397
9398    // Only used by bench, not safe to call otherwise accounts can conflict with the
9399    // accounts cache!
9400    pub fn update_accounts_bench(accounts: &Accounts, pubkeys: &[Pubkey], slot: u64) {
9401        for pubkey in pubkeys {
9402            let amount = thread_rng().gen_range(0, 10);
9403            let account = AccountSharedData::new(amount, 0, AccountSharedData::default().owner());
9404            accounts.store_slow_uncached(slot, pubkey, &account);
9405        }
9406    }
9407}
9408
9409#[cfg(test)]
9410pub mod tests {
9411    use {
9412        super::*,
9413        crate::{
9414            accounts::Accounts,
9415            accounts_hash::MERKLE_FANOUT,
9416            accounts_index::{
9417                tests::*, AccountIndex, AccountSecondaryIndexes,
9418                AccountSecondaryIndexesIncludeExclude, ReadAccountMapEntry, RefCount,
9419            },
9420            append_vec::{test_utils::TempFile, AccountMeta, StoredMeta},
9421            cache_hash_data_stats::CacheHashDataStats,
9422            inline_spl_token,
9423            secondary_index::MAX_NUM_LARGEST_INDEX_KEYS_RETURNED,
9424        },
9425        assert_matches::assert_matches,
9426        itertools::Itertools,
9427        rand::{distributions::Uniform, prelude::SliceRandom, thread_rng, Rng},
9428        solana_sdk::{
9429            account::{
9430                accounts_equal, Account, AccountSharedData, ReadableAccount, WritableAccount,
9431            },
9432            hash::HASH_BYTES,
9433            pubkey::PUBKEY_BYTES,
9434            system_program,
9435        },
9436        std::{
9437            iter::FromIterator,
9438            str::FromStr,
9439            sync::atomic::AtomicBool,
9440            thread::{self, Builder, JoinHandle},
9441        },
9442    };
9443
9444    fn linear_ancestors(end_slot: u64) -> Ancestors {
9445        let mut ancestors: Ancestors = vec![(0, 0)].into_iter().collect();
9446        for i in 1..end_slot {
9447            ancestors.insert(i, (i - 1) as usize);
9448        }
9449        ancestors
9450    }
9451
9452    fn empty_storages<'a>() -> SortedStorages<'a> {
9453        SortedStorages::new(&[])
9454    }
9455
9456    impl AccountsDb {
9457        fn scan_snapshot_stores(
9458            &self,
9459            storage: &SortedStorages,
9460            stats: &mut crate::accounts_hash::HashStats,
9461            bins: usize,
9462            bin_range: &Range<usize>,
9463            check_hash: bool,
9464        ) -> Result<Vec<CacheHashDataFile>, BankHashVerificationError> {
9465            let temp_dir = TempDir::new().unwrap();
9466            let accounts_hash_cache_path = temp_dir.path().to_path_buf();
9467            self.scan_snapshot_stores_with_cache(
9468                &CacheHashData::new(accounts_hash_cache_path),
9469                storage,
9470                stats,
9471                bins,
9472                bin_range,
9473                &CalcAccountsHashConfig {
9474                    check_hash,
9475                    ..CalcAccountsHashConfig::default()
9476                },
9477                None,
9478            )
9479        }
9480
9481        fn load_without_fixed_root(
9482            &self,
9483            ancestors: &Ancestors,
9484            pubkey: &Pubkey,
9485        ) -> Option<(AccountSharedData, Slot)> {
9486            self.do_load(
9487                ancestors,
9488                pubkey,
9489                None,
9490                LoadHint::Unspecified,
9491                // callers of this expect zero lamport accounts that exist in the index to be returned as Some(empty)
9492                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
9493            )
9494        }
9495
9496        fn get_storage_for_slot(&self, slot: Slot) -> Option<Arc<AccountStorageEntry>> {
9497            self.storage.get_slot_storage_entry(slot)
9498        }
9499
9500        // used by serde_snapshot tests
9501        pub fn set_accounts_hash_for_tests(&self, slot: Slot, accounts_hash: AccountsHash) {
9502            self.set_accounts_hash(slot, accounts_hash);
9503        }
9504
9505        // used by serde_snapshot tests
9506        pub fn set_accounts_delta_hash_for_tests(
9507            &self,
9508            slot: Slot,
9509            accounts_delta_hash: AccountsDeltaHash,
9510        ) {
9511            self.set_accounts_delta_hash(slot, accounts_delta_hash);
9512        }
9513    }
9514
9515    /// This impl exists until this feature is activated:
9516    ///  ignore slot when calculating an account hash #28420
9517    /// For now, all test code will continue to work thanks to this impl
9518    /// Tests will use INCLUDE_SLOT_IN_HASH_TESTS for 'include_slot_in_hash' calls.
9519    impl<'a, T: ReadableAccount + Sync> StorableAccounts<'a, T> for (Slot, &'a [(&'a Pubkey, &'a T)]) {
9520        fn pubkey(&self, index: usize) -> &Pubkey {
9521            self.1[index].0
9522        }
9523        fn account(&self, index: usize) -> &T {
9524            self.1[index].1
9525        }
9526        fn slot(&self, _index: usize) -> Slot {
9527            // per-index slot is not unique per slot when per-account slot is not included in the source data
9528            self.target_slot()
9529        }
9530        fn target_slot(&self) -> Slot {
9531            self.0
9532        }
9533        fn len(&self) -> usize {
9534            self.1.len()
9535        }
9536        fn include_slot_in_hash(&self) -> IncludeSlotInHash {
9537            INCLUDE_SLOT_IN_HASH_TESTS
9538        }
9539    }
9540
9541    /// this tuple contains slot info PER account
9542    impl<'a, T: ReadableAccount + Sync> StorableAccounts<'a, T>
9543        for (Slot, &'a [(&'a Pubkey, &'a T, Slot)])
9544    {
9545        fn pubkey(&self, index: usize) -> &Pubkey {
9546            self.1[index].0
9547        }
9548        fn account(&self, index: usize) -> &T {
9549            self.1[index].1
9550        }
9551        fn slot(&self, index: usize) -> Slot {
9552            // note that this could be different than 'target_slot()' PER account
9553            self.1[index].2
9554        }
9555        fn target_slot(&self) -> Slot {
9556            self.0
9557        }
9558        fn len(&self) -> usize {
9559            self.1.len()
9560        }
9561        fn contains_multiple_slots(&self) -> bool {
9562            let len = self.len();
9563            if len > 0 {
9564                let slot = self.slot(0);
9565                // true if any item has a different slot than the first item
9566                (1..len).any(|i| slot != self.slot(i))
9567            } else {
9568                false
9569            }
9570        }
9571        fn include_slot_in_hash(&self) -> IncludeSlotInHash {
9572            INCLUDE_SLOT_IN_HASH_TESTS
9573        }
9574    }
9575
9576    impl CurrentAncientAppendVec {
9577        /// note this requires that 'slot_and_append_vec' is Some
9578        fn append_vec_id(&self) -> AppendVecId {
9579            self.append_vec().append_vec_id()
9580        }
9581    }
9582
9583    impl<'a> BankHashLamportsVerifyConfig<'a> {
9584        fn new_for_test(
9585            ancestors: &'a Ancestors,
9586            epoch_schedule: &'a EpochSchedule,
9587            rent_collector: &'a RentCollector,
9588        ) -> BankHashLamportsVerifyConfig<'a> {
9589            BankHashLamportsVerifyConfig {
9590                ancestors,
9591                test_hash_calculation: true,
9592                epoch_schedule,
9593                rent_collector,
9594                ignore_mismatch: false,
9595                store_detailed_debug_info: false,
9596                use_bg_thread_pool: false,
9597            }
9598        }
9599    }
9600
9601    #[test]
9602    fn test_maybe_unref_accounts_already_in_ancient() {
9603        let db = AccountsDb::new_single_for_tests();
9604        let slot0 = 0;
9605        let slot1 = 1;
9606        let available_bytes = 1_000_000;
9607        let mut current_ancient = CurrentAncientAppendVec::default();
9608
9609        // setup 'to_store'
9610        let pubkey = Pubkey::from([1; 32]);
9611        let account_size = 3;
9612
9613        let account = AccountSharedData::default();
9614
9615        let account_meta = AccountMeta {
9616            lamports: 1,
9617            owner: Pubkey::from([2; 32]),
9618            executable: false,
9619            rent_epoch: 0,
9620        };
9621        let offset = 3;
9622        let hash = Hash::new(&[2; 32]);
9623        let stored_meta = StoredMeta {
9624            /// global write version
9625            write_version_obsolete: 0,
9626            /// key for the account
9627            pubkey,
9628            data_len: 43,
9629        };
9630        let account = StoredAccountMeta {
9631            meta: &stored_meta,
9632            /// account data
9633            account_meta: &account_meta,
9634            data: account.data(),
9635            offset,
9636            stored_size: account_size,
9637            hash: &hash,
9638        };
9639        let map = vec![&account];
9640        let alive_total_bytes = account.stored_size;
9641        let to_store = AccountsToStore::new(available_bytes, &map, alive_total_bytes, slot0);
9642        // Done: setup 'to_store'
9643
9644        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
9645        let _existing_append_vec = db.create_and_insert_store(slot0, 1000, "test");
9646        {
9647            let _shrink_in_progress = current_ancient.create_ancient_append_vec(slot0, &db);
9648        }
9649        let mut ancient_slot_pubkeys = AncientSlotPubkeys::default();
9650        assert!(ancient_slot_pubkeys.inner.is_none());
9651        // same slot as current_ancient, so no-op
9652        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
9653            slot0,
9654            &db,
9655            &current_ancient,
9656            &to_store,
9657        );
9658        assert!(ancient_slot_pubkeys.inner.is_none());
9659        // different slot than current_ancient, so update 'ancient_slot_pubkeys'
9660        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
9661        let _existing_append_vec = db.create_and_insert_store(slot1, 1000, "test");
9662        let _shrink_in_progress = current_ancient.create_ancient_append_vec(slot1, &db);
9663        let slot2 = 2;
9664        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
9665            slot2,
9666            &db,
9667            &current_ancient,
9668            &to_store,
9669        );
9670        assert!(ancient_slot_pubkeys.inner.is_some());
9671        assert_eq!(ancient_slot_pubkeys.inner.as_ref().unwrap().slot, slot1);
9672        assert!(ancient_slot_pubkeys
9673            .inner
9674            .as_ref()
9675            .unwrap()
9676            .pubkeys
9677            .contains(&pubkey));
9678        assert_eq!(
9679            ancient_slot_pubkeys.inner.as_ref().unwrap().pubkeys.len(),
9680            1
9681        );
9682    }
9683
9684    #[test]
9685    fn test_get_keys_to_unref_ancient() {
9686        let rent_epoch = 0;
9687        let lamports = 0;
9688        let executable = false;
9689        let owner = Pubkey::default();
9690        let data = Vec::new();
9691
9692        let pubkey = solana_sdk::pubkey::new_rand();
9693        let pubkey2 = solana_sdk::pubkey::new_rand();
9694        let pubkey3 = solana_sdk::pubkey::new_rand();
9695        let pubkey4 = solana_sdk::pubkey::new_rand();
9696
9697        let meta = StoredMeta {
9698            write_version_obsolete: 5,
9699            pubkey,
9700            data_len: 7,
9701        };
9702        let meta2 = StoredMeta {
9703            write_version_obsolete: 5,
9704            pubkey: pubkey2,
9705            data_len: 7,
9706        };
9707        let meta3 = StoredMeta {
9708            write_version_obsolete: 5,
9709            pubkey: pubkey3,
9710            data_len: 7,
9711        };
9712        let meta4 = StoredMeta {
9713            write_version_obsolete: 5,
9714            pubkey: pubkey4,
9715            data_len: 7,
9716        };
9717        let account_meta = AccountMeta {
9718            lamports,
9719            owner,
9720            executable,
9721            rent_epoch,
9722        };
9723        let offset = 99;
9724        let stored_size = 101;
9725        let hash = Hash::new_unique();
9726        let stored_account = StoredAccountMeta {
9727            meta: &meta,
9728            account_meta: &account_meta,
9729            data: &data,
9730            offset,
9731            stored_size,
9732            hash: &hash,
9733        };
9734        let stored_account2 = StoredAccountMeta {
9735            meta: &meta2,
9736            account_meta: &account_meta,
9737            data: &data,
9738            offset,
9739            stored_size,
9740            hash: &hash,
9741        };
9742        let stored_account3 = StoredAccountMeta {
9743            meta: &meta3,
9744            account_meta: &account_meta,
9745            data: &data,
9746            offset,
9747            stored_size,
9748            hash: &hash,
9749        };
9750        let stored_account4 = StoredAccountMeta {
9751            meta: &meta4,
9752            account_meta: &account_meta,
9753            data: &data,
9754            offset,
9755            stored_size,
9756            hash: &hash,
9757        };
9758        let mut existing_ancient_pubkeys = HashSet::default();
9759        let accounts = [&stored_account];
9760        // pubkey NOT in existing_ancient_pubkeys, so do NOT unref, but add to existing_ancient_pubkeys
9761        let unrefs =
9762            AccountsDb::get_keys_to_unref_ancient(&accounts, &mut existing_ancient_pubkeys);
9763        assert!(unrefs.is_empty());
9764        assert_eq!(
9765            existing_ancient_pubkeys.iter().collect::<Vec<_>>(),
9766            vec![&pubkey]
9767        );
9768        // pubkey already in existing_ancient_pubkeys, so DO unref
9769        let unrefs =
9770            AccountsDb::get_keys_to_unref_ancient(&accounts, &mut existing_ancient_pubkeys);
9771        assert_eq!(
9772            existing_ancient_pubkeys.iter().collect::<Vec<_>>(),
9773            vec![&pubkey]
9774        );
9775        assert_eq!(unrefs.iter().cloned().collect::<Vec<_>>(), vec![&pubkey]);
9776        // pubkey2 NOT in existing_ancient_pubkeys, so do NOT unref, but add to existing_ancient_pubkeys
9777        let accounts = [&stored_account2];
9778        let unrefs =
9779            AccountsDb::get_keys_to_unref_ancient(&accounts, &mut existing_ancient_pubkeys);
9780        assert!(unrefs.is_empty());
9781        assert_eq!(
9782            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9783            vec![&pubkey, &pubkey2]
9784                .into_iter()
9785                .sorted()
9786                .collect::<Vec<_>>()
9787        );
9788        // pubkey2 already in existing_ancient_pubkeys, so DO unref
9789        let unrefs =
9790            AccountsDb::get_keys_to_unref_ancient(&accounts, &mut existing_ancient_pubkeys);
9791        assert_eq!(
9792            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9793            vec![&pubkey, &pubkey2]
9794                .into_iter()
9795                .sorted()
9796                .collect::<Vec<_>>()
9797        );
9798        assert_eq!(unrefs.iter().cloned().collect::<Vec<_>>(), vec![&pubkey2]);
9799        // pubkey3/4 NOT in existing_ancient_pubkeys, so do NOT unref, but add to existing_ancient_pubkeys
9800        let accounts = [&stored_account3, &stored_account4];
9801        let unrefs =
9802            AccountsDb::get_keys_to_unref_ancient(&accounts, &mut existing_ancient_pubkeys);
9803        assert!(unrefs.is_empty());
9804        assert_eq!(
9805            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9806            vec![&pubkey, &pubkey2, &pubkey3, &pubkey4]
9807                .into_iter()
9808                .sorted()
9809                .collect::<Vec<_>>()
9810        );
9811        // pubkey3/4 already in existing_ancient_pubkeys, so DO unref
9812        let unrefs =
9813            AccountsDb::get_keys_to_unref_ancient(&accounts, &mut existing_ancient_pubkeys);
9814        assert_eq!(
9815            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9816            vec![&pubkey, &pubkey2, &pubkey3, &pubkey4]
9817                .into_iter()
9818                .sorted()
9819                .collect::<Vec<_>>()
9820        );
9821        assert_eq!(
9822            unrefs.iter().cloned().sorted().collect::<Vec<_>>(),
9823            vec![&pubkey3, &pubkey4]
9824                .into_iter()
9825                .sorted()
9826                .collect::<Vec<_>>()
9827        );
9828    }
9829
9830    #[test]
9831    #[should_panic(
9832        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
9833    )]
9834    fn test_accountsdb_scan_snapshot_stores_illegal_range_start() {
9835        let mut stats = HashStats::default();
9836        let bounds = Range { start: 2, end: 2 };
9837        let accounts_db = AccountsDb::new_single_for_tests();
9838
9839        accounts_db
9840            .scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false)
9841            .unwrap();
9842    }
9843    #[test]
9844    #[should_panic(
9845        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
9846    )]
9847    fn test_accountsdb_scan_snapshot_stores_illegal_range_end() {
9848        let mut stats = HashStats::default();
9849        let bounds = Range { start: 1, end: 3 };
9850
9851        let accounts_db = AccountsDb::new_single_for_tests();
9852        accounts_db
9853            .scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false)
9854            .unwrap();
9855    }
9856
9857    #[test]
9858    #[should_panic(
9859        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
9860    )]
9861    fn test_accountsdb_scan_snapshot_stores_illegal_range_inverse() {
9862        let mut stats = HashStats::default();
9863        let bounds = Range { start: 1, end: 0 };
9864
9865        let accounts_db = AccountsDb::new_single_for_tests();
9866        accounts_db
9867            .scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false)
9868            .unwrap();
9869    }
9870
9871    fn sample_storages_and_account_in_slot(
9872        slot: Slot,
9873        accounts: &AccountsDb,
9874    ) -> (
9875        Vec<Arc<AccountStorageEntry>>,
9876        Vec<CalculateHashIntermediate>,
9877    ) {
9878        let pubkey0 = Pubkey::from([0u8; 32]);
9879        let pubkey127 = Pubkey::from([0x7fu8; 32]);
9880        let pubkey128 = Pubkey::from([0x80u8; 32]);
9881        let pubkey255 = Pubkey::from([0xffu8; 32]);
9882
9883        let mut raw_expected = vec![
9884            CalculateHashIntermediate::new(Hash::default(), 1, pubkey0),
9885            CalculateHashIntermediate::new(Hash::default(), 128, pubkey127),
9886            CalculateHashIntermediate::new(Hash::default(), 129, pubkey128),
9887            CalculateHashIntermediate::new(Hash::default(), 256, pubkey255),
9888        ];
9889
9890        let expected_hashes = vec![
9891            Hash::from_str("5K3NW73xFHwgTWVe4LyCg4QfQda8f88uZj2ypDx2kmmH").unwrap(),
9892            Hash::from_str("84ozw83MZ8oeSF4hRAg7SeW1Tqs9LMXagX1BrDRjtZEx").unwrap(),
9893            Hash::from_str("5XqtnEJ41CG2JWNp7MAg9nxkRUAnyjLxfsKsdrLxQUbC").unwrap(),
9894            Hash::from_str("DpvwJcznzwULYh19Zu5CuAA4AT6WTBe4H6n15prATmqj").unwrap(),
9895        ];
9896
9897        let mut raw_accounts = Vec::default();
9898
9899        for i in 0..raw_expected.len() {
9900            raw_accounts.push(AccountSharedData::new(
9901                raw_expected[i].lamports,
9902                1,
9903                AccountSharedData::default().owner(),
9904            ));
9905            let hash = AccountsDb::hash_account(
9906                slot,
9907                &raw_accounts[i],
9908                &raw_expected[i].pubkey,
9909                INCLUDE_SLOT_IN_HASH_TESTS,
9910            );
9911            if slot == 1 {
9912                assert_eq!(hash, expected_hashes[i]);
9913            }
9914            raw_expected[i].hash = hash;
9915        }
9916
9917        let to_store = raw_accounts
9918            .iter()
9919            .zip(raw_expected.iter())
9920            .map(|(account, intermediate)| (&intermediate.pubkey, account))
9921            .collect::<Vec<_>>();
9922
9923        accounts.store_for_tests(slot, &to_store[..]);
9924        accounts.add_root_and_flush_write_cache(slot);
9925
9926        let (storages, slots) = accounts.get_snapshot_storages(..=slot, None);
9927        assert_eq!(storages.len(), slots.len());
9928        storages
9929            .iter()
9930            .zip(slots.iter())
9931            .for_each(|(storage, slot)| {
9932                assert_eq!(&storage.slot(), slot);
9933            });
9934        (storages, raw_expected)
9935    }
9936
9937    fn sample_storages_and_accounts(
9938        accounts: &AccountsDb,
9939    ) -> (
9940        Vec<Arc<AccountStorageEntry>>,
9941        Vec<CalculateHashIntermediate>,
9942    ) {
9943        sample_storages_and_account_in_slot(1, accounts)
9944    }
9945
9946    fn get_storage_refs(input: &[Arc<AccountStorageEntry>]) -> SortedStorages {
9947        SortedStorages::new(input)
9948    }
9949
9950    /// helper to compare expected binned data with scan result in cache files
9951    /// result: return from scanning
9952    /// expected: binned data expected
9953    /// bins: # bins total to divide pubkeys into
9954    /// start_bin_index: bin # that was the minimum # we were scanning for 0<=start_bin_index<bins
9955    /// bin_range: end_exclusive-start_bin_index passed to scan
9956    fn assert_scan(
9957        result: Vec<CacheHashDataFile>,
9958        expected: Vec<BinnedHashData>,
9959        bins: usize,
9960        start_bin_index: usize,
9961        bin_range: usize,
9962    ) {
9963        assert_eq!(expected.len(), result.len());
9964
9965        for cache_file in &result {
9966            let mut result2 = (0..bin_range).map(|_| Vec::default()).collect::<Vec<_>>();
9967            cache_file.load_all(
9968                &mut result2,
9969                start_bin_index,
9970                &PubkeyBinCalculator24::new(bins),
9971                &mut CacheHashDataStats::default(),
9972            );
9973            assert_eq!(
9974                convert_to_slice(&[result2]),
9975                expected,
9976                "bins: {bins}, start_bin_index: {start_bin_index}"
9977            );
9978        }
9979    }
9980
9981    #[test]
9982    fn test_accountsdb_scan_snapshot_stores() {
9983        solana_logger::setup();
9984        let accounts_db = AccountsDb::new_single_for_tests();
9985        let (storages, raw_expected) = sample_storages_and_accounts(&accounts_db);
9986
9987        let bins = 1;
9988        let mut stats = HashStats::default();
9989
9990        let result = accounts_db
9991            .scan_snapshot_stores(
9992                &get_storage_refs(&storages),
9993                &mut stats,
9994                bins,
9995                &Range {
9996                    start: 0,
9997                    end: bins,
9998                },
9999                false,
10000            )
10001            .unwrap();
10002        assert_scan(result, vec![vec![raw_expected.clone()]], bins, 0, bins);
10003
10004        let bins = 2;
10005        let accounts_db = AccountsDb::new_single_for_tests();
10006        let result = accounts_db
10007            .scan_snapshot_stores(
10008                &get_storage_refs(&storages),
10009                &mut stats,
10010                bins,
10011                &Range {
10012                    start: 0,
10013                    end: bins,
10014                },
10015                false,
10016            )
10017            .unwrap();
10018        let mut expected = vec![Vec::new(); bins];
10019        expected[0].push(raw_expected[0].clone());
10020        expected[0].push(raw_expected[1].clone());
10021        expected[bins - 1].push(raw_expected[2].clone());
10022        expected[bins - 1].push(raw_expected[3].clone());
10023        assert_scan(result, vec![expected], bins, 0, bins);
10024
10025        let bins = 4;
10026        let accounts_db = AccountsDb::new_single_for_tests();
10027        let result = accounts_db
10028            .scan_snapshot_stores(
10029                &get_storage_refs(&storages),
10030                &mut stats,
10031                bins,
10032                &Range {
10033                    start: 0,
10034                    end: bins,
10035                },
10036                false,
10037            )
10038            .unwrap();
10039        let mut expected = vec![Vec::new(); bins];
10040        expected[0].push(raw_expected[0].clone());
10041        expected[1].push(raw_expected[1].clone());
10042        expected[2].push(raw_expected[2].clone());
10043        expected[bins - 1].push(raw_expected[3].clone());
10044        assert_scan(result, vec![expected], bins, 0, bins);
10045
10046        let bins = 256;
10047        let accounts_db = AccountsDb::new_single_for_tests();
10048        let result = accounts_db
10049            .scan_snapshot_stores(
10050                &get_storage_refs(&storages),
10051                &mut stats,
10052                bins,
10053                &Range {
10054                    start: 0,
10055                    end: bins,
10056                },
10057                false,
10058            )
10059            .unwrap();
10060        let mut expected = vec![Vec::new(); bins];
10061        expected[0].push(raw_expected[0].clone());
10062        expected[127].push(raw_expected[1].clone());
10063        expected[128].push(raw_expected[2].clone());
10064        expected[bins - 1].push(raw_expected.last().unwrap().clone());
10065        assert_scan(result, vec![expected], bins, 0, bins);
10066    }
10067
10068    #[test]
10069    fn test_accountsdb_scan_snapshot_stores_2nd_chunk() {
10070        let accounts_db = AccountsDb::new_single_for_tests();
10071        // enough stores to get to 2nd chunk
10072        let bins = 1;
10073        let slot = MAX_ITEMS_PER_CHUNK as Slot;
10074        let (storages, raw_expected) = sample_storages_and_account_in_slot(slot, &accounts_db);
10075        let storage_data = vec![(&storages[0], slot)];
10076
10077        let sorted_storages =
10078            SortedStorages::new_debug(&storage_data[..], 0, MAX_ITEMS_PER_CHUNK as usize + 1);
10079
10080        let mut stats = HashStats::default();
10081        let result = accounts_db
10082            .scan_snapshot_stores(
10083                &sorted_storages,
10084                &mut stats,
10085                bins,
10086                &Range {
10087                    start: 0,
10088                    end: bins,
10089                },
10090                false,
10091            )
10092            .unwrap();
10093
10094        assert_scan(result, vec![vec![raw_expected]], bins, 0, bins);
10095    }
10096
10097    #[test]
10098    fn test_accountsdb_scan_snapshot_stores_binning() {
10099        let mut stats = HashStats::default();
10100        let accounts_db = AccountsDb::new_single_for_tests();
10101        let (storages, raw_expected) = sample_storages_and_accounts(&accounts_db);
10102
10103        // just the first bin of 2
10104        let bins = 2;
10105        let half_bins = bins / 2;
10106        let result = accounts_db
10107            .scan_snapshot_stores(
10108                &get_storage_refs(&storages),
10109                &mut stats,
10110                bins,
10111                &Range {
10112                    start: 0,
10113                    end: half_bins,
10114                },
10115                false,
10116            )
10117            .unwrap();
10118        let mut expected = vec![Vec::new(); half_bins];
10119        expected[0].push(raw_expected[0].clone());
10120        expected[0].push(raw_expected[1].clone());
10121        assert_scan(result, vec![expected], bins, 0, half_bins);
10122
10123        // just the second bin of 2
10124        let accounts_db = AccountsDb::new_single_for_tests();
10125        let result = accounts_db
10126            .scan_snapshot_stores(
10127                &get_storage_refs(&storages),
10128                &mut stats,
10129                bins,
10130                &Range {
10131                    start: 1,
10132                    end: bins,
10133                },
10134                false,
10135            )
10136            .unwrap();
10137
10138        let mut expected = vec![Vec::new(); half_bins];
10139        let starting_bin_index = 0;
10140        expected[starting_bin_index].push(raw_expected[2].clone());
10141        expected[starting_bin_index].push(raw_expected[3].clone());
10142        assert_scan(result, vec![expected], bins, 1, bins - 1);
10143
10144        // 1 bin at a time of 4
10145        let bins = 4;
10146        let accounts_db = AccountsDb::new_single_for_tests();
10147
10148        for (bin, expected_item) in raw_expected.iter().enumerate().take(bins) {
10149            let result = accounts_db
10150                .scan_snapshot_stores(
10151                    &get_storage_refs(&storages),
10152                    &mut stats,
10153                    bins,
10154                    &Range {
10155                        start: bin,
10156                        end: bin + 1,
10157                    },
10158                    false,
10159                )
10160                .unwrap();
10161            let mut expected = vec![Vec::new(); 1];
10162            expected[0].push(expected_item.clone());
10163            assert_scan(result, vec![expected], bins, bin, 1);
10164        }
10165
10166        let bins = 256;
10167        let bin_locations = vec![0, 127, 128, 255];
10168        let range = 1;
10169        for bin in 0..bins {
10170            let accounts_db = AccountsDb::new_single_for_tests();
10171            let result = accounts_db
10172                .scan_snapshot_stores(
10173                    &get_storage_refs(&storages),
10174                    &mut stats,
10175                    bins,
10176                    &Range {
10177                        start: bin,
10178                        end: bin + range,
10179                    },
10180                    false,
10181                )
10182                .unwrap();
10183            let mut expected = vec![];
10184            if let Some(index) = bin_locations.iter().position(|&r| r == bin) {
10185                expected = vec![Vec::new(); range];
10186                expected[0].push(raw_expected[index].clone());
10187            }
10188            let mut result2 = (0..range).map(|_| Vec::default()).collect::<Vec<_>>();
10189            if let Some(m) = result.get(0) {
10190                m.load_all(
10191                    &mut result2,
10192                    bin,
10193                    &PubkeyBinCalculator24::new(bins),
10194                    &mut CacheHashDataStats::default(),
10195                );
10196            } else {
10197                result2 = vec![];
10198            }
10199
10200            assert_eq!(result2, expected);
10201        }
10202    }
10203
10204    #[test]
10205    fn test_accountsdb_scan_snapshot_stores_binning_2nd_chunk() {
10206        let accounts_db = AccountsDb::new_single_for_tests();
10207        // enough stores to get to 2nd chunk
10208        // range is for only 1 bin out of 256.
10209        let bins = 256;
10210        let slot = MAX_ITEMS_PER_CHUNK as Slot;
10211        let (storages, raw_expected) = sample_storages_and_account_in_slot(slot, &accounts_db);
10212        let storage_data = vec![(&storages[0], slot)];
10213
10214        let sorted_storages =
10215            SortedStorages::new_debug(&storage_data[..], 0, MAX_ITEMS_PER_CHUNK as usize + 1);
10216
10217        let mut stats = HashStats::default();
10218        let range = 1;
10219        let start = 127;
10220        let result = accounts_db
10221            .scan_snapshot_stores(
10222                &sorted_storages,
10223                &mut stats,
10224                bins,
10225                &Range {
10226                    start,
10227                    end: start + range,
10228                },
10229                false,
10230            )
10231            .unwrap();
10232        assert_eq!(result.len(), 1); // 2 chunks, but 1 is empty so not included
10233        let mut expected = vec![Vec::new(); range];
10234        expected[0].push(raw_expected[1].clone());
10235        let mut result2 = (0..range).map(|_| Vec::default()).collect::<Vec<_>>();
10236        result[0].load_all(
10237            &mut result2,
10238            0,
10239            &PubkeyBinCalculator24::new(range),
10240            &mut CacheHashDataStats::default(),
10241        );
10242        assert_eq!(result2.len(), 1);
10243        assert_eq!(result2, expected);
10244    }
10245
10246    #[test]
10247    fn test_accountsdb_calculate_accounts_hash_from_storages_simple() {
10248        solana_logger::setup();
10249
10250        let (storages, _size, _slot_expected) = sample_storage();
10251        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10252        let result = db
10253            .calculate_accounts_hash_from_storages(
10254                &CalcAccountsHashConfig::default(),
10255                &get_storage_refs(&storages),
10256                HashStats::default(),
10257            )
10258            .unwrap();
10259        let expected_hash = Hash::from_str("GKot5hBsd81kMupNCXHaqbhv3huEbxAFMLnpcX2hniwn").unwrap();
10260        let expected_accounts_hash = AccountsHash(expected_hash);
10261        assert_eq!(result, (expected_accounts_hash, 0));
10262    }
10263
10264    #[test]
10265    fn test_accountsdb_calculate_accounts_hash_from_storages() {
10266        solana_logger::setup();
10267
10268        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10269        let (storages, raw_expected) = sample_storages_and_accounts(&db);
10270        let expected_hash =
10271            AccountsHasher::compute_merkle_root_loop(raw_expected.clone(), MERKLE_FANOUT, |item| {
10272                &item.hash
10273            });
10274        let sum = raw_expected.iter().map(|item| item.lamports).sum();
10275        let result = db
10276            .calculate_accounts_hash_from_storages(
10277                &CalcAccountsHashConfig::default(),
10278                &get_storage_refs(&storages),
10279                HashStats::default(),
10280            )
10281            .unwrap();
10282
10283        let expected_accounts_hash = AccountsHash(expected_hash);
10284        assert_eq!(result, (expected_accounts_hash, sum));
10285    }
10286
10287    fn sample_storage() -> (Vec<Arc<AccountStorageEntry>>, usize, Slot) {
10288        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10289        let slot_expected: Slot = 0;
10290        let size: usize = 123;
10291        let data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
10292
10293        let arc = Arc::new(data);
10294        let storages = vec![arc];
10295        (storages, size, slot_expected)
10296    }
10297
10298    #[derive(Clone)]
10299    struct TestScan {
10300        calls: Arc<AtomicU64>,
10301        pubkey: Pubkey,
10302        slot_expected: Slot,
10303        accum: BinnedHashData,
10304        current_slot: Slot,
10305        value_to_use_for_lamports: u64,
10306    }
10307
10308    impl AppendVecScan for TestScan {
10309        fn filter(&mut self, _pubkey: &Pubkey) -> bool {
10310            true
10311        }
10312        fn set_slot(&mut self, slot: Slot) {
10313            self.current_slot = slot;
10314        }
10315        fn init_accum(&mut self, _count: usize) {}
10316        fn get_accum(&mut self) -> BinnedHashData {
10317            std::mem::take(&mut self.accum)
10318        }
10319        fn set_accum(&mut self, accum: BinnedHashData) {
10320            self.accum = accum;
10321        }
10322        fn found_account(&mut self, loaded_account: &LoadedAccount) {
10323            self.calls.fetch_add(1, Ordering::Relaxed);
10324            assert_eq!(loaded_account.pubkey(), &self.pubkey);
10325            assert_eq!(self.slot_expected, self.current_slot);
10326            self.accum.push(vec![CalculateHashIntermediate::new(
10327                Hash::default(),
10328                self.value_to_use_for_lamports,
10329                self.pubkey,
10330            )]);
10331        }
10332        fn scanning_complete(self) -> BinnedHashData {
10333            self.accum
10334        }
10335    }
10336
10337    #[test]
10338    fn test_accountsdb_scan_account_storage_no_bank() {
10339        solana_logger::setup();
10340
10341        let expected = 1;
10342        let tf = crate::append_vec::test_utils::get_append_vec_path(
10343            "test_accountsdb_scan_account_storage_no_bank",
10344        );
10345        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10346        let slot_expected: Slot = 0;
10347        let size: usize = 123;
10348        let mut data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
10349        let av = AccountsFile::AppendVec(AppendVec::new(&tf.path, true, 1024 * 1024));
10350        data.accounts = av;
10351
10352        let storage = Arc::new(data);
10353        let pubkey = solana_sdk::pubkey::new_rand();
10354        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
10355        let mark_alive = false;
10356        append_single_account_with_default_hash(&storage, &pubkey, &acc, 1, mark_alive, None);
10357
10358        let calls = Arc::new(AtomicU64::new(0));
10359        let temp_dir = TempDir::new().unwrap();
10360        let accounts_hash_cache_path = temp_dir.path().to_path_buf();
10361        let accounts_db = AccountsDb::new_single_for_tests();
10362
10363        let test_scan = TestScan {
10364            calls: calls.clone(),
10365            pubkey,
10366            slot_expected,
10367            accum: Vec::default(),
10368            current_slot: 0,
10369            value_to_use_for_lamports: expected,
10370        };
10371
10372        let result = accounts_db.scan_account_storage_no_bank(
10373            &CacheHashData::new(accounts_hash_cache_path),
10374            &CalcAccountsHashConfig::default(),
10375            &get_storage_refs(&[storage]),
10376            test_scan,
10377            &Range { start: 0, end: 1 },
10378            &HashStats::default(),
10379        );
10380        assert_eq!(calls.load(Ordering::Relaxed), 1);
10381        assert_scan(
10382            result,
10383            vec![vec![vec![CalculateHashIntermediate::new(
10384                Hash::default(),
10385                expected,
10386                pubkey,
10387            )]]],
10388            1,
10389            0,
10390            1,
10391        );
10392    }
10393
10394    fn convert_to_slice(
10395        input: &[Vec<Vec<CalculateHashIntermediate>>],
10396    ) -> Vec<Vec<&[CalculateHashIntermediate]>> {
10397        input
10398            .iter()
10399            .map(|v| v.iter().map(|v| &v[..]).collect::<Vec<_>>())
10400            .collect::<Vec<_>>()
10401    }
10402
10403    pub(crate) fn append_single_account_with_default_hash(
10404        storage: &AccountStorageEntry,
10405        pubkey: &Pubkey,
10406        account: &AccountSharedData,
10407        write_version: StoredMetaWriteVersion,
10408        mark_alive: bool,
10409        add_to_index: Option<&AccountsIndex<AccountInfo>>,
10410    ) {
10411        let slot = storage.slot();
10412        let accounts = [(pubkey, account)];
10413        let slice = &accounts[..];
10414        let account_data = (slot, slice);
10415        let hash = Hash::default();
10416        let storable_accounts =
10417            StorableAccountsWithHashesAndWriteVersions::new_with_hashes_and_write_versions(
10418                &account_data,
10419                vec![&hash],
10420                vec![write_version],
10421            );
10422        let old_written = storage.written_bytes();
10423        let offsets = storage
10424            .accounts
10425            .append_accounts(&storable_accounts, 0)
10426            .unwrap();
10427        if mark_alive {
10428            // updates 'alive_bytes' on the storage
10429            storage.add_account((storage.written_bytes() - old_written) as usize);
10430        }
10431
10432        if let Some(index) = add_to_index {
10433            let account_info = AccountInfo::new(
10434                StorageLocation::AppendVec(storage.append_vec_id(), offsets[0]),
10435                (offsets[1] - offsets[0]).try_into().unwrap(),
10436                account.lamports(),
10437            );
10438            index.upsert(
10439                slot,
10440                slot,
10441                pubkey,
10442                account,
10443                &AccountSecondaryIndexes::default(),
10444                account_info,
10445                &mut Vec::default(),
10446                UpsertReclaim::IgnoreReclaims,
10447            );
10448        }
10449    }
10450
10451    #[test]
10452    fn test_accountsdb_scan_account_storage_no_bank_one_slot() {
10453        solana_logger::setup();
10454
10455        let expected = 1;
10456        let tf = crate::append_vec::test_utils::get_append_vec_path(
10457            "test_accountsdb_scan_account_storage_no_bank",
10458        );
10459        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10460        let slot_expected: Slot = 0;
10461        let size: usize = 123;
10462        let mut data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
10463        let av = AccountsFile::AppendVec(AppendVec::new(&tf.path, true, 1024 * 1024));
10464        data.accounts = av;
10465
10466        let storage = Arc::new(data);
10467        let pubkey = solana_sdk::pubkey::new_rand();
10468        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
10469        let mark_alive = false;
10470        append_single_account_with_default_hash(&storage, &pubkey, &acc, 1, mark_alive, None);
10471
10472        let calls = Arc::new(AtomicU64::new(0));
10473
10474        let mut test_scan = TestScan {
10475            calls: calls.clone(),
10476            pubkey,
10477            slot_expected,
10478            accum: Vec::default(),
10479            current_slot: 0,
10480            value_to_use_for_lamports: expected,
10481        };
10482
10483        AccountsDb::scan_single_account_storage(&storage, &mut test_scan);
10484        let accum = test_scan.scanning_complete();
10485        assert_eq!(calls.load(Ordering::Relaxed), 1);
10486        assert_eq!(
10487            accum
10488                .iter()
10489                .flatten()
10490                .map(|a| a.lamports)
10491                .collect::<Vec<_>>(),
10492            vec![expected]
10493        );
10494    }
10495
10496    fn append_sample_data_to_storage(
10497        storage: &Arc<AccountStorageEntry>,
10498        pubkey: &Pubkey,
10499        write_version: StoredMetaWriteVersion,
10500        mark_alive: bool,
10501        account_data_size: Option<u64>,
10502    ) {
10503        let acc = AccountSharedData::new(
10504            1,
10505            account_data_size.unwrap_or(48) as usize,
10506            AccountSharedData::default().owner(),
10507        );
10508        append_single_account_with_default_hash(
10509            storage,
10510            pubkey,
10511            &acc,
10512            write_version,
10513            mark_alive,
10514            None,
10515        );
10516    }
10517
10518    fn sample_storage_with_entries(
10519        tf: &TempFile,
10520        write_version: StoredMetaWriteVersion,
10521        slot: Slot,
10522        pubkey: &Pubkey,
10523        mark_alive: bool,
10524    ) -> Arc<AccountStorageEntry> {
10525        sample_storage_with_entries_id(tf, write_version, slot, pubkey, 0, mark_alive, None)
10526    }
10527
10528    fn sample_storage_with_entries_id(
10529        tf: &TempFile,
10530        write_version: StoredMetaWriteVersion,
10531        slot: Slot,
10532        pubkey: &Pubkey,
10533        id: AppendVecId,
10534        mark_alive: bool,
10535        account_data_size: Option<u64>,
10536    ) -> Arc<AccountStorageEntry> {
10537        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10538        let size: usize = aligned_stored_size(account_data_size.unwrap_or(123) as usize);
10539        let mut data = AccountStorageEntry::new(&paths[0], slot, id, size as u64);
10540        let av = AccountsFile::AppendVec(AppendVec::new(&tf.path, true, (1024 * 1024).max(size)));
10541        data.accounts = av;
10542
10543        let arc = Arc::new(data);
10544        append_sample_data_to_storage(&arc, pubkey, write_version, mark_alive, account_data_size);
10545        arc
10546    }
10547
10548    #[test]
10549    fn test_accountsdb_scan_multiple_account_storage_no_bank_one_slot() {
10550        solana_logger::setup();
10551
10552        let slot_expected: Slot = 0;
10553        let tf = crate::append_vec::test_utils::get_append_vec_path(
10554            "test_accountsdb_scan_account_storage_no_bank",
10555        );
10556        let write_version1 = 0;
10557        let pubkey1 = solana_sdk::pubkey::new_rand();
10558        let pubkey2 = solana_sdk::pubkey::new_rand();
10559        let mark_alive = false;
10560        let storage =
10561            sample_storage_with_entries(&tf, write_version1, slot_expected, &pubkey1, mark_alive);
10562        let lamports = storage.accounts.account_iter().next().unwrap().lamports();
10563        let calls = Arc::new(AtomicU64::new(0));
10564        let mut scanner = TestScanSimple {
10565            current_slot: 0,
10566            slot_expected,
10567            pubkey1,
10568            pubkey2,
10569            accum: Vec::default(),
10570            calls: calls.clone(),
10571        };
10572        AccountsDb::scan_single_account_storage(&storage, &mut scanner);
10573        let accum = scanner.scanning_complete();
10574        assert_eq!(calls.load(Ordering::Relaxed), 1);
10575        assert_eq!(
10576            accum
10577                .iter()
10578                .flatten()
10579                .map(|a| a.lamports)
10580                .collect::<Vec<_>>(),
10581            vec![lamports]
10582        );
10583    }
10584
10585    #[derive(Clone)]
10586    struct TestScanSimple {
10587        current_slot: Slot,
10588        slot_expected: Slot,
10589        calls: Arc<AtomicU64>,
10590        accum: BinnedHashData,
10591        pubkey1: Pubkey,
10592        pubkey2: Pubkey,
10593    }
10594
10595    impl AppendVecScan for TestScanSimple {
10596        fn set_slot(&mut self, slot: Slot) {
10597            self.current_slot = slot;
10598        }
10599        fn filter(&mut self, _pubkey: &Pubkey) -> bool {
10600            true
10601        }
10602        fn init_accum(&mut self, _count: usize) {}
10603        fn found_account(&mut self, loaded_account: &LoadedAccount) {
10604            self.calls.fetch_add(1, Ordering::Relaxed);
10605            let first = loaded_account.pubkey() == &self.pubkey1;
10606            assert!(first || loaded_account.pubkey() == &self.pubkey2);
10607            assert_eq!(self.slot_expected, self.current_slot);
10608            if first {
10609                assert!(self.accum.is_empty());
10610            } else {
10611                assert_eq!(self.accum.len(), 1);
10612            }
10613            self.accum.push(vec![CalculateHashIntermediate {
10614                hash: Hash::default(),
10615                lamports: loaded_account.lamports(),
10616                pubkey: Pubkey::default(),
10617            }]);
10618        }
10619        fn scanning_complete(self) -> BinnedHashData {
10620            self.accum
10621        }
10622        fn get_accum(&mut self) -> BinnedHashData {
10623            std::mem::take(&mut self.accum)
10624        }
10625        fn set_accum(&mut self, accum: BinnedHashData) {
10626            self.accum = accum;
10627        }
10628    }
10629
10630    #[test]
10631    fn test_accountsdb_add_root() {
10632        solana_logger::setup();
10633        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10634        let key = Pubkey::default();
10635        let account0 = AccountSharedData::new(1, 0, &key);
10636
10637        db.store_for_tests(0, &[(&key, &account0)]);
10638        db.add_root(0);
10639        let ancestors = vec![(1, 1)].into_iter().collect();
10640        assert_eq!(
10641            db.load_without_fixed_root(&ancestors, &key),
10642            Some((account0, 0))
10643        );
10644    }
10645
10646    #[test]
10647    fn test_accountsdb_latest_ancestor() {
10648        solana_logger::setup();
10649        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10650        let key = Pubkey::default();
10651        let account0 = AccountSharedData::new(1, 0, &key);
10652
10653        db.store_for_tests(0, &[(&key, &account0)]);
10654
10655        let account1 = AccountSharedData::new(0, 0, &key);
10656        db.store_for_tests(1, &[(&key, &account1)]);
10657
10658        let ancestors = vec![(1, 1)].into_iter().collect();
10659        assert_eq!(
10660            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10661            &account1
10662        );
10663
10664        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
10665        assert_eq!(
10666            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10667            &account1
10668        );
10669
10670        let mut accounts = Vec::new();
10671        db.unchecked_scan_accounts(
10672            "",
10673            &ancestors,
10674            |_, account, _| {
10675                accounts.push(account.take_account());
10676            },
10677            &ScanConfig::default(),
10678        );
10679        assert_eq!(accounts, vec![account1]);
10680    }
10681
10682    #[test]
10683    fn test_accountsdb_latest_ancestor_with_root() {
10684        solana_logger::setup();
10685        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10686        let key = Pubkey::default();
10687        let account0 = AccountSharedData::new(1, 0, &key);
10688
10689        db.store_for_tests(0, &[(&key, &account0)]);
10690
10691        let account1 = AccountSharedData::new(0, 0, &key);
10692        db.store_for_tests(1, &[(&key, &account1)]);
10693        db.add_root(0);
10694
10695        let ancestors = vec![(1, 1)].into_iter().collect();
10696        assert_eq!(
10697            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10698            &account1
10699        );
10700
10701        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
10702        assert_eq!(
10703            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10704            &account1
10705        );
10706    }
10707
10708    #[test]
10709    fn test_accountsdb_root_one_slot() {
10710        solana_logger::setup();
10711        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10712
10713        let key = Pubkey::default();
10714        let account0 = AccountSharedData::new(1, 0, &key);
10715
10716        // store value 1 in the "root", i.e. db zero
10717        db.store_for_tests(0, &[(&key, &account0)]);
10718
10719        // now we have:
10720        //
10721        //                       root0 -> key.lamports==1
10722        //                        / \
10723        //                       /   \
10724        //  key.lamports==0 <- slot1    \
10725        //                             slot2 -> key.lamports==1
10726        //                                       (via root0)
10727
10728        // store value 0 in one child
10729        let account1 = AccountSharedData::new(0, 0, &key);
10730        db.store_for_tests(1, &[(&key, &account1)]);
10731
10732        // masking accounts is done at the Accounts level, at accountsDB we see
10733        // original account (but could also accept "None", which is implemented
10734        // at the Accounts level)
10735        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
10736        assert_eq!(
10737            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10738            &account1
10739        );
10740
10741        // we should see 1 token in slot 2
10742        let ancestors = vec![(0, 0), (2, 2)].into_iter().collect();
10743        assert_eq!(
10744            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10745            &account0
10746        );
10747
10748        db.add_root(0);
10749
10750        let ancestors = vec![(1, 1)].into_iter().collect();
10751        assert_eq!(
10752            db.load_without_fixed_root(&ancestors, &key),
10753            Some((account1, 1))
10754        );
10755        let ancestors = vec![(2, 2)].into_iter().collect();
10756        assert_eq!(
10757            db.load_without_fixed_root(&ancestors, &key),
10758            Some((account0, 0))
10759        ); // original value
10760    }
10761
10762    #[test]
10763    fn test_accountsdb_add_root_many() {
10764        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10765
10766        let mut pubkeys: Vec<Pubkey> = vec![];
10767        create_account(&db, &mut pubkeys, 0, 100, 0, 0);
10768        for _ in 1..100 {
10769            let idx = thread_rng().gen_range(0, 99);
10770            let ancestors = vec![(0, 0)].into_iter().collect();
10771            let account = db
10772                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10773                .unwrap();
10774            let default_account = AccountSharedData::from(Account {
10775                lamports: (idx + 1) as u64,
10776                ..Account::default()
10777            });
10778            assert_eq!((default_account, 0), account);
10779        }
10780
10781        db.add_root(0);
10782
10783        // check that all the accounts appear with a new root
10784        for _ in 1..100 {
10785            let idx = thread_rng().gen_range(0, 99);
10786            let ancestors = vec![(0, 0)].into_iter().collect();
10787            let account0 = db
10788                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10789                .unwrap();
10790            let ancestors = vec![(1, 1)].into_iter().collect();
10791            let account1 = db
10792                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10793                .unwrap();
10794            let default_account = AccountSharedData::from(Account {
10795                lamports: (idx + 1) as u64,
10796                ..Account::default()
10797            });
10798            assert_eq!(&default_account, &account0.0);
10799            assert_eq!(&default_account, &account1.0);
10800        }
10801    }
10802
10803    #[test]
10804    fn test_accountsdb_count_stores() {
10805        solana_logger::setup();
10806        let db = AccountsDb::new_single_for_tests();
10807
10808        let mut pubkeys: Vec<Pubkey> = vec![];
10809        create_account(&db, &mut pubkeys, 0, 2, DEFAULT_FILE_SIZE as usize / 3, 0);
10810        db.add_root_and_flush_write_cache(0);
10811        check_storage(&db, 0, 2);
10812
10813        let pubkey = solana_sdk::pubkey::new_rand();
10814        let account = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 3, &pubkey);
10815        db.store_for_tests(1, &[(&pubkey, &account)]);
10816        db.store_for_tests(1, &[(&pubkeys[0], &account)]);
10817        // adding root doesn't change anything
10818        db.calculate_accounts_delta_hash(1);
10819        db.add_root_and_flush_write_cache(1);
10820        {
10821            let slot_0_store = &db.storage.get_slot_storage_entry(0).unwrap();
10822            let slot_1_store = &db.storage.get_slot_storage_entry(1).unwrap();
10823            assert_eq!(slot_0_store.count(), 2);
10824            assert_eq!(slot_1_store.count(), 2);
10825            assert_eq!(slot_0_store.approx_stored_count(), 2);
10826            assert_eq!(slot_1_store.approx_stored_count(), 2);
10827        }
10828
10829        // overwrite old rooted account version; only the r_slot_0_stores.count() should be
10830        // decremented
10831        // slot 2 is not a root and should be ignored by clean
10832        db.store_for_tests(2, &[(&pubkeys[0], &account)]);
10833        db.clean_accounts_for_tests();
10834        {
10835            let slot_0_store = &db.storage.get_slot_storage_entry(0).unwrap();
10836            let slot_1_store = &db.storage.get_slot_storage_entry(1).unwrap();
10837            assert_eq!(slot_0_store.count(), 1);
10838            assert_eq!(slot_1_store.count(), 2);
10839            assert_eq!(slot_0_store.approx_stored_count(), 2);
10840            assert_eq!(slot_1_store.approx_stored_count(), 2);
10841        }
10842    }
10843
10844    #[test]
10845    fn test_accounts_unsquashed() {
10846        let key = Pubkey::default();
10847
10848        // 1 token in the "root", i.e. db zero
10849        let db0 = AccountsDb::new(Vec::new(), &ClusterType::Development);
10850        let account0 = AccountSharedData::new(1, 0, &key);
10851        db0.store_for_tests(0, &[(&key, &account0)]);
10852
10853        // 0 lamports in the child
10854        let account1 = AccountSharedData::new(0, 0, &key);
10855        db0.store_for_tests(1, &[(&key, &account1)]);
10856
10857        // masking accounts is done at the Accounts level, at accountsDB we see
10858        // original account
10859        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
10860        assert_eq!(
10861            db0.load_without_fixed_root(&ancestors, &key),
10862            Some((account1, 1))
10863        );
10864        let ancestors = vec![(0, 0)].into_iter().collect();
10865        assert_eq!(
10866            db0.load_without_fixed_root(&ancestors, &key),
10867            Some((account0, 0))
10868        );
10869    }
10870
10871    fn run_test_remove_unrooted_slot(is_cached: bool) {
10872        let unrooted_slot = 9;
10873        let unrooted_bank_id = 9;
10874        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10875        let key = Pubkey::default();
10876        let account0 = AccountSharedData::new(1, 0, &key);
10877        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
10878        if is_cached {
10879            db.store_cached((unrooted_slot, &[(&key, &account0)][..]), None);
10880        } else {
10881            db.store_for_tests(unrooted_slot, &[(&key, &account0)]);
10882        }
10883        assert!(db.get_bank_hash_stats(unrooted_slot).is_some());
10884        assert!(db
10885            .accounts_index
10886            .get(&key, Some(&ancestors), None)
10887            .is_some());
10888        assert_load_account(&db, unrooted_slot, key, 1);
10889
10890        // Purge the slot
10891        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
10892        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
10893        assert!(db.get_bank_hash_stats(unrooted_slot).is_none());
10894        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_none());
10895        assert!(db.storage.get_slot_storage_entry(unrooted_slot).is_none());
10896        assert!(db.accounts_index.get_account_read_entry(&key).is_none());
10897        assert!(db
10898            .accounts_index
10899            .get(&key, Some(&ancestors), None)
10900            .is_none());
10901
10902        // Test we can store for the same slot again and get the right information
10903        let account0 = AccountSharedData::new(2, 0, &key);
10904        db.store_for_tests(unrooted_slot, &[(&key, &account0)]);
10905        assert_load_account(&db, unrooted_slot, key, 2);
10906    }
10907
10908    #[test]
10909    fn test_remove_unrooted_slot_cached() {
10910        run_test_remove_unrooted_slot(true);
10911    }
10912
10913    #[test]
10914    fn test_remove_unrooted_slot_storage() {
10915        run_test_remove_unrooted_slot(false);
10916    }
10917
10918    #[test]
10919    fn test_remove_unrooted_slot_snapshot() {
10920        solana_logger::setup();
10921        let unrooted_slot = 9;
10922        let unrooted_bank_id = 9;
10923        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10924        let key = solana_sdk::pubkey::new_rand();
10925        let account0 = AccountSharedData::new(1, 0, &key);
10926        db.store_for_tests(unrooted_slot, &[(&key, &account0)]);
10927
10928        // Purge the slot
10929        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
10930
10931        // Add a new root
10932        let key2 = solana_sdk::pubkey::new_rand();
10933        let new_root = unrooted_slot + 1;
10934        db.store_for_tests(new_root, &[(&key2, &account0)]);
10935        db.add_root_and_flush_write_cache(new_root);
10936
10937        db.calculate_accounts_delta_hash(new_root);
10938        db.update_accounts_hash_for_tests(new_root, &linear_ancestors(new_root), false, false);
10939
10940        // Simulate reconstruction from snapshot
10941        let db = reconstruct_accounts_db_via_serialization(&db, new_root);
10942
10943        // Check root account exists
10944        assert_load_account(&db, new_root, key2, 1);
10945
10946        // Check purged account stays gone
10947        let unrooted_slot_ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
10948        assert!(db
10949            .load_without_fixed_root(&unrooted_slot_ancestors, &key)
10950            .is_none());
10951    }
10952
10953    fn create_account(
10954        accounts: &AccountsDb,
10955        pubkeys: &mut Vec<Pubkey>,
10956        slot: Slot,
10957        num: usize,
10958        space: usize,
10959        num_vote: usize,
10960    ) {
10961        let ancestors = vec![(slot, 0)].into_iter().collect();
10962        for t in 0..num {
10963            let pubkey = solana_sdk::pubkey::new_rand();
10964            let account =
10965                AccountSharedData::new((t + 1) as u64, space, AccountSharedData::default().owner());
10966            pubkeys.push(pubkey);
10967            assert!(accounts
10968                .load_without_fixed_root(&ancestors, &pubkey)
10969                .is_none());
10970            accounts.store_for_tests(slot, &[(&pubkey, &account)]);
10971        }
10972        for t in 0..num_vote {
10973            let pubkey = solana_sdk::pubkey::new_rand();
10974            let account =
10975                AccountSharedData::new((num + t + 1) as u64, space, &solana_vote_program::id());
10976            pubkeys.push(pubkey);
10977            let ancestors = vec![(slot, 0)].into_iter().collect();
10978            assert!(accounts
10979                .load_without_fixed_root(&ancestors, &pubkey)
10980                .is_none());
10981            accounts.store_for_tests(slot, &[(&pubkey, &account)]);
10982        }
10983    }
10984
10985    fn update_accounts(accounts: &AccountsDb, pubkeys: &[Pubkey], slot: Slot, range: usize) {
10986        for _ in 1..1000 {
10987            let idx = thread_rng().gen_range(0, range);
10988            let ancestors = vec![(slot, 0)].into_iter().collect();
10989            if let Some((mut account, _)) =
10990                accounts.load_without_fixed_root(&ancestors, &pubkeys[idx])
10991            {
10992                account.checked_add_lamports(1).unwrap();
10993                accounts.store_for_tests(slot, &[(&pubkeys[idx], &account)]);
10994                if account.is_zero_lamport() {
10995                    let ancestors = vec![(slot, 0)].into_iter().collect();
10996                    assert!(accounts
10997                        .load_without_fixed_root(&ancestors, &pubkeys[idx])
10998                        .is_none());
10999                } else {
11000                    let default_account = AccountSharedData::from(Account {
11001                        lamports: account.lamports(),
11002                        ..Account::default()
11003                    });
11004                    assert_eq!(default_account, account);
11005                }
11006            }
11007        }
11008    }
11009
11010    fn check_storage(accounts: &AccountsDb, slot: Slot, count: usize) {
11011        assert!(accounts.storage.get_slot_storage_entry(slot).is_some());
11012        let store = accounts.storage.get_slot_storage_entry(slot).unwrap();
11013        let total_count = store.count();
11014        assert_eq!(store.status(), AccountStorageStatus::Available);
11015        assert_eq!(total_count, count);
11016        let (expected_store_count, actual_store_count): (usize, usize) =
11017            (store.approx_stored_count(), store.all_accounts().len());
11018        assert_eq!(expected_store_count, actual_store_count);
11019    }
11020
11021    fn check_accounts(
11022        accounts: &AccountsDb,
11023        pubkeys: &[Pubkey],
11024        slot: Slot,
11025        num: usize,
11026        count: usize,
11027    ) {
11028        let ancestors = vec![(slot, 0)].into_iter().collect();
11029        for _ in 0..num {
11030            let idx = thread_rng().gen_range(0, num);
11031            let account = accounts.load_without_fixed_root(&ancestors, &pubkeys[idx]);
11032            let account1 = Some((
11033                AccountSharedData::new(
11034                    (idx + count) as u64,
11035                    0,
11036                    AccountSharedData::default().owner(),
11037                ),
11038                slot,
11039            ));
11040            assert_eq!(account, account1);
11041        }
11042    }
11043
11044    #[allow(clippy::needless_range_loop)]
11045    fn modify_accounts(
11046        accounts: &AccountsDb,
11047        pubkeys: &[Pubkey],
11048        slot: Slot,
11049        num: usize,
11050        count: usize,
11051    ) {
11052        for idx in 0..num {
11053            let account = AccountSharedData::new(
11054                (idx + count) as u64,
11055                0,
11056                AccountSharedData::default().owner(),
11057            );
11058            accounts.store_for_tests(slot, &[(&pubkeys[idx], &account)]);
11059        }
11060    }
11061
11062    #[test]
11063    fn test_account_one() {
11064        let (_accounts_dirs, paths) = get_temp_accounts_paths(1).unwrap();
11065        let db = AccountsDb::new(paths, &ClusterType::Development);
11066        let mut pubkeys: Vec<Pubkey> = vec![];
11067        create_account(&db, &mut pubkeys, 0, 1, 0, 0);
11068        let ancestors = vec![(0, 0)].into_iter().collect();
11069        let account = db.load_without_fixed_root(&ancestors, &pubkeys[0]).unwrap();
11070        let default_account = AccountSharedData::from(Account {
11071            lamports: 1,
11072            ..Account::default()
11073        });
11074        assert_eq!((default_account, 0), account);
11075    }
11076
11077    #[test]
11078    fn test_account_many() {
11079        let (_accounts_dirs, paths) = get_temp_accounts_paths(2).unwrap();
11080        let db = AccountsDb::new(paths, &ClusterType::Development);
11081        let mut pubkeys: Vec<Pubkey> = vec![];
11082        create_account(&db, &mut pubkeys, 0, 100, 0, 0);
11083        check_accounts(&db, &pubkeys, 0, 100, 1);
11084    }
11085
11086    #[test]
11087    fn test_account_update() {
11088        let accounts = AccountsDb::new_single_for_tests();
11089        let mut pubkeys: Vec<Pubkey> = vec![];
11090        create_account(&accounts, &mut pubkeys, 0, 100, 0, 0);
11091        update_accounts(&accounts, &pubkeys, 0, 99);
11092        accounts.add_root_and_flush_write_cache(0);
11093        check_storage(&accounts, 0, 100);
11094    }
11095
11096    #[test]
11097    fn test_account_grow_many() {
11098        let (_accounts_dir, paths) = get_temp_accounts_paths(2).unwrap();
11099        let size = 4096;
11100        let accounts = AccountsDb::new_sized(paths, size);
11101        let mut keys = vec![];
11102        for i in 0..9 {
11103            let key = solana_sdk::pubkey::new_rand();
11104            let account = AccountSharedData::new(i + 1, size as usize / 4, &key);
11105            accounts.store_for_tests(0, &[(&key, &account)]);
11106            keys.push(key);
11107        }
11108        let ancestors = vec![(0, 0)].into_iter().collect();
11109        for (i, key) in keys.iter().enumerate() {
11110            assert_eq!(
11111                accounts
11112                    .load_without_fixed_root(&ancestors, key)
11113                    .unwrap()
11114                    .0
11115                    .lamports(),
11116                (i as u64) + 1
11117            );
11118        }
11119
11120        let mut append_vec_histogram = HashMap::new();
11121        let mut all_slots = vec![];
11122        for slot_storage in accounts.storage.iter() {
11123            all_slots.push(slot_storage.0)
11124        }
11125        for slot in all_slots {
11126            *append_vec_histogram.entry(slot).or_insert(0) += 1;
11127        }
11128        for count in append_vec_histogram.values() {
11129            assert!(*count >= 2);
11130        }
11131    }
11132
11133    #[test]
11134    fn test_account_grow() {
11135        for pass in 0..27 {
11136            let accounts = AccountsDb::new_single_for_tests();
11137
11138            let status = [AccountStorageStatus::Available, AccountStorageStatus::Full];
11139            let pubkey1 = solana_sdk::pubkey::new_rand();
11140            let account1 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey1);
11141            accounts.store_for_tests(0, &[(&pubkey1, &account1)]);
11142            if pass == 0 {
11143                accounts.add_root_and_flush_write_cache(0);
11144                let store = &accounts.storage.get_slot_storage_entry(0).unwrap();
11145                assert_eq!(store.count(), 1);
11146                assert_eq!(store.status(), AccountStorageStatus::Available);
11147                continue;
11148            }
11149
11150            let pubkey2 = solana_sdk::pubkey::new_rand();
11151            let account2 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey2);
11152            accounts.store_for_tests(0, &[(&pubkey2, &account2)]);
11153
11154            if pass == 1 {
11155                accounts.add_root_and_flush_write_cache(0);
11156                assert_eq!(accounts.storage.len(), 1);
11157                let store = &accounts.storage.get_slot_storage_entry(0).unwrap();
11158                assert_eq!(store.count(), 2);
11159                assert_eq!(store.status(), AccountStorageStatus::Available);
11160                continue;
11161            }
11162            let ancestors = vec![(0, 0)].into_iter().collect();
11163            assert_eq!(
11164                accounts
11165                    .load_without_fixed_root(&ancestors, &pubkey1)
11166                    .unwrap()
11167                    .0,
11168                account1
11169            );
11170            assert_eq!(
11171                accounts
11172                    .load_without_fixed_root(&ancestors, &pubkey2)
11173                    .unwrap()
11174                    .0,
11175                account2
11176            );
11177
11178            // lots of writes, but they are all duplicates
11179            for i in 0..25 {
11180                accounts.store_for_tests(0, &[(&pubkey1, &account1)]);
11181                let flush = pass == i + 2;
11182                if flush {
11183                    accounts.add_root_and_flush_write_cache(0);
11184                    assert_eq!(accounts.storage.len(), 1);
11185                    let store = &accounts.storage.get_slot_storage_entry(0).unwrap();
11186                    assert_eq!(store.status(), status[0]);
11187                }
11188                let ancestors = vec![(0, 0)].into_iter().collect();
11189                assert_eq!(
11190                    accounts
11191                        .load_without_fixed_root(&ancestors, &pubkey1)
11192                        .unwrap()
11193                        .0,
11194                    account1
11195                );
11196                assert_eq!(
11197                    accounts
11198                        .load_without_fixed_root(&ancestors, &pubkey2)
11199                        .unwrap()
11200                        .0,
11201                    account2
11202                );
11203                if flush {
11204                    break;
11205                }
11206            }
11207        }
11208    }
11209
11210    #[test]
11211    fn test_lazy_gc_slot() {
11212        solana_logger::setup();
11213        //This test is pedantic
11214        //A slot is purged when a non root bank is cleaned up.  If a slot is behind root but it is
11215        //not root, it means we are retaining dead banks.
11216        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11217        let pubkey = solana_sdk::pubkey::new_rand();
11218        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11219        //store an account
11220        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11221        accounts.add_root_and_flush_write_cache(0);
11222
11223        let ancestors = vec![(0, 0)].into_iter().collect();
11224        let id = {
11225            let (lock, idx) = accounts
11226                .accounts_index
11227                .get_for_tests(&pubkey, Some(&ancestors), None)
11228                .unwrap();
11229            lock.slot_list()[idx].1.store_id()
11230        };
11231        accounts.calculate_accounts_delta_hash(0);
11232
11233        //slot is still there, since gc is lazy
11234        assert_eq!(
11235            accounts
11236                .storage
11237                .get_slot_storage_entry(0)
11238                .unwrap()
11239                .append_vec_id(),
11240            id
11241        );
11242
11243        //store causes clean
11244        accounts.store_for_tests(1, &[(&pubkey, &account)]);
11245
11246        // generate delta state for slot 1, so clean operates on it.
11247        accounts.calculate_accounts_delta_hash(1);
11248
11249        //slot is gone
11250        accounts.print_accounts_stats("pre-clean");
11251        accounts.add_root_and_flush_write_cache(1);
11252        assert!(accounts.storage.get_slot_storage_entry(0).is_some());
11253        accounts.clean_accounts_for_tests();
11254        assert!(accounts.storage.get_slot_storage_entry(0).is_none());
11255
11256        //new value is there
11257        let ancestors = vec![(1, 1)].into_iter().collect();
11258        assert_eq!(
11259            accounts.load_without_fixed_root(&ancestors, &pubkey),
11260            Some((account, 1))
11261        );
11262    }
11263
11264    impl AccountsDb {
11265        fn all_account_count_in_append_vec(&self, slot: Slot) -> usize {
11266            let store = self.storage.get_slot_storage_entry(slot);
11267            if let Some(store) = store {
11268                let count = store.all_accounts().len();
11269                let stored_count = store.approx_stored_count();
11270                assert_eq!(stored_count, count);
11271                count
11272            } else {
11273                0
11274            }
11275        }
11276
11277        pub fn ref_count_for_pubkey(&self, pubkey: &Pubkey) -> RefCount {
11278            self.accounts_index.ref_count_from_storage(pubkey)
11279        }
11280    }
11281
11282    #[test]
11283    fn test_clean_zero_lamport_and_dead_slot() {
11284        solana_logger::setup();
11285
11286        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11287        let pubkey1 = solana_sdk::pubkey::new_rand();
11288        let pubkey2 = solana_sdk::pubkey::new_rand();
11289        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
11290        let zero_lamport_account =
11291            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11292
11293        // Store two accounts
11294        accounts.store_for_tests(0, &[(&pubkey1, &account)]);
11295        accounts.store_for_tests(0, &[(&pubkey2, &account)]);
11296
11297        // Make sure both accounts are in the same AppendVec in slot 0, which
11298        // will prevent pubkey1 from being cleaned up later even when it's a
11299        // zero-lamport account
11300        let ancestors = vec![(0, 1)].into_iter().collect();
11301        let (slot1, account_info1) = accounts
11302            .accounts_index
11303            .get_for_tests(&pubkey1, Some(&ancestors), None)
11304            .map(|(account_list1, index1)| account_list1.slot_list()[index1])
11305            .unwrap();
11306        let (slot2, account_info2) = accounts
11307            .accounts_index
11308            .get_for_tests(&pubkey2, Some(&ancestors), None)
11309            .map(|(account_list2, index2)| account_list2.slot_list()[index2])
11310            .unwrap();
11311        assert_eq!(slot1, 0);
11312        assert_eq!(slot1, slot2);
11313        assert_eq!(account_info1.storage_location(), StorageLocation::Cached);
11314        assert_eq!(
11315            account_info1.storage_location(),
11316            account_info2.storage_location()
11317        );
11318
11319        // Update account 1 in slot 1
11320        accounts.store_for_tests(1, &[(&pubkey1, &account)]);
11321
11322        // Update account 1 as  zero lamports account
11323        accounts.store_for_tests(2, &[(&pubkey1, &zero_lamport_account)]);
11324
11325        // Pubkey 1 was the only account in slot 1, and it was updated in slot 2, so
11326        // slot 1 should be purged
11327        accounts.calculate_accounts_delta_hash(0);
11328        accounts.add_root_and_flush_write_cache(0);
11329        accounts.calculate_accounts_delta_hash(1);
11330        accounts.add_root_and_flush_write_cache(1);
11331        accounts.calculate_accounts_delta_hash(2);
11332        accounts.add_root_and_flush_write_cache(2);
11333
11334        // Slot 1 should be removed, slot 0 cannot be removed because it still has
11335        // the latest update for pubkey 2
11336        accounts.clean_accounts_for_tests();
11337        assert!(accounts.storage.get_slot_storage_entry(0).is_some());
11338        assert!(accounts.storage.get_slot_storage_entry(1).is_none());
11339
11340        // Slot 1 should be cleaned because all it's accounts are
11341        // zero lamports, and are not present in any other slot's
11342        // storage entries
11343        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11344    }
11345
11346    #[test]
11347    fn test_clean_multiple_zero_lamport_decrements_index_ref_count() {
11348        solana_logger::setup();
11349
11350        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11351        let pubkey1 = solana_sdk::pubkey::new_rand();
11352        let pubkey2 = solana_sdk::pubkey::new_rand();
11353        let zero_lamport_account =
11354            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11355
11356        // Store 2 accounts in slot 0, then update account 1 in two more slots
11357        accounts.store_for_tests(0, &[(&pubkey1, &zero_lamport_account)]);
11358        accounts.store_for_tests(0, &[(&pubkey2, &zero_lamport_account)]);
11359        accounts.store_for_tests(1, &[(&pubkey1, &zero_lamport_account)]);
11360        accounts.store_for_tests(2, &[(&pubkey1, &zero_lamport_account)]);
11361        // Root all slots
11362        accounts.calculate_accounts_delta_hash(0);
11363        accounts.add_root_and_flush_write_cache(0);
11364        accounts.calculate_accounts_delta_hash(1);
11365        accounts.add_root_and_flush_write_cache(1);
11366        accounts.calculate_accounts_delta_hash(2);
11367        accounts.add_root_and_flush_write_cache(2);
11368
11369        // Account ref counts should match how many slots they were stored in
11370        // Account 1 = 3 slots; account 2 = 1 slot
11371        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 3);
11372        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 1);
11373
11374        accounts.clean_accounts_for_tests();
11375        // Slots 0 and 1 should each have been cleaned because all of their
11376        // accounts are zero lamports
11377        assert!(accounts.storage.get_slot_storage_entry(0).is_none());
11378        assert!(accounts.storage.get_slot_storage_entry(1).is_none());
11379        // Slot 2 only has a zero lamport account as well. But, calc_delete_dependencies()
11380        // should exclude slot 2 from the clean due to changes in other slots
11381        assert!(accounts.storage.get_slot_storage_entry(2).is_some());
11382        // Index ref counts should be consistent with the slot stores. Account 1 ref count
11383        // should be 1 since slot 2 is the only alive slot; account 2 should have a ref
11384        // count of 0 due to slot 0 being dead
11385        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 1);
11386        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 0);
11387
11388        accounts.clean_accounts_for_tests();
11389        // Slot 2 will now be cleaned, which will leave account 1 with a ref count of 0
11390        assert!(accounts.storage.get_slot_storage_entry(2).is_none());
11391        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
11392    }
11393
11394    #[test]
11395    fn test_clean_zero_lamport_and_old_roots() {
11396        solana_logger::setup();
11397
11398        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11399        let pubkey = solana_sdk::pubkey::new_rand();
11400        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11401        let zero_lamport_account =
11402            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11403
11404        // Store a zero-lamport account
11405        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11406        accounts.store_for_tests(1, &[(&pubkey, &zero_lamport_account)]);
11407
11408        // Simulate rooting the zero-lamport account, should be a
11409        // candidate for cleaning
11410        accounts.calculate_accounts_delta_hash(0);
11411        accounts.add_root_and_flush_write_cache(0);
11412        accounts.calculate_accounts_delta_hash(1);
11413        accounts.add_root_and_flush_write_cache(1);
11414
11415        // Slot 0 should be removed, and
11416        // zero-lamport account should be cleaned
11417        accounts.clean_accounts_for_tests();
11418
11419        assert!(accounts.storage.get_slot_storage_entry(0).is_none());
11420        assert!(accounts.storage.get_slot_storage_entry(1).is_none());
11421
11422        // Slot 0 should be cleaned because all it's accounts have been
11423        // updated in the rooted slot 1
11424        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11425
11426        // Slot 1 should be cleaned because all it's accounts are
11427        // zero lamports, and are not present in any other slot's
11428        // storage entries
11429        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11430
11431        // zero lamport account, should no longer exist in accounts index
11432        // because it has been removed
11433        assert!(accounts
11434            .accounts_index
11435            .get_for_tests(&pubkey, None, None)
11436            .is_none());
11437    }
11438
11439    #[test]
11440    fn test_clean_old_with_normal_account() {
11441        solana_logger::setup();
11442
11443        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11444        let pubkey = solana_sdk::pubkey::new_rand();
11445        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11446        //store an account
11447        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11448        accounts.store_for_tests(1, &[(&pubkey, &account)]);
11449
11450        // simulate slots are rooted after while
11451        accounts.calculate_accounts_delta_hash(0);
11452        accounts.add_root_and_flush_write_cache(0);
11453        accounts.calculate_accounts_delta_hash(1);
11454        accounts.add_root_and_flush_write_cache(1);
11455
11456        //even if rooted, old state isn't cleaned up
11457        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
11458        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11459
11460        accounts.clean_accounts_for_tests();
11461
11462        //now old state is cleaned up
11463        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11464        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11465    }
11466
11467    #[test]
11468    fn test_clean_old_with_zero_lamport_account() {
11469        solana_logger::setup();
11470
11471        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11472        let pubkey1 = solana_sdk::pubkey::new_rand();
11473        let pubkey2 = solana_sdk::pubkey::new_rand();
11474        let normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11475        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11476        //store an account
11477        accounts.store_for_tests(0, &[(&pubkey1, &normal_account)]);
11478        accounts.store_for_tests(1, &[(&pubkey1, &zero_account)]);
11479        accounts.store_for_tests(0, &[(&pubkey2, &normal_account)]);
11480        accounts.store_for_tests(1, &[(&pubkey2, &normal_account)]);
11481
11482        //simulate slots are rooted after while
11483        accounts.calculate_accounts_delta_hash(0);
11484        accounts.add_root_and_flush_write_cache(0);
11485        accounts.calculate_accounts_delta_hash(1);
11486        accounts.add_root_and_flush_write_cache(1);
11487
11488        //even if rooted, old state isn't cleaned up
11489        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
11490        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
11491
11492        accounts.print_accounts_stats("");
11493
11494        accounts.clean_accounts_for_tests();
11495
11496        //Old state behind zero-lamport account is cleaned up
11497        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11498        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
11499    }
11500
11501    #[test]
11502    fn test_clean_old_with_both_normal_and_zero_lamport_accounts() {
11503        solana_logger::setup();
11504
11505        let mut accounts = AccountsDb::new_with_config_for_tests(
11506            Vec::new(),
11507            &ClusterType::Development,
11508            spl_token_mint_index_enabled(),
11509            AccountShrinkThreshold::default(),
11510        );
11511        let pubkey1 = solana_sdk::pubkey::new_rand();
11512        let pubkey2 = solana_sdk::pubkey::new_rand();
11513
11514        // Set up account to be added to secondary index
11515        let mint_key = Pubkey::new_unique();
11516        let mut account_data_with_mint = vec![0; inline_spl_token::Account::get_packed_len()];
11517        account_data_with_mint[..PUBKEY_BYTES].clone_from_slice(&(mint_key.to_bytes()));
11518
11519        let mut normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11520        normal_account.set_owner(inline_spl_token::id());
11521        normal_account.set_data(account_data_with_mint.clone());
11522        let mut zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11523        zero_account.set_owner(inline_spl_token::id());
11524        zero_account.set_data(account_data_with_mint);
11525
11526        //store an account
11527        accounts.store_for_tests(0, &[(&pubkey1, &normal_account)]);
11528        accounts.store_for_tests(0, &[(&pubkey1, &normal_account)]);
11529        accounts.store_for_tests(1, &[(&pubkey1, &zero_account)]);
11530        accounts.store_for_tests(0, &[(&pubkey2, &normal_account)]);
11531        accounts.store_for_tests(2, &[(&pubkey2, &normal_account)]);
11532
11533        //simulate slots are rooted after while
11534        accounts.calculate_accounts_delta_hash(0);
11535        accounts.add_root_and_flush_write_cache(0);
11536        accounts.calculate_accounts_delta_hash(1);
11537        accounts.add_root_and_flush_write_cache(1);
11538        accounts.calculate_accounts_delta_hash(2);
11539        accounts.add_root_and_flush_write_cache(2);
11540
11541        //even if rooted, old state isn't cleaned up
11542        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
11543        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11544        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
11545
11546        // Secondary index should still find both pubkeys
11547        let mut found_accounts = HashSet::new();
11548        let index_key = IndexKey::SplTokenMint(mint_key);
11549        let bank_id = 0;
11550        accounts
11551            .accounts_index
11552            .index_scan_accounts(
11553                &Ancestors::default(),
11554                bank_id,
11555                index_key,
11556                |key, _| {
11557                    found_accounts.insert(*key);
11558                },
11559                &ScanConfig::default(),
11560            )
11561            .unwrap();
11562        assert_eq!(found_accounts.len(), 2);
11563        assert!(found_accounts.contains(&pubkey1));
11564        assert!(found_accounts.contains(&pubkey2));
11565
11566        {
11567            accounts.account_indexes.keys = Some(AccountSecondaryIndexesIncludeExclude {
11568                exclude: true,
11569                keys: [mint_key].iter().cloned().collect::<HashSet<Pubkey>>(),
11570            });
11571            // Secondary index can't be used - do normal scan: should still find both pubkeys
11572            let mut found_accounts = HashSet::new();
11573            let used_index = accounts
11574                .index_scan_accounts(
11575                    &Ancestors::default(),
11576                    bank_id,
11577                    index_key,
11578                    |account| {
11579                        found_accounts.insert(*account.unwrap().0);
11580                    },
11581                    &ScanConfig::default(),
11582                )
11583                .unwrap();
11584            assert!(!used_index);
11585            assert_eq!(found_accounts.len(), 2);
11586            assert!(found_accounts.contains(&pubkey1));
11587            assert!(found_accounts.contains(&pubkey2));
11588
11589            accounts.account_indexes.keys = None;
11590
11591            // Secondary index can now be used since it isn't marked as excluded
11592            let mut found_accounts = HashSet::new();
11593            let used_index = accounts
11594                .index_scan_accounts(
11595                    &Ancestors::default(),
11596                    bank_id,
11597                    index_key,
11598                    |account| {
11599                        found_accounts.insert(*account.unwrap().0);
11600                    },
11601                    &ScanConfig::default(),
11602                )
11603                .unwrap();
11604            assert!(used_index);
11605            assert_eq!(found_accounts.len(), 2);
11606            assert!(found_accounts.contains(&pubkey1));
11607            assert!(found_accounts.contains(&pubkey2));
11608
11609            accounts.account_indexes.keys = None;
11610        }
11611
11612        accounts.clean_accounts_for_tests();
11613
11614        //both zero lamport and normal accounts are cleaned up
11615        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11616        // The only store to slot 1 was a zero lamport account, should
11617        // be purged by zero-lamport cleaning logic because slot 1 is
11618        // rooted
11619        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11620        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
11621
11622        // `pubkey1`, a zero lamport account, should no longer exist in accounts index
11623        // because it has been removed by the clean
11624        assert!(accounts
11625            .accounts_index
11626            .get_for_tests(&pubkey1, None, None)
11627            .is_none());
11628
11629        // Secondary index should have purged `pubkey1` as well
11630        let mut found_accounts = vec![];
11631        accounts
11632            .accounts_index
11633            .index_scan_accounts(
11634                &Ancestors::default(),
11635                bank_id,
11636                IndexKey::SplTokenMint(mint_key),
11637                |key, _| found_accounts.push(*key),
11638                &ScanConfig::default(),
11639            )
11640            .unwrap();
11641        assert_eq!(found_accounts, vec![pubkey2]);
11642    }
11643
11644    #[test]
11645    fn test_clean_max_slot_zero_lamport_account() {
11646        solana_logger::setup();
11647
11648        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11649        let pubkey = solana_sdk::pubkey::new_rand();
11650        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11651        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11652
11653        // store an account, make it a zero lamport account
11654        // in slot 1
11655        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11656        accounts.store_for_tests(1, &[(&pubkey, &zero_account)]);
11657
11658        // simulate slots are rooted after while
11659        accounts.calculate_accounts_delta_hash(0);
11660        accounts.add_root_and_flush_write_cache(0);
11661        accounts.calculate_accounts_delta_hash(1);
11662        accounts.add_root_and_flush_write_cache(1);
11663
11664        // Only clean up to account 0, should not purge slot 0 based on
11665        // updates in later slots in slot 1
11666        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
11667        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11668        accounts.clean_accounts(Some(0), false, None);
11669        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
11670        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11671        assert!(accounts
11672            .accounts_index
11673            .get_for_tests(&pubkey, None, None)
11674            .is_some());
11675
11676        // Now the account can be cleaned up
11677        accounts.clean_accounts(Some(1), false, None);
11678        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11679        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11680
11681        // The zero lamport account, should no longer exist in accounts index
11682        // because it has been removed
11683        assert!(accounts
11684            .accounts_index
11685            .get_for_tests(&pubkey, None, None)
11686            .is_none());
11687    }
11688
11689    #[test]
11690    fn test_uncleaned_roots_with_account() {
11691        solana_logger::setup();
11692
11693        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11694        let pubkey = solana_sdk::pubkey::new_rand();
11695        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11696        //store an account
11697        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11698        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11699
11700        // simulate slots are rooted after while
11701        accounts.add_root_and_flush_write_cache(0);
11702        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
11703
11704        //now uncleaned roots are cleaned up
11705        accounts.clean_accounts_for_tests();
11706        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11707    }
11708
11709    #[test]
11710    fn test_uncleaned_roots_with_no_account() {
11711        solana_logger::setup();
11712
11713        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
11714
11715        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11716
11717        // simulate slots are rooted after while
11718        accounts.add_root_and_flush_write_cache(0);
11719        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
11720
11721        //now uncleaned roots are cleaned up
11722        accounts.clean_accounts_for_tests();
11723        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11724    }
11725
11726    #[test]
11727    fn test_accounts_db_serialize1() {
11728        for pass in 0..2 {
11729            solana_logger::setup();
11730            let accounts = AccountsDb::new_single_for_tests();
11731            let mut pubkeys: Vec<Pubkey> = vec![];
11732
11733            // Create 100 accounts in slot 0
11734            create_account(&accounts, &mut pubkeys, 0, 100, 0, 0);
11735            if pass == 0 {
11736                accounts.add_root_and_flush_write_cache(0);
11737                check_storage(&accounts, 0, 100);
11738                accounts.clean_accounts_for_tests();
11739                check_accounts(&accounts, &pubkeys, 0, 100, 1);
11740                // clean should have done nothing
11741                continue;
11742            }
11743
11744            // do some updates to those accounts and re-check
11745            modify_accounts(&accounts, &pubkeys, 0, 100, 2);
11746            accounts.add_root_and_flush_write_cache(0);
11747            check_storage(&accounts, 0, 100);
11748            check_accounts(&accounts, &pubkeys, 0, 100, 2);
11749            accounts.calculate_accounts_delta_hash(0);
11750
11751            let mut pubkeys1: Vec<Pubkey> = vec![];
11752
11753            // CREATE SLOT 1
11754            let latest_slot = 1;
11755
11756            // Modify the first 10 of the accounts from slot 0 in slot 1
11757            modify_accounts(&accounts, &pubkeys, latest_slot, 10, 3);
11758            // Overwrite account 30 from slot 0 with lamports=0 into slot 1.
11759            // Slot 1 should now have 10 + 1 = 11 accounts
11760            let account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11761            accounts.store_for_tests(latest_slot, &[(&pubkeys[30], &account)]);
11762
11763            // Create 10 new accounts in slot 1, should now have 11 + 10 = 21
11764            // accounts
11765            create_account(&accounts, &mut pubkeys1, latest_slot, 10, 0, 0);
11766
11767            accounts.calculate_accounts_delta_hash(latest_slot);
11768            accounts.add_root_and_flush_write_cache(latest_slot);
11769            check_storage(&accounts, 1, 21);
11770
11771            // CREATE SLOT 2
11772            let latest_slot = 2;
11773            let mut pubkeys2: Vec<Pubkey> = vec![];
11774
11775            // Modify first 20 of the accounts from slot 0 in slot 2
11776            modify_accounts(&accounts, &pubkeys, latest_slot, 20, 4);
11777            accounts.clean_accounts_for_tests();
11778            // Overwrite account 31 from slot 0 with lamports=0 into slot 2.
11779            // Slot 2 should now have 20 + 1 = 21 accounts
11780            let account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11781            accounts.store_for_tests(latest_slot, &[(&pubkeys[31], &account)]);
11782
11783            // Create 10 new accounts in slot 2. Slot 2 should now have
11784            // 21 + 10 = 31 accounts
11785            create_account(&accounts, &mut pubkeys2, latest_slot, 10, 0, 0);
11786
11787            accounts.calculate_accounts_delta_hash(latest_slot);
11788            accounts.add_root_and_flush_write_cache(latest_slot);
11789            check_storage(&accounts, 2, 31);
11790
11791            let ancestors = linear_ancestors(latest_slot);
11792            accounts.update_accounts_hash_for_tests(latest_slot, &ancestors, false, false);
11793
11794            accounts.clean_accounts_for_tests();
11795            // The first 20 accounts of slot 0 have been updated in slot 2, as well as
11796            // accounts 30 and  31 (overwritten with zero-lamport accounts in slot 1 and
11797            // slot 2 respectively), so only 78 accounts are left in slot 0's storage entries.
11798            check_storage(&accounts, 0, 78);
11799            // 10 of the 21 accounts have been modified in slot 2, so only 11
11800            // accounts left in slot 1.
11801            check_storage(&accounts, 1, 11);
11802            check_storage(&accounts, 2, 31);
11803
11804            let daccounts = reconstruct_accounts_db_via_serialization(&accounts, latest_slot);
11805
11806            assert_eq!(
11807                daccounts.write_version.load(Ordering::Acquire),
11808                accounts.write_version.load(Ordering::Acquire)
11809            );
11810
11811            // Get the hashes for the latest slot, which should be the only hashes in the
11812            // map on the deserialized AccountsDb
11813            assert_eq!(daccounts.accounts_delta_hashes.lock().unwrap().len(), 1);
11814            assert_eq!(daccounts.accounts_hashes.lock().unwrap().len(), 1);
11815            assert_eq!(
11816                daccounts.get_accounts_delta_hash(latest_slot).unwrap(),
11817                accounts.get_accounts_delta_hash(latest_slot).unwrap(),
11818            );
11819            assert_eq!(
11820                daccounts.get_accounts_hash(latest_slot).unwrap(),
11821                accounts.get_accounts_hash(latest_slot).unwrap(),
11822            );
11823
11824            daccounts.print_count_and_status("daccounts");
11825
11826            // Don't check the first 35 accounts which have not been modified on slot 0
11827            check_accounts(&daccounts, &pubkeys[35..], 0, 65, 37);
11828            check_accounts(&daccounts, &pubkeys1, 1, 10, 1);
11829            check_storage(&daccounts, 0, 100);
11830            check_storage(&daccounts, 1, 21);
11831            check_storage(&daccounts, 2, 31);
11832
11833            assert_eq!(
11834                daccounts.update_accounts_hash_for_tests(latest_slot, &ancestors, false, false,),
11835                accounts.update_accounts_hash_for_tests(latest_slot, &ancestors, false, false,)
11836            );
11837        }
11838    }
11839
11840    fn assert_load_account(
11841        accounts: &AccountsDb,
11842        slot: Slot,
11843        pubkey: Pubkey,
11844        expected_lamports: u64,
11845    ) {
11846        let ancestors = vec![(slot, 0)].into_iter().collect();
11847        let (account, slot) = accounts
11848            .load_without_fixed_root(&ancestors, &pubkey)
11849            .unwrap();
11850        assert_eq!((account.lamports(), slot), (expected_lamports, slot));
11851    }
11852
11853    fn assert_not_load_account(accounts: &AccountsDb, slot: Slot, pubkey: Pubkey) {
11854        let ancestors = vec![(slot, 0)].into_iter().collect();
11855        let load = accounts.load_without_fixed_root(&ancestors, &pubkey);
11856        assert!(load.is_none(), "{load:?}");
11857    }
11858
11859    fn reconstruct_accounts_db_via_serialization(accounts: &AccountsDb, slot: Slot) -> AccountsDb {
11860        let daccounts =
11861            crate::serde_snapshot::reconstruct_accounts_db_via_serialization(accounts, slot);
11862        daccounts.print_count_and_status("daccounts");
11863        daccounts
11864    }
11865
11866    fn assert_no_stores(accounts: &AccountsDb, slot: Slot) {
11867        let store = accounts.storage.get_slot_storage_entry(slot);
11868        assert!(store.is_none());
11869    }
11870
11871    #[test]
11872    fn test_accounts_db_purge_keep_live() {
11873        solana_logger::setup();
11874        let some_lamport = 223;
11875        let zero_lamport = 0;
11876        let no_data = 0;
11877        let owner = *AccountSharedData::default().owner();
11878
11879        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11880        let pubkey = solana_sdk::pubkey::new_rand();
11881
11882        let account2 = AccountSharedData::new(some_lamport, no_data, &owner);
11883        let pubkey2 = solana_sdk::pubkey::new_rand();
11884
11885        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11886
11887        let accounts = AccountsDb::new_single_for_tests();
11888        accounts.calculate_accounts_delta_hash(0);
11889        accounts.add_root_and_flush_write_cache(0);
11890
11891        // Step A
11892        let mut current_slot = 1;
11893        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
11894        // Store another live account to slot 1 which will prevent any purge
11895        // since the store count will not be zero
11896        accounts.store_for_tests(current_slot, &[(&pubkey2, &account2)]);
11897        accounts.calculate_accounts_delta_hash(current_slot);
11898        accounts.add_root_and_flush_write_cache(current_slot);
11899        let (slot1, account_info1) = accounts
11900            .accounts_index
11901            .get_for_tests(&pubkey, None, None)
11902            .map(|(account_list1, index1)| account_list1.slot_list()[index1])
11903            .unwrap();
11904        let (slot2, account_info2) = accounts
11905            .accounts_index
11906            .get_for_tests(&pubkey2, None, None)
11907            .map(|(account_list2, index2)| account_list2.slot_list()[index2])
11908            .unwrap();
11909        assert_eq!(slot1, current_slot);
11910        assert_eq!(slot1, slot2);
11911        assert_eq!(account_info1.store_id(), account_info2.store_id());
11912
11913        // Step B
11914        current_slot += 1;
11915        let zero_lamport_slot = current_slot;
11916        accounts.store_for_tests(current_slot, &[(&pubkey, &zero_lamport_account)]);
11917        accounts.calculate_accounts_delta_hash(current_slot);
11918        accounts.add_root_and_flush_write_cache(current_slot);
11919
11920        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
11921
11922        current_slot += 1;
11923        accounts.calculate_accounts_delta_hash(current_slot);
11924        accounts.add_root_and_flush_write_cache(current_slot);
11925
11926        accounts.print_accounts_stats("pre_purge");
11927
11928        accounts.clean_accounts_for_tests();
11929
11930        accounts.print_accounts_stats("post_purge");
11931
11932        // The earlier entry for pubkey in the account index is purged,
11933        let (slot_list_len, index_slot) = {
11934            let account_entry = accounts
11935                .accounts_index
11936                .get_account_read_entry(&pubkey)
11937                .unwrap();
11938            let slot_list = account_entry.slot_list();
11939            (slot_list.len(), slot_list[0].0)
11940        };
11941        assert_eq!(slot_list_len, 1);
11942        // Zero lamport entry was not the one purged
11943        assert_eq!(index_slot, zero_lamport_slot);
11944        // The ref count should still be 2 because no slots were purged
11945        assert_eq!(accounts.ref_count_for_pubkey(&pubkey), 2);
11946
11947        // storage for slot 1 had 2 accounts, now has 1 after pubkey 1
11948        // was reclaimed
11949        check_storage(&accounts, 1, 1);
11950        // storage for slot 2 had 1 accounts, now has 1
11951        check_storage(&accounts, 2, 1);
11952    }
11953
11954    #[test]
11955    fn test_accounts_db_purge1() {
11956        solana_logger::setup();
11957        let some_lamport = 223;
11958        let zero_lamport = 0;
11959        let no_data = 0;
11960        let owner = *AccountSharedData::default().owner();
11961
11962        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11963        let pubkey = solana_sdk::pubkey::new_rand();
11964
11965        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11966
11967        let accounts = AccountsDb::new_single_for_tests();
11968        accounts.add_root(0);
11969
11970        let mut current_slot = 1;
11971        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
11972        accounts.calculate_accounts_delta_hash(current_slot);
11973        accounts.add_root_and_flush_write_cache(current_slot);
11974
11975        current_slot += 1;
11976        accounts.store_for_tests(current_slot, &[(&pubkey, &zero_lamport_account)]);
11977        accounts.calculate_accounts_delta_hash(current_slot);
11978        accounts.add_root_and_flush_write_cache(current_slot);
11979
11980        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
11981
11982        // Otherwise slot 2 will not be removed
11983        current_slot += 1;
11984        accounts.calculate_accounts_delta_hash(current_slot);
11985        accounts.add_root_and_flush_write_cache(current_slot);
11986
11987        accounts.print_accounts_stats("pre_purge");
11988
11989        let ancestors = linear_ancestors(current_slot);
11990        info!("ancestors: {:?}", ancestors);
11991        let hash = accounts.update_accounts_hash_for_tests(current_slot, &ancestors, true, true);
11992
11993        accounts.clean_accounts_for_tests();
11994
11995        assert_eq!(
11996            accounts.update_accounts_hash_for_tests(current_slot, &ancestors, true, true),
11997            hash
11998        );
11999
12000        accounts.print_accounts_stats("post_purge");
12001
12002        // Make sure the index is for pubkey cleared
12003        assert!(accounts
12004            .accounts_index
12005            .get_account_read_entry(&pubkey)
12006            .is_none());
12007
12008        // slot 1 & 2 should not have any stores
12009        assert_no_stores(&accounts, 1);
12010        assert_no_stores(&accounts, 2);
12011    }
12012
12013    #[test]
12014    fn test_accounts_db_serialize_zero_and_free() {
12015        solana_logger::setup();
12016
12017        let some_lamport = 223;
12018        let zero_lamport = 0;
12019        let no_data = 0;
12020        let owner = *AccountSharedData::default().owner();
12021
12022        let account = AccountSharedData::new(some_lamport, no_data, &owner);
12023        let pubkey = solana_sdk::pubkey::new_rand();
12024        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
12025
12026        let account2 = AccountSharedData::new(some_lamport + 1, no_data, &owner);
12027        let pubkey2 = solana_sdk::pubkey::new_rand();
12028
12029        let filler_account = AccountSharedData::new(some_lamport, no_data, &owner);
12030        let filler_account_pubkey = solana_sdk::pubkey::new_rand();
12031
12032        let accounts = AccountsDb::new_single_for_tests();
12033
12034        let mut current_slot = 1;
12035        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
12036        accounts.add_root(current_slot);
12037
12038        current_slot += 1;
12039        accounts.store_for_tests(current_slot, &[(&pubkey, &zero_lamport_account)]);
12040        accounts.store_for_tests(current_slot, &[(&pubkey2, &account2)]);
12041
12042        // Store the account a few times.
12043        // use to be: store enough accounts such that an additional store for slot 2 is created.
12044        // but we use the write cache now
12045        for _ in 0..3 {
12046            accounts.store_for_tests(current_slot, &[(&filler_account_pubkey, &filler_account)]);
12047        }
12048        accounts.add_root_and_flush_write_cache(current_slot);
12049
12050        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
12051
12052        accounts.print_accounts_stats("accounts");
12053
12054        accounts.clean_accounts_for_tests();
12055
12056        accounts.print_accounts_stats("accounts_post_purge");
12057
12058        accounts.calculate_accounts_delta_hash(current_slot);
12059        accounts.update_accounts_hash_for_tests(
12060            current_slot,
12061            &linear_ancestors(current_slot),
12062            false,
12063            false,
12064        );
12065        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
12066
12067        accounts.print_accounts_stats("reconstructed");
12068
12069        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
12070    }
12071
12072    fn with_chained_zero_lamport_accounts<F>(f: F)
12073    where
12074        F: Fn(AccountsDb, Slot) -> AccountsDb,
12075    {
12076        let some_lamport = 223;
12077        let zero_lamport = 0;
12078        let dummy_lamport = 999;
12079        let no_data = 0;
12080        let owner = *AccountSharedData::default().owner();
12081
12082        let account = AccountSharedData::new(some_lamport, no_data, &owner);
12083        let account2 = AccountSharedData::new(some_lamport + 100_001, no_data, &owner);
12084        let account3 = AccountSharedData::new(some_lamport + 100_002, no_data, &owner);
12085        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
12086
12087        let pubkey = solana_sdk::pubkey::new_rand();
12088        let purged_pubkey1 = solana_sdk::pubkey::new_rand();
12089        let purged_pubkey2 = solana_sdk::pubkey::new_rand();
12090
12091        let dummy_account = AccountSharedData::new(dummy_lamport, no_data, &owner);
12092        let dummy_pubkey = Pubkey::default();
12093
12094        let accounts = AccountsDb::new_single_for_tests();
12095
12096        let mut current_slot = 1;
12097        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
12098        accounts.store_for_tests(current_slot, &[(&purged_pubkey1, &account2)]);
12099        accounts.add_root_and_flush_write_cache(current_slot);
12100
12101        current_slot += 1;
12102        accounts.store_for_tests(current_slot, &[(&purged_pubkey1, &zero_lamport_account)]);
12103        accounts.store_for_tests(current_slot, &[(&purged_pubkey2, &account3)]);
12104        accounts.add_root_and_flush_write_cache(current_slot);
12105
12106        current_slot += 1;
12107        accounts.store_for_tests(current_slot, &[(&purged_pubkey2, &zero_lamport_account)]);
12108        accounts.add_root_and_flush_write_cache(current_slot);
12109
12110        current_slot += 1;
12111        accounts.store_for_tests(current_slot, &[(&dummy_pubkey, &dummy_account)]);
12112        accounts.add_root_and_flush_write_cache(current_slot);
12113
12114        accounts.print_accounts_stats("pre_f");
12115        accounts.calculate_accounts_delta_hash(current_slot);
12116        accounts.update_accounts_hash_for_tests(4, &Ancestors::default(), false, false);
12117
12118        let accounts = f(accounts, current_slot);
12119
12120        accounts.print_accounts_stats("post_f");
12121
12122        assert_load_account(&accounts, current_slot, pubkey, some_lamport);
12123        assert_load_account(&accounts, current_slot, purged_pubkey1, 0);
12124        assert_load_account(&accounts, current_slot, purged_pubkey2, 0);
12125        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
12126
12127        let ancestors = Ancestors::default();
12128        let epoch_schedule = EpochSchedule::default();
12129        let rent_collector = RentCollector::default();
12130        let config = BankHashLamportsVerifyConfig::new_for_test(
12131            &ancestors,
12132            &epoch_schedule,
12133            &rent_collector,
12134        );
12135
12136        accounts
12137            .verify_bank_hash_and_lamports(4, 1222, config)
12138            .unwrap();
12139    }
12140
12141    #[test]
12142    fn test_accounts_purge_chained_purge_before_snapshot_restore() {
12143        solana_logger::setup();
12144        with_chained_zero_lamport_accounts(|accounts, current_slot| {
12145            accounts.clean_accounts_for_tests();
12146            reconstruct_accounts_db_via_serialization(&accounts, current_slot)
12147        });
12148    }
12149
12150    #[test]
12151    fn test_accounts_purge_chained_purge_after_snapshot_restore() {
12152        solana_logger::setup();
12153        with_chained_zero_lamport_accounts(|accounts, current_slot| {
12154            let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
12155            accounts.print_accounts_stats("after_reconstruct");
12156            accounts.clean_accounts_for_tests();
12157            reconstruct_accounts_db_via_serialization(&accounts, current_slot)
12158        });
12159    }
12160
12161    #[test]
12162    #[ignore]
12163    fn test_store_account_stress() {
12164        let slot = 42;
12165        let num_threads = 2;
12166
12167        let min_file_bytes = std::mem::size_of::<StoredMeta>()
12168            + std::mem::size_of::<crate::append_vec::AccountMeta>();
12169
12170        let db = Arc::new(AccountsDb::new_sized(Vec::new(), min_file_bytes as u64));
12171
12172        db.add_root(slot);
12173        let thread_hdls: Vec<_> = (0..num_threads)
12174            .map(|_| {
12175                let db = db.clone();
12176                std::thread::Builder::new()
12177                    .name("account-writers".to_string())
12178                    .spawn(move || {
12179                        let pubkey = solana_sdk::pubkey::new_rand();
12180                        let mut account = AccountSharedData::new(1, 0, &pubkey);
12181                        let mut i = 0;
12182                        loop {
12183                            let account_bal = thread_rng().gen_range(1, 99);
12184                            account.set_lamports(account_bal);
12185                            db.store_for_tests(slot, &[(&pubkey, &account)]);
12186
12187                            let (account, slot) = db
12188                                .load_without_fixed_root(&Ancestors::default(), &pubkey)
12189                                .unwrap_or_else(|| {
12190                                    panic!("Could not fetch stored account {pubkey}, iter {i}")
12191                                });
12192                            assert_eq!(slot, slot);
12193                            assert_eq!(account.lamports(), account_bal);
12194                            i += 1;
12195                        }
12196                    })
12197                    .unwrap()
12198            })
12199            .collect();
12200
12201        for t in thread_hdls {
12202            t.join().unwrap();
12203        }
12204    }
12205
12206    #[test]
12207    fn test_accountsdb_scan_accounts() {
12208        solana_logger::setup();
12209        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12210        let key = Pubkey::default();
12211        let key0 = solana_sdk::pubkey::new_rand();
12212        let account0 = AccountSharedData::new(1, 0, &key);
12213
12214        db.store_for_tests(0, &[(&key0, &account0)]);
12215
12216        let key1 = solana_sdk::pubkey::new_rand();
12217        let account1 = AccountSharedData::new(2, 0, &key);
12218        db.store_for_tests(1, &[(&key1, &account1)]);
12219
12220        let ancestors = vec![(0, 0)].into_iter().collect();
12221        let mut accounts = Vec::new();
12222        db.unchecked_scan_accounts(
12223            "",
12224            &ancestors,
12225            |_, account, _| {
12226                accounts.push(account.take_account());
12227            },
12228            &ScanConfig::default(),
12229        );
12230        assert_eq!(accounts, vec![account0]);
12231
12232        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
12233        let mut accounts = Vec::new();
12234        db.unchecked_scan_accounts(
12235            "",
12236            &ancestors,
12237            |_, account, _| {
12238                accounts.push(account.take_account());
12239            },
12240            &ScanConfig::default(),
12241        );
12242        assert_eq!(accounts.len(), 2);
12243    }
12244
12245    #[test]
12246    fn test_cleanup_key_not_removed() {
12247        solana_logger::setup();
12248        let db = AccountsDb::new_single_for_tests();
12249
12250        let key = Pubkey::default();
12251        let key0 = solana_sdk::pubkey::new_rand();
12252        let account0 = AccountSharedData::new(1, 0, &key);
12253
12254        db.store_for_tests(0, &[(&key0, &account0)]);
12255
12256        let key1 = solana_sdk::pubkey::new_rand();
12257        let account1 = AccountSharedData::new(2, 0, &key);
12258        db.store_for_tests(1, &[(&key1, &account1)]);
12259
12260        db.print_accounts_stats("pre");
12261
12262        let slots: HashSet<Slot> = vec![1].into_iter().collect();
12263        let purge_keys = vec![(key1, slots)];
12264        let _ = db.purge_keys_exact(purge_keys.iter());
12265
12266        let account2 = AccountSharedData::new(3, 0, &key);
12267        db.store_for_tests(2, &[(&key1, &account2)]);
12268
12269        db.print_accounts_stats("post");
12270        let ancestors = vec![(2, 0)].into_iter().collect();
12271        assert_eq!(
12272            db.load_without_fixed_root(&ancestors, &key1)
12273                .unwrap()
12274                .0
12275                .lamports(),
12276            3
12277        );
12278    }
12279
12280    #[test]
12281    fn test_store_large_account() {
12282        solana_logger::setup();
12283        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12284
12285        let key = Pubkey::default();
12286        let data_len = DEFAULT_FILE_SIZE as usize + 7;
12287        let account = AccountSharedData::new(1, data_len, &key);
12288
12289        db.store_for_tests(0, &[(&key, &account)]);
12290
12291        let ancestors = vec![(0, 0)].into_iter().collect();
12292        let ret = db.load_without_fixed_root(&ancestors, &key).unwrap();
12293        assert_eq!(ret.0.data().len(), data_len);
12294    }
12295
12296    #[test]
12297    fn test_stored_readable_account() {
12298        let lamports = 1;
12299        let owner = Pubkey::new_unique();
12300        let executable = true;
12301        let rent_epoch = 2;
12302        let meta = StoredMeta {
12303            write_version_obsolete: 5,
12304            pubkey: Pubkey::new_unique(),
12305            data_len: 7,
12306        };
12307        let account_meta = AccountMeta {
12308            lamports,
12309            owner,
12310            executable,
12311            rent_epoch,
12312        };
12313        let data = Vec::new();
12314        let account = Account {
12315            lamports,
12316            owner,
12317            executable,
12318            rent_epoch,
12319            data: data.clone(),
12320        };
12321        let offset = 99;
12322        let stored_size = 101;
12323        let hash = Hash::new_unique();
12324        let stored_account = StoredAccountMeta {
12325            meta: &meta,
12326            account_meta: &account_meta,
12327            data: &data,
12328            offset,
12329            stored_size,
12330            hash: &hash,
12331        };
12332        assert!(accounts_equal(&account, &stored_account));
12333    }
12334
12335    #[test]
12336    fn test_hash_stored_account() {
12337        // This test uses some UNSAFE tricks to detect most of account's field
12338        // addition and deletion without changing the hash code
12339
12340        const ACCOUNT_DATA_LEN: usize = 3;
12341        // the type of InputTuple elements must not contain references;
12342        // they should be simple scalars or data blobs
12343        type InputTuple = (
12344            Slot,
12345            StoredMeta,
12346            AccountMeta,
12347            [u8; ACCOUNT_DATA_LEN],
12348            usize, // for StoredAccountMeta::offset
12349            Hash,
12350        );
12351        const INPUT_LEN: usize = std::mem::size_of::<InputTuple>();
12352        type InputBlob = [u8; INPUT_LEN];
12353        let mut blob: InputBlob = [0u8; INPUT_LEN];
12354
12355        // spray memory with decreasing counts so that, data layout can be detected.
12356        for (i, byte) in blob.iter_mut().enumerate() {
12357            *byte = (INPUT_LEN - i) as u8;
12358        }
12359
12360        //UNSAFE: forcibly cast the special byte pattern to actual account fields.
12361        let (slot, meta, account_meta, data, offset, hash): InputTuple =
12362            unsafe { std::mem::transmute::<InputBlob, InputTuple>(blob) };
12363
12364        let stored_account = StoredAccountMeta {
12365            meta: &meta,
12366            account_meta: &account_meta,
12367            data: &data,
12368            offset,
12369            stored_size: CACHE_VIRTUAL_STORED_SIZE as usize,
12370            hash: &hash,
12371        };
12372        let account = stored_account.clone_account();
12373
12374        let expected_account_hash = if cfg!(debug_assertions) {
12375            Hash::from_str("6qtBXmRrLdTdAV5bK6bZZJxQA4fPSUBxzQGq2BQSat25").unwrap()
12376        } else {
12377            Hash::from_str("5HL9MtsQmxZQ8XSgcAhSkqnrayQFXUY8FT1JsHjDNKbi").unwrap()
12378        };
12379
12380        assert_eq!(
12381            AccountsDb::hash_account(
12382                slot,
12383                &stored_account,
12384                stored_account.pubkey(),
12385                INCLUDE_SLOT_IN_HASH_TESTS
12386            ),
12387            expected_account_hash,
12388            "StoredAccountMeta's data layout might be changed; update hashing if needed."
12389        );
12390        assert_eq!(
12391            AccountsDb::hash_account(
12392                slot,
12393                &account,
12394                stored_account.pubkey(),
12395                INCLUDE_SLOT_IN_HASH_TESTS
12396            ),
12397            expected_account_hash,
12398            "Account-based hashing must be consistent with StoredAccountMeta-based one."
12399        );
12400    }
12401
12402    #[test]
12403    fn test_bank_hash_stats() {
12404        solana_logger::setup();
12405        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12406
12407        let key = Pubkey::default();
12408        let some_data_len = 5;
12409        let some_slot: Slot = 0;
12410        let account = AccountSharedData::new(1, some_data_len, &key);
12411        let ancestors = vec![(some_slot, 0)].into_iter().collect();
12412
12413        db.store_for_tests(some_slot, &[(&key, &account)]);
12414        let mut account = db.load_without_fixed_root(&ancestors, &key).unwrap().0;
12415        account.checked_sub_lamports(1).unwrap();
12416        account.set_executable(true);
12417        db.store_for_tests(some_slot, &[(&key, &account)]);
12418        db.add_root(some_slot);
12419
12420        let stats = db.get_bank_hash_stats(some_slot).unwrap();
12421        assert_eq!(stats.num_updated_accounts, 1);
12422        assert_eq!(stats.num_removed_accounts, 1);
12423        assert_eq!(stats.num_lamports_stored, 1);
12424        assert_eq!(stats.total_data_len, 2 * some_data_len as u64);
12425        assert_eq!(stats.num_executable_accounts, 1);
12426    }
12427
12428    // this test tests check_hash=true, which is unsupported behavior at the moment. It cannot be enabled by anything but these tests.
12429    #[ignore]
12430    #[test]
12431    fn test_calculate_accounts_hash_check_hash_mismatch() {
12432        solana_logger::setup();
12433        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12434
12435        let key = solana_sdk::pubkey::new_rand();
12436        let some_data_len = 0;
12437        let some_slot: Slot = 0;
12438        let account = AccountSharedData::new(1, some_data_len, &key);
12439
12440        let ancestors = vec![(some_slot, 0)].into_iter().collect();
12441
12442        // put wrong hash value in store so we get a mismatch
12443        db.store_accounts_unfrozen(
12444            (some_slot, &[(&key, &account)][..]),
12445            Some(vec![&Hash::default()]),
12446            &StoreTo::Storage(&db.find_storage_candidate(some_slot, 1)),
12447            None,
12448            StoreReclaims::Default,
12449        );
12450        db.add_root(some_slot);
12451        let check_hash = true;
12452        for data_source in [
12453            CalcAccountsHashDataSource::IndexForTests,
12454            CalcAccountsHashDataSource::Storages,
12455        ] {
12456            assert!(db
12457                .calculate_accounts_hash(
12458                    data_source,
12459                    some_slot,
12460                    &CalcAccountsHashConfig {
12461                        use_bg_thread_pool: true, // is_startup used to be false
12462                        check_hash,
12463                        ancestors: Some(&ancestors),
12464                        ..CalcAccountsHashConfig::default()
12465                    },
12466                )
12467                .is_err());
12468        }
12469    }
12470
12471    // something we can get a ref to
12472    lazy_static! {
12473        pub static ref EPOCH_SCHEDULE: EpochSchedule = EpochSchedule::default();
12474        pub static ref RENT_COLLECTOR: RentCollector = RentCollector::default();
12475    }
12476
12477    impl<'a> CalcAccountsHashConfig<'a> {
12478        fn default() -> Self {
12479            Self {
12480                use_bg_thread_pool: false,
12481                check_hash: false,
12482                ancestors: None,
12483                epoch_schedule: &EPOCH_SCHEDULE,
12484                rent_collector: &RENT_COLLECTOR,
12485                store_detailed_debug_info_on_failure: false,
12486            }
12487        }
12488    }
12489
12490    // this test tests check_hash=true, which is unsupported behavior at the moment. It cannot be enabled by anything but these tests.
12491    #[ignore]
12492    #[test]
12493    fn test_calculate_accounts_hash_check_hash() {
12494        solana_logger::setup();
12495        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12496
12497        let key = solana_sdk::pubkey::new_rand();
12498        let some_data_len = 0;
12499        let some_slot: Slot = 0;
12500        let account = AccountSharedData::new(1, some_data_len, &key);
12501
12502        let ancestors = vec![(some_slot, 0)].into_iter().collect();
12503
12504        db.store_for_tests(some_slot, &[(&key, &account)]);
12505        db.add_root(some_slot);
12506        let check_hash = true;
12507        assert_eq!(
12508            db.calculate_accounts_hash(
12509                CalcAccountsHashDataSource::Storages,
12510                some_slot,
12511                &CalcAccountsHashConfig {
12512                    use_bg_thread_pool: true, // is_startup used to be false
12513                    check_hash,
12514                    ancestors: Some(&ancestors),
12515                    ..CalcAccountsHashConfig::default()
12516                },
12517            )
12518            .unwrap(),
12519            db.calculate_accounts_hash(
12520                CalcAccountsHashDataSource::IndexForTests,
12521                some_slot,
12522                &CalcAccountsHashConfig {
12523                    use_bg_thread_pool: true, // is_startup used to be false
12524                    check_hash,
12525                    ancestors: Some(&ancestors),
12526                    ..CalcAccountsHashConfig::default()
12527                },
12528            )
12529            .unwrap(),
12530        );
12531    }
12532
12533    #[test]
12534    fn test_verify_bank_hash() {
12535        use BankHashVerificationError::*;
12536        solana_logger::setup();
12537        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12538
12539        let key = solana_sdk::pubkey::new_rand();
12540        let some_data_len = 0;
12541        let some_slot: Slot = 0;
12542        let account = AccountSharedData::new(1, some_data_len, &key);
12543        let ancestors = vec![(some_slot, 0)].into_iter().collect();
12544        let epoch_schedule = EpochSchedule::default();
12545        let rent_collector = RentCollector::default();
12546
12547        db.store_for_tests(some_slot, &[(&key, &account)]);
12548        db.add_root_and_flush_write_cache(some_slot);
12549        db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
12550
12551        let config = BankHashLamportsVerifyConfig::new_for_test(
12552            &ancestors,
12553            &epoch_schedule,
12554            &rent_collector,
12555        );
12556
12557        assert_matches!(
12558            db.verify_bank_hash_and_lamports(some_slot, 1, config.clone()),
12559            Ok(_)
12560        );
12561
12562        db.remove_bank_hash_info(&some_slot);
12563
12564        assert_matches!(
12565            db.verify_bank_hash_and_lamports(some_slot, 1, config.clone()),
12566            Err(MissingBankHash)
12567        );
12568
12569        db.set_accounts_hash(some_slot, AccountsHash(Hash::new(&[0xca; HASH_BYTES])));
12570
12571        assert_matches!(
12572            db.verify_bank_hash_and_lamports(some_slot, 1, config),
12573            Err(MismatchedBankHash)
12574        );
12575    }
12576
12577    #[test]
12578    fn test_verify_bank_capitalization() {
12579        for pass in 0..2 {
12580            use BankHashVerificationError::*;
12581            solana_logger::setup();
12582            let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12583
12584            let key = solana_sdk::pubkey::new_rand();
12585            let some_data_len = 0;
12586            let some_slot: Slot = 0;
12587            let account = AccountSharedData::new(1, some_data_len, &key);
12588            let ancestors = vec![(some_slot, 0)].into_iter().collect();
12589            let epoch_schedule = EpochSchedule::default();
12590            let rent_collector = RentCollector::default();
12591            let config = BankHashLamportsVerifyConfig::new_for_test(
12592                &ancestors,
12593                &epoch_schedule,
12594                &rent_collector,
12595            );
12596
12597            db.store_for_tests(some_slot, &[(&key, &account)]);
12598            if pass == 0 {
12599                db.add_root_and_flush_write_cache(some_slot);
12600                db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
12601
12602                assert_matches!(
12603                    db.verify_bank_hash_and_lamports(some_slot, 1, config.clone()),
12604                    Ok(_)
12605                );
12606                continue;
12607            }
12608
12609            let native_account_pubkey = solana_sdk::pubkey::new_rand();
12610            db.store_for_tests(
12611                some_slot,
12612                &[(
12613                    &native_account_pubkey,
12614                    &solana_sdk::native_loader::create_loadable_account_for_test("foo"),
12615                )],
12616            );
12617            db.add_root_and_flush_write_cache(some_slot);
12618            db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
12619
12620            assert_matches!(
12621                db.verify_bank_hash_and_lamports(some_slot, 2, config.clone()),
12622                Ok(_)
12623            );
12624
12625            assert_matches!(
12626                db.verify_bank_hash_and_lamports(some_slot, 10, config),
12627                Err(MismatchedTotalLamports(expected, actual)) if expected == 2 && actual == 10
12628            );
12629        }
12630    }
12631
12632    #[test]
12633    fn test_verify_bank_hash_no_account() {
12634        solana_logger::setup();
12635        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12636
12637        let some_slot: Slot = 0;
12638        let ancestors = vec![(some_slot, 0)].into_iter().collect();
12639
12640        db.add_root(some_slot);
12641        db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
12642
12643        let epoch_schedule = EpochSchedule::default();
12644        let rent_collector = RentCollector::default();
12645        let config = BankHashLamportsVerifyConfig::new_for_test(
12646            &ancestors,
12647            &epoch_schedule,
12648            &rent_collector,
12649        );
12650
12651        assert_matches!(
12652            db.verify_bank_hash_and_lamports(some_slot, 0, config),
12653            Ok(_)
12654        );
12655    }
12656
12657    #[test]
12658    fn test_verify_bank_hash_bad_account_hash() {
12659        use BankHashVerificationError::*;
12660        solana_logger::setup();
12661        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12662
12663        let key = Pubkey::default();
12664        let some_data_len = 0;
12665        let some_slot: Slot = 0;
12666        let account = AccountSharedData::new(1, some_data_len, &key);
12667        let ancestors = vec![(some_slot, 0)].into_iter().collect();
12668
12669        let accounts = &[(&key, &account)][..];
12670        db.update_accounts_hash_for_tests(some_slot, &ancestors, false, false);
12671
12672        // provide bogus account hashes
12673        let some_hash = Hash::new(&[0xca; HASH_BYTES]);
12674        db.store_accounts_unfrozen(
12675            (some_slot, accounts),
12676            Some(vec![&some_hash]),
12677            &StoreTo::Storage(&db.find_storage_candidate(some_slot, 1)),
12678            None,
12679            StoreReclaims::Default,
12680        );
12681        db.add_root(some_slot);
12682
12683        let epoch_schedule = EpochSchedule::default();
12684        let rent_collector = RentCollector::default();
12685        let config = BankHashLamportsVerifyConfig::new_for_test(
12686            &ancestors,
12687            &epoch_schedule,
12688            &rent_collector,
12689        );
12690
12691        assert_matches!(
12692            db.verify_bank_hash_and_lamports(some_slot, 1, config),
12693            Err(MismatchedBankHash)
12694        );
12695    }
12696
12697    #[test]
12698    fn test_storage_finder() {
12699        solana_logger::setup();
12700        let db = AccountsDb::new_sized(Vec::new(), 16 * 1024);
12701        let key = solana_sdk::pubkey::new_rand();
12702        let lamports = 100;
12703        let data_len = 8190;
12704        let account = AccountSharedData::new(lamports, data_len, &solana_sdk::pubkey::new_rand());
12705        // pre-populate with a smaller empty store
12706        db.create_and_insert_store(1, 8192, "test_storage_finder");
12707        db.store_for_tests(1, &[(&key, &account)]);
12708    }
12709
12710    #[test]
12711    fn test_get_snapshot_storages_empty() {
12712        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12713        assert!(db.get_snapshot_storages(..=0, None).0.is_empty());
12714    }
12715
12716    #[test]
12717    fn test_get_snapshot_storages_only_older_than_or_equal_to_snapshot_slot() {
12718        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12719
12720        let key = Pubkey::default();
12721        let account = AccountSharedData::new(1, 0, &key);
12722        let before_slot = 0;
12723        let base_slot = before_slot + 1;
12724        let after_slot = base_slot + 1;
12725
12726        db.store_for_tests(base_slot, &[(&key, &account)]);
12727        db.add_root_and_flush_write_cache(base_slot);
12728        assert!(db.get_snapshot_storages(..=before_slot, None).0.is_empty());
12729
12730        assert_eq!(1, db.get_snapshot_storages(..=base_slot, None).0.len());
12731        assert_eq!(1, db.get_snapshot_storages(..=after_slot, None).0.len());
12732    }
12733
12734    #[test]
12735    fn test_get_snapshot_storages_only_non_empty() {
12736        for pass in 0..2 {
12737            let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12738
12739            let key = Pubkey::default();
12740            let account = AccountSharedData::new(1, 0, &key);
12741            let base_slot = 0;
12742            let after_slot = base_slot + 1;
12743
12744            db.store_for_tests(base_slot, &[(&key, &account)]);
12745            if pass == 0 {
12746                db.add_root_and_flush_write_cache(base_slot);
12747                db.storage.remove(&base_slot, false);
12748                assert!(db.get_snapshot_storages(..=after_slot, None).0.is_empty());
12749                continue;
12750            }
12751
12752            db.store_for_tests(base_slot, &[(&key, &account)]);
12753            db.add_root_and_flush_write_cache(base_slot);
12754            assert_eq!(1, db.get_snapshot_storages(..=after_slot, None).0.len());
12755        }
12756    }
12757
12758    #[test]
12759    fn test_get_snapshot_storages_only_roots() {
12760        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12761
12762        let key = Pubkey::default();
12763        let account = AccountSharedData::new(1, 0, &key);
12764        let base_slot = 0;
12765        let after_slot = base_slot + 1;
12766
12767        db.store_for_tests(base_slot, &[(&key, &account)]);
12768        assert!(db.get_snapshot_storages(..=after_slot, None).0.is_empty());
12769
12770        db.add_root_and_flush_write_cache(base_slot);
12771        assert_eq!(1, db.get_snapshot_storages(..=after_slot, None).0.len());
12772    }
12773
12774    #[test]
12775    fn test_get_snapshot_storages_exclude_empty() {
12776        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12777
12778        let key = Pubkey::default();
12779        let account = AccountSharedData::new(1, 0, &key);
12780        let base_slot = 0;
12781        let after_slot = base_slot + 1;
12782
12783        db.store_for_tests(base_slot, &[(&key, &account)]);
12784        db.add_root_and_flush_write_cache(base_slot);
12785        assert_eq!(1, db.get_snapshot_storages(..=after_slot, None).0.len());
12786
12787        db.storage
12788            .get_slot_storage_entry(0)
12789            .unwrap()
12790            .remove_account(0, true);
12791        assert!(db.get_snapshot_storages(..=after_slot, None).0.is_empty());
12792    }
12793
12794    #[test]
12795    fn test_get_snapshot_storages_with_base_slot() {
12796        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12797
12798        let key = Pubkey::default();
12799        let account = AccountSharedData::new(1, 0, &key);
12800
12801        let slot = 10;
12802        db.store_for_tests(slot, &[(&key, &account)]);
12803        db.add_root_and_flush_write_cache(slot);
12804        assert_eq!(
12805            0,
12806            db.get_snapshot_storages(slot + 1..=slot + 1, None).0.len()
12807        );
12808        assert_eq!(1, db.get_snapshot_storages(slot..=slot + 1, None).0.len());
12809    }
12810
12811    #[test]
12812    #[should_panic(expected = "double remove of account in slot: 0/store: 0!!")]
12813    fn test_storage_remove_account_double_remove() {
12814        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
12815        let pubkey = solana_sdk::pubkey::new_rand();
12816        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12817        accounts.store_for_tests(0, &[(&pubkey, &account)]);
12818        accounts.add_root_and_flush_write_cache(0);
12819        let storage_entry = accounts.storage.get_slot_storage_entry(0).unwrap();
12820        storage_entry.remove_account(0, true);
12821        storage_entry.remove_account(0, true);
12822    }
12823
12824    #[test]
12825    fn test_accounts_purge_long_chained_after_snapshot_restore() {
12826        solana_logger::setup();
12827        let old_lamport = 223;
12828        let zero_lamport = 0;
12829        let no_data = 0;
12830        let owner = *AccountSharedData::default().owner();
12831
12832        let account = AccountSharedData::new(old_lamport, no_data, &owner);
12833        let account2 = AccountSharedData::new(old_lamport + 100_001, no_data, &owner);
12834        let account3 = AccountSharedData::new(old_lamport + 100_002, no_data, &owner);
12835        let dummy_account = AccountSharedData::new(99_999_999, no_data, &owner);
12836        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
12837
12838        let pubkey = solana_sdk::pubkey::new_rand();
12839        let dummy_pubkey = solana_sdk::pubkey::new_rand();
12840        let purged_pubkey1 = solana_sdk::pubkey::new_rand();
12841        let purged_pubkey2 = solana_sdk::pubkey::new_rand();
12842
12843        let mut current_slot = 0;
12844        let accounts = AccountsDb::new_single_for_tests();
12845
12846        // create intermediate updates to purged_pubkey1 so that
12847        // generate_index must add slots as root last at once
12848        current_slot += 1;
12849        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
12850        accounts.store_for_tests(current_slot, &[(&purged_pubkey1, &account2)]);
12851        accounts.add_root_and_flush_write_cache(current_slot);
12852
12853        current_slot += 1;
12854        accounts.store_for_tests(current_slot, &[(&purged_pubkey1, &account2)]);
12855        accounts.add_root_and_flush_write_cache(current_slot);
12856
12857        current_slot += 1;
12858        accounts.store_for_tests(current_slot, &[(&purged_pubkey1, &account2)]);
12859        accounts.add_root_and_flush_write_cache(current_slot);
12860
12861        current_slot += 1;
12862        accounts.store_for_tests(current_slot, &[(&purged_pubkey1, &zero_lamport_account)]);
12863        accounts.store_for_tests(current_slot, &[(&purged_pubkey2, &account3)]);
12864        accounts.add_root_and_flush_write_cache(current_slot);
12865
12866        current_slot += 1;
12867        accounts.store_for_tests(current_slot, &[(&purged_pubkey2, &zero_lamport_account)]);
12868        accounts.add_root_and_flush_write_cache(current_slot);
12869
12870        current_slot += 1;
12871        accounts.store_for_tests(current_slot, &[(&dummy_pubkey, &dummy_account)]);
12872        accounts.add_root_and_flush_write_cache(current_slot);
12873
12874        accounts.print_count_and_status("before reconstruct");
12875        accounts.calculate_accounts_delta_hash(current_slot);
12876        accounts.update_accounts_hash_for_tests(
12877            current_slot,
12878            &linear_ancestors(current_slot),
12879            false,
12880            false,
12881        );
12882        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
12883        accounts.print_count_and_status("before purge zero");
12884        accounts.clean_accounts_for_tests();
12885        accounts.print_count_and_status("after purge zero");
12886
12887        assert_load_account(&accounts, current_slot, pubkey, old_lamport);
12888        assert_load_account(&accounts, current_slot, purged_pubkey1, 0);
12889        assert_load_account(&accounts, current_slot, purged_pubkey2, 0);
12890    }
12891
12892    fn do_full_clean_refcount(store1_first: bool, store_size: u64) {
12893        let pubkey1 = Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
12894        let pubkey2 = Pubkey::from_str("My22211111111111111111111111111111111111111").unwrap();
12895        let pubkey3 = Pubkey::from_str("My33311111111111111111111111111111111111111").unwrap();
12896
12897        let old_lamport = 223;
12898        let zero_lamport = 0;
12899        let dummy_lamport = 999_999;
12900
12901        // size data so only 1 fits in a 4k store
12902        let data_size = 2200;
12903
12904        let owner = *AccountSharedData::default().owner();
12905
12906        let account = AccountSharedData::new(old_lamport, data_size, &owner);
12907        let account2 = AccountSharedData::new(old_lamport + 100_001, data_size, &owner);
12908        let account3 = AccountSharedData::new(old_lamport + 100_002, data_size, &owner);
12909        let account4 = AccountSharedData::new(dummy_lamport, data_size, &owner);
12910        let zero_lamport_account = AccountSharedData::new(zero_lamport, data_size, &owner);
12911
12912        let mut current_slot = 0;
12913        let accounts = AccountsDb::new_sized_no_extra_stores(Vec::new(), store_size);
12914
12915        // A: Initialize AccountsDb with pubkey1 and pubkey2
12916        current_slot += 1;
12917        if store1_first {
12918            accounts.store_for_tests(current_slot, &[(&pubkey1, &account)]);
12919            accounts.store_for_tests(current_slot, &[(&pubkey2, &account)]);
12920        } else {
12921            accounts.store_for_tests(current_slot, &[(&pubkey2, &account)]);
12922            accounts.store_for_tests(current_slot, &[(&pubkey1, &account)]);
12923        }
12924        accounts.calculate_accounts_delta_hash(current_slot);
12925        accounts.add_root_and_flush_write_cache(current_slot);
12926
12927        info!("post A");
12928        accounts.print_accounts_stats("Post-A");
12929
12930        // B: Test multiple updates to pubkey1 in a single slot/storage
12931        current_slot += 1;
12932        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
12933        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
12934        accounts.store_for_tests(current_slot, &[(&pubkey1, &account2)]);
12935        accounts.store_for_tests(current_slot, &[(&pubkey1, &account2)]);
12936        accounts.add_root_and_flush_write_cache(current_slot);
12937        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
12938        // Stores to same pubkey, same slot only count once towards the
12939        // ref count
12940        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12941        accounts.calculate_accounts_delta_hash(current_slot);
12942        accounts.add_root_and_flush_write_cache(current_slot);
12943
12944        accounts.print_accounts_stats("Post-B pre-clean");
12945
12946        accounts.clean_accounts_for_tests();
12947
12948        info!("post B");
12949        accounts.print_accounts_stats("Post-B");
12950
12951        // C: more updates to trigger clean of previous updates
12952        current_slot += 1;
12953        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12954        accounts.store_for_tests(current_slot, &[(&pubkey1, &account3)]);
12955        accounts.store_for_tests(current_slot, &[(&pubkey2, &account3)]);
12956        accounts.store_for_tests(current_slot, &[(&pubkey3, &account4)]);
12957        accounts.add_root_and_flush_write_cache(current_slot);
12958        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12959        accounts.calculate_accounts_delta_hash(current_slot);
12960
12961        info!("post C");
12962
12963        accounts.print_accounts_stats("Post-C");
12964
12965        // D: Make all keys 0-lamport, cleans all keys
12966        current_slot += 1;
12967        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12968        accounts.store_for_tests(current_slot, &[(&pubkey1, &zero_lamport_account)]);
12969        accounts.store_for_tests(current_slot, &[(&pubkey2, &zero_lamport_account)]);
12970        accounts.store_for_tests(current_slot, &[(&pubkey3, &zero_lamport_account)]);
12971
12972        let snapshot_stores = accounts.get_snapshot_storages(..=current_slot, None).0;
12973        let total_accounts: usize = snapshot_stores.iter().map(|s| s.all_accounts().len()).sum();
12974        assert!(!snapshot_stores.is_empty());
12975        assert!(total_accounts > 0);
12976
12977        info!("post D");
12978        accounts.print_accounts_stats("Post-D");
12979
12980        accounts.calculate_accounts_delta_hash(current_slot);
12981        accounts.add_root_and_flush_write_cache(current_slot);
12982        accounts.clean_accounts_for_tests();
12983
12984        accounts.print_accounts_stats("Post-D clean");
12985
12986        let total_accounts_post_clean: usize =
12987            snapshot_stores.iter().map(|s| s.all_accounts().len()).sum();
12988        assert_eq!(total_accounts, total_accounts_post_clean);
12989
12990        // should clean all 3 pubkeys
12991        assert_eq!(accounts.ref_count_for_pubkey(&pubkey1), 0);
12992        assert_eq!(accounts.ref_count_for_pubkey(&pubkey2), 0);
12993        assert_eq!(accounts.ref_count_for_pubkey(&pubkey3), 0);
12994    }
12995
12996    #[test]
12997    fn test_full_clean_refcount() {
12998        solana_logger::setup();
12999
13000        // Setup 3 scenarios which try to differentiate between pubkey1 being in an
13001        // Available slot or a Full slot which would cause a different reset behavior
13002        // when pubkey1 is cleaned and therefore cause the ref count to be incorrect
13003        // preventing a removal of that key.
13004        //
13005        // do stores with a 4mb size so only 1 store is created per slot
13006        do_full_clean_refcount(false, 4 * 1024 * 1024);
13007
13008        // do stores with a 4k size and store pubkey1 first
13009        do_full_clean_refcount(false, 4096);
13010
13011        // do stores with a 4k size and store pubkey1 2nd
13012        do_full_clean_refcount(true, 4096);
13013    }
13014
13015    #[test]
13016    fn test_accounts_clean_after_snapshot_restore_then_old_revives() {
13017        solana_logger::setup();
13018        let old_lamport = 223;
13019        let zero_lamport = 0;
13020        let no_data = 0;
13021        let dummy_lamport = 999_999;
13022        let owner = *AccountSharedData::default().owner();
13023
13024        let account = AccountSharedData::new(old_lamport, no_data, &owner);
13025        let account2 = AccountSharedData::new(old_lamport + 100_001, no_data, &owner);
13026        let account3 = AccountSharedData::new(old_lamport + 100_002, no_data, &owner);
13027        let dummy_account = AccountSharedData::new(dummy_lamport, no_data, &owner);
13028        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
13029
13030        let pubkey1 = solana_sdk::pubkey::new_rand();
13031        let pubkey2 = solana_sdk::pubkey::new_rand();
13032        let dummy_pubkey = solana_sdk::pubkey::new_rand();
13033
13034        let mut current_slot = 0;
13035        let accounts = AccountsDb::new_single_for_tests();
13036
13037        // A: Initialize AccountsDb with pubkey1 and pubkey2
13038        current_slot += 1;
13039        accounts.store_for_tests(current_slot, &[(&pubkey1, &account)]);
13040        accounts.store_for_tests(current_slot, &[(&pubkey2, &account)]);
13041        accounts.calculate_accounts_delta_hash(current_slot);
13042        accounts.add_root(current_slot);
13043
13044        // B: Test multiple updates to pubkey1 in a single slot/storage
13045        current_slot += 1;
13046        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
13047        accounts.add_root_and_flush_write_cache(current_slot - 1);
13048        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
13049        accounts.store_for_tests(current_slot, &[(&pubkey1, &account2)]);
13050        accounts.store_for_tests(current_slot, &[(&pubkey1, &account2)]);
13051        accounts.add_root_and_flush_write_cache(current_slot);
13052        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
13053        // Stores to same pubkey, same slot only count once towards the
13054        // ref count
13055        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
13056        accounts.calculate_accounts_delta_hash(current_slot);
13057
13058        // C: Yet more update to trigger lazy clean of step A
13059        current_slot += 1;
13060        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
13061        accounts.store_for_tests(current_slot, &[(&pubkey1, &account3)]);
13062        accounts.add_root_and_flush_write_cache(current_slot);
13063        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
13064        accounts.calculate_accounts_delta_hash(current_slot);
13065        accounts.add_root_and_flush_write_cache(current_slot);
13066
13067        // D: Make pubkey1 0-lamport; also triggers clean of step B
13068        current_slot += 1;
13069        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
13070        accounts.store_for_tests(current_slot, &[(&pubkey1, &zero_lamport_account)]);
13071        accounts.add_root_and_flush_write_cache(current_slot);
13072        // had to be a root to flush, but clean won't work as this test expects if it is a root
13073        // so, remove the root from alive_roots, then restore it after clean
13074        accounts
13075            .accounts_index
13076            .roots_tracker
13077            .write()
13078            .unwrap()
13079            .alive_roots
13080            .remove(&current_slot);
13081        accounts.clean_accounts_for_tests();
13082        accounts
13083            .accounts_index
13084            .roots_tracker
13085            .write()
13086            .unwrap()
13087            .alive_roots
13088            .insert(current_slot);
13089
13090        assert_eq!(
13091            // Removed one reference from the dead slot (reference only counted once
13092            // even though there were two stores to the pubkey in that slot)
13093            3, /* == 3 - 1 + 1 */
13094            accounts.ref_count_for_pubkey(&pubkey1)
13095        );
13096        accounts.calculate_accounts_delta_hash(current_slot);
13097        accounts.add_root(current_slot);
13098
13099        // E: Avoid missing bank hash error
13100        current_slot += 1;
13101        accounts.store_for_tests(current_slot, &[(&dummy_pubkey, &dummy_account)]);
13102        accounts.calculate_accounts_delta_hash(current_slot);
13103        accounts.add_root(current_slot);
13104
13105        assert_load_account(&accounts, current_slot, pubkey1, zero_lamport);
13106        assert_load_account(&accounts, current_slot, pubkey2, old_lamport);
13107        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
13108
13109        // At this point, there is no index entries for A and B
13110        // If step C and step D should be purged, snapshot restore would cause
13111        // pubkey1 to be revived as the state of step A.
13112        // So, prevent that from happening by introducing refcount
13113        ((current_slot - 1)..=current_slot).for_each(|slot| accounts.flush_root_write_cache(slot));
13114        accounts.clean_accounts_for_tests();
13115        accounts.update_accounts_hash_for_tests(
13116            current_slot,
13117            &linear_ancestors(current_slot),
13118            false,
13119            false,
13120        );
13121        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
13122        accounts.clean_accounts_for_tests();
13123
13124        info!("pubkey: {}", pubkey1);
13125        accounts.print_accounts_stats("pre_clean");
13126        assert_load_account(&accounts, current_slot, pubkey1, zero_lamport);
13127        assert_load_account(&accounts, current_slot, pubkey2, old_lamport);
13128        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
13129
13130        // F: Finally, make Step A cleanable
13131        current_slot += 1;
13132        accounts.store_for_tests(current_slot, &[(&pubkey2, &account)]);
13133        accounts.calculate_accounts_delta_hash(current_slot);
13134        accounts.add_root(current_slot);
13135
13136        // Do clean
13137        accounts.flush_root_write_cache(current_slot);
13138        accounts.clean_accounts_for_tests();
13139
13140        // 2nd clean needed to clean-up pubkey1
13141        accounts.clean_accounts_for_tests();
13142
13143        // Ensure pubkey2 is cleaned from the index finally
13144        assert_not_load_account(&accounts, current_slot, pubkey1);
13145        assert_load_account(&accounts, current_slot, pubkey2, old_lamport);
13146        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
13147    }
13148
13149    #[test]
13150    fn test_clean_stored_dead_slots_empty() {
13151        let accounts = AccountsDb::new_single_for_tests();
13152        let mut dead_slots = HashSet::new();
13153        dead_slots.insert(10);
13154        accounts.clean_stored_dead_slots(&dead_slots, None, &HashSet::default());
13155    }
13156
13157    #[test]
13158    fn test_shrink_all_slots_none() {
13159        for startup in &[false, true] {
13160            let accounts = AccountsDb::new_single_for_tests();
13161
13162            for _ in 0..10 {
13163                accounts.shrink_candidate_slots();
13164            }
13165
13166            accounts.shrink_all_slots(*startup, None);
13167        }
13168    }
13169
13170    #[test]
13171    fn test_shrink_stale_slots_processed() {
13172        solana_logger::setup();
13173
13174        for startup in &[false, true] {
13175            let accounts = AccountsDb::new_single_for_tests();
13176
13177            let pubkey_count = 100;
13178            let pubkeys: Vec<_> = (0..pubkey_count)
13179                .map(|_| solana_sdk::pubkey::new_rand())
13180                .collect();
13181
13182            let some_lamport = 223;
13183            let no_data = 0;
13184            let owner = *AccountSharedData::default().owner();
13185
13186            let account = AccountSharedData::new(some_lamport, no_data, &owner);
13187
13188            let mut current_slot = 0;
13189
13190            current_slot += 1;
13191            for pubkey in &pubkeys {
13192                accounts.store_for_tests(current_slot, &[(pubkey, &account)]);
13193            }
13194            let shrink_slot = current_slot;
13195            accounts.calculate_accounts_delta_hash(current_slot);
13196            accounts.add_root_and_flush_write_cache(current_slot);
13197
13198            current_slot += 1;
13199            let pubkey_count_after_shrink = 10;
13200            let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
13201
13202            for pubkey in updated_pubkeys {
13203                accounts.store_for_tests(current_slot, &[(pubkey, &account)]);
13204            }
13205            accounts.calculate_accounts_delta_hash(current_slot);
13206            accounts.add_root_and_flush_write_cache(current_slot);
13207
13208            accounts.clean_accounts_for_tests();
13209
13210            assert_eq!(
13211                pubkey_count,
13212                accounts.all_account_count_in_append_vec(shrink_slot)
13213            );
13214            accounts.shrink_all_slots(*startup, None);
13215            assert_eq!(
13216                pubkey_count_after_shrink,
13217                accounts.all_account_count_in_append_vec(shrink_slot)
13218            );
13219
13220            let no_ancestors = Ancestors::default();
13221
13222            let epoch_schedule = EpochSchedule::default();
13223            let rent_collector = RentCollector::default();
13224            let config = BankHashLamportsVerifyConfig::new_for_test(
13225                &no_ancestors,
13226                &epoch_schedule,
13227                &rent_collector,
13228            );
13229
13230            accounts.update_accounts_hash_for_tests(current_slot, &no_ancestors, false, false);
13231            accounts
13232                .verify_bank_hash_and_lamports(current_slot, 22300, config.clone())
13233                .unwrap();
13234
13235            let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
13236            accounts
13237                .verify_bank_hash_and_lamports(current_slot, 22300, config)
13238                .unwrap();
13239
13240            // repeating should be no-op
13241            accounts.shrink_all_slots(*startup, None);
13242            assert_eq!(
13243                pubkey_count_after_shrink,
13244                accounts.all_account_count_in_append_vec(shrink_slot)
13245            );
13246        }
13247    }
13248
13249    #[test]
13250    fn test_shrink_candidate_slots() {
13251        solana_logger::setup();
13252
13253        let mut accounts = AccountsDb::new_single_for_tests();
13254
13255        let pubkey_count = 30000;
13256        let pubkeys: Vec<_> = (0..pubkey_count)
13257            .map(|_| solana_sdk::pubkey::new_rand())
13258            .collect();
13259
13260        let some_lamport = 223;
13261        let no_data = 0;
13262        let owner = *AccountSharedData::default().owner();
13263
13264        let account = AccountSharedData::new(some_lamport, no_data, &owner);
13265
13266        let mut current_slot = 0;
13267
13268        current_slot += 1;
13269        for pubkey in &pubkeys {
13270            accounts.store_for_tests(current_slot, &[(pubkey, &account)]);
13271        }
13272        let shrink_slot = current_slot;
13273        accounts.calculate_accounts_delta_hash(current_slot);
13274        accounts.add_root_and_flush_write_cache(current_slot);
13275
13276        current_slot += 1;
13277        let pubkey_count_after_shrink = 25000;
13278        let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
13279
13280        for pubkey in updated_pubkeys {
13281            accounts.store_for_tests(current_slot, &[(pubkey, &account)]);
13282        }
13283        accounts.calculate_accounts_delta_hash(current_slot);
13284        accounts.add_root_and_flush_write_cache(current_slot);
13285        accounts.clean_accounts_for_tests();
13286
13287        assert_eq!(
13288            pubkey_count,
13289            accounts.all_account_count_in_append_vec(shrink_slot)
13290        );
13291
13292        // Only, try to shrink stale slots, nothing happens because shrink ratio
13293        // is not small enough to do a shrink
13294        // Note this shrink ratio had to change because we are WAY over-allocating append vecs when we flush the write cache at the moment.
13295        accounts.shrink_ratio = AccountShrinkThreshold::TotalSpace { shrink_ratio: 0.4 };
13296        accounts.shrink_candidate_slots();
13297        assert_eq!(
13298            pubkey_count,
13299            accounts.all_account_count_in_append_vec(shrink_slot)
13300        );
13301
13302        // Now, do full-shrink.
13303        accounts.shrink_all_slots(false, None);
13304        assert_eq!(
13305            pubkey_count_after_shrink,
13306            accounts.all_account_count_in_append_vec(shrink_slot)
13307        );
13308    }
13309
13310    #[test]
13311    fn test_select_candidates_by_total_usage_no_candidates() {
13312        // no input candidates -- none should be selected
13313        solana_logger::setup();
13314        let candidates: ShrinkCandidates = HashMap::new();
13315
13316        let (selected_candidates, next_candidates) = AccountsDb::select_candidates_by_total_usage(
13317            &candidates,
13318            DEFAULT_ACCOUNTS_SHRINK_RATIO,
13319            None,
13320        );
13321
13322        assert_eq!(0, selected_candidates.len());
13323        assert_eq!(0, next_candidates.len());
13324    }
13325
13326    #[test]
13327    fn test_select_candidates_by_total_usage_3_way_split_condition() {
13328        // three candidates, one selected for shrink, one is put back to the candidate list and one is ignored
13329        solana_logger::setup();
13330        let mut candidates: ShrinkCandidates = HashMap::new();
13331
13332        let common_store_path = Path::new("");
13333        let slot_id_1 = 12;
13334        let store_file_size = 2 * PAGE_SIZE;
13335
13336        let store1_id = 22;
13337        let store1 = Arc::new(AccountStorageEntry::new(
13338            common_store_path,
13339            slot_id_1,
13340            store1_id,
13341            store_file_size,
13342        ));
13343        store1.alive_bytes.store(0, Ordering::Release);
13344
13345        candidates.insert(slot_id_1, store1.clone());
13346
13347        let slot_id_2 = 13;
13348
13349        let store2_id = 44;
13350        let store2 = Arc::new(AccountStorageEntry::new(
13351            common_store_path,
13352            slot_id_2,
13353            store2_id,
13354            store_file_size,
13355        ));
13356
13357        // The store2's alive_ratio is 0.5: as its page aligned alive size is 1 page.
13358        let store2_alive_bytes = (PAGE_SIZE - 1) as usize;
13359        store2
13360            .alive_bytes
13361            .store(store2_alive_bytes, Ordering::Release);
13362        candidates.insert(slot_id_2, store2.clone());
13363
13364        let slot_id_3 = 14;
13365        let store3_id = 55;
13366        let entry3 = Arc::new(AccountStorageEntry::new(
13367            common_store_path,
13368            slot_id_3,
13369            store3_id,
13370            store_file_size,
13371        ));
13372
13373        // The store3's alive ratio is 1.0 as its page-aligned alive size is 2 pages
13374        let store3_alive_bytes = (PAGE_SIZE + 1) as usize;
13375        entry3
13376            .alive_bytes
13377            .store(store3_alive_bytes, Ordering::Release);
13378
13379        candidates.insert(slot_id_3, entry3);
13380
13381        // Set the target alive ratio to 0.6 so that we can just get rid of store1, the remaining two stores
13382        // alive ratio can be > the target ratio: the actual ratio is 0.75 because of 3 alive pages / 4 total pages.
13383        // The target ratio is also set to larger than store2's alive ratio: 0.5 so that it would be added
13384        // to the candidates list for next round.
13385        let target_alive_ratio = 0.6;
13386        let (selected_candidates, next_candidates) =
13387            AccountsDb::select_candidates_by_total_usage(&candidates, target_alive_ratio, None);
13388        assert_eq!(1, selected_candidates.len());
13389        assert_eq!(
13390            selected_candidates[&slot_id_1].append_vec_id(),
13391            store1.append_vec_id()
13392        );
13393        assert_eq!(1, next_candidates.len());
13394        assert_eq!(
13395            next_candidates[&slot_id_2].append_vec_id(),
13396            store2.append_vec_id()
13397        );
13398    }
13399
13400    #[test]
13401    fn test_select_candidates_by_total_usage_2_way_split_condition() {
13402        // three candidates, 2 are selected for shrink, one is ignored
13403        solana_logger::setup();
13404        let mut candidates: ShrinkCandidates = HashMap::new();
13405
13406        let common_store_path = Path::new("");
13407        let slot_id_1 = 12;
13408        let store_file_size = 2 * PAGE_SIZE;
13409
13410        let store1_id = 22;
13411        let store1 = Arc::new(AccountStorageEntry::new(
13412            common_store_path,
13413            slot_id_1,
13414            store1_id,
13415            store_file_size,
13416        ));
13417        store1.alive_bytes.store(0, Ordering::Release);
13418
13419        candidates.insert(slot_id_1, store1.clone());
13420
13421        let slot_id_2 = 13;
13422        let store2_id = 44;
13423        let store2 = Arc::new(AccountStorageEntry::new(
13424            common_store_path,
13425            slot_id_2,
13426            store2_id,
13427            store_file_size,
13428        ));
13429
13430        // The store2's alive_ratio is 0.5: as its page aligned alive size is 1 page.
13431        let store2_alive_bytes = (PAGE_SIZE - 1) as usize;
13432        store2
13433            .alive_bytes
13434            .store(store2_alive_bytes, Ordering::Release);
13435        candidates.insert(slot_id_2, store2.clone());
13436
13437        let slot_id_3 = 14;
13438        let store3_id = 55;
13439        let entry3 = Arc::new(AccountStorageEntry::new(
13440            common_store_path,
13441            slot_id_3,
13442            store3_id,
13443            store_file_size,
13444        ));
13445
13446        // The store3's alive ratio is 1.0 as its page-aligned alive size is 2 pages
13447        let store3_alive_bytes = (PAGE_SIZE + 1) as usize;
13448        entry3
13449            .alive_bytes
13450            .store(store3_alive_bytes, Ordering::Release);
13451
13452        candidates.insert(slot_id_3, entry3);
13453
13454        // Set the target ratio to default (0.8), both store1 and store2 must be selected and store3 is ignored.
13455        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
13456        let (selected_candidates, next_candidates) =
13457            AccountsDb::select_candidates_by_total_usage(&candidates, target_alive_ratio, None);
13458        assert_eq!(2, selected_candidates.len());
13459        assert_eq!(
13460            selected_candidates[&slot_id_1].append_vec_id(),
13461            store1.append_vec_id()
13462        );
13463        assert_eq!(
13464            selected_candidates[&slot_id_2].append_vec_id(),
13465            store2.append_vec_id()
13466        );
13467        assert_eq!(0, next_candidates.len());
13468    }
13469
13470    #[test]
13471    fn test_select_candidates_by_total_usage_all_clean() {
13472        // 2 candidates, they must be selected to achieve the target alive ratio
13473        solana_logger::setup();
13474        let mut candidates: ShrinkCandidates = HashMap::new();
13475
13476        let slot1 = 12;
13477        let common_store_path = Path::new("");
13478
13479        let store_file_size = 4 * PAGE_SIZE;
13480        let store1_id = 22;
13481        let store1 = Arc::new(AccountStorageEntry::new(
13482            common_store_path,
13483            slot1,
13484            store1_id,
13485            store_file_size,
13486        ));
13487
13488        // store1 has 1 page-aligned alive bytes, its alive ratio is 1/4: 0.25
13489        let store1_alive_bytes = (PAGE_SIZE - 1) as usize;
13490        store1
13491            .alive_bytes
13492            .store(store1_alive_bytes, Ordering::Release);
13493
13494        candidates.insert(slot1, store1.clone());
13495
13496        let store2_id = 44;
13497        let slot2 = 44;
13498        let store2 = Arc::new(AccountStorageEntry::new(
13499            common_store_path,
13500            slot2,
13501            store2_id,
13502            store_file_size,
13503        ));
13504
13505        // store2 has 2 page-aligned bytes, its alive ratio is 2/4: 0.5
13506        let store2_alive_bytes = (PAGE_SIZE + 1) as usize;
13507        store2
13508            .alive_bytes
13509            .store(store2_alive_bytes, Ordering::Release);
13510
13511        candidates.insert(slot2, store2.clone());
13512
13513        for newest_ancient_slot in [None, Some(slot1), Some(slot2)] {
13514            // Set the target ratio to default (0.8), both stores from the two different slots must be selected.
13515            let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
13516            let (selected_candidates, next_candidates) =
13517                AccountsDb::select_candidates_by_total_usage(
13518                    &candidates,
13519                    target_alive_ratio,
13520                    newest_ancient_slot,
13521                );
13522            assert_eq!(
13523                if newest_ancient_slot == Some(slot1) {
13524                    1
13525                } else if newest_ancient_slot == Some(slot2) {
13526                    0
13527                } else {
13528                    2
13529                },
13530                selected_candidates.len()
13531            );
13532            assert_eq!(
13533                newest_ancient_slot.is_none(),
13534                selected_candidates.contains(&slot1)
13535            );
13536
13537            if newest_ancient_slot.is_none() {
13538                assert_eq!(
13539                    selected_candidates[&slot1].append_vec_id(),
13540                    store1.append_vec_id()
13541                );
13542            }
13543            if newest_ancient_slot != Some(slot2) {
13544                assert!(selected_candidates.contains(&slot2));
13545
13546                assert_eq!(
13547                    selected_candidates[&slot2].append_vec_id(),
13548                    store2.append_vec_id()
13549                );
13550            }
13551            assert_eq!(0, next_candidates.len());
13552        }
13553    }
13554
13555    const UPSERT_POPULATE_RECLAIMS: UpsertReclaim = UpsertReclaim::PopulateReclaims;
13556
13557    // returns the rooted entries and the storage ref count
13558    fn roots_and_ref_count<T: IndexValue>(
13559        index: &AccountsIndex<T>,
13560        locked_account_entry: &ReadAccountMapEntry<T>,
13561        max_inclusive: Option<Slot>,
13562    ) -> (SlotList<T>, RefCount) {
13563        (
13564            index.get_rooted_entries(locked_account_entry.slot_list(), max_inclusive),
13565            locked_account_entry.ref_count(),
13566        )
13567    }
13568
13569    #[test]
13570    fn test_delete_dependencies() {
13571        solana_logger::setup();
13572        let accounts_index = AccountsIndex::default_for_tests();
13573        let key0 = Pubkey::new_from_array([0u8; 32]);
13574        let key1 = Pubkey::new_from_array([1u8; 32]);
13575        let key2 = Pubkey::new_from_array([2u8; 32]);
13576        let info0 = AccountInfo::new(StorageLocation::AppendVec(0, 0), 0, 0);
13577        let info1 = AccountInfo::new(StorageLocation::AppendVec(1, 0), 0, 0);
13578        let info2 = AccountInfo::new(StorageLocation::AppendVec(2, 0), 0, 0);
13579        let info3 = AccountInfo::new(StorageLocation::AppendVec(3, 0), 0, 0);
13580        let mut reclaims = vec![];
13581        accounts_index.upsert(
13582            0,
13583            0,
13584            &key0,
13585            &AccountSharedData::default(),
13586            &AccountSecondaryIndexes::default(),
13587            info0,
13588            &mut reclaims,
13589            UPSERT_POPULATE_RECLAIMS,
13590        );
13591        accounts_index.upsert(
13592            1,
13593            1,
13594            &key0,
13595            &AccountSharedData::default(),
13596            &AccountSecondaryIndexes::default(),
13597            info1,
13598            &mut reclaims,
13599            UPSERT_POPULATE_RECLAIMS,
13600        );
13601        accounts_index.upsert(
13602            1,
13603            1,
13604            &key1,
13605            &AccountSharedData::default(),
13606            &AccountSecondaryIndexes::default(),
13607            info1,
13608            &mut reclaims,
13609            UPSERT_POPULATE_RECLAIMS,
13610        );
13611        accounts_index.upsert(
13612            2,
13613            2,
13614            &key1,
13615            &AccountSharedData::default(),
13616            &AccountSecondaryIndexes::default(),
13617            info2,
13618            &mut reclaims,
13619            UPSERT_POPULATE_RECLAIMS,
13620        );
13621        accounts_index.upsert(
13622            2,
13623            2,
13624            &key2,
13625            &AccountSharedData::default(),
13626            &AccountSecondaryIndexes::default(),
13627            info2,
13628            &mut reclaims,
13629            UPSERT_POPULATE_RECLAIMS,
13630        );
13631        accounts_index.upsert(
13632            3,
13633            3,
13634            &key2,
13635            &AccountSharedData::default(),
13636            &AccountSecondaryIndexes::default(),
13637            info3,
13638            &mut reclaims,
13639            UPSERT_POPULATE_RECLAIMS,
13640        );
13641        accounts_index.add_root(0);
13642        accounts_index.add_root(1);
13643        accounts_index.add_root(2);
13644        accounts_index.add_root(3);
13645        let mut purges = HashMap::new();
13646        let (key0_entry, _) = accounts_index.get_for_tests(&key0, None, None).unwrap();
13647        purges.insert(
13648            key0,
13649            roots_and_ref_count(&accounts_index, &key0_entry, None),
13650        );
13651        let (key1_entry, _) = accounts_index.get_for_tests(&key1, None, None).unwrap();
13652        purges.insert(
13653            key1,
13654            roots_and_ref_count(&accounts_index, &key1_entry, None),
13655        );
13656        let (key2_entry, _) = accounts_index.get_for_tests(&key2, None, None).unwrap();
13657        purges.insert(
13658            key2,
13659            roots_and_ref_count(&accounts_index, &key2_entry, None),
13660        );
13661        for (key, (list, ref_count)) in &purges {
13662            info!(" purge {} ref_count {} =>", key, ref_count);
13663            for x in list {
13664                info!("  {:?}", x);
13665            }
13666        }
13667
13668        let mut store_counts = HashMap::new();
13669        store_counts.insert(0, (0, HashSet::from_iter(vec![key0])));
13670        store_counts.insert(1, (0, HashSet::from_iter(vec![key0, key1])));
13671        store_counts.insert(2, (0, HashSet::from_iter(vec![key1, key2])));
13672        store_counts.insert(3, (1, HashSet::from_iter(vec![key2])));
13673        AccountsDb::calc_delete_dependencies(&purges, &mut store_counts, None);
13674        let mut stores: Vec<_> = store_counts.keys().cloned().collect();
13675        stores.sort_unstable();
13676        for store in &stores {
13677            info!(
13678                "store: {:?} : {:?}",
13679                store,
13680                store_counts.get(store).unwrap()
13681            );
13682        }
13683        for x in 0..3 {
13684            // if the store count doesn't exist for this id, then it is implied to be > 0
13685            assert!(store_counts
13686                .get(&x)
13687                .map(|entry| entry.0 >= 1)
13688                .unwrap_or(true));
13689        }
13690    }
13691
13692    #[test]
13693    fn test_account_balance_for_capitalization_sysvar() {
13694        let normal_sysvar = solana_sdk::account::create_account_for_test(
13695            &solana_sdk::slot_history::SlotHistory::default(),
13696        );
13697        assert_eq!(normal_sysvar.lamports(), 1);
13698    }
13699
13700    #[test]
13701    fn test_account_balance_for_capitalization_native_program() {
13702        let normal_native_program =
13703            solana_sdk::native_loader::create_loadable_account_for_test("foo");
13704        assert_eq!(normal_native_program.lamports(), 1);
13705    }
13706
13707    #[test]
13708    fn test_checked_sum_for_capitalization_normal() {
13709        assert_eq!(
13710            AccountsDb::checked_sum_for_capitalization(vec![1, 2].into_iter()),
13711            3
13712        );
13713    }
13714
13715    #[test]
13716    #[should_panic(expected = "overflow is detected while summing capitalization")]
13717    fn test_checked_sum_for_capitalization_overflow() {
13718        assert_eq!(
13719            AccountsDb::checked_sum_for_capitalization(vec![1, u64::max_value()].into_iter()),
13720            3
13721        );
13722    }
13723
13724    #[test]
13725    fn test_store_overhead() {
13726        solana_logger::setup();
13727        let accounts = AccountsDb::new_single_for_tests();
13728        let account = AccountSharedData::default();
13729        let pubkey = solana_sdk::pubkey::new_rand();
13730        accounts.store_for_tests(0, &[(&pubkey, &account)]);
13731        accounts.add_root_and_flush_write_cache(0);
13732        let store = accounts.storage.get_slot_storage_entry(0).unwrap();
13733        let total_len = store.accounts.len();
13734        info!("total: {}", total_len);
13735        assert_eq!(total_len, STORE_META_OVERHEAD);
13736    }
13737
13738    #[test]
13739    fn test_store_clean_after_shrink() {
13740        solana_logger::setup();
13741        let accounts = AccountsDb::new_with_config_for_tests(
13742            vec![],
13743            &ClusterType::Development,
13744            AccountSecondaryIndexes::default(),
13745            AccountShrinkThreshold::default(),
13746        );
13747
13748        let account = AccountSharedData::new(1, 16 * 4096, &Pubkey::default());
13749        let pubkey1 = solana_sdk::pubkey::new_rand();
13750        accounts.store_cached((0, &[(&pubkey1, &account)][..]), None);
13751
13752        let pubkey2 = solana_sdk::pubkey::new_rand();
13753        accounts.store_cached((0, &[(&pubkey2, &account)][..]), None);
13754
13755        let zero_account = AccountSharedData::new(0, 1, &Pubkey::default());
13756        accounts.store_cached((1, &[(&pubkey1, &zero_account)][..]), None);
13757
13758        // Add root 0 and flush separately
13759        accounts.calculate_accounts_delta_hash(0);
13760        accounts.add_root(0);
13761        accounts.flush_accounts_cache(true, None);
13762
13763        // clear out the dirty keys
13764        accounts.clean_accounts_for_tests();
13765
13766        // flush 1
13767        accounts.calculate_accounts_delta_hash(1);
13768        accounts.add_root(1);
13769        accounts.flush_accounts_cache(true, None);
13770
13771        accounts.print_accounts_stats("pre-clean");
13772
13773        // clean to remove pubkey1 from 0,
13774        // shrink to shrink pubkey1 from 0
13775        // then another clean to remove pubkey1 from slot 1
13776        accounts.clean_accounts_for_tests();
13777
13778        accounts.shrink_candidate_slots();
13779
13780        accounts.clean_accounts_for_tests();
13781
13782        accounts.print_accounts_stats("post-clean");
13783        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
13784    }
13785
13786    #[test]
13787    fn test_store_reuse() {
13788        solana_logger::setup();
13789        let accounts = AccountsDb::new_sized_caching(vec![], 4096);
13790
13791        let size = 100;
13792        let num_accounts: usize = 100;
13793        let mut keys = Vec::new();
13794        for i in 0..num_accounts {
13795            let account = AccountSharedData::new((i + 1) as u64, size, &Pubkey::default());
13796            let pubkey = solana_sdk::pubkey::new_rand();
13797            accounts.store_cached((0 as Slot, &[(&pubkey, &account)][..]), None);
13798            keys.push(pubkey);
13799        }
13800        // get delta hash to feed these accounts to clean
13801        accounts.calculate_accounts_delta_hash(0);
13802        accounts.add_root(0);
13803        // we have to flush just slot 0
13804        // if we slot 0 and 1 together, then they are cleaned and slot 0 doesn't contain the accounts
13805        // this test wants to clean and then allow us to shrink
13806        accounts.flush_accounts_cache(true, None);
13807
13808        for (i, key) in keys[1..].iter().enumerate() {
13809            let account =
13810                AccountSharedData::new((1 + i + num_accounts) as u64, size, &Pubkey::default());
13811            accounts.store_cached((1 as Slot, &[(key, &account)][..]), None);
13812        }
13813        accounts.calculate_accounts_delta_hash(1);
13814        accounts.add_root(1);
13815        accounts.flush_accounts_cache(true, None);
13816        accounts.clean_accounts_for_tests();
13817        accounts.shrink_all_slots(false, None);
13818
13819        // Clean again to flush the dirty stores
13820        // and allow them to be recycled in the next step
13821        accounts.clean_accounts_for_tests();
13822        accounts.print_accounts_stats("post-shrink");
13823        let num_stores = accounts.recycle_stores.read().unwrap().entry_count();
13824        assert!(num_stores > 0);
13825
13826        let mut account_refs = Vec::new();
13827        let num_to_store = 20;
13828        for (i, key) in keys[..num_to_store].iter().enumerate() {
13829            let account = AccountSharedData::new(
13830                (1 + i + 2 * num_accounts) as u64,
13831                i + 20,
13832                &Pubkey::default(),
13833            );
13834            accounts.store_uncached(2, &[(key, &account)]);
13835            account_refs.push(account);
13836        }
13837        assert!(accounts.recycle_stores.read().unwrap().entry_count() < num_stores);
13838
13839        accounts.print_accounts_stats("post-store");
13840
13841        let mut ancestors = Ancestors::default();
13842        ancestors.insert(1, 0);
13843        ancestors.insert(2, 1);
13844        for (key, account_ref) in keys[..num_to_store].iter().zip(account_refs) {
13845            assert_eq!(
13846                accounts.load_without_fixed_root(&ancestors, key).unwrap().0,
13847                account_ref
13848            );
13849        }
13850    }
13851
13852    #[test]
13853    #[should_panic(expected = "We've run out of storage ids!")]
13854    fn test_wrapping_append_vec_id() {
13855        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13856
13857        let zero_lamport_account =
13858            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13859
13860        // set 'next' id to the max possible value
13861        db.next_id.store(AppendVecId::MAX, Ordering::Release);
13862        let slots = 3;
13863        let keys = (0..slots).map(|_| Pubkey::new_unique()).collect::<Vec<_>>();
13864        // write unique keys to successive slots
13865        keys.iter().enumerate().for_each(|(slot, key)| {
13866            let slot = slot as Slot;
13867            db.store_for_tests(slot, &[(key, &zero_lamport_account)]);
13868            db.calculate_accounts_delta_hash(slot);
13869            db.add_root_and_flush_write_cache(slot);
13870        });
13871        assert_eq!(slots - 1, db.next_id.load(Ordering::Acquire));
13872        let ancestors = Ancestors::default();
13873        keys.iter().for_each(|key| {
13874            assert!(db.load_without_fixed_root(&ancestors, key).is_some());
13875        });
13876    }
13877
13878    #[test]
13879    #[should_panic(expected = "We've run out of storage ids!")]
13880    fn test_reuse_append_vec_id() {
13881        solana_logger::setup();
13882        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13883
13884        let zero_lamport_account =
13885            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13886
13887        // set 'next' id to the max possible value
13888        db.next_id.store(AppendVecId::MAX, Ordering::Release);
13889        let slots = 3;
13890        let keys = (0..slots).map(|_| Pubkey::new_unique()).collect::<Vec<_>>();
13891        // write unique keys to successive slots
13892        keys.iter().enumerate().for_each(|(slot, key)| {
13893            let slot = slot as Slot;
13894            db.store_for_tests(slot, &[(key, &zero_lamport_account)]);
13895            db.calculate_accounts_delta_hash(slot);
13896            db.add_root_and_flush_write_cache(slot);
13897            // reset next_id to what it was previously to cause us to re-use the same id
13898            db.next_id.store(AppendVecId::MAX, Ordering::Release);
13899        });
13900        let ancestors = Ancestors::default();
13901        keys.iter().for_each(|key| {
13902            assert!(db.load_without_fixed_root(&ancestors, key).is_some());
13903        });
13904    }
13905
13906    #[test]
13907    fn test_zero_lamport_new_root_not_cleaned() {
13908        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13909        let account_key = Pubkey::new_unique();
13910        let zero_lamport_account =
13911            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13912
13913        // Store zero lamport account into slots 0 and 1, root both slots
13914        db.store_for_tests(0, &[(&account_key, &zero_lamport_account)]);
13915        db.store_for_tests(1, &[(&account_key, &zero_lamport_account)]);
13916        db.calculate_accounts_delta_hash(0);
13917        db.add_root_and_flush_write_cache(0);
13918        db.calculate_accounts_delta_hash(1);
13919        db.add_root_and_flush_write_cache(1);
13920
13921        // Only clean zero lamport accounts up to slot 0
13922        db.clean_accounts(Some(0), false, None);
13923
13924        // Should still be able to find zero lamport account in slot 1
13925        assert_eq!(
13926            db.load_without_fixed_root(&Ancestors::default(), &account_key),
13927            Some((zero_lamport_account, 1))
13928        );
13929    }
13930
13931    #[test]
13932    fn test_store_load_cached() {
13933        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13934        let key = Pubkey::default();
13935        let account0 = AccountSharedData::new(1, 0, &key);
13936        let slot = 0;
13937        db.store_cached((slot, &[(&key, &account0)][..]), None);
13938
13939        // Load with no ancestors and no root will return nothing
13940        assert!(db
13941            .load_without_fixed_root(&Ancestors::default(), &key)
13942            .is_none());
13943
13944        // Load with ancestors not equal to `slot` will return nothing
13945        let ancestors = vec![(slot + 1, 1)].into_iter().collect();
13946        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
13947
13948        // Load with ancestors equal to `slot` will return the account
13949        let ancestors = vec![(slot, 1)].into_iter().collect();
13950        assert_eq!(
13951            db.load_without_fixed_root(&ancestors, &key),
13952            Some((account0.clone(), slot))
13953        );
13954
13955        // Adding root will return the account even without ancestors
13956        db.add_root(slot);
13957        assert_eq!(
13958            db.load_without_fixed_root(&Ancestors::default(), &key),
13959            Some((account0, slot))
13960        );
13961    }
13962
13963    #[test]
13964    fn test_store_flush_load_cached() {
13965        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13966        let key = Pubkey::default();
13967        let account0 = AccountSharedData::new(1, 0, &key);
13968        let slot = 0;
13969        db.store_cached((slot, &[(&key, &account0)][..]), None);
13970        db.mark_slot_frozen(slot);
13971
13972        // No root was added yet, requires an ancestor to find
13973        // the account
13974        db.flush_accounts_cache(true, None);
13975        let ancestors = vec![(slot, 1)].into_iter().collect();
13976        assert_eq!(
13977            db.load_without_fixed_root(&ancestors, &key),
13978            Some((account0.clone(), slot))
13979        );
13980
13981        // Add root then flush
13982        db.add_root(slot);
13983        db.flush_accounts_cache(true, None);
13984        assert_eq!(
13985            db.load_without_fixed_root(&Ancestors::default(), &key),
13986            Some((account0, slot))
13987        );
13988    }
13989
13990    #[test]
13991    fn test_flush_accounts_cache() {
13992        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13993        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
13994
13995        let unrooted_slot = 4;
13996        let root5 = 5;
13997        let root6 = 6;
13998        let unrooted_key = solana_sdk::pubkey::new_rand();
13999        let key5 = solana_sdk::pubkey::new_rand();
14000        let key6 = solana_sdk::pubkey::new_rand();
14001        db.store_cached((unrooted_slot, &[(&unrooted_key, &account0)][..]), None);
14002        db.store_cached((root5, &[(&key5, &account0)][..]), None);
14003        db.store_cached((root6, &[(&key6, &account0)][..]), None);
14004        for slot in &[unrooted_slot, root5, root6] {
14005            db.mark_slot_frozen(*slot);
14006        }
14007        db.add_root(root5);
14008        db.add_root(root6);
14009
14010        // Unrooted slot should be able to be fetched before the flush
14011        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
14012        assert_eq!(
14013            db.load_without_fixed_root(&ancestors, &unrooted_key),
14014            Some((account0.clone(), unrooted_slot))
14015        );
14016        db.flush_accounts_cache(true, None);
14017
14018        // After the flush, the unrooted slot is still in the cache
14019        assert!(db
14020            .load_without_fixed_root(&ancestors, &unrooted_key)
14021            .is_some());
14022        assert!(db
14023            .accounts_index
14024            .get_account_read_entry(&unrooted_key)
14025            .is_some());
14026        assert_eq!(db.accounts_cache.num_slots(), 1);
14027        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_some());
14028        assert_eq!(
14029            db.load_without_fixed_root(&Ancestors::default(), &key5),
14030            Some((account0.clone(), root5))
14031        );
14032        assert_eq!(
14033            db.load_without_fixed_root(&Ancestors::default(), &key6),
14034            Some((account0, root6))
14035        );
14036    }
14037
14038    fn max_cache_slots() -> usize {
14039        // this used to be the limiting factor - used here to facilitate tests.
14040        200
14041    }
14042
14043    #[test]
14044    fn test_flush_accounts_cache_if_needed() {
14045        run_test_flush_accounts_cache_if_needed(0, 2 * max_cache_slots());
14046        run_test_flush_accounts_cache_if_needed(2 * max_cache_slots(), 0);
14047        run_test_flush_accounts_cache_if_needed(max_cache_slots() - 1, 0);
14048        run_test_flush_accounts_cache_if_needed(0, max_cache_slots() - 1);
14049        run_test_flush_accounts_cache_if_needed(max_cache_slots(), 0);
14050        run_test_flush_accounts_cache_if_needed(0, max_cache_slots());
14051        run_test_flush_accounts_cache_if_needed(2 * max_cache_slots(), 2 * max_cache_slots());
14052        run_test_flush_accounts_cache_if_needed(max_cache_slots() - 1, max_cache_slots() - 1);
14053        run_test_flush_accounts_cache_if_needed(max_cache_slots(), max_cache_slots());
14054    }
14055
14056    fn run_test_flush_accounts_cache_if_needed(num_roots: usize, num_unrooted: usize) {
14057        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
14058        db.write_cache_limit_bytes = Some(max_cache_slots() as u64);
14059        let space = 1; // # data bytes per account. write cache counts data len
14060        let account0 = AccountSharedData::new(1, space, &Pubkey::default());
14061        let mut keys = vec![];
14062        let num_slots = 2 * max_cache_slots();
14063        for i in 0..num_roots + num_unrooted {
14064            let key = Pubkey::new_unique();
14065            db.store_cached((i as Slot, &[(&key, &account0)][..]), None);
14066            keys.push(key);
14067            db.mark_slot_frozen(i as Slot);
14068            if i < num_roots {
14069                db.add_root(i as Slot);
14070            }
14071        }
14072
14073        db.flush_accounts_cache(false, None);
14074
14075        let total_slots = num_roots + num_unrooted;
14076        // If there's <= the max size, then nothing will be flushed from the slot
14077        if total_slots <= max_cache_slots() {
14078            assert_eq!(db.accounts_cache.num_slots(), total_slots);
14079        } else {
14080            // Otherwise, all the roots are flushed, and only at most max_cache_slots()
14081            // of the unrooted slots are kept in the cache
14082            let expected_size = std::cmp::min(num_unrooted, max_cache_slots());
14083            if expected_size > 0 {
14084                // +1: slot is 1-based. slot 1 has 1 byte of data
14085                for unrooted_slot in (total_slots - expected_size + 1)..total_slots {
14086                    assert!(
14087                        db.accounts_cache
14088                            .slot_cache(unrooted_slot as Slot)
14089                            .is_some(),
14090                        "unrooted_slot: {unrooted_slot}, total_slots: {total_slots}, expected_size: {expected_size}"
14091                    );
14092                }
14093            }
14094        }
14095
14096        // Should still be able to fetch all the accounts after flush
14097        for (slot, key) in (0..num_slots as Slot).zip(keys) {
14098            let ancestors = if slot < num_roots as Slot {
14099                Ancestors::default()
14100            } else {
14101                vec![(slot, 1)].into_iter().collect()
14102            };
14103            assert_eq!(
14104                db.load_without_fixed_root(&ancestors, &key),
14105                Some((account0.clone(), slot))
14106            );
14107        }
14108    }
14109
14110    #[test]
14111    fn test_read_only_accounts_cache() {
14112        let db = Arc::new(AccountsDb::new_with_config_for_tests(
14113            Vec::new(),
14114            &ClusterType::Development,
14115            AccountSecondaryIndexes::default(),
14116            AccountShrinkThreshold::default(),
14117        ));
14118
14119        let account_key = Pubkey::new_unique();
14120        let zero_lamport_account =
14121            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
14122        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14123        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
14124        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
14125
14126        db.add_root(0);
14127        db.add_root(1);
14128        db.clean_accounts_for_tests();
14129        db.flush_accounts_cache(true, None);
14130        db.clean_accounts_for_tests();
14131        db.add_root(2);
14132
14133        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
14134        let account = db
14135            .load_with_fixed_root(&Ancestors::default(), &account_key)
14136            .map(|(account, _)| account)
14137            .unwrap();
14138        assert_eq!(account.lamports(), 1);
14139        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
14140        let account = db
14141            .load_with_fixed_root(&Ancestors::default(), &account_key)
14142            .map(|(account, _)| account)
14143            .unwrap();
14144        assert_eq!(account.lamports(), 1);
14145        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
14146        db.store_cached((2, &[(&account_key, &zero_lamport_account)][..]), None);
14147        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
14148        let account = db
14149            .load_with_fixed_root(&Ancestors::default(), &account_key)
14150            .map(|(account, _)| account);
14151        assert!(account.is_none());
14152        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
14153    }
14154
14155    #[test]
14156    fn test_account_matches_owners() {
14157        let db = Arc::new(AccountsDb::new_with_config_for_tests(
14158            Vec::new(),
14159            &ClusterType::Development,
14160            AccountSecondaryIndexes::default(),
14161            AccountShrinkThreshold::default(),
14162        ));
14163
14164        let owners: Vec<Pubkey> = (0..2).map(|_| Pubkey::new_unique()).collect();
14165        let owners_refs: Vec<&Pubkey> = owners.iter().collect();
14166
14167        let account1_key = Pubkey::new_unique();
14168        let account1 = AccountSharedData::new(321, 10, &owners[0]);
14169
14170        let account2_key = Pubkey::new_unique();
14171        let account2 = AccountSharedData::new(1, 1, &owners[1]);
14172
14173        let account3_key = Pubkey::new_unique();
14174        let account3 = AccountSharedData::new(1, 1, &Pubkey::new_unique());
14175
14176        // Account with 0 lamports
14177        let account4_key = Pubkey::new_unique();
14178        let account4 = AccountSharedData::new(0, 1, &owners[1]);
14179
14180        db.store_cached((0, &[(&account1_key, &account1)][..]), None);
14181        db.store_cached((1, &[(&account2_key, &account2)][..]), None);
14182        db.store_cached((2, &[(&account3_key, &account3)][..]), None);
14183        db.store_cached((3, &[(&account4_key, &account4)][..]), None);
14184
14185        db.add_root(0);
14186        db.add_root(1);
14187        db.add_root(2);
14188        db.add_root(3);
14189
14190        // Flush the cache so that the account meta will be read from the storage
14191        db.flush_accounts_cache(true, None);
14192        db.clean_accounts_for_tests();
14193
14194        assert_eq!(
14195            db.account_matches_owners(&Ancestors::default(), &account1_key, &owners_refs),
14196            Ok(0)
14197        );
14198        assert_eq!(
14199            db.account_matches_owners(&Ancestors::default(), &account2_key, &owners_refs),
14200            Ok(1)
14201        );
14202        assert_eq!(
14203            db.account_matches_owners(&Ancestors::default(), &account3_key, &owners_refs),
14204            Err(MatchAccountOwnerError::NoMatch)
14205        );
14206        assert_eq!(
14207            db.account_matches_owners(&Ancestors::default(), &account4_key, &owners_refs),
14208            Err(MatchAccountOwnerError::NoMatch)
14209        );
14210        assert_eq!(
14211            db.account_matches_owners(&Ancestors::default(), &Pubkey::new_unique(), &owners_refs),
14212            Err(MatchAccountOwnerError::UnableToLoad)
14213        );
14214
14215        // Flush the cache and load account1 (so that it's in the cache)
14216        db.flush_accounts_cache(true, None);
14217        db.clean_accounts_for_tests();
14218        let _ = db
14219            .do_load(
14220                &Ancestors::default(),
14221                &account1_key,
14222                Some(0),
14223                LoadHint::Unspecified,
14224                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
14225            )
14226            .unwrap();
14227
14228        assert_eq!(
14229            db.account_matches_owners(&Ancestors::default(), &account1_key, &owners_refs),
14230            Ok(0)
14231        );
14232        assert_eq!(
14233            db.account_matches_owners(&Ancestors::default(), &account2_key, &owners_refs),
14234            Ok(1)
14235        );
14236        assert_eq!(
14237            db.account_matches_owners(&Ancestors::default(), &account3_key, &owners_refs),
14238            Err(MatchAccountOwnerError::NoMatch)
14239        );
14240        assert_eq!(
14241            db.account_matches_owners(&Ancestors::default(), &account4_key, &owners_refs),
14242            Err(MatchAccountOwnerError::NoMatch)
14243        );
14244        assert_eq!(
14245            db.account_matches_owners(&Ancestors::default(), &Pubkey::new_unique(), &owners_refs),
14246            Err(MatchAccountOwnerError::UnableToLoad)
14247        );
14248    }
14249
14250    /// a test that will accept either answer
14251    const LOAD_ZERO_LAMPORTS_ANY_TESTS: LoadZeroLamports = LoadZeroLamports::None;
14252
14253    #[test]
14254    fn test_flush_cache_clean() {
14255        let db = Arc::new(AccountsDb::new_with_config_for_tests(
14256            Vec::new(),
14257            &ClusterType::Development,
14258            AccountSecondaryIndexes::default(),
14259            AccountShrinkThreshold::default(),
14260        ));
14261
14262        let account_key = Pubkey::new_unique();
14263        let zero_lamport_account =
14264            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
14265        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14266        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
14267        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
14268
14269        db.add_root(0);
14270        db.add_root(1);
14271
14272        // Clean should not remove anything yet as nothing has been flushed
14273        db.clean_accounts_for_tests();
14274        let account = db
14275            .do_load(
14276                &Ancestors::default(),
14277                &account_key,
14278                Some(0),
14279                LoadHint::Unspecified,
14280                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
14281            )
14282            .unwrap();
14283        assert_eq!(account.0.lamports(), 0);
14284        // since this item is in the cache, it should not be in the read only cache
14285        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
14286
14287        // Flush, then clean again. Should not need another root to initiate the cleaning
14288        // because `accounts_index.uncleaned_roots` should be correct
14289        db.flush_accounts_cache(true, None);
14290        db.clean_accounts_for_tests();
14291        assert!(db
14292            .do_load(
14293                &Ancestors::default(),
14294                &account_key,
14295                Some(0),
14296                LoadHint::Unspecified,
14297                LOAD_ZERO_LAMPORTS_ANY_TESTS
14298            )
14299            .is_none());
14300    }
14301
14302    #[test]
14303    fn test_flush_cache_dont_clean_zero_lamport_account() {
14304        let db = Arc::new(AccountsDb::new_with_config_for_tests(
14305            Vec::new(),
14306            &ClusterType::Development,
14307            AccountSecondaryIndexes::default(),
14308            AccountShrinkThreshold::default(),
14309        ));
14310
14311        let zero_lamport_account_key = Pubkey::new_unique();
14312        let other_account_key = Pubkey::new_unique();
14313
14314        let original_lamports = 1;
14315        let slot0_account =
14316            AccountSharedData::new(original_lamports, 1, AccountSharedData::default().owner());
14317        let zero_lamport_account =
14318            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
14319
14320        // Store into slot 0, and then flush the slot to storage
14321        db.store_cached(
14322            (0, &[(&zero_lamport_account_key, &slot0_account)][..]),
14323            None,
14324        );
14325        // Second key keeps other lamport account entry for slot 0 alive,
14326        // preventing clean of the zero_lamport_account in slot 1.
14327        db.store_cached((0, &[(&other_account_key, &slot0_account)][..]), None);
14328        db.add_root(0);
14329        db.flush_accounts_cache(true, None);
14330        assert!(db.storage.get_slot_storage_entry(0).is_some());
14331
14332        // Store into slot 1, a dummy slot that will be dead and purged before flush
14333        db.store_cached(
14334            (1, &[(&zero_lamport_account_key, &zero_lamport_account)][..]),
14335            None,
14336        );
14337
14338        // Store into slot 2, which makes all updates from slot 1 outdated.
14339        // This means slot 1 is a dead slot. Later, slot 1 will be cleaned/purged
14340        // before it even reaches storage, but this purge of slot 1should not affect
14341        // the refcount of `zero_lamport_account_key` because cached keys do not bump
14342        // the refcount in the index. This means clean should *not* remove
14343        // `zero_lamport_account_key` from slot 2
14344        db.store_cached(
14345            (2, &[(&zero_lamport_account_key, &zero_lamport_account)][..]),
14346            None,
14347        );
14348        db.add_root(1);
14349        db.add_root(2);
14350
14351        // Flush, then clean. Should not need another root to initiate the cleaning
14352        // because `accounts_index.uncleaned_roots` should be correct
14353        db.flush_accounts_cache(true, None);
14354        db.clean_accounts_for_tests();
14355
14356        // The `zero_lamport_account_key` is still alive in slot 1, so refcount for the
14357        // pubkey should be 2
14358        assert_eq!(
14359            db.accounts_index
14360                .ref_count_from_storage(&zero_lamport_account_key),
14361            2
14362        );
14363        assert_eq!(
14364            db.accounts_index.ref_count_from_storage(&other_account_key),
14365            1
14366        );
14367
14368        // The zero-lamport account in slot 2 should not be purged yet, because the
14369        // entry in slot 1 is blocking cleanup of the zero-lamport account.
14370        let max_root = None;
14371        // Fine to simulate a transaction load since we are not doing any out of band
14372        // removals, only using clean_accounts
14373        let load_hint = LoadHint::FixedMaxRoot;
14374        assert_eq!(
14375            db.do_load(
14376                &Ancestors::default(),
14377                &zero_lamport_account_key,
14378                max_root,
14379                load_hint,
14380                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
14381            )
14382            .unwrap()
14383            .0
14384            .lamports(),
14385            0
14386        );
14387    }
14388
14389    struct ScanTracker {
14390        t_scan: JoinHandle<()>,
14391        exit: Arc<AtomicBool>,
14392    }
14393
14394    impl ScanTracker {
14395        fn exit(self) -> thread::Result<()> {
14396            self.exit.store(true, Ordering::Relaxed);
14397            self.t_scan.join()
14398        }
14399    }
14400
14401    fn setup_scan(
14402        db: Arc<AccountsDb>,
14403        scan_ancestors: Arc<Ancestors>,
14404        bank_id: BankId,
14405        stall_key: Pubkey,
14406    ) -> ScanTracker {
14407        let exit = Arc::new(AtomicBool::new(false));
14408        let exit_ = exit.clone();
14409        let ready = Arc::new(AtomicBool::new(false));
14410        let ready_ = ready.clone();
14411
14412        let t_scan = Builder::new()
14413            .name("scan".to_string())
14414            .spawn(move || {
14415                db.scan_accounts(
14416                    &scan_ancestors,
14417                    bank_id,
14418                    |maybe_account| {
14419                        ready_.store(true, Ordering::Relaxed);
14420                        if let Some((pubkey, _, _)) = maybe_account {
14421                            if *pubkey == stall_key {
14422                                loop {
14423                                    if exit_.load(Ordering::Relaxed) {
14424                                        break;
14425                                    } else {
14426                                        sleep(Duration::from_millis(10));
14427                                    }
14428                                }
14429                            }
14430                        }
14431                    },
14432                    &ScanConfig::default(),
14433                )
14434                .unwrap();
14435            })
14436            .unwrap();
14437
14438        // Wait for scan to start
14439        while !ready.load(Ordering::Relaxed) {
14440            sleep(Duration::from_millis(10));
14441        }
14442
14443        ScanTracker { t_scan, exit }
14444    }
14445
14446    #[test]
14447    fn test_scan_flush_accounts_cache_then_clean_drop() {
14448        let db = Arc::new(AccountsDb::new_with_config_for_tests(
14449            Vec::new(),
14450            &ClusterType::Development,
14451            AccountSecondaryIndexes::default(),
14452            AccountShrinkThreshold::default(),
14453        ));
14454        let account_key = Pubkey::new_unique();
14455        let account_key2 = Pubkey::new_unique();
14456        let zero_lamport_account =
14457            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
14458        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14459        let slot2_account = AccountSharedData::new(2, 1, AccountSharedData::default().owner());
14460
14461        /*
14462            Store zero lamport account into slots 0, 1, 2 where
14463            root slots are 0, 2, and slot 1 is unrooted.
14464                                    0 (root)
14465                                /        \
14466                              1            2 (root)
14467        */
14468        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
14469        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
14470        // Fodder for the scan so that the lock on `account_key` is not held
14471        db.store_cached((1, &[(&account_key2, &slot1_account)][..]), None);
14472        db.store_cached((2, &[(&account_key, &slot2_account)][..]), None);
14473        db.calculate_accounts_delta_hash(0);
14474
14475        let max_scan_root = 0;
14476        db.add_root(max_scan_root);
14477        let scan_ancestors: Arc<Ancestors> = Arc::new(vec![(0, 1), (1, 1)].into_iter().collect());
14478        let bank_id = 0;
14479        let scan_tracker = setup_scan(db.clone(), scan_ancestors.clone(), bank_id, account_key2);
14480
14481        // Add a new root 2
14482        let new_root = 2;
14483        db.calculate_accounts_delta_hash(new_root);
14484        db.add_root(new_root);
14485
14486        // Check that the scan is properly set up
14487        assert_eq!(
14488            db.accounts_index.min_ongoing_scan_root().unwrap(),
14489            max_scan_root
14490        );
14491
14492        // If we specify a requested_flush_root == 2, then `slot 2 <= max_flush_slot` will
14493        // be flushed even though `slot 2 > max_scan_root`. The unrooted slot 1 should
14494        // remain in the cache
14495        db.flush_accounts_cache(true, Some(new_root));
14496        assert_eq!(db.accounts_cache.num_slots(), 1);
14497        assert!(db.accounts_cache.slot_cache(1).is_some());
14498
14499        // Intra cache cleaning should not clean the entry for `account_key` from slot 0,
14500        // even though it was updated in slot `2` because of the ongoing scan
14501        let account = db
14502            .do_load(
14503                &Ancestors::default(),
14504                &account_key,
14505                Some(0),
14506                LoadHint::Unspecified,
14507                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
14508            )
14509            .unwrap();
14510        assert_eq!(account.0.lamports(), zero_lamport_account.lamports());
14511
14512        // Run clean, unrooted slot 1 should not be purged, and still readable from the cache,
14513        // because we're still doing a scan on it.
14514        db.clean_accounts_for_tests();
14515        let account = db
14516            .do_load(
14517                &scan_ancestors,
14518                &account_key,
14519                Some(max_scan_root),
14520                LoadHint::Unspecified,
14521                LOAD_ZERO_LAMPORTS_ANY_TESTS,
14522            )
14523            .unwrap();
14524        assert_eq!(account.0.lamports(), slot1_account.lamports());
14525
14526        // When the scan is over, clean should not panic and should not purge something
14527        // still in the cache.
14528        scan_tracker.exit().unwrap();
14529        db.clean_accounts_for_tests();
14530        let account = db
14531            .do_load(
14532                &scan_ancestors,
14533                &account_key,
14534                Some(max_scan_root),
14535                LoadHint::Unspecified,
14536                LOAD_ZERO_LAMPORTS_ANY_TESTS,
14537            )
14538            .unwrap();
14539        assert_eq!(account.0.lamports(), slot1_account.lamports());
14540
14541        // Simulate dropping the bank, which finally removes the slot from the cache
14542        let bank_id = 1;
14543        db.purge_slot(1, bank_id, false);
14544        assert!(db
14545            .do_load(
14546                &scan_ancestors,
14547                &account_key,
14548                Some(max_scan_root),
14549                LoadHint::Unspecified,
14550                LOAD_ZERO_LAMPORTS_ANY_TESTS
14551            )
14552            .is_none());
14553    }
14554
14555    impl AccountsDb {
14556        fn get_and_assert_single_storage(&self, slot: Slot) -> Arc<AccountStorageEntry> {
14557            self.storage.get_slot_storage_entry(slot).unwrap()
14558        }
14559    }
14560
14561    #[test]
14562    fn test_alive_bytes() {
14563        let accounts_db = AccountsDb::new_with_config_for_tests(
14564            Vec::new(),
14565            &ClusterType::Development,
14566            AccountSecondaryIndexes::default(),
14567            AccountShrinkThreshold::default(),
14568        );
14569        let slot: Slot = 0;
14570        let num_keys = 10;
14571
14572        for data_size in 0..num_keys {
14573            let account = AccountSharedData::new(1, data_size, &Pubkey::default());
14574            accounts_db.store_cached((slot, &[(&Pubkey::new_unique(), &account)][..]), None);
14575        }
14576
14577        accounts_db.add_root(slot);
14578        accounts_db.flush_accounts_cache(true, None);
14579
14580        // Flushing cache should only create one storage entry
14581        let storage0 = accounts_db.get_and_assert_single_storage(slot);
14582        let accounts = storage0.all_accounts();
14583
14584        for account in accounts {
14585            let before_size = storage0.alive_bytes.load(Ordering::Acquire);
14586            let account_info = accounts_db
14587                .accounts_index
14588                .get_account_read_entry(account.pubkey())
14589                .map(|locked_entry| {
14590                    // Should only be one entry per key, since every key was only stored to slot 0
14591                    locked_entry.slot_list()[0]
14592                })
14593                .unwrap();
14594            let removed_data_size = account_info.1.stored_size();
14595            // Fetching the account from storage should return the same
14596            // stored size as in the index.
14597            assert_eq!(removed_data_size, account.stored_size as StoredSize);
14598            assert_eq!(account_info.0, slot);
14599            let reclaims = vec![account_info];
14600            accounts_db.remove_dead_accounts(reclaims.iter(), None, None, true);
14601            let after_size = storage0.alive_bytes.load(Ordering::Acquire);
14602            assert_eq!(before_size, after_size + account.stored_size);
14603        }
14604    }
14605
14606    fn setup_accounts_db_cache_clean(
14607        num_slots: usize,
14608        scan_slot: Option<Slot>,
14609        write_cache_limit_bytes: Option<u64>,
14610    ) -> (Arc<AccountsDb>, Vec<Pubkey>, Vec<Slot>, Option<ScanTracker>) {
14611        let mut accounts_db = AccountsDb::new_with_config_for_tests(
14612            Vec::new(),
14613            &ClusterType::Development,
14614            AccountSecondaryIndexes::default(),
14615            AccountShrinkThreshold::default(),
14616        );
14617        accounts_db.write_cache_limit_bytes = write_cache_limit_bytes;
14618        let accounts_db = Arc::new(accounts_db);
14619
14620        let slots: Vec<_> = (0..num_slots as Slot).collect();
14621        let stall_slot = num_slots as Slot;
14622        let scan_stall_key = Pubkey::new_unique();
14623        let keys: Vec<Pubkey> = std::iter::repeat_with(Pubkey::new_unique)
14624            .take(num_slots)
14625            .collect();
14626        if scan_slot.is_some() {
14627            accounts_db.store_cached(
14628                // Store it in a slot that isn't returned in `slots`
14629                (
14630                    stall_slot,
14631                    &[(
14632                        &scan_stall_key,
14633                        &AccountSharedData::new(1, 0, &Pubkey::default()),
14634                    )][..],
14635                ),
14636                None,
14637            );
14638        }
14639
14640        // Store some subset of the keys in slots 0..num_slots
14641        let mut scan_tracker = None;
14642        for slot in &slots {
14643            for key in &keys[*slot as usize..] {
14644                let space = 1; // 1 byte allows us to track by size
14645                accounts_db.store_cached(
14646                    (
14647                        *slot,
14648                        &[(key, &AccountSharedData::new(1, space, &Pubkey::default()))][..],
14649                    ),
14650                    None,
14651                );
14652            }
14653            accounts_db.add_root(*slot as Slot);
14654            if Some(*slot) == scan_slot {
14655                let ancestors = Arc::new(vec![(stall_slot, 1), (*slot, 1)].into_iter().collect());
14656                let bank_id = 0;
14657                scan_tracker = Some(setup_scan(
14658                    accounts_db.clone(),
14659                    ancestors,
14660                    bank_id,
14661                    scan_stall_key,
14662                ));
14663                assert_eq!(
14664                    accounts_db.accounts_index.min_ongoing_scan_root().unwrap(),
14665                    *slot
14666                );
14667            }
14668        }
14669
14670        accounts_db.accounts_cache.remove_slot(stall_slot);
14671
14672        // If there's <= max_cache_slots(), no slots should be flushed
14673        if accounts_db.accounts_cache.num_slots() <= max_cache_slots() {
14674            accounts_db.flush_accounts_cache(false, None);
14675            assert_eq!(accounts_db.accounts_cache.num_slots(), num_slots);
14676        }
14677
14678        (accounts_db, keys, slots, scan_tracker)
14679    }
14680
14681    #[test]
14682    fn test_accounts_db_cache_clean_dead_slots() {
14683        let num_slots = 10;
14684        let (accounts_db, keys, mut slots, _) =
14685            setup_accounts_db_cache_clean(num_slots, None, None);
14686        let last_dead_slot = (num_slots - 1) as Slot;
14687        assert_eq!(*slots.last().unwrap(), last_dead_slot);
14688        let alive_slot = last_dead_slot as Slot + 1;
14689        slots.push(alive_slot);
14690        for key in &keys {
14691            // Store a slot that overwrites all previous keys, rendering all previous keys dead
14692            accounts_db.store_cached(
14693                (
14694                    alive_slot,
14695                    &[(key, &AccountSharedData::new(1, 0, &Pubkey::default()))][..],
14696                ),
14697                None,
14698            );
14699            accounts_db.add_root(alive_slot);
14700        }
14701
14702        // Before the flush, we can find entries in the database for slots < alive_slot if we specify
14703        // a smaller max root
14704        for key in &keys {
14705            assert!(accounts_db
14706                .do_load(
14707                    &Ancestors::default(),
14708                    key,
14709                    Some(last_dead_slot),
14710                    LoadHint::Unspecified,
14711                    LOAD_ZERO_LAMPORTS_ANY_TESTS
14712                )
14713                .is_some());
14714        }
14715
14716        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
14717        accounts_db.flush_accounts_cache(true, None);
14718        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
14719        let mut uncleaned_roots = accounts_db
14720            .accounts_index
14721            .clear_uncleaned_roots(None)
14722            .into_iter()
14723            .collect::<Vec<_>>();
14724        uncleaned_roots.sort_unstable();
14725        assert_eq!(uncleaned_roots, slots);
14726        assert_eq!(
14727            accounts_db.accounts_cache.fetch_max_flush_root(),
14728            alive_slot,
14729        );
14730
14731        // Specifying a max_root < alive_slot, should not return any more entries,
14732        // as those have been purged from the accounts index for the dead slots.
14733        for key in &keys {
14734            assert!(accounts_db
14735                .do_load(
14736                    &Ancestors::default(),
14737                    key,
14738                    Some(last_dead_slot),
14739                    LoadHint::Unspecified,
14740                    LOAD_ZERO_LAMPORTS_ANY_TESTS
14741                )
14742                .is_none());
14743        }
14744        // Each slot should only have one entry in the storage, since all other accounts were
14745        // cleaned due to later updates
14746        for slot in &slots {
14747            if let ScanStorageResult::Stored(slot_accounts) = accounts_db.scan_account_storage(
14748                *slot as Slot,
14749                |_| Some(0),
14750                |slot_accounts: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
14751                    slot_accounts.insert(*loaded_account.pubkey());
14752                },
14753            ) {
14754                if *slot == alive_slot {
14755                    assert_eq!(slot_accounts.len(), keys.len());
14756                } else {
14757                    assert!(slot_accounts.is_empty());
14758                }
14759            } else {
14760                panic!("Expected slot to be in storage, not cache");
14761            }
14762        }
14763    }
14764
14765    #[test]
14766    fn test_accounts_db_cache_clean() {
14767        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(10, None, None);
14768
14769        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
14770        accounts_db.flush_accounts_cache(true, None);
14771        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
14772        let mut uncleaned_roots = accounts_db
14773            .accounts_index
14774            .clear_uncleaned_roots(None)
14775            .into_iter()
14776            .collect::<Vec<_>>();
14777        uncleaned_roots.sort_unstable();
14778        assert_eq!(uncleaned_roots, slots);
14779        assert_eq!(
14780            accounts_db.accounts_cache.fetch_max_flush_root(),
14781            *slots.last().unwrap()
14782        );
14783
14784        // Each slot should only have one entry in the storage, since all other accounts were
14785        // cleaned due to later updates
14786        for slot in &slots {
14787            if let ScanStorageResult::Stored(slot_account) = accounts_db.scan_account_storage(
14788                *slot as Slot,
14789                |_| Some(0),
14790                |slot_account: &Arc<RwLock<Pubkey>>, loaded_account: LoadedAccount| {
14791                    *slot_account.write().unwrap() = *loaded_account.pubkey();
14792                },
14793            ) {
14794                assert_eq!(*slot_account.read().unwrap(), keys[*slot as usize]);
14795            } else {
14796                panic!("Everything should have been flushed")
14797            }
14798        }
14799    }
14800
14801    fn run_test_accounts_db_cache_clean_max_root(
14802        num_slots: usize,
14803        requested_flush_root: Slot,
14804        scan_root: Option<Slot>,
14805    ) {
14806        assert!(requested_flush_root < (num_slots as Slot));
14807        let (accounts_db, keys, slots, scan_tracker) =
14808            setup_accounts_db_cache_clean(num_slots, scan_root, Some(max_cache_slots() as u64));
14809        let is_cache_at_limit = num_slots - requested_flush_root as usize - 1 > max_cache_slots();
14810
14811        // If:
14812        // 1) `requested_flush_root` is specified,
14813        // 2) not at the cache limit, i.e. `is_cache_at_limit == false`, then
14814        // `flush_accounts_cache()` should clean and flush only slots <= requested_flush_root,
14815        accounts_db.flush_accounts_cache(true, Some(requested_flush_root));
14816
14817        if !is_cache_at_limit {
14818            // Should flush all slots between 0..=requested_flush_root
14819            assert_eq!(
14820                accounts_db.accounts_cache.num_slots(),
14821                slots.len() - requested_flush_root as usize - 1
14822            );
14823        } else {
14824            // Otherwise, if we are at the cache limit, all roots will be flushed
14825            assert_eq!(accounts_db.accounts_cache.num_slots(), 0,);
14826        }
14827
14828        let mut uncleaned_roots = accounts_db
14829            .accounts_index
14830            .clear_uncleaned_roots(None)
14831            .into_iter()
14832            .collect::<Vec<_>>();
14833        uncleaned_roots.sort_unstable();
14834
14835        let expected_max_flushed_root = if !is_cache_at_limit {
14836            // Should flush all slots between 0..=requested_flush_root
14837            requested_flush_root
14838        } else {
14839            // Otherwise, if we are at the cache limit, all roots will be flushed
14840            num_slots as Slot - 1
14841        };
14842
14843        assert_eq!(
14844            uncleaned_roots,
14845            slots[0..=expected_max_flushed_root as usize].to_vec()
14846        );
14847        assert_eq!(
14848            accounts_db.accounts_cache.fetch_max_flush_root(),
14849            expected_max_flushed_root,
14850        );
14851
14852        for slot in &slots {
14853            let slot_accounts = accounts_db.scan_account_storage(
14854                *slot as Slot,
14855                |loaded_account: LoadedAccount| {
14856                    assert!(
14857                        !is_cache_at_limit,
14858                        "When cache is at limit, all roots should have been flushed to storage"
14859                    );
14860                    // All slots <= requested_flush_root should have been flushed, regardless
14861                    // of ongoing scans
14862                    assert!(*slot > requested_flush_root);
14863                    Some(*loaded_account.pubkey())
14864                },
14865                |slot_accounts: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
14866                    slot_accounts.insert(*loaded_account.pubkey());
14867                    if !is_cache_at_limit {
14868                        // Only true when the limit hasn't been reached and there are still
14869                        // slots left in the cache
14870                        assert!(*slot <= requested_flush_root);
14871                    }
14872                },
14873            );
14874
14875            let slot_accounts = match slot_accounts {
14876                ScanStorageResult::Cached(slot_accounts) => {
14877                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
14878                }
14879                ScanStorageResult::Stored(slot_accounts) => {
14880                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
14881                }
14882            };
14883
14884            let expected_accounts =
14885                if *slot >= requested_flush_root || *slot >= scan_root.unwrap_or(Slot::MAX) {
14886                    // 1) If slot > `requested_flush_root`, then  either:
14887                    //   a) If `is_cache_at_limit == false`, still in the cache
14888                    //   b) if `is_cache_at_limit == true`, were not cleaned before being flushed to storage.
14889                    //
14890                    // In both cases all the *original* updates at index `slot` were uncleaned and thus
14891                    // should be discoverable by this scan.
14892                    //
14893                    // 2) If slot == `requested_flush_root`, the slot was not cleaned before being flushed to storage,
14894                    // so it also contains all the original updates.
14895                    //
14896                    // 3) If *slot >= scan_root, then we should not clean it either
14897                    keys[*slot as usize..]
14898                        .iter()
14899                        .cloned()
14900                        .collect::<HashSet<Pubkey>>()
14901                } else {
14902                    // Slots less than `requested_flush_root` and `scan_root` were cleaned in the cache before being flushed
14903                    // to storage, should only contain one account
14904                    std::iter::once(keys[*slot as usize]).collect::<HashSet<Pubkey>>()
14905                };
14906
14907            assert_eq!(slot_accounts, expected_accounts);
14908        }
14909
14910        if let Some(scan_tracker) = scan_tracker {
14911            scan_tracker.exit().unwrap();
14912        }
14913    }
14914
14915    #[test]
14916    fn test_accounts_db_cache_clean_max_root() {
14917        let requested_flush_root = 5;
14918        run_test_accounts_db_cache_clean_max_root(10, requested_flush_root, None);
14919    }
14920
14921    #[test]
14922    fn test_accounts_db_cache_clean_max_root_with_scan() {
14923        let requested_flush_root = 5;
14924        run_test_accounts_db_cache_clean_max_root(
14925            10,
14926            requested_flush_root,
14927            Some(requested_flush_root - 1),
14928        );
14929        run_test_accounts_db_cache_clean_max_root(
14930            10,
14931            requested_flush_root,
14932            Some(requested_flush_root + 1),
14933        );
14934    }
14935
14936    #[test]
14937    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit() {
14938        let requested_flush_root = 5;
14939        // Test that if there are > max_cache_slots() in the cache after flush, then more roots
14940        // will be flushed
14941        run_test_accounts_db_cache_clean_max_root(
14942            max_cache_slots() + requested_flush_root as usize + 2,
14943            requested_flush_root,
14944            None,
14945        );
14946    }
14947
14948    #[test]
14949    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit_and_scan() {
14950        let requested_flush_root = 5;
14951        // Test that if there are > max_cache_slots() in the cache after flush, then more roots
14952        // will be flushed
14953        run_test_accounts_db_cache_clean_max_root(
14954            max_cache_slots() + requested_flush_root as usize + 2,
14955            requested_flush_root,
14956            Some(requested_flush_root - 1),
14957        );
14958        run_test_accounts_db_cache_clean_max_root(
14959            max_cache_slots() + requested_flush_root as usize + 2,
14960            requested_flush_root,
14961            Some(requested_flush_root + 1),
14962        );
14963    }
14964
14965    fn run_flush_rooted_accounts_cache(should_clean: bool) {
14966        let num_slots = 10;
14967        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(num_slots, None, None);
14968        let mut cleaned_bytes = 0;
14969        let mut cleaned_accounts = 0;
14970        let should_clean_tracker = if should_clean {
14971            Some((&mut cleaned_bytes, &mut cleaned_accounts))
14972        } else {
14973            None
14974        };
14975
14976        // If no cleaning is specified, then flush everything
14977        accounts_db.flush_rooted_accounts_cache(None, should_clean_tracker);
14978        for slot in &slots {
14979            let slot_accounts = if let ScanStorageResult::Stored(slot_accounts) = accounts_db
14980                .scan_account_storage(
14981                    *slot as Slot,
14982                    |_| Some(0),
14983                    |slot_account: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
14984                        slot_account.insert(*loaded_account.pubkey());
14985                    },
14986                ) {
14987                slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
14988            } else {
14989                panic!("All roots should have been flushed to storage");
14990            };
14991            let expected_accounts = if !should_clean || slot == slots.last().unwrap() {
14992                // The slot was not cleaned before being flushed to storage,
14993                // so it also contains all the original updates.
14994                keys[*slot as usize..]
14995                    .iter()
14996                    .cloned()
14997                    .collect::<HashSet<Pubkey>>()
14998            } else {
14999                // If clean was specified, only the latest slot should have all the updates.
15000                // All these other slots have been cleaned before flush
15001                std::iter::once(keys[*slot as usize]).collect::<HashSet<Pubkey>>()
15002            };
15003            assert_eq!(slot_accounts, expected_accounts);
15004        }
15005    }
15006
15007    #[test]
15008    fn test_flush_rooted_accounts_cache_with_clean() {
15009        run_flush_rooted_accounts_cache(true);
15010    }
15011
15012    #[test]
15013    fn test_flush_rooted_accounts_cache_without_clean() {
15014        run_flush_rooted_accounts_cache(false);
15015    }
15016
15017    fn run_test_shrink_unref(do_intra_cache_clean: bool) {
15018        let db = AccountsDb::new_with_config_for_tests(
15019            Vec::new(),
15020            &ClusterType::Development,
15021            AccountSecondaryIndexes::default(),
15022            AccountShrinkThreshold::default(),
15023        );
15024        let account_key1 = Pubkey::new_unique();
15025        let account_key2 = Pubkey::new_unique();
15026        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
15027
15028        // Store into slot 0
15029        db.store_cached((0, &[(&account_key1, &account1)][..]), None);
15030        db.store_cached((0, &[(&account_key2, &account1)][..]), None);
15031        db.add_root(0);
15032        if !do_intra_cache_clean {
15033            // If we don't want the cache doing purges before flush,
15034            // then we cannot flush multiple roots at once, otherwise the later
15035            // roots will clean the earlier roots before they are stored.
15036            // Thus flush the roots individually
15037            db.flush_accounts_cache(true, None);
15038
15039            // Add an additional ref within the same slot to pubkey 1
15040            db.store_uncached(0, &[(&account_key1, &account1)]);
15041        }
15042
15043        // Make account_key1 in slot 0 outdated by updating in rooted slot 1
15044        db.store_cached((1, &[(&account_key1, &account1)][..]), None);
15045        db.add_root(1);
15046        // Flushes all roots
15047        db.flush_accounts_cache(true, None);
15048        db.calculate_accounts_delta_hash(0);
15049        db.calculate_accounts_delta_hash(1);
15050
15051        // Clean to remove outdated entry from slot 0
15052        db.clean_accounts(Some(1), false, None);
15053
15054        // Shrink Slot 0
15055        let slot0_store = db.get_and_assert_single_storage(0);
15056        {
15057            let mut shrink_candidate_slots = db.shrink_candidate_slots.lock().unwrap();
15058            shrink_candidate_slots.insert(0, slot0_store);
15059        }
15060        db.shrink_candidate_slots();
15061
15062        // Make slot 0 dead by updating the remaining key
15063        db.store_cached((2, &[(&account_key2, &account1)][..]), None);
15064        db.add_root(2);
15065
15066        // Flushes all roots
15067        db.flush_accounts_cache(true, None);
15068
15069        // Should be one store before clean for slot 0
15070        db.get_and_assert_single_storage(0);
15071        db.calculate_accounts_delta_hash(2);
15072        db.clean_accounts(Some(2), false, None);
15073
15074        // No stores should exist for slot 0 after clean
15075        assert_no_storages_at_slot(&db, 0);
15076
15077        // Ref count for `account_key1` (account removed earlier by shrink)
15078        // should be 1, since it was only stored in slot 0 and 1, and slot 0
15079        // is now dead
15080        assert_eq!(db.accounts_index.ref_count_from_storage(&account_key1), 1);
15081    }
15082
15083    #[test]
15084    fn test_shrink_unref() {
15085        run_test_shrink_unref(false)
15086    }
15087
15088    #[test]
15089    fn test_shrink_unref_with_intra_slot_cleaning() {
15090        run_test_shrink_unref(true)
15091    }
15092
15093    #[test]
15094    fn test_partial_clean() {
15095        solana_logger::setup();
15096        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
15097        let account_key1 = Pubkey::new_unique();
15098        let account_key2 = Pubkey::new_unique();
15099        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
15100        let account2 = AccountSharedData::new(2, 0, AccountSharedData::default().owner());
15101        let account3 = AccountSharedData::new(3, 0, AccountSharedData::default().owner());
15102        let account4 = AccountSharedData::new(4, 0, AccountSharedData::default().owner());
15103
15104        // Store accounts into slots 0 and 1
15105        db.store_uncached(0, &[(&account_key1, &account1)]);
15106        db.store_uncached(0, &[(&account_key2, &account1)]);
15107        db.store_uncached(1, &[(&account_key1, &account2)]);
15108        db.calculate_accounts_delta_hash(0);
15109        db.calculate_accounts_delta_hash(1);
15110
15111        db.print_accounts_stats("pre-clean1");
15112
15113        // clean accounts - no accounts should be cleaned, since no rooted slots
15114        //
15115        // Checking that the uncleaned_pubkeys are not pre-maturely removed
15116        // such that when the slots are rooted, and can actually be cleaned, then the
15117        // delta keys are still there.
15118        db.clean_accounts_for_tests();
15119
15120        db.print_accounts_stats("post-clean1");
15121        // Check stores > 0
15122        assert!(!db.storage.is_empty_entry(0));
15123        assert!(!db.storage.is_empty_entry(1));
15124
15125        // root slot 0
15126        db.add_root_and_flush_write_cache(0);
15127
15128        // store into slot 2
15129        db.store_uncached(2, &[(&account_key2, &account3)]);
15130        db.store_uncached(2, &[(&account_key1, &account3)]);
15131        db.calculate_accounts_delta_hash(2);
15132
15133        db.clean_accounts_for_tests();
15134        db.print_accounts_stats("post-clean2");
15135
15136        // root slots 1
15137        db.add_root_and_flush_write_cache(1);
15138        db.clean_accounts_for_tests();
15139
15140        db.print_accounts_stats("post-clean3");
15141
15142        db.store_uncached(3, &[(&account_key2, &account4)]);
15143        db.calculate_accounts_delta_hash(3);
15144        db.add_root_and_flush_write_cache(3);
15145
15146        // Check that we can clean where max_root=3 and slot=2 is not rooted
15147        db.clean_accounts_for_tests();
15148
15149        assert!(db.uncleaned_pubkeys.is_empty());
15150
15151        db.print_accounts_stats("post-clean4");
15152
15153        assert!(db.storage.is_empty_entry(0));
15154        assert!(!db.storage.is_empty_entry(1));
15155    }
15156
15157    #[test]
15158    fn test_recycle_stores_expiration() {
15159        solana_logger::setup();
15160
15161        let common_store_path = Path::new("");
15162        let common_slot_id = 12;
15163        let store_file_size = 1000;
15164
15165        let store1_id = 22;
15166        let entry1 = Arc::new(AccountStorageEntry::new(
15167            common_store_path,
15168            common_slot_id,
15169            store1_id,
15170            store_file_size,
15171        ));
15172
15173        let store2_id = 44;
15174        let entry2 = Arc::new(AccountStorageEntry::new(
15175            common_store_path,
15176            common_slot_id,
15177            store2_id,
15178            store_file_size,
15179        ));
15180
15181        let mut recycle_stores = RecycleStores::default();
15182        recycle_stores.add_entry(entry1);
15183        recycle_stores.add_entry(entry2);
15184        assert_eq!(recycle_stores.entry_count(), 2);
15185
15186        // no expiration for newly added entries
15187        let expired = recycle_stores.expire_old_entries();
15188        assert_eq!(
15189            expired
15190                .iter()
15191                .map(|e| e.append_vec_id())
15192                .collect::<Vec<_>>(),
15193            Vec::<AppendVecId>::new()
15194        );
15195        assert_eq!(
15196            recycle_stores
15197                .iter()
15198                .map(|(_, e)| e.append_vec_id())
15199                .collect::<Vec<_>>(),
15200            vec![store1_id, store2_id]
15201        );
15202        assert_eq!(recycle_stores.entry_count(), 2);
15203        assert_eq!(recycle_stores.total_bytes(), store_file_size * 2);
15204
15205        // expiration for only too old entries
15206        recycle_stores.entries[0].0 = Instant::now()
15207            .checked_sub(Duration::from_secs(EXPIRATION_TTL_SECONDS + 1))
15208            .unwrap();
15209        let expired = recycle_stores.expire_old_entries();
15210        assert_eq!(
15211            expired
15212                .iter()
15213                .map(|e| e.append_vec_id())
15214                .collect::<Vec<_>>(),
15215            vec![store1_id]
15216        );
15217        assert_eq!(
15218            recycle_stores
15219                .iter()
15220                .map(|(_, e)| e.append_vec_id())
15221                .collect::<Vec<_>>(),
15222            vec![store2_id]
15223        );
15224        assert_eq!(recycle_stores.entry_count(), 1);
15225        assert_eq!(recycle_stores.total_bytes(), store_file_size);
15226    }
15227
15228    const RACY_SLEEP_MS: u64 = 10;
15229    const RACE_TIME: u64 = 5;
15230
15231    fn start_load_thread(
15232        with_retry: bool,
15233        ancestors: Ancestors,
15234        db: Arc<AccountsDb>,
15235        exit: Arc<AtomicBool>,
15236        pubkey: Arc<Pubkey>,
15237        expected_lamports: impl Fn(&(AccountSharedData, Slot)) -> u64 + Send + 'static,
15238    ) -> JoinHandle<()> {
15239        let load_hint = if with_retry {
15240            LoadHint::FixedMaxRoot
15241        } else {
15242            LoadHint::Unspecified
15243        };
15244
15245        std::thread::Builder::new()
15246            .name("account-do-load".to_string())
15247            .spawn(move || {
15248                loop {
15249                    if exit.load(Ordering::Relaxed) {
15250                        return;
15251                    }
15252                    // Meddle load_limit to cover all branches of implementation.
15253                    // There should absolutely no behaviorial difference; the load_limit triggered
15254                    // slow branch should only affect the performance.
15255                    // Ordering::Relaxed is ok because of no data dependencies; the modified field is
15256                    // completely free-standing cfg(test) control-flow knob.
15257                    db.load_limit
15258                        .store(thread_rng().gen_range(0, 10) as u64, Ordering::Relaxed);
15259
15260                    // Load should never be unable to find this key
15261                    let loaded_account = db
15262                        .do_load(
15263                            &ancestors,
15264                            &pubkey,
15265                            None,
15266                            load_hint,
15267                            LOAD_ZERO_LAMPORTS_ANY_TESTS,
15268                        )
15269                        .unwrap();
15270                    // slot + 1 == account.lamports because of the account-cache-flush thread
15271                    assert_eq!(
15272                        loaded_account.0.lamports(),
15273                        expected_lamports(&loaded_account)
15274                    );
15275                }
15276            })
15277            .unwrap()
15278    }
15279
15280    fn do_test_load_account_and_cache_flush_race(with_retry: bool) {
15281        solana_logger::setup();
15282
15283        let mut db = AccountsDb::new_with_config_for_tests(
15284            Vec::new(),
15285            &ClusterType::Development,
15286            AccountSecondaryIndexes::default(),
15287            AccountShrinkThreshold::default(),
15288        );
15289        db.load_delay = RACY_SLEEP_MS;
15290        let db = Arc::new(db);
15291        let pubkey = Arc::new(Pubkey::new_unique());
15292        let exit = Arc::new(AtomicBool::new(false));
15293        db.store_cached(
15294            (
15295                0,
15296                &[(
15297                    pubkey.as_ref(),
15298                    &AccountSharedData::new(1, 0, AccountSharedData::default().owner()),
15299                )][..],
15300            ),
15301            None,
15302        );
15303        db.add_root(0);
15304        db.flush_accounts_cache(true, None);
15305
15306        let t_flush_accounts_cache = {
15307            let db = db.clone();
15308            let exit = exit.clone();
15309            let pubkey = pubkey.clone();
15310            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
15311            std::thread::Builder::new()
15312                .name("account-cache-flush".to_string())
15313                .spawn(move || {
15314                    let mut slot: Slot = 1;
15315                    loop {
15316                        if exit.load(Ordering::Relaxed) {
15317                            return;
15318                        }
15319                        account.set_lamports(slot + 1);
15320                        db.store_cached((slot, &[(pubkey.as_ref(), &account)][..]), None);
15321                        db.add_root(slot);
15322                        sleep(Duration::from_millis(RACY_SLEEP_MS));
15323                        db.flush_accounts_cache(true, None);
15324                        slot += 1;
15325                    }
15326                })
15327                .unwrap()
15328        };
15329
15330        let t_do_load = start_load_thread(
15331            with_retry,
15332            Ancestors::default(),
15333            db,
15334            exit.clone(),
15335            pubkey,
15336            |(_, slot)| slot + 1,
15337        );
15338
15339        sleep(Duration::from_secs(RACE_TIME));
15340        exit.store(true, Ordering::Relaxed);
15341        t_flush_accounts_cache.join().unwrap();
15342        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
15343    }
15344
15345    #[test]
15346    fn test_load_account_and_cache_flush_race_with_retry() {
15347        do_test_load_account_and_cache_flush_race(true);
15348    }
15349
15350    #[test]
15351    fn test_load_account_and_cache_flush_race_without_retry() {
15352        do_test_load_account_and_cache_flush_race(false);
15353    }
15354
15355    fn do_test_load_account_and_shrink_race(with_retry: bool) {
15356        let mut db = AccountsDb::new_with_config_for_tests(
15357            Vec::new(),
15358            &ClusterType::Development,
15359            AccountSecondaryIndexes::default(),
15360            AccountShrinkThreshold::default(),
15361        );
15362        db.load_delay = RACY_SLEEP_MS;
15363        let db = Arc::new(db);
15364        let pubkey = Arc::new(Pubkey::new_unique());
15365        let exit = Arc::new(AtomicBool::new(false));
15366        let slot = 1;
15367
15368        // Store an account
15369        let lamports = 42;
15370        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
15371        account.set_lamports(lamports);
15372        db.store_uncached(slot, &[(&pubkey, &account)]);
15373
15374        // Set the slot as a root so account loads will see the contents of this slot
15375        db.add_root(slot);
15376
15377        let t_shrink_accounts = {
15378            let db = db.clone();
15379            let exit = exit.clone();
15380
15381            std::thread::Builder::new()
15382                .name("account-shrink".to_string())
15383                .spawn(move || loop {
15384                    if exit.load(Ordering::Relaxed) {
15385                        return;
15386                    }
15387                    // Simulate adding shrink candidates from clean_accounts()
15388                    let store = db.get_and_assert_single_storage(slot);
15389                    db.shrink_candidate_slots
15390                        .lock()
15391                        .unwrap()
15392                        .insert(slot, store.clone());
15393                    db.shrink_candidate_slots();
15394                })
15395                .unwrap()
15396        };
15397
15398        let t_do_load = start_load_thread(
15399            with_retry,
15400            Ancestors::default(),
15401            db,
15402            exit.clone(),
15403            pubkey,
15404            move |_| lamports,
15405        );
15406
15407        sleep(Duration::from_secs(RACE_TIME));
15408        exit.store(true, Ordering::Relaxed);
15409        t_shrink_accounts.join().unwrap();
15410        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
15411    }
15412
15413    #[test]
15414    fn test_load_account_and_shrink_race_with_retry() {
15415        do_test_load_account_and_shrink_race(true);
15416    }
15417
15418    #[test]
15419    fn test_load_account_and_shrink_race_without_retry() {
15420        do_test_load_account_and_shrink_race(false);
15421    }
15422
15423    #[test]
15424    fn test_cache_flush_delayed_remove_unrooted_race() {
15425        let mut db = AccountsDb::new_with_config_for_tests(
15426            Vec::new(),
15427            &ClusterType::Development,
15428            AccountSecondaryIndexes::default(),
15429            AccountShrinkThreshold::default(),
15430        );
15431        db.load_delay = RACY_SLEEP_MS;
15432        let db = Arc::new(db);
15433        let slot = 10;
15434        let bank_id = 10;
15435
15436        let lamports = 42;
15437        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
15438        account.set_lamports(lamports);
15439
15440        // Start up a thread to flush the accounts cache
15441        let (flush_trial_start_sender, flush_trial_start_receiver) = unbounded();
15442        let (flush_done_sender, flush_done_receiver) = unbounded();
15443        let t_flush_cache = {
15444            let db = db.clone();
15445            std::thread::Builder::new()
15446                .name("account-cache-flush".to_string())
15447                .spawn(move || loop {
15448                    // Wait for the signal to start a trial
15449                    if flush_trial_start_receiver.recv().is_err() {
15450                        return;
15451                    }
15452                    db.flush_slot_cache(10);
15453                    flush_done_sender.send(()).unwrap();
15454                })
15455                .unwrap()
15456        };
15457
15458        // Start up a thread remove the slot
15459        let (remove_trial_start_sender, remove_trial_start_receiver) = unbounded();
15460        let (remove_done_sender, remove_done_receiver) = unbounded();
15461        let t_remove = {
15462            let db = db.clone();
15463            std::thread::Builder::new()
15464                .name("account-remove".to_string())
15465                .spawn(move || loop {
15466                    // Wait for the signal to start a trial
15467                    if remove_trial_start_receiver.recv().is_err() {
15468                        return;
15469                    }
15470                    db.remove_unrooted_slots(&[(slot, bank_id)]);
15471                    remove_done_sender.send(()).unwrap();
15472                })
15473                .unwrap()
15474        };
15475
15476        let num_trials = 10;
15477        for _ in 0..num_trials {
15478            let pubkey = Pubkey::new_unique();
15479            db.store_cached((slot, &[(&pubkey, &account)][..]), None);
15480            // Wait for both threads to finish
15481            flush_trial_start_sender.send(()).unwrap();
15482            remove_trial_start_sender.send(()).unwrap();
15483            let _ = flush_done_receiver.recv();
15484            let _ = remove_done_receiver.recv();
15485        }
15486
15487        drop(flush_trial_start_sender);
15488        drop(remove_trial_start_sender);
15489        t_flush_cache.join().unwrap();
15490        t_remove.join().unwrap();
15491    }
15492
15493    #[test]
15494    fn test_cache_flush_remove_unrooted_race_multiple_slots() {
15495        let db = AccountsDb::new_with_config_for_tests(
15496            Vec::new(),
15497            &ClusterType::Development,
15498            AccountSecondaryIndexes::default(),
15499            AccountShrinkThreshold::default(),
15500        );
15501        let db = Arc::new(db);
15502        let num_cached_slots = 100;
15503
15504        let num_trials = 100;
15505        let (new_trial_start_sender, new_trial_start_receiver) = unbounded();
15506        let (flush_done_sender, flush_done_receiver) = unbounded();
15507        // Start up a thread to flush the accounts cache
15508        let t_flush_cache = {
15509            let db = db.clone();
15510
15511            std::thread::Builder::new()
15512                .name("account-cache-flush".to_string())
15513                .spawn(move || loop {
15514                    // Wait for the signal to start a trial
15515                    if new_trial_start_receiver.recv().is_err() {
15516                        return;
15517                    }
15518                    for slot in 0..num_cached_slots {
15519                        db.flush_slot_cache(slot);
15520                    }
15521                    flush_done_sender.send(()).unwrap();
15522                })
15523                .unwrap()
15524        };
15525
15526        let exit = Arc::new(AtomicBool::new(false));
15527
15528        let t_spurious_signal = {
15529            let db = db.clone();
15530            let exit = exit.clone();
15531            std::thread::Builder::new()
15532                .name("account-cache-flush".to_string())
15533                .spawn(move || loop {
15534                    if exit.load(Ordering::Relaxed) {
15535                        return;
15536                    }
15537                    // Simulate spurious wake-up that can happen, but is too rare to
15538                    // otherwise depend on in tests.
15539                    db.remove_unrooted_slots_synchronization.signal.notify_all();
15540                })
15541                .unwrap()
15542        };
15543
15544        // Run multiple trials. Has the added benefit of rewriting the same slots after we've
15545        // dumped them in previous trials.
15546        for _ in 0..num_trials {
15547            // Store an account
15548            let lamports = 42;
15549            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
15550            account.set_lamports(lamports);
15551
15552            // Pick random 50% of the slots to pass to `remove_unrooted_slots()`
15553            let mut all_slots: Vec<(Slot, BankId)> = (0..num_cached_slots)
15554                .map(|slot| {
15555                    let bank_id = slot + 1;
15556                    (slot, bank_id)
15557                })
15558                .collect();
15559            all_slots.shuffle(&mut rand::thread_rng());
15560            let slots_to_dump = &all_slots[0..num_cached_slots as usize / 2];
15561            let slots_to_keep = &all_slots[num_cached_slots as usize / 2..];
15562
15563            // Set up a one account per slot across many different slots, track which
15564            // pubkey was stored in each slot.
15565            let slot_to_pubkey_map: HashMap<Slot, Pubkey> = (0..num_cached_slots)
15566                .map(|slot| {
15567                    let pubkey = Pubkey::new_unique();
15568                    db.store_cached((slot, &[(&pubkey, &account)][..]), None);
15569                    (slot, pubkey)
15570                })
15571                .collect();
15572
15573            // Signal the flushing shred to start flushing
15574            new_trial_start_sender.send(()).unwrap();
15575
15576            // Here we want to test both:
15577            // 1) Flush thread starts flushing a slot before we try dumping it.
15578            // 2) Flushing thread trying to flush while/after we're trying to dump the slot,
15579            // in which case flush should ignore/move past the slot to be dumped
15580            //
15581            // Hence, we split into chunks to get the dumping of each chunk to race with the
15582            // flushes. If we were to dump the entire chunk at once, then this reduces the possibility
15583            // of the flush occurring first since the dumping logic reserves all the slots it's about
15584            // to dump immediately.
15585
15586            for chunks in slots_to_dump.chunks(slots_to_dump.len() / 2) {
15587                db.remove_unrooted_slots(chunks);
15588            }
15589
15590            // Check that all the slots in `slots_to_dump` were completely removed from the
15591            // cache, storage, and index
15592
15593            for (slot, _) in slots_to_dump {
15594                assert_no_storages_at_slot(&db, *slot);
15595                assert!(db.accounts_cache.slot_cache(*slot).is_none());
15596                let account_in_slot = slot_to_pubkey_map[slot];
15597                let item = db.accounts_index.get_account_read_entry(&account_in_slot);
15598                assert!(item.is_none(), "item: {item:?}");
15599            }
15600
15601            // Wait for flush to finish before starting next trial
15602
15603            flush_done_receiver.recv().unwrap();
15604
15605            for (slot, bank_id) in slots_to_keep {
15606                let account_in_slot = slot_to_pubkey_map[slot];
15607                assert!(db
15608                    .load(
15609                        &Ancestors::from(vec![(*slot, 0)]),
15610                        &account_in_slot,
15611                        LoadHint::FixedMaxRoot
15612                    )
15613                    .is_some());
15614                // Clear for next iteration so that `assert!(self.storage.get_slot_storage_entry(purged_slot).is_none());`
15615                // in `purge_slot_pubkeys()` doesn't trigger
15616                db.remove_unrooted_slots(&[(*slot, *bank_id)]);
15617            }
15618        }
15619
15620        exit.store(true, Ordering::Relaxed);
15621        drop(new_trial_start_sender);
15622        t_flush_cache.join().unwrap();
15623
15624        t_spurious_signal.join().unwrap();
15625    }
15626
15627    #[test]
15628    fn test_collect_uncleaned_slots_up_to_slot() {
15629        solana_logger::setup();
15630        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
15631
15632        let slot1 = 11;
15633        let slot2 = 222;
15634        let slot3 = 3333;
15635
15636        let pubkey1 = Pubkey::new_unique();
15637        let pubkey2 = Pubkey::new_unique();
15638        let pubkey3 = Pubkey::new_unique();
15639
15640        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
15641        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
15642        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
15643
15644        let mut uncleaned_slots1 = db.collect_uncleaned_slots_up_to_slot(slot1);
15645        let mut uncleaned_slots2 = db.collect_uncleaned_slots_up_to_slot(slot2);
15646        let mut uncleaned_slots3 = db.collect_uncleaned_slots_up_to_slot(slot3);
15647
15648        uncleaned_slots1.sort_unstable();
15649        uncleaned_slots2.sort_unstable();
15650        uncleaned_slots3.sort_unstable();
15651
15652        assert_eq!(uncleaned_slots1, [slot1]);
15653        assert_eq!(uncleaned_slots2, [slot1, slot2]);
15654        assert_eq!(uncleaned_slots3, [slot1, slot2, slot3]);
15655    }
15656
15657    #[test]
15658    fn test_remove_uncleaned_slots_and_collect_pubkeys() {
15659        solana_logger::setup();
15660        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
15661
15662        let slot1 = 11;
15663        let slot2 = 222;
15664        let slot3 = 3333;
15665
15666        let pubkey1 = Pubkey::new_unique();
15667        let pubkey2 = Pubkey::new_unique();
15668        let pubkey3 = Pubkey::new_unique();
15669
15670        let account1 = AccountSharedData::new(0, 0, &pubkey1);
15671        let account2 = AccountSharedData::new(0, 0, &pubkey2);
15672        let account3 = AccountSharedData::new(0, 0, &pubkey3);
15673
15674        db.store_for_tests(slot1, &[(&pubkey1, &account1)]);
15675        db.store_for_tests(slot2, &[(&pubkey2, &account2)]);
15676        db.store_for_tests(slot3, &[(&pubkey3, &account3)]);
15677
15678        db.add_root(slot1);
15679        // slot 2 is _not_ a root on purpose
15680        db.add_root(slot3);
15681
15682        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
15683        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
15684        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
15685
15686        let uncleaned_pubkeys1 = db
15687            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot1])
15688            .into_iter()
15689            .flatten()
15690            .collect::<Vec<_>>();
15691        let uncleaned_pubkeys2 = db
15692            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot2])
15693            .into_iter()
15694            .flatten()
15695            .collect::<Vec<_>>();
15696        let uncleaned_pubkeys3 = db
15697            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot3])
15698            .into_iter()
15699            .flatten()
15700            .collect::<Vec<_>>();
15701
15702        assert!(uncleaned_pubkeys1.contains(&pubkey1));
15703        assert!(!uncleaned_pubkeys1.contains(&pubkey2));
15704        assert!(!uncleaned_pubkeys1.contains(&pubkey3));
15705
15706        assert!(!uncleaned_pubkeys2.contains(&pubkey1));
15707        assert!(uncleaned_pubkeys2.contains(&pubkey2));
15708        assert!(!uncleaned_pubkeys2.contains(&pubkey3));
15709
15710        assert!(!uncleaned_pubkeys3.contains(&pubkey1));
15711        assert!(!uncleaned_pubkeys3.contains(&pubkey2));
15712        assert!(uncleaned_pubkeys3.contains(&pubkey3));
15713    }
15714
15715    #[test]
15716    fn test_remove_uncleaned_slots_and_collect_pubkeys_up_to_slot() {
15717        solana_logger::setup();
15718        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
15719
15720        let slot1 = 11;
15721        let slot2 = 222;
15722        let slot3 = 3333;
15723
15724        let pubkey1 = Pubkey::new_unique();
15725        let pubkey2 = Pubkey::new_unique();
15726        let pubkey3 = Pubkey::new_unique();
15727
15728        let account1 = AccountSharedData::new(0, 0, &pubkey1);
15729        let account2 = AccountSharedData::new(0, 0, &pubkey2);
15730        let account3 = AccountSharedData::new(0, 0, &pubkey3);
15731
15732        db.store_for_tests(slot1, &[(&pubkey1, &account1)]);
15733        db.store_for_tests(slot2, &[(&pubkey2, &account2)]);
15734        db.store_for_tests(slot3, &[(&pubkey3, &account3)]);
15735
15736        // slot 1 is _not_ a root on purpose
15737        db.add_root(slot2);
15738        db.add_root(slot3);
15739
15740        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
15741        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
15742        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
15743
15744        let uncleaned_pubkeys = db
15745            .remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(slot3)
15746            .into_iter()
15747            .flatten()
15748            .collect::<Vec<_>>();
15749
15750        assert!(uncleaned_pubkeys.contains(&pubkey1));
15751        assert!(uncleaned_pubkeys.contains(&pubkey2));
15752        assert!(uncleaned_pubkeys.contains(&pubkey3));
15753    }
15754
15755    #[test]
15756    fn test_shrink_productive() {
15757        solana_logger::setup();
15758        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, 1024);
15759        let store = Arc::new(s1);
15760        assert!(!AccountsDb::is_shrinking_productive(0, &store));
15761
15762        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, PAGE_SIZE * 4);
15763        let store = Arc::new(s1);
15764        store.add_account((3 * PAGE_SIZE as usize) - 1);
15765        store.add_account(10);
15766        store.remove_account(10, false);
15767        assert!(AccountsDb::is_shrinking_productive(0, &store));
15768
15769        store.add_account(PAGE_SIZE as usize);
15770        assert!(!AccountsDb::is_shrinking_productive(0, &store));
15771    }
15772
15773    #[test]
15774    fn test_is_candidate_for_shrink() {
15775        solana_logger::setup();
15776
15777        let mut accounts = AccountsDb::new_single_for_tests();
15778        let common_store_path = Path::new("");
15779        let store_file_size = 2 * PAGE_SIZE;
15780        let entry = Arc::new(AccountStorageEntry::new(
15781            common_store_path,
15782            0,
15783            1,
15784            store_file_size,
15785        ));
15786        match accounts.shrink_ratio {
15787            AccountShrinkThreshold::TotalSpace { shrink_ratio } => {
15788                assert_eq!(
15789                    (DEFAULT_ACCOUNTS_SHRINK_RATIO * 100.) as u64,
15790                    (shrink_ratio * 100.) as u64
15791                )
15792            }
15793            AccountShrinkThreshold::IndividualStore { shrink_ratio: _ } => {
15794                panic!("Expect the default to be TotalSpace")
15795            }
15796        }
15797        entry.alive_bytes.store(3000, Ordering::Release);
15798        assert!(accounts.is_candidate_for_shrink(&entry, false));
15799        entry.alive_bytes.store(5000, Ordering::Release);
15800        assert!(!accounts.is_candidate_for_shrink(&entry, false));
15801        accounts.shrink_ratio = AccountShrinkThreshold::TotalSpace { shrink_ratio: 0.3 };
15802        entry.alive_bytes.store(3000, Ordering::Release);
15803        assert!(accounts.is_candidate_for_shrink(&entry, false));
15804        accounts.shrink_ratio = AccountShrinkThreshold::IndividualStore { shrink_ratio: 0.3 };
15805        assert!(!accounts.is_candidate_for_shrink(&entry, false));
15806    }
15807
15808    #[test]
15809    fn test_calculate_storage_count_and_alive_bytes() {
15810        let accounts = AccountsDb::new_single_for_tests();
15811        let shared_key = solana_sdk::pubkey::new_rand();
15812        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15813        let slot0 = 0;
15814        accounts.store_for_tests(slot0, &[(&shared_key, &account)]);
15815        accounts.add_root_and_flush_write_cache(slot0);
15816
15817        let storage = accounts.storage.get_slot_storage_entry(slot0).unwrap();
15818        let storage_info = StorageSizeAndCountMap::default();
15819        let accounts_map = accounts.process_storage_slot(&storage);
15820        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
15821        assert_eq!(storage_info.len(), 1);
15822        for entry in storage_info.iter() {
15823            assert_eq!(
15824                (entry.key(), entry.value().count, entry.value().stored_size),
15825                (&0, 1, 144)
15826            );
15827        }
15828    }
15829
15830    #[test]
15831    fn test_calculate_storage_count_and_alive_bytes_0_accounts() {
15832        let accounts = AccountsDb::new_single_for_tests();
15833        // empty store
15834        let storage = accounts.create_and_insert_store(0, 1, "test");
15835        let storage_info = StorageSizeAndCountMap::default();
15836        let accounts_map = accounts.process_storage_slot(&storage);
15837        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
15838        assert!(storage_info.is_empty());
15839    }
15840
15841    #[test]
15842    fn test_calculate_storage_count_and_alive_bytes_2_accounts() {
15843        let accounts = AccountsDb::new_single_for_tests();
15844        let keys = [
15845            solana_sdk::pubkey::Pubkey::from([0; 32]),
15846            solana_sdk::pubkey::Pubkey::from([255; 32]),
15847        ];
15848        // make sure accounts are in 2 different bins
15849        assert!(
15850            (accounts.accounts_index.bins() == 1)
15851                ^ (accounts
15852                    .accounts_index
15853                    .bin_calculator
15854                    .bin_from_pubkey(&keys[0])
15855                    != accounts
15856                        .accounts_index
15857                        .bin_calculator
15858                        .bin_from_pubkey(&keys[1]))
15859        );
15860        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15861        let account_big = AccountSharedData::new(1, 1000, AccountSharedData::default().owner());
15862        let slot0 = 0;
15863        accounts.store_for_tests(slot0, &[(&keys[0], &account)]);
15864        accounts.store_for_tests(slot0, &[(&keys[1], &account_big)]);
15865        accounts.add_root_and_flush_write_cache(slot0);
15866
15867        let storage = accounts.storage.get_slot_storage_entry(slot0).unwrap();
15868        let storage_info = StorageSizeAndCountMap::default();
15869        let accounts_map = accounts.process_storage_slot(&storage);
15870        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
15871        assert_eq!(storage_info.len(), 1);
15872        for entry in storage_info.iter() {
15873            assert_eq!(
15874                (entry.key(), entry.value().count, entry.value().stored_size),
15875                (&0, 2, 1280)
15876            );
15877        }
15878    }
15879
15880    #[test]
15881    fn test_set_storage_count_and_alive_bytes() {
15882        let accounts = AccountsDb::new_single_for_tests();
15883
15884        // make sure we have storage 0
15885        let shared_key = solana_sdk::pubkey::new_rand();
15886        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15887        let slot0 = 0;
15888        accounts.store_for_tests(slot0, &[(&shared_key, &account)]);
15889        accounts.add_root_and_flush_write_cache(slot0);
15890
15891        // fake out the store count to avoid the assert
15892        for (_, store) in accounts.storage.iter() {
15893            store.alive_bytes.store(0, Ordering::Release);
15894        }
15895
15896        // populate based on made up hash data
15897        let dashmap = DashMap::default();
15898        dashmap.insert(
15899            0,
15900            StorageSizeAndCount {
15901                stored_size: 2,
15902                count: 3,
15903            },
15904        );
15905        accounts.set_storage_count_and_alive_bytes(dashmap, &mut GenerateIndexTimings::default());
15906        assert_eq!(accounts.storage.len(), 1);
15907        for (_, store) in accounts.storage.iter() {
15908            assert_eq!(store.append_vec_id(), 0);
15909            assert_eq!(store.count_and_status.read().unwrap().0, 3);
15910            assert_eq!(store.alive_bytes.load(Ordering::Acquire), 2);
15911        }
15912    }
15913
15914    #[test]
15915    fn test_purge_alive_unrooted_slots_after_clean() {
15916        let accounts = AccountsDb::new_single_for_tests();
15917
15918        // Key shared between rooted and nonrooted slot
15919        let shared_key = solana_sdk::pubkey::new_rand();
15920        // Key to keep the storage entry for the unrooted slot alive
15921        let unrooted_key = solana_sdk::pubkey::new_rand();
15922        let slot0 = 0;
15923        let slot1 = 1;
15924
15925        // Store accounts with greater than 0 lamports
15926        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15927        accounts.store_for_tests(slot0, &[(&shared_key, &account)]);
15928        accounts.store_for_tests(slot0, &[(&unrooted_key, &account)]);
15929
15930        // Simulate adding dirty pubkeys on bank freeze. Note this is
15931        // not a rooted slot
15932        accounts.calculate_accounts_delta_hash(slot0);
15933
15934        // On the next *rooted* slot, update the `shared_key` account to zero lamports
15935        let zero_lamport_account =
15936            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
15937        accounts.store_for_tests(slot1, &[(&shared_key, &zero_lamport_account)]);
15938
15939        // Simulate adding dirty pubkeys on bank freeze, set root
15940        accounts.calculate_accounts_delta_hash(slot1);
15941        accounts.add_root_and_flush_write_cache(slot1);
15942
15943        // The later rooted zero-lamport update to `shared_key` cannot be cleaned
15944        // because it is kept alive by the unrooted slot.
15945        accounts.clean_accounts_for_tests();
15946        assert!(accounts
15947            .accounts_index
15948            .get_account_read_entry(&shared_key)
15949            .is_some());
15950
15951        // Simulate purge_slot() all from AccountsBackgroundService
15952        accounts.purge_slot(slot0, 0, true);
15953
15954        // Now clean should clean up the remaining key
15955        accounts.clean_accounts_for_tests();
15956        assert!(accounts
15957            .accounts_index
15958            .get_account_read_entry(&shared_key)
15959            .is_none());
15960        assert_no_storages_at_slot(&accounts, slot0);
15961    }
15962
15963    /// asserts that not only are there 0 append vecs, but there is not even an entry in the storage map for 'slot'
15964    fn assert_no_storages_at_slot(db: &AccountsDb, slot: Slot) {
15965        assert!(db.storage.get_slot_storage_entry(slot).is_none());
15966    }
15967
15968    /// Test to make sure `clean_accounts()` works properly with the `last_full_snapshot_slot`
15969    /// parameter.  Basically:
15970    ///
15971    /// - slot 1: set Account1's balance to non-zero
15972    /// - slot 2: set Account1's balance to a different non-zero amount
15973    /// - slot 3: set Account1's balance to zero
15974    /// - call `clean_accounts()` with `max_clean_root` set to 2
15975    ///     - ensure Account1 has *not* been purged
15976    ///     - ensure the store from slot 1 is cleaned up
15977    /// - call `clean_accounts()` with `last_full_snapshot_slot` set to 2
15978    ///     - ensure Account1 has *not* been purged
15979    /// - call `clean_accounts()` with `last_full_snapshot_slot` set to 3
15980    ///     - ensure Account1 *has* been purged
15981    #[test]
15982    fn test_clean_accounts_with_last_full_snapshot_slot() {
15983        solana_logger::setup();
15984        let accounts_db = AccountsDb::new_single_for_tests();
15985        let pubkey = solana_sdk::pubkey::new_rand();
15986        let owner = solana_sdk::pubkey::new_rand();
15987        let space = 0;
15988
15989        let slot1: Slot = 1;
15990        let account = AccountSharedData::new(111, space, &owner);
15991        accounts_db.store_cached((slot1, &[(&pubkey, &account)][..]), None);
15992        accounts_db.calculate_accounts_delta_hash(slot1);
15993        accounts_db.add_root_and_flush_write_cache(slot1);
15994
15995        let slot2: Slot = 2;
15996        let account = AccountSharedData::new(222, space, &owner);
15997        accounts_db.store_cached((slot2, &[(&pubkey, &account)][..]), None);
15998        accounts_db.calculate_accounts_delta_hash(slot2);
15999        accounts_db.add_root_and_flush_write_cache(slot2);
16000
16001        let slot3: Slot = 3;
16002        let account = AccountSharedData::new(0, space, &owner);
16003        accounts_db.store_cached((slot3, &[(&pubkey, &account)][..]), None);
16004        accounts_db.calculate_accounts_delta_hash(slot3);
16005        accounts_db.add_root_and_flush_write_cache(slot3);
16006
16007        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 3);
16008
16009        accounts_db.clean_accounts(Some(slot2), false, Some(slot2));
16010        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 2);
16011
16012        accounts_db.clean_accounts(None, false, Some(slot2));
16013        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 1);
16014
16015        accounts_db.clean_accounts(None, false, Some(slot3));
16016        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 0);
16017    }
16018
16019    #[test]
16020    fn test_filter_zero_lamport_clean_for_incremental_snapshots() {
16021        solana_logger::setup();
16022        let slot = 10;
16023
16024        struct TestParameters {
16025            last_full_snapshot_slot: Option<Slot>,
16026            max_clean_root: Option<Slot>,
16027            should_contain: bool,
16028        }
16029
16030        let do_test = |test_params: TestParameters| {
16031            let account_info = AccountInfo::new(StorageLocation::AppendVec(42, 128), 234, 0);
16032            let pubkey = solana_sdk::pubkey::new_rand();
16033            let mut key_set = HashSet::default();
16034            key_set.insert(pubkey);
16035            let store_count = 0;
16036            let mut store_counts = HashMap::default();
16037            store_counts.insert(account_info.store_id(), (store_count, key_set));
16038            let mut purges_zero_lamports = HashMap::default();
16039            purges_zero_lamports.insert(pubkey, (vec![(slot, account_info)], 1));
16040
16041            let accounts_db = AccountsDb::new_single_for_tests();
16042            accounts_db.filter_zero_lamport_clean_for_incremental_snapshots(
16043                test_params.max_clean_root,
16044                test_params.last_full_snapshot_slot,
16045                &store_counts,
16046                &mut purges_zero_lamports,
16047            );
16048
16049            assert_eq!(
16050                purges_zero_lamports.contains_key(&pubkey),
16051                test_params.should_contain
16052            );
16053        };
16054
16055        // Scenario 1: last full snapshot is NONE
16056        // In this scenario incremental snapshots are OFF, so always purge
16057        {
16058            let last_full_snapshot_slot = None;
16059
16060            do_test(TestParameters {
16061                last_full_snapshot_slot,
16062                max_clean_root: Some(slot),
16063                should_contain: true,
16064            });
16065
16066            do_test(TestParameters {
16067                last_full_snapshot_slot,
16068                max_clean_root: None,
16069                should_contain: true,
16070            });
16071        }
16072
16073        // Scenario 2: last full snapshot is GREATER THAN zero lamport account slot
16074        // In this scenario always purge, and just test the various permutations of
16075        // `should_filter_for_incremental_snapshots` based on `max_clean_root`.
16076        {
16077            let last_full_snapshot_slot = Some(slot + 1);
16078
16079            do_test(TestParameters {
16080                last_full_snapshot_slot,
16081                max_clean_root: last_full_snapshot_slot,
16082                should_contain: true,
16083            });
16084
16085            do_test(TestParameters {
16086                last_full_snapshot_slot,
16087                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
16088                should_contain: true,
16089            });
16090
16091            do_test(TestParameters {
16092                last_full_snapshot_slot,
16093                max_clean_root: None,
16094                should_contain: true,
16095            });
16096        }
16097
16098        // Scenario 3: last full snapshot is EQUAL TO zero lamport account slot
16099        // In this scenario always purge, as it's the same as Scenario 2.
16100        {
16101            let last_full_snapshot_slot = Some(slot);
16102
16103            do_test(TestParameters {
16104                last_full_snapshot_slot,
16105                max_clean_root: last_full_snapshot_slot,
16106                should_contain: true,
16107            });
16108
16109            do_test(TestParameters {
16110                last_full_snapshot_slot,
16111                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
16112                should_contain: true,
16113            });
16114
16115            do_test(TestParameters {
16116                last_full_snapshot_slot,
16117                max_clean_root: None,
16118                should_contain: true,
16119            });
16120        }
16121
16122        // Scenario 4: last full snapshot is LESS THAN zero lamport account slot
16123        // In this scenario do *not* purge, except when `should_filter_for_incremental_snapshots`
16124        // is false
16125        {
16126            let last_full_snapshot_slot = Some(slot - 1);
16127
16128            do_test(TestParameters {
16129                last_full_snapshot_slot,
16130                max_clean_root: last_full_snapshot_slot,
16131                should_contain: true,
16132            });
16133
16134            do_test(TestParameters {
16135                last_full_snapshot_slot,
16136                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
16137                should_contain: false,
16138            });
16139
16140            do_test(TestParameters {
16141                last_full_snapshot_slot,
16142                max_clean_root: None,
16143                should_contain: false,
16144            });
16145        }
16146    }
16147
16148    #[test]
16149    fn test_calc_alive_ancient_historical_roots() {
16150        let db = AccountsDb::new_single_for_tests();
16151        let min_root = 0;
16152        let result = db.calc_alive_ancient_historical_roots(min_root);
16153        assert!(result.is_empty());
16154        for extra in 1..3 {
16155            let result = db.calc_alive_ancient_historical_roots(extra);
16156            assert_eq!(result, HashSet::default(), "extra: {extra}");
16157        }
16158
16159        let extra = 3;
16160        let active_root = 2;
16161        db.accounts_index.add_root(active_root);
16162        let result = db.calc_alive_ancient_historical_roots(extra);
16163        let expected_alive_roots = [active_root].into_iter().collect();
16164        assert_eq!(result, expected_alive_roots, "extra: {extra}");
16165    }
16166
16167    impl AccountsDb {
16168        /// useful to adapt tests written prior to introduction of the write cache
16169        /// to use the write cache
16170        pub fn add_root_and_flush_write_cache(&self, slot: Slot) {
16171            self.add_root(slot);
16172            self.flush_root_write_cache(slot);
16173        }
16174
16175        /// useful to adapt tests written prior to introduction of the write cache
16176        /// to use the write cache
16177        fn flush_root_write_cache(&self, root: Slot) {
16178            assert!(
16179                self.accounts_index
16180                    .roots_tracker
16181                    .read()
16182                    .unwrap()
16183                    .alive_roots
16184                    .contains(&root),
16185                "slot: {root}"
16186            );
16187            self.flush_accounts_cache(true, Some(root));
16188        }
16189
16190        /// callers used to call store_uncached. But, this is not allowed anymore.
16191        pub fn store_for_tests(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
16192            self.store(
16193                (slot, accounts, INCLUDE_SLOT_IN_HASH_TESTS),
16194                &StoreTo::Cache,
16195                None,
16196                StoreReclaims::Default,
16197            );
16198        }
16199
16200        /// helper function to test unref_accounts or clean_dead_slots_from_accounts_index
16201        fn test_unref(
16202            &self,
16203            call_unref: bool,
16204            purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
16205            purged_stored_account_slots: &mut AccountSlots,
16206            pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
16207        ) {
16208            if call_unref {
16209                self.unref_accounts(
16210                    purged_slot_pubkeys,
16211                    purged_stored_account_slots,
16212                    pubkeys_removed_from_accounts_index,
16213                );
16214            } else {
16215                let empty_vec = Vec::default();
16216                self.clean_dead_slots_from_accounts_index(
16217                    empty_vec.iter(),
16218                    purged_slot_pubkeys,
16219                    Some(purged_stored_account_slots),
16220                    pubkeys_removed_from_accounts_index,
16221                );
16222            }
16223        }
16224    }
16225
16226    #[test]
16227    /// test 'unref' parameter 'pubkeys_removed_from_accounts_index'
16228    fn test_unref_pubkeys_removed_from_accounts_index() {
16229        let slot1 = 1;
16230        let pk1 = Pubkey::from([1; 32]);
16231        for already_removed in [false, true] {
16232            let mut pubkeys_removed_from_accounts_index =
16233                PubkeysRemovedFromAccountsIndex::default();
16234            if already_removed {
16235                pubkeys_removed_from_accounts_index.insert(pk1);
16236            }
16237            // pk1 in slot1, purge it
16238            let db = AccountsDb::new_single_for_tests();
16239            let mut purged_slot_pubkeys = HashSet::default();
16240            purged_slot_pubkeys.insert((slot1, pk1));
16241            let mut reclaims = SlotList::default();
16242            db.accounts_index.upsert(
16243                slot1,
16244                slot1,
16245                &pk1,
16246                &AccountSharedData::default(),
16247                &AccountSecondaryIndexes::default(),
16248                AccountInfo::default(),
16249                &mut reclaims,
16250                UpsertReclaim::IgnoreReclaims,
16251            );
16252
16253            let mut purged_stored_account_slots = AccountSlots::default();
16254            db.test_unref(
16255                true,
16256                purged_slot_pubkeys,
16257                &mut purged_stored_account_slots,
16258                &pubkeys_removed_from_accounts_index,
16259            );
16260            assert_eq!(
16261                vec![(pk1, vec![slot1].into_iter().collect::<HashSet<_>>())],
16262                purged_stored_account_slots.into_iter().collect::<Vec<_>>()
16263            );
16264            let expected = u64::from(already_removed);
16265            assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), expected);
16266        }
16267    }
16268
16269    #[test]
16270    fn test_unref_accounts() {
16271        let pubkeys_removed_from_accounts_index = PubkeysRemovedFromAccountsIndex::default();
16272        for call_unref in [false, true] {
16273            {
16274                let db = AccountsDb::new_single_for_tests();
16275                let mut purged_stored_account_slots = AccountSlots::default();
16276
16277                db.test_unref(
16278                    call_unref,
16279                    HashSet::default(),
16280                    &mut purged_stored_account_slots,
16281                    &pubkeys_removed_from_accounts_index,
16282                );
16283                assert!(purged_stored_account_slots.is_empty());
16284            }
16285
16286            let slot1 = 1;
16287            let slot2 = 2;
16288            let pk1 = Pubkey::from([1; 32]);
16289            let pk2 = Pubkey::from([2; 32]);
16290            {
16291                // pk1 in slot1, purge it
16292                let db = AccountsDb::new_single_for_tests();
16293                let mut purged_slot_pubkeys = HashSet::default();
16294                purged_slot_pubkeys.insert((slot1, pk1));
16295                let mut reclaims = SlotList::default();
16296                db.accounts_index.upsert(
16297                    slot1,
16298                    slot1,
16299                    &pk1,
16300                    &AccountSharedData::default(),
16301                    &AccountSecondaryIndexes::default(),
16302                    AccountInfo::default(),
16303                    &mut reclaims,
16304                    UpsertReclaim::IgnoreReclaims,
16305                );
16306
16307                let mut purged_stored_account_slots = AccountSlots::default();
16308                db.test_unref(
16309                    call_unref,
16310                    purged_slot_pubkeys,
16311                    &mut purged_stored_account_slots,
16312                    &pubkeys_removed_from_accounts_index,
16313                );
16314                assert_eq!(
16315                    vec![(pk1, vec![slot1].into_iter().collect::<HashSet<_>>())],
16316                    purged_stored_account_slots.into_iter().collect::<Vec<_>>()
16317                );
16318                assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), 0);
16319            }
16320            {
16321                let db = AccountsDb::new_single_for_tests();
16322                let mut purged_stored_account_slots = AccountSlots::default();
16323                let mut purged_slot_pubkeys = HashSet::default();
16324                let mut reclaims = SlotList::default();
16325                // pk1 and pk2 both in slot1 and slot2, so each has refcount of 2
16326                for slot in [slot1, slot2] {
16327                    for pk in [pk1, pk2] {
16328                        db.accounts_index.upsert(
16329                            slot,
16330                            slot,
16331                            &pk,
16332                            &AccountSharedData::default(),
16333                            &AccountSecondaryIndexes::default(),
16334                            AccountInfo::default(),
16335                            &mut reclaims,
16336                            UpsertReclaim::IgnoreReclaims,
16337                        );
16338                    }
16339                }
16340                // purge pk1 from both 1 and 2 and pk2 from slot 1
16341                let purges = vec![(slot1, pk1), (slot1, pk2), (slot2, pk1)];
16342                purges.into_iter().for_each(|(slot, pk)| {
16343                    purged_slot_pubkeys.insert((slot, pk));
16344                });
16345                db.test_unref(
16346                    call_unref,
16347                    purged_slot_pubkeys,
16348                    &mut purged_stored_account_slots,
16349                    &pubkeys_removed_from_accounts_index,
16350                );
16351                for (pk, slots) in vec![(pk1, vec![slot1, slot2]), (pk2, vec![slot1])] {
16352                    let result = purged_stored_account_slots.remove(&pk).unwrap();
16353                    assert_eq!(result, slots.into_iter().collect::<HashSet<_>>());
16354                }
16355                assert!(purged_stored_account_slots.is_empty());
16356                assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), 0);
16357                assert_eq!(db.accounts_index.ref_count_from_storage(&pk2), 1);
16358            }
16359        }
16360    }
16361
16362    #[test]
16363    fn test_many_unrefs() {
16364        let db = AccountsDb::new_single_for_tests();
16365        let mut purged_stored_account_slots = AccountSlots::default();
16366        let mut reclaims = SlotList::default();
16367        let pk1 = Pubkey::from([1; 32]);
16368        // make sure we have > 1 batch. Bigger numbers cost more in test time here.
16369        let n = (UNREF_ACCOUNTS_BATCH_SIZE + 1) as Slot;
16370        // put the pubkey into the acct idx in 'n' slots
16371        let purged_slot_pubkeys = (0..n)
16372            .map(|slot| {
16373                db.accounts_index.upsert(
16374                    slot,
16375                    slot,
16376                    &pk1,
16377                    &AccountSharedData::default(),
16378                    &AccountSecondaryIndexes::default(),
16379                    AccountInfo::default(),
16380                    &mut reclaims,
16381                    UpsertReclaim::IgnoreReclaims,
16382                );
16383                (slot, pk1)
16384            })
16385            .collect::<HashSet<_>>();
16386
16387        assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), n);
16388        // unref all 'n' slots
16389        db.unref_accounts(
16390            purged_slot_pubkeys,
16391            &mut purged_stored_account_slots,
16392            &HashSet::default(),
16393        );
16394        assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), 0);
16395    }
16396
16397    #[test]
16398    fn test_get_one_epoch_old_slot_for_hash_calc_scan() {
16399        let mut db = AccountsDb::new_single_for_tests();
16400        let config = CalcAccountsHashConfig::default();
16401        let slot = config.epoch_schedule.slots_per_epoch;
16402        assert_ne!(slot, 0);
16403        let offset = 10;
16404        assert_eq!(
16405            db.get_one_epoch_old_slot_for_hash_calc_scan(slot + offset, &config),
16406            0
16407        );
16408        db.ancient_append_vec_offset = Some(0);
16409        assert_eq!(
16410            db.get_one_epoch_old_slot_for_hash_calc_scan(slot, &config),
16411            0
16412        );
16413        assert_eq!(
16414            db.get_one_epoch_old_slot_for_hash_calc_scan(slot + offset, &config),
16415            offset
16416        );
16417    }
16418
16419    #[test]
16420    fn test_mark_dirty_dead_stores_empty() {
16421        let db = AccountsDb::new_single_for_tests();
16422        let slot = 0;
16423        for add_dirty_stores in [false, true] {
16424            let dead_storages = db.mark_dirty_dead_stores(slot, add_dirty_stores, None, false);
16425            assert!(dead_storages.is_empty());
16426            assert!(db.dirty_stores.is_empty());
16427        }
16428    }
16429
16430    #[test]
16431    fn test_mark_dirty_dead_stores_no_shrink_in_progress() {
16432        // None for shrink_in_progress, 1 existing store at the slot
16433        // There should be no more append vecs at that slot after the call to mark_dirty_dead_stores.
16434        // This tests the case where this slot was combined into an ancient append vec from an older slot and
16435        // there is no longer an append vec at this slot.
16436        for add_dirty_stores in [false, true] {
16437            let slot = 0;
16438            let db = AccountsDb::new_single_for_tests();
16439            let size = 1;
16440            let existing_store = db.create_and_insert_store(slot, size, "test");
16441            let old_id = existing_store.append_vec_id();
16442            let dead_storages = db.mark_dirty_dead_stores(slot, add_dirty_stores, None, false);
16443            assert!(db.storage.get_slot_storage_entry(slot).is_none());
16444            assert_eq!(dead_storages.len(), 1);
16445            assert_eq!(dead_storages.first().unwrap().append_vec_id(), old_id);
16446            if add_dirty_stores {
16447                assert_eq!(1, db.dirty_stores.len());
16448                let dirty_store = db.dirty_stores.get(&(slot, old_id)).unwrap();
16449                assert_eq!(dirty_store.append_vec_id(), old_id);
16450            } else {
16451                assert!(db.dirty_stores.is_empty());
16452            }
16453            assert!(db.storage.is_empty_entry(slot));
16454        }
16455    }
16456
16457    #[test]
16458    fn test_mark_dirty_dead_stores() {
16459        let slot = 0;
16460
16461        // use shrink_in_progress to cause us to drop the initial store
16462        for add_dirty_stores in [false, true] {
16463            let db = AccountsDb::new_single_for_tests();
16464            let size = 1;
16465            let old_store = db.create_and_insert_store(slot, size, "test");
16466            let old_id = old_store.append_vec_id();
16467            let shrink_in_progress = db.get_store_for_shrink(slot, 100);
16468            let dead_storages =
16469                db.mark_dirty_dead_stores(slot, add_dirty_stores, Some(shrink_in_progress), false);
16470            assert!(db.storage.get_slot_storage_entry(slot).is_some());
16471            assert_eq!(dead_storages.len(), 1);
16472            assert_eq!(dead_storages.first().unwrap().append_vec_id(), old_id);
16473            if add_dirty_stores {
16474                assert_eq!(1, db.dirty_stores.len());
16475                let dirty_store = db.dirty_stores.get(&(slot, old_id)).unwrap();
16476                assert_eq!(dirty_store.append_vec_id(), old_id);
16477            } else {
16478                assert!(db.dirty_stores.is_empty());
16479            }
16480            assert!(db.storage.get_slot_storage_entry(slot).is_some());
16481        }
16482    }
16483
16484    #[test]
16485    fn test_split_storages_ancient_chunks() {
16486        let storages = SortedStorages::empty();
16487        assert_eq!(storages.max_slot_inclusive(), 0);
16488        let result = SplitAncientStorages::new(0, &storages);
16489        assert_eq!(result, SplitAncientStorages::default());
16490    }
16491
16492    /// get all the ranges the splitter produces
16493    fn get_all_slot_ranges(splitter: &SplitAncientStorages) -> Vec<Option<Range<Slot>>> {
16494        (0..splitter.chunk_count)
16495            .map(|chunk| {
16496                assert_eq!(
16497                    splitter.get_starting_slot_from_normal_chunk(chunk),
16498                    if chunk == 0 {
16499                        splitter.normal_slot_range.start
16500                    } else {
16501                        (splitter.first_chunk_start + ((chunk as Slot) - 1) * MAX_ITEMS_PER_CHUNK)
16502                            .max(splitter.normal_slot_range.start)
16503                    },
16504                    "chunk: {chunk}, num_chunks: {}, splitter: {:?}",
16505                    splitter.chunk_count,
16506                    splitter,
16507                );
16508                splitter.get_slot_range(chunk)
16509            })
16510            .collect::<Vec<_>>()
16511    }
16512
16513    /// test function to make sure the split range covers exactly every slot in the original range
16514    fn verify_all_slots_covered_exactly_once(
16515        splitter: &SplitAncientStorages,
16516        overall_range: &Range<Slot>,
16517    ) {
16518        // verify all slots covered exactly once
16519        let result = get_all_slot_ranges(splitter);
16520        let mut expected = overall_range.start;
16521        result.iter().for_each(|range| {
16522            if let Some(range) = range {
16523                assert!(
16524                    overall_range.start == range.start || range.start % MAX_ITEMS_PER_CHUNK == 0
16525                );
16526                for slot in range.clone() {
16527                    assert_eq!(slot, expected);
16528                    expected += 1;
16529                }
16530            }
16531        });
16532        assert_eq!(expected, overall_range.end);
16533    }
16534
16535    /// new splitter for test
16536    /// without any ancient append vecs
16537    fn new_splitter(range: &Range<Slot>) -> SplitAncientStorages {
16538        let splitter =
16539            SplitAncientStorages::new_with_ancient_info(range, Vec::default(), range.start);
16540
16541        verify_all_slots_covered_exactly_once(&splitter, range);
16542
16543        splitter
16544    }
16545
16546    /// new splitter for test
16547    /// without any ancient append vecs
16548    fn new_splitter2(start: Slot, count: Slot) -> SplitAncientStorages {
16549        new_splitter(&Range {
16550            start,
16551            end: start + count,
16552        })
16553    }
16554
16555    #[test]
16556    fn test_split_storages_splitter_simple() {
16557        let plus_1 = MAX_ITEMS_PER_CHUNK + 1;
16558        let plus_2 = plus_1 + 1;
16559
16560        // starting at 0 is aligned with beginning, so 1st chunk is unnecessary since beginning slot starts at boundary
16561        // second chunk is the final chunk, which is not full (does not have 2500 entries)
16562        let splitter = new_splitter2(0, 1);
16563        let result = get_all_slot_ranges(&splitter);
16564        assert_eq!(result, [Some(0..1), None]);
16565
16566        // starting at 1 is not aligned with beginning, but since we don't have enough for a full chunk, it gets returned in the last chunk
16567        let splitter = new_splitter2(1, 1);
16568        let result = get_all_slot_ranges(&splitter);
16569        assert_eq!(result, [Some(1..2), None]);
16570
16571        // 1 full chunk, aligned
16572        let splitter = new_splitter2(0, MAX_ITEMS_PER_CHUNK);
16573        let result = get_all_slot_ranges(&splitter);
16574        assert_eq!(result, [Some(0..MAX_ITEMS_PER_CHUNK), None, None]);
16575
16576        // 1 full chunk + 1, aligned
16577        let splitter = new_splitter2(0, plus_1);
16578        let result = get_all_slot_ranges(&splitter);
16579        assert_eq!(
16580            result,
16581            [
16582                Some(0..MAX_ITEMS_PER_CHUNK),
16583                Some(MAX_ITEMS_PER_CHUNK..plus_1),
16584                None
16585            ]
16586        );
16587
16588        // 1 full chunk + 2, aligned
16589        let splitter = new_splitter2(0, plus_2);
16590        let result = get_all_slot_ranges(&splitter);
16591        assert_eq!(
16592            result,
16593            [
16594                Some(0..MAX_ITEMS_PER_CHUNK),
16595                Some(MAX_ITEMS_PER_CHUNK..plus_2),
16596                None
16597            ]
16598        );
16599
16600        // 1 full chunk, mis-aligned by 1
16601        let offset = 1;
16602        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK);
16603        let result = get_all_slot_ranges(&splitter);
16604        assert_eq!(
16605            result,
16606            [
16607                Some(offset..MAX_ITEMS_PER_CHUNK),
16608                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK + offset),
16609                None
16610            ]
16611        );
16612
16613        // starting at 1 is not aligned with beginning
16614        let offset = 1;
16615        let splitter = new_splitter2(offset, plus_1);
16616        let result = get_all_slot_ranges(&splitter);
16617        assert_eq!(
16618            result,
16619            [
16620                Some(offset..MAX_ITEMS_PER_CHUNK),
16621                Some(MAX_ITEMS_PER_CHUNK..plus_1 + offset),
16622                None
16623            ],
16624            "{splitter:?}"
16625        );
16626
16627        // 2 full chunks, aligned
16628        let offset = 0;
16629        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK * 2);
16630        let result = get_all_slot_ranges(&splitter);
16631        assert_eq!(
16632            result,
16633            [
16634                Some(offset..MAX_ITEMS_PER_CHUNK),
16635                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK * 2),
16636                None,
16637                None
16638            ],
16639            "{splitter:?}"
16640        );
16641
16642        // 2 full chunks + 1, mis-aligned
16643        let offset = 1;
16644        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK * 2);
16645        let result = get_all_slot_ranges(&splitter);
16646        assert_eq!(
16647            result,
16648            [
16649                Some(offset..MAX_ITEMS_PER_CHUNK),
16650                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK * 2),
16651                Some(MAX_ITEMS_PER_CHUNK * 2..MAX_ITEMS_PER_CHUNK * 2 + offset),
16652                None,
16653            ],
16654            "{splitter:?}"
16655        );
16656
16657        // 3 full chunks - 1, mis-aligned by 2
16658        // we need ALL the chunks here
16659        let offset = 2;
16660        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK * 3 - 1);
16661        let result = get_all_slot_ranges(&splitter);
16662        assert_eq!(
16663            result,
16664            [
16665                Some(offset..MAX_ITEMS_PER_CHUNK),
16666                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK * 2),
16667                Some(MAX_ITEMS_PER_CHUNK * 2..MAX_ITEMS_PER_CHUNK * 3),
16668                Some(MAX_ITEMS_PER_CHUNK * 3..MAX_ITEMS_PER_CHUNK * 3 + 1),
16669            ],
16670            "{splitter:?}"
16671        );
16672
16673        // 1 full chunk - 1, mis-aligned by 2
16674        // we need ALL the chunks here
16675        let offset = 2;
16676        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
16677        let result = get_all_slot_ranges(&splitter);
16678        assert_eq!(
16679            result,
16680            [
16681                Some(offset..MAX_ITEMS_PER_CHUNK),
16682                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK + 1),
16683            ],
16684            "{splitter:?}"
16685        );
16686
16687        // 1 full chunk - 1, aligned at big offset
16688        // huge offset
16689        // we need ALL the chunks here
16690        let offset = MAX_ITEMS_PER_CHUNK * 100;
16691        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
16692        let result = get_all_slot_ranges(&splitter);
16693        assert_eq!(
16694            result,
16695            [Some(offset..MAX_ITEMS_PER_CHUNK * 101 - 1), None,],
16696            "{splitter:?}"
16697        );
16698
16699        // 1 full chunk - 1, mis-aligned by 2 at big offset
16700        // huge offset
16701        // we need ALL the chunks here
16702        let offset = MAX_ITEMS_PER_CHUNK * 100 + 2;
16703        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
16704        let result = get_all_slot_ranges(&splitter);
16705        assert_eq!(
16706            result,
16707            [
16708                Some(offset..MAX_ITEMS_PER_CHUNK * 101),
16709                Some(MAX_ITEMS_PER_CHUNK * 101..MAX_ITEMS_PER_CHUNK * 101 + 1),
16710            ],
16711            "{splitter:?}"
16712        );
16713    }
16714
16715    #[test]
16716    fn test_split_storages_splitter_large_offset() {
16717        solana_logger::setup();
16718        // 1 full chunk - 1, mis-aligned by 2 at big offset
16719        // huge offset
16720        // we need ALL the chunks here
16721        let offset = MAX_ITEMS_PER_CHUNK * 100 + 2;
16722        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
16723        let result = get_all_slot_ranges(&splitter);
16724        assert_eq!(
16725            result,
16726            [
16727                Some(offset..MAX_ITEMS_PER_CHUNK * 101),
16728                Some(MAX_ITEMS_PER_CHUNK * 101..MAX_ITEMS_PER_CHUNK * 101 + 1),
16729            ],
16730            "{splitter:?}"
16731        );
16732    }
16733
16734    #[test]
16735    fn test_split_storages_parametric_splitter() {
16736        for offset_multiplier in [1, 1000] {
16737            for offset in [
16738                0,
16739                1,
16740                2,
16741                MAX_ITEMS_PER_CHUNK - 2,
16742                MAX_ITEMS_PER_CHUNK - 1,
16743                MAX_ITEMS_PER_CHUNK,
16744                MAX_ITEMS_PER_CHUNK + 1,
16745            ] {
16746                for full_chunks in [0, 1, 2, 3] {
16747                    for reduced_items in [0, 1, 2] {
16748                        for added_items in [0, 1, 2] {
16749                            // this will verify the entire range correctly
16750                            _ = new_splitter2(
16751                                offset * offset_multiplier,
16752                                (full_chunks * MAX_ITEMS_PER_CHUNK + added_items)
16753                                    .saturating_sub(reduced_items),
16754                            );
16755                        }
16756                    }
16757                }
16758            }
16759        }
16760    }
16761
16762    #[test]
16763    fn test_add_uncleaned_pubkeys_after_shrink() {
16764        let db = AccountsDb::new_single_for_tests();
16765        let slot = 0;
16766        let pubkey = Pubkey::from([1; 32]);
16767        db.add_uncleaned_pubkeys_after_shrink(slot, vec![pubkey].into_iter());
16768        assert_eq!(&*db.uncleaned_pubkeys.get(&slot).unwrap(), &vec![pubkey]);
16769    }
16770
16771    #[test]
16772    fn test_get_ancient_slots() {
16773        // test permutations of ancient, non-ancient, ancient with sparse slot #s and not
16774        for sparse in [false, true] {
16775            let (slot1_ancient, slot2, slot3_ancient, slot1_plus_ancient) = if sparse {
16776                (1, 10, 20, 5)
16777            } else {
16778                // we only test with 2 ancient append vecs when sparse
16779                (1, 2, 3, 4 /* irrelevant */)
16780            };
16781
16782            let db = AccountsDb::new_single_for_tests();
16783            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
16784            let _existing_append_vec = db.create_and_insert_store(slot1_ancient, 1000, "test");
16785
16786            let ancient = db
16787                .get_store_for_shrink(slot1_ancient, get_ancient_append_vec_capacity())
16788                .new_storage()
16789                .clone();
16790            let _existing_append_vec = db.create_and_insert_store(slot1_plus_ancient, 1000, "test");
16791            let ancient_1_plus = db
16792                .get_store_for_shrink(slot1_plus_ancient, get_ancient_append_vec_capacity())
16793                .new_storage()
16794                .clone();
16795            let _existing_append_vec = db.create_and_insert_store(slot3_ancient, 1000, "test");
16796            let ancient3 =
16797                db.get_store_for_shrink(slot3_ancient, get_ancient_append_vec_capacity());
16798            let temp_dir = TempDir::new().unwrap();
16799            let path = temp_dir.path();
16800            let id = 1;
16801            let size = 1;
16802            let non_ancient_storage = Arc::new(AccountStorageEntry::new(path, slot2, id, size));
16803            let raw_storages = vec![non_ancient_storage.clone()];
16804            let snapshot_storages = SortedStorages::new(&raw_storages);
16805            // test without an ancient append vec
16806            let one_epoch_old_slot = 0;
16807            let ancient_slots =
16808                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16809            assert_eq!(Vec::<Slot>::default(), ancient_slots);
16810            let one_epoch_old_slot = 3;
16811            let ancient_slots =
16812                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16813            assert_eq!(Vec::<Slot>::default(), ancient_slots);
16814
16815            // now test with an ancient append vec
16816            let raw_storages = vec![ancient.clone()];
16817            let snapshot_storages = SortedStorages::new(&raw_storages);
16818            let one_epoch_old_slot = 0;
16819            let ancient_slots =
16820                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16821            assert_eq!(Vec::<Slot>::default(), ancient_slots);
16822            let one_epoch_old_slot = slot2 + 1;
16823            let ancient_slots =
16824                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16825            assert_eq!(vec![slot1_ancient], ancient_slots);
16826
16827            // now test with an ancient append vec and then a non-ancient append vec
16828            let raw_storages = vec![ancient.clone(), non_ancient_storage.clone()];
16829            let snapshot_storages = SortedStorages::new(&raw_storages);
16830            let one_epoch_old_slot = 0;
16831            let ancient_slots =
16832                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16833            assert_eq!(Vec::<Slot>::default(), ancient_slots);
16834            let one_epoch_old_slot = slot2 + 1;
16835            let ancient_slots =
16836                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16837            assert_eq!(vec![slot1_ancient], ancient_slots);
16838
16839            // ancient, non-ancient, ancient
16840            let raw_storages = vec![
16841                ancient.clone(),
16842                non_ancient_storage.clone(),
16843                ancient3.new_storage().clone(),
16844            ];
16845            let snapshot_storages = SortedStorages::new(&raw_storages);
16846            let one_epoch_old_slot = 0;
16847            let ancient_slots =
16848                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16849            assert_eq!(Vec::<Slot>::default(), ancient_slots);
16850            let one_epoch_old_slot = slot3_ancient + 1;
16851            let ancient_slots =
16852                SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16853            assert_eq!(vec![slot1_ancient], ancient_slots);
16854
16855            if sparse {
16856                // ancient, ancient, non-ancient, ancient
16857                let raw_storages = vec![
16858                    Arc::clone(&ancient),
16859                    Arc::clone(&ancient_1_plus),
16860                    non_ancient_storage,
16861                    Arc::clone(ancient3.new_storage()),
16862                ];
16863                let snapshot_storages = SortedStorages::new(&raw_storages[..]);
16864                let one_epoch_old_slot = 0;
16865                let ancient_slots =
16866                    SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16867                assert_eq!(Vec::<Slot>::default(), ancient_slots);
16868                let one_epoch_old_slot = slot3_ancient + 1;
16869                let ancient_slots =
16870                    SplitAncientStorages::get_ancient_slots(one_epoch_old_slot, &snapshot_storages);
16871                assert_eq!(vec![slot1_ancient, slot1_plus_ancient], ancient_slots);
16872            }
16873        }
16874    }
16875
16876    #[test]
16877    fn test_hash_storage_info() {
16878        {
16879            let mut hasher = std::collections::hash_map::DefaultHasher::new();
16880            let storages = None;
16881            let slot = 1;
16882            let load = AccountsDb::hash_storage_info(&mut hasher, storages, slot);
16883            let hash = hasher.finish();
16884            assert_eq!(15130871412783076140, hash);
16885            assert!(load);
16886        }
16887        {
16888            let mut hasher = std::collections::hash_map::DefaultHasher::new();
16889            let slot: Slot = 0;
16890            let tf = crate::append_vec::test_utils::get_append_vec_path(
16891                "test_accountsdb_scan_account_storage_no_bank",
16892            );
16893            let write_version1 = 0;
16894            let pubkey1 = solana_sdk::pubkey::new_rand();
16895            let mark_alive = false;
16896            let storage =
16897                sample_storage_with_entries(&tf, write_version1, slot, &pubkey1, mark_alive);
16898
16899            let load = AccountsDb::hash_storage_info(&mut hasher, Some(&storage), slot);
16900            let hash = hasher.finish();
16901            // can't assert hash here - it is a function of mod date
16902            assert!(load);
16903            let slot = 2; // changed this
16904            let mut hasher = std::collections::hash_map::DefaultHasher::new();
16905            let load = AccountsDb::hash_storage_info(&mut hasher, Some(&storage), slot);
16906            let hash2 = hasher.finish();
16907            assert_ne!(hash, hash2); // slot changed, these should be different
16908                                     // can't assert hash here - it is a function of mod date
16909            assert!(load);
16910            let mut hasher = std::collections::hash_map::DefaultHasher::new();
16911            append_sample_data_to_storage(
16912                &storage,
16913                &solana_sdk::pubkey::new_rand(),
16914                write_version1,
16915                false,
16916                None,
16917            );
16918            let load = AccountsDb::hash_storage_info(&mut hasher, Some(&storage), slot);
16919            let hash3 = hasher.finish();
16920            assert_ne!(hash2, hash3); // moddate and written size changed
16921                                      // can't assert hash here - it is a function of mod date
16922            assert!(load);
16923            let mut hasher = std::collections::hash_map::DefaultHasher::new();
16924            let load = AccountsDb::hash_storage_info(&mut hasher, Some(&storage), slot);
16925            let hash4 = hasher.finish();
16926            assert_eq!(hash4, hash3); // same
16927                                      // can't assert hash here - it is a function of mod date
16928            assert!(load);
16929        }
16930    }
16931
16932    #[test]
16933    fn test_get_accounts_hash_complete_one_epoch_old() {
16934        let db = AccountsDb::new_single_for_tests();
16935        assert_eq!(db.get_accounts_hash_complete_one_epoch_old(), 0);
16936        let epoch_schedule = EpochSchedule::default();
16937        let completed_slot = epoch_schedule.slots_per_epoch;
16938        db.notify_accounts_hash_calculated_complete(completed_slot, &epoch_schedule);
16939        assert_eq!(db.get_accounts_hash_complete_one_epoch_old(), 0);
16940        let offset = 1;
16941        let completed_slot = completed_slot + offset;
16942        db.notify_accounts_hash_calculated_complete(completed_slot, &epoch_schedule);
16943        let earliest = AccountsDb::get_slot_one_epoch_prior(completed_slot, &epoch_schedule);
16944        assert_eq!(db.get_accounts_hash_complete_one_epoch_old(), earliest);
16945        let offset = 5;
16946        let completed_slot = completed_slot + offset;
16947        db.notify_accounts_hash_calculated_complete(completed_slot, &epoch_schedule);
16948        let earliest = AccountsDb::get_slot_one_epoch_prior(completed_slot, &epoch_schedule);
16949        assert_eq!(db.get_accounts_hash_complete_one_epoch_old(), earliest);
16950    }
16951
16952    #[test]
16953    #[should_panic(expected = "called `Option::unwrap()` on a `None` value")]
16954    fn test_current_ancient_slot_assert() {
16955        let current_ancient = CurrentAncientAppendVec::default();
16956        _ = current_ancient.slot();
16957    }
16958
16959    #[test]
16960    #[should_panic(expected = "called `Option::unwrap()` on a `None` value")]
16961    fn test_current_ancient_append_vec_assert() {
16962        let current_ancient = CurrentAncientAppendVec::default();
16963        _ = current_ancient.append_vec();
16964    }
16965
16966    #[test]
16967    fn test_current_ancient_simple() {
16968        let slot = 1;
16969        let slot2 = 2;
16970        let slot3 = 3;
16971        {
16972            // new
16973            let db = AccountsDb::new_single_for_tests();
16974            let size = 1000;
16975            let append_vec = db.create_and_insert_store(slot, size, "test");
16976            let mut current_ancient = CurrentAncientAppendVec::new(slot, append_vec.clone());
16977            assert_eq!(current_ancient.slot(), slot);
16978            assert_eq!(current_ancient.append_vec_id(), append_vec.append_vec_id());
16979            assert_eq!(
16980                current_ancient.append_vec().append_vec_id(),
16981                append_vec.append_vec_id()
16982            );
16983
16984            let _shrink_in_progress = current_ancient.create_if_necessary(slot2, &db);
16985            assert_eq!(current_ancient.slot(), slot);
16986            assert_eq!(current_ancient.append_vec_id(), append_vec.append_vec_id());
16987        }
16988
16989        {
16990            // create_if_necessary
16991            let db = AccountsDb::new_single_for_tests();
16992            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
16993            let _existing_append_vec = db.create_and_insert_store(slot2, 1000, "test");
16994
16995            let mut current_ancient = CurrentAncientAppendVec::default();
16996            let mut _shrink_in_progress = current_ancient.create_if_necessary(slot2, &db);
16997            let id = current_ancient.append_vec_id();
16998            assert_eq!(current_ancient.slot(), slot2);
16999            assert!(is_ancient(&current_ancient.append_vec().accounts));
17000            let slot3 = 3;
17001            // should do nothing
17002            let _shrink_in_progress = current_ancient.create_if_necessary(slot3, &db);
17003            assert_eq!(current_ancient.slot(), slot2);
17004            assert_eq!(current_ancient.append_vec_id(), id);
17005            assert!(is_ancient(&current_ancient.append_vec().accounts));
17006        }
17007
17008        {
17009            // create_ancient_append_vec
17010            let db = AccountsDb::new_single_for_tests();
17011            let mut current_ancient = CurrentAncientAppendVec::default();
17012            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17013            let _existing_append_vec = db.create_and_insert_store(slot2, 1000, "test");
17014
17015            {
17016                let _shrink_in_progress = current_ancient.create_ancient_append_vec(slot2, &db);
17017            }
17018            let id = current_ancient.append_vec_id();
17019            assert_eq!(current_ancient.slot(), slot2);
17020            assert!(is_ancient(&current_ancient.append_vec().accounts));
17021
17022            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17023            let _existing_append_vec = db.create_and_insert_store(slot3, 1000, "test");
17024
17025            let mut _shrink_in_progress = current_ancient.create_ancient_append_vec(slot3, &db);
17026            assert_eq!(current_ancient.slot(), slot3);
17027            assert!(is_ancient(&current_ancient.append_vec().accounts));
17028            assert_ne!(current_ancient.append_vec_id(), id);
17029        }
17030    }
17031
17032    #[test]
17033    fn test_get_sorted_potential_ancient_slots() {
17034        let db = AccountsDb::new_single_for_tests();
17035        assert!(db.get_sorted_potential_ancient_slots().is_empty());
17036        let root0 = 0;
17037        db.add_root(root0);
17038        let root1 = 1;
17039        let root2 = 2;
17040        db.add_root(root1);
17041        assert!(db.get_sorted_potential_ancient_slots().is_empty());
17042        let epoch_schedule = EpochSchedule::default();
17043        let completed_slot = epoch_schedule.slots_per_epoch;
17044        db.notify_accounts_hash_calculated_complete(completed_slot, &epoch_schedule);
17045        // get_sorted_potential_ancient_slots uses 'less than' as opposed to 'less or equal'
17046        // so, we need to get more than an epoch away to get the first valid root
17047        assert!(db.get_sorted_potential_ancient_slots().is_empty());
17048        let completed_slot = epoch_schedule.slots_per_epoch + root1;
17049        db.notify_accounts_hash_calculated_complete(completed_slot, &epoch_schedule);
17050        assert_eq!(db.get_sorted_potential_ancient_slots(), vec![root0]);
17051        let completed_slot = epoch_schedule.slots_per_epoch + root2;
17052        db.notify_accounts_hash_calculated_complete(completed_slot, &epoch_schedule);
17053        assert_eq!(db.get_sorted_potential_ancient_slots(), vec![root0, root1]);
17054        db.accounts_index
17055            .roots_tracker
17056            .write()
17057            .unwrap()
17058            .alive_roots
17059            .remove(&root0);
17060        assert_eq!(db.get_sorted_potential_ancient_slots(), vec![root1]);
17061    }
17062
17063    #[test]
17064    fn test_shrink_collect_simple() {
17065        solana_logger::setup();
17066        let account_counts = [
17067            1,
17068            SHRINK_COLLECT_CHUNK_SIZE,
17069            SHRINK_COLLECT_CHUNK_SIZE + 1,
17070            SHRINK_COLLECT_CHUNK_SIZE * 2,
17071        ];
17072        // 2 = append_opposite_alive_account + append_opposite_zero_lamport_account
17073        let max_appended_accounts = 2;
17074        let max_num_accounts = *account_counts.iter().max().unwrap();
17075        let pubkeys = (0..(max_num_accounts + max_appended_accounts))
17076            .map(|_| solana_sdk::pubkey::new_rand())
17077            .collect::<Vec<_>>();
17078        // write accounts, maybe remove from index
17079        // check shrink_collect results
17080        for lamports in [0, 1] {
17081            for space in [0, 8] {
17082                if lamports == 0 && space != 0 {
17083                    // illegal - zero lamport accounts are written with 0 space
17084                    continue;
17085                }
17086                for alive in [false, true] {
17087                    for append_opposite_alive_account in [false, true] {
17088                        for append_opposite_zero_lamport_account in [true, false] {
17089                            for mut account_count in account_counts {
17090                                let mut normal_account_count = account_count;
17091                                let mut pubkey_opposite_zero_lamports = None;
17092                                if append_opposite_zero_lamport_account {
17093                                    pubkey_opposite_zero_lamports = Some(&pubkeys[account_count]);
17094                                    normal_account_count += 1;
17095                                    account_count += 1;
17096                                }
17097                                let mut pubkey_opposite_alive = None;
17098                                if append_opposite_alive_account {
17099                                    // this needs to happen AFTER append_opposite_zero_lamport_account
17100                                    pubkey_opposite_alive = Some(&pubkeys[account_count]);
17101                                    account_count += 1;
17102                                }
17103                                debug!("space: {space}, lamports: {lamports}, alive: {alive}, account_count: {account_count}, append_opposite_alive_account: {append_opposite_alive_account}, append_opposite_zero_lamport_account: {append_opposite_zero_lamport_account}, normal_account_count: {normal_account_count}");
17104                                let db = AccountsDb::new_single_for_tests();
17105                                let slot5 = 5;
17106                                let mut account = AccountSharedData::new(
17107                                    lamports,
17108                                    space,
17109                                    AccountSharedData::default().owner(),
17110                                );
17111                                let mut to_purge = Vec::default();
17112                                for pubkey in pubkeys.iter().take(account_count) {
17113                                    // store in append vec and index
17114                                    let old_lamports = account.lamports();
17115                                    if Some(pubkey) == pubkey_opposite_zero_lamports {
17116                                        account.set_lamports(u64::from(old_lamports == 0));
17117                                    }
17118
17119                                    db.store_for_tests(slot5, &[(pubkey, &account)]);
17120                                    account.set_lamports(old_lamports);
17121                                    let mut alive = alive;
17122                                    if append_opposite_alive_account
17123                                        && Some(pubkey) == pubkey_opposite_alive
17124                                    {
17125                                        // invert this for one special pubkey
17126                                        alive = !alive;
17127                                    }
17128                                    if !alive {
17129                                        // remove from index so pubkey is 'dead'
17130                                        to_purge.push(*pubkey);
17131                                    }
17132                                }
17133                                db.add_root_and_flush_write_cache(slot5);
17134                                to_purge.iter().for_each(|pubkey| {
17135                                    db.accounts_index.purge_exact(
17136                                        pubkey,
17137                                        &([slot5].into_iter().collect::<HashSet<_>>()),
17138                                        &mut Vec::default(),
17139                                    );
17140                                });
17141
17142                                let storage = db.get_storage_for_slot(slot5).unwrap();
17143                                let unique_accounts = db
17144                                    .get_unique_accounts_from_storage_for_shrink(
17145                                        &storage,
17146                                        &ShrinkStats::default(),
17147                                    );
17148
17149                                let shrink_collect = db.shrink_collect::<AliveAccounts<'_>>(
17150                                    &storage,
17151                                    &unique_accounts,
17152                                    &ShrinkStats::default(),
17153                                );
17154                                let expect_single_opposite_alive_account =
17155                                    if append_opposite_alive_account {
17156                                        vec![*pubkey_opposite_alive.unwrap()]
17157                                    } else {
17158                                        vec![]
17159                                    };
17160
17161                                let expected_alive_accounts = if alive {
17162                                    pubkeys[..normal_account_count]
17163                                        .iter()
17164                                        .filter(|p| Some(p) != pubkey_opposite_alive.as_ref())
17165                                        .sorted()
17166                                        .cloned()
17167                                        .collect::<Vec<_>>()
17168                                } else {
17169                                    expect_single_opposite_alive_account.clone()
17170                                };
17171
17172                                let expected_unrefed = if alive {
17173                                    expect_single_opposite_alive_account.clone()
17174                                } else {
17175                                    pubkeys[..normal_account_count]
17176                                        .iter()
17177                                        .sorted()
17178                                        .cloned()
17179                                        .collect::<Vec<_>>()
17180                                };
17181
17182                                assert_eq!(shrink_collect.slot, slot5);
17183
17184                                assert_eq!(
17185                                    shrink_collect
17186                                        .alive_accounts
17187                                        .accounts
17188                                        .iter()
17189                                        .map(|account| *account.pubkey())
17190                                        .sorted()
17191                                        .collect::<Vec<_>>(),
17192                                    expected_alive_accounts
17193                                );
17194                                assert_eq!(
17195                                    shrink_collect
17196                                        .unrefed_pubkeys
17197                                        .iter()
17198                                        .sorted()
17199                                        .cloned()
17200                                        .cloned()
17201                                        .collect::<Vec<_>>(),
17202                                    expected_unrefed
17203                                );
17204
17205                                let alive_total_one_account = 136 + space;
17206                                if alive {
17207                                    assert_eq!(
17208                                        shrink_collect.aligned_total_bytes,
17209                                        PAGE_SIZE
17210                                            * if account_count >= 100 {
17211                                                4
17212                                            } else if account_count >= 50 {
17213                                                2
17214                                            } else {
17215                                                1
17216                                            }
17217                                    );
17218                                    let mut expected_alive_total_bytes =
17219                                        alive_total_one_account * normal_account_count;
17220                                    if append_opposite_zero_lamport_account {
17221                                        // zero lamport accounts store size=0 data
17222                                        expected_alive_total_bytes -= space;
17223                                    }
17224                                    assert_eq!(
17225                                        shrink_collect.alive_total_bytes,
17226                                        expected_alive_total_bytes
17227                                    );
17228                                } else if append_opposite_alive_account {
17229                                    assert_eq!(shrink_collect.aligned_total_bytes, 4096);
17230                                    assert_eq!(
17231                                        shrink_collect.alive_total_bytes,
17232                                        alive_total_one_account
17233                                    );
17234                                } else {
17235                                    assert_eq!(shrink_collect.aligned_total_bytes, 0);
17236                                    assert_eq!(shrink_collect.alive_total_bytes, 0);
17237                                }
17238                                // these constants are multiples of page size (4096).
17239                                // They are determined by what size append vec gets created when the write cache is flushed to an append vec.
17240                                // Thus, they are dependent on the # of accounts that are written. They were identified by hitting the asserts and noting the value
17241                                // for shrink_collect.original_bytes at each account_count and then encoding it here.
17242                                let expected_capacity = if account_count >= 100 {
17243                                    16384
17244                                } else if account_count >= 50 {
17245                                    8192
17246                                } else {
17247                                    4096
17248                                };
17249                                assert_eq!(shrink_collect.capacity, expected_capacity);
17250                                assert_eq!(shrink_collect.total_starting_accounts, account_count);
17251                                let mut expected_all_are_zero_lamports = lamports == 0;
17252                                if !append_opposite_alive_account {
17253                                    expected_all_are_zero_lamports |= !alive;
17254                                }
17255                                if append_opposite_zero_lamport_account && lamports == 0 && alive {
17256                                    expected_all_are_zero_lamports =
17257                                        !expected_all_are_zero_lamports;
17258                                }
17259                                assert_eq!(
17260                                    shrink_collect.all_are_zero_lamports,
17261                                    expected_all_are_zero_lamports
17262                                );
17263                            }
17264                        }
17265                    }
17266                }
17267            }
17268        }
17269    }
17270
17271    pub(crate) const CAN_RANDOMLY_SHRINK_FALSE: bool = false;
17272
17273    #[test]
17274    fn test_combine_ancient_slots_empty() {
17275        solana_logger::setup();
17276        let db = AccountsDb::new_single_for_tests();
17277        // empty slots
17278        db.combine_ancient_slots(Vec::default(), CAN_RANDOMLY_SHRINK_FALSE);
17279    }
17280
17281    #[test]
17282    fn test_combine_ancient_slots_simple() {
17283        for alive in [false, true] {
17284            _ = get_one_ancient_append_vec_and_others(alive, 0);
17285        }
17286    }
17287
17288    pub(crate) fn get_all_accounts(
17289        db: &AccountsDb,
17290        slots: Range<Slot>,
17291    ) -> Vec<(Pubkey, AccountSharedData)> {
17292        slots
17293            .filter_map(|slot| {
17294                let storage = db.storage.get_slot_storage_entry(slot);
17295                storage.map(|storage| {
17296                    storage
17297                        .accounts
17298                        .account_iter()
17299                        .map(|account| (*account.pubkey(), account.to_account_shared_data()))
17300                        .collect::<Vec<_>>()
17301                })
17302            })
17303            .flatten()
17304            .collect::<Vec<_>>()
17305    }
17306
17307    pub(crate) fn compare_all_accounts(
17308        one: &[(Pubkey, AccountSharedData)],
17309        two: &[(Pubkey, AccountSharedData)],
17310    ) {
17311        let mut failures = 0;
17312        let mut two_indexes = (0..two.len()).collect::<Vec<_>>();
17313        one.iter().for_each(|(pubkey, account)| {
17314            for i in 0..two_indexes.len() {
17315                let pubkey2 = two[two_indexes[i]].0;
17316                if pubkey2 == *pubkey {
17317                    if !accounts_equal(account, &two[two_indexes[i]].1) {
17318                        failures += 1;
17319                    }
17320                    two_indexes.remove(i);
17321                    break;
17322                }
17323            }
17324        });
17325        // helper method to reduce the volume of logged data to help identify differences
17326        // modify this when you hit a failure
17327        let clean = |accounts: &[(Pubkey, AccountSharedData)]| {
17328            accounts
17329                .iter()
17330                .map(|(_pubkey, account)| account.lamports())
17331                .collect::<Vec<_>>()
17332        };
17333        assert_eq!(
17334            failures,
17335            0,
17336            "one: {:?}, two: {:?}, two_indexes: {:?}",
17337            clean(one),
17338            clean(two),
17339            two_indexes,
17340        );
17341        assert!(
17342            two_indexes.is_empty(),
17343            "one: {one:?}, two: {two:?}, two_indexes: {two_indexes:?}"
17344        );
17345    }
17346
17347    #[test]
17348    fn test_shrink_ancient_overflow() {
17349        solana_logger::setup();
17350
17351        let num_normal_slots = 2;
17352        // build an ancient append vec at slot 'ancient_slot'
17353        let (db, ancient_slot) = get_one_ancient_append_vec_and_others(true, num_normal_slots);
17354
17355        let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
17356        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
17357
17358        let ancient = db.storage.get_slot_storage_entry(ancient_slot).unwrap();
17359        let initial_len = ancient.alive_bytes();
17360        // set size of ancient to be 'full'
17361        adjust_append_vec_len_for_tests(&ancient, ancient.accounts.capacity() as usize);
17362
17363        // combine 1 normal append vec into existing ancient append vec
17364        // this will overflow the original ancient append vec because of the marking full above
17365        db.combine_ancient_slots(
17366            (ancient_slot..max_slot_inclusive).collect(),
17367            CAN_RANDOMLY_SHRINK_FALSE,
17368        );
17369
17370        // Restore size of ancient so we don't read garbage accounts when comparing. Now that we have created a second ancient append vec,
17371        // This first one is happy to be quite empty.
17372        adjust_append_vec_len_for_tests(&ancient, initial_len);
17373
17374        compare_all_accounts(
17375            &initial_accounts,
17376            &get_all_accounts(&db, ancient_slot..max_slot_inclusive),
17377        );
17378
17379        // the append vec at max_slot_inclusive-1 should NOT have been removed since we created an ancient append vec there
17380        assert!(is_ancient(
17381            &db.storage
17382                .get_slot_storage_entry(max_slot_inclusive - 1)
17383                .unwrap()
17384                .accounts
17385        ));
17386
17387        // combine normal append vec(s) into existing ancient append vec
17388        // this will overflow the original ancient append vec because of the marking full above
17389        db.combine_ancient_slots(
17390            (ancient_slot..=max_slot_inclusive).collect(),
17391            CAN_RANDOMLY_SHRINK_FALSE,
17392        );
17393
17394        // now, combine the next slot into the one that was just overflow
17395        compare_all_accounts(
17396            &initial_accounts,
17397            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
17398        );
17399
17400        // 2 ancients and then missing (because combined into 2nd ancient)
17401        assert!(is_ancient(
17402            &db.storage
17403                .get_slot_storage_entry(ancient_slot)
17404                .unwrap()
17405                .accounts
17406        ));
17407        assert!(is_ancient(
17408            &db.storage
17409                .get_slot_storage_entry(max_slot_inclusive - 1)
17410                .unwrap()
17411                .accounts
17412        ));
17413        assert!(db
17414            .storage
17415            .get_slot_storage_entry(max_slot_inclusive)
17416            .is_none());
17417    }
17418
17419    #[test]
17420    fn test_shrink_ancient() {
17421        solana_logger::setup();
17422
17423        let num_normal_slots = 1;
17424        // build an ancient append vec at slot 'ancient_slot'
17425        let (db, ancient_slot) = get_one_ancient_append_vec_and_others(true, num_normal_slots);
17426
17427        let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
17428        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
17429        compare_all_accounts(
17430            &initial_accounts,
17431            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
17432        );
17433
17434        // combine normal append vec(s) into existing ancient append vec
17435        db.combine_ancient_slots(
17436            (ancient_slot..=max_slot_inclusive).collect(),
17437            CAN_RANDOMLY_SHRINK_FALSE,
17438        );
17439
17440        compare_all_accounts(
17441            &initial_accounts,
17442            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
17443        );
17444
17445        // create a 2nd ancient append vec at 'next_slot'
17446        let next_slot = max_slot_inclusive + 1;
17447        create_storages_and_update_index(&db, None, next_slot, num_normal_slots, true, None);
17448        let max_slot_inclusive = next_slot + (num_normal_slots as Slot);
17449
17450        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
17451        compare_all_accounts(
17452            &initial_accounts,
17453            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
17454        );
17455
17456        db.combine_ancient_slots(
17457            (next_slot..=max_slot_inclusive).collect(),
17458            CAN_RANDOMLY_SHRINK_FALSE,
17459        );
17460
17461        compare_all_accounts(
17462            &initial_accounts,
17463            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
17464        );
17465
17466        // now, shrink the second ancient append vec into the first one
17467        let mut current_ancient = CurrentAncientAppendVec::new(
17468            ancient_slot,
17469            db.get_storage_for_slot(ancient_slot).unwrap(),
17470        );
17471        let mut dropped_roots = Vec::default();
17472        db.combine_one_store_into_ancient(
17473            next_slot,
17474            &db.get_storage_for_slot(next_slot).unwrap(),
17475            &mut current_ancient,
17476            &mut AncientSlotPubkeys::default(),
17477            &mut dropped_roots,
17478        );
17479        assert!(db.storage.is_empty_entry(next_slot));
17480        // this removes the storages entry completely from the hashmap for 'next_slot'.
17481        // Otherwise, we have a zero length vec in that hashmap
17482        db.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
17483        assert!(db.storage.get_slot_storage_entry(next_slot).is_none());
17484
17485        // include all the slots we put into the ancient append vec - they should contain nothing
17486        compare_all_accounts(
17487            &initial_accounts,
17488            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
17489        );
17490        // look at just the ancient append vec
17491        compare_all_accounts(
17492            &initial_accounts,
17493            &get_all_accounts(&db, ancient_slot..(ancient_slot + 1)),
17494        );
17495        // make sure there is only 1 ancient append vec at the ancient slot
17496        assert!(db.storage.get_slot_storage_entry(ancient_slot).is_some());
17497        assert!(is_ancient(
17498            &db.storage
17499                .get_slot_storage_entry(ancient_slot)
17500                .unwrap()
17501                .accounts
17502        ));
17503        ((ancient_slot + 1)..=max_slot_inclusive)
17504            .for_each(|slot| assert!(db.storage.get_slot_storage_entry(slot).is_none()));
17505    }
17506
17507    #[test]
17508    fn test_combine_ancient_slots_append() {
17509        solana_logger::setup();
17510        // combine 2-4 slots into a single ancient append vec
17511        for num_normal_slots in 1..3 {
17512            // but some slots contain only dead accounts
17513            for dead_accounts in 0..=num_normal_slots {
17514                let mut originals = Vec::default();
17515                // ancient_slot: contains ancient append vec
17516                // ancient_slot + 1: contains normal append vec with 1 alive account
17517                let (db, ancient_slot) =
17518                    get_one_ancient_append_vec_and_others(true, num_normal_slots);
17519
17520                let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
17521
17522                for slot in ancient_slot..=max_slot_inclusive {
17523                    originals.push(db.get_storage_for_slot(slot).unwrap());
17524                }
17525
17526                {
17527                    // remove the intended dead slots from the index so they look dead
17528                    for (count_marked_dead, original) in originals.iter().skip(1).enumerate() {
17529                        // skip the ancient one
17530                        if count_marked_dead >= dead_accounts {
17531                            break;
17532                        }
17533                        let original = original.accounts.account_iter().next().unwrap();
17534                        let slot = ancient_slot + 1 + (count_marked_dead as Slot);
17535                        _ = db.purge_keys_exact(
17536                            [(
17537                                *original.pubkey(),
17538                                vec![slot].into_iter().collect::<HashSet<_>>(),
17539                            )]
17540                            .iter(),
17541                        );
17542                    }
17543                    // the entries from these original append vecs should not expect to be in the final ancient append vec
17544                    for _ in 0..dead_accounts {
17545                        originals.remove(1); // remove the first non-ancient original entry each time
17546                    }
17547                }
17548
17549                // combine normal append vec(s) into existing ancient append vec
17550                db.combine_ancient_slots(
17551                    (ancient_slot..=max_slot_inclusive).collect(),
17552                    CAN_RANDOMLY_SHRINK_FALSE,
17553                );
17554
17555                // normal slots should have been appended to the ancient append vec in the first slot
17556                assert!(db.storage.get_slot_storage_entry(ancient_slot).is_some());
17557                let ancient = db.get_storage_for_slot(ancient_slot).unwrap();
17558                assert!(is_ancient(&ancient.accounts));
17559                let first_alive = ancient_slot + 1 + (dead_accounts as Slot);
17560                for slot in first_alive..=max_slot_inclusive {
17561                    assert!(db.storage.get_slot_storage_entry(slot).is_none());
17562                }
17563
17564                let GetUniqueAccountsResult {
17565                    stored_accounts: mut after_stored_accounts,
17566                    ..
17567                } = db.get_unique_accounts_from_storage(&ancient);
17568                assert_eq!(
17569                    after_stored_accounts.len(),
17570                    num_normal_slots + 1 - dead_accounts,
17571                    "normal_slots: {num_normal_slots}, dead_accounts: {dead_accounts}"
17572                );
17573                for original in &originals {
17574                    let original = original.accounts.account_iter().next().unwrap();
17575
17576                    let i = after_stored_accounts
17577                        .iter()
17578                        .enumerate()
17579                        .find_map(|(i, stored_ancient)| {
17580                            (stored_ancient.pubkey() == original.pubkey()).then_some({
17581                                assert!(accounts_equal(stored_ancient, &original));
17582                                i
17583                            })
17584                        })
17585                        .expect("did not find account");
17586                    after_stored_accounts.remove(i);
17587                }
17588                assert!(
17589                    after_stored_accounts.is_empty(),
17590                    "originals: {}, num_normal_slots: {}",
17591                    originals.len(),
17592                    num_normal_slots
17593                );
17594            }
17595        }
17596    }
17597
17598    fn populate_index(db: &AccountsDb, slots: Range<Slot>) {
17599        slots.into_iter().for_each(|slot| {
17600            if let Some(storage) = db.get_storage_for_slot(slot) {
17601                storage.accounts.account_iter().for_each(|account| {
17602                    let info = AccountInfo::new(
17603                        StorageLocation::AppendVec(storage.append_vec_id(), account.offset),
17604                        account.stored_size as u32,
17605                        account.lamports(),
17606                    );
17607                    db.accounts_index.upsert(
17608                        slot,
17609                        slot,
17610                        account.pubkey(),
17611                        &account,
17612                        &AccountSecondaryIndexes::default(),
17613                        info,
17614                        &mut Vec::default(),
17615                        UpsertReclaim::IgnoreReclaims,
17616                    );
17617                })
17618            }
17619        })
17620    }
17621
17622    pub(crate) fn remove_account_for_tests(
17623        storage: &AccountStorageEntry,
17624        num_bytes: usize,
17625        reset_accounts: bool,
17626    ) {
17627        storage.remove_account(num_bytes, reset_accounts);
17628    }
17629
17630    pub(crate) fn create_storages_and_update_index(
17631        db: &AccountsDb,
17632        tf: Option<&TempFile>,
17633        starting_slot: Slot,
17634        num_slots: usize,
17635        alive: bool,
17636        account_data_size: Option<u64>,
17637    ) {
17638        if num_slots == 0 {
17639            return;
17640        }
17641
17642        let local_tf = (tf.is_none()).then(|| {
17643            crate::append_vec::test_utils::get_append_vec_path("create_storages_and_update_index")
17644        });
17645        let tf = tf.unwrap_or_else(|| local_tf.as_ref().unwrap());
17646
17647        let write_version1 = 0;
17648        let starting_id = db
17649            .storage
17650            .iter()
17651            .map(|storage| storage.1.append_vec_id())
17652            .max()
17653            .unwrap_or(999);
17654        for i in 0..num_slots {
17655            let id = starting_id + (i as AppendVecId);
17656            let pubkey1 = solana_sdk::pubkey::new_rand();
17657            let storage = sample_storage_with_entries_id(
17658                tf,
17659                write_version1,
17660                starting_slot + (i as Slot),
17661                &pubkey1,
17662                id,
17663                alive,
17664                account_data_size,
17665            );
17666            insert_store(db, Arc::clone(&storage));
17667        }
17668
17669        let storage = db.get_storage_for_slot(starting_slot).unwrap();
17670        let created_accounts = db.get_unique_accounts_from_storage(&storage);
17671        assert_eq!(created_accounts.stored_accounts.len(), 1);
17672
17673        if alive {
17674            populate_index(db, starting_slot..(starting_slot + (num_slots as Slot) + 1));
17675        }
17676    }
17677
17678    pub(crate) fn create_db_with_storages_and_index(
17679        alive: bool,
17680        num_slots: usize,
17681        account_data_size: Option<u64>,
17682    ) -> (AccountsDb, Slot) {
17683        solana_logger::setup();
17684
17685        let db = AccountsDb::new_single_for_tests();
17686
17687        // create a single append vec with a single account in a slot
17688        // add the pubkey to index if alive
17689        // call combine_ancient_slots with the slot
17690        // verify we create an ancient appendvec that has alive accounts and does not have dead accounts
17691
17692        let slot1 = 1;
17693        create_storages_and_update_index(&db, None, slot1, num_slots, alive, account_data_size);
17694
17695        let slot1 = slot1 as Slot;
17696        (db, slot1)
17697    }
17698
17699    fn get_one_ancient_append_vec_and_others(
17700        alive: bool,
17701        num_normal_slots: usize,
17702    ) -> (AccountsDb, Slot) {
17703        let (db, slot1) = create_db_with_storages_and_index(alive, num_normal_slots + 1, None);
17704        let storage = db.get_storage_for_slot(slot1).unwrap();
17705        let created_accounts = db.get_unique_accounts_from_storage(&storage);
17706
17707        db.combine_ancient_slots(vec![slot1], CAN_RANDOMLY_SHRINK_FALSE);
17708        assert!(db.storage.get_slot_storage_entry(slot1).is_some());
17709        let ancient = db.get_storage_for_slot(slot1).unwrap();
17710        assert_eq!(alive, is_ancient(&ancient.accounts));
17711        let after_store = db.get_storage_for_slot(slot1).unwrap();
17712        let GetUniqueAccountsResult {
17713            stored_accounts: after_stored_accounts,
17714            capacity: after_capacity,
17715        } = db.get_unique_accounts_from_storage(&after_store);
17716        if alive {
17717            assert_ne!(created_accounts.capacity, after_capacity);
17718        } else {
17719            assert_eq!(created_accounts.capacity, after_capacity);
17720        }
17721        assert_eq!(created_accounts.stored_accounts.len(), 1);
17722        // always 1 account: either we leave the append vec alone if it is all dead
17723        // or we create a new one and copy into it if account is alive
17724        assert_eq!(after_stored_accounts.len(), 1);
17725        (db, slot1)
17726    }
17727
17728    #[test]
17729    fn test_handle_dropped_roots_for_ancient() {
17730        solana_logger::setup();
17731        let db = AccountsDb::new_single_for_tests();
17732        db.handle_dropped_roots_for_ancient(std::iter::empty::<Slot>());
17733        let slot0 = 0;
17734        let dropped_roots = vec![slot0];
17735        assert!(db.get_bank_hash_stats(slot0).is_some());
17736        db.accounts_index.add_root(slot0);
17737        db.accounts_index.add_uncleaned_roots([slot0].into_iter());
17738        assert!(db.accounts_index.is_uncleaned_root(slot0));
17739        assert!(db.accounts_index.is_alive_root(slot0));
17740        db.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
17741        assert!(db.get_bank_hash_stats(slot0).is_none());
17742        assert!(!db.accounts_index.is_uncleaned_root(slot0));
17743        assert!(!db.accounts_index.is_alive_root(slot0));
17744    }
17745
17746    fn insert_store(db: &AccountsDb, append_vec: Arc<AccountStorageEntry>) {
17747        db.storage.insert(append_vec.slot(), append_vec);
17748    }
17749
17750    #[test]
17751    #[should_panic(expected = "assertion failed: self.storage.remove")]
17752    fn test_handle_dropped_roots_for_ancient_assert() {
17753        solana_logger::setup();
17754        let common_store_path = Path::new("");
17755        let store_file_size = 2 * PAGE_SIZE;
17756        let entry = Arc::new(AccountStorageEntry::new(
17757            common_store_path,
17758            0,
17759            1,
17760            store_file_size,
17761        ));
17762        let db = AccountsDb::new_single_for_tests();
17763        let slot0 = 0;
17764        let dropped_roots = vec![slot0];
17765        insert_store(&db, entry);
17766        db.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
17767    }
17768
17769    #[test]
17770    fn test_should_move_to_ancient_append_vec() {
17771        solana_logger::setup();
17772        let db = AccountsDb::new_single_for_tests();
17773        let slot5 = 5;
17774        let tf = crate::append_vec::test_utils::get_append_vec_path(
17775            "test_should_move_to_ancient_append_vec",
17776        );
17777        let write_version1 = 0;
17778        let pubkey1 = solana_sdk::pubkey::new_rand();
17779        let storage = sample_storage_with_entries(&tf, write_version1, slot5, &pubkey1, false);
17780        let mut current_ancient = CurrentAncientAppendVec::default();
17781
17782        let should_move = db.should_move_to_ancient_append_vec(
17783            &storage,
17784            &mut current_ancient,
17785            slot5,
17786            CAN_RANDOMLY_SHRINK_FALSE,
17787        );
17788        assert!(current_ancient.slot_and_append_vec.is_none());
17789        // slot is not ancient, so it is good to move
17790        assert!(should_move);
17791
17792        current_ancient = CurrentAncientAppendVec::new(slot5, Arc::clone(&storage)); // just 'some', contents don't matter
17793        let should_move = db.should_move_to_ancient_append_vec(
17794            &storage,
17795            &mut current_ancient,
17796            slot5,
17797            CAN_RANDOMLY_SHRINK_FALSE,
17798        );
17799        // should have kept the same 'current_ancient'
17800        assert_eq!(current_ancient.slot(), slot5);
17801        assert_eq!(current_ancient.append_vec().slot(), slot5);
17802        assert_eq!(current_ancient.append_vec_id(), storage.append_vec_id());
17803
17804        // slot is not ancient, so it is good to move
17805        assert!(should_move);
17806
17807        // now, create an ancient slot and make sure that it does NOT think it needs to be moved and that it becomes the ancient append vec to use
17808        let mut current_ancient = CurrentAncientAppendVec::default();
17809        let slot1_ancient = 1;
17810        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17811        let _existing_append_vec = db.create_and_insert_store(slot1_ancient, 1000, "test");
17812        let ancient1 = db
17813            .get_store_for_shrink(slot1_ancient, get_ancient_append_vec_capacity())
17814            .new_storage()
17815            .clone();
17816        let should_move = db.should_move_to_ancient_append_vec(
17817            &ancient1,
17818            &mut current_ancient,
17819            slot1_ancient,
17820            CAN_RANDOMLY_SHRINK_FALSE,
17821        );
17822        assert!(!should_move);
17823        assert_eq!(current_ancient.append_vec_id(), ancient1.append_vec_id());
17824        assert_eq!(current_ancient.slot(), slot1_ancient);
17825
17826        // current is ancient1
17827        // try to move ancient2
17828        // current should become ancient2
17829        let slot2_ancient = 2;
17830        let mut current_ancient = CurrentAncientAppendVec::new(slot1_ancient, ancient1.clone());
17831        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17832        let _existing_append_vec = db.create_and_insert_store(slot2_ancient, 1000, "test");
17833        let ancient2 = db
17834            .get_store_for_shrink(slot2_ancient, get_ancient_append_vec_capacity())
17835            .new_storage()
17836            .clone();
17837        let should_move = db.should_move_to_ancient_append_vec(
17838            &ancient2,
17839            &mut current_ancient,
17840            slot2_ancient,
17841            CAN_RANDOMLY_SHRINK_FALSE,
17842        );
17843        assert!(!should_move);
17844        assert_eq!(current_ancient.append_vec_id(), ancient2.append_vec_id());
17845        assert_eq!(current_ancient.slot(), slot2_ancient);
17846
17847        // now try a full ancient append vec
17848        // current is None
17849        let slot3_full_ancient = 3;
17850        let mut current_ancient = CurrentAncientAppendVec::default();
17851        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17852        let _existing_append_vec = db.create_and_insert_store(slot3_full_ancient, 1000, "test");
17853        let full_ancient_3 = make_full_ancient_append_vec(&db, slot3_full_ancient, false);
17854        let should_move = db.should_move_to_ancient_append_vec(
17855            &full_ancient_3.new_storage().clone(),
17856            &mut current_ancient,
17857            slot3_full_ancient,
17858            CAN_RANDOMLY_SHRINK_FALSE,
17859        );
17860        assert!(!should_move);
17861        assert_eq!(
17862            current_ancient.append_vec_id(),
17863            full_ancient_3.new_storage().append_vec_id()
17864        );
17865        assert_eq!(current_ancient.slot(), slot3_full_ancient);
17866
17867        // now set current_ancient to something
17868        let mut current_ancient = CurrentAncientAppendVec::new(slot1_ancient, ancient1.clone());
17869        let should_move = db.should_move_to_ancient_append_vec(
17870            &full_ancient_3.new_storage().clone(),
17871            &mut current_ancient,
17872            slot3_full_ancient,
17873            CAN_RANDOMLY_SHRINK_FALSE,
17874        );
17875        assert!(!should_move);
17876        assert_eq!(
17877            current_ancient.append_vec_id(),
17878            full_ancient_3.new_storage().append_vec_id()
17879        );
17880        assert_eq!(current_ancient.slot(), slot3_full_ancient);
17881
17882        // now mark the full ancient as candidate for shrink
17883        adjust_alive_bytes(full_ancient_3.new_storage(), 0);
17884
17885        // should shrink here, returning none for current
17886        let mut current_ancient = CurrentAncientAppendVec::default();
17887        let should_move = db.should_move_to_ancient_append_vec(
17888            &full_ancient_3.new_storage().clone(),
17889            &mut current_ancient,
17890            slot3_full_ancient,
17891            CAN_RANDOMLY_SHRINK_FALSE,
17892        );
17893        assert!(should_move);
17894        assert!(current_ancient.slot_and_append_vec.is_none());
17895
17896        // should return true here, returning current from prior
17897        // now set current_ancient to something and see if it still goes to None
17898        let mut current_ancient = CurrentAncientAppendVec::new(slot1_ancient, ancient1.clone());
17899        let should_move = db.should_move_to_ancient_append_vec(
17900            &Arc::clone(full_ancient_3.new_storage()),
17901            &mut current_ancient,
17902            slot3_full_ancient,
17903            CAN_RANDOMLY_SHRINK_FALSE,
17904        );
17905        assert!(should_move);
17906        assert_eq!(current_ancient.append_vec_id(), ancient1.append_vec_id());
17907        assert_eq!(current_ancient.slot(), slot1_ancient);
17908    }
17909
17910    fn adjust_alive_bytes(storage: &Arc<AccountStorageEntry>, alive_bytes: usize) {
17911        storage.alive_bytes.store(alive_bytes, Ordering::Release);
17912    }
17913
17914    /// cause 'ancient' to appear to contain 'len' bytes
17915    fn adjust_append_vec_len_for_tests(ancient: &Arc<AccountStorageEntry>, len: usize) {
17916        assert!(is_ancient(&ancient.accounts));
17917        ancient.accounts.set_current_len_for_tests(len);
17918        adjust_alive_bytes(ancient, len);
17919    }
17920
17921    fn make_ancient_append_vec_full(ancient: &Arc<AccountStorageEntry>, mark_alive: bool) {
17922        for _ in 0..100 {
17923            append_sample_data_to_storage(ancient, &Pubkey::default(), 0, mark_alive, None);
17924        }
17925        // since we're not adding to the index, this is how we specify that all these accounts are alive
17926        adjust_alive_bytes(ancient, ancient.capacity() as usize);
17927    }
17928
17929    fn make_full_ancient_append_vec(
17930        db: &AccountsDb,
17931        slot: Slot,
17932        mark_alive: bool,
17933    ) -> ShrinkInProgress<'_> {
17934        let full = db.get_store_for_shrink(slot, get_ancient_append_vec_capacity());
17935        make_ancient_append_vec_full(full.new_storage(), mark_alive);
17936        full
17937    }
17938
17939    #[test]
17940    fn test_calculate_incremental_accounts_hash() {
17941        let accounts_db =
17942            AccountsDb::new_for_tests_with_caching(Vec::new(), &ClusterType::Development);
17943
17944        let owner = Pubkey::new_unique();
17945        let mut accounts: Vec<_> = (0..10)
17946            .map(|_| (Pubkey::new_unique(), AccountSharedData::new(0, 0, &owner)))
17947            .collect();
17948
17949        // store some accounts into slot 0
17950        let slot = 0;
17951        {
17952            accounts[0].1.set_lamports(0);
17953            accounts[1].1.set_lamports(1);
17954            accounts[2].1.set_lamports(10);
17955            accounts[3].1.set_lamports(100);
17956            //accounts[4].1.set_lamports(1_000); <-- will be added next slot
17957
17958            let accounts = vec![
17959                (&accounts[0].0, &accounts[0].1),
17960                (&accounts[1].0, &accounts[1].1),
17961                (&accounts[2].0, &accounts[2].1),
17962                (&accounts[3].0, &accounts[3].1),
17963            ];
17964            accounts_db.store_cached((slot, accounts.as_slice()), None);
17965            accounts_db.add_root_and_flush_write_cache(slot);
17966        }
17967
17968        // store some accounts into slot 1
17969        let slot = slot + 1;
17970        {
17971            //accounts[0].1.set_lamports(0);      <-- unchanged
17972            accounts[1].1.set_lamports(0); /*     <-- drain account */
17973            //accounts[2].1.set_lamports(10);     <-- unchanged
17974            //accounts[3].1.set_lamports(100);    <-- unchanged
17975            accounts[4].1.set_lamports(1_000); /* <-- add account */
17976
17977            let accounts = vec![
17978                (&accounts[1].0, &accounts[1].1),
17979                (&accounts[4].0, &accounts[4].1),
17980            ];
17981            accounts_db.store_cached((slot, accounts.as_slice()), None);
17982            accounts_db.add_root_and_flush_write_cache(slot);
17983        }
17984
17985        // calculate the full accounts hash
17986        let full_accounts_hash = {
17987            accounts_db.clean_accounts(Some(slot - 1), false, None);
17988            let (storages, _) = accounts_db.get_snapshot_storages(..=slot, None);
17989            let storages = SortedStorages::new(&storages);
17990            accounts_db
17991                .calculate_accounts_hash_from_storages(
17992                    &CalcAccountsHashConfig::default(),
17993                    &storages,
17994                    HashStats::default(),
17995                )
17996                .unwrap()
17997        };
17998        assert_eq!(full_accounts_hash.1, 1_110);
17999        let full_accounts_hash_slot = slot;
18000
18001        // Calculate the expected full accounts hash here and ensure it matches.
18002        // Ensure the zero-lamport accounts are NOT included in the full accounts hash.
18003        let full_account_hashes = [(2, 0), (3, 0), (4, 1)].into_iter().map(|(index, slot)| {
18004            let (pubkey, account) = &accounts[index];
18005            AccountsDb::hash_account(slot, account, pubkey, INCLUDE_SLOT_IN_HASH_TESTS)
18006        });
18007        let expected_accounts_hash = AccountsHash(compute_merkle_root(full_account_hashes));
18008        assert_eq!(full_accounts_hash.0, expected_accounts_hash);
18009
18010        // store accounts into slot 2
18011        let slot = slot + 1;
18012        {
18013            //accounts[0].1.set_lamports(0);         <-- unchanged
18014            //accounts[1].1.set_lamports(0);         <-- unchanged
18015            accounts[2].1.set_lamports(0); /*        <-- drain account */
18016            //accounts[3].1.set_lamports(100);       <-- unchanged
18017            //accounts[4].1.set_lamports(1_000);     <-- unchanged
18018            accounts[5].1.set_lamports(10_000); /*   <-- add account */
18019            accounts[6].1.set_lamports(100_000); /*  <-- add account */
18020            //accounts[7].1.set_lamports(1_000_000); <-- will be added next slot
18021
18022            let accounts = vec![
18023                (&accounts[2].0, &accounts[2].1),
18024                (&accounts[5].0, &accounts[5].1),
18025                (&accounts[6].0, &accounts[6].1),
18026            ];
18027            accounts_db.store_cached((slot, accounts.as_slice()), None);
18028            accounts_db.add_root_and_flush_write_cache(slot);
18029        }
18030
18031        // store accounts into slot 3
18032        let slot = slot + 1;
18033        {
18034            //accounts[0].1.set_lamports(0);          <-- unchanged
18035            //accounts[1].1.set_lamports(0);          <-- unchanged
18036            //accounts[2].1.set_lamports(0);          <-- unchanged
18037            accounts[3].1.set_lamports(0); /*         <-- drain account */
18038            //accounts[4].1.set_lamports(1_000);      <-- unchanged
18039            accounts[5].1.set_lamports(0); /*         <-- drain account */
18040            //accounts[6].1.set_lamports(100_000);    <-- unchanged
18041            accounts[7].1.set_lamports(1_000_000); /* <-- add account */
18042
18043            let accounts = vec![
18044                (&accounts[3].0, &accounts[3].1),
18045                (&accounts[5].0, &accounts[5].1),
18046                (&accounts[7].0, &accounts[7].1),
18047            ];
18048            accounts_db.store_cached((slot, accounts.as_slice()), None);
18049            accounts_db.add_root_and_flush_write_cache(slot);
18050        }
18051
18052        // calculate the incremental accounts hash
18053        let incremental_accounts_hash = {
18054            accounts_db.clean_accounts(Some(slot - 1), false, Some(full_accounts_hash_slot));
18055            let (storages, _) =
18056                accounts_db.get_snapshot_storages(full_accounts_hash_slot + 1..=slot, None);
18057            let storages = SortedStorages::new(&storages);
18058            accounts_db
18059                .calculate_incremental_accounts_hash(
18060                    &CalcAccountsHashConfig::default(),
18061                    &storages,
18062                    full_accounts_hash_slot,
18063                    HashStats::default(),
18064                )
18065                .unwrap()
18066        };
18067        assert_eq!(incremental_accounts_hash.1, 1_100_000);
18068
18069        // Ensure the zero-lamport accounts are included in the IAH.
18070        // Accounts 2, 3, and 5 are all zero-lamports.
18071        let incremental_account_hashes =
18072            [(2, 2), (3, 3), (5, 3), (6, 2), (7, 3)]
18073                .into_iter()
18074                .map(|(index, slot)| {
18075                    let (pubkey, account) = &accounts[index];
18076                    if account.is_zero_lamport() {
18077                        // For incremental accounts hash, the hash of a zero lamport account is the hash of its pubkey.
18078                        // Ensure this implementation detail remains in sync with AccountsHasher::de_dup_in_parallel().
18079                        let hash = blake3::hash(bytemuck::bytes_of(pubkey));
18080                        Hash::new_from_array(hash.into())
18081                    } else {
18082                        AccountsDb::hash_account(slot, account, pubkey, INCLUDE_SLOT_IN_HASH_TESTS)
18083                    }
18084                });
18085        let expected_accounts_hash = AccountsHash(compute_merkle_root(incremental_account_hashes));
18086        assert_eq!(incremental_accounts_hash.0, expected_accounts_hash);
18087    }
18088
18089    fn compute_merkle_root(hashes: impl IntoIterator<Item = Hash>) -> Hash {
18090        let hashes = hashes.into_iter().collect();
18091        AccountsHasher::compute_merkle_root_recurse(hashes, MERKLE_FANOUT)
18092    }
18093
18094    #[test]
18095    fn test_get_largest_keys() {
18096        solana_logger::setup();
18097        // Constants
18098        const NUM_DUMMY_ACCOUNTS: usize = 50;
18099        const MAX_CHILD_ACCOUNTS: usize = 100;
18100        let mut slot = 0;
18101
18102        // Set secondary indexes
18103        let account_indexes = AccountSecondaryIndexes {
18104            keys: None,
18105            indexes: HashSet::from([AccountIndex::ProgramId]),
18106        };
18107
18108        // AccountDB Setup
18109        let accounts_db = AccountsDb::new_single_for_tests_with_secondary_indexes(account_indexes);
18110
18111        // Assert that list is empty. No accounts added yet.
18112        let mut test_largest_keys = accounts_db.accounts_index.get_largest_keys(
18113            &AccountIndex::ProgramId,
18114            MAX_NUM_LARGEST_INDEX_KEYS_RETURNED,
18115        );
18116        assert_eq!(0, test_largest_keys.len());
18117
18118        // Add some basic system owned accounts
18119        let mut dummy_account_pubkeys = Vec::with_capacity(NUM_DUMMY_ACCOUNTS);
18120        let mut num_generator = thread_rng();
18121        let key_size_range = Uniform::new_inclusive(0, MAX_CHILD_ACCOUNTS);
18122        for i in 1..=NUM_DUMMY_ACCOUNTS {
18123            let pubkey = Pubkey::new_unique();
18124            dummy_account_pubkeys.push(pubkey);
18125            let account = AccountSharedData::from(Account {
18126                lamports: 11111111,
18127                owner: system_program::id(),
18128                ..Account::default()
18129            });
18130            // Store account in the AccountsDB
18131            accounts_db.store_for_tests(slot, &[(&dummy_account_pubkeys[i - 1], &account)]);
18132            slot += 1;
18133            // Check that the system pubkey increments each time
18134            test_largest_keys = accounts_db.accounts_index.get_largest_keys(
18135                &AccountIndex::ProgramId,
18136                MAX_NUM_LARGEST_INDEX_KEYS_RETURNED,
18137            );
18138            assert_eq!(test_largest_keys.len(), 1);
18139            let number_system_owned_accounts = test_largest_keys[0].0;
18140            assert_eq!(i, number_system_owned_accounts);
18141        }
18142
18143        // Now add a random number of accounts each owned by one of the newly
18144        // created dummy pubkeys
18145        for dummy_account in &dummy_account_pubkeys {
18146            // Add child accounts to each dummy account
18147            let num_children = (&mut num_generator).sample_iter(key_size_range).next();
18148            for j in 0..num_children.unwrap_or(0) {
18149                let child_pubkey = Pubkey::new_unique();
18150                let child_account = AccountSharedData::from(Account {
18151                    lamports: ((j as u64) + 1) * 1000,
18152                    owner: *dummy_account,
18153                    ..Account::default()
18154                });
18155                accounts_db.store_for_tests(slot, &[(&child_pubkey, &child_account)]);
18156                slot += 1;
18157            }
18158            // Check for entries with the same key size for sub sorting by pubkey
18159            let existing_key_size_position = test_largest_keys
18160                .iter()
18161                .position(|(x, _)| *x == num_children.unwrap_or(0));
18162            // Find where it should go and insert it
18163            let key_position = match test_largest_keys
18164                .binary_search_by_key(&num_children.unwrap_or(0), |(size, _)| *size)
18165            {
18166                Ok(found_position) => found_position,
18167                Err(woudbe_position) => woudbe_position,
18168            };
18169            test_largest_keys.insert(key_position, (num_children.unwrap_or(0), *dummy_account));
18170            // If there were indeed more elements with the same key size sort them by Pubkey
18171            if existing_key_size_position.is_some() {
18172                // Obtain a slice of mutable references to all elements with the same key_size
18173                let mut sub_slice = test_largest_keys
18174                    .split_mut(|(k, _)| *k != num_children.unwrap_or(0))
18175                    .flatten()
18176                    .collect_vec();
18177                // Sort them...
18178                let mut sorting_buffer = sub_slice.iter().map(|x| *(*x)).collect_vec();
18179                sorting_buffer.sort_unstable_by_key(|(_, v)| *v);
18180                // Copy back into the list
18181                for i in 0..sub_slice.len() {
18182                    *(sub_slice[i]) = (sorting_buffer[i].0, sorting_buffer[i].1);
18183                }
18184            }
18185            // Prune list
18186            while test_largest_keys.len() > MAX_NUM_LARGEST_INDEX_KEYS_RETURNED {
18187                test_largest_keys.remove(0);
18188            }
18189        }
18190
18191        // Verify secondary index list matches expected list built above.
18192        let largest_keys = accounts_db.accounts_index.get_largest_keys(
18193            &AccountIndex::ProgramId,
18194            MAX_NUM_LARGEST_INDEX_KEYS_RETURNED,
18195        );
18196        // Reverse the tracking vector and check for equality
18197        // Note: Backend stores the `key_size_index` in ascending key size, but
18198        // `get_largest_keys` returns the data in descending order, ie. the largest at the top.
18199        test_largest_keys = test_largest_keys.into_iter().rev().collect_vec();
18200        assert_eq!(test_largest_keys, largest_keys);
18201
18202        // Test queries for a partial list and past max return size
18203        let mut largest_program_id_keys = Vec::<(usize, Pubkey)>::new();
18204        for i in 0..=MAX_NUM_LARGEST_INDEX_KEYS_RETURNED + 1 {
18205            largest_program_id_keys = accounts_db
18206                .accounts_index
18207                .get_largest_keys(&AccountIndex::ProgramId, i);
18208            if i <= MAX_NUM_LARGEST_INDEX_KEYS_RETURNED {
18209                assert_eq!(largest_program_id_keys.len(), i);
18210            } else {
18211                assert_eq!(
18212                    largest_program_id_keys.len(),
18213                    MAX_NUM_LARGEST_INDEX_KEYS_RETURNED
18214                );
18215            }
18216        }
18217
18218        // Root the bank preparing for removal
18219        (0..slot).for_each(|slot| {
18220            accounts_db.calculate_accounts_delta_hash(slot);
18221            accounts_db.add_root_and_flush_write_cache(slot);
18222        });
18223
18224        // Test Removal of Keys
18225        // First just remove a single key
18226        let mut smallest_key: Pubkey;
18227        let mut smallest_key_size: usize;
18228        let mut smallest_key_inner_keys: Vec<Pubkey>;
18229        let mut try_again = 0;
18230        let zero_lamport_account =
18231            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
18232        loop {
18233            smallest_key = largest_program_id_keys[largest_program_id_keys.len() - 1 - try_again].1;
18234            smallest_key_size =
18235                largest_program_id_keys[largest_program_id_keys.len() - 1 - try_again].0;
18236            let mut collector = Vec::new();
18237            accounts_db
18238                .scan_accounts(
18239                    &Ancestors::default(),
18240                    0,
18241                    |some_account_tuple| {
18242                        if let Some(mapped_account_tuple) = some_account_tuple
18243                            .filter(|(_, account, _)| {
18244                                Accounts::is_loadable(account.lamports())
18245                                    && account.owner() == &smallest_key
18246                            })
18247                            .map(|(pubkey, account, _slot)| (*pubkey, account))
18248                        {
18249                            collector.push(mapped_account_tuple)
18250                        }
18251                    },
18252                    &ScanConfig::new(true),
18253                )
18254                .ok();
18255            smallest_key_inner_keys = collector.into_iter().map(|(k, _)| k).collect_vec();
18256            let single_inner_key = smallest_key_inner_keys.pop().unwrap();
18257            // Overwrite the account as a 0 lamport account and clean.
18258            accounts_db.store_for_tests(slot, &[(&single_inner_key, &zero_lamport_account)]);
18259            accounts_db.calculate_accounts_delta_hash(slot);
18260            accounts_db.add_root_and_flush_write_cache(slot);
18261            slot += 1;
18262            accounts_db.clean_accounts_for_tests();
18263            // Read back
18264            largest_program_id_keys = accounts_db.accounts_index.get_largest_keys(
18265                &AccountIndex::ProgramId,
18266                MAX_NUM_LARGEST_INDEX_KEYS_RETURNED,
18267            );
18268            // Ensure the below check is comparing the same pubkey in case there were ties in the list for keysize.
18269            if largest_program_id_keys[largest_program_id_keys.len() - 1 - try_again].1
18270                == smallest_key
18271            {
18272                break;
18273            }
18274            // If there were a duplicate keysize, just move up in the largest key list.
18275            // worst case use the second largest key and keep removing till the tie is broken.
18276            else if try_again < largest_program_id_keys.len() - 2 {
18277                try_again += 1;
18278            }
18279        }
18280        // Make sure outer key size decreased
18281        assert_eq!(
18282            smallest_key_size - 1,
18283            largest_program_id_keys[largest_program_id_keys.len() - 1 - try_again].0
18284        );
18285
18286        // Test removal of multiple keys
18287        for key in smallest_key_inner_keys {
18288            accounts_db.store_for_tests(slot, &[(&key, &zero_lamport_account)]);
18289        }
18290        accounts_db.calculate_accounts_delta_hash(slot);
18291        accounts_db.add_root_and_flush_write_cache(slot);
18292        accounts_db.clean_accounts_for_tests();
18293        // Read back
18294        largest_program_id_keys = accounts_db.accounts_index.get_largest_keys(
18295            &AccountIndex::ProgramId,
18296            MAX_NUM_LARGEST_INDEX_KEYS_RETURNED,
18297        );
18298        // Since all inner keys were removed, make sure outer key is gone too.
18299        let outer_key_removed = !largest_program_id_keys
18300            .iter()
18301            .any(|(_, v)| *v == smallest_key);
18302        assert!(outer_key_removed);
18303    }
18304}