solana_accounts_db/
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
21mod geyser_plugin_utils;
22mod scan_account_storage;
23pub mod stats;
24
25#[cfg(feature = "dev-context-only-utils")]
26use qualifier_attr::qualifiers;
27use {
28    crate::{
29        account_info::{AccountInfo, Offset, StorageLocation},
30        account_storage::{
31            meta::StoredAccountMeta, AccountStorage, AccountStorageStatus, ShrinkInProgress,
32        },
33        accounts_cache::{AccountsCache, CachedAccount, SlotCache},
34        accounts_db::stats::{
35            AccountsStats, BankHashStats, CleanAccountsStats, FlushStats, PurgeStats,
36            ShrinkAncientStats, ShrinkStats, ShrinkStatsSub, StoreAccountsTiming,
37        },
38        accounts_file::{
39            AccountsFile, AccountsFileError, AccountsFileProvider, MatchAccountOwnerError,
40            StorageAccess, ALIGN_BOUNDARY_OFFSET,
41        },
42        accounts_hash::{
43            AccountHash, AccountLtHash, AccountsDeltaHash, AccountsHash, AccountsHashKind,
44            AccountsHasher, AccountsLtHash, CalcAccountsHashConfig, CalculateHashIntermediate,
45            HashStats, IncrementalAccountsHash, SerdeAccountsDeltaHash, SerdeAccountsHash,
46            SerdeIncrementalAccountsHash, ZeroLamportAccounts, ZERO_LAMPORT_ACCOUNT_HASH,
47            ZERO_LAMPORT_ACCOUNT_LT_HASH,
48        },
49        accounts_index::{
50            in_mem_accounts_index::StartupStats, AccountSecondaryIndexes, AccountsIndex,
51            AccountsIndexConfig, AccountsIndexRootsStats, AccountsIndexScanResult, DiskIndexValue,
52            IndexKey, IndexValue, IsCached, RefCount, ScanConfig, ScanFilter, ScanResult, SlotList,
53            UpsertReclaim, ZeroLamport, ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS,
54            ACCOUNTS_INDEX_CONFIG_FOR_TESTING,
55        },
56        accounts_index_storage::Startup,
57        accounts_partition::RentPayingAccountsByPartition,
58        accounts_update_notifier_interface::AccountsUpdateNotifier,
59        active_stats::{ActiveStatItem, ActiveStats},
60        ancestors::Ancestors,
61        ancient_append_vecs::{
62            get_ancient_append_vec_capacity, is_ancient, AccountsToStore, StorageSelector,
63        },
64        append_vec::{aligned_stored_size, STORE_META_OVERHEAD},
65        cache_hash_data::{CacheHashData, DeletionPolicy as CacheHashDeletionPolicy},
66        contains::Contains,
67        epoch_accounts_hash::EpochAccountsHashManager,
68        partitioned_rewards::{PartitionedEpochRewardsConfig, TestPartitionedEpochRewards},
69        read_only_accounts_cache::ReadOnlyAccountsCache,
70        sorted_storages::SortedStorages,
71        storable_accounts::{StorableAccounts, StorableAccountsBySlot},
72        u64_align, utils,
73        verify_accounts_hash_in_background::VerifyAccountsHashInBackground,
74    },
75    crossbeam_channel::{unbounded, Receiver, Sender, TryRecvError},
76    dashmap::{DashMap, DashSet},
77    log::*,
78    rand::{thread_rng, Rng},
79    rayon::{prelude::*, ThreadPool},
80    seqlock::SeqLock,
81    smallvec::SmallVec,
82    solana_lattice_hash::lt_hash::LtHash,
83    solana_measure::{meas_dur, measure::Measure, measure_us},
84    solana_nohash_hasher::{IntMap, IntSet},
85    solana_rayon_threadlimit::get_thread_count,
86    solana_sdk::{
87        account::{Account, AccountSharedData, ReadableAccount},
88        clock::{BankId, Epoch, Slot},
89        epoch_schedule::EpochSchedule,
90        genesis_config::GenesisConfig,
91        hash::Hash,
92        pubkey::Pubkey,
93        rent_collector::RentCollector,
94        saturating_add_assign,
95        transaction::SanitizedTransaction,
96    },
97    std::{
98        borrow::Cow,
99        boxed::Box,
100        collections::{BTreeSet, HashMap, HashSet, VecDeque},
101        fs,
102        hash::{Hash as StdHash, Hasher as StdHasher},
103        io::Result as IoResult,
104        num::{NonZeroUsize, Saturating},
105        ops::{Range, RangeBounds},
106        path::{Path, PathBuf},
107        sync::{
108            atomic::{AtomicBool, AtomicU32, AtomicU64, AtomicUsize, Ordering},
109            Arc, Condvar, Mutex, RwLock,
110        },
111        thread::{sleep, Builder},
112        time::{Duration, Instant},
113    },
114    tempfile::TempDir,
115};
116
117// when the accounts write cache exceeds this many bytes, we will flush it
118// this can be specified on the command line, too (--accounts-db-cache-limit-mb)
119const WRITE_CACHE_LIMIT_BYTES_DEFAULT: u64 = 15_000_000_000;
120const SCAN_SLOT_PAR_ITER_THRESHOLD: usize = 4000;
121
122const UNREF_ACCOUNTS_BATCH_SIZE: usize = 10_000;
123
124const DEFAULT_FILE_SIZE: u64 = 4 * 1024 * 1024;
125const DEFAULT_NUM_DIRS: u32 = 4;
126
127// When calculating hashes, it is helpful to break the pubkeys found into bins based on the pubkey value.
128// More bins means smaller vectors to sort, copy, etc.
129pub const PUBKEY_BINS_FOR_CALCULATING_HASHES: usize = 65536;
130
131// Without chunks, we end up with 1 output vec for each outer snapshot storage.
132// This results in too many vectors to be efficient.
133// Chunks when scanning storages to calculate hashes.
134// If this is too big, we don't get enough parallelism of scanning storages.
135// If this is too small, then we produce too many output vectors to iterate.
136// Metrics indicate a sweet spot in the 2.5k-5k range for mnb.
137const MAX_ITEMS_PER_CHUNK: Slot = 2_500;
138
139// When getting accounts for shrinking from the index, this is the # of accounts to lookup per thread.
140// This allows us to split up accounts index accesses across multiple threads.
141const SHRINK_COLLECT_CHUNK_SIZE: usize = 50;
142
143/// The number of shrink candidate slots that is small enough so that
144/// additional storages from ancient slots can be added to the
145/// candidates for shrinking.
146const SHRINK_INSERT_ANCIENT_THRESHOLD: usize = 10;
147
148/// Default value for the number of ancient storages the ancient slot
149/// combining should converge to.
150pub const MAX_ANCIENT_SLOTS_DEFAULT: usize = 100_000;
151
152#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
153pub enum CreateAncientStorage {
154    /// ancient storages are created by appending
155    Append,
156    /// ancient storages are created by 1-shot write to pack multiple accounts together more efficiently with new formats
157    #[default]
158    Pack,
159}
160
161#[derive(Debug)]
162enum StoreTo<'a> {
163    /// write to cache
164    Cache,
165    /// write to storage
166    Storage(&'a Arc<AccountStorageEntry>),
167}
168
169impl<'a> StoreTo<'a> {
170    fn is_cached(&self) -> bool {
171        matches!(self, StoreTo::Cache)
172    }
173}
174
175#[derive(Debug, Clone, Copy, PartialEq, Eq)]
176pub(crate) enum ScanAccountStorageData {
177    /// callback for accounts in storage will not include `data`
178    NoData,
179    /// return data (&[u8]) for each account.
180    /// This can be expensive to get and is not necessary for many scan operations.
181    DataRefForStorage,
182}
183
184#[derive(Default, Debug)]
185/// hold alive accounts
186/// alive means in the accounts index
187pub(crate) struct AliveAccounts<'a> {
188    /// slot the accounts are currently stored in
189    pub(crate) slot: Slot,
190    pub(crate) accounts: Vec<&'a AccountFromStorage>,
191    pub(crate) bytes: usize,
192}
193
194/// separate pubkeys into those with a single refcount and those with > 1 refcount
195#[derive(Debug)]
196pub(crate) struct ShrinkCollectAliveSeparatedByRefs<'a> {
197    /// accounts where ref_count = 1
198    pub(crate) one_ref: AliveAccounts<'a>,
199    /// account where ref_count > 1, but this slot contains the alive entry with the highest slot
200    pub(crate) many_refs_this_is_newest_alive: AliveAccounts<'a>,
201    /// account where ref_count > 1, and this slot is NOT the highest alive entry in the index for the pubkey
202    pub(crate) many_refs_old_alive: AliveAccounts<'a>,
203}
204
205/// Configuration Parameters for running accounts hash and total lamports verification
206#[derive(Debug, Clone)]
207pub struct VerifyAccountsHashAndLamportsConfig<'a> {
208    /// bank ancestors
209    pub ancestors: &'a Ancestors,
210    /// true to verify hash calculation
211    pub test_hash_calculation: bool,
212    /// epoch_schedule
213    pub epoch_schedule: &'a EpochSchedule,
214    /// rent_collector
215    pub rent_collector: &'a RentCollector,
216    /// true to ignore mismatches
217    pub ignore_mismatch: bool,
218    /// true to dump debug log if mismatch happens
219    pub store_detailed_debug_info: bool,
220    /// true to use dedicated background thread pool for verification
221    pub use_bg_thread_pool: bool,
222}
223
224pub(crate) trait ShrinkCollectRefs<'a>: Sync + Send {
225    fn with_capacity(capacity: usize, slot: Slot) -> Self;
226    fn collect(&mut self, other: Self);
227    fn add(
228        &mut self,
229        ref_count: u64,
230        account: &'a AccountFromStorage,
231        slot_list: &[(Slot, AccountInfo)],
232    );
233    fn len(&self) -> usize;
234    fn alive_bytes(&self) -> usize;
235    fn alive_accounts(&self) -> &Vec<&'a AccountFromStorage>;
236}
237
238impl<'a> ShrinkCollectRefs<'a> for AliveAccounts<'a> {
239    fn collect(&mut self, mut other: Self) {
240        self.bytes = self.bytes.saturating_add(other.bytes);
241        self.accounts.append(&mut other.accounts);
242    }
243    fn with_capacity(capacity: usize, slot: Slot) -> Self {
244        Self {
245            accounts: Vec::with_capacity(capacity),
246            bytes: 0,
247            slot,
248        }
249    }
250    fn add(
251        &mut self,
252        _ref_count: u64,
253        account: &'a AccountFromStorage,
254        _slot_list: &[(Slot, AccountInfo)],
255    ) {
256        self.accounts.push(account);
257        self.bytes = self.bytes.saturating_add(account.stored_size());
258    }
259    fn len(&self) -> usize {
260        self.accounts.len()
261    }
262    fn alive_bytes(&self) -> usize {
263        self.bytes
264    }
265    fn alive_accounts(&self) -> &Vec<&'a AccountFromStorage> {
266        &self.accounts
267    }
268}
269
270impl<'a> ShrinkCollectRefs<'a> for ShrinkCollectAliveSeparatedByRefs<'a> {
271    fn collect(&mut self, other: Self) {
272        self.one_ref.collect(other.one_ref);
273        self.many_refs_this_is_newest_alive
274            .collect(other.many_refs_this_is_newest_alive);
275        self.many_refs_old_alive.collect(other.many_refs_old_alive);
276    }
277    fn with_capacity(capacity: usize, slot: Slot) -> Self {
278        Self {
279            one_ref: AliveAccounts::with_capacity(capacity, slot),
280            many_refs_this_is_newest_alive: AliveAccounts::with_capacity(0, slot),
281            many_refs_old_alive: AliveAccounts::with_capacity(0, slot),
282        }
283    }
284    fn add(
285        &mut self,
286        ref_count: u64,
287        account: &'a AccountFromStorage,
288        slot_list: &[(Slot, AccountInfo)],
289    ) {
290        let other = if ref_count == 1 {
291            &mut self.one_ref
292        } else if slot_list.len() == 1
293            || !slot_list
294                .iter()
295                .any(|(slot_list_slot, _info)| slot_list_slot > &self.many_refs_old_alive.slot)
296        {
297            // this entry is alive but is newer than any other slot in the index
298            &mut self.many_refs_this_is_newest_alive
299        } else {
300            // This entry is alive but is older than at least one other slot in the index.
301            // We would expect clean to get rid of the entry for THIS slot at some point, but clean hasn't done that yet.
302            &mut self.many_refs_old_alive
303        };
304        other.add(ref_count, account, slot_list);
305    }
306    fn len(&self) -> usize {
307        self.one_ref
308            .len()
309            .saturating_add(self.many_refs_old_alive.len())
310            .saturating_add(self.many_refs_this_is_newest_alive.len())
311    }
312    fn alive_bytes(&self) -> usize {
313        self.one_ref
314            .alive_bytes()
315            .saturating_add(self.many_refs_old_alive.alive_bytes())
316            .saturating_add(self.many_refs_this_is_newest_alive.alive_bytes())
317    }
318    fn alive_accounts(&self) -> &Vec<&'a AccountFromStorage> {
319        unimplemented!("illegal use");
320    }
321}
322
323pub enum StoreReclaims {
324    /// normal reclaim mode
325    Default,
326    /// do not return reclaims from accounts index upsert
327    Ignore,
328}
329
330/// while combining into ancient append vecs, we need to keep track of the current one that is receiving new data
331/// The pattern for callers is:
332/// 1. this is a mut local
333/// 2. do some version of create/new
334/// 3. use it (slot, append_vec, etc.)
335/// 4. re-create it sometimes
336/// 5. goto 3
337///
338/// If a caller uses it before initializing it, it will be a runtime unwrap() error, similar to an assert.
339/// That condition is an illegal use pattern and is justifiably an assertable condition.
340#[derive(Default)]
341struct CurrentAncientAccountsFile {
342    slot_and_accounts_file: Option<(Slot, Arc<AccountStorageEntry>)>,
343}
344
345impl CurrentAncientAccountsFile {
346    fn new(slot: Slot, append_vec: Arc<AccountStorageEntry>) -> CurrentAncientAccountsFile {
347        Self {
348            slot_and_accounts_file: Some((slot, append_vec)),
349        }
350    }
351
352    /// Create ancient accounts file for a slot
353    ///     min_bytes: the new accounts file needs to have at least this capacity
354    #[must_use]
355    fn create_ancient_accounts_file<'a>(
356        &mut self,
357        slot: Slot,
358        db: &'a AccountsDb,
359        min_bytes: usize,
360    ) -> ShrinkInProgress<'a> {
361        let size = get_ancient_append_vec_capacity().max(min_bytes as u64);
362        let shrink_in_progress = db.get_store_for_shrink(slot, size);
363        *self = Self::new(slot, Arc::clone(shrink_in_progress.new_storage()));
364        shrink_in_progress
365    }
366    #[must_use]
367    fn create_if_necessary<'a>(
368        &mut self,
369        slot: Slot,
370        db: &'a AccountsDb,
371        min_bytes: usize,
372    ) -> Option<ShrinkInProgress<'a>> {
373        if self.slot_and_accounts_file.is_none() {
374            Some(self.create_ancient_accounts_file(slot, db, min_bytes))
375        } else {
376            None
377        }
378    }
379
380    /// note this requires that 'slot_and_accounts_file' is Some
381    fn slot(&self) -> Slot {
382        self.slot_and_accounts_file.as_ref().unwrap().0
383    }
384
385    /// note this requires that 'slot_and_accounts_file' is Some
386    fn accounts_file(&self) -> &Arc<AccountStorageEntry> {
387        &self.slot_and_accounts_file.as_ref().unwrap().1
388    }
389
390    /// helper function to cleanup call to 'store_accounts_frozen'
391    /// return timing and bytes written
392    fn store_ancient_accounts(
393        &self,
394        db: &AccountsDb,
395        accounts_to_store: &AccountsToStore,
396        storage_selector: StorageSelector,
397    ) -> (StoreAccountsTiming, u64) {
398        let accounts = accounts_to_store.get(storage_selector);
399
400        let previous_available = self.accounts_file().accounts.remaining_bytes();
401
402        let accounts = [(accounts_to_store.slot(), accounts)];
403        let storable_accounts = StorableAccountsBySlot::new(self.slot(), &accounts, db);
404        let timing = db.store_accounts_frozen(storable_accounts, self.accounts_file());
405        let bytes_written =
406            previous_available.saturating_sub(self.accounts_file().accounts.remaining_bytes());
407        assert_eq!(
408            bytes_written,
409            u64_align!(accounts_to_store.get_bytes(storage_selector)) as u64
410        );
411
412        (timing, bytes_written)
413    }
414}
415
416/// specifies how to return zero lamport accounts from a load
417#[derive(Clone, Copy)]
418enum LoadZeroLamports {
419    /// return None if loaded account has zero lamports
420    None,
421    /// return Some(account with zero lamports) if loaded account has zero lamports
422    /// This used to be the only behavior.
423    /// Note that this is non-deterministic if clean is running asynchronously.
424    /// If a zero lamport account exists in the index, then Some is returned.
425    /// Once it is cleaned from the index, None is returned.
426    #[cfg(feature = "dev-context-only-utils")]
427    SomeWithZeroLamportAccountForTests,
428}
429
430#[derive(Debug)]
431struct AncientSlotPubkeysInner {
432    pubkeys: HashSet<Pubkey>,
433    slot: Slot,
434}
435
436#[derive(Debug, Default)]
437struct AncientSlotPubkeys {
438    inner: Option<AncientSlotPubkeysInner>,
439}
440
441impl AncientSlotPubkeys {
442    /// All accounts in 'slot' will be moved to 'current_ancient'
443    /// If 'slot' is different than the 'current_ancient'.slot, then an account in 'slot' may ALREADY be in the current ancient append vec.
444    /// 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'.
445    /// 'self' is also changed to accumulate the pubkeys that now exist in 'current_ancient'
446    /// 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
447    ///  pubkeys in the new 'current_ancient'.append_vec
448    fn maybe_unref_accounts_already_in_ancient(
449        &mut self,
450        slot: Slot,
451        db: &AccountsDb,
452        current_ancient: &CurrentAncientAccountsFile,
453        to_store: &AccountsToStore,
454    ) {
455        if slot != current_ancient.slot() {
456            // we are taking accounts from 'slot' and putting them into 'current_ancient.slot()'
457            // StorageSelector::Primary here because only the accounts that are moving from 'slot' to 'current_ancient.slot()'
458            // Any overflow accounts will get written into a new append vec AT 'slot', so they don't need to be unrefed
459            let accounts = to_store.get(StorageSelector::Primary);
460            if Some(current_ancient.slot()) != self.inner.as_ref().map(|ap| ap.slot) {
461                let mut pubkeys = HashSet::new();
462                current_ancient
463                    .accounts_file()
464                    .accounts
465                    .scan_pubkeys(|pubkey| {
466                        pubkeys.insert(*pubkey);
467                    });
468                self.inner = Some(AncientSlotPubkeysInner {
469                    pubkeys,
470                    slot: current_ancient.slot(),
471                });
472            }
473            // 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
474            // unwrap cannot fail because the code above will cause us to set it to Some(...) if it is None
475            db.unref_accounts_already_in_storage(
476                accounts,
477                self.inner.as_mut().map(|p| &mut p.pubkeys).unwrap(),
478            );
479        }
480    }
481}
482
483#[derive(Debug)]
484pub(crate) struct ShrinkCollect<'a, T: ShrinkCollectRefs<'a>> {
485    pub(crate) slot: Slot,
486    pub(crate) capacity: u64,
487    pub(crate) pubkeys_to_unref: Vec<&'a Pubkey>,
488    pub(crate) zero_lamport_single_ref_pubkeys: Vec<&'a Pubkey>,
489    pub(crate) alive_accounts: T,
490    /// total size in storage of all alive accounts
491    pub(crate) alive_total_bytes: usize,
492    pub(crate) total_starting_accounts: usize,
493    /// true if all alive accounts are zero lamports
494    pub(crate) all_are_zero_lamports: bool,
495}
496
497pub const ACCOUNTS_DB_CONFIG_FOR_TESTING: AccountsDbConfig = AccountsDbConfig {
498    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_TESTING),
499    account_indexes: None,
500    base_working_path: None,
501    accounts_hash_cache_path: None,
502    shrink_paths: None,
503    shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION,
504    read_cache_limit_bytes: None,
505    write_cache_limit_bytes: None,
506    ancient_append_vec_offset: None,
507    skip_initial_hash_calc: false,
508    exhaustively_verify_refcounts: false,
509    create_ancient_storage: CreateAncientStorage::Pack,
510    test_partitioned_epoch_rewards: TestPartitionedEpochRewards::CompareResults,
511    test_skip_rewrites_but_include_in_bank_hash: false,
512    storage_access: StorageAccess::Mmap,
513    scan_filter_for_shrinking: ScanFilter::OnlyAbnormalWithVerify,
514    enable_experimental_accumulator_hash: false,
515    num_clean_threads: None,
516    num_foreground_threads: None,
517    num_hash_threads: None,
518};
519pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig {
520    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS),
521    account_indexes: None,
522    base_working_path: None,
523    accounts_hash_cache_path: None,
524    shrink_paths: None,
525    shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION,
526    read_cache_limit_bytes: None,
527    write_cache_limit_bytes: None,
528    ancient_append_vec_offset: None,
529    skip_initial_hash_calc: false,
530    exhaustively_verify_refcounts: false,
531    create_ancient_storage: CreateAncientStorage::Pack,
532    test_partitioned_epoch_rewards: TestPartitionedEpochRewards::None,
533    test_skip_rewrites_but_include_in_bank_hash: false,
534    storage_access: StorageAccess::Mmap,
535    scan_filter_for_shrinking: ScanFilter::OnlyAbnormalWithVerify,
536    enable_experimental_accumulator_hash: false,
537    num_clean_threads: None,
538    num_foreground_threads: None,
539    num_hash_threads: None,
540};
541
542pub type BinnedHashData = Vec<Vec<CalculateHashIntermediate>>;
543
544struct LoadAccountsIndexForShrink<'a, T: ShrinkCollectRefs<'a>> {
545    /// all alive accounts
546    alive_accounts: T,
547    /// pubkeys that are going to be unref'd in the accounts index after we are
548    /// done with shrinking, because they are dead
549    pubkeys_to_unref: Vec<&'a Pubkey>,
550    /// pubkeys that are the last remaining zero lamport instance of an account
551    zero_lamport_single_ref_pubkeys: Vec<&'a Pubkey>,
552    /// true if all alive accounts are zero lamport accounts
553    all_are_zero_lamports: bool,
554}
555
556/// reference an account found during scanning a storage. This is a byval struct to replace
557/// `StoredAccountMeta`
558#[derive(Debug, PartialEq, Copy, Clone)]
559pub struct AccountFromStorage {
560    pub index_info: AccountInfo,
561    pub data_len: u64,
562    pub pubkey: Pubkey,
563}
564
565impl ZeroLamport for AccountFromStorage {
566    fn is_zero_lamport(&self) -> bool {
567        self.index_info.is_zero_lamport()
568    }
569}
570
571impl AccountFromStorage {
572    pub fn pubkey(&self) -> &Pubkey {
573        &self.pubkey
574    }
575    pub fn stored_size(&self) -> usize {
576        aligned_stored_size(self.data_len as usize)
577    }
578    pub fn data_len(&self) -> usize {
579        self.data_len as usize
580    }
581    pub fn new(account: &StoredAccountMeta) -> Self {
582        // the id is irrelevant in this account info. This structure is only used DURING shrink operations.
583        // In those cases, there is only 1 append vec id per slot when we read the accounts.
584        // Any value of storage id in account info works fine when we want the 'normal' storage.
585        let storage_id = 0;
586        AccountFromStorage {
587            index_info: AccountInfo::new(
588                StorageLocation::AppendVec(storage_id, account.offset()),
589                account.lamports(),
590            ),
591            pubkey: *account.pubkey(),
592            data_len: account.data_len() as u64,
593        }
594    }
595}
596
597pub struct GetUniqueAccountsResult {
598    pub stored_accounts: Vec<AccountFromStorage>,
599    pub capacity: u64,
600    pub num_duplicated_accounts: usize,
601}
602
603pub struct AccountsAddRootTiming {
604    pub index_us: u64,
605    pub cache_us: u64,
606    pub store_us: u64,
607}
608
609/// Slots older the "number of slots in an epoch minus this number"
610/// than max root are treated as ancient and subject to packing.
611/// |  older  |<-          slots in an epoch          ->| max root
612/// |  older  |<-    offset   ->|                       |
613/// |          ancient          |        modern         |
614///
615/// If this is negative, this many slots older than the number of
616/// slots in epoch are still treated as modern (ie. non-ancient).
617/// |  older  |<- abs(offset) ->|<- slots in an epoch ->| max root
618/// | ancient |                 modern                  |
619///
620/// Note that another constant MAX_ANCIENT_SLOTS_DEFAULT sets a
621/// threshold for combining ancient storages so that their overall
622/// number is under a certain limit, whereas this constant establishes
623/// the distance from the max root slot beyond which storages holding
624/// the account data for the slots are considered ancient by the
625/// shrinking algorithm.
626const ANCIENT_APPEND_VEC_DEFAULT_OFFSET: Option<i64> = Some(100_000);
627
628#[derive(Debug, Default, Clone)]
629pub struct AccountsDbConfig {
630    pub index: Option<AccountsIndexConfig>,
631    pub account_indexes: Option<AccountSecondaryIndexes>,
632    /// Base directory for various necessary files
633    pub base_working_path: Option<PathBuf>,
634    pub accounts_hash_cache_path: Option<PathBuf>,
635    pub shrink_paths: Option<Vec<PathBuf>>,
636    pub shrink_ratio: AccountShrinkThreshold,
637    /// The low and high watermark sizes for the read cache, in bytes.
638    /// If None, defaults will be used.
639    pub read_cache_limit_bytes: Option<(usize, usize)>,
640    pub write_cache_limit_bytes: Option<u64>,
641    /// if None, ancient append vecs are set to ANCIENT_APPEND_VEC_DEFAULT_OFFSET
642    /// Some(offset) means include slots up to (max_slot - (slots_per_epoch - 'offset'))
643    pub ancient_append_vec_offset: Option<i64>,
644    pub test_skip_rewrites_but_include_in_bank_hash: bool,
645    pub skip_initial_hash_calc: bool,
646    pub exhaustively_verify_refcounts: bool,
647    /// how to create ancient storages
648    pub create_ancient_storage: CreateAncientStorage,
649    pub test_partitioned_epoch_rewards: TestPartitionedEpochRewards,
650    pub storage_access: StorageAccess,
651    pub scan_filter_for_shrinking: ScanFilter,
652    pub enable_experimental_accumulator_hash: bool,
653    /// Number of threads for background cleaning operations (`thread_pool_clean')
654    pub num_clean_threads: Option<NonZeroUsize>,
655    /// Number of threads for foreground operations (`thread_pool`)
656    pub num_foreground_threads: Option<NonZeroUsize>,
657    /// Number of threads for background accounts hashing (`thread_pool_hash`)
658    pub num_hash_threads: Option<NonZeroUsize>,
659}
660
661#[cfg(not(test))]
662const ABSURD_CONSECUTIVE_FAILED_ITERATIONS: usize = 100;
663
664#[derive(Debug, Clone, Copy)]
665pub enum AccountShrinkThreshold {
666    /// Measure the total space sparseness across all candidates
667    /// And select the candidates by using the top sparse account storage entries to shrink.
668    /// The value is the overall shrink threshold measured as ratio of the total live bytes
669    /// over the total bytes.
670    TotalSpace { shrink_ratio: f64 },
671    /// Use the following option to shrink all stores whose alive ratio is below
672    /// the specified threshold.
673    IndividualStore { shrink_ratio: f64 },
674}
675pub const DEFAULT_ACCOUNTS_SHRINK_OPTIMIZE_TOTAL_SPACE: bool = true;
676pub const DEFAULT_ACCOUNTS_SHRINK_RATIO: f64 = 0.80;
677// The default extra account space in percentage from the ideal target
678const DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION: AccountShrinkThreshold =
679    AccountShrinkThreshold::TotalSpace {
680        shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_RATIO,
681    };
682
683impl Default for AccountShrinkThreshold {
684    fn default() -> AccountShrinkThreshold {
685        DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION
686    }
687}
688
689pub enum ScanStorageResult<R, B> {
690    Cached(Vec<R>),
691    Stored(B),
692}
693
694#[derive(Debug, Default)]
695pub struct IndexGenerationInfo {
696    pub accounts_data_len: u64,
697    pub rent_paying_accounts_by_partition: RentPayingAccountsByPartition,
698    /// The lt hash of the old/duplicate accounts identified during index generation.
699    /// Will be used when verifying the accounts lt hash, after rebuilding a Bank.
700    pub duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
701}
702
703#[derive(Debug, Default)]
704struct SlotIndexGenerationInfo {
705    insert_time_us: u64,
706    num_accounts: u64,
707    num_accounts_rent_paying: usize,
708    accounts_data_len: u64,
709    amount_to_top_off_rent: u64,
710    rent_paying_accounts_by_partition: Vec<Pubkey>,
711    all_accounts_are_zero_lamports: bool,
712}
713
714/// The lt hash of old/duplicate accounts
715///
716/// Accumulation of all the duplicate accounts found during index generation.
717/// These accounts need to have their lt hashes mixed *out*.
718/// This is the final value, that when applied to all the storages at startup,
719/// will produce the correct accounts lt hash.
720#[derive(Debug, Clone)]
721pub struct DuplicatesLtHash(pub LtHash);
722
723impl Default for DuplicatesLtHash {
724    fn default() -> Self {
725        Self(LtHash::identity())
726    }
727}
728
729#[derive(Default, Debug)]
730struct GenerateIndexTimings {
731    pub total_time_us: u64,
732    pub index_time: u64,
733    pub scan_time: u64,
734    pub insertion_time_us: u64,
735    pub min_bin_size_in_mem: usize,
736    pub max_bin_size_in_mem: usize,
737    pub total_items_in_mem: usize,
738    pub storage_size_storages_us: u64,
739    pub index_flush_us: u64,
740    pub rent_paying: AtomicUsize,
741    pub amount_to_top_off_rent: AtomicU64,
742    pub total_including_duplicates: u64,
743    pub accounts_data_len_dedup_time_us: u64,
744    pub total_duplicate_slot_keys: u64,
745    pub total_num_unique_duplicate_keys: u64,
746    pub num_duplicate_accounts: u64,
747    pub populate_duplicate_keys_us: u64,
748    pub total_slots: u64,
749    pub slots_to_clean: u64,
750    pub par_duplicates_lt_hash_us: AtomicU64,
751    pub all_accounts_are_zero_lamports_slots: u64,
752}
753
754#[derive(Default, Debug, PartialEq, Eq)]
755struct StorageSizeAndCount {
756    /// total size stored, including both alive and dead bytes
757    pub stored_size: usize,
758    /// number of accounts in the storage including both alive and dead accounts
759    pub count: usize,
760}
761type StorageSizeAndCountMap = DashMap<AccountsFileId, StorageSizeAndCount>;
762
763impl GenerateIndexTimings {
764    pub fn report(&self, startup_stats: &StartupStats) {
765        datapoint_info!(
766            "generate_index",
767            ("overall_us", self.total_time_us, i64),
768            // we cannot accurately measure index insertion time because of many threads and lock contention
769            ("total_us", self.index_time, i64),
770            ("scan_stores_us", self.scan_time, i64),
771            ("insertion_time_us", self.insertion_time_us, i64),
772            ("min_bin_size_in_mem", self.min_bin_size_in_mem as i64, i64),
773            ("max_bin_size_in_mem", self.max_bin_size_in_mem as i64, i64),
774            (
775                "storage_size_storages_us",
776                self.storage_size_storages_us as i64,
777                i64
778            ),
779            ("index_flush_us", self.index_flush_us as i64, i64),
780            (
781                "total_rent_paying",
782                self.rent_paying.load(Ordering::Relaxed) as i64,
783                i64
784            ),
785            (
786                "amount_to_top_off_rent",
787                self.amount_to_top_off_rent.load(Ordering::Relaxed) as i64,
788                i64
789            ),
790            (
791                "total_items_including_duplicates",
792                self.total_including_duplicates as i64,
793                i64
794            ),
795            ("total_items_in_mem", self.total_items_in_mem as i64, i64),
796            (
797                "accounts_data_len_dedup_time_us",
798                self.accounts_data_len_dedup_time_us as i64,
799                i64
800            ),
801            (
802                "total_duplicate_slot_keys",
803                self.total_duplicate_slot_keys as i64,
804                i64
805            ),
806            (
807                "total_num_unique_duplicate_keys",
808                self.total_num_unique_duplicate_keys as i64,
809                i64
810            ),
811            (
812                "num_duplicate_accounts",
813                self.num_duplicate_accounts as i64,
814                i64
815            ),
816            (
817                "populate_duplicate_keys_us",
818                self.populate_duplicate_keys_us as i64,
819                i64
820            ),
821            ("total_slots", self.total_slots, i64),
822            ("slots_to_clean", self.slots_to_clean, i64),
823            (
824                "copy_data_us",
825                startup_stats.copy_data_us.swap(0, Ordering::Relaxed),
826                i64
827            ),
828            (
829                "par_duplicates_lt_hash_us",
830                self.par_duplicates_lt_hash_us.load(Ordering::Relaxed),
831                i64
832            ),
833            (
834                "all_accounts_are_zero_lamports_slots",
835                self.all_accounts_are_zero_lamports_slots,
836                i64
837            ),
838        );
839    }
840}
841
842impl IndexValue for AccountInfo {}
843impl DiskIndexValue for AccountInfo {}
844
845impl ZeroLamport for AccountSharedData {
846    fn is_zero_lamport(&self) -> bool {
847        self.lamports() == 0
848    }
849}
850
851impl ZeroLamport for Account {
852    fn is_zero_lamport(&self) -> bool {
853        self.lamports() == 0
854    }
855}
856
857struct MultiThreadProgress<'a> {
858    last_update: Instant,
859    my_last_report_count: u64,
860    total_count: &'a AtomicU64,
861    report_delay_secs: u64,
862    first_caller: bool,
863    ultimate_count: u64,
864    start_time: Instant,
865}
866
867impl<'a> MultiThreadProgress<'a> {
868    fn new(total_count: &'a AtomicU64, report_delay_secs: u64, ultimate_count: u64) -> Self {
869        Self {
870            last_update: Instant::now(),
871            my_last_report_count: 0,
872            total_count,
873            report_delay_secs,
874            first_caller: false,
875            ultimate_count,
876            start_time: Instant::now(),
877        }
878    }
879    fn report(&mut self, my_current_count: u64) {
880        let now = Instant::now();
881        if now.duration_since(self.last_update).as_secs() >= self.report_delay_secs {
882            let my_total_newly_processed_slots_since_last_report =
883                my_current_count - self.my_last_report_count;
884
885            self.my_last_report_count = my_current_count;
886            let previous_total_processed_slots_across_all_threads = self.total_count.fetch_add(
887                my_total_newly_processed_slots_since_last_report,
888                Ordering::Relaxed,
889            );
890            self.first_caller =
891                self.first_caller || 0 == previous_total_processed_slots_across_all_threads;
892            if self.first_caller {
893                let total = previous_total_processed_slots_across_all_threads
894                    + my_total_newly_processed_slots_since_last_report;
895                info!(
896                    "generating index: {}/{} slots... ({}/s)",
897                    total,
898                    self.ultimate_count,
899                    total / self.start_time.elapsed().as_secs().max(1),
900                );
901            }
902            self.last_update = now;
903        }
904    }
905}
906
907/// An offset into the AccountsDb::storage vector
908pub type AtomicAccountsFileId = AtomicU32;
909pub type AccountsFileId = u32;
910
911type AccountSlots = HashMap<Pubkey, IntSet<Slot>>;
912type SlotOffsets = IntMap<Slot, IntSet<Offset>>;
913type ReclaimResult = (AccountSlots, SlotOffsets);
914type PubkeysRemovedFromAccountsIndex = HashSet<Pubkey>;
915type ShrinkCandidates = IntSet<Slot>;
916
917// Some hints for applicability of additional sanity checks for the do_load fast-path;
918// Slower fallback code path will be taken if the fast path has failed over the retry
919// threshold, regardless of these hints. Also, load cannot fail not-deterministically
920// even under very rare circumstances, unlike previously did allow.
921#[derive(Clone, Copy, Debug, PartialEq, Eq)]
922pub enum LoadHint {
923    // Caller hints that it's loading transactions for a block which is
924    // descended from the current root, and at the tip of its fork.
925    // Thereby, further this assumes AccountIndex::max_root should not increase
926    // during this load, meaning there should be no squash.
927    // Overall, this enables us to assert!() strictly while running the fast-path for
928    // account loading, while maintaining the determinism of account loading and resultant
929    // transaction execution thereof.
930    FixedMaxRoot,
931    /// same as `FixedMaxRoot`, except do not populate the read cache on load
932    FixedMaxRootDoNotPopulateReadCache,
933    // Caller can't hint the above safety assumption. Generally RPC and miscellaneous
934    // other call-site falls into this category. The likelihood of slower path is slightly
935    // increased as well.
936    Unspecified,
937}
938
939#[derive(Debug)]
940pub enum LoadedAccountAccessor<'a> {
941    // StoredAccountMeta can't be held directly here due to its lifetime dependency to
942    // AccountStorageEntry
943    Stored(Option<(Arc<AccountStorageEntry>, usize)>),
944    // None value in Cached variant means the cache was flushed
945    Cached(Option<Cow<'a, CachedAccount>>),
946}
947
948impl<'a> LoadedAccountAccessor<'a> {
949    fn check_and_get_loaded_account_shared_data(&mut self) -> AccountSharedData {
950        // all of these following .expect() and .unwrap() are like serious logic errors,
951        // ideal for representing this as rust type system....
952
953        match self {
954            LoadedAccountAccessor::Stored(Some((maybe_storage_entry, offset))) => {
955                // If we do find the storage entry, we can guarantee that the storage entry is
956                // safe to read from because we grabbed a reference to the storage entry while it
957                // was still in the storage map. This means even if the storage entry is removed
958                // from the storage map after we grabbed the storage entry, the recycler should not
959                // reset the storage entry until we drop the reference to the storage entry.
960                maybe_storage_entry.get_account_shared_data(*offset).expect(
961                    "If a storage entry was found in the storage map, it must not have been reset \
962                     yet",
963                )
964            }
965            _ => self.check_and_get_loaded_account(|loaded_account| loaded_account.take_account()),
966        }
967    }
968
969    fn check_and_get_loaded_account<T>(
970        &mut self,
971        callback: impl for<'local> FnMut(LoadedAccount<'local>) -> T,
972    ) -> T {
973        // all of these following .expect() and .unwrap() are like serious logic errors,
974        // ideal for representing this as rust type system....
975
976        match self {
977            LoadedAccountAccessor::Cached(None) | LoadedAccountAccessor::Stored(None) => {
978                panic!(
979                    "Should have already been taken care of when creating this \
980                     LoadedAccountAccessor"
981                );
982            }
983            LoadedAccountAccessor::Cached(Some(_cached_account)) => {
984                // Cached(Some(x)) variant always produces `Some` for get_loaded_account() since
985                // it just returns the inner `x` without additional fetches
986                self.get_loaded_account(callback).unwrap()
987            }
988            LoadedAccountAccessor::Stored(Some(_maybe_storage_entry)) => {
989                // If we do find the storage entry, we can guarantee that the storage entry is
990                // safe to read from because we grabbed a reference to the storage entry while it
991                // was still in the storage map. This means even if the storage entry is removed
992                // from the storage map after we grabbed the storage entry, the recycler should not
993                // reset the storage entry until we drop the reference to the storage entry.
994                self.get_loaded_account(callback).expect(
995                    "If a storage entry was found in the storage map, it must not have been reset \
996                     yet",
997                )
998            }
999        }
1000    }
1001
1002    fn get_loaded_account<T>(
1003        &mut self,
1004        mut callback: impl for<'local> FnMut(LoadedAccount<'local>) -> T,
1005    ) -> Option<T> {
1006        match self {
1007            LoadedAccountAccessor::Cached(cached_account) => {
1008                let cached_account: Cow<'a, CachedAccount> = cached_account.take().expect(
1009                    "Cache flushed/purged should be handled before trying to fetch account",
1010                );
1011                Some(callback(LoadedAccount::Cached(cached_account)))
1012            }
1013            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
1014                // storage entry may not be present if slot was cleaned up in
1015                // between reading the accounts index and calling this function to
1016                // get account meta from the storage entry here
1017                maybe_storage_entry
1018                    .as_ref()
1019                    .and_then(|(storage_entry, offset)| {
1020                        storage_entry
1021                            .accounts
1022                            .get_stored_account_meta_callback(*offset, |account| {
1023                                callback(LoadedAccount::Stored(account))
1024                            })
1025                    })
1026            }
1027        }
1028    }
1029
1030    fn account_matches_owners(&self, owners: &[Pubkey]) -> Result<usize, MatchAccountOwnerError> {
1031        match self {
1032            LoadedAccountAccessor::Cached(cached_account) => cached_account
1033                .as_ref()
1034                .and_then(|cached_account| {
1035                    if cached_account.account.is_zero_lamport() {
1036                        None
1037                    } else {
1038                        owners
1039                            .iter()
1040                            .position(|entry| cached_account.account.owner() == entry)
1041                    }
1042                })
1043                .ok_or(MatchAccountOwnerError::NoMatch),
1044            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
1045                // storage entry may not be present if slot was cleaned up in
1046                // between reading the accounts index and calling this function to
1047                // get account meta from the storage entry here
1048                maybe_storage_entry
1049                    .as_ref()
1050                    .map(|(storage_entry, offset)| {
1051                        storage_entry
1052                            .accounts
1053                            .account_matches_owners(*offset, owners)
1054                    })
1055                    .unwrap_or(Err(MatchAccountOwnerError::UnableToLoad))
1056            }
1057        }
1058    }
1059}
1060
1061pub enum LoadedAccount<'a> {
1062    Stored(StoredAccountMeta<'a>),
1063    Cached(Cow<'a, CachedAccount>),
1064}
1065
1066impl<'a> LoadedAccount<'a> {
1067    pub fn loaded_hash(&self) -> AccountHash {
1068        match self {
1069            LoadedAccount::Stored(stored_account_meta) => *stored_account_meta.hash(),
1070            LoadedAccount::Cached(cached_account) => cached_account.hash(),
1071        }
1072    }
1073
1074    pub fn pubkey(&self) -> &Pubkey {
1075        match self {
1076            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.pubkey(),
1077            LoadedAccount::Cached(cached_account) => cached_account.pubkey(),
1078        }
1079    }
1080
1081    pub fn take_account(&self) -> AccountSharedData {
1082        match self {
1083            LoadedAccount::Stored(stored_account_meta) => {
1084                stored_account_meta.to_account_shared_data()
1085            }
1086            LoadedAccount::Cached(cached_account) => match cached_account {
1087                Cow::Owned(cached_account) => cached_account.account.clone(),
1088                Cow::Borrowed(cached_account) => cached_account.account.clone(),
1089            },
1090        }
1091    }
1092
1093    pub fn is_cached(&self) -> bool {
1094        match self {
1095            LoadedAccount::Stored(_) => false,
1096            LoadedAccount::Cached(_) => true,
1097        }
1098    }
1099
1100    /// data_len can be calculated without having access to `&data` in future implementations
1101    pub fn data_len(&self) -> usize {
1102        self.data().len()
1103    }
1104}
1105
1106impl<'a> ReadableAccount for LoadedAccount<'a> {
1107    fn lamports(&self) -> u64 {
1108        match self {
1109            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.lamports(),
1110            LoadedAccount::Cached(cached_account) => cached_account.account.lamports(),
1111        }
1112    }
1113    fn data(&self) -> &[u8] {
1114        match self {
1115            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.data(),
1116            LoadedAccount::Cached(cached_account) => cached_account.account.data(),
1117        }
1118    }
1119    fn owner(&self) -> &Pubkey {
1120        match self {
1121            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.owner(),
1122            LoadedAccount::Cached(cached_account) => cached_account.account.owner(),
1123        }
1124    }
1125    fn executable(&self) -> bool {
1126        match self {
1127            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.executable(),
1128            LoadedAccount::Cached(cached_account) => cached_account.account.executable(),
1129        }
1130    }
1131    fn rent_epoch(&self) -> Epoch {
1132        match self {
1133            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.rent_epoch(),
1134            LoadedAccount::Cached(cached_account) => cached_account.account.rent_epoch(),
1135        }
1136    }
1137    fn to_account_shared_data(&self) -> AccountSharedData {
1138        self.take_account()
1139    }
1140}
1141
1142#[derive(Debug)]
1143pub enum AccountsHashVerificationError {
1144    MissingAccountsHash,
1145    MismatchedAccountsHash,
1146    MismatchedTotalLamports(u64, u64),
1147}
1148
1149#[derive(Default)]
1150struct CleanKeyTimings {
1151    collect_delta_keys_us: u64,
1152    delta_insert_us: u64,
1153    dirty_store_processing_us: u64,
1154    delta_key_count: u64,
1155    dirty_pubkeys_count: u64,
1156    oldest_dirty_slot: Slot,
1157    /// number of ancient append vecs that were scanned because they were dirty when clean started
1158    dirty_ancient_stores: usize,
1159}
1160
1161/// Persistent storage structure holding the accounts
1162#[derive(Debug)]
1163pub struct AccountStorageEntry {
1164    pub(crate) id: AccountsFileId,
1165
1166    pub(crate) slot: Slot,
1167
1168    /// storage holding the accounts
1169    pub accounts: AccountsFile,
1170
1171    /// Keeps track of the number of accounts stored in a specific AppendVec.
1172    ///  This is periodically checked to reuse the stores that do not have
1173    ///  any accounts in it
1174    /// status corresponding to the storage, lets us know that
1175    ///  the append_vec, once maxed out, then emptied, can be reclaimed
1176    count_and_status: SeqLock<(usize, AccountStorageStatus)>,
1177
1178    alive_bytes: AtomicUsize,
1179}
1180
1181impl AccountStorageEntry {
1182    pub fn new(
1183        path: &Path,
1184        slot: Slot,
1185        id: AccountsFileId,
1186        file_size: u64,
1187        provider: AccountsFileProvider,
1188    ) -> Self {
1189        let tail = AccountsFile::file_name(slot, id);
1190        let path = Path::new(path).join(tail);
1191        let accounts = provider.new_writable(path, file_size);
1192
1193        Self {
1194            id,
1195            slot,
1196            accounts,
1197            count_and_status: SeqLock::new((0, AccountStorageStatus::Available)),
1198            alive_bytes: AtomicUsize::new(0),
1199        }
1200    }
1201
1202    /// open a new instance of the storage that is readonly
1203    fn reopen_as_readonly(&self, storage_access: StorageAccess) -> Option<Self> {
1204        if storage_access != StorageAccess::File {
1205            // if we are only using mmap, then no reason to re-open
1206            return None;
1207        }
1208
1209        let count_and_status = self.count_and_status.lock_write();
1210        self.accounts.reopen_as_readonly().map(|accounts| Self {
1211            id: self.id,
1212            slot: self.slot,
1213            count_and_status: SeqLock::new(*count_and_status),
1214            alive_bytes: AtomicUsize::new(self.alive_bytes()),
1215            accounts,
1216        })
1217    }
1218
1219    pub fn new_existing(
1220        slot: Slot,
1221        id: AccountsFileId,
1222        accounts: AccountsFile,
1223        _num_accounts: usize,
1224    ) -> Self {
1225        Self {
1226            id,
1227            slot,
1228            accounts,
1229            count_and_status: SeqLock::new((0, AccountStorageStatus::Available)),
1230            alive_bytes: AtomicUsize::new(0),
1231        }
1232    }
1233
1234    pub fn set_status(&self, mut status: AccountStorageStatus) {
1235        let mut count_and_status = self.count_and_status.lock_write();
1236
1237        let count = count_and_status.0;
1238
1239        if status == AccountStorageStatus::Full && count == 0 {
1240            // this case arises when the append_vec is full (store_ptrs fails),
1241            //  but all accounts have already been removed from the storage
1242            //
1243            // the only time it's safe to call reset() on an append_vec is when
1244            //  every account has been removed
1245            //          **and**
1246            //  the append_vec has previously been completely full
1247            //
1248            self.accounts.reset();
1249            status = AccountStorageStatus::Available;
1250        }
1251
1252        *count_and_status = (count, status);
1253    }
1254
1255    pub fn status(&self) -> AccountStorageStatus {
1256        self.count_and_status.read().1
1257    }
1258
1259    pub fn count(&self) -> usize {
1260        self.count_and_status.read().0
1261    }
1262
1263    pub fn alive_bytes(&self) -> usize {
1264        self.alive_bytes.load(Ordering::Acquire)
1265    }
1266
1267    pub fn written_bytes(&self) -> u64 {
1268        self.accounts.len() as u64
1269    }
1270
1271    pub fn capacity(&self) -> u64 {
1272        self.accounts.capacity()
1273    }
1274
1275    pub fn has_accounts(&self) -> bool {
1276        self.count() > 0
1277    }
1278
1279    pub fn slot(&self) -> Slot {
1280        self.slot
1281    }
1282
1283    pub fn id(&self) -> AccountsFileId {
1284        self.id
1285    }
1286
1287    pub fn flush(&self) -> Result<(), AccountsFileError> {
1288        self.accounts.flush()
1289    }
1290
1291    fn get_account_shared_data(&self, offset: usize) -> Option<AccountSharedData> {
1292        self.accounts.get_account_shared_data(offset)
1293    }
1294
1295    fn add_accounts(&self, num_accounts: usize, num_bytes: usize) {
1296        let mut count_and_status = self.count_and_status.lock_write();
1297        *count_and_status = (count_and_status.0 + num_accounts, count_and_status.1);
1298        self.alive_bytes.fetch_add(num_bytes, Ordering::Release);
1299    }
1300
1301    fn try_available(&self) -> bool {
1302        let mut count_and_status = self.count_and_status.lock_write();
1303        let (count, status) = *count_and_status;
1304
1305        if status == AccountStorageStatus::Available {
1306            *count_and_status = (count, AccountStorageStatus::Candidate);
1307            true
1308        } else {
1309            false
1310        }
1311    }
1312
1313    /// returns # of accounts remaining in the storage
1314    fn remove_accounts(
1315        &self,
1316        num_bytes: usize,
1317        reset_accounts: bool,
1318        num_accounts: usize,
1319    ) -> usize {
1320        let mut count_and_status = self.count_and_status.lock_write();
1321        let (mut count, mut status) = *count_and_status;
1322
1323        if count == num_accounts && status == AccountStorageStatus::Full && reset_accounts {
1324            // this case arises when we remove the last account from the
1325            //  storage, but we've learned from previous write attempts that
1326            //  the storage is full
1327            //
1328            // the only time it's safe to call reset() on an append_vec is when
1329            //  every account has been removed
1330            //          **and**
1331            //  the append_vec has previously been completely full
1332            //
1333            // otherwise, the storage may be in flight with a store()
1334            //   call
1335            self.accounts.reset();
1336            status = AccountStorageStatus::Available;
1337        }
1338
1339        // Some code path is removing accounts too many; this may result in an
1340        // unintended reveal of old state for unrelated accounts.
1341        assert!(
1342            count >= num_accounts,
1343            "double remove of account in slot: {}/store: {}!!",
1344            self.slot(),
1345            self.id(),
1346        );
1347
1348        self.alive_bytes.fetch_sub(num_bytes, Ordering::Release);
1349        count = count.saturating_sub(num_accounts);
1350        *count_and_status = (count, status);
1351        count
1352    }
1353
1354    /// Returns the path to the underlying accounts storage file
1355    pub fn path(&self) -> &Path {
1356        self.accounts.path()
1357    }
1358}
1359
1360pub fn get_temp_accounts_paths(count: u32) -> IoResult<(Vec<TempDir>, Vec<PathBuf>)> {
1361    let temp_dirs: IoResult<Vec<TempDir>> = (0..count).map(|_| TempDir::new()).collect();
1362    let temp_dirs = temp_dirs?;
1363
1364    let paths: IoResult<Vec<_>> = temp_dirs
1365        .iter()
1366        .map(|temp_dir| {
1367            utils::create_accounts_run_and_snapshot_dirs(temp_dir)
1368                .map(|(run_dir, _snapshot_dir)| run_dir)
1369        })
1370        .collect();
1371    let paths = paths?;
1372    Ok((temp_dirs, paths))
1373}
1374
1375#[derive(Default, Debug)]
1376struct CleaningInfo {
1377    slot_list: SlotList<AccountInfo>,
1378    ref_count: u64,
1379    /// Indicates if this account might have a zero lamport index entry.
1380    /// If false, the account *shall* not have zero lamport index entries.
1381    /// If true, the account *might* have zero lamport index entries.
1382    might_contain_zero_lamport_entry: bool,
1383}
1384
1385/// This is the return type of AccountsDb::construct_candidate_clean_keys.
1386/// It's a collection of pubkeys with associated information to
1387/// facilitate the decision making about which accounts can be removed
1388/// from the accounts index. In addition, the minimal dirty slot is
1389/// included in the returned value.
1390type CleaningCandidates = (Box<[RwLock<HashMap<Pubkey, CleaningInfo>>]>, Option<Slot>);
1391
1392/// Removing unrooted slots in Accounts Background Service needs to be synchronized with flushing
1393/// slots from the Accounts Cache.  This keeps track of those slots and the Mutex + Condvar for
1394/// synchronization.
1395#[derive(Debug, Default)]
1396struct RemoveUnrootedSlotsSynchronization {
1397    // slots being flushed from the cache or being purged
1398    slots_under_contention: Mutex<IntSet<Slot>>,
1399    signal: Condvar,
1400}
1401
1402type AccountInfoAccountsIndex = AccountsIndex<AccountInfo, AccountInfo>;
1403
1404// This structure handles the load/store of the accounts
1405#[derive(Debug)]
1406pub struct AccountsDb {
1407    /// Keeps tracks of index into AppendVec on a per slot basis
1408    pub accounts_index: AccountInfoAccountsIndex,
1409
1410    /// Some(offset) iff we want to squash old append vecs together into 'ancient append vecs'
1411    /// Some(offset) means for slots up to (max_slot - (slots_per_epoch - 'offset')), put them in ancient append vecs
1412    pub ancient_append_vec_offset: Option<i64>,
1413
1414    /// true iff we want to skip the initial hash calculation on startup
1415    pub skip_initial_hash_calc: bool,
1416
1417    pub storage: AccountStorage,
1418
1419    /// from AccountsDbConfig
1420    create_ancient_storage: CreateAncientStorage,
1421
1422    /// true if this client should skip rewrites but still include those rewrites in the bank hash as if rewrites had occurred.
1423    pub test_skip_rewrites_but_include_in_bank_hash: bool,
1424
1425    pub accounts_cache: AccountsCache,
1426
1427    write_cache_limit_bytes: Option<u64>,
1428
1429    sender_bg_hasher: Option<Sender<Vec<CachedAccount>>>,
1430    read_only_accounts_cache: ReadOnlyAccountsCache,
1431
1432    /// distribute the accounts across storage lists
1433    pub next_id: AtomicAccountsFileId,
1434
1435    /// Set of shrinkable stores organized by map of slot to storage id
1436    pub shrink_candidate_slots: Mutex<ShrinkCandidates>,
1437
1438    pub write_version: AtomicU64,
1439
1440    /// Set of storage paths to pick from
1441    pub paths: Vec<PathBuf>,
1442
1443    /// Base directory for various necessary files
1444    base_working_path: PathBuf,
1445    // used by tests - held until we are dropped
1446    #[allow(dead_code)]
1447    base_working_temp_dir: Option<TempDir>,
1448
1449    accounts_hash_cache_path: PathBuf,
1450
1451    shrink_paths: Vec<PathBuf>,
1452
1453    /// Directory of paths this accounts_db needs to hold/remove
1454    #[allow(dead_code)]
1455    pub temp_paths: Option<Vec<TempDir>>,
1456
1457    /// Starting file size of appendvecs
1458    file_size: u64,
1459
1460    /// Thread pool used for par_iter
1461    pub thread_pool: ThreadPool,
1462
1463    pub thread_pool_clean: ThreadPool,
1464
1465    pub thread_pool_hash: ThreadPool,
1466
1467    bank_hash_stats: Mutex<HashMap<Slot, BankHashStats>>,
1468    accounts_delta_hashes: Mutex<HashMap<Slot, AccountsDeltaHash>>,
1469    accounts_hashes: Mutex<HashMap<Slot, (AccountsHash, /*capitalization*/ u64)>>,
1470    incremental_accounts_hashes:
1471        Mutex<HashMap<Slot, (IncrementalAccountsHash, /*capitalization*/ u64)>>,
1472
1473    pub stats: AccountsStats,
1474
1475    clean_accounts_stats: CleanAccountsStats,
1476
1477    // Stats for purges called outside of clean_accounts()
1478    external_purge_slots_stats: PurgeStats,
1479
1480    pub shrink_stats: ShrinkStats,
1481
1482    pub(crate) shrink_ancient_stats: ShrinkAncientStats,
1483
1484    pub account_indexes: AccountSecondaryIndexes,
1485
1486    /// Set of unique keys per slot which is used
1487    /// to drive clean_accounts
1488    /// Generated by calculate_accounts_delta_hash
1489    uncleaned_pubkeys: DashMap<Slot, Vec<Pubkey>>,
1490
1491    #[cfg(test)]
1492    load_delay: u64,
1493
1494    #[cfg(test)]
1495    load_limit: AtomicU64,
1496
1497    /// true if drop_callback is attached to the bank.
1498    is_bank_drop_callback_enabled: AtomicBool,
1499
1500    /// Set of slots currently being flushed by `flush_slot_cache()` or removed
1501    /// by `remove_unrooted_slot()`. Used to ensure `remove_unrooted_slots(slots)`
1502    /// can safely clear the set of unrooted slots `slots`.
1503    remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization,
1504
1505    shrink_ratio: AccountShrinkThreshold,
1506
1507    /// Set of stores which are recently rooted or had accounts removed
1508    /// such that potentially a 0-lamport account update could be present which
1509    /// means we can remove the account from the index entirely.
1510    dirty_stores: DashMap<Slot, Arc<AccountStorageEntry>>,
1511
1512    /// Zero-lamport accounts that are *not* purged during clean because they need to stay alive
1513    /// for incremental snapshot support.
1514    zero_lamport_accounts_to_purge_after_full_snapshot: DashSet<(Slot, Pubkey)>,
1515
1516    /// GeyserPlugin accounts update notifier
1517    accounts_update_notifier: Option<AccountsUpdateNotifier>,
1518
1519    pub(crate) active_stats: ActiveStats,
1520
1521    pub verify_accounts_hash_in_bg: VerifyAccountsHashInBackground,
1522
1523    /// Used to disable logging dead slots during removal.
1524    /// allow disabling noisy log
1525    pub log_dead_slots: AtomicBool,
1526
1527    /// debug feature to scan every append vec and verify refcounts are equal
1528    exhaustively_verify_refcounts: bool,
1529
1530    /// storage format to use for new storages
1531    accounts_file_provider: AccountsFileProvider,
1532
1533    /// method to use for accessing storages
1534    storage_access: StorageAccess,
1535
1536    /// index scan filtering for shrinking
1537    scan_filter_for_shrinking: ScanFilter,
1538
1539    /// this will live here until the feature for partitioned epoch rewards is activated.
1540    /// At that point, this and other code can be deleted.
1541    pub partitioned_epoch_rewards_config: PartitionedEpochRewardsConfig,
1542
1543    /// the full accounts hash calculation as of a predetermined block height 'N'
1544    /// to be included in the bank hash at a predetermined block height 'M'
1545    /// The cadence is once per epoch, all nodes calculate a full accounts hash as of a known slot calculated using 'N'
1546    /// Some time later (to allow for slow calculation time), the bank hash at a slot calculated using 'M' includes the full accounts hash.
1547    /// Thus, the state of all accounts on a validator is known to be correct at least once per epoch.
1548    pub epoch_accounts_hash_manager: EpochAccountsHashManager,
1549
1550    /// The latest full snapshot slot dictates how to handle zero lamport accounts
1551    /// Note, this is None if we're told to *not* take snapshots
1552    latest_full_snapshot_slot: SeqLock<Option<Slot>>,
1553
1554    /// Flag to indicate if the experimental accounts lattice hash is enabled.
1555    /// (For R&D only; a feature-gate also exists to turn this on and make it a part of consensus.)
1556    pub is_experimental_accumulator_hash_enabled: AtomicBool,
1557
1558    /// These are the ancient storages that could be valuable to
1559    /// shrink, sorted by amount of dead bytes.  The elements
1560    /// are sorted from the largest dead bytes to the smallest.
1561    /// Members are Slot and capacity. If capacity is smaller, then
1562    /// that means the storage was already shrunk.
1563    pub(crate) best_ancient_slots_to_shrink: RwLock<VecDeque<(Slot, u64)>>,
1564}
1565
1566/// results from 'split_storages_ancient'
1567#[derive(Debug, Default, PartialEq)]
1568struct SplitAncientStorages {
1569    /// # ancient slots
1570    ancient_slot_count: usize,
1571    /// the specific ancient slots
1572    ancient_slots: Vec<Slot>,
1573    /// lowest slot that is not an ancient append vec
1574    first_non_ancient_slot: Slot,
1575    /// slot # of beginning of first aligned chunk starting from the first non ancient slot
1576    first_chunk_start: Slot,
1577    /// # non-ancient slots to scan
1578    non_ancient_slot_count: usize,
1579    /// # chunks to use to iterate the storages
1580    /// all ancient chunks, the special 0 and last chunks for non-full chunks, and all the 'full' chunks of normal slots
1581    chunk_count: usize,
1582    /// start and end(exclusive) of normal (non-ancient) slots to be scanned
1583    normal_slot_range: Range<Slot>,
1584}
1585
1586impl SplitAncientStorages {
1587    /// When calculating accounts hash, we break the slots/storages into chunks that remain the same during an entire epoch.
1588    /// a slot is in this chunk of slots:
1589    /// start:         (slot / MAX_ITEMS_PER_CHUNK) * MAX_ITEMS_PER_CHUNK
1590    /// end_exclusive: start + MAX_ITEMS_PER_CHUNK
1591    /// So a slot remains in the same chunk whenever it is included in the accounts hash.
1592    /// When the slot gets deleted or gets consumed in an ancient append vec, it will no longer be in its chunk.
1593    /// The results of scanning a chunk of appendvecs can be cached to avoid scanning large amounts of data over and over.
1594    fn new(oldest_non_ancient_slot: Option<Slot>, snapshot_storages: &SortedStorages) -> Self {
1595        let range = snapshot_storages.range();
1596
1597        let (ancient_slots, first_non_ancient_slot) = if let Some(oldest_non_ancient_slot) =
1598            oldest_non_ancient_slot
1599        {
1600            // any ancient append vecs should definitely be cached
1601            // We need to break the ranges into:
1602            // 1. individual ancient append vecs (may be empty)
1603            // 2. first unevenly divided chunk starting at 1 epoch old slot (may be empty)
1604            // 3. evenly divided full chunks in the middle
1605            // 4. unevenly divided chunk of most recent slots (may be empty)
1606            let ancient_slots =
1607                Self::get_ancient_slots(oldest_non_ancient_slot, snapshot_storages, |storage| {
1608                    storage.capacity() > get_ancient_append_vec_capacity() * 50 / 100
1609                });
1610
1611            let first_non_ancient_slot = ancient_slots
1612                .last()
1613                .map(|last_ancient_slot| last_ancient_slot.saturating_add(1))
1614                .unwrap_or(range.start);
1615
1616            (ancient_slots, first_non_ancient_slot)
1617        } else {
1618            (vec![], range.start)
1619        };
1620
1621        Self::new_with_ancient_info(range, ancient_slots, first_non_ancient_slot)
1622    }
1623
1624    /// return all ancient append vec slots from the early slots referenced by 'snapshot_storages'
1625    /// `treat_as_ancient` returns true if the storage at this slot is large and should be treated individually by accounts hash calculation.
1626    /// `treat_as_ancient` is a fn so that we can test this well. Otherwise, we have to generate large append vecs to pass the intended checks.
1627    fn get_ancient_slots(
1628        oldest_non_ancient_slot: Slot,
1629        snapshot_storages: &SortedStorages,
1630        treat_as_ancient: impl Fn(&AccountStorageEntry) -> bool,
1631    ) -> Vec<Slot> {
1632        let range = snapshot_storages.range();
1633        let mut i = 0;
1634        let mut len_truncate = 0;
1635        let mut possible_ancient_slots = snapshot_storages
1636            .iter_range(&(range.start..oldest_non_ancient_slot))
1637            .filter_map(|(slot, storage)| {
1638                storage.map(|storage| {
1639                    i += 1;
1640                    if treat_as_ancient(storage) {
1641                        // even though the slot is in range of being an ancient append vec, if it isn't actually a large append vec,
1642                        // then we are better off treating all these slots as normally cacheable to reduce work in dedup.
1643                        // Since this one is large, for the moment, this one becomes the highest slot where we want to individually cache files.
1644                        len_truncate = i;
1645                    }
1646                    slot
1647                })
1648            })
1649            .collect::<Vec<_>>();
1650        possible_ancient_slots.truncate(len_truncate);
1651        possible_ancient_slots
1652    }
1653
1654    /// create once ancient slots have been identified
1655    /// This is easier to test, removing SortedStorages as a type to deal with here.
1656    fn new_with_ancient_info(
1657        range: &Range<Slot>,
1658        ancient_slots: Vec<Slot>,
1659        first_non_ancient_slot: Slot,
1660    ) -> Self {
1661        if range.is_empty() {
1662            // Corner case mainly for tests, but gives us a consistent base case. Makes more sense to return default here than anything else.
1663            // caller is asking to split for empty set of slots
1664            return SplitAncientStorages::default();
1665        }
1666
1667        let max_slot_inclusive = range.end.saturating_sub(1);
1668        let ancient_slot_count = ancient_slots.len();
1669        let first_chunk_start = ((first_non_ancient_slot + MAX_ITEMS_PER_CHUNK)
1670            / MAX_ITEMS_PER_CHUNK)
1671            * MAX_ITEMS_PER_CHUNK;
1672
1673        let non_ancient_slot_count = (max_slot_inclusive - first_non_ancient_slot + 1) as usize;
1674
1675        let normal_slot_range = Range {
1676            start: first_non_ancient_slot,
1677            end: range.end,
1678        };
1679
1680        // 2 is for 2 special chunks - unaligned slots at the beginning and end
1681        let chunk_count =
1682            ancient_slot_count + 2 + non_ancient_slot_count / (MAX_ITEMS_PER_CHUNK as usize);
1683
1684        SplitAncientStorages {
1685            ancient_slot_count,
1686            ancient_slots,
1687            first_non_ancient_slot,
1688            first_chunk_start,
1689            non_ancient_slot_count,
1690            chunk_count,
1691            normal_slot_range,
1692        }
1693    }
1694
1695    /// given 'normal_chunk', return the starting slot of that chunk in the normal/non-ancient range
1696    /// a normal_chunk is 0<=normal_chunk<=non_ancient_chunk_count
1697    /// non_ancient_chunk_count is chunk_count-ancient_slot_count
1698    fn get_starting_slot_from_normal_chunk(&self, normal_chunk: usize) -> Slot {
1699        if normal_chunk == 0 {
1700            self.normal_slot_range.start
1701        } else {
1702            assert!(
1703                normal_chunk.saturating_add(self.ancient_slot_count) < self.chunk_count,
1704                "out of bounds: {}, {}",
1705                normal_chunk,
1706                self.chunk_count
1707            );
1708
1709            let normal_chunk = normal_chunk.saturating_sub(1);
1710            (self.first_chunk_start + MAX_ITEMS_PER_CHUNK * (normal_chunk as Slot))
1711                .max(self.normal_slot_range.start)
1712        }
1713    }
1714
1715    /// ancient slots are the first chunks
1716    fn is_chunk_ancient(&self, chunk: usize) -> bool {
1717        chunk < self.ancient_slot_count
1718    }
1719
1720    /// given chunk in 0<=chunk<self.chunk_count
1721    /// return the range of slots in that chunk
1722    /// None indicates the range is empty for that chunk.
1723    fn get_slot_range(&self, chunk: usize) -> Option<Range<Slot>> {
1724        let range = if self.is_chunk_ancient(chunk) {
1725            // ancient append vecs are handled individually
1726            let slot = self.ancient_slots[chunk];
1727            Range {
1728                start: slot,
1729                end: slot + 1,
1730            }
1731        } else {
1732            // normal chunks are after ancient chunks
1733            let normal_chunk = chunk - self.ancient_slot_count;
1734            if normal_chunk == 0 {
1735                // first slot
1736                Range {
1737                    start: self.normal_slot_range.start,
1738                    end: self.first_chunk_start.min(self.normal_slot_range.end),
1739                }
1740            } else {
1741                // normal full chunk or the last chunk
1742                let first_slot = self.get_starting_slot_from_normal_chunk(normal_chunk);
1743                Range {
1744                    start: first_slot,
1745                    end: (first_slot + MAX_ITEMS_PER_CHUNK).min(self.normal_slot_range.end),
1746                }
1747            }
1748        };
1749        // return empty range as None
1750        (!range.is_empty()).then_some(range)
1751    }
1752}
1753
1754pub fn quarter_thread_count() -> usize {
1755    std::cmp::max(2, num_cpus::get() / 4)
1756}
1757
1758pub fn make_min_priority_thread_pool() -> ThreadPool {
1759    // Use lower thread count to reduce priority.
1760    let num_threads = quarter_thread_count();
1761    rayon::ThreadPoolBuilder::new()
1762        .thread_name(|i| format!("solAccountsLo{i:02}"))
1763        .num_threads(num_threads)
1764        .build()
1765        .unwrap()
1766}
1767
1768/// Returns the default number of threads to use for background accounts hashing
1769pub fn default_num_hash_threads() -> NonZeroUsize {
1770    // 1/8 of the number of cpus and up to 6 threads gives good balance for the system.
1771    let num_threads = (num_cpus::get() / 8).clamp(2, 6);
1772    NonZeroUsize::new(num_threads).unwrap()
1773}
1774
1775pub fn make_hash_thread_pool(num_threads: Option<NonZeroUsize>) -> ThreadPool {
1776    let num_threads = num_threads.unwrap_or_else(default_num_hash_threads).get();
1777    rayon::ThreadPoolBuilder::new()
1778        .thread_name(|i| format!("solAcctHash{i:02}"))
1779        .num_threads(num_threads)
1780        .build()
1781        .unwrap()
1782}
1783
1784pub fn default_num_foreground_threads() -> usize {
1785    get_thread_count()
1786}
1787
1788#[cfg(feature = "frozen-abi")]
1789impl solana_frozen_abi::abi_example::AbiExample for AccountsDb {
1790    fn example() -> Self {
1791        let accounts_db = AccountsDb::new_single_for_tests();
1792        let key = Pubkey::default();
1793        let some_data_len = 5;
1794        let some_slot: Slot = 0;
1795        let account = AccountSharedData::new(1, some_data_len, &key);
1796        accounts_db.store_uncached(some_slot, &[(&key, &account)]);
1797        accounts_db.add_root(0);
1798
1799        accounts_db
1800    }
1801}
1802
1803impl<'a> ZeroLamport for StoredAccountMeta<'a> {
1804    fn is_zero_lamport(&self) -> bool {
1805        self.lamports() == 0
1806    }
1807}
1808
1809#[derive(Clone, Debug, Eq, PartialEq)]
1810pub struct PubkeyHashAccount {
1811    pub pubkey: Pubkey,
1812    pub hash: AccountHash,
1813    pub account: AccountSharedData,
1814}
1815
1816impl AccountsDb {
1817    pub const DEFAULT_ACCOUNTS_HASH_CACHE_DIR: &'static str = "accounts_hash_cache";
1818
1819    // read only cache does not update lru on read of an entry unless it has been at least this many ms since the last lru update
1820    const READ_ONLY_CACHE_MS_TO_SKIP_LRU_UPDATE: u32 = 100;
1821
1822    // The default high and low watermark sizes for the accounts read cache.
1823    // If the cache size exceeds MAX_SIZE_HI, it'll evict entries until the size is <= MAX_SIZE_LO.
1824    const DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_LO: usize = 400 * 1024 * 1024;
1825    const DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_HI: usize = 410 * 1024 * 1024;
1826
1827    pub fn default_for_tests() -> Self {
1828        Self::new_single_for_tests()
1829    }
1830
1831    pub fn new_single_for_tests() -> Self {
1832        AccountsDb::new_for_tests(Vec::new())
1833    }
1834
1835    pub fn new_single_for_tests_with_provider(file_provider: AccountsFileProvider) -> Self {
1836        AccountsDb::new_for_tests_with_provider(Vec::new(), file_provider)
1837    }
1838
1839    pub fn new_for_tests(paths: Vec<PathBuf>) -> Self {
1840        Self::new_for_tests_with_provider(paths, AccountsFileProvider::default())
1841    }
1842
1843    fn new_for_tests_with_provider(
1844        paths: Vec<PathBuf>,
1845        accounts_file_provider: AccountsFileProvider,
1846    ) -> Self {
1847        let mut db = AccountsDb::new_with_config(
1848            paths,
1849            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
1850            None,
1851            Arc::default(),
1852        );
1853        db.accounts_file_provider = accounts_file_provider;
1854        db
1855    }
1856
1857    pub fn new_with_config(
1858        paths: Vec<PathBuf>,
1859        accounts_db_config: Option<AccountsDbConfig>,
1860        accounts_update_notifier: Option<AccountsUpdateNotifier>,
1861        exit: Arc<AtomicBool>,
1862    ) -> Self {
1863        let accounts_db_config = accounts_db_config.unwrap_or_default();
1864        let accounts_index = AccountsIndex::new(accounts_db_config.index.clone(), exit);
1865
1866        let base_working_path = accounts_db_config.base_working_path.clone();
1867        let (base_working_path, base_working_temp_dir) =
1868            if let Some(base_working_path) = base_working_path {
1869                (base_working_path, None)
1870            } else {
1871                let base_working_temp_dir = TempDir::new().unwrap();
1872                let base_working_path = base_working_temp_dir.path().to_path_buf();
1873                (base_working_path, Some(base_working_temp_dir))
1874            };
1875
1876        let (paths, temp_paths) = if paths.is_empty() {
1877            // Create a temporary set of accounts directories, used primarily
1878            // for testing
1879            let (temp_dirs, temp_paths) = get_temp_accounts_paths(DEFAULT_NUM_DIRS).unwrap();
1880            (temp_paths, Some(temp_dirs))
1881        } else {
1882            (paths, None)
1883        };
1884
1885        let shrink_paths = accounts_db_config
1886            .shrink_paths
1887            .clone()
1888            .unwrap_or_else(|| paths.clone());
1889
1890        let accounts_hash_cache_path = accounts_db_config.accounts_hash_cache_path.clone();
1891        let accounts_hash_cache_path = accounts_hash_cache_path.unwrap_or_else(|| {
1892            let accounts_hash_cache_path =
1893                base_working_path.join(Self::DEFAULT_ACCOUNTS_HASH_CACHE_DIR);
1894            if !accounts_hash_cache_path.exists() {
1895                fs::create_dir(&accounts_hash_cache_path).expect("create accounts hash cache dir");
1896            }
1897            accounts_hash_cache_path
1898        });
1899
1900        let test_partitioned_epoch_rewards = accounts_db_config.test_partitioned_epoch_rewards;
1901        let partitioned_epoch_rewards_config: PartitionedEpochRewardsConfig =
1902            PartitionedEpochRewardsConfig::new(test_partitioned_epoch_rewards);
1903
1904        let read_cache_size = accounts_db_config.read_cache_limit_bytes.unwrap_or((
1905            Self::DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_LO,
1906            Self::DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_HI,
1907        ));
1908
1909        let bank_hash_stats = Mutex::new(HashMap::from([(0, BankHashStats::default())]));
1910
1911        // Increase the stack for foreground threads
1912        // rayon needs a lot of stack
1913        const ACCOUNTS_STACK_SIZE: usize = 8 * 1024 * 1024;
1914        let num_foreground_threads = accounts_db_config
1915            .num_foreground_threads
1916            .map(Into::into)
1917            .unwrap_or_else(default_num_foreground_threads);
1918        let thread_pool = rayon::ThreadPoolBuilder::new()
1919            .num_threads(num_foreground_threads)
1920            .thread_name(|i| format!("solAccounts{i:02}"))
1921            .stack_size(ACCOUNTS_STACK_SIZE)
1922            .build()
1923            .expect("new rayon threadpool");
1924
1925        let num_clean_threads = accounts_db_config
1926            .num_clean_threads
1927            .map(Into::into)
1928            .unwrap_or_else(quarter_thread_count);
1929        let thread_pool_clean = rayon::ThreadPoolBuilder::new()
1930            .thread_name(|i| format!("solAccountsLo{i:02}"))
1931            .num_threads(num_clean_threads)
1932            .build()
1933            .expect("new rayon threadpool");
1934
1935        let thread_pool_hash = make_hash_thread_pool(accounts_db_config.num_hash_threads);
1936
1937        let mut new = Self {
1938            accounts_index,
1939            paths,
1940            base_working_path,
1941            base_working_temp_dir,
1942            accounts_hash_cache_path,
1943            temp_paths,
1944            shrink_paths,
1945            skip_initial_hash_calc: accounts_db_config.skip_initial_hash_calc,
1946            ancient_append_vec_offset: accounts_db_config
1947                .ancient_append_vec_offset
1948                .or(ANCIENT_APPEND_VEC_DEFAULT_OFFSET),
1949            account_indexes: accounts_db_config.account_indexes.unwrap_or_default(),
1950            shrink_ratio: accounts_db_config.shrink_ratio,
1951            accounts_update_notifier,
1952            create_ancient_storage: accounts_db_config.create_ancient_storage,
1953            read_only_accounts_cache: ReadOnlyAccountsCache::new(
1954                read_cache_size.0,
1955                read_cache_size.1,
1956                Self::READ_ONLY_CACHE_MS_TO_SKIP_LRU_UPDATE,
1957            ),
1958            write_cache_limit_bytes: accounts_db_config.write_cache_limit_bytes,
1959            partitioned_epoch_rewards_config,
1960            exhaustively_verify_refcounts: accounts_db_config.exhaustively_verify_refcounts,
1961            test_skip_rewrites_but_include_in_bank_hash: accounts_db_config
1962                .test_skip_rewrites_but_include_in_bank_hash,
1963            storage_access: accounts_db_config.storage_access,
1964            scan_filter_for_shrinking: accounts_db_config.scan_filter_for_shrinking,
1965            is_experimental_accumulator_hash_enabled: accounts_db_config
1966                .enable_experimental_accumulator_hash
1967                .into(),
1968            bank_hash_stats,
1969            thread_pool,
1970            thread_pool_clean,
1971            thread_pool_hash,
1972            verify_accounts_hash_in_bg: VerifyAccountsHashInBackground::default(),
1973            active_stats: ActiveStats::default(),
1974            storage: AccountStorage::default(),
1975            accounts_cache: AccountsCache::default(),
1976            sender_bg_hasher: None,
1977            uncleaned_pubkeys: DashMap::new(),
1978            next_id: AtomicAccountsFileId::new(0),
1979            shrink_candidate_slots: Mutex::new(ShrinkCandidates::default()),
1980            write_version: AtomicU64::new(0),
1981            file_size: DEFAULT_FILE_SIZE,
1982            accounts_delta_hashes: Mutex::new(HashMap::new()),
1983            accounts_hashes: Mutex::new(HashMap::new()),
1984            incremental_accounts_hashes: Mutex::new(HashMap::new()),
1985            external_purge_slots_stats: PurgeStats::default(),
1986            clean_accounts_stats: CleanAccountsStats::default(),
1987            shrink_stats: ShrinkStats::default(),
1988            shrink_ancient_stats: ShrinkAncientStats::default(),
1989            stats: AccountsStats::default(),
1990            #[cfg(test)]
1991            load_delay: u64::default(),
1992            #[cfg(test)]
1993            load_limit: AtomicU64::default(),
1994            is_bank_drop_callback_enabled: AtomicBool::default(),
1995            remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization::default(),
1996            dirty_stores: DashMap::default(),
1997            zero_lamport_accounts_to_purge_after_full_snapshot: DashSet::default(),
1998            log_dead_slots: AtomicBool::new(true),
1999            accounts_file_provider: AccountsFileProvider::default(),
2000            epoch_accounts_hash_manager: EpochAccountsHashManager::new_invalid(),
2001            latest_full_snapshot_slot: SeqLock::new(None),
2002            best_ancient_slots_to_shrink: RwLock::default(),
2003        };
2004
2005        new.start_background_hasher();
2006        {
2007            for path in new.paths.iter() {
2008                std::fs::create_dir_all(path).expect("Create directory failed.");
2009            }
2010        }
2011        new
2012    }
2013
2014    pub fn file_size(&self) -> u64 {
2015        self.file_size
2016    }
2017
2018    /// Get the base working directory
2019    pub fn get_base_working_path(&self) -> PathBuf {
2020        self.base_working_path.clone()
2021    }
2022
2023    /// Returns true if there is an accounts update notifier.
2024    pub fn has_accounts_update_notifier(&self) -> bool {
2025        self.accounts_update_notifier.is_some()
2026    }
2027
2028    fn next_id(&self) -> AccountsFileId {
2029        let next_id = self.next_id.fetch_add(1, Ordering::AcqRel);
2030        assert!(
2031            next_id != AccountsFileId::MAX,
2032            "We've run out of storage ids!"
2033        );
2034        next_id
2035    }
2036
2037    fn new_storage_entry(&self, slot: Slot, path: &Path, size: u64) -> AccountStorageEntry {
2038        AccountStorageEntry::new(
2039            path,
2040            slot,
2041            self.next_id(),
2042            size,
2043            self.accounts_file_provider,
2044        )
2045    }
2046
2047    /// Returns if the experimental accounts lattice hash is enabled
2048    pub fn is_experimental_accumulator_hash_enabled(&self) -> bool {
2049        self.is_experimental_accumulator_hash_enabled
2050            .load(Ordering::Acquire)
2051    }
2052
2053    /// Sets if the experimental accounts lattice hash is enabled
2054    pub fn set_is_experimental_accumulator_hash_enabled(&self, is_enabled: bool) {
2055        self.is_experimental_accumulator_hash_enabled
2056            .store(is_enabled, Ordering::Release);
2057    }
2058
2059    /// While scanning cleaning candidates obtain slots that can be
2060    /// reclaimed for each pubkey. In addition, if the pubkey is
2061    /// removed from the index, insert in pubkeys_removed_from_accounts_index.
2062    fn collect_reclaims(
2063        &self,
2064        pubkey: &Pubkey,
2065        max_clean_root_inclusive: Option<Slot>,
2066        ancient_account_cleans: &AtomicU64,
2067        epoch_schedule: &EpochSchedule,
2068        pubkeys_removed_from_accounts_index: &Mutex<PubkeysRemovedFromAccountsIndex>,
2069    ) -> SlotList<AccountInfo> {
2070        let one_epoch_old = self.get_oldest_non_ancient_slot(epoch_schedule);
2071        let mut clean_rooted = Measure::start("clean_old_root-ms");
2072        let mut reclaims = Vec::new();
2073        let removed_from_index = self.accounts_index.clean_rooted_entries(
2074            pubkey,
2075            &mut reclaims,
2076            max_clean_root_inclusive,
2077        );
2078        if removed_from_index {
2079            pubkeys_removed_from_accounts_index
2080                .lock()
2081                .unwrap()
2082                .insert(*pubkey);
2083        }
2084        if !reclaims.is_empty() {
2085            // figure out how many ancient accounts have been reclaimed
2086            let old_reclaims = reclaims
2087                .iter()
2088                .filter_map(|(slot, _)| (slot < &one_epoch_old).then_some(1))
2089                .sum();
2090            ancient_account_cleans.fetch_add(old_reclaims, Ordering::Relaxed);
2091        }
2092        clean_rooted.stop();
2093        self.clean_accounts_stats
2094            .clean_old_root_us
2095            .fetch_add(clean_rooted.as_us(), Ordering::Relaxed);
2096        reclaims
2097    }
2098
2099    /// Reclaim older states of accounts older than max_clean_root_inclusive for AccountsDb bloat mitigation.
2100    /// Any accounts which are removed from the accounts index are returned in PubkeysRemovedFromAccountsIndex.
2101    /// These should NOT be unref'd later from the accounts index.
2102    fn clean_accounts_older_than_root(
2103        &self,
2104        reclaims: &SlotList<AccountInfo>,
2105        pubkeys_removed_from_accounts_index: &HashSet<Pubkey>,
2106    ) -> ReclaimResult {
2107        let mut measure = Measure::start("clean_old_root_reclaims");
2108
2109        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
2110        // and those stores may be used for background hashing.
2111        let reset_accounts = false;
2112
2113        let reclaim_result = self.handle_reclaims(
2114            (!reclaims.is_empty()).then(|| reclaims.iter()),
2115            None,
2116            reset_accounts,
2117            pubkeys_removed_from_accounts_index,
2118            HandleReclaims::ProcessDeadSlots(&self.clean_accounts_stats.purge_stats),
2119        );
2120        measure.stop();
2121        debug!("{}", measure);
2122        self.clean_accounts_stats
2123            .clean_old_root_reclaim_us
2124            .fetch_add(measure.as_us(), Ordering::Relaxed);
2125        reclaim_result
2126    }
2127
2128    fn do_reset_uncleaned_roots(&self, max_clean_root: Option<Slot>) {
2129        let mut measure = Measure::start("reset");
2130        self.accounts_index.reset_uncleaned_roots(max_clean_root);
2131        measure.stop();
2132        self.clean_accounts_stats
2133            .reset_uncleaned_roots_us
2134            .fetch_add(measure.as_us(), Ordering::Relaxed);
2135    }
2136
2137    /// increment store_counts to non-zero for all stores that can not be deleted.
2138    /// a store cannot be deleted if:
2139    /// 1. one of the pubkeys in the store has account info to a store whose store count is not going to zero
2140    /// 2. a pubkey we were planning to remove is not removing all stores that contain the account
2141    fn calc_delete_dependencies(
2142        &self,
2143        candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>],
2144        store_counts: &mut HashMap<Slot, (usize, HashSet<Pubkey>)>,
2145        min_slot: Option<Slot>,
2146    ) {
2147        // Another pass to check if there are some filtered accounts which
2148        // do not match the criteria of deleting all appendvecs which contain them
2149        // then increment their storage count.
2150        let mut already_counted = IntSet::default();
2151        for (bin_index, bin) in candidates.iter().enumerate() {
2152            let bin = bin.read().unwrap();
2153            for (
2154                pubkey,
2155                CleaningInfo {
2156                    slot_list,
2157                    ref_count,
2158                    ..
2159                },
2160            ) in bin.iter()
2161            {
2162                let mut failed_slot = None;
2163                let all_stores_being_deleted = slot_list.len() as RefCount == *ref_count;
2164                if all_stores_being_deleted {
2165                    let mut delete = true;
2166                    for (slot, _account_info) in slot_list {
2167                        if let Some(count) = store_counts.get(slot).map(|s| s.0) {
2168                            debug!(
2169                                "calc_delete_dependencies()
2170                            slot: {slot},
2171                            count len: {count}"
2172                            );
2173                            if count == 0 {
2174                                // this store CAN be removed
2175                                continue;
2176                            }
2177                        }
2178                        // One of the pubkeys in the store has account info to a store whose store count is not going to zero.
2179                        // If the store cannot be found, that also means store isn't being deleted.
2180                        failed_slot = Some(*slot);
2181                        delete = false;
2182                        break;
2183                    }
2184                    if delete {
2185                        // this pubkey can be deleted from all stores it is in
2186                        continue;
2187                    }
2188                } else {
2189                    // a pubkey we were planning to remove is not removing all stores that contain the account
2190                    debug!(
2191                        "calc_delete_dependencies(),
2192                    pubkey: {},
2193                    slot_list: {:?},
2194                    slot_list_len: {},
2195                    ref_count: {}",
2196                        pubkey,
2197                        slot_list,
2198                        slot_list.len(),
2199                        ref_count,
2200                    );
2201                }
2202
2203                // increment store_counts to non-zero for all stores that can not be deleted.
2204                let mut pending_stores = IntSet::default();
2205                for (slot, _account_info) in slot_list {
2206                    if !already_counted.contains(slot) {
2207                        pending_stores.insert(*slot);
2208                    }
2209                }
2210                while !pending_stores.is_empty() {
2211                    let slot = pending_stores.iter().next().cloned().unwrap();
2212                    if Some(slot) == min_slot {
2213                        if let Some(failed_slot) = failed_slot.take() {
2214                            info!(
2215                                "calc_delete_dependencies, oldest slot is not able to be deleted \
2216                                 because of {pubkey} in slot {failed_slot}"
2217                            );
2218                        } else {
2219                            info!(
2220                                "calc_delete_dependencies, oldest slot is not able to be deleted \
2221                                 because of {pubkey}, slot list len: {}, ref count: {ref_count}",
2222                                slot_list.len()
2223                            );
2224                        }
2225                    }
2226
2227                    pending_stores.remove(&slot);
2228                    if !already_counted.insert(slot) {
2229                        continue;
2230                    }
2231                    // the point of all this code: remove the store count for all stores we cannot remove
2232                    if let Some(store_count) = store_counts.remove(&slot) {
2233                        // all pubkeys in this store also cannot be removed from all stores they are in
2234                        let affected_pubkeys = &store_count.1;
2235                        for key in affected_pubkeys {
2236                            let candidates_bin_index =
2237                                self.accounts_index.bin_calculator.bin_from_pubkey(key);
2238                            let mut update_pending_stores =
2239                                |bin: &HashMap<Pubkey, CleaningInfo>| {
2240                                    for (slot, _account_info) in &bin.get(key).unwrap().slot_list {
2241                                        if !already_counted.contains(slot) {
2242                                            pending_stores.insert(*slot);
2243                                        }
2244                                    }
2245                                };
2246                            if candidates_bin_index == bin_index {
2247                                update_pending_stores(&bin);
2248                            } else {
2249                                update_pending_stores(
2250                                    &candidates[candidates_bin_index].read().unwrap(),
2251                                );
2252                            }
2253                        }
2254                    }
2255                }
2256            }
2257        }
2258    }
2259
2260    fn background_hasher(receiver: Receiver<Vec<CachedAccount>>) {
2261        info!("Background account hasher has started");
2262        loop {
2263            let result = receiver.try_recv();
2264            match result {
2265                Ok(accounts) => {
2266                    for account in accounts {
2267                        // if we hold the only ref, then this account doesn't need to be hashed, we ignore this account and it will disappear
2268                        if Arc::strong_count(&account) > 1 {
2269                            // this will cause the hash to be calculated and store inside account if it needs to be calculated
2270                            let _ = (*account).hash();
2271                        };
2272                    }
2273                }
2274                Err(TryRecvError::Empty) => {
2275                    sleep(Duration::from_millis(5));
2276                }
2277                Err(err @ TryRecvError::Disconnected) => {
2278                    info!("Background account hasher is stopping because: {err}");
2279                    break;
2280                }
2281            }
2282        }
2283        info!("Background account hasher has stopped");
2284    }
2285
2286    fn start_background_hasher(&mut self) {
2287        let (sender, receiver) = unbounded();
2288        Builder::new()
2289            .name("solDbStoreHashr".to_string())
2290            .spawn(move || {
2291                Self::background_hasher(receiver);
2292            })
2293            .unwrap();
2294        self.sender_bg_hasher = Some(sender);
2295    }
2296
2297    #[must_use]
2298    pub fn purge_keys_exact<'a, C>(
2299        &'a self,
2300        pubkey_to_slot_set: impl Iterator<Item = &'a (Pubkey, C)>,
2301    ) -> (Vec<(Slot, AccountInfo)>, PubkeysRemovedFromAccountsIndex)
2302    where
2303        C: Contains<'a, Slot> + 'a,
2304    {
2305        let mut reclaims = Vec::new();
2306        let mut dead_keys = Vec::new();
2307
2308        let mut purge_exact_count = 0;
2309        let (_, purge_exact_us) = measure_us!(for (pubkey, slots_set) in pubkey_to_slot_set {
2310            purge_exact_count += 1;
2311            let is_empty = self
2312                .accounts_index
2313                .purge_exact(pubkey, slots_set, &mut reclaims);
2314            if is_empty {
2315                dead_keys.push(pubkey);
2316            }
2317        });
2318
2319        let (pubkeys_removed_from_accounts_index, handle_dead_keys_us) = measure_us!(self
2320            .accounts_index
2321            .handle_dead_keys(&dead_keys, &self.account_indexes));
2322
2323        self.stats
2324            .purge_exact_count
2325            .fetch_add(purge_exact_count, Ordering::Relaxed);
2326        self.stats
2327            .handle_dead_keys_us
2328            .fetch_add(handle_dead_keys_us, Ordering::Relaxed);
2329        self.stats
2330            .purge_exact_us
2331            .fetch_add(purge_exact_us, Ordering::Relaxed);
2332        (reclaims, pubkeys_removed_from_accounts_index)
2333    }
2334
2335    fn max_clean_root(&self, proposed_clean_root: Option<Slot>) -> Option<Slot> {
2336        match (
2337            self.accounts_index.min_ongoing_scan_root(),
2338            proposed_clean_root,
2339        ) {
2340            (None, None) => None,
2341            (Some(min_scan_root), None) => Some(min_scan_root),
2342            (None, Some(proposed_clean_root)) => Some(proposed_clean_root),
2343            (Some(min_scan_root), Some(proposed_clean_root)) => {
2344                Some(std::cmp::min(min_scan_root, proposed_clean_root))
2345            }
2346        }
2347    }
2348
2349    /// get the oldest slot that is within one epoch of the highest known root.
2350    /// The slot will have been offset by `self.ancient_append_vec_offset`
2351    fn get_oldest_non_ancient_slot(&self, epoch_schedule: &EpochSchedule) -> Slot {
2352        self.get_oldest_non_ancient_slot_from_slot(
2353            epoch_schedule,
2354            self.accounts_index.max_root_inclusive(),
2355        )
2356    }
2357
2358    /// get the oldest slot that is within one epoch of `max_root_inclusive`.
2359    /// The slot will have been offset by `self.ancient_append_vec_offset`
2360    fn get_oldest_non_ancient_slot_from_slot(
2361        &self,
2362        epoch_schedule: &EpochSchedule,
2363        max_root_inclusive: Slot,
2364    ) -> Slot {
2365        let mut result = max_root_inclusive;
2366        if let Some(offset) = self.ancient_append_vec_offset {
2367            result = Self::apply_offset_to_slot(result, offset);
2368        }
2369        result = Self::apply_offset_to_slot(
2370            result,
2371            -((epoch_schedule.slots_per_epoch as i64).saturating_sub(1)),
2372        );
2373        result.min(max_root_inclusive)
2374    }
2375
2376    /// Collect all the uncleaned slots, up to a max slot
2377    ///
2378    /// Search through the uncleaned Pubkeys and return all the slots, up to a maximum slot.
2379    fn collect_uncleaned_slots_up_to_slot(&self, max_slot_inclusive: Slot) -> Vec<Slot> {
2380        self.uncleaned_pubkeys
2381            .iter()
2382            .filter_map(|entry| {
2383                let slot = *entry.key();
2384                (slot <= max_slot_inclusive).then_some(slot)
2385            })
2386            .collect()
2387    }
2388
2389    /// For each slot in the list of uncleaned slots, up to a maximum
2390    /// slot, remove it from the `uncleaned_pubkeys` and move all the
2391    /// pubkeys to `candidates` for cleaning.
2392    fn remove_uncleaned_slots_up_to_slot_and_move_pubkeys(
2393        &self,
2394        max_slot_inclusive: Slot,
2395        candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>],
2396    ) {
2397        let uncleaned_slots = self.collect_uncleaned_slots_up_to_slot(max_slot_inclusive);
2398        for uncleaned_slot in uncleaned_slots.into_iter() {
2399            if let Some((_removed_slot, mut removed_pubkeys)) =
2400                self.uncleaned_pubkeys.remove(&uncleaned_slot)
2401            {
2402                // Sort all keys by bin index so that we can insert
2403                // them in `candidates` more efficiently.
2404                removed_pubkeys.sort_by(|a, b| {
2405                    self.accounts_index
2406                        .bin_calculator
2407                        .bin_from_pubkey(a)
2408                        .cmp(&self.accounts_index.bin_calculator.bin_from_pubkey(b))
2409                });
2410                if let Some(first_removed_pubkey) = removed_pubkeys.first() {
2411                    let mut prev_bin = self
2412                        .accounts_index
2413                        .bin_calculator
2414                        .bin_from_pubkey(first_removed_pubkey);
2415                    let mut candidates_bin = candidates[prev_bin].write().unwrap();
2416                    for removed_pubkey in removed_pubkeys {
2417                        let curr_bin = self
2418                            .accounts_index
2419                            .bin_calculator
2420                            .bin_from_pubkey(&removed_pubkey);
2421                        if curr_bin != prev_bin {
2422                            candidates_bin = candidates[curr_bin].write().unwrap();
2423                            prev_bin = curr_bin;
2424                        }
2425                        // Conservatively mark the candidate might have a zero lamport entry for
2426                        // correctness so that scan WILL try to look in disk if it is
2427                        // not in-mem. These keys are from 1) recently processed
2428                        // slots, 2) zero lamports found in shrink. Therefore, they are very likely
2429                        // to be in-memory, and seldomly do we need to look them up in disk.
2430                        candidates_bin.insert(
2431                            removed_pubkey,
2432                            CleaningInfo {
2433                                might_contain_zero_lamport_entry: true,
2434                                ..Default::default()
2435                            },
2436                        );
2437                    }
2438                }
2439            }
2440        }
2441    }
2442
2443    fn count_pubkeys(candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>]) -> u64 {
2444        candidates
2445            .iter()
2446            .map(|x| x.read().unwrap().len())
2447            .sum::<usize>() as u64
2448    }
2449
2450    /// Construct a list of candidates for cleaning from:
2451    /// - dirty_stores      -- set of stores which had accounts
2452    ///                        removed or recently rooted;
2453    /// - uncleaned_pubkeys -- the delta set of updated pubkeys in
2454    ///                        rooted slots from the last clean.
2455    ///
2456    /// The function also returns the minimum slot we encountered.
2457    fn construct_candidate_clean_keys(
2458        &self,
2459        max_clean_root_inclusive: Option<Slot>,
2460        is_startup: bool,
2461        timings: &mut CleanKeyTimings,
2462        epoch_schedule: &EpochSchedule,
2463        old_storages_policy: OldStoragesPolicy,
2464    ) -> CleaningCandidates {
2465        let oldest_non_ancient_slot = self.get_oldest_non_ancient_slot(epoch_schedule);
2466        let mut dirty_store_processing_time = Measure::start("dirty_store_processing");
2467        let max_root_inclusive = self.accounts_index.max_root_inclusive();
2468        let max_slot_inclusive = max_clean_root_inclusive.unwrap_or(max_root_inclusive);
2469
2470        if old_storages_policy == OldStoragesPolicy::Clean {
2471            let slot_one_epoch_old =
2472                max_root_inclusive.saturating_sub(epoch_schedule.slots_per_epoch);
2473            // do nothing special for these 100 old storages that will likely get cleaned up shortly
2474            let acceptable_straggler_slot_count = 100;
2475            let old_slot_cutoff =
2476                slot_one_epoch_old.saturating_sub(acceptable_straggler_slot_count);
2477            let (old_storages, old_slots) = self.get_snapshot_storages(..old_slot_cutoff);
2478            let num_old_storages = old_storages.len();
2479            self.accounts_index
2480                .add_uncleaned_roots(old_slots.iter().copied());
2481            for (old_slot, old_storage) in std::iter::zip(old_slots, old_storages) {
2482                self.dirty_stores.entry(old_slot).or_insert(old_storage);
2483            }
2484            info!("Marked {num_old_storages} old storages as dirty");
2485        }
2486
2487        let mut dirty_stores = Vec::with_capacity(self.dirty_stores.len());
2488        // find the oldest dirty slot
2489        // we'll add logging if that append vec cannot be marked dead
2490        let mut min_dirty_slot = None::<u64>;
2491        self.dirty_stores.retain(|slot, store| {
2492            if *slot > max_slot_inclusive {
2493                true
2494            } else {
2495                min_dirty_slot = min_dirty_slot.map(|min| min.min(*slot)).or(Some(*slot));
2496                dirty_stores.push((*slot, store.clone()));
2497                false
2498            }
2499        });
2500        let dirty_stores_len = dirty_stores.len();
2501        let num_bins = self.accounts_index.bins();
2502        let candidates: Box<_> =
2503            std::iter::repeat_with(|| RwLock::new(HashMap::<Pubkey, CleaningInfo>::new()))
2504                .take(num_bins)
2505                .collect();
2506
2507        let insert_candidate = |pubkey, is_zero_lamport| {
2508            let index = self.accounts_index.bin_calculator.bin_from_pubkey(&pubkey);
2509            let mut candidates_bin = candidates[index].write().unwrap();
2510            candidates_bin
2511                .entry(pubkey)
2512                .or_default()
2513                .might_contain_zero_lamport_entry |= is_zero_lamport;
2514        };
2515
2516        let dirty_ancient_stores = AtomicUsize::default();
2517        let mut dirty_store_routine = || {
2518            let chunk_size = 1.max(dirty_stores_len.saturating_div(rayon::current_num_threads()));
2519            let oldest_dirty_slots: Vec<u64> = dirty_stores
2520                .par_chunks(chunk_size)
2521                .map(|dirty_store_chunk| {
2522                    let mut oldest_dirty_slot = max_slot_inclusive.saturating_add(1);
2523                    dirty_store_chunk.iter().for_each(|(slot, store)| {
2524                        if *slot < oldest_non_ancient_slot {
2525                            dirty_ancient_stores.fetch_add(1, Ordering::Relaxed);
2526                        }
2527                        oldest_dirty_slot = oldest_dirty_slot.min(*slot);
2528
2529                        store.accounts.scan_index(|index| {
2530                            let pubkey = index.index_info.pubkey;
2531                            let is_zero_lamport = index.index_info.lamports == 0;
2532                            insert_candidate(pubkey, is_zero_lamport);
2533                        });
2534                    });
2535                    oldest_dirty_slot
2536                })
2537                .collect();
2538            timings.oldest_dirty_slot = *oldest_dirty_slots
2539                .iter()
2540                .min()
2541                .unwrap_or(&max_slot_inclusive.saturating_add(1));
2542        };
2543
2544        if is_startup {
2545            // Free to consume all the cores during startup
2546            dirty_store_routine();
2547        } else {
2548            self.thread_pool_clean.install(|| {
2549                dirty_store_routine();
2550            });
2551        }
2552        timings.dirty_pubkeys_count = Self::count_pubkeys(&candidates);
2553        trace!(
2554            "dirty_stores.len: {} pubkeys.len: {}",
2555            dirty_stores_len,
2556            timings.dirty_pubkeys_count,
2557        );
2558        dirty_store_processing_time.stop();
2559        timings.dirty_store_processing_us += dirty_store_processing_time.as_us();
2560        timings.dirty_ancient_stores = dirty_ancient_stores.load(Ordering::Relaxed);
2561
2562        let mut collect_delta_keys = Measure::start("key_create");
2563        self.remove_uncleaned_slots_up_to_slot_and_move_pubkeys(max_slot_inclusive, &candidates);
2564        collect_delta_keys.stop();
2565        timings.collect_delta_keys_us += collect_delta_keys.as_us();
2566
2567        timings.delta_key_count = Self::count_pubkeys(&candidates);
2568
2569        // Check if we should purge any of the
2570        // zero_lamport_accounts_to_purge_later, based on the
2571        // latest_full_snapshot_slot.
2572        let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
2573        assert!(
2574            latest_full_snapshot_slot.is_some()
2575                || self
2576                    .zero_lamport_accounts_to_purge_after_full_snapshot
2577                    .is_empty(),
2578            "if snapshots are disabled, then zero_lamport_accounts_to_purge_later should always \
2579             be empty"
2580        );
2581        if let Some(latest_full_snapshot_slot) = latest_full_snapshot_slot {
2582            self.zero_lamport_accounts_to_purge_after_full_snapshot
2583                .retain(|(slot, pubkey)| {
2584                    let is_candidate_for_clean =
2585                        max_slot_inclusive >= *slot && latest_full_snapshot_slot >= *slot;
2586                    if is_candidate_for_clean {
2587                        insert_candidate(*pubkey, true);
2588                    }
2589                    !is_candidate_for_clean
2590                });
2591        }
2592
2593        (candidates, min_dirty_slot)
2594    }
2595
2596    /// Call clean_accounts() with the common parameters that tests/benches use.
2597    pub fn clean_accounts_for_tests(&self) {
2598        self.clean_accounts(
2599            None,
2600            false,
2601            &EpochSchedule::default(),
2602            if self.ancient_append_vec_offset.is_some() {
2603                OldStoragesPolicy::Leave
2604            } else {
2605                OldStoragesPolicy::Clean
2606            },
2607        )
2608    }
2609
2610    /// called with cli argument to verify refcounts are correct on all accounts
2611    /// this is very slow
2612    fn exhaustively_verify_refcounts(&self, max_slot_inclusive: Option<Slot>) {
2613        let max_slot_inclusive =
2614            max_slot_inclusive.unwrap_or_else(|| self.accounts_index.max_root_inclusive());
2615        info!("exhaustively verifying refcounts as of slot: {max_slot_inclusive}");
2616        let pubkey_refcount = DashMap::<Pubkey, Vec<Slot>>::default();
2617        let slots = self.storage.all_slots();
2618        // populate
2619        slots.into_par_iter().for_each(|slot| {
2620            if slot > max_slot_inclusive {
2621                return;
2622            }
2623            if let Some(storage) = self.storage.get_slot_storage_entry(slot) {
2624                storage.accounts.scan_accounts(|account| {
2625                    let pk = account.pubkey();
2626                    match pubkey_refcount.entry(*pk) {
2627                        dashmap::mapref::entry::Entry::Occupied(mut occupied_entry) => {
2628                            if !occupied_entry.get().iter().any(|s| s == &slot) {
2629                                occupied_entry.get_mut().push(slot);
2630                            }
2631                        }
2632                        dashmap::mapref::entry::Entry::Vacant(vacant_entry) => {
2633                            vacant_entry.insert(vec![slot]);
2634                        }
2635                    }
2636                });
2637            }
2638        });
2639        let total = pubkey_refcount.len();
2640        let failed = AtomicBool::default();
2641        let threads = quarter_thread_count();
2642        let per_batch = total / threads;
2643        (0..=threads).into_par_iter().for_each(|attempt| {
2644            pubkey_refcount
2645                .iter()
2646                .skip(attempt * per_batch)
2647                .take(per_batch)
2648                .for_each(|entry| {
2649                    if failed.load(Ordering::Relaxed) {
2650                        return;
2651                    }
2652
2653                    self.accounts_index
2654                        .get_and_then(entry.key(), |index_entry| {
2655                            if let Some(index_entry) = index_entry {
2656                                match (index_entry.ref_count() as usize).cmp(&entry.value().len()) {
2657                                    std::cmp::Ordering::Equal => {
2658                                        // ref counts match, nothing to do here
2659                                    }
2660                                    std::cmp::Ordering::Greater => {
2661                                        let slot_list = index_entry.slot_list.read().unwrap();
2662                                        let num_too_new = slot_list
2663                                            .iter()
2664                                            .filter(|(slot, _)| slot > &max_slot_inclusive)
2665                                            .count();
2666
2667                                        if ((index_entry.ref_count() as usize) - num_too_new)
2668                                            > entry.value().len()
2669                                        {
2670                                            failed.store(true, Ordering::Relaxed);
2671                                            error!(
2672                                                "exhaustively_verify_refcounts: {} refcount too \
2673                                                 large: {}, should be: {}, {:?}, {:?}, too_new: \
2674                                                 {num_too_new}",
2675                                                entry.key(),
2676                                                index_entry.ref_count(),
2677                                                entry.value().len(),
2678                                                *entry.value(),
2679                                                slot_list
2680                                            );
2681                                        }
2682                                    }
2683                                    std::cmp::Ordering::Less => {
2684                                        error!(
2685                                            "exhaustively_verify_refcounts: {} refcount too \
2686                                             small: {}, should be: {}, {:?}, {:?}",
2687                                            entry.key(),
2688                                            index_entry.ref_count(),
2689                                            entry.value().len(),
2690                                            *entry.value(),
2691                                            index_entry.slot_list.read().unwrap()
2692                                        );
2693                                    }
2694                                }
2695                            };
2696                            (false, ())
2697                        });
2698                });
2699        });
2700        if failed.load(Ordering::Relaxed) {
2701            panic!("exhaustively_verify_refcounts failed");
2702        }
2703    }
2704
2705    // Purge zero lamport accounts and older rooted account states as garbage
2706    // collection
2707    // Only remove those accounts where the entire rooted history of the account
2708    // can be purged because there are no live append vecs in the ancestors
2709    pub fn clean_accounts(
2710        &self,
2711        max_clean_root_inclusive: Option<Slot>,
2712        is_startup: bool,
2713        epoch_schedule: &EpochSchedule,
2714        old_storages_policy: OldStoragesPolicy,
2715    ) {
2716        if self.exhaustively_verify_refcounts {
2717            self.exhaustively_verify_refcounts(max_clean_root_inclusive);
2718        }
2719
2720        let _guard = self.active_stats.activate(ActiveStatItem::Clean);
2721
2722        let ancient_account_cleans = AtomicU64::default();
2723        let purges_old_accounts_count = AtomicU64::default();
2724
2725        let mut measure_all = Measure::start("clean_accounts");
2726        let max_clean_root_inclusive = self.max_clean_root(max_clean_root_inclusive);
2727
2728        self.report_store_stats();
2729
2730        let mut key_timings = CleanKeyTimings::default();
2731        let (candidates, min_dirty_slot) = self.construct_candidate_clean_keys(
2732            max_clean_root_inclusive,
2733            is_startup,
2734            &mut key_timings,
2735            epoch_schedule,
2736            old_storages_policy,
2737        );
2738
2739        let num_candidates = Self::count_pubkeys(&candidates);
2740        let mut accounts_scan = Measure::start("accounts_scan");
2741        let uncleaned_roots = self.accounts_index.clone_uncleaned_roots();
2742        let found_not_zero_accum = AtomicU64::new(0);
2743        let not_found_on_fork_accum = AtomicU64::new(0);
2744        let missing_accum = AtomicU64::new(0);
2745        let useful_accum = AtomicU64::new(0);
2746        let reclaims: SlotList<AccountInfo> = Vec::with_capacity(num_candidates as usize);
2747        let reclaims = Mutex::new(reclaims);
2748        let pubkeys_removed_from_accounts_index: PubkeysRemovedFromAccountsIndex = HashSet::new();
2749        let pubkeys_removed_from_accounts_index = Mutex::new(pubkeys_removed_from_accounts_index);
2750        // parallel scan the index.
2751        let do_clean_scan = || {
2752            candidates.par_iter().for_each(|candidates_bin| {
2753                let mut found_not_zero = 0;
2754                let mut not_found_on_fork = 0;
2755                let mut missing = 0;
2756                let mut useful = 0;
2757                let mut purges_old_accounts_local = 0;
2758                let mut candidates_bin = candidates_bin.write().unwrap();
2759                // Iterate over each HashMap entry to
2760                // avoid capturing the HashMap in the
2761                // closure passed to scan thus making
2762                // conflicting read and write borrows.
2763                candidates_bin.retain(|candidate_pubkey, candidate_info| {
2764                    let mut should_purge = false;
2765                    self.accounts_index.scan(
2766                        [*candidate_pubkey].iter(),
2767                        |_candidate_pubkey, slot_list_and_ref_count, _entry| {
2768                            let mut useless = true;
2769                            if let Some((slot_list, ref_count)) = slot_list_and_ref_count {
2770                                // find the highest rooted slot in the slot list
2771                                let index_in_slot_list = self.accounts_index.latest_slot(
2772                                    None,
2773                                    slot_list,
2774                                    max_clean_root_inclusive,
2775                                );
2776
2777                                match index_in_slot_list {
2778                                    Some(index_in_slot_list) => {
2779                                        // found info relative to max_clean_root
2780                                        let (slot, account_info) = &slot_list[index_in_slot_list];
2781                                        if account_info.is_zero_lamport() {
2782                                            useless = false;
2783                                            // The latest one is zero lamports. We may be able to purge it.
2784                                            // Add all the rooted entries that contain this pubkey.
2785                                            // We know the highest rooted entry is zero lamports.
2786                                            candidate_info.slot_list =
2787                                                self.accounts_index.get_rooted_entries(
2788                                                    slot_list,
2789                                                    max_clean_root_inclusive,
2790                                                );
2791                                            candidate_info.ref_count = ref_count;
2792                                        } else {
2793                                            found_not_zero += 1;
2794                                        }
2795
2796                                        // If this candidate has multiple rooted slot list entries,
2797                                        // we should reclaim the older ones.
2798                                        if slot_list.len() > 1
2799                                            && *slot
2800                                                <= max_clean_root_inclusive.unwrap_or(Slot::MAX)
2801                                        {
2802                                            should_purge = true;
2803                                            purges_old_accounts_local += 1;
2804                                            useless = false;
2805                                        }
2806                                        // Note, this next if-block is only kept to maintain the
2807                                        // `uncleaned_roots_slot_list_1` stat.
2808                                        if uncleaned_roots.contains(slot) {
2809                                            // Assertion enforced by `accounts_index.get()`, the latest slot
2810                                            // will not be greater than the given `max_clean_root`
2811                                            if let Some(max_clean_root_inclusive) =
2812                                                max_clean_root_inclusive
2813                                            {
2814                                                assert!(slot <= &max_clean_root_inclusive);
2815                                            }
2816                                            if slot_list.len() == 1 {
2817                                                self.clean_accounts_stats
2818                                                    .uncleaned_roots_slot_list_1
2819                                                    .fetch_add(1, Ordering::Relaxed);
2820                                            }
2821                                        }
2822                                    }
2823                                    None => {
2824                                        // This pubkey is in the index but not in a root slot, so clean
2825                                        // it up by adding it to the to-be-purged list.
2826                                        //
2827                                        // Also, this pubkey must have been touched by some slot since
2828                                        // it was in the dirty list, so we assume that the slot it was
2829                                        // touched in must be unrooted.
2830                                        not_found_on_fork += 1;
2831                                        should_purge = true;
2832                                        purges_old_accounts_local += 1;
2833                                        useless = false;
2834                                    }
2835                                }
2836                            } else {
2837                                missing += 1;
2838                            }
2839                            if !useless {
2840                                useful += 1;
2841                            }
2842                            AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
2843                        },
2844                        None,
2845                        false,
2846                        if candidate_info.might_contain_zero_lamport_entry {
2847                            ScanFilter::All
2848                        } else {
2849                            self.scan_filter_for_shrinking
2850                        },
2851                    );
2852                    if should_purge {
2853                        let reclaims_new = self.collect_reclaims(
2854                            candidate_pubkey,
2855                            max_clean_root_inclusive,
2856                            &ancient_account_cleans,
2857                            epoch_schedule,
2858                            &pubkeys_removed_from_accounts_index,
2859                        );
2860                        if !reclaims_new.is_empty() {
2861                            reclaims.lock().unwrap().extend(reclaims_new);
2862                        }
2863                    }
2864                    !candidate_info.slot_list.is_empty()
2865                });
2866                found_not_zero_accum.fetch_add(found_not_zero, Ordering::Relaxed);
2867                not_found_on_fork_accum.fetch_add(not_found_on_fork, Ordering::Relaxed);
2868                missing_accum.fetch_add(missing, Ordering::Relaxed);
2869                useful_accum.fetch_add(useful, Ordering::Relaxed);
2870                purges_old_accounts_count.fetch_add(purges_old_accounts_local, Ordering::Relaxed);
2871            });
2872        };
2873        if is_startup {
2874            do_clean_scan();
2875        } else {
2876            self.thread_pool_clean.install(do_clean_scan);
2877        }
2878
2879        accounts_scan.stop();
2880        let retained_keys_count = Self::count_pubkeys(&candidates);
2881        let reclaims = reclaims.into_inner().unwrap();
2882        let mut pubkeys_removed_from_accounts_index =
2883            pubkeys_removed_from_accounts_index.into_inner().unwrap();
2884        let mut clean_old_rooted = Measure::start("clean_old_roots");
2885        let (purged_account_slots, removed_accounts) =
2886            self.clean_accounts_older_than_root(&reclaims, &pubkeys_removed_from_accounts_index);
2887        self.do_reset_uncleaned_roots(max_clean_root_inclusive);
2888        clean_old_rooted.stop();
2889
2890        let mut store_counts_time = Measure::start("store_counts");
2891
2892        // Calculate store counts as if everything was purged
2893        // Then purge if we can
2894        let mut store_counts: HashMap<Slot, (usize, HashSet<Pubkey>)> = HashMap::new();
2895        for candidates_bin in candidates.iter() {
2896            for (
2897                pubkey,
2898                CleaningInfo {
2899                    slot_list,
2900                    ref_count,
2901                    ..
2902                },
2903            ) in candidates_bin.write().unwrap().iter_mut()
2904            {
2905                debug_assert!(!slot_list.is_empty(), "candidate slot_list can't be empty");
2906                if purged_account_slots.contains_key(pubkey) {
2907                    *ref_count = self.accounts_index.ref_count_from_storage(pubkey);
2908                }
2909                slot_list.retain(|(slot, account_info)| {
2910                    let was_slot_purged = purged_account_slots
2911                        .get(pubkey)
2912                        .map(|slots_removed| slots_removed.contains(slot))
2913                        .unwrap_or(false);
2914                    if was_slot_purged {
2915                        // No need to look up the slot storage below if the entire
2916                        // slot was purged
2917                        return false;
2918                    }
2919                    // Check if this update in `slot` to the account with `key` was reclaimed earlier by
2920                    // `clean_accounts_older_than_root()`
2921                    let was_reclaimed = removed_accounts
2922                        .get(slot)
2923                        .map(|store_removed| store_removed.contains(&account_info.offset()))
2924                        .unwrap_or(false);
2925                    if was_reclaimed {
2926                        return false;
2927                    }
2928                    if let Some(store_count) = store_counts.get_mut(slot) {
2929                        store_count.0 -= 1;
2930                        store_count.1.insert(*pubkey);
2931                    } else {
2932                        let mut key_set = HashSet::new();
2933                        key_set.insert(*pubkey);
2934                        assert!(
2935                            !account_info.is_cached(),
2936                            "The Accounts Cache must be flushed first for this account info. \
2937                             pubkey: {}, slot: {}",
2938                            *pubkey,
2939                            *slot
2940                        );
2941                        let count = self
2942                            .storage
2943                            .get_account_storage_entry(*slot, account_info.store_id())
2944                            .map(|store| store.count())
2945                            .unwrap()
2946                            - 1;
2947                        debug!(
2948                            "store_counts, inserting slot: {}, store id: {}, count: {}",
2949                            slot,
2950                            account_info.store_id(),
2951                            count
2952                        );
2953                        store_counts.insert(*slot, (count, key_set));
2954                    }
2955                    true
2956                });
2957            }
2958        }
2959        store_counts_time.stop();
2960
2961        let mut calc_deps_time = Measure::start("calc_deps");
2962        self.calc_delete_dependencies(&candidates, &mut store_counts, min_dirty_slot);
2963        calc_deps_time.stop();
2964
2965        let mut purge_filter = Measure::start("purge_filter");
2966        self.filter_zero_lamport_clean_for_incremental_snapshots(
2967            max_clean_root_inclusive,
2968            &store_counts,
2969            &candidates,
2970        );
2971        purge_filter.stop();
2972
2973        let mut reclaims_time = Measure::start("reclaims");
2974        // Recalculate reclaims with new purge set
2975        let mut pubkey_to_slot_set = Vec::new();
2976        for candidates_bin in candidates.iter() {
2977            let candidates_bin = candidates_bin.read().unwrap();
2978            let mut bin_set = candidates_bin
2979                .iter()
2980                .filter_map(|(pubkey, cleaning_info)| {
2981                    let CleaningInfo {
2982                        slot_list,
2983                        ref_count: _,
2984                        ..
2985                    } = cleaning_info;
2986                    (!slot_list.is_empty()).then_some((
2987                        *pubkey,
2988                        slot_list
2989                            .iter()
2990                            .map(|(slot, _)| *slot)
2991                            .collect::<HashSet<Slot>>(),
2992                    ))
2993                })
2994                .collect::<Vec<_>>();
2995            pubkey_to_slot_set.append(&mut bin_set);
2996        }
2997
2998        let (reclaims, pubkeys_removed_from_accounts_index2) =
2999            self.purge_keys_exact(pubkey_to_slot_set.iter());
3000        pubkeys_removed_from_accounts_index.extend(pubkeys_removed_from_accounts_index2);
3001
3002        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
3003        // and those stores may be used for background hashing.
3004        let reset_accounts = false;
3005        self.handle_reclaims(
3006            (!reclaims.is_empty()).then(|| reclaims.iter()),
3007            None,
3008            reset_accounts,
3009            &pubkeys_removed_from_accounts_index,
3010            HandleReclaims::ProcessDeadSlots(&self.clean_accounts_stats.purge_stats),
3011        );
3012
3013        reclaims_time.stop();
3014        measure_all.stop();
3015
3016        self.clean_accounts_stats.report();
3017        datapoint_info!(
3018            "clean_accounts",
3019            ("max_clean_root", max_clean_root_inclusive, Option<i64>),
3020            ("total_us", measure_all.as_us(), i64),
3021            (
3022                "collect_delta_keys_us",
3023                key_timings.collect_delta_keys_us,
3024                i64
3025            ),
3026            ("oldest_dirty_slot", key_timings.oldest_dirty_slot, i64),
3027            (
3028                "pubkeys_removed_from_accounts_index",
3029                pubkeys_removed_from_accounts_index.len(),
3030                i64
3031            ),
3032            (
3033                "dirty_ancient_stores",
3034                key_timings.dirty_ancient_stores,
3035                i64
3036            ),
3037            (
3038                "dirty_store_processing_us",
3039                key_timings.dirty_store_processing_us,
3040                i64
3041            ),
3042            ("accounts_scan", accounts_scan.as_us() as i64, i64),
3043            ("clean_old_rooted", clean_old_rooted.as_us() as i64, i64),
3044            ("store_counts", store_counts_time.as_us() as i64, i64),
3045            ("purge_filter", purge_filter.as_us() as i64, i64),
3046            ("calc_deps", calc_deps_time.as_us() as i64, i64),
3047            ("reclaims", reclaims_time.as_us() as i64, i64),
3048            ("delta_insert_us", key_timings.delta_insert_us, i64),
3049            ("delta_key_count", key_timings.delta_key_count, i64),
3050            ("dirty_pubkeys_count", key_timings.dirty_pubkeys_count, i64),
3051            ("useful_keys", useful_accum.load(Ordering::Relaxed), i64),
3052            ("total_keys_count", num_candidates, i64),
3053            ("retained_keys_count", retained_keys_count, i64),
3054            (
3055                "scan_found_not_zero",
3056                found_not_zero_accum.load(Ordering::Relaxed),
3057                i64
3058            ),
3059            (
3060                "scan_not_found_on_fork",
3061                not_found_on_fork_accum.load(Ordering::Relaxed),
3062                i64
3063            ),
3064            ("scan_missing", missing_accum.load(Ordering::Relaxed), i64),
3065            ("uncleaned_roots_len", uncleaned_roots.len(), i64),
3066            (
3067                "uncleaned_roots_slot_list_1",
3068                self.clean_accounts_stats
3069                    .uncleaned_roots_slot_list_1
3070                    .swap(0, Ordering::Relaxed),
3071                i64
3072            ),
3073            (
3074                "get_account_sizes_us",
3075                self.clean_accounts_stats
3076                    .get_account_sizes_us
3077                    .swap(0, Ordering::Relaxed),
3078                i64
3079            ),
3080            (
3081                "slots_cleaned",
3082                self.clean_accounts_stats
3083                    .slots_cleaned
3084                    .swap(0, Ordering::Relaxed),
3085                i64
3086            ),
3087            (
3088                "clean_old_root_us",
3089                self.clean_accounts_stats
3090                    .clean_old_root_us
3091                    .swap(0, Ordering::Relaxed),
3092                i64
3093            ),
3094            (
3095                "clean_old_root_reclaim_us",
3096                self.clean_accounts_stats
3097                    .clean_old_root_reclaim_us
3098                    .swap(0, Ordering::Relaxed),
3099                i64
3100            ),
3101            (
3102                "reset_uncleaned_roots_us",
3103                self.clean_accounts_stats
3104                    .reset_uncleaned_roots_us
3105                    .swap(0, Ordering::Relaxed),
3106                i64
3107            ),
3108            (
3109                "remove_dead_accounts_remove_us",
3110                self.clean_accounts_stats
3111                    .remove_dead_accounts_remove_us
3112                    .swap(0, Ordering::Relaxed),
3113                i64
3114            ),
3115            (
3116                "remove_dead_accounts_shrink_us",
3117                self.clean_accounts_stats
3118                    .remove_dead_accounts_shrink_us
3119                    .swap(0, Ordering::Relaxed),
3120                i64
3121            ),
3122            (
3123                "clean_stored_dead_slots_us",
3124                self.clean_accounts_stats
3125                    .clean_stored_dead_slots_us
3126                    .swap(0, Ordering::Relaxed),
3127                i64
3128            ),
3129            (
3130                "roots_added",
3131                self.accounts_index.roots_added.swap(0, Ordering::Relaxed),
3132                i64
3133            ),
3134            (
3135                "purge_older_root_entries_one_slot_list",
3136                self.accounts_index
3137                    .purge_older_root_entries_one_slot_list
3138                    .swap(0, Ordering::Relaxed),
3139                i64
3140            ),
3141            (
3142                "roots_removed",
3143                self.accounts_index.roots_removed.swap(0, Ordering::Relaxed),
3144                i64
3145            ),
3146            (
3147                "active_scans",
3148                self.accounts_index.active_scans.load(Ordering::Relaxed),
3149                i64
3150            ),
3151            (
3152                "max_distance_to_min_scan_slot",
3153                self.accounts_index
3154                    .max_distance_to_min_scan_slot
3155                    .swap(0, Ordering::Relaxed),
3156                i64
3157            ),
3158            (
3159                "unref_zero_count",
3160                self.accounts_index
3161                    .unref_zero_count
3162                    .swap(0, Ordering::Relaxed),
3163                i64
3164            ),
3165            (
3166                "ancient_account_cleans",
3167                ancient_account_cleans.load(Ordering::Relaxed),
3168                i64
3169            ),
3170            (
3171                "purges_old_accounts_count",
3172                purges_old_accounts_count.load(Ordering::Relaxed),
3173                i64
3174            ),
3175            ("next_store_id", self.next_id.load(Ordering::Relaxed), i64),
3176        );
3177    }
3178
3179    /// Removes the accounts in the input `reclaims` from the tracked "count" of
3180    /// their corresponding  storage entries. Note this does not actually free
3181    /// the memory from the storage entries until all the storage entries for
3182    /// a given slot `S` are empty, at which point `process_dead_slots` will
3183    /// remove all the storage entries for `S`.
3184    ///
3185    /// # Arguments
3186    /// * `reclaims` - The accounts to remove from storage entries' "count". Note here
3187    ///    that we should not remove cache entries, only entries for accounts actually
3188    ///    stored in a storage entry.
3189    ///
3190    /// * `expected_single_dead_slot` - A correctness assertion. If this is equal to `Some(S)`,
3191    ///    then the function will check that the only slot being cleaned up in `reclaims`
3192    ///    is the slot == `S`. This is true for instance when `handle_reclaims` is called
3193    ///    from store or slot shrinking, as those should only touch the slot they are
3194    ///    currently storing to or shrinking.
3195    ///
3196    /// * `reset_accounts` - Reset the append_vec store when the store is dead (count==0)
3197    ///    From the clean and shrink paths it should be false since there may be an in-progress
3198    ///    hash operation and the stores may hold accounts that need to be unref'ed.
3199    /// * `pubkeys_removed_from_accounts_index` - These keys have already been removed from the accounts index
3200    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
3201    /// * `handle_reclaims`. `purge_stats` are stats used to track performance of purging dead slots if
3202    ///    value is `ProcessDeadSlots`.
3203    ///    Otherwise, there can be no dead slots
3204    ///    that happen as a result of this call, and the function will check that no slots are
3205    ///    cleaned up/removed via `process_dead_slots`. For instance, on store, no slots should
3206    ///    be cleaned up, but during the background clean accounts purges accounts from old rooted
3207    ///    slots, so outdated slots may be removed.
3208    fn handle_reclaims<'a, I>(
3209        &'a self,
3210        reclaims: Option<I>,
3211        expected_single_dead_slot: Option<Slot>,
3212        reset_accounts: bool,
3213        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3214        handle_reclaims: HandleReclaims<'a>,
3215    ) -> ReclaimResult
3216    where
3217        I: Iterator<Item = &'a (Slot, AccountInfo)>,
3218    {
3219        let mut reclaim_result = ReclaimResult::default();
3220        if let Some(reclaims) = reclaims {
3221            let (dead_slots, reclaimed_offsets) =
3222                self.remove_dead_accounts(reclaims, expected_single_dead_slot, reset_accounts);
3223            reclaim_result.1 = reclaimed_offsets;
3224
3225            if let HandleReclaims::ProcessDeadSlots(purge_stats) = handle_reclaims {
3226                if let Some(expected_single_dead_slot) = expected_single_dead_slot {
3227                    assert!(dead_slots.len() <= 1);
3228                    if dead_slots.len() == 1 {
3229                        assert!(dead_slots.contains(&expected_single_dead_slot));
3230                    }
3231                }
3232
3233                self.process_dead_slots(
3234                    &dead_slots,
3235                    Some(&mut reclaim_result.0),
3236                    purge_stats,
3237                    pubkeys_removed_from_accounts_index,
3238                );
3239            } else {
3240                assert!(dead_slots.is_empty());
3241            }
3242        }
3243        reclaim_result
3244    }
3245
3246    /// During clean, some zero-lamport accounts that are marked for purge should *not* actually
3247    /// get purged.  Filter out those accounts here by removing them from 'candidates'.
3248    /// Candidates may contain entries with empty slots list in CleaningInfo.
3249    /// The function removes such entries from 'candidates'.
3250    ///
3251    /// When using incremental snapshots, do not purge zero-lamport accounts if the slot is higher
3252    /// than the latest full snapshot slot.  This is to protect against the following scenario:
3253    ///
3254    ///   ```text
3255    ///   A full snapshot is taken, including account 'alpha' with a non-zero balance.  In a later slot,
3256    ///   alpha's lamports go to zero.  Eventually, cleaning runs.  Without this change,
3257    ///   alpha would be cleaned up and removed completely. Finally, an incremental snapshot is taken.
3258    ///
3259    ///   Later, the incremental and full snapshots are used to rebuild the bank and accounts
3260    ///   database (e.x. if the node restarts).  The full snapshot _does_ contain alpha
3261    ///   and its balance is non-zero.  However, since alpha was cleaned up in a slot after the full
3262    ///   snapshot slot (due to having zero lamports), the incremental snapshot would not contain alpha.
3263    ///   Thus, the accounts database will contain the old, incorrect info for alpha with a non-zero
3264    ///   balance.  Very bad!
3265    ///   ```
3266    ///
3267    /// This filtering step can be skipped if there is no `latest_full_snapshot_slot`, or if the
3268    /// `max_clean_root_inclusive` is less-than-or-equal-to the `latest_full_snapshot_slot`.
3269    fn filter_zero_lamport_clean_for_incremental_snapshots(
3270        &self,
3271        max_clean_root_inclusive: Option<Slot>,
3272        store_counts: &HashMap<Slot, (usize, HashSet<Pubkey>)>,
3273        candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>],
3274    ) {
3275        let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
3276        let should_filter_for_incremental_snapshots = max_clean_root_inclusive.unwrap_or(Slot::MAX)
3277            > latest_full_snapshot_slot.unwrap_or(Slot::MAX);
3278        assert!(
3279            latest_full_snapshot_slot.is_some() || !should_filter_for_incremental_snapshots,
3280            "if filtering for incremental snapshots, then snapshots should be enabled",
3281        );
3282
3283        for bin in candidates {
3284            let mut bin = bin.write().unwrap();
3285            bin.retain(|pubkey, cleaning_info| {
3286                let CleaningInfo {
3287                    slot_list,
3288                    ref_count: _,
3289                    ..
3290                } = cleaning_info;
3291                debug_assert!(!slot_list.is_empty(), "candidate slot_list can't be empty");
3292                // Only keep candidates where the entire history of the account in the root set
3293                // can be purged. All AppendVecs for those updates are dead.
3294                for (slot, _account_info) in slot_list.iter() {
3295                    if let Some(store_count) = store_counts.get(slot) {
3296                        if store_count.0 != 0 {
3297                            // one store this pubkey is in is not being removed, so this pubkey cannot be removed at all
3298                            return false;
3299                        }
3300                    } else {
3301                        // store is not being removed, so this pubkey cannot be removed at all
3302                        return false;
3303                    }
3304                }
3305
3306                // Exit early if not filtering more for incremental snapshots
3307                if !should_filter_for_incremental_snapshots {
3308                    return true;
3309                }
3310
3311                // Safety: We exited early if the slot list was empty,
3312                // so we're guaranteed here that `.max_by_key()` returns Some.
3313                let (slot, account_info) = slot_list
3314                    .iter()
3315                    .max_by_key(|(slot, _account_info)| slot)
3316                    .unwrap();
3317
3318                // Do *not* purge zero-lamport accounts if the slot is greater than the last full
3319                // snapshot slot.  Since we're `retain`ing the accounts-to-purge, I felt creating
3320                // the `cannot_purge` variable made this easier to understand.  Accounts that do
3321                // not get purged here are added to a list so they be considered for purging later
3322                // (i.e. after the next full snapshot).
3323                assert!(account_info.is_zero_lamport());
3324                let cannot_purge = *slot > latest_full_snapshot_slot.unwrap();
3325                if cannot_purge {
3326                    self.zero_lamport_accounts_to_purge_after_full_snapshot
3327                        .insert((*slot, *pubkey));
3328                }
3329                !cannot_purge
3330            });
3331        }
3332    }
3333
3334    // Must be kept private!, does sensitive cleanup that should only be called from
3335    // supported pipelines in AccountsDb
3336    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
3337    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
3338    fn process_dead_slots(
3339        &self,
3340        dead_slots: &IntSet<Slot>,
3341        purged_account_slots: Option<&mut AccountSlots>,
3342        purge_stats: &PurgeStats,
3343        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3344    ) {
3345        if dead_slots.is_empty() {
3346            return;
3347        }
3348        let mut clean_dead_slots = Measure::start("reclaims::clean_dead_slots");
3349        self.clean_stored_dead_slots(
3350            dead_slots,
3351            purged_account_slots,
3352            pubkeys_removed_from_accounts_index,
3353        );
3354        clean_dead_slots.stop();
3355
3356        let mut purge_removed_slots = Measure::start("reclaims::purge_removed_slots");
3357        self.purge_dead_slots_from_storage(dead_slots.iter(), purge_stats);
3358        purge_removed_slots.stop();
3359
3360        // If the slot is dead, remove the need to shrink the storages as
3361        // the storage entries will be purged.
3362        {
3363            let mut list = self.shrink_candidate_slots.lock().unwrap();
3364            for slot in dead_slots {
3365                list.remove(slot);
3366            }
3367        }
3368
3369        debug!(
3370            "process_dead_slots({}): {} {} {:?}",
3371            dead_slots.len(),
3372            clean_dead_slots,
3373            purge_removed_slots,
3374            dead_slots,
3375        );
3376    }
3377
3378    /// load the account index entry for the first `count` items in `accounts`
3379    /// store a reference to all alive accounts in `alive_accounts`
3380    /// store all pubkeys dead in `slot_to_shrink` in `pubkeys_to_unref`
3381    /// return sum of account size for all alive accounts
3382    fn load_accounts_index_for_shrink<'a, T: ShrinkCollectRefs<'a>>(
3383        &self,
3384        accounts: &'a [AccountFromStorage],
3385        stats: &ShrinkStats,
3386        slot_to_shrink: Slot,
3387    ) -> LoadAccountsIndexForShrink<'a, T> {
3388        let count = accounts.len();
3389        let mut alive_accounts = T::with_capacity(count, slot_to_shrink);
3390        let mut pubkeys_to_unref = Vec::with_capacity(count);
3391        let mut zero_lamport_single_ref_pubkeys = Vec::with_capacity(count);
3392
3393        let mut alive = 0;
3394        let mut dead = 0;
3395        let mut index = 0;
3396        let mut index_scan_returned_some_count = 0;
3397        let mut index_scan_returned_none_count = 0;
3398        let mut all_are_zero_lamports = true;
3399        let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
3400        self.accounts_index.scan(
3401            accounts.iter().map(|account| account.pubkey()),
3402            |pubkey, slots_refs, _entry| {
3403                let stored_account = &accounts[index];
3404                let mut do_populate_accounts_for_shrink = |ref_count, slot_list| {
3405                    if stored_account.is_zero_lamport()
3406                        && ref_count == 1
3407                        && latest_full_snapshot_slot
3408                            .map(|latest_full_snapshot_slot| {
3409                                latest_full_snapshot_slot >= slot_to_shrink
3410                            })
3411                            .unwrap_or(true)
3412                    {
3413                        // only do this if our slot is prior to the latest full snapshot
3414                        // we found a zero lamport account that is the only instance of this account. We can delete it completely.
3415                        zero_lamport_single_ref_pubkeys.push(pubkey);
3416                        self.add_uncleaned_pubkeys_after_shrink(
3417                            slot_to_shrink,
3418                            [*pubkey].into_iter(),
3419                        );
3420                    } else {
3421                        all_are_zero_lamports &= stored_account.is_zero_lamport();
3422                        alive_accounts.add(ref_count, stored_account, slot_list);
3423                        alive += 1;
3424                    }
3425                };
3426                if let Some((slot_list, ref_count)) = slots_refs {
3427                    index_scan_returned_some_count += 1;
3428                    let is_alive = slot_list.iter().any(|(slot, _acct_info)| {
3429                        // 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
3430                        *slot == slot_to_shrink
3431                    });
3432
3433                    if !is_alive {
3434                        // This pubkey was found in the storage, but no longer exists in the index.
3435                        // It would have had a ref to the storage from the initial store, but it will
3436                        // not exist in the re-written slot. Unref it to keep the index consistent with
3437                        // rewriting the storage entries.
3438                        pubkeys_to_unref.push(pubkey);
3439                        dead += 1;
3440                    } else {
3441                        do_populate_accounts_for_shrink(ref_count, slot_list);
3442                    }
3443                } else {
3444                    index_scan_returned_none_count += 1;
3445                    // getting None here means the account is 'normal' and was written to disk. This means it must have ref_count=1 and
3446                    // slot_list.len() = 1. This means it must be alive in this slot. This is by far the most common case.
3447                    // Note that we could get Some(...) here if the account is in the in mem index because it is hot.
3448                    // Note this could also mean the account isn't on disk either. That would indicate a bug in accounts db.
3449                    // Account is alive.
3450                    let ref_count = 1;
3451                    let slot_list = [(slot_to_shrink, AccountInfo::default())];
3452                    do_populate_accounts_for_shrink(ref_count, &slot_list);
3453                }
3454                index += 1;
3455                AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
3456            },
3457            None,
3458            false,
3459            self.scan_filter_for_shrinking,
3460        );
3461        assert_eq!(index, std::cmp::min(accounts.len(), count));
3462        stats
3463            .index_scan_returned_some
3464            .fetch_add(index_scan_returned_some_count, Ordering::Relaxed);
3465        stats
3466            .index_scan_returned_none
3467            .fetch_add(index_scan_returned_none_count, Ordering::Relaxed);
3468        stats.alive_accounts.fetch_add(alive, Ordering::Relaxed);
3469        stats.dead_accounts.fetch_add(dead, Ordering::Relaxed);
3470
3471        LoadAccountsIndexForShrink {
3472            alive_accounts,
3473            pubkeys_to_unref,
3474            zero_lamport_single_ref_pubkeys,
3475            all_are_zero_lamports,
3476        }
3477    }
3478
3479    /// get all accounts in all the storages passed in
3480    /// for duplicate pubkeys, the account with the highest write_value is returned
3481    pub fn get_unique_accounts_from_storage(
3482        &self,
3483        store: &AccountStorageEntry,
3484    ) -> GetUniqueAccountsResult {
3485        let capacity = store.capacity();
3486        let mut stored_accounts = Vec::with_capacity(store.count());
3487        store.accounts.scan_index(|info| {
3488            // file_id is unused and can be anything. We will always be loading whatever storage is in the slot.
3489            let file_id = 0;
3490            stored_accounts.push(AccountFromStorage {
3491                index_info: AccountInfo::new(
3492                    StorageLocation::AppendVec(file_id, info.index_info.offset),
3493                    info.index_info.lamports,
3494                ),
3495                pubkey: info.index_info.pubkey,
3496                data_len: info.index_info.data_len,
3497            });
3498        });
3499
3500        // sort by pubkey to keep account index lookups close
3501        let num_duplicated_accounts = Self::sort_and_remove_dups(&mut stored_accounts);
3502
3503        GetUniqueAccountsResult {
3504            stored_accounts,
3505            capacity,
3506            num_duplicated_accounts,
3507        }
3508    }
3509
3510    #[cfg(feature = "dev-context-only-utils")]
3511    pub fn set_storage_access(&mut self, storage_access: StorageAccess) {
3512        self.storage_access = storage_access;
3513    }
3514
3515    /// Sort `accounts` by pubkey and removes all but the *last* of consecutive
3516    /// accounts in the vector with the same pubkey.
3517    ///
3518    /// Return the number of duplicated elements in the vector.
3519    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
3520    fn sort_and_remove_dups(accounts: &mut Vec<AccountFromStorage>) -> usize {
3521        // stable sort because we want the most recent only
3522        accounts.sort_by(|a, b| a.pubkey().cmp(b.pubkey()));
3523        let len0 = accounts.len();
3524        if accounts.len() > 1 {
3525            let mut last = 0;
3526            let mut curr = 1;
3527
3528            while curr < accounts.len() {
3529                if accounts[curr].pubkey() == accounts[last].pubkey() {
3530                    accounts[last] = accounts[curr];
3531                } else {
3532                    last += 1;
3533                    accounts[last] = accounts[curr];
3534                }
3535                curr += 1;
3536            }
3537            accounts.truncate(last + 1);
3538        }
3539        len0 - accounts.len()
3540    }
3541
3542    pub(crate) fn get_unique_accounts_from_storage_for_shrink(
3543        &self,
3544        store: &AccountStorageEntry,
3545        stats: &ShrinkStats,
3546    ) -> GetUniqueAccountsResult {
3547        let (result, storage_read_elapsed_us) =
3548            measure_us!(self.get_unique_accounts_from_storage(store));
3549        stats
3550            .storage_read_elapsed
3551            .fetch_add(storage_read_elapsed_us, Ordering::Relaxed);
3552        stats
3553            .num_duplicated_accounts
3554            .fetch_add(result.num_duplicated_accounts as u64, Ordering::Relaxed);
3555        result
3556    }
3557
3558    /// shared code for shrinking normal slots and combining into ancient append vecs
3559    /// note 'unique_accounts' is passed by ref so we can return references to data within it, avoiding self-references
3560    pub(crate) fn shrink_collect<'a: 'b, 'b, T: ShrinkCollectRefs<'b>>(
3561        &self,
3562        store: &'a AccountStorageEntry,
3563        unique_accounts: &'b GetUniqueAccountsResult,
3564        stats: &ShrinkStats,
3565    ) -> ShrinkCollect<'b, T> {
3566        let slot = store.slot();
3567
3568        let GetUniqueAccountsResult {
3569            stored_accounts,
3570            capacity,
3571            num_duplicated_accounts,
3572        } = unique_accounts;
3573
3574        let mut index_read_elapsed = Measure::start("index_read_elapsed");
3575
3576        let len = stored_accounts.len();
3577        let alive_accounts_collect = Mutex::new(T::with_capacity(len, slot));
3578        let pubkeys_to_unref_collect = Mutex::new(Vec::with_capacity(len));
3579        let zero_lamport_single_ref_pubkeys_collect = Mutex::new(Vec::with_capacity(len));
3580        stats
3581            .accounts_loaded
3582            .fetch_add(len as u64, Ordering::Relaxed);
3583        stats
3584            .num_duplicated_accounts
3585            .fetch_add(*num_duplicated_accounts as u64, Ordering::Relaxed);
3586        let all_are_zero_lamports_collect = Mutex::new(true);
3587        self.thread_pool_clean.install(|| {
3588            stored_accounts
3589                .par_chunks(SHRINK_COLLECT_CHUNK_SIZE)
3590                .for_each(|stored_accounts| {
3591                    let LoadAccountsIndexForShrink {
3592                        alive_accounts,
3593                        mut pubkeys_to_unref,
3594                        all_are_zero_lamports,
3595                        mut zero_lamport_single_ref_pubkeys,
3596                    } = self.load_accounts_index_for_shrink(stored_accounts, stats, slot);
3597
3598                    // collect
3599                    alive_accounts_collect
3600                        .lock()
3601                        .unwrap()
3602                        .collect(alive_accounts);
3603                    pubkeys_to_unref_collect
3604                        .lock()
3605                        .unwrap()
3606                        .append(&mut pubkeys_to_unref);
3607                    zero_lamport_single_ref_pubkeys_collect
3608                        .lock()
3609                        .unwrap()
3610                        .append(&mut zero_lamport_single_ref_pubkeys);
3611                    if !all_are_zero_lamports {
3612                        *all_are_zero_lamports_collect.lock().unwrap() = false;
3613                    }
3614                });
3615        });
3616
3617        let alive_accounts = alive_accounts_collect.into_inner().unwrap();
3618        let pubkeys_to_unref = pubkeys_to_unref_collect.into_inner().unwrap();
3619        let zero_lamport_single_ref_pubkeys = zero_lamport_single_ref_pubkeys_collect
3620            .into_inner()
3621            .unwrap();
3622
3623        index_read_elapsed.stop();
3624        stats
3625            .index_read_elapsed
3626            .fetch_add(index_read_elapsed.as_us(), Ordering::Relaxed);
3627
3628        let alive_total_bytes = alive_accounts.alive_bytes();
3629
3630        stats
3631            .accounts_removed
3632            .fetch_add(len - alive_accounts.len(), Ordering::Relaxed);
3633        stats.bytes_removed.fetch_add(
3634            capacity.saturating_sub(alive_total_bytes as u64),
3635            Ordering::Relaxed,
3636        );
3637        stats
3638            .bytes_written
3639            .fetch_add(alive_total_bytes as u64, Ordering::Relaxed);
3640
3641        ShrinkCollect {
3642            slot,
3643            capacity: *capacity,
3644            pubkeys_to_unref,
3645            zero_lamport_single_ref_pubkeys,
3646            alive_accounts,
3647            alive_total_bytes,
3648            total_starting_accounts: len,
3649            all_are_zero_lamports: all_are_zero_lamports_collect.into_inner().unwrap(),
3650        }
3651    }
3652
3653    /// These accounts were found during shrink of `slot` to be slot_list=[slot] and ref_count == 1 and lamports = 0.
3654    /// This means this slot contained the only account data for this pubkey and it is zero lamport.
3655    /// Thus, we did NOT treat this as an alive account, so we did NOT copy the zero lamport account to the new
3656    /// storage. So, the account will no longer be alive or exist at `slot`.
3657    /// So, first, remove the ref count since this newly shrunk storage will no longer access it.
3658    /// Second, remove `slot` from the index entry's slot list. If the slot list is now empty, then the
3659    /// pubkey can be removed completely from the index.
3660    /// In parallel with this code (which is running in the bg), the same pubkey could be revived and written to
3661    /// as part of tx processing. In that case, the slot list will contain a slot in the write cache and the
3662    /// index entry will NOT be deleted.
3663    fn remove_zero_lamport_single_ref_accounts_after_shrink(
3664        &self,
3665        zero_lamport_single_ref_pubkeys: &[&Pubkey],
3666        slot: Slot,
3667        stats: &ShrinkStats,
3668        do_assert: bool,
3669    ) {
3670        stats.purged_zero_lamports.fetch_add(
3671            zero_lamport_single_ref_pubkeys.len() as u64,
3672            Ordering::Relaxed,
3673        );
3674
3675        // we have to unref before we `purge_keys_exact`. Otherwise, we could race with the foreground with tx processing
3676        // reviving this index entry and then we'd unref the revived version, which is a refcount bug.
3677
3678        self.accounts_index.scan(
3679            zero_lamport_single_ref_pubkeys.iter().cloned(),
3680            |_pubkey, _slots_refs, _entry| AccountsIndexScanResult::Unref,
3681            if do_assert {
3682                Some(AccountsIndexScanResult::UnrefAssert0)
3683            } else {
3684                Some(AccountsIndexScanResult::UnrefLog0)
3685            },
3686            false,
3687            ScanFilter::All,
3688        );
3689
3690        zero_lamport_single_ref_pubkeys.iter().for_each(|k| {
3691            _ = self.purge_keys_exact([&(**k, slot)].into_iter());
3692        });
3693    }
3694
3695    /// common code from shrink and combine_ancient_slots
3696    /// get rid of all original store_ids in the slot
3697    pub(crate) fn remove_old_stores_shrink<'a, T: ShrinkCollectRefs<'a>>(
3698        &self,
3699        shrink_collect: &ShrinkCollect<'a, T>,
3700        stats: &ShrinkStats,
3701        shrink_in_progress: Option<ShrinkInProgress>,
3702        shrink_can_be_active: bool,
3703    ) {
3704        let mut time = Measure::start("remove_old_stores_shrink");
3705
3706        // handle the zero lamport alive accounts before calling clean
3707        // We have to update the index entries for these zero lamport pubkeys before we remove the storage in `mark_dirty_dead_stores`
3708        // that contained the accounts.
3709        self.remove_zero_lamport_single_ref_accounts_after_shrink(
3710            &shrink_collect.zero_lamport_single_ref_pubkeys,
3711            shrink_collect.slot,
3712            stats,
3713            false,
3714        );
3715
3716        // Purge old, overwritten storage entries
3717        // This has the side effect of dropping `shrink_in_progress`, which removes the old storage completely. The
3718        // index has to be correct before we drop the old storage.
3719        let dead_storages = self.mark_dirty_dead_stores(
3720            shrink_collect.slot,
3721            // If all accounts are zero lamports, then we want to mark the entire OLD append vec as dirty.
3722            // otherwise, we'll call 'add_uncleaned_pubkeys_after_shrink' just on the unref'd keys below.
3723            shrink_collect.all_are_zero_lamports,
3724            shrink_in_progress,
3725            shrink_can_be_active,
3726        );
3727        let dead_storages_len = dead_storages.len();
3728
3729        if !shrink_collect.all_are_zero_lamports {
3730            self.add_uncleaned_pubkeys_after_shrink(
3731                shrink_collect.slot,
3732                shrink_collect.pubkeys_to_unref.iter().cloned().cloned(),
3733            );
3734        }
3735
3736        let (_, drop_storage_entries_elapsed) = measure_us!(drop(dead_storages));
3737        time.stop();
3738
3739        self.stats
3740            .dropped_stores
3741            .fetch_add(dead_storages_len as u64, Ordering::Relaxed);
3742        stats
3743            .drop_storage_entries_elapsed
3744            .fetch_add(drop_storage_entries_elapsed, Ordering::Relaxed);
3745        stats
3746            .remove_old_stores_shrink_us
3747            .fetch_add(time.as_us(), Ordering::Relaxed);
3748    }
3749
3750    pub(crate) fn unref_shrunk_dead_accounts<'a>(
3751        &self,
3752        pubkeys: impl Iterator<Item = &'a Pubkey>,
3753        slot: Slot,
3754    ) {
3755        self.accounts_index.scan(
3756            pubkeys,
3757            |pubkey, slot_refs, _entry| {
3758                if slot_refs.is_none() {
3759                    // We also expect that the accounts index must contain an
3760                    // entry for `pubkey`. Log a warning for now. In future,
3761                    // we will panic when this happens.
3762                    warn!(
3763                        "pubkey {pubkey} in slot {slot} was NOT found in accounts index during \
3764                         shrink"
3765                    );
3766                    datapoint_warn!(
3767                        "accounts_db-shink_pubkey_missing_from_index",
3768                        ("store_slot", slot, i64),
3769                        ("pubkey", pubkey.to_string(), String),
3770                    )
3771                }
3772                AccountsIndexScanResult::Unref
3773            },
3774            None,
3775            false,
3776            ScanFilter::All,
3777        );
3778    }
3779
3780    /// Shrinks `store` by rewriting the alive accounts to a new storage
3781    fn shrink_storage(&self, store: &AccountStorageEntry) {
3782        let slot = store.slot();
3783        if self.accounts_cache.contains(slot) {
3784            // 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.
3785            // 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.
3786            // 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.
3787            // During the second window, once an append vec has been created for the slot, it could be possible to try to shrink that slot.
3788            // Shrink no-ops before this function if there is no store for the slot (notice this function requires 'store' to be passed).
3789            // So, if we enter this function but the slot is still in the write cache, reasonable behavior is to skip shrinking this slot.
3790            // Flush will ONLY write alive accounts to the append vec, which is what shrink does anyway.
3791            // Flush then adds the slot to 'uncleaned_roots', which causes clean to take a look at the slot.
3792            // Clean causes us to mark accounts as dead, which causes shrink to later take a look at the slot.
3793            // This could be an assert, but it could lead to intermittency in tests.
3794            // It is 'correct' to ignore calls to shrink when a slot is still in the write cache.
3795            return;
3796        }
3797        let unique_accounts =
3798            self.get_unique_accounts_from_storage_for_shrink(store, &self.shrink_stats);
3799        debug!("do_shrink_slot_store: slot: {}", slot);
3800        let shrink_collect =
3801            self.shrink_collect::<AliveAccounts<'_>>(store, &unique_accounts, &self.shrink_stats);
3802
3803        // This shouldn't happen if alive_bytes is accurate.
3804        // However, it is possible that the remaining alive bytes could be 0. In that case, the whole slot should be marked dead by clean.
3805        if Self::should_not_shrink(
3806            shrink_collect.alive_total_bytes as u64,
3807            shrink_collect.capacity,
3808        ) || shrink_collect.alive_total_bytes == 0
3809        {
3810            if shrink_collect.alive_total_bytes == 0 {
3811                // clean needs to take care of this dead slot
3812                self.accounts_index.add_uncleaned_roots([slot]);
3813            }
3814
3815            if !shrink_collect.all_are_zero_lamports {
3816                // if all are zero lamports, then we expect that we would like to mark the whole slot dead, but we cannot. That's clean's job.
3817                info!(
3818                    "Unexpected shrink for slot {} alive {} capacity {}, likely caused by a bug \
3819                     for calculating alive bytes.",
3820                    slot, shrink_collect.alive_total_bytes, shrink_collect.capacity
3821                );
3822            }
3823
3824            self.shrink_stats
3825                .skipped_shrink
3826                .fetch_add(1, Ordering::Relaxed);
3827            return;
3828        }
3829
3830        self.unref_shrunk_dead_accounts(shrink_collect.pubkeys_to_unref.iter().cloned(), slot);
3831
3832        let total_accounts_after_shrink = shrink_collect.alive_accounts.len();
3833        debug!(
3834            "shrinking: slot: {}, accounts: ({} => {}) bytes: {} original: {}",
3835            slot,
3836            shrink_collect.total_starting_accounts,
3837            total_accounts_after_shrink,
3838            shrink_collect.alive_total_bytes,
3839            shrink_collect.capacity,
3840        );
3841
3842        let mut stats_sub = ShrinkStatsSub::default();
3843        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
3844        let (shrink_in_progress, time_us) =
3845            measure_us!(self.get_store_for_shrink(slot, shrink_collect.alive_total_bytes as u64));
3846        stats_sub.create_and_insert_store_elapsed_us = Saturating(time_us);
3847
3848        // here, we're writing back alive_accounts. That should be an atomic operation
3849        // without use of rather wide locks in this whole function, because we're
3850        // mutating rooted slots; There should be no writers to them.
3851        let accounts = [(slot, &shrink_collect.alive_accounts.alive_accounts()[..])];
3852        let storable_accounts = StorableAccountsBySlot::new(slot, &accounts, self);
3853        stats_sub.store_accounts_timing =
3854            self.store_accounts_frozen(storable_accounts, shrink_in_progress.new_storage());
3855
3856        rewrite_elapsed.stop();
3857        stats_sub.rewrite_elapsed_us = Saturating(rewrite_elapsed.as_us());
3858
3859        // `store_accounts_frozen()` above may have purged accounts from some
3860        // other storage entries (the ones that were just overwritten by this
3861        // new storage entry). This means some of those stores might have caused
3862        // this slot to be read to `self.shrink_candidate_slots`, so delete
3863        // those here
3864        self.shrink_candidate_slots.lock().unwrap().remove(&slot);
3865
3866        self.remove_old_stores_shrink(
3867            &shrink_collect,
3868            &self.shrink_stats,
3869            Some(shrink_in_progress),
3870            false,
3871        );
3872
3873        self.reopen_storage_as_readonly_shrinking_in_progress_ok(slot);
3874
3875        Self::update_shrink_stats(&self.shrink_stats, stats_sub, true);
3876        self.shrink_stats.report();
3877    }
3878
3879    pub(crate) fn update_shrink_stats(
3880        shrink_stats: &ShrinkStats,
3881        stats_sub: ShrinkStatsSub,
3882        increment_count: bool,
3883    ) {
3884        if increment_count {
3885            shrink_stats
3886                .num_slots_shrunk
3887                .fetch_add(1, Ordering::Relaxed);
3888        }
3889        shrink_stats.create_and_insert_store_elapsed.fetch_add(
3890            stats_sub.create_and_insert_store_elapsed_us.0,
3891            Ordering::Relaxed,
3892        );
3893        shrink_stats.store_accounts_elapsed.fetch_add(
3894            stats_sub.store_accounts_timing.store_accounts_elapsed,
3895            Ordering::Relaxed,
3896        );
3897        shrink_stats.update_index_elapsed.fetch_add(
3898            stats_sub.store_accounts_timing.update_index_elapsed,
3899            Ordering::Relaxed,
3900        );
3901        shrink_stats.handle_reclaims_elapsed.fetch_add(
3902            stats_sub.store_accounts_timing.handle_reclaims_elapsed,
3903            Ordering::Relaxed,
3904        );
3905        shrink_stats
3906            .rewrite_elapsed
3907            .fetch_add(stats_sub.rewrite_elapsed_us.0, Ordering::Relaxed);
3908        shrink_stats
3909            .unpackable_slots_count
3910            .fetch_add(stats_sub.unpackable_slots_count.0 as u64, Ordering::Relaxed);
3911        shrink_stats.newest_alive_packed_count.fetch_add(
3912            stats_sub.newest_alive_packed_count.0 as u64,
3913            Ordering::Relaxed,
3914        );
3915    }
3916
3917    /// get stores for 'slot'
3918    /// Drop 'shrink_in_progress', which will cause the old store to be removed from the storage map.
3919    /// For 'shrink_in_progress'.'old_storage' which is not retained, insert in 'dead_storages' and optionally 'dirty_stores'
3920    /// This is the end of the life cycle of `shrink_in_progress`.
3921    pub fn mark_dirty_dead_stores(
3922        &self,
3923        slot: Slot,
3924        add_dirty_stores: bool,
3925        shrink_in_progress: Option<ShrinkInProgress>,
3926        shrink_can_be_active: bool,
3927    ) -> Vec<Arc<AccountStorageEntry>> {
3928        let mut dead_storages = Vec::default();
3929
3930        let mut not_retaining_store = |store: &Arc<AccountStorageEntry>| {
3931            if add_dirty_stores {
3932                self.dirty_stores.insert(slot, store.clone());
3933            }
3934            dead_storages.push(store.clone());
3935        };
3936
3937        if let Some(shrink_in_progress) = shrink_in_progress {
3938            // shrink is in progress, so 1 new append vec to keep, 1 old one to throw away
3939            not_retaining_store(shrink_in_progress.old_storage());
3940            // dropping 'shrink_in_progress' removes the old append vec that was being shrunk from db's storage
3941        } else if let Some(store) = self.storage.remove(&slot, shrink_can_be_active) {
3942            // no shrink in progress, so all append vecs in this slot are dead
3943            not_retaining_store(&store);
3944        }
3945
3946        dead_storages
3947    }
3948
3949    /// we are done writing to the storage at `slot`. It can be re-opened as read-only if that would help
3950    /// system performance.
3951    pub(crate) fn reopen_storage_as_readonly_shrinking_in_progress_ok(&self, slot: Slot) {
3952        if let Some(storage) = self
3953            .storage
3954            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
3955        {
3956            if let Some(new_storage) = storage.reopen_as_readonly(self.storage_access) {
3957                // consider here the race condition of tx processing having looked up something in the index,
3958                // which could return (slot, append vec id). We want the lookup for the storage to get a storage
3959                // that works whether the lookup occurs before or after the replace call here.
3960                // So, the two storages have to be exactly equivalent wrt offsets, counts, len, id, etc.
3961                assert_eq!(storage.id(), new_storage.id());
3962                assert_eq!(storage.accounts.len(), new_storage.accounts.len());
3963                self.storage
3964                    .replace_storage_with_equivalent(slot, Arc::new(new_storage));
3965            }
3966        }
3967    }
3968
3969    /// return a store that can contain 'size' bytes
3970    pub fn get_store_for_shrink(&self, slot: Slot, size: u64) -> ShrinkInProgress<'_> {
3971        let shrunken_store = self.create_store(slot, size, "shrink", self.shrink_paths.as_slice());
3972        self.storage.shrinking_in_progress(slot, shrunken_store)
3973    }
3974
3975    // Reads all accounts in given slot's AppendVecs and filter only to alive,
3976    // then create a minimum AppendVec filled with the alive.
3977    fn shrink_slot_forced(&self, slot: Slot) {
3978        debug!("shrink_slot_forced: slot: {}", slot);
3979
3980        if let Some(store) = self
3981            .storage
3982            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
3983        {
3984            if Self::is_shrinking_productive(&store) {
3985                self.shrink_storage(&store)
3986            }
3987        }
3988    }
3989
3990    fn all_slots_in_storage(&self) -> Vec<Slot> {
3991        self.storage.all_slots()
3992    }
3993
3994    /// Given the input `ShrinkCandidates`, this function sorts the stores by their alive ratio
3995    /// in increasing order with the most sparse entries in the front. It will then simulate the
3996    /// shrinking by working on the most sparse entries first and if the overall alive ratio is
3997    /// achieved, it will stop and return:
3998    /// first tuple element: the filtered-down candidates and
3999    /// second duple element: the candidates which
4000    /// are skipped in this round and might be eligible for the future shrink.
4001    fn select_candidates_by_total_usage(
4002        &self,
4003        shrink_slots: &ShrinkCandidates,
4004        shrink_ratio: f64,
4005    ) -> (IntMap<Slot, Arc<AccountStorageEntry>>, ShrinkCandidates) {
4006        struct StoreUsageInfo {
4007            slot: Slot,
4008            alive_ratio: f64,
4009            store: Arc<AccountStorageEntry>,
4010        }
4011        let mut store_usage: Vec<StoreUsageInfo> = Vec::with_capacity(shrink_slots.len());
4012        let mut total_alive_bytes: u64 = 0;
4013        let mut total_bytes: u64 = 0;
4014        for slot in shrink_slots {
4015            let Some(store) = self.storage.get_slot_storage_entry(*slot) else {
4016                continue;
4017            };
4018            let alive_bytes = store.alive_bytes();
4019            total_alive_bytes += alive_bytes as u64;
4020            total_bytes += store.capacity();
4021            let alive_ratio = alive_bytes as f64 / store.capacity() as f64;
4022            store_usage.push(StoreUsageInfo {
4023                slot: *slot,
4024                alive_ratio,
4025                store: store.clone(),
4026            });
4027        }
4028        store_usage.sort_by(|a, b| {
4029            a.alive_ratio
4030                .partial_cmp(&b.alive_ratio)
4031                .unwrap_or(std::cmp::Ordering::Equal)
4032        });
4033
4034        // Working from the beginning of store_usage which are the most sparse and see when we can stop
4035        // shrinking while still achieving the overall goals.
4036        let mut shrink_slots = IntMap::default();
4037        let mut shrink_slots_next_batch = ShrinkCandidates::default();
4038        for usage in &store_usage {
4039            let store = &usage.store;
4040            let alive_ratio = (total_alive_bytes as f64) / (total_bytes as f64);
4041            debug!(
4042                "alive_ratio: {:?} store_id: {:?}, store_ratio: {:?} requirement: {:?}, \
4043                 total_bytes: {:?} total_alive_bytes: {:?}",
4044                alive_ratio,
4045                usage.store.id(),
4046                usage.alive_ratio,
4047                shrink_ratio,
4048                total_bytes,
4049                total_alive_bytes
4050            );
4051            if alive_ratio > shrink_ratio {
4052                // we have reached our goal, stop
4053                debug!(
4054                    "Shrinking goal can be achieved at slot {:?}, total_alive_bytes: {:?} \
4055                     total_bytes: {:?}, alive_ratio: {:}, shrink_ratio: {:?}",
4056                    usage.slot, total_alive_bytes, total_bytes, alive_ratio, shrink_ratio
4057                );
4058                if usage.alive_ratio < shrink_ratio {
4059                    shrink_slots_next_batch.insert(usage.slot);
4060                } else {
4061                    break;
4062                }
4063            } else {
4064                let current_store_size = store.capacity();
4065                let after_shrink_size = store.alive_bytes() as u64;
4066                let bytes_saved = current_store_size.saturating_sub(after_shrink_size);
4067                total_bytes -= bytes_saved;
4068                shrink_slots.insert(usage.slot, Arc::clone(store));
4069            }
4070        }
4071        (shrink_slots, shrink_slots_next_batch)
4072    }
4073
4074    fn get_roots_less_than(&self, slot: Slot) -> Vec<Slot> {
4075        self.accounts_index
4076            .roots_tracker
4077            .read()
4078            .unwrap()
4079            .alive_roots
4080            .get_all_less_than(slot)
4081    }
4082
4083    /// return all slots that are more than one epoch old and thus could already be an ancient append vec
4084    /// or which could need to be combined into a new or existing ancient append vec
4085    /// offset is used to combine newer slots than we normally would. This is designed to be used for testing.
4086    fn get_sorted_potential_ancient_slots(&self, oldest_non_ancient_slot: Slot) -> Vec<Slot> {
4087        let mut ancient_slots = self.get_roots_less_than(oldest_non_ancient_slot);
4088        ancient_slots.sort_unstable();
4089        ancient_slots
4090    }
4091
4092    /// get a sorted list of slots older than an epoch
4093    /// squash those slots into ancient append vecs
4094    pub fn shrink_ancient_slots(&self, epoch_schedule: &EpochSchedule) {
4095        if self.ancient_append_vec_offset.is_none() {
4096            return;
4097        }
4098
4099        let oldest_non_ancient_slot = self.get_oldest_non_ancient_slot(epoch_schedule);
4100        let can_randomly_shrink = true;
4101        let sorted_slots = self.get_sorted_potential_ancient_slots(oldest_non_ancient_slot);
4102        if self.create_ancient_storage == CreateAncientStorage::Append {
4103            self.combine_ancient_slots(sorted_slots, can_randomly_shrink);
4104        } else {
4105            self.combine_ancient_slots_packed(sorted_slots, can_randomly_shrink);
4106        }
4107    }
4108
4109    /// 'accounts' that exist in the current slot we are combining into a different ancient slot
4110    /// 'existing_ancient_pubkeys': pubkeys that exist currently in the ancient append vec slot
4111    /// returns the pubkeys that are in 'accounts' that are already in 'existing_ancient_pubkeys'
4112    /// Also updated 'existing_ancient_pubkeys' to include all pubkeys in 'accounts' since they will soon be written into the ancient slot.
4113    fn get_keys_to_unref_ancient<'a>(
4114        accounts: &'a [&AccountFromStorage],
4115        existing_ancient_pubkeys: &mut HashSet<Pubkey>,
4116    ) -> HashSet<&'a Pubkey> {
4117        let mut unref = HashSet::<&Pubkey>::default();
4118        // 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.
4119        // 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.
4120        // The slot that currently references the account is going away, so unref to maintain # slots that reference the pubkey = refcount.
4121        accounts.iter().for_each(|account| {
4122            let key = account.pubkey();
4123            if !existing_ancient_pubkeys.insert(*key) {
4124                // this key exists BOTH in 'accounts' and already in the ancient append vec, so we need to unref it
4125                unref.insert(key);
4126            }
4127        });
4128        unref
4129    }
4130
4131    /// 'accounts' are about to be appended to an ancient append vec. That ancient append vec may already have some accounts.
4132    /// Unref each account in 'accounts' that already exists in 'existing_ancient_pubkeys'.
4133    /// As a side effect, on exit, 'existing_ancient_pubkeys' will now contain all pubkeys in 'accounts'.
4134    fn unref_accounts_already_in_storage(
4135        &self,
4136        accounts: &[&AccountFromStorage],
4137        existing_ancient_pubkeys: &mut HashSet<Pubkey>,
4138    ) {
4139        let unref = Self::get_keys_to_unref_ancient(accounts, existing_ancient_pubkeys);
4140
4141        self.unref_pubkeys(
4142            unref.iter().cloned(),
4143            unref.len(),
4144            &PubkeysRemovedFromAccountsIndex::default(),
4145        );
4146    }
4147
4148    /// get the storage from 'slot' to squash
4149    /// or None if this slot should be skipped
4150    /// side effect could be updating 'current_ancient'
4151    fn get_storage_to_move_to_ancient_accounts_file(
4152        &self,
4153        slot: Slot,
4154        current_ancient: &mut CurrentAncientAccountsFile,
4155        can_randomly_shrink: bool,
4156    ) -> Option<Arc<AccountStorageEntry>> {
4157        self.storage
4158            .get_slot_storage_entry(slot)
4159            .and_then(|storage| {
4160                self.should_move_to_ancient_accounts_file(
4161                    &storage,
4162                    current_ancient,
4163                    slot,
4164                    can_randomly_shrink,
4165                )
4166                .then_some(storage)
4167            })
4168    }
4169
4170    /// return true if the accounts in this slot should be moved to an ancient append vec
4171    /// otherwise, return false and the caller can skip this slot
4172    /// side effect could be updating 'current_ancient'
4173    /// can_randomly_shrink: true if ancient append vecs that otherwise don't qualify to be shrunk can be randomly shrunk
4174    ///  this is convenient for a running system
4175    ///  this is not useful for testing
4176    fn should_move_to_ancient_accounts_file(
4177        &self,
4178        storage: &Arc<AccountStorageEntry>,
4179        current_ancient: &mut CurrentAncientAccountsFile,
4180        slot: Slot,
4181        can_randomly_shrink: bool,
4182    ) -> bool {
4183        let accounts = &storage.accounts;
4184
4185        self.shrink_ancient_stats
4186            .slots_considered
4187            .fetch_add(1, Ordering::Relaxed);
4188
4189        // if an append vec is at least 80% of the ideal capacity of an ancient append vec, that's close enough.
4190        // If we packed, then we end up allocating exact size ancient append vecs. Those will likely never be exactly the ideal ancient capacity.
4191        if accounts.capacity() * 100 / get_ancient_append_vec_capacity() > 80 {
4192            self.shrink_ancient_stats
4193                .ancient_scanned
4194                .fetch_add(1, Ordering::Relaxed);
4195
4196            // randomly shrink ancient slots
4197            // this exercises the ancient shrink code more often
4198            let written_bytes = storage.written_bytes();
4199            let mut alive_ratio = 0;
4200            let is_candidate = if written_bytes > 0 {
4201                alive_ratio = (storage.alive_bytes() as u64) * 100 / written_bytes;
4202                alive_ratio < 90
4203            } else {
4204                false
4205            };
4206            if is_candidate || (can_randomly_shrink && thread_rng().gen_range(0..10000) == 0) {
4207                // we are a candidate for shrink, so either append us to the previous append vec
4208                // or recreate us as a new append vec and eliminate the dead accounts
4209                info!(
4210                    "ancient_append_vec: shrinking full ancient: {}, random: {}, alive_ratio: {}",
4211                    slot, !is_candidate, alive_ratio
4212                );
4213                if !is_candidate {
4214                    self.shrink_ancient_stats
4215                        .random_shrink
4216                        .fetch_add(1, Ordering::Relaxed);
4217                }
4218                self.shrink_ancient_stats
4219                    .ancient_append_vecs_shrunk
4220                    .fetch_add(1, Ordering::Relaxed);
4221                return true;
4222            }
4223            if storage.accounts.can_append() {
4224                // this slot is ancient and can become the 'current' ancient for other slots to be squashed into
4225                *current_ancient = CurrentAncientAccountsFile::new(slot, Arc::clone(storage));
4226            } else {
4227                *current_ancient = CurrentAncientAccountsFile::default();
4228            }
4229            return false; // we're done with this slot - this slot IS the ancient append vec
4230        }
4231
4232        // otherwise, yes, squash this slot into the current ancient append vec or create one at this slot
4233        true
4234    }
4235
4236    /// Combine all account data from storages in 'sorted_slots' into ancient append vecs.
4237    /// This keeps us from accumulating append vecs for each slot older than an epoch.
4238    fn combine_ancient_slots(&self, sorted_slots: Vec<Slot>, can_randomly_shrink: bool) {
4239        if sorted_slots.is_empty() {
4240            return;
4241        }
4242
4243        let mut total = Measure::start("combine_ancient_slots");
4244        let mut guard = None;
4245
4246        // the ancient append vec currently being written to
4247        let mut current_ancient = CurrentAncientAccountsFile::default();
4248        let mut dropped_roots = vec![];
4249
4250        // we have to keep track of what pubkeys exist in the current ancient append vec so we can unref correctly
4251        let mut ancient_slot_pubkeys = AncientSlotPubkeys::default();
4252
4253        let len = sorted_slots.len();
4254        for slot in sorted_slots {
4255            let Some(old_storage) = self.get_storage_to_move_to_ancient_accounts_file(
4256                slot,
4257                &mut current_ancient,
4258                can_randomly_shrink,
4259            ) else {
4260                // nothing to squash for this slot
4261                continue;
4262            };
4263
4264            if guard.is_none() {
4265                // we are now doing interesting work in squashing ancient
4266                guard = Some(self.active_stats.activate(ActiveStatItem::SquashAncient));
4267                info!(
4268                    "ancient_append_vec: combine_ancient_slots first slot: {}, num_roots: {}",
4269                    slot, len
4270                );
4271            }
4272
4273            self.combine_one_store_into_ancient(
4274                slot,
4275                &old_storage,
4276                &mut current_ancient,
4277                &mut ancient_slot_pubkeys,
4278                &mut dropped_roots,
4279            );
4280        }
4281
4282        self.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
4283
4284        total.stop();
4285        self.shrink_ancient_stats
4286            .total_us
4287            .fetch_add(total.as_us(), Ordering::Relaxed);
4288
4289        // only log when we moved some accounts to ancient append vecs or we've exceeded 100ms
4290        // results will continue to accumulate otherwise
4291        if guard.is_some() || self.shrink_ancient_stats.total_us.load(Ordering::Relaxed) > 100_000 {
4292            self.shrink_ancient_stats.report();
4293        }
4294    }
4295
4296    /// put entire alive contents of 'old_storage' into the current ancient append vec or a newly created ancient append vec
4297    fn combine_one_store_into_ancient(
4298        &self,
4299        slot: Slot,
4300        old_storage: &Arc<AccountStorageEntry>,
4301        current_ancient: &mut CurrentAncientAccountsFile,
4302        ancient_slot_pubkeys: &mut AncientSlotPubkeys,
4303        dropped_roots: &mut Vec<Slot>,
4304    ) {
4305        let unique_accounts = self.get_unique_accounts_from_storage_for_shrink(
4306            old_storage,
4307            &self.shrink_ancient_stats.shrink_stats,
4308        );
4309        let shrink_collect = self.shrink_collect::<AliveAccounts<'_>>(
4310            old_storage,
4311            &unique_accounts,
4312            &self.shrink_ancient_stats.shrink_stats,
4313        );
4314
4315        // could follow what shrink does more closely
4316        if shrink_collect.total_starting_accounts == 0 || shrink_collect.alive_total_bytes == 0 {
4317            return; // skipping slot with no useful accounts to write
4318        }
4319
4320        let mut stats_sub = ShrinkStatsSub::default();
4321        let mut bytes_remaining_to_write = shrink_collect.alive_total_bytes;
4322        let (mut shrink_in_progress, create_and_insert_store_elapsed_us) = measure_us!(
4323            current_ancient.create_if_necessary(slot, self, shrink_collect.alive_total_bytes)
4324        );
4325        stats_sub.create_and_insert_store_elapsed_us =
4326            Saturating(create_and_insert_store_elapsed_us);
4327        let available_bytes = current_ancient.accounts_file().accounts.remaining_bytes();
4328        // split accounts in 'slot' into:
4329        // 'Primary', which can fit in 'current_ancient'
4330        // 'Overflow', which will have to go into a new ancient append vec at 'slot'
4331        let to_store = AccountsToStore::new(
4332            available_bytes,
4333            shrink_collect.alive_accounts.alive_accounts(),
4334            shrink_collect.alive_total_bytes,
4335            slot,
4336        );
4337
4338        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
4339            slot,
4340            self,
4341            current_ancient,
4342            &to_store,
4343        );
4344
4345        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
4346        // write what we can to the current ancient storage
4347        let (store_accounts_timing, bytes_written) =
4348            current_ancient.store_ancient_accounts(self, &to_store, StorageSelector::Primary);
4349        stats_sub.store_accounts_timing = store_accounts_timing;
4350        bytes_remaining_to_write = bytes_remaining_to_write.saturating_sub(bytes_written as usize);
4351
4352        // handle accounts from 'slot' which did not fit into the current ancient append vec
4353        if to_store.has_overflow() {
4354            // We need a new ancient append vec at this slot.
4355            // Assert: it cannot be the case that we already had an ancient append vec at this slot and
4356            // yet that ancient append vec does not have room for the accounts stored at this slot currently
4357            assert_ne!(slot, current_ancient.slot());
4358
4359            // we filled one up
4360            self.reopen_storage_as_readonly_shrinking_in_progress_ok(current_ancient.slot());
4361
4362            // Now we create an ancient append vec at `slot` to store the overflows.
4363            let (shrink_in_progress_overflow, time_us) = measure_us!(current_ancient
4364                .create_ancient_accounts_file(
4365                    slot,
4366                    self,
4367                    to_store.get_bytes(StorageSelector::Overflow)
4368                ));
4369            stats_sub.create_and_insert_store_elapsed_us += time_us;
4370            // We cannot possibly be shrinking the original slot that created an ancient append vec
4371            // AND not have enough room in the ancient append vec at that slot
4372            // to hold all the contents of that slot.
4373            // We need this new 'shrink_in_progress' to be used in 'remove_old_stores_shrink' below.
4374            // All non-overflow accounts were put in a prior slot's ancient append vec. All overflow accounts
4375            // are essentially being shrunk into a new ancient append vec in 'slot'.
4376            assert!(shrink_in_progress.is_none());
4377            shrink_in_progress = Some(shrink_in_progress_overflow);
4378
4379            // write the overflow accounts to the next ancient storage
4380            let (store_accounts_timing, bytes_written) =
4381                current_ancient.store_ancient_accounts(self, &to_store, StorageSelector::Overflow);
4382            bytes_remaining_to_write =
4383                bytes_remaining_to_write.saturating_sub(bytes_written as usize);
4384
4385            stats_sub
4386                .store_accounts_timing
4387                .accumulate(&store_accounts_timing);
4388        }
4389        assert_eq!(bytes_remaining_to_write, 0);
4390        rewrite_elapsed.stop();
4391        stats_sub.rewrite_elapsed_us = Saturating(rewrite_elapsed.as_us());
4392
4393        if slot != current_ancient.slot() {
4394            // all append vecs in this slot have been combined into an ancient append vec
4395            dropped_roots.push(slot);
4396        }
4397
4398        self.remove_old_stores_shrink(
4399            &shrink_collect,
4400            &self.shrink_ancient_stats.shrink_stats,
4401            shrink_in_progress,
4402            false,
4403        );
4404
4405        // 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.
4406        self.shrink_candidate_slots.lock().unwrap().remove(&slot);
4407
4408        Self::update_shrink_stats(&self.shrink_ancient_stats.shrink_stats, stats_sub, true);
4409    }
4410
4411    /// each slot in 'dropped_roots' has been combined into an ancient append vec.
4412    /// We are done with the slot now forever.
4413    pub(crate) fn handle_dropped_roots_for_ancient(
4414        &self,
4415        dropped_roots: impl Iterator<Item = Slot>,
4416    ) {
4417        let mut accounts_delta_hashes = self.accounts_delta_hashes.lock().unwrap();
4418        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
4419
4420        dropped_roots.for_each(|slot| {
4421            self.accounts_index.clean_dead_slot(slot);
4422            accounts_delta_hashes.remove(&slot);
4423            bank_hash_stats.remove(&slot);
4424            // the storage has been removed from this slot and recycled or dropped
4425            assert!(self.storage.remove(&slot, false).is_none());
4426            debug_assert!(
4427                !self
4428                    .accounts_index
4429                    .roots_tracker
4430                    .read()
4431                    .unwrap()
4432                    .alive_roots
4433                    .contains(&slot),
4434                "slot: {slot}"
4435            );
4436        });
4437    }
4438
4439    /// add all 'pubkeys' into the set of pubkeys that are 'uncleaned', associated with 'slot'
4440    /// clean will visit these pubkeys next time it runs
4441    fn add_uncleaned_pubkeys_after_shrink(
4442        &self,
4443        slot: Slot,
4444        pubkeys: impl Iterator<Item = Pubkey>,
4445    ) {
4446        /*
4447        This is only called during 'shrink'-type operations.
4448        Original accounts were separated into 'accounts' and 'pubkeys_to_unref'.
4449        These sets correspond to 'alive' and 'dead'.
4450        'alive' means this account in this slot is in the accounts index.
4451        'dead' means this account in this slot is NOT in the accounts index.
4452        If dead, nobody will care if this version of this account is not written into the newly shrunk append vec for this slot.
4453        For all dead accounts, they were already unrefed and are now absent in the new append vec.
4454        This means that another version of this pubkey could possibly now be cleaned since this one is now gone.
4455        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.
4456        So, for all unrefed accounts, send them to clean to be revisited next time clean runs.
4457        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.
4458        Clean doesn't care about alive accounts that remain alive.
4459        Except... A slightly different case is if ALL the alive accounts in this slot are zero lamport accounts, then it is possible that
4460        this slot can be marked dead. So, if all alive accounts are zero lamports, we send the entire OLD/pre-shrunk append vec
4461        to clean so that all the pubkeys are visited.
4462        It is a performance optimization to not send the ENTIRE old/pre-shrunk append vec to clean in the normal case.
4463        */
4464
4465        let mut uncleaned_pubkeys = self.uncleaned_pubkeys.entry(slot).or_default();
4466        uncleaned_pubkeys.extend(pubkeys);
4467    }
4468
4469    pub fn shrink_candidate_slots(&self, epoch_schedule: &EpochSchedule) -> usize {
4470        let oldest_non_ancient_slot = self.get_oldest_non_ancient_slot(epoch_schedule);
4471
4472        let shrink_candidates_slots =
4473            std::mem::take(&mut *self.shrink_candidate_slots.lock().unwrap());
4474        self.shrink_stats
4475            .initial_candidates_count
4476            .store(shrink_candidates_slots.len() as u64, Ordering::Relaxed);
4477
4478        let candidates_count = shrink_candidates_slots.len();
4479        let ((mut shrink_slots, shrink_slots_next_batch), select_time_us) = measure_us!({
4480            if let AccountShrinkThreshold::TotalSpace { shrink_ratio } = self.shrink_ratio {
4481                let (shrink_slots, shrink_slots_next_batch) =
4482                    self.select_candidates_by_total_usage(&shrink_candidates_slots, shrink_ratio);
4483                (shrink_slots, Some(shrink_slots_next_batch))
4484            } else {
4485                (
4486                    // lookup storage for each slot
4487                    shrink_candidates_slots
4488                        .into_iter()
4489                        .filter_map(|slot| {
4490                            self.storage
4491                                .get_slot_storage_entry(slot)
4492                                .map(|storage| (slot, storage))
4493                        })
4494                        .collect(),
4495                    None,
4496                )
4497            }
4498        });
4499
4500        // If there are too few slots to shrink, add an ancient slot
4501        // for shrinking.
4502        if shrink_slots.len() < SHRINK_INSERT_ANCIENT_THRESHOLD {
4503            let mut ancients = self.best_ancient_slots_to_shrink.write().unwrap();
4504            while let Some((slot, capacity)) = ancients.pop_front() {
4505                if let Some(store) = self.storage.get_slot_storage_entry(slot) {
4506                    if !shrink_slots.contains(&slot)
4507                        && capacity == store.capacity()
4508                        && Self::is_candidate_for_shrink(self, &store)
4509                    {
4510                        let ancient_bytes_added_to_shrink = store.alive_bytes() as u64;
4511                        shrink_slots.insert(slot, store);
4512                        self.shrink_stats
4513                            .ancient_bytes_added_to_shrink
4514                            .fetch_add(ancient_bytes_added_to_shrink, Ordering::Relaxed);
4515                        self.shrink_stats
4516                            .ancient_slots_added_to_shrink
4517                            .fetch_add(1, Ordering::Relaxed);
4518                        break;
4519                    }
4520                }
4521            }
4522        }
4523        if shrink_slots.is_empty()
4524            && shrink_slots_next_batch
4525                .as_ref()
4526                .map(|s| s.is_empty())
4527                .unwrap_or(true)
4528        {
4529            return 0;
4530        }
4531
4532        let _guard = (!shrink_slots.is_empty())
4533            .then_some(|| self.active_stats.activate(ActiveStatItem::Shrink));
4534
4535        let num_selected = shrink_slots.len();
4536        let (_, shrink_all_us) = measure_us!({
4537            self.thread_pool_clean.install(|| {
4538                shrink_slots
4539                    .into_par_iter()
4540                    .for_each(|(slot, slot_shrink_candidate)| {
4541                        if self.ancient_append_vec_offset.is_some()
4542                            && slot < oldest_non_ancient_slot
4543                        {
4544                            self.shrink_stats
4545                                .num_ancient_slots_shrunk
4546                                .fetch_add(1, Ordering::Relaxed);
4547                        }
4548                        self.shrink_storage(&slot_shrink_candidate);
4549                    });
4550            })
4551        });
4552
4553        let mut pended_counts: usize = 0;
4554        if let Some(shrink_slots_next_batch) = shrink_slots_next_batch {
4555            let mut shrink_slots = self.shrink_candidate_slots.lock().unwrap();
4556            pended_counts = shrink_slots_next_batch.len();
4557            for slot in shrink_slots_next_batch {
4558                shrink_slots.insert(slot);
4559            }
4560        }
4561
4562        datapoint_info!(
4563            "shrink_candidate_slots",
4564            ("select_time_us", select_time_us, i64),
4565            ("shrink_all_us", shrink_all_us, i64),
4566            ("candidates_count", candidates_count, i64),
4567            ("selected_count", num_selected, i64),
4568            ("deferred_to_next_round_count", pended_counts, i64)
4569        );
4570
4571        num_selected
4572    }
4573
4574    /// This is only called at startup from bank when we are being extra careful such as when we downloaded a snapshot.
4575    /// Also called from tests.
4576    /// `newest_slot_skip_shrink_inclusive` is used to avoid shrinking the slot we are loading a snapshot from. If we shrink that slot, we affect
4577    /// the bank hash calculation verification at startup.
4578    pub fn shrink_all_slots(
4579        &self,
4580        is_startup: bool,
4581        epoch_schedule: &EpochSchedule,
4582        newest_slot_skip_shrink_inclusive: Option<Slot>,
4583    ) {
4584        let _guard = self.active_stats.activate(ActiveStatItem::Shrink);
4585        const DIRTY_STORES_CLEANING_THRESHOLD: usize = 10_000;
4586        const OUTER_CHUNK_SIZE: usize = 2000;
4587        let mut slots = self.all_slots_in_storage();
4588        if let Some(newest_slot_skip_shrink_inclusive) = newest_slot_skip_shrink_inclusive {
4589            // at startup, we cannot shrink the slot that we're about to replay and recalculate bank hash for.
4590            // That storage's contents are used to verify the bank hash (and accounts delta hash) of the startup slot.
4591            slots.retain(|slot| slot < &newest_slot_skip_shrink_inclusive);
4592        }
4593
4594        // if we are restoring from incremental + full snapshot, then we cannot clean past latest_full_snapshot_slot.
4595        // If we were to clean past that, then we could mark accounts prior to latest_full_snapshot_slot as dead.
4596        // If we mark accounts prior to latest_full_snapshot_slot as dead, then we could shrink those accounts away.
4597        // If we shrink accounts away, then when we run the full hash of all accounts calculation up to latest_full_snapshot_slot,
4598        // then we will get the wrong answer, because some accounts may be GONE from the slot range up to latest_full_snapshot_slot.
4599        // So, we can only clean UP TO and including latest_full_snapshot_slot.
4600        // As long as we don't mark anything as dead at slots > latest_full_snapshot_slot, then shrink will have nothing to do for
4601        // slots > latest_full_snapshot_slot.
4602        let maybe_clean = || {
4603            if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
4604                let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
4605                self.clean_accounts(
4606                    latest_full_snapshot_slot,
4607                    is_startup,
4608                    epoch_schedule,
4609                    // Leave any old storages alone for now.  Once the validator is running
4610                    // normal, calls to clean_accounts() will have the correct policy based
4611                    // on if ancient storages are enabled or not.
4612                    OldStoragesPolicy::Leave,
4613                );
4614            }
4615        };
4616
4617        if is_startup {
4618            let threads = num_cpus::get();
4619            let inner_chunk_size = std::cmp::max(OUTER_CHUNK_SIZE / threads, 1);
4620            slots.chunks(OUTER_CHUNK_SIZE).for_each(|chunk| {
4621                chunk.par_chunks(inner_chunk_size).for_each(|slots| {
4622                    for slot in slots {
4623                        self.shrink_slot_forced(*slot);
4624                    }
4625                });
4626                maybe_clean();
4627            });
4628        } else {
4629            for slot in slots {
4630                self.shrink_slot_forced(slot);
4631                maybe_clean();
4632            }
4633        }
4634    }
4635
4636    pub fn scan_accounts<F>(
4637        &self,
4638        ancestors: &Ancestors,
4639        bank_id: BankId,
4640        mut scan_func: F,
4641        config: &ScanConfig,
4642    ) -> ScanResult<()>
4643    where
4644        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4645    {
4646        // This can error out if the slots being scanned over are aborted
4647        self.accounts_index.scan_accounts(
4648            ancestors,
4649            bank_id,
4650            |pubkey, (account_info, slot)| {
4651                let account_slot = self
4652                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4653                    .get_loaded_account(|loaded_account| {
4654                        (pubkey, loaded_account.take_account(), slot)
4655                    });
4656                scan_func(account_slot)
4657            },
4658            config,
4659        )?;
4660
4661        Ok(())
4662    }
4663
4664    pub fn unchecked_scan_accounts<F>(
4665        &self,
4666        metric_name: &'static str,
4667        ancestors: &Ancestors,
4668        mut scan_func: F,
4669        config: &ScanConfig,
4670    ) where
4671        F: FnMut(&Pubkey, LoadedAccount, Slot),
4672    {
4673        self.accounts_index.unchecked_scan_accounts(
4674            metric_name,
4675            ancestors,
4676            |pubkey, (account_info, slot)| {
4677                self.get_account_accessor(slot, pubkey, &account_info.storage_location())
4678                    .get_loaded_account(|loaded_account| {
4679                        scan_func(pubkey, loaded_account, slot);
4680                    });
4681            },
4682            config,
4683        );
4684    }
4685
4686    /// Only guaranteed to be safe when called from rent collection
4687    pub fn range_scan_accounts<F, R>(
4688        &self,
4689        metric_name: &'static str,
4690        ancestors: &Ancestors,
4691        range: R,
4692        config: &ScanConfig,
4693        mut scan_func: F,
4694    ) where
4695        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4696        R: RangeBounds<Pubkey> + std::fmt::Debug,
4697    {
4698        self.accounts_index.range_scan_accounts(
4699            metric_name,
4700            ancestors,
4701            range,
4702            config,
4703            |pubkey, (account_info, slot)| {
4704                // unlike other scan fns, this is called from Bank::collect_rent_eagerly(),
4705                // which is on-consensus processing in the banking/replaying stage.
4706                // This requires infallible and consistent account loading.
4707                // So, we unwrap Option<LoadedAccount> from get_loaded_account() here.
4708                // This is safe because this closure is invoked with the account_info,
4709                // while we lock the index entry at AccountsIndex::do_scan_accounts() ultimately,
4710                // meaning no other subsystems can invalidate the account_info before making their
4711                // changes to the index entry.
4712                // For details, see the comment in retry_to_get_account_accessor()
4713                if let Some(account_slot) = self
4714                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4715                    .get_loaded_account(|loaded_account| {
4716                        (pubkey, loaded_account.take_account(), slot)
4717                    })
4718                {
4719                    scan_func(Some(account_slot))
4720                }
4721            },
4722        );
4723    }
4724
4725    pub fn index_scan_accounts<F>(
4726        &self,
4727        ancestors: &Ancestors,
4728        bank_id: BankId,
4729        index_key: IndexKey,
4730        mut scan_func: F,
4731        config: &ScanConfig,
4732    ) -> ScanResult<bool>
4733    where
4734        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4735    {
4736        let key = match &index_key {
4737            IndexKey::ProgramId(key) => key,
4738            IndexKey::SplTokenMint(key) => key,
4739            IndexKey::SplTokenOwner(key) => key,
4740        };
4741        if !self.account_indexes.include_key(key) {
4742            // the requested key was not indexed in the secondary index, so do a normal scan
4743            let used_index = false;
4744            self.scan_accounts(ancestors, bank_id, scan_func, config)?;
4745            return Ok(used_index);
4746        }
4747
4748        self.accounts_index.index_scan_accounts(
4749            ancestors,
4750            bank_id,
4751            index_key,
4752            |pubkey, (account_info, slot)| {
4753                let account_slot = self
4754                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4755                    .get_loaded_account(|loaded_account| {
4756                        (pubkey, loaded_account.take_account(), slot)
4757                    });
4758                scan_func(account_slot)
4759            },
4760            config,
4761        )?;
4762        let used_index = true;
4763        Ok(used_index)
4764    }
4765
4766    /// Scan a specific slot through all the account storage
4767    pub(crate) fn scan_account_storage<R, B>(
4768        &self,
4769        slot: Slot,
4770        cache_map_func: impl Fn(&LoadedAccount) -> Option<R> + Sync,
4771        storage_scan_func: impl Fn(&B, &LoadedAccount, Option<&[u8]>) + Sync,
4772        scan_account_storage_data: ScanAccountStorageData,
4773    ) -> ScanStorageResult<R, B>
4774    where
4775        R: Send,
4776        B: Send + Default + Sync,
4777    {
4778        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
4779            // If we see the slot in the cache, then all the account information
4780            // is in this cached slot
4781            if slot_cache.len() > SCAN_SLOT_PAR_ITER_THRESHOLD {
4782                ScanStorageResult::Cached(self.thread_pool.install(|| {
4783                    slot_cache
4784                        .par_iter()
4785                        .filter_map(|cached_account| {
4786                            cache_map_func(&LoadedAccount::Cached(Cow::Borrowed(
4787                                cached_account.value(),
4788                            )))
4789                        })
4790                        .collect()
4791                }))
4792            } else {
4793                ScanStorageResult::Cached(
4794                    slot_cache
4795                        .iter()
4796                        .filter_map(|cached_account| {
4797                            cache_map_func(&LoadedAccount::Cached(Cow::Borrowed(
4798                                cached_account.value(),
4799                            )))
4800                        })
4801                        .collect(),
4802                )
4803            }
4804        } else {
4805            let retval = B::default();
4806            // If the slot is not in the cache, then all the account information must have
4807            // been flushed. This is guaranteed because we only remove the rooted slot from
4808            // the cache *after* we've finished flushing in `flush_slot_cache`.
4809            // Regarding `shrinking_in_progress_ok`:
4810            // This fn could be running in the foreground, so shrinking could be running in the background, independently.
4811            // Even if shrinking is running, there will be 0-1 active storages to scan here at any point.
4812            // When a concurrent shrink completes, the active storage at this slot will
4813            // be replaced with an equivalent storage with only alive accounts in it.
4814            // A shrink on this slot could have completed anytime before the call here, a shrink could currently be in progress,
4815            // or the shrink could complete immediately or anytime after this call. This has always been true.
4816            // So, whether we get a never-shrunk, an about-to-be shrunk, or a will-be-shrunk-in-future storage here to scan,
4817            // all are correct and possible in a normally running system.
4818            if let Some(storage) = self
4819                .storage
4820                .get_slot_storage_entry_shrinking_in_progress_ok(slot)
4821            {
4822                storage.accounts.scan_accounts(|account| {
4823                    let loaded_account = LoadedAccount::Stored(account);
4824                    let data = (scan_account_storage_data
4825                        == ScanAccountStorageData::DataRefForStorage)
4826                        .then_some(loaded_account.data());
4827                    storage_scan_func(&retval, &loaded_account, data)
4828                });
4829            }
4830
4831            ScanStorageResult::Stored(retval)
4832        }
4833    }
4834
4835    /// Insert a default bank hash stats for `slot`
4836    ///
4837    /// This fn is called when creating a new bank from parent.
4838    pub fn insert_default_bank_hash_stats(&self, slot: Slot, parent_slot: Slot) {
4839        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
4840        if bank_hash_stats.get(&slot).is_some() {
4841            error!(
4842                "set_hash: already exists; multiple forks with shared slot {slot} as child \
4843                 (parent: {parent_slot})!?"
4844            );
4845            return;
4846        }
4847        bank_hash_stats.insert(slot, BankHashStats::default());
4848    }
4849
4850    pub fn load(
4851        &self,
4852        ancestors: &Ancestors,
4853        pubkey: &Pubkey,
4854        load_hint: LoadHint,
4855    ) -> Option<(AccountSharedData, Slot)> {
4856        self.do_load(ancestors, pubkey, None, load_hint, LoadZeroLamports::None)
4857    }
4858
4859    /// Return Ok(index_of_matching_owner) if the account owner at `offset` is one of the pubkeys in `owners`.
4860    /// Return Err(MatchAccountOwnerError::NoMatch) if the account has 0 lamports or the owner is not one of
4861    /// the pubkeys in `owners`.
4862    /// Return Err(MatchAccountOwnerError::UnableToLoad) if the account could not be accessed.
4863    pub fn account_matches_owners(
4864        &self,
4865        ancestors: &Ancestors,
4866        account: &Pubkey,
4867        owners: &[Pubkey],
4868    ) -> Result<usize, MatchAccountOwnerError> {
4869        let (slot, storage_location, _maybe_account_accesor) = self
4870            .read_index_for_accessor_or_load_slow(ancestors, account, None, false)
4871            .ok_or(MatchAccountOwnerError::UnableToLoad)?;
4872
4873        if !storage_location.is_cached() {
4874            let result = self.read_only_accounts_cache.load(*account, slot);
4875            if let Some(account) = result {
4876                return if account.is_zero_lamport() {
4877                    Err(MatchAccountOwnerError::NoMatch)
4878                } else {
4879                    owners
4880                        .iter()
4881                        .position(|entry| account.owner() == entry)
4882                        .ok_or(MatchAccountOwnerError::NoMatch)
4883                };
4884            }
4885        }
4886
4887        let (account_accessor, _slot) = self
4888            .retry_to_get_account_accessor(
4889                slot,
4890                storage_location,
4891                ancestors,
4892                account,
4893                None,
4894                LoadHint::Unspecified,
4895            )
4896            .ok_or(MatchAccountOwnerError::UnableToLoad)?;
4897        account_accessor.account_matches_owners(owners)
4898    }
4899
4900    /// load the account with `pubkey` into the read only accounts cache.
4901    /// The goal is to make subsequent loads (which caller expects to occur) to find the account quickly.
4902    pub fn load_account_into_read_cache(&self, ancestors: &Ancestors, pubkey: &Pubkey) {
4903        self.do_load_with_populate_read_cache(
4904            ancestors,
4905            pubkey,
4906            None,
4907            LoadHint::Unspecified,
4908            true,
4909            // no return from this function, so irrelevant
4910            LoadZeroLamports::None,
4911        );
4912    }
4913
4914    /// note this returns None for accounts with zero lamports
4915    pub fn load_with_fixed_root(
4916        &self,
4917        ancestors: &Ancestors,
4918        pubkey: &Pubkey,
4919    ) -> Option<(AccountSharedData, Slot)> {
4920        self.load(ancestors, pubkey, LoadHint::FixedMaxRoot)
4921    }
4922
4923    fn read_index_for_accessor_or_load_slow<'a>(
4924        &'a self,
4925        ancestors: &Ancestors,
4926        pubkey: &'a Pubkey,
4927        max_root: Option<Slot>,
4928        clone_in_lock: bool,
4929    ) -> Option<(Slot, StorageLocation, Option<LoadedAccountAccessor<'a>>)> {
4930        self.accounts_index.get_with_and_then(
4931            pubkey,
4932            Some(ancestors),
4933            max_root,
4934            true,
4935            |(slot, account_info)| {
4936                let storage_location = account_info.storage_location();
4937                let account_accessor = clone_in_lock
4938                    .then(|| self.get_account_accessor(slot, pubkey, &storage_location));
4939                (slot, storage_location, account_accessor)
4940            },
4941        )
4942    }
4943
4944    fn retry_to_get_account_accessor<'a>(
4945        &'a self,
4946        mut slot: Slot,
4947        mut storage_location: StorageLocation,
4948        ancestors: &'a Ancestors,
4949        pubkey: &'a Pubkey,
4950        max_root: Option<Slot>,
4951        load_hint: LoadHint,
4952    ) -> Option<(LoadedAccountAccessor<'a>, Slot)> {
4953        // Happy drawing time! :)
4954        //
4955        // Reader                               | Accessed data source for cached/stored
4956        // -------------------------------------+----------------------------------
4957        // R1 read_index_for_accessor_or_load_slow()| cached/stored: index
4958        //          |                           |
4959        //        <(store_id, offset, ..)>      |
4960        //          V                           |
4961        // R2 retry_to_get_account_accessor()/  | cached: map of caches & entry for (slot, pubkey)
4962        //        get_account_accessor()        | stored: map of stores
4963        //          |                           |
4964        //        <Accessor>                    |
4965        //          V                           |
4966        // R3 check_and_get_loaded_account()/   | cached: N/A (note: basically noop unwrap)
4967        //        get_loaded_account()          | stored: store's entry for slot
4968        //          |                           |
4969        //        <LoadedAccount>               |
4970        //          V                           |
4971        // R4 take_account()                    | cached/stored: entry of cache/storage for (slot, pubkey)
4972        //          |                           |
4973        //        <AccountSharedData>           |
4974        //          V                           |
4975        //    Account!!                         V
4976        //
4977        // Flusher                              | Accessed data source for cached/stored
4978        // -------------------------------------+----------------------------------
4979        // F1 flush_slot_cache()                | N/A
4980        //          |                           |
4981        //          V                           |
4982        // F2 store_accounts_frozen()/          | map of stores (creates new entry)
4983        //        write_accounts_to_storage()   |
4984        //          |                           |
4985        //          V                           |
4986        // F3 store_accounts_frozen()/          | index
4987        //        update_index()                | (replaces existing store_id, offset in caches)
4988        //          |                           |
4989        //          V                           |
4990        // F4 accounts_cache.remove_slot()      | map of caches (removes old entry)
4991        //                                      V
4992        //
4993        // Remarks for flusher: So, for any reading operations, it's a race condition where F4 happens
4994        // between R1 and R2. In that case, retrying from R1 is safu because F3 should have
4995        // been occurred.
4996        //
4997        // Shrinker                             | Accessed data source for stored
4998        // -------------------------------------+----------------------------------
4999        // S1 do_shrink_slot_store()            | N/A
5000        //          |                           |
5001        //          V                           |
5002        // S2 store_accounts_frozen()/          | map of stores (creates new entry)
5003        //        write_accounts_to_storage()   |
5004        //          |                           |
5005        //          V                           |
5006        // S3 store_accounts_frozen()/          | index
5007        //        update_index()                | (replaces existing store_id, offset in stores)
5008        //          |                           |
5009        //          V                           |
5010        // S4 do_shrink_slot_store()/           | map of stores (removes old entry)
5011        //        dead_storages
5012        //
5013        // Remarks for shrinker: So, for any reading operations, it's a race condition
5014        // where S4 happens between R1 and R2. In that case, retrying from R1 is safu because S3 should have
5015        // been occurred, and S3 atomically replaced the index accordingly.
5016        //
5017        // Cleaner                              | Accessed data source for stored
5018        // -------------------------------------+----------------------------------
5019        // C1 clean_accounts()                  | N/A
5020        //          |                           |
5021        //          V                           |
5022        // C2 clean_accounts()/                 | index
5023        //        purge_keys_exact()            | (removes existing store_id, offset for stores)
5024        //          |                           |
5025        //          V                           |
5026        // C3 clean_accounts()/                 | map of stores (removes old entry)
5027        //        handle_reclaims()             |
5028        //
5029        // Remarks for cleaner: So, for any reading operations, it's a race condition
5030        // where C3 happens between R1 and R2. In that case, retrying from R1 is safu.
5031        // In that case, None would be returned while bailing out at R1.
5032        //
5033        // Purger                                 | Accessed data source for cached/stored
5034        // ---------------------------------------+----------------------------------
5035        // P1 purge_slot()                        | N/A
5036        //          |                             |
5037        //          V                             |
5038        // P2 purge_slots_from_cache_and_store()  | map of caches/stores (removes old entry)
5039        //          |                             |
5040        //          V                             |
5041        // P3 purge_slots_from_cache_and_store()/ | index
5042        //       purge_slot_cache()/              |
5043        //          purge_slot_cache_pubkeys()    | (removes existing store_id, offset for cache)
5044        //       purge_slot_storage()/            |
5045        //          purge_keys_exact()            | (removes accounts index entries)
5046        //          handle_reclaims()             | (removes storage entries)
5047        //      OR                                |
5048        //    clean_accounts()/                   |
5049        //        clean_accounts_older_than_root()| (removes existing store_id, offset for stores)
5050        //                                        V
5051        //
5052        // Remarks for purger: So, for any reading operations, it's a race condition
5053        // where P2 happens between R1 and R2. In that case, retrying from R1 is safu.
5054        // In that case, we may bail at index read retry when P3 hasn't been run
5055
5056        #[cfg(test)]
5057        {
5058            // Give some time for cache flushing to occur here for unit tests
5059            sleep(Duration::from_millis(self.load_delay));
5060        }
5061
5062        // Failsafe for potential race conditions with other subsystems
5063        let mut num_acceptable_failed_iterations = 0;
5064        loop {
5065            let account_accessor = self.get_account_accessor(slot, pubkey, &storage_location);
5066            match account_accessor {
5067                LoadedAccountAccessor::Cached(Some(_)) | LoadedAccountAccessor::Stored(Some(_)) => {
5068                    // Great! There was no race, just return :) This is the most usual situation
5069                    return Some((account_accessor, slot));
5070                }
5071                LoadedAccountAccessor::Cached(None) => {
5072                    num_acceptable_failed_iterations += 1;
5073                    // Cache was flushed in between checking the index and retrieving from the cache,
5074                    // so retry. This works because in accounts cache flush, an account is written to
5075                    // storage *before* it is removed from the cache
5076                    match load_hint {
5077                        LoadHint::FixedMaxRootDoNotPopulateReadCache | LoadHint::FixedMaxRoot => {
5078                            // it's impossible for this to fail for transaction loads from
5079                            // replaying/banking more than once.
5080                            // This is because:
5081                            // 1) For a slot `X` that's being replayed, there is only one
5082                            // latest ancestor containing the latest update for the account, and this
5083                            // ancestor can only be flushed once.
5084                            // 2) The root cannot move while replaying, so the index cannot continually
5085                            // find more up to date entries than the current `slot`
5086                            assert!(num_acceptable_failed_iterations <= 1);
5087                        }
5088                        LoadHint::Unspecified => {
5089                            // Because newer root can be added to the index (= not fixed),
5090                            // multiple flush race conditions can be observed under very rare
5091                            // condition, at least theoretically
5092                        }
5093                    }
5094                }
5095                LoadedAccountAccessor::Stored(None) => {
5096                    match load_hint {
5097                        LoadHint::FixedMaxRootDoNotPopulateReadCache | LoadHint::FixedMaxRoot => {
5098                            // When running replay on the validator, or banking stage on the leader,
5099                            // it should be very rare that the storage entry doesn't exist if the
5100                            // entry in the accounts index is the latest version of this account.
5101                            //
5102                            // There are only a few places where the storage entry may not exist
5103                            // after reading the index:
5104                            // 1) Shrink has removed the old storage entry and rewritten to
5105                            // a newer storage entry
5106                            // 2) The `pubkey` asked for in this function is a zero-lamport account,
5107                            // and the storage entry holding this account qualified for zero-lamport clean.
5108                            //
5109                            // In both these cases, it should be safe to retry and recheck the accounts
5110                            // index indefinitely, without incrementing num_acceptable_failed_iterations.
5111                            // That's because if the root is fixed, there should be a bounded number
5112                            // of pending cleans/shrinks (depends how far behind the AccountsBackgroundService
5113                            // is), termination to the desired condition is guaranteed.
5114                            //
5115                            // Also note that in both cases, if we do find the storage entry,
5116                            // we can guarantee that the storage entry is safe to read from because
5117                            // we grabbed a reference to the storage entry while it was still in the
5118                            // storage map. This means even if the storage entry is removed from the storage
5119                            // map after we grabbed the storage entry, the recycler should not reset the
5120                            // storage entry until we drop the reference to the storage entry.
5121                            //
5122                            // eh, no code in this arm? yes!
5123                        }
5124                        LoadHint::Unspecified => {
5125                            // RPC get_account() may have fetched an old root from the index that was
5126                            // either:
5127                            // 1) Cleaned up by clean_accounts(), so the accounts index has been updated
5128                            // and the storage entries have been removed.
5129                            // 2) Dropped by purge_slots() because the slot was on a minor fork, which
5130                            // removes the slots' storage entries but doesn't purge from the accounts index
5131                            // (account index cleanup is left to clean for stored slots). Note that
5132                            // this generally is impossible to occur in the wild because the RPC
5133                            // should hold the slot's bank, preventing it from being purged() to
5134                            // begin with.
5135                            num_acceptable_failed_iterations += 1;
5136                        }
5137                    }
5138                }
5139            }
5140            #[cfg(not(test))]
5141            let load_limit = ABSURD_CONSECUTIVE_FAILED_ITERATIONS;
5142
5143            #[cfg(test)]
5144            let load_limit = self.load_limit.load(Ordering::Relaxed);
5145
5146            let fallback_to_slow_path = if num_acceptable_failed_iterations >= load_limit {
5147                // The latest version of the account existed in the index, but could not be
5148                // fetched from storage. This means a race occurred between this function and clean
5149                // accounts/purge_slots
5150                let message = format!(
5151                    "do_load() failed to get key: {pubkey} from storage, latest attempt was for \
5152                     slot: {slot}, storage_location: {storage_location:?}, load_hint: \
5153                     {load_hint:?}",
5154                );
5155                datapoint_warn!("accounts_db-do_load_warn", ("warn", message, String));
5156                true
5157            } else {
5158                false
5159            };
5160
5161            // Because reading from the cache/storage failed, retry from the index read
5162            let (new_slot, new_storage_location, maybe_account_accessor) = self
5163                .read_index_for_accessor_or_load_slow(
5164                    ancestors,
5165                    pubkey,
5166                    max_root,
5167                    fallback_to_slow_path,
5168                )?;
5169            // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5170
5171            if new_slot == slot && new_storage_location.is_store_id_equal(&storage_location) {
5172                let message = format!(
5173                    "Bad index entry detected ({}, {}, {:?}, {:?}, {:?}, {:?})",
5174                    pubkey,
5175                    slot,
5176                    storage_location,
5177                    load_hint,
5178                    new_storage_location,
5179                    self.accounts_index.get_cloned(pubkey)
5180                );
5181                // Considering that we've failed to get accessor above and further that
5182                // the index still returned the same (slot, store_id) tuple, offset must be same
5183                // too.
5184                assert!(
5185                    new_storage_location.is_offset_equal(&storage_location),
5186                    "{message}"
5187                );
5188
5189                // If the entry was missing from the cache, that means it must have been flushed,
5190                // and the accounts index is always updated before cache flush, so store_id must
5191                // not indicate being cached at this point.
5192                assert!(!new_storage_location.is_cached(), "{message}");
5193
5194                // If this is not a cache entry, then this was a minor fork slot
5195                // that had its storage entries cleaned up by purge_slots() but hasn't been
5196                // cleaned yet. That means this must be rpc access and not replay/banking at the
5197                // very least. Note that purge shouldn't occur even for RPC as caller must hold all
5198                // of ancestor slots..
5199                assert_eq!(load_hint, LoadHint::Unspecified, "{message}");
5200
5201                // Everything being assert!()-ed, let's panic!() here as it's an error condition
5202                // after all....
5203                // That reasoning is based on the fact all of code-path reaching this fn
5204                // retry_to_get_account_accessor() must outlive the Arc<Bank> (and its all
5205                // ancestors) over this fn invocation, guaranteeing the prevention of being purged,
5206                // first of all.
5207                // For details, see the comment in AccountIndex::do_checked_scan_accounts(),
5208                // which is referring back here.
5209                panic!("{message}");
5210            } else if fallback_to_slow_path {
5211                // the above bad-index-entry check must had been checked first to retain the same
5212                // behavior
5213                return Some((
5214                    maybe_account_accessor.expect("must be some if clone_in_lock=true"),
5215                    new_slot,
5216                ));
5217            }
5218
5219            slot = new_slot;
5220            storage_location = new_storage_location;
5221        }
5222    }
5223
5224    fn do_load(
5225        &self,
5226        ancestors: &Ancestors,
5227        pubkey: &Pubkey,
5228        max_root: Option<Slot>,
5229        load_hint: LoadHint,
5230        load_zero_lamports: LoadZeroLamports,
5231    ) -> Option<(AccountSharedData, Slot)> {
5232        self.do_load_with_populate_read_cache(
5233            ancestors,
5234            pubkey,
5235            max_root,
5236            load_hint,
5237            false,
5238            load_zero_lamports,
5239        )
5240    }
5241
5242    /// Load account with `pubkey` and maybe put into read cache.
5243    ///
5244    /// If the account is not already cached, invoke `should_put_in_read_cache_fn`.
5245    /// The caller can inspect the account and indicate if it should be put into the read cache or not.
5246    ///
5247    /// Return the account and the slot when the account was last stored.
5248    /// Return None for ZeroLamport accounts.
5249    pub fn load_account_with(
5250        &self,
5251        ancestors: &Ancestors,
5252        pubkey: &Pubkey,
5253        should_put_in_read_cache_fn: impl Fn(&AccountSharedData) -> bool,
5254    ) -> Option<(AccountSharedData, Slot)> {
5255        let (slot, storage_location, _maybe_account_accesor) =
5256            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, None, false)?;
5257        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5258
5259        let in_write_cache = storage_location.is_cached();
5260        if !in_write_cache {
5261            let result = self.read_only_accounts_cache.load(*pubkey, slot);
5262            if let Some(account) = result {
5263                if account.is_zero_lamport() {
5264                    return None;
5265                }
5266                return Some((account, slot));
5267            }
5268        }
5269
5270        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
5271            slot,
5272            storage_location,
5273            ancestors,
5274            pubkey,
5275            None,
5276            LoadHint::Unspecified,
5277        )?;
5278
5279        // note that the account being in the cache could be different now than it was previously
5280        // since the cache could be flushed in between the 2 calls.
5281        let in_write_cache = matches!(account_accessor, LoadedAccountAccessor::Cached(_));
5282        let account = account_accessor.check_and_get_loaded_account_shared_data();
5283        if account.is_zero_lamport() {
5284            return None;
5285        }
5286
5287        if !in_write_cache && should_put_in_read_cache_fn(&account) {
5288            /*
5289            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
5290            safe/reflect 'A''s latest state on this fork.
5291            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
5292            not the read-only cache, after it's been updated in replay of slot 'S'.
5293            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
5294            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
5295            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
5296            which means '(S', A)' does not exist in the write cache yet.
5297            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
5298            must exist in the write cache, which is a contradiction.
5299            */
5300            self.read_only_accounts_cache
5301                .store(*pubkey, slot, account.clone());
5302        }
5303        Some((account, slot))
5304    }
5305
5306    /// if 'load_into_read_cache_only', then return value is meaningless.
5307    ///   The goal is to get the account into the read-only cache.
5308    fn do_load_with_populate_read_cache(
5309        &self,
5310        ancestors: &Ancestors,
5311        pubkey: &Pubkey,
5312        max_root: Option<Slot>,
5313        load_hint: LoadHint,
5314        load_into_read_cache_only: bool,
5315        load_zero_lamports: LoadZeroLamports,
5316    ) -> Option<(AccountSharedData, Slot)> {
5317        #[cfg(not(test))]
5318        assert!(max_root.is_none());
5319
5320        let (slot, storage_location, _maybe_account_accesor) =
5321            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
5322        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5323
5324        let in_write_cache = storage_location.is_cached();
5325        if !load_into_read_cache_only {
5326            if !in_write_cache {
5327                let result = self.read_only_accounts_cache.load(*pubkey, slot);
5328                if let Some(account) = result {
5329                    if matches!(load_zero_lamports, LoadZeroLamports::None)
5330                        && account.is_zero_lamport()
5331                    {
5332                        return None;
5333                    }
5334                    return Some((account, slot));
5335                }
5336            }
5337        } else {
5338            // goal is to load into read cache
5339            if in_write_cache {
5340                // no reason to load in read cache. already in write cache
5341                return None;
5342            }
5343            if self.read_only_accounts_cache.in_cache(pubkey, slot) {
5344                // already in read cache
5345                return None;
5346            }
5347        }
5348
5349        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
5350            slot,
5351            storage_location,
5352            ancestors,
5353            pubkey,
5354            max_root,
5355            load_hint,
5356        )?;
5357        // note that the account being in the cache could be different now than it was previously
5358        // since the cache could be flushed in between the 2 calls.
5359        let in_write_cache = matches!(account_accessor, LoadedAccountAccessor::Cached(_));
5360        let account = account_accessor.check_and_get_loaded_account_shared_data();
5361        if matches!(load_zero_lamports, LoadZeroLamports::None) && account.is_zero_lamport() {
5362            return None;
5363        }
5364
5365        if !in_write_cache && load_hint != LoadHint::FixedMaxRootDoNotPopulateReadCache {
5366            /*
5367            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
5368            safe/reflect 'A''s latest state on this fork.
5369            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
5370            not the read-only cache, after it's been updated in replay of slot 'S'.
5371            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
5372            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
5373            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
5374            which means '(S', A)' does not exist in the write cache yet.
5375            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
5376            must exist in the write cache, which is a contradiction.
5377            */
5378            self.read_only_accounts_cache
5379                .store(*pubkey, slot, account.clone());
5380        }
5381        Some((account, slot))
5382    }
5383
5384    pub fn load_account_hash(
5385        &self,
5386        ancestors: &Ancestors,
5387        pubkey: &Pubkey,
5388        max_root: Option<Slot>,
5389        load_hint: LoadHint,
5390    ) -> Option<AccountHash> {
5391        let (slot, storage_location, _maybe_account_accesor) =
5392            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
5393        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5394
5395        let (mut account_accessor, _) = self.retry_to_get_account_accessor(
5396            slot,
5397            storage_location,
5398            ancestors,
5399            pubkey,
5400            max_root,
5401            load_hint,
5402        )?;
5403        account_accessor
5404            .check_and_get_loaded_account(|loaded_account| Some(loaded_account.loaded_hash()))
5405    }
5406
5407    fn get_account_accessor<'a>(
5408        &'a self,
5409        slot: Slot,
5410        pubkey: &'a Pubkey,
5411        storage_location: &StorageLocation,
5412    ) -> LoadedAccountAccessor<'a> {
5413        match storage_location {
5414            StorageLocation::Cached => {
5415                let maybe_cached_account = self.accounts_cache.load(slot, pubkey).map(Cow::Owned);
5416                LoadedAccountAccessor::Cached(maybe_cached_account)
5417            }
5418            StorageLocation::AppendVec(store_id, offset) => {
5419                let maybe_storage_entry = self
5420                    .storage
5421                    .get_account_storage_entry(slot, *store_id)
5422                    .map(|account_storage_entry| (account_storage_entry, *offset));
5423                LoadedAccountAccessor::Stored(maybe_storage_entry)
5424            }
5425        }
5426    }
5427
5428    fn find_storage_candidate(&self, slot: Slot) -> Arc<AccountStorageEntry> {
5429        let mut get_slot_stores = Measure::start("get_slot_stores");
5430        let store = self.storage.get_slot_storage_entry(slot);
5431        get_slot_stores.stop();
5432        self.stats
5433            .store_get_slot_store
5434            .fetch_add(get_slot_stores.as_us(), Ordering::Relaxed);
5435        let mut find_existing = Measure::start("find_existing");
5436        if let Some(store) = store {
5437            if store.try_available() {
5438                let ret = store.clone();
5439                drop(store);
5440                find_existing.stop();
5441                self.stats
5442                    .store_find_existing
5443                    .fetch_add(find_existing.as_us(), Ordering::Relaxed);
5444                return ret;
5445            }
5446        }
5447        find_existing.stop();
5448        self.stats
5449            .store_find_existing
5450            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
5451
5452        let store = self.create_store(slot, self.file_size, "store", &self.paths);
5453
5454        // try_available is like taking a lock on the store,
5455        // preventing other threads from using it.
5456        // It must succeed here and happen before insert,
5457        // otherwise another thread could also grab it from the index.
5458        assert!(store.try_available());
5459        self.insert_store(slot, store.clone());
5460        store
5461    }
5462
5463    fn has_space_available(&self, slot: Slot, size: u64) -> bool {
5464        let store = self.storage.get_slot_storage_entry(slot).unwrap();
5465        if store.status() == AccountStorageStatus::Available
5466            && store.accounts.remaining_bytes() >= size
5467        {
5468            return true;
5469        }
5470        false
5471    }
5472
5473    fn create_store(
5474        &self,
5475        slot: Slot,
5476        size: u64,
5477        from: &str,
5478        paths: &[PathBuf],
5479    ) -> Arc<AccountStorageEntry> {
5480        self.stats
5481            .create_store_count
5482            .fetch_add(1, Ordering::Relaxed);
5483        let path_index = thread_rng().gen_range(0..paths.len());
5484        let store = Arc::new(self.new_storage_entry(slot, Path::new(&paths[path_index]), size));
5485
5486        debug!(
5487            "creating store: {} slot: {} len: {} size: {} from: {} path: {}",
5488            store.id(),
5489            slot,
5490            store.accounts.len(),
5491            store.accounts.capacity(),
5492            from,
5493            store.accounts.path().display(),
5494        );
5495
5496        store
5497    }
5498
5499    fn create_and_insert_store(
5500        &self,
5501        slot: Slot,
5502        size: u64,
5503        from: &str,
5504    ) -> Arc<AccountStorageEntry> {
5505        self.create_and_insert_store_with_paths(slot, size, from, &self.paths)
5506    }
5507
5508    fn create_and_insert_store_with_paths(
5509        &self,
5510        slot: Slot,
5511        size: u64,
5512        from: &str,
5513        paths: &[PathBuf],
5514    ) -> Arc<AccountStorageEntry> {
5515        let store = self.create_store(slot, size, from, paths);
5516        let store_for_index = store.clone();
5517
5518        self.insert_store(slot, store_for_index);
5519        store
5520    }
5521
5522    fn insert_store(&self, slot: Slot, store: Arc<AccountStorageEntry>) {
5523        self.storage.insert(slot, store)
5524    }
5525
5526    pub fn enable_bank_drop_callback(&self) {
5527        self.is_bank_drop_callback_enabled
5528            .store(true, Ordering::Release);
5529    }
5530
5531    /// This should only be called after the `Bank::drop()` runs in bank.rs, See BANK_DROP_SAFETY
5532    /// comment below for more explanation.
5533    ///   * `is_serialized_with_abs` - indicates whether this call runs sequentially with all other
5534    ///        accounts_db relevant calls, such as shrinking, purging etc., in account background
5535    ///        service.
5536    pub fn purge_slot(&self, slot: Slot, bank_id: BankId, is_serialized_with_abs: bool) {
5537        if self.is_bank_drop_callback_enabled.load(Ordering::Acquire) && !is_serialized_with_abs {
5538            panic!(
5539                "bad drop callpath detected; Bank::drop() must run serially with other logic in
5540                ABS like clean_accounts()"
5541            )
5542        }
5543
5544        // BANK_DROP_SAFETY: Because this function only runs once the bank is dropped,
5545        // we know that there are no longer any ongoing scans on this bank, because scans require
5546        // and hold a reference to the bank at the tip of the fork they're scanning. Hence it's
5547        // safe to remove this bank_id from the `removed_bank_ids` list at this point.
5548        if self
5549            .accounts_index
5550            .removed_bank_ids
5551            .lock()
5552            .unwrap()
5553            .remove(&bank_id)
5554        {
5555            // If this slot was already cleaned up, no need to do any further cleans
5556            return;
5557        }
5558
5559        self.purge_slots(std::iter::once(&slot));
5560    }
5561
5562    /// Purges every slot in `removed_slots` from both the cache and storage. This includes
5563    /// entries in the accounts index, cache entries, and any backing storage entries.
5564    pub fn purge_slots_from_cache_and_store<'a>(
5565        &self,
5566        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5567        purge_stats: &PurgeStats,
5568        log_accounts: bool,
5569    ) {
5570        let mut remove_cache_elapsed_across_slots = 0;
5571        let mut num_cached_slots_removed = 0;
5572        let mut total_removed_cached_bytes = 0;
5573        if log_accounts {
5574            if let Some(min) = removed_slots.clone().min() {
5575                info!(
5576                    "purge_slots_from_cache_and_store: {:?}",
5577                    self.get_pubkey_hash_for_slot(*min).0
5578                );
5579            }
5580        }
5581        for remove_slot in removed_slots {
5582            // This function is only currently safe with respect to `flush_slot_cache()` because
5583            // both functions run serially in AccountsBackgroundService.
5584            let mut remove_cache_elapsed = Measure::start("remove_cache_elapsed");
5585            // Note: we cannot remove this slot from the slot cache until we've removed its
5586            // entries from the accounts index first. This is because `scan_accounts()` relies on
5587            // holding the index lock, finding the index entry, and then looking up the entry
5588            // in the cache. If it fails to find that entry, it will panic in `get_loaded_account()`
5589            if let Some(slot_cache) = self.accounts_cache.slot_cache(*remove_slot) {
5590                // If the slot is still in the cache, remove the backing storages for
5591                // the slot and from the Accounts Index
5592                num_cached_slots_removed += 1;
5593                total_removed_cached_bytes += slot_cache.total_bytes();
5594                self.purge_slot_cache(*remove_slot, slot_cache);
5595                remove_cache_elapsed.stop();
5596                remove_cache_elapsed_across_slots += remove_cache_elapsed.as_us();
5597                // Nobody else should have removed the slot cache entry yet
5598                assert!(self.accounts_cache.remove_slot(*remove_slot).is_some());
5599            } else {
5600                self.purge_slot_storage(*remove_slot, purge_stats);
5601            }
5602            // It should not be possible that a slot is neither in the cache or storage. Even in
5603            // a slot with all ticks, `Bank::new_from_parent()` immediately stores some sysvars
5604            // on bank creation.
5605        }
5606
5607        purge_stats
5608            .remove_cache_elapsed
5609            .fetch_add(remove_cache_elapsed_across_slots, Ordering::Relaxed);
5610        purge_stats
5611            .num_cached_slots_removed
5612            .fetch_add(num_cached_slots_removed, Ordering::Relaxed);
5613        purge_stats
5614            .total_removed_cached_bytes
5615            .fetch_add(total_removed_cached_bytes, Ordering::Relaxed);
5616    }
5617
5618    /// Purge the backing storage entries for the given slot, does not purge from
5619    /// the cache!
5620    fn purge_dead_slots_from_storage<'a>(
5621        &'a self,
5622        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5623        purge_stats: &PurgeStats,
5624    ) {
5625        // Check all slots `removed_slots` are no longer "relevant" roots.
5626        // Note that the slots here could have been rooted slots, but if they're passed here
5627        // for removal it means:
5628        // 1) All updates in that old root have been outdated by updates in newer roots
5629        // 2) Those slots/roots should have already been purged from the accounts index root
5630        // tracking metadata via `accounts_index.clean_dead_slot()`.
5631        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5632        assert!(self
5633            .accounts_index
5634            .get_rooted_from_list(removed_slots.clone())
5635            .is_empty());
5636        safety_checks_elapsed.stop();
5637        purge_stats
5638            .safety_checks_elapsed
5639            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5640
5641        let mut total_removed_stored_bytes = 0;
5642        let mut all_removed_slot_storages = vec![];
5643
5644        let mut remove_storage_entries_elapsed = Measure::start("remove_storage_entries_elapsed");
5645        for remove_slot in removed_slots {
5646            // Remove the storage entries and collect some metrics
5647            if let Some(store) = self.storage.remove(remove_slot, false) {
5648                total_removed_stored_bytes += store.accounts.capacity();
5649                all_removed_slot_storages.push(store);
5650            }
5651        }
5652        remove_storage_entries_elapsed.stop();
5653        let num_stored_slots_removed = all_removed_slot_storages.len();
5654
5655        // Backing mmaps for removed storages entries explicitly dropped here outside
5656        // of any locks
5657        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
5658        drop(all_removed_slot_storages);
5659        drop_storage_entries_elapsed.stop();
5660
5661        purge_stats
5662            .remove_storage_entries_elapsed
5663            .fetch_add(remove_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5664        purge_stats
5665            .drop_storage_entries_elapsed
5666            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5667        purge_stats
5668            .num_stored_slots_removed
5669            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
5670        purge_stats
5671            .total_removed_storage_entries
5672            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
5673        purge_stats
5674            .total_removed_stored_bytes
5675            .fetch_add(total_removed_stored_bytes, Ordering::Relaxed);
5676        self.stats
5677            .dropped_stores
5678            .fetch_add(num_stored_slots_removed as u64, Ordering::Relaxed);
5679    }
5680
5681    fn purge_slot_cache(&self, purged_slot: Slot, slot_cache: SlotCache) {
5682        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
5683        let pubkey_to_slot_set: Vec<(Pubkey, Slot)> = slot_cache
5684            .iter()
5685            .map(|account| {
5686                purged_slot_pubkeys.insert((purged_slot, *account.key()));
5687                (*account.key(), purged_slot)
5688            })
5689            .collect();
5690        self.purge_slot_cache_pubkeys(
5691            purged_slot,
5692            purged_slot_pubkeys,
5693            pubkey_to_slot_set,
5694            true,
5695            &HashSet::default(),
5696        );
5697    }
5698
5699    fn purge_slot_cache_pubkeys(
5700        &self,
5701        purged_slot: Slot,
5702        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
5703        pubkey_to_slot_set: Vec<(Pubkey, Slot)>,
5704        is_dead: bool,
5705        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
5706    ) {
5707        // Slot purged from cache should not exist in the backing store
5708        assert!(self
5709            .storage
5710            .get_slot_storage_entry_shrinking_in_progress_ok(purged_slot)
5711            .is_none());
5712        let num_purged_keys = pubkey_to_slot_set.len();
5713        let (reclaims, _) = self.purge_keys_exact(pubkey_to_slot_set.iter());
5714        assert_eq!(reclaims.len(), num_purged_keys);
5715        if is_dead {
5716            self.remove_dead_slots_metadata(
5717                std::iter::once(&purged_slot),
5718                purged_slot_pubkeys,
5719                None,
5720                pubkeys_removed_from_accounts_index,
5721            );
5722        }
5723    }
5724
5725    fn purge_slot_storage(&self, remove_slot: Slot, purge_stats: &PurgeStats) {
5726        // Because AccountsBackgroundService synchronously flushes from the accounts cache
5727        // and handles all Bank::drop() (the cleanup function that leads to this
5728        // function call), then we don't need to worry above an overlapping cache flush
5729        // with this function call. This means, if we get into this case, we can be
5730        // confident that the entire state for this slot has been flushed to the storage
5731        // already.
5732        let mut scan_storages_elapsed = Measure::start("scan_storages_elapsed");
5733        let mut stored_keys = HashSet::new();
5734        if let Some(storage) = self
5735            .storage
5736            .get_slot_storage_entry_shrinking_in_progress_ok(remove_slot)
5737        {
5738            storage.accounts.scan_pubkeys(|pk| {
5739                stored_keys.insert((*pk, remove_slot));
5740            });
5741        }
5742        scan_storages_elapsed.stop();
5743        purge_stats
5744            .scan_storages_elapsed
5745            .fetch_add(scan_storages_elapsed.as_us(), Ordering::Relaxed);
5746
5747        let mut purge_accounts_index_elapsed = Measure::start("purge_accounts_index_elapsed");
5748        // Purge this slot from the accounts index
5749        let (reclaims, pubkeys_removed_from_accounts_index) =
5750            self.purge_keys_exact(stored_keys.iter());
5751        purge_accounts_index_elapsed.stop();
5752        purge_stats
5753            .purge_accounts_index_elapsed
5754            .fetch_add(purge_accounts_index_elapsed.as_us(), Ordering::Relaxed);
5755
5756        // `handle_reclaims()` should remove all the account index entries and
5757        // storage entries
5758        let mut handle_reclaims_elapsed = Measure::start("handle_reclaims_elapsed");
5759        // Slot should be dead after removing all its account entries
5760        let expected_dead_slot = Some(remove_slot);
5761        self.handle_reclaims(
5762            (!reclaims.is_empty()).then(|| reclaims.iter()),
5763            expected_dead_slot,
5764            false,
5765            &pubkeys_removed_from_accounts_index,
5766            HandleReclaims::ProcessDeadSlots(purge_stats),
5767        );
5768        handle_reclaims_elapsed.stop();
5769        purge_stats
5770            .handle_reclaims_elapsed
5771            .fetch_add(handle_reclaims_elapsed.as_us(), Ordering::Relaxed);
5772        // After handling the reclaimed entries, this slot's
5773        // storage entries should be purged from self.storage
5774        assert!(
5775            self.storage.get_slot_storage_entry(remove_slot).is_none(),
5776            "slot {remove_slot} is not none"
5777        );
5778    }
5779
5780    fn purge_slots<'a>(&self, slots: impl Iterator<Item = &'a Slot> + Clone) {
5781        // `add_root()` should be called first
5782        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5783        let non_roots = slots
5784            // Only safe to check when there are duplicate versions of a slot
5785            // because ReplayStage will not make new roots before dumping the
5786            // duplicate slots first. Thus we will not be in a case where we
5787            // root slot `S`, then try to dump some other version of slot `S`, the
5788            // dumping has to finish first
5789            //
5790            // Also note roots are never removed via `remove_unrooted_slot()`, so
5791            // it's safe to filter them out here as they won't need deletion from
5792            // self.accounts_index.removed_bank_ids in `purge_slots_from_cache_and_store()`.
5793            .filter(|slot| !self.accounts_index.is_alive_root(**slot));
5794        safety_checks_elapsed.stop();
5795        self.external_purge_slots_stats
5796            .safety_checks_elapsed
5797            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5798        self.purge_slots_from_cache_and_store(non_roots, &self.external_purge_slots_stats, false);
5799        self.external_purge_slots_stats
5800            .report("external_purge_slots_stats", Some(1000));
5801    }
5802
5803    pub fn remove_unrooted_slots(&self, remove_slots: &[(Slot, BankId)]) {
5804        let rooted_slots = self
5805            .accounts_index
5806            .get_rooted_from_list(remove_slots.iter().map(|(slot, _)| slot));
5807        assert!(
5808            rooted_slots.is_empty(),
5809            "Trying to remove accounts for rooted slots {rooted_slots:?}"
5810        );
5811
5812        let RemoveUnrootedSlotsSynchronization {
5813            slots_under_contention,
5814            signal,
5815        } = &self.remove_unrooted_slots_synchronization;
5816
5817        {
5818            // Slots that are currently being flushed by flush_slot_cache()
5819
5820            let mut currently_contended_slots = slots_under_contention.lock().unwrap();
5821
5822            // Slots that are currently being flushed by flush_slot_cache() AND
5823            // we want to remove in this function
5824            let mut remaining_contended_flush_slots: Vec<Slot> = remove_slots
5825                .iter()
5826                .filter_map(|(remove_slot, _)| {
5827                    // Reserve the slots that we want to purge that aren't currently
5828                    // being flushed to prevent cache from flushing those slots in
5829                    // the future.
5830                    //
5831                    // Note that the single replay thread has to remove a specific slot `N`
5832                    // before another version of the same slot can be replayed. This means
5833                    // multiple threads should not call `remove_unrooted_slots()` simultaneously
5834                    // with the same slot.
5835                    let is_being_flushed = !currently_contended_slots.insert(*remove_slot);
5836                    // If the cache is currently flushing this slot, add it to the list
5837                    is_being_flushed.then_some(remove_slot)
5838                })
5839                .cloned()
5840                .collect();
5841
5842            // Wait for cache flushes to finish
5843            loop {
5844                if !remaining_contended_flush_slots.is_empty() {
5845                    // Wait for the signal that the cache has finished flushing a slot
5846                    //
5847                    // Don't wait if the remaining_contended_flush_slots is empty, otherwise
5848                    // we may never get a signal since there's no cache flush thread to
5849                    // do the signaling
5850                    currently_contended_slots = signal.wait(currently_contended_slots).unwrap();
5851                } else {
5852                    // There are no slots being flushed to wait on, so it's safe to continue
5853                    // to purging the slots we want to purge!
5854                    break;
5855                }
5856
5857                // For each slot the cache flush has finished, mark that we're about to start
5858                // purging these slots by reserving it in `currently_contended_slots`.
5859                remaining_contended_flush_slots.retain(|flush_slot| {
5860                    // returns true if slot was already in set. This means slot is being flushed
5861                    !currently_contended_slots.insert(*flush_slot)
5862                });
5863            }
5864        }
5865
5866        // Mark down these slots are about to be purged so that new attempts to scan these
5867        // banks fail, and any ongoing scans over these slots will detect that they should abort
5868        // their results
5869        {
5870            let mut locked_removed_bank_ids = self.accounts_index.removed_bank_ids.lock().unwrap();
5871            for (_slot, remove_bank_id) in remove_slots.iter() {
5872                locked_removed_bank_ids.insert(*remove_bank_id);
5873            }
5874        }
5875
5876        let remove_unrooted_purge_stats = PurgeStats::default();
5877        self.purge_slots_from_cache_and_store(
5878            remove_slots.iter().map(|(slot, _)| slot),
5879            &remove_unrooted_purge_stats,
5880            true,
5881        );
5882        remove_unrooted_purge_stats.report("remove_unrooted_slots_purge_slots_stats", None);
5883
5884        let mut currently_contended_slots = slots_under_contention.lock().unwrap();
5885        for (remove_slot, _) in remove_slots {
5886            assert!(currently_contended_slots.remove(remove_slot));
5887        }
5888    }
5889
5890    /// Calculates the `AccountLtHash` of `account`
5891    pub fn lt_hash_account(account: &impl ReadableAccount, pubkey: &Pubkey) -> AccountLtHash {
5892        if account.lamports() == 0 {
5893            return ZERO_LAMPORT_ACCOUNT_LT_HASH;
5894        }
5895
5896        let hasher = Self::hash_account_helper(account, pubkey);
5897        let lt_hash = LtHash::with(&hasher);
5898        AccountLtHash(lt_hash)
5899    }
5900
5901    /// Calculates the `AccountHash` of `account`
5902    pub fn hash_account<T: ReadableAccount>(account: &T, pubkey: &Pubkey) -> AccountHash {
5903        if account.lamports() == 0 {
5904            return ZERO_LAMPORT_ACCOUNT_HASH;
5905        }
5906
5907        let hasher = Self::hash_account_helper(account, pubkey);
5908        let hash = Hash::new_from_array(hasher.finalize().into());
5909        AccountHash(hash)
5910    }
5911
5912    /// Hashes `account` and returns the underlying Hasher
5913    fn hash_account_helper(account: &impl ReadableAccount, pubkey: &Pubkey) -> blake3::Hasher {
5914        let mut hasher = blake3::Hasher::new();
5915
5916        // allocate a buffer on the stack that's big enough
5917        // to hold a token account or a stake account
5918        const META_SIZE: usize = 8 /* lamports */ + 8 /* rent_epoch */ + 1 /* executable */ + 32 /* owner */ + 32 /* pubkey */;
5919        const DATA_SIZE: usize = 200; // stake accounts are 200 B and token accounts are 165-182ish B
5920        const BUFFER_SIZE: usize = META_SIZE + DATA_SIZE;
5921        let mut buffer = SmallVec::<[u8; BUFFER_SIZE]>::new();
5922
5923        // collect lamports, rent_epoch into buffer to hash
5924        buffer.extend_from_slice(&account.lamports().to_le_bytes());
5925        buffer.extend_from_slice(&account.rent_epoch().to_le_bytes());
5926
5927        let data = account.data();
5928        if data.len() > DATA_SIZE {
5929            // For larger accounts whose data can't fit into the buffer, update the hash now.
5930            hasher.update(&buffer);
5931            buffer.clear();
5932
5933            // hash account's data
5934            hasher.update(data);
5935        } else {
5936            // For small accounts whose data can fit into the buffer, append it to the buffer.
5937            buffer.extend_from_slice(data);
5938        }
5939
5940        // collect exec_flag, owner, pubkey into buffer to hash
5941        buffer.push(account.executable().into());
5942        buffer.extend_from_slice(account.owner().as_ref());
5943        buffer.extend_from_slice(pubkey.as_ref());
5944        hasher.update(&buffer);
5945
5946        hasher
5947    }
5948
5949    fn write_accounts_to_storage<'a>(
5950        &self,
5951        slot: Slot,
5952        storage: &AccountStorageEntry,
5953        accounts_and_meta_to_store: &impl StorableAccounts<'a>,
5954    ) -> Vec<AccountInfo> {
5955        let mut infos: Vec<AccountInfo> = Vec::with_capacity(accounts_and_meta_to_store.len());
5956        let mut total_append_accounts_us = 0;
5957        while infos.len() < accounts_and_meta_to_store.len() {
5958            let mut append_accounts = Measure::start("append_accounts");
5959            let stored_accounts_info = storage
5960                .accounts
5961                .append_accounts(accounts_and_meta_to_store, infos.len());
5962            append_accounts.stop();
5963            total_append_accounts_us += append_accounts.as_us();
5964            let Some(stored_accounts_info) = stored_accounts_info else {
5965                storage.set_status(AccountStorageStatus::Full);
5966
5967                // See if an account overflows the append vecs in the slot.
5968                accounts_and_meta_to_store.account_default_if_zero_lamport(
5969                    infos.len(),
5970                    |account| {
5971                        let data_len = account.data().len();
5972                        let data_len = (data_len + STORE_META_OVERHEAD) as u64;
5973                        if !self.has_space_available(slot, data_len) {
5974                            info!(
5975                                "write_accounts_to_storage, no space: {}, {}, {}, {}, {}",
5976                                storage.accounts.capacity(),
5977                                storage.accounts.remaining_bytes(),
5978                                data_len,
5979                                infos.len(),
5980                                accounts_and_meta_to_store.len()
5981                            );
5982                            let special_store_size = std::cmp::max(data_len * 2, self.file_size);
5983                            self.create_and_insert_store(slot, special_store_size, "large create");
5984                        }
5985                    },
5986                );
5987                continue;
5988            };
5989
5990            let store_id = storage.id();
5991            for (i, offset) in stored_accounts_info.offsets.iter().enumerate() {
5992                infos.push(AccountInfo::new(
5993                    StorageLocation::AppendVec(store_id, *offset),
5994                    accounts_and_meta_to_store
5995                        .account_default_if_zero_lamport(i, |account| account.lamports()),
5996                ));
5997            }
5998            storage.add_accounts(
5999                stored_accounts_info.offsets.len(),
6000                stored_accounts_info.size,
6001            );
6002
6003            // restore the state to available
6004            storage.set_status(AccountStorageStatus::Available);
6005        }
6006
6007        self.stats
6008            .store_append_accounts
6009            .fetch_add(total_append_accounts_us, Ordering::Relaxed);
6010
6011        infos
6012    }
6013
6014    pub fn mark_slot_frozen(&self, slot: Slot) {
6015        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
6016            slot_cache.mark_slot_frozen();
6017            slot_cache.report_slot_store_metrics();
6018        }
6019        self.accounts_cache.report_size();
6020    }
6021
6022    // These functions/fields are only usable from a dev context (i.e. tests and benches)
6023    #[cfg(feature = "dev-context-only-utils")]
6024    pub fn flush_accounts_cache_slot_for_tests(&self, slot: Slot) {
6025        self.flush_slot_cache(slot);
6026    }
6027
6028    /// true if write cache is too big
6029    fn should_aggressively_flush_cache(&self) -> bool {
6030        self.write_cache_limit_bytes
6031            .unwrap_or(WRITE_CACHE_LIMIT_BYTES_DEFAULT)
6032            < self.accounts_cache.size()
6033    }
6034
6035    // `force_flush` flushes all the cached roots `<= requested_flush_root`. It also then
6036    // flushes:
6037    // 1) excess remaining roots or unrooted slots while 'should_aggressively_flush_cache' is true
6038    pub fn flush_accounts_cache(&self, force_flush: bool, requested_flush_root: Option<Slot>) {
6039        #[cfg(not(test))]
6040        assert!(requested_flush_root.is_some());
6041
6042        if !force_flush && !self.should_aggressively_flush_cache() {
6043            return;
6044        }
6045
6046        // Flush only the roots <= requested_flush_root, so that snapshotting has all
6047        // the relevant roots in storage.
6048        let mut flush_roots_elapsed = Measure::start("flush_roots_elapsed");
6049        let mut account_bytes_saved = 0;
6050        let mut num_accounts_saved = 0;
6051
6052        let _guard = self.active_stats.activate(ActiveStatItem::Flush);
6053
6054        // Note even if force_flush is false, we will still flush all roots <= the
6055        // given `requested_flush_root`, even if some of the later roots cannot be used for
6056        // cleaning due to an ongoing scan
6057        let (total_new_cleaned_roots, num_cleaned_roots_flushed, mut flush_stats) = self
6058            .flush_rooted_accounts_cache(
6059                requested_flush_root,
6060                Some((&mut account_bytes_saved, &mut num_accounts_saved)),
6061            );
6062        flush_roots_elapsed.stop();
6063
6064        // Note we don't purge unrooted slots here because there may be ongoing scans/references
6065        // for those slot, let the Bank::drop() implementation do cleanup instead on dead
6066        // banks
6067
6068        // If 'should_aggressively_flush_cache', then flush the excess ones to storage
6069        let (total_new_excess_roots, num_excess_roots_flushed, flush_stats_aggressively) =
6070            if self.should_aggressively_flush_cache() {
6071                // Start by flushing the roots
6072                //
6073                // Cannot do any cleaning on roots past `requested_flush_root` because future
6074                // snapshots may need updates from those later slots, hence we pass `None`
6075                // for `should_clean`.
6076                self.flush_rooted_accounts_cache(None, None)
6077            } else {
6078                (0, 0, FlushStats::default())
6079            };
6080        flush_stats.accumulate(&flush_stats_aggressively);
6081
6082        let mut excess_slot_count = 0;
6083        let mut unflushable_unrooted_slot_count = 0;
6084        let max_flushed_root = self.accounts_cache.fetch_max_flush_root();
6085        if self.should_aggressively_flush_cache() {
6086            let old_slots = self.accounts_cache.cached_frozen_slots();
6087            excess_slot_count = old_slots.len();
6088            let mut flush_stats = FlushStats::default();
6089            old_slots.into_iter().for_each(|old_slot| {
6090                // Don't flush slots that are known to be unrooted
6091                if old_slot > max_flushed_root {
6092                    if self.should_aggressively_flush_cache() {
6093                        if let Some(stats) = self.flush_slot_cache(old_slot) {
6094                            flush_stats.accumulate(&stats);
6095                        }
6096                    }
6097                } else {
6098                    unflushable_unrooted_slot_count += 1;
6099                }
6100            });
6101            datapoint_info!(
6102                "accounts_db-flush_accounts_cache_aggressively",
6103                ("num_flushed", flush_stats.num_flushed.0, i64),
6104                ("num_purged", flush_stats.num_purged.0, i64),
6105                ("total_flush_size", flush_stats.total_size.0, i64),
6106                ("total_cache_size", self.accounts_cache.size(), i64),
6107                ("total_frozen_slots", excess_slot_count, i64),
6108                ("total_slots", self.accounts_cache.num_slots(), i64),
6109            );
6110        }
6111
6112        datapoint_info!(
6113            "accounts_db-flush_accounts_cache",
6114            ("total_new_cleaned_roots", total_new_cleaned_roots, i64),
6115            ("num_cleaned_roots_flushed", num_cleaned_roots_flushed, i64),
6116            ("total_new_excess_roots", total_new_excess_roots, i64),
6117            ("num_excess_roots_flushed", num_excess_roots_flushed, i64),
6118            ("excess_slot_count", excess_slot_count, i64),
6119            (
6120                "unflushable_unrooted_slot_count",
6121                unflushable_unrooted_slot_count,
6122                i64
6123            ),
6124            (
6125                "flush_roots_elapsed",
6126                flush_roots_elapsed.as_us() as i64,
6127                i64
6128            ),
6129            ("account_bytes_saved", account_bytes_saved, i64),
6130            ("num_accounts_saved", num_accounts_saved, i64),
6131            (
6132                "store_accounts_total_us",
6133                flush_stats.store_accounts_total_us.0,
6134                i64
6135            ),
6136            (
6137                "update_index_us",
6138                flush_stats.store_accounts_timing.update_index_elapsed,
6139                i64
6140            ),
6141            (
6142                "store_accounts_elapsed_us",
6143                flush_stats.store_accounts_timing.store_accounts_elapsed,
6144                i64
6145            ),
6146            (
6147                "handle_reclaims_elapsed_us",
6148                flush_stats.store_accounts_timing.handle_reclaims_elapsed,
6149                i64
6150            ),
6151        );
6152    }
6153
6154    fn flush_rooted_accounts_cache(
6155        &self,
6156        requested_flush_root: Option<Slot>,
6157        should_clean: Option<(&mut usize, &mut usize)>,
6158    ) -> (usize, usize, FlushStats) {
6159        let max_clean_root = should_clean.as_ref().and_then(|_| {
6160            // If there is a long running scan going on, this could prevent any cleaning
6161            // based on updates from slots > `max_clean_root`.
6162            self.max_clean_root(requested_flush_root)
6163        });
6164
6165        let mut written_accounts = HashSet::new();
6166
6167        // If `should_clean` is None, then`should_flush_f` is also None, which will cause
6168        // `flush_slot_cache` to flush all accounts to storage without cleaning any accounts.
6169        let mut should_flush_f = should_clean.map(|(account_bytes_saved, num_accounts_saved)| {
6170            move |&pubkey: &Pubkey, account: &AccountSharedData| {
6171                // if not in hashset, then not flushed previously, so flush it
6172                let should_flush = written_accounts.insert(pubkey);
6173                if !should_flush {
6174                    *account_bytes_saved += account.data().len();
6175                    *num_accounts_saved += 1;
6176                    // If a later root already wrote this account, no point
6177                    // in flushing it
6178                }
6179                should_flush
6180            }
6181        });
6182
6183        // Always flush up to `requested_flush_root`, which is necessary for things like snapshotting.
6184        let cached_roots: BTreeSet<Slot> = self.accounts_cache.clear_roots(requested_flush_root);
6185
6186        // Iterate from highest to lowest so that we don't need to flush earlier
6187        // outdated updates in earlier roots
6188        let mut num_roots_flushed = 0;
6189        let mut flush_stats = FlushStats::default();
6190        for &root in cached_roots.iter().rev() {
6191            if let Some(stats) =
6192                self.flush_slot_cache_with_clean(root, should_flush_f.as_mut(), max_clean_root)
6193            {
6194                num_roots_flushed += 1;
6195                flush_stats.accumulate(&stats);
6196            }
6197
6198            // Regardless of whether this slot was *just* flushed from the cache by the above
6199            // `flush_slot_cache()`, we should update the `max_flush_root`.
6200            // This is because some rooted slots may be flushed to storage *before* they are marked as root.
6201            // This can occur for instance when
6202            //  the cache is overwhelmed, we flushed some yet to be rooted frozen slots
6203            // These slots may then *later* be marked as root, so we still need to handle updating the
6204            // `max_flush_root` in the accounts cache.
6205            self.accounts_cache.set_max_flush_root(root);
6206        }
6207
6208        // Only add to the uncleaned roots set *after* we've flushed the previous roots,
6209        // so that clean will actually be able to clean the slots.
6210        let num_new_roots = cached_roots.len();
6211        self.accounts_index.add_uncleaned_roots(cached_roots);
6212        (num_new_roots, num_roots_flushed, flush_stats)
6213    }
6214
6215    fn do_flush_slot_cache(
6216        &self,
6217        slot: Slot,
6218        slot_cache: &SlotCache,
6219        mut should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
6220        max_clean_root: Option<Slot>,
6221    ) -> FlushStats {
6222        let mut flush_stats = FlushStats::default();
6223        let iter_items: Vec<_> = slot_cache.iter().collect();
6224        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
6225        let mut pubkey_to_slot_set: Vec<(Pubkey, Slot)> = vec![];
6226        if should_flush_f.is_some() {
6227            if let Some(max_clean_root) = max_clean_root {
6228                if slot > max_clean_root {
6229                    // Only if the root is greater than the `max_clean_root` do we
6230                    // have to prevent cleaning, otherwise, just default to `should_flush_f`
6231                    // for any slots <= `max_clean_root`
6232                    should_flush_f = None;
6233                }
6234            }
6235        }
6236
6237        let accounts: Vec<(&Pubkey, &AccountSharedData)> = iter_items
6238            .iter()
6239            .filter_map(|iter_item| {
6240                let key = iter_item.key();
6241                let account = &iter_item.value().account;
6242                let should_flush = should_flush_f
6243                    .as_mut()
6244                    .map(|should_flush_f| should_flush_f(key, account))
6245                    .unwrap_or(true);
6246                if should_flush {
6247                    flush_stats.total_size += aligned_stored_size(account.data().len()) as u64;
6248                    flush_stats.num_flushed += 1;
6249                    Some((key, account))
6250                } else {
6251                    // If we don't flush, we have to remove the entry from the
6252                    // index, since it's equivalent to purging
6253                    purged_slot_pubkeys.insert((slot, *key));
6254                    pubkey_to_slot_set.push((*key, slot));
6255                    flush_stats.num_purged += 1;
6256                    None
6257                }
6258            })
6259            .collect();
6260
6261        let is_dead_slot = accounts.is_empty();
6262        // Remove the account index entries from earlier roots that are outdated by later roots.
6263        // Safe because queries to the index will be reading updates from later roots.
6264        self.purge_slot_cache_pubkeys(
6265            slot,
6266            purged_slot_pubkeys,
6267            pubkey_to_slot_set,
6268            is_dead_slot,
6269            &HashSet::default(),
6270        );
6271
6272        if !is_dead_slot {
6273            // This ensures that all updates are written to an AppendVec, before any
6274            // updates to the index happen, so anybody that sees a real entry in the index,
6275            // will be able to find the account in storage
6276            let flushed_store =
6277                self.create_and_insert_store(slot, flush_stats.total_size.0, "flush_slot_cache");
6278            let (store_accounts_timing_inner, store_accounts_total_inner_us) =
6279                measure_us!(self.store_accounts_frozen((slot, &accounts[..]), &flushed_store,));
6280            flush_stats.store_accounts_timing = store_accounts_timing_inner;
6281            flush_stats.store_accounts_total_us = Saturating(store_accounts_total_inner_us);
6282
6283            // If the above sizing function is correct, just one AppendVec is enough to hold
6284            // all the data for the slot
6285            assert!(self.storage.get_slot_storage_entry(slot).is_some());
6286            self.reopen_storage_as_readonly_shrinking_in_progress_ok(slot);
6287        }
6288
6289        // Remove this slot from the cache, which will to AccountsDb's new readers should look like an
6290        // atomic switch from the cache to storage.
6291        // There is some racy condition for existing readers who just has read exactly while
6292        // flushing. That case is handled by retry_to_get_account_accessor()
6293        assert!(self.accounts_cache.remove_slot(slot).is_some());
6294
6295        flush_stats
6296    }
6297
6298    /// flush all accounts in this slot
6299    fn flush_slot_cache(&self, slot: Slot) -> Option<FlushStats> {
6300        self.flush_slot_cache_with_clean(slot, None::<&mut fn(&_, &_) -> bool>, None)
6301    }
6302
6303    /// `should_flush_f` is an optional closure that determines whether a given
6304    /// account should be flushed. Passing `None` will by default flush all
6305    /// accounts
6306    fn flush_slot_cache_with_clean(
6307        &self,
6308        slot: Slot,
6309        should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
6310        max_clean_root: Option<Slot>,
6311    ) -> Option<FlushStats> {
6312        if self
6313            .remove_unrooted_slots_synchronization
6314            .slots_under_contention
6315            .lock()
6316            .unwrap()
6317            .insert(slot)
6318        {
6319            // We have not seen this slot, flush it.
6320            let flush_stats = self.accounts_cache.slot_cache(slot).map(|slot_cache| {
6321                #[cfg(test)]
6322                {
6323                    // Give some time for cache flushing to occur here for unit tests
6324                    sleep(Duration::from_millis(self.load_delay));
6325                }
6326                // Since we added the slot to `slots_under_contention` AND this slot
6327                // still exists in the cache, we know the slot cannot be removed
6328                // by any other threads past this point. We are now responsible for
6329                // flushing this slot.
6330                self.do_flush_slot_cache(slot, &slot_cache, should_flush_f, max_clean_root)
6331            });
6332
6333            // Nobody else should have been purging this slot, so should not have been removed
6334            // from `self.remove_unrooted_slots_synchronization`.
6335            assert!(self
6336                .remove_unrooted_slots_synchronization
6337                .slots_under_contention
6338                .lock()
6339                .unwrap()
6340                .remove(&slot));
6341
6342            // Signal to any threads blocked on `remove_unrooted_slots(slot)` that we have finished
6343            // flushing
6344            self.remove_unrooted_slots_synchronization
6345                .signal
6346                .notify_all();
6347            flush_stats
6348        } else {
6349            // We have already seen this slot. It is already under flushing. Skip.
6350            None
6351        }
6352    }
6353
6354    fn write_accounts_to_cache<'a, 'b>(
6355        &self,
6356        slot: Slot,
6357        accounts_and_meta_to_store: &impl StorableAccounts<'b>,
6358        txs: Option<&[&SanitizedTransaction]>,
6359    ) -> Vec<AccountInfo> {
6360        let mut current_write_version = if self.accounts_update_notifier.is_some() {
6361            self.write_version
6362                .fetch_add(accounts_and_meta_to_store.len() as u64, Ordering::AcqRel)
6363        } else {
6364            0
6365        };
6366
6367        let (account_infos, cached_accounts) = (0..accounts_and_meta_to_store.len())
6368            .map(|index| {
6369                let txn = txs.map(|txs| *txs.get(index).expect("txs must be present if provided"));
6370                let mut account_info = AccountInfo::default();
6371                accounts_and_meta_to_store.account_default_if_zero_lamport(index, |account| {
6372                    let account_shared_data = account.to_account_shared_data();
6373                    let pubkey = account.pubkey();
6374                    account_info = AccountInfo::new(StorageLocation::Cached, account.lamports());
6375
6376                    self.notify_account_at_accounts_update(
6377                        slot,
6378                        &account_shared_data,
6379                        &txn,
6380                        pubkey,
6381                        current_write_version,
6382                    );
6383                    saturating_add_assign!(current_write_version, 1);
6384
6385                    let cached_account =
6386                        self.accounts_cache.store(slot, pubkey, account_shared_data);
6387                    (account_info, cached_account)
6388                })
6389            })
6390            .unzip();
6391
6392        // hash this accounts in bg
6393        if let Some(ref sender) = &self.sender_bg_hasher {
6394            let _ = sender.send(cached_accounts);
6395        };
6396
6397        account_infos
6398    }
6399
6400    fn store_accounts_to<'a: 'c, 'b, 'c>(
6401        &self,
6402        accounts: &'c impl StorableAccounts<'b>,
6403        store_to: &StoreTo,
6404        transactions: Option<&'a [&'a SanitizedTransaction]>,
6405    ) -> Vec<AccountInfo> {
6406        let mut calc_stored_meta_time = Measure::start("calc_stored_meta");
6407        let slot = accounts.target_slot();
6408        if self
6409            .read_only_accounts_cache
6410            .can_slot_be_in_cache(accounts.target_slot())
6411        {
6412            (0..accounts.len()).for_each(|index| {
6413                accounts.account(index, |account| {
6414                    // based on the patterns of how a validator writes accounts, it is almost always the case that there is no read only cache entry
6415                    // for this pubkey and slot. So, we can give that hint to the `remove` for performance.
6416                    self.read_only_accounts_cache
6417                        .remove_assume_not_present(*account.pubkey());
6418                })
6419            });
6420        }
6421        calc_stored_meta_time.stop();
6422        self.stats
6423            .calc_stored_meta
6424            .fetch_add(calc_stored_meta_time.as_us(), Ordering::Relaxed);
6425
6426        match store_to {
6427            StoreTo::Cache => self.write_accounts_to_cache(slot, accounts, transactions),
6428            StoreTo::Storage(storage) => self.write_accounts_to_storage(slot, storage, accounts),
6429        }
6430    }
6431
6432    fn report_store_stats(&self) {
6433        let mut total_count = 0;
6434        let mut newest_slot = 0;
6435        let mut oldest_slot = u64::MAX;
6436        let mut total_bytes = 0;
6437        let mut total_alive_bytes = 0;
6438        for (slot, store) in self.storage.iter() {
6439            total_count += 1;
6440            newest_slot = std::cmp::max(newest_slot, slot);
6441
6442            oldest_slot = std::cmp::min(oldest_slot, slot);
6443
6444            total_alive_bytes += store.alive_bytes();
6445            total_bytes += store.capacity();
6446        }
6447        info!(
6448            "total_stores: {total_count}, newest_slot: {newest_slot}, oldest_slot: {oldest_slot}"
6449        );
6450
6451        let total_alive_ratio = if total_bytes > 0 {
6452            total_alive_bytes as f64 / total_bytes as f64
6453        } else {
6454            0.
6455        };
6456
6457        datapoint_info!(
6458            "accounts_db-stores",
6459            ("total_count", total_count, i64),
6460            ("total_bytes", total_bytes, i64),
6461            ("total_alive_bytes", total_alive_bytes, i64),
6462            ("total_alive_ratio", total_alive_ratio, f64),
6463        );
6464        datapoint_info!(
6465            "accounts_db-perf-stats",
6466            (
6467                "delta_hash_num",
6468                self.stats.delta_hash_num.swap(0, Ordering::Relaxed),
6469                i64
6470            ),
6471            (
6472                "delta_hash_scan_us",
6473                self.stats
6474                    .delta_hash_scan_time_total_us
6475                    .swap(0, Ordering::Relaxed),
6476                i64
6477            ),
6478            (
6479                "delta_hash_accumulate_us",
6480                self.stats
6481                    .delta_hash_accumulate_time_total_us
6482                    .swap(0, Ordering::Relaxed),
6483                i64
6484            ),
6485            (
6486                "skipped_rewrites_num",
6487                self.stats.skipped_rewrites_num.swap(0, Ordering::Relaxed),
6488                i64
6489            ),
6490        );
6491    }
6492
6493    pub fn checked_sum_for_capitalization<T: Iterator<Item = u64>>(balances: T) -> u64 {
6494        AccountsHasher::checked_cast_for_capitalization(balances.map(|b| b as u128).sum::<u128>())
6495    }
6496
6497    pub fn calculate_accounts_hash_from_index(
6498        &self,
6499        max_slot: Slot,
6500        config: &CalcAccountsHashConfig<'_>,
6501    ) -> (AccountsHash, u64) {
6502        let mut collect = Measure::start("collect");
6503        let keys: Vec<_> = self
6504            .accounts_index
6505            .account_maps
6506            .iter()
6507            .flat_map(|map| {
6508                let mut keys = map.keys();
6509                keys.sort_unstable(); // hashmap is not ordered, but bins are relative to each other
6510                keys
6511            })
6512            .collect();
6513        collect.stop();
6514
6515        // Pick a chunk size big enough to allow us to produce output vectors that are smaller than the overall size.
6516        // We'll also accumulate the lamports within each chunk and fewer chunks results in less contention to accumulate the sum.
6517        let chunks = crate::accounts_hash::MERKLE_FANOUT.pow(4);
6518        let total_lamports = Mutex::<u64>::new(0);
6519
6520        let get_account_hashes = || {
6521            keys.par_chunks(chunks)
6522                .map(|pubkeys| {
6523                    let mut sum = 0u128;
6524                    let account_hashes: Vec<Hash> = pubkeys
6525                        .iter()
6526                        .filter_map(|pubkey| {
6527                            let index_entry = self.accounts_index.get_cloned(pubkey)?;
6528                            self.accounts_index
6529                                .get_account_info_with_and_then(
6530                                    &index_entry,
6531                                    config.ancestors,
6532                                    Some(max_slot),
6533                                    |(slot, account_info)| {
6534                                        if account_info.is_zero_lamport() {
6535                                            return None;
6536                                        }
6537                                        self.get_account_accessor(
6538                                            slot,
6539                                            pubkey,
6540                                            &account_info.storage_location(),
6541                                        )
6542                                        .get_loaded_account(|loaded_account| {
6543                                            let mut loaded_hash = loaded_account.loaded_hash();
6544                                            let balance = loaded_account.lamports();
6545                                            let hash_is_missing =
6546                                                loaded_hash == AccountHash(Hash::default());
6547                                            if hash_is_missing {
6548                                                let computed_hash = Self::hash_account(
6549                                                    &loaded_account,
6550                                                    loaded_account.pubkey(),
6551                                                );
6552                                                loaded_hash = computed_hash;
6553                                            }
6554                                            sum += balance as u128;
6555                                            loaded_hash.0
6556                                        })
6557                                    },
6558                                )
6559                                .flatten()
6560                        })
6561                        .collect();
6562                    let mut total = total_lamports.lock().unwrap();
6563                    *total = AccountsHasher::checked_cast_for_capitalization(*total as u128 + sum);
6564                    account_hashes
6565                })
6566                .collect()
6567        };
6568
6569        let mut scan = Measure::start("scan");
6570        let account_hashes: Vec<Vec<Hash>> = self.thread_pool_clean.install(get_account_hashes);
6571        scan.stop();
6572
6573        let total_lamports = *total_lamports.lock().unwrap();
6574
6575        let mut hash_time = Measure::start("hash");
6576        let (accumulated_hash, hash_total) = AccountsHasher::calculate_hash(account_hashes);
6577        hash_time.stop();
6578
6579        datapoint_info!(
6580            "calculate_accounts_hash_from_index",
6581            ("accounts_scan", scan.as_us(), i64),
6582            ("hash", hash_time.as_us(), i64),
6583            ("hash_total", hash_total, i64),
6584            ("collect", collect.as_us(), i64),
6585        );
6586
6587        let accounts_hash = AccountsHash(accumulated_hash);
6588        (accounts_hash, total_lamports)
6589    }
6590
6591    /// Calculates the accounts lt hash
6592    ///
6593    /// Only intended to be called at startup (or by tests).
6594    /// Only intended to be used while testing the experimental accumulator hash.
6595    pub fn calculate_accounts_lt_hash_at_startup_from_index(
6596        &self,
6597        ancestors: &Ancestors,
6598        startup_slot: Slot,
6599    ) -> AccountsLtHash {
6600        debug_assert!(self.is_experimental_accumulator_hash_enabled());
6601
6602        // This impl iterates over all the index bins in parallel, and computes the lt hash
6603        // sequentially per bin.  Then afterwards reduces to a single lt hash.
6604        // This implementation is quite fast.  Runtime is about 150 seconds on mnb as of 10/2/2024.
6605        // The sequential implementation took about 6,275 seconds!
6606        // A different parallel implementation that iterated over the bins *sequentially* and then
6607        // hashed the accounts *within* a bin in parallel took about 600 seconds.  That impl uses
6608        // less memory, as only a single index bin is loaded into mem at a time.
6609        let lt_hash = self
6610            .accounts_index
6611            .account_maps
6612            .par_iter()
6613            .fold(
6614                LtHash::identity,
6615                |mut accumulator_lt_hash, accounts_index_bin| {
6616                    for pubkey in accounts_index_bin.keys() {
6617                        let account_lt_hash = self
6618                            .accounts_index
6619                            .get_with_and_then(
6620                                &pubkey,
6621                                Some(ancestors),
6622                                Some(startup_slot),
6623                                false,
6624                                |(slot, account_info)| {
6625                                    (!account_info.is_zero_lamport()).then(|| {
6626                                        self.get_account_accessor(
6627                                            slot,
6628                                            &pubkey,
6629                                            &account_info.storage_location(),
6630                                        )
6631                                        .get_loaded_account(|loaded_account| {
6632                                            Self::lt_hash_account(&loaded_account, &pubkey)
6633                                        })
6634                                        // SAFETY: The index said this pubkey exists, so
6635                                        // there must be an account to load.
6636                                        .unwrap()
6637                                    })
6638                                },
6639                            )
6640                            .flatten();
6641                        if let Some(account_lt_hash) = account_lt_hash {
6642                            accumulator_lt_hash.mix_in(&account_lt_hash.0);
6643                        }
6644                    }
6645                    accumulator_lt_hash
6646                },
6647            )
6648            .reduce(LtHash::identity, |mut accum, elem| {
6649                accum.mix_in(&elem);
6650                accum
6651            });
6652
6653        AccountsLtHash(lt_hash)
6654    }
6655
6656    /// Calculates the accounts lt hash
6657    ///
6658    /// Intended to be used to verify the accounts lt hash at startup.
6659    ///
6660    /// The `duplicates_lt_hash` is the old/duplicate accounts to mix *out* of the storages.
6661    /// This value comes from index generation.
6662    pub fn calculate_accounts_lt_hash_at_startup_from_storages(
6663        &self,
6664        storages: &[Arc<AccountStorageEntry>],
6665        duplicates_lt_hash: &DuplicatesLtHash,
6666    ) -> AccountsLtHash {
6667        debug_assert!(self.is_experimental_accumulator_hash_enabled());
6668
6669        let mut lt_hash = storages
6670            .par_iter()
6671            .fold(LtHash::identity, |mut accum, storage| {
6672                storage.accounts.scan_accounts(|stored_account_meta| {
6673                    let account_lt_hash =
6674                        Self::lt_hash_account(&stored_account_meta, stored_account_meta.pubkey());
6675                    accum.mix_in(&account_lt_hash.0);
6676                });
6677                accum
6678            })
6679            .reduce(LtHash::identity, |mut accum, elem| {
6680                accum.mix_in(&elem);
6681                accum
6682            });
6683
6684        lt_hash.mix_out(&duplicates_lt_hash.0);
6685
6686        AccountsLtHash(lt_hash)
6687    }
6688
6689    /// This is only valid to call from tests.
6690    /// run the accounts hash calculation and store the results
6691    pub fn update_accounts_hash_for_tests(
6692        &self,
6693        slot: Slot,
6694        ancestors: &Ancestors,
6695        debug_verify: bool,
6696        is_startup: bool,
6697    ) -> (AccountsHash, u64) {
6698        self.update_accounts_hash_with_verify_from(
6699            CalcAccountsHashDataSource::IndexForTests,
6700            debug_verify,
6701            slot,
6702            ancestors,
6703            None,
6704            &EpochSchedule::default(),
6705            &RentCollector::default(),
6706            is_startup,
6707        )
6708    }
6709
6710    fn update_old_slot_stats(&self, stats: &HashStats, storage: Option<&Arc<AccountStorageEntry>>) {
6711        if let Some(storage) = storage {
6712            stats.roots_older_than_epoch.fetch_add(1, Ordering::Relaxed);
6713            let num_accounts = storage.count();
6714            let sizes = storage.capacity();
6715            stats
6716                .append_vec_sizes_older_than_epoch
6717                .fetch_add(sizes as usize, Ordering::Relaxed);
6718            stats
6719                .accounts_in_roots_older_than_epoch
6720                .fetch_add(num_accounts, Ordering::Relaxed);
6721        }
6722    }
6723
6724    /// return slot + offset, where offset can be +/-
6725    fn apply_offset_to_slot(slot: Slot, offset: i64) -> Slot {
6726        if offset > 0 {
6727            slot.saturating_add(offset as u64)
6728        } else {
6729            slot.saturating_sub(offset.unsigned_abs())
6730        }
6731    }
6732
6733    /// `oldest_non_ancient_slot` is only applicable when `Append` is used for ancient append vec packing.
6734    /// If `Pack` is used for ancient append vec packing, return None.
6735    /// Otherwise, return a slot 'max_slot_inclusive' - (slots_per_epoch - `self.ancient_append_vec_offset`)
6736    /// If ancient append vecs are not enabled, return 0.
6737    fn get_oldest_non_ancient_slot_for_hash_calc_scan(
6738        &self,
6739        max_slot_inclusive: Slot,
6740        config: &CalcAccountsHashConfig<'_>,
6741    ) -> Option<Slot> {
6742        if self.create_ancient_storage == CreateAncientStorage::Pack {
6743            // oldest_non_ancient_slot is only applicable when ancient storages are created with `Append`. When ancient storages are created with `Pack`, ancient storages
6744            // can be created in between non-ancient storages. Return None, because oldest_non_ancient_slot is not applicable here.
6745            None
6746        } else if self.ancient_append_vec_offset.is_some() {
6747            // For performance, this is required when ancient appendvecs are enabled
6748            Some(
6749                self.get_oldest_non_ancient_slot_from_slot(
6750                    config.epoch_schedule,
6751                    max_slot_inclusive,
6752                ),
6753            )
6754        } else {
6755            // This causes the entire range to be chunked together, treating older append vecs just like new ones.
6756            // This performs well if there are many old append vecs that haven't been cleaned yet.
6757            // 0 will have the effect of causing ALL older append vecs to be chunked together, just like every other append vec.
6758            Some(0)
6759        }
6760    }
6761
6762    /// hash info about 'storage' into 'hasher'
6763    /// return true iff storage is valid for loading from cache
6764    fn hash_storage_info(
6765        hasher: &mut impl StdHasher,
6766        storage: &AccountStorageEntry,
6767        slot: Slot,
6768    ) -> bool {
6769        // hash info about this storage
6770        storage.written_bytes().hash(hasher);
6771        slot.hash(hasher);
6772        let storage_file = storage.accounts.path();
6773        storage_file.hash(hasher);
6774        let Ok(metadata) = std::fs::metadata(storage_file) else {
6775            return false;
6776        };
6777        let Ok(amod) = metadata.modified() else {
6778            return false;
6779        };
6780        let amod = amod
6781            .duration_since(std::time::UNIX_EPOCH)
6782            .unwrap()
6783            .as_secs();
6784        amod.hash(hasher);
6785
6786        // if we made it here, we have hashed info and we should try to load from the cache
6787        true
6788    }
6789
6790    pub fn calculate_accounts_hash_from(
6791        &self,
6792        data_source: CalcAccountsHashDataSource,
6793        slot: Slot,
6794        config: &CalcAccountsHashConfig<'_>,
6795    ) -> (AccountsHash, u64) {
6796        match data_source {
6797            CalcAccountsHashDataSource::Storages => {
6798                if self.accounts_cache.contains_any_slots(slot) {
6799                    // this indicates a race condition
6800                    inc_new_counter_info!("accounts_hash_items_in_write_cache", 1);
6801                }
6802
6803                let mut collect_time = Measure::start("collect");
6804                let (combined_maps, slots) = self.get_snapshot_storages(..=slot);
6805                collect_time.stop();
6806
6807                let mut sort_time = Measure::start("sort_storages");
6808                let min_root = self.accounts_index.min_alive_root();
6809                let storages = SortedStorages::new_with_slots(
6810                    combined_maps.iter().zip(slots),
6811                    min_root,
6812                    Some(slot),
6813                );
6814                sort_time.stop();
6815
6816                let mut timings = HashStats {
6817                    collect_snapshots_us: collect_time.as_us(),
6818                    storage_sort_us: sort_time.as_us(),
6819                    ..HashStats::default()
6820                };
6821                timings.calc_storage_size_quartiles(&combined_maps);
6822
6823                self.calculate_accounts_hash(config, &storages, timings)
6824            }
6825            CalcAccountsHashDataSource::IndexForTests => {
6826                self.calculate_accounts_hash_from_index(slot, config)
6827            }
6828        }
6829    }
6830
6831    fn calculate_accounts_hash_with_verify_from(
6832        &self,
6833        data_source: CalcAccountsHashDataSource,
6834        debug_verify: bool,
6835        slot: Slot,
6836        config: CalcAccountsHashConfig<'_>,
6837        expected_capitalization: Option<u64>,
6838    ) -> (AccountsHash, u64) {
6839        let (accounts_hash, total_lamports) =
6840            self.calculate_accounts_hash_from(data_source, slot, &config);
6841        if debug_verify {
6842            // calculate the other way (store or non-store) and verify results match.
6843            let data_source_other = match data_source {
6844                CalcAccountsHashDataSource::IndexForTests => CalcAccountsHashDataSource::Storages,
6845                CalcAccountsHashDataSource::Storages => CalcAccountsHashDataSource::IndexForTests,
6846            };
6847            let (accounts_hash_other, total_lamports_other) =
6848                self.calculate_accounts_hash_from(data_source_other, slot, &config);
6849
6850            let success = accounts_hash == accounts_hash_other
6851                && total_lamports == total_lamports_other
6852                && total_lamports == expected_capitalization.unwrap_or(total_lamports);
6853            assert!(
6854                success,
6855                "calculate_accounts_hash_with_verify mismatch. hashes: {}, {}; lamports: {}, {}; \
6856                 expected lamports: {:?}, data source: {:?}, slot: {}",
6857                accounts_hash.0,
6858                accounts_hash_other.0,
6859                total_lamports,
6860                total_lamports_other,
6861                expected_capitalization,
6862                data_source,
6863                slot
6864            );
6865        }
6866        (accounts_hash, total_lamports)
6867    }
6868
6869    /// run the accounts hash calculation and store the results
6870    #[allow(clippy::too_many_arguments)]
6871    pub fn update_accounts_hash_with_verify_from(
6872        &self,
6873        data_source: CalcAccountsHashDataSource,
6874        debug_verify: bool,
6875        slot: Slot,
6876        ancestors: &Ancestors,
6877        expected_capitalization: Option<u64>,
6878        epoch_schedule: &EpochSchedule,
6879        rent_collector: &RentCollector,
6880        is_startup: bool,
6881    ) -> (AccountsHash, u64) {
6882        let (accounts_hash, total_lamports) = self.calculate_accounts_hash_with_verify_from(
6883            data_source,
6884            debug_verify,
6885            slot,
6886            CalcAccountsHashConfig {
6887                use_bg_thread_pool: !is_startup,
6888                ancestors: Some(ancestors),
6889                epoch_schedule,
6890                rent_collector,
6891                store_detailed_debug_info_on_failure: false,
6892            },
6893            expected_capitalization,
6894        );
6895        self.set_accounts_hash(slot, (accounts_hash, total_lamports));
6896        (accounts_hash, total_lamports)
6897    }
6898
6899    /// Calculate the full accounts hash for `storages` and save the results at `slot`
6900    pub fn update_accounts_hash(
6901        &self,
6902        config: &CalcAccountsHashConfig<'_>,
6903        storages: &SortedStorages<'_>,
6904        slot: Slot,
6905        stats: HashStats,
6906    ) -> (AccountsHash, /*capitalization*/ u64) {
6907        let accounts_hash = self.calculate_accounts_hash(config, storages, stats);
6908        let old_accounts_hash = self.set_accounts_hash(slot, accounts_hash);
6909        if let Some(old_accounts_hash) = old_accounts_hash {
6910            warn!(
6911                "Accounts hash was already set for slot {slot}! old: {old_accounts_hash:?}, new: \
6912                 {accounts_hash:?}"
6913            );
6914        }
6915        accounts_hash
6916    }
6917
6918    /// Calculate the incremental accounts hash for `storages` and save the results at `slot`
6919    pub fn update_incremental_accounts_hash(
6920        &self,
6921        config: &CalcAccountsHashConfig<'_>,
6922        storages: &SortedStorages<'_>,
6923        slot: Slot,
6924        stats: HashStats,
6925    ) -> (IncrementalAccountsHash, /*capitalization*/ u64) {
6926        let incremental_accounts_hash =
6927            self.calculate_incremental_accounts_hash(config, storages, stats);
6928        let old_incremental_accounts_hash =
6929            self.set_incremental_accounts_hash(slot, incremental_accounts_hash);
6930        if let Some(old_incremental_accounts_hash) = old_incremental_accounts_hash {
6931            warn!(
6932                "Incremental accounts hash was already set for slot {slot}! old: \
6933                 {old_incremental_accounts_hash:?}, new: {incremental_accounts_hash:?}"
6934            );
6935        }
6936        incremental_accounts_hash
6937    }
6938
6939    /// Set the accounts hash for `slot`
6940    ///
6941    /// returns the previous accounts hash for `slot`
6942    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
6943    fn set_accounts_hash(
6944        &self,
6945        slot: Slot,
6946        accounts_hash: (AccountsHash, /*capitalization*/ u64),
6947    ) -> Option<(AccountsHash, /*capitalization*/ u64)> {
6948        self.accounts_hashes
6949            .lock()
6950            .unwrap()
6951            .insert(slot, accounts_hash)
6952    }
6953
6954    /// After deserializing a snapshot, set the accounts hash for the new AccountsDb
6955    pub fn set_accounts_hash_from_snapshot(
6956        &mut self,
6957        slot: Slot,
6958        accounts_hash: SerdeAccountsHash,
6959        capitalization: u64,
6960    ) -> Option<(AccountsHash, /*capitalization*/ u64)> {
6961        self.set_accounts_hash(slot, (accounts_hash.into(), capitalization))
6962    }
6963
6964    /// Get the accounts hash for `slot`
6965    pub fn get_accounts_hash(&self, slot: Slot) -> Option<(AccountsHash, /*capitalization*/ u64)> {
6966        self.accounts_hashes.lock().unwrap().get(&slot).cloned()
6967    }
6968
6969    /// Get all accounts hashes
6970    pub fn get_accounts_hashes(&self) -> HashMap<Slot, (AccountsHash, /*capitalization*/ u64)> {
6971        self.accounts_hashes.lock().unwrap().clone()
6972    }
6973
6974    /// Set the incremental accounts hash for `slot`
6975    ///
6976    /// returns the previous incremental accounts hash for `slot`
6977    pub fn set_incremental_accounts_hash(
6978        &self,
6979        slot: Slot,
6980        incremental_accounts_hash: (IncrementalAccountsHash, /*capitalization*/ u64),
6981    ) -> Option<(IncrementalAccountsHash, /*capitalization*/ u64)> {
6982        self.incremental_accounts_hashes
6983            .lock()
6984            .unwrap()
6985            .insert(slot, incremental_accounts_hash)
6986    }
6987
6988    /// After deserializing a snapshot, set the incremental accounts hash for the new AccountsDb
6989    pub fn set_incremental_accounts_hash_from_snapshot(
6990        &mut self,
6991        slot: Slot,
6992        incremental_accounts_hash: SerdeIncrementalAccountsHash,
6993        capitalization: u64,
6994    ) -> Option<(IncrementalAccountsHash, /*capitalization*/ u64)> {
6995        self.set_incremental_accounts_hash(slot, (incremental_accounts_hash.into(), capitalization))
6996    }
6997
6998    /// Get the incremental accounts hash for `slot`
6999    pub fn get_incremental_accounts_hash(
7000        &self,
7001        slot: Slot,
7002    ) -> Option<(IncrementalAccountsHash, /*capitalization*/ u64)> {
7003        self.incremental_accounts_hashes
7004            .lock()
7005            .unwrap()
7006            .get(&slot)
7007            .cloned()
7008    }
7009
7010    /// Get all incremental accounts hashes
7011    pub fn get_incremental_accounts_hashes(
7012        &self,
7013    ) -> HashMap<Slot, (IncrementalAccountsHash, /*capitalization*/ u64)> {
7014        self.incremental_accounts_hashes.lock().unwrap().clone()
7015    }
7016
7017    /// Purge accounts hashes that are older than `latest_full_snapshot_slot`
7018    ///
7019    /// Should only be called by AccountsHashVerifier, since it consumes the accounts hashes and
7020    /// knows which ones are still needed.
7021    pub fn purge_old_accounts_hashes(&self, latest_full_snapshot_slot: Slot) {
7022        self.accounts_hashes
7023            .lock()
7024            .unwrap()
7025            .retain(|&slot, _| slot >= latest_full_snapshot_slot);
7026        self.incremental_accounts_hashes
7027            .lock()
7028            .unwrap()
7029            .retain(|&slot, _| slot >= latest_full_snapshot_slot);
7030    }
7031
7032    fn sort_slot_storage_scan(accum: &mut BinnedHashData) -> u64 {
7033        let (_, sort_time) = measure_us!(accum.iter_mut().for_each(|items| {
7034            // sort_by vs unstable because slot and write_version are already in order
7035            items.sort_by(AccountsHasher::compare_two_hash_entries);
7036        }));
7037        sort_time
7038    }
7039
7040    /// normal code path returns the common cache path
7041    /// when called after a failure has been detected, redirect the cache storage to a separate folder for debugging later
7042    fn get_cache_hash_data(
7043        accounts_hash_cache_path: PathBuf,
7044        config: &CalcAccountsHashConfig<'_>,
7045        kind: CalcAccountsHashKind,
7046        slot: Slot,
7047        storages_start_slot: Slot,
7048    ) -> CacheHashData {
7049        let accounts_hash_cache_path = if !config.store_detailed_debug_info_on_failure {
7050            accounts_hash_cache_path
7051        } else {
7052            // this path executes when we are failing with a hash mismatch
7053            let failed_dir = accounts_hash_cache_path
7054                .join("failed_calculate_accounts_hash_cache")
7055                .join(slot.to_string());
7056            _ = std::fs::remove_dir_all(&failed_dir);
7057            failed_dir
7058        };
7059        let deletion_policy = match kind {
7060            CalcAccountsHashKind::Full => CacheHashDeletionPolicy::AllUnused,
7061            CalcAccountsHashKind::Incremental => {
7062                CacheHashDeletionPolicy::UnusedAtLeast(storages_start_slot)
7063            }
7064        };
7065        CacheHashData::new(accounts_hash_cache_path, deletion_policy)
7066    }
7067
7068    /// Calculate the full accounts hash
7069    ///
7070    /// This is intended to be used by startup verification, and also AccountsHashVerifier.
7071    /// Uses account storage files as the data source for the calculation.
7072    pub fn calculate_accounts_hash(
7073        &self,
7074        config: &CalcAccountsHashConfig<'_>,
7075        storages: &SortedStorages<'_>,
7076        stats: HashStats,
7077    ) -> (AccountsHash, u64) {
7078        let (accounts_hash, capitalization) = self.calculate_accounts_hash_from_storages(
7079            config,
7080            storages,
7081            stats,
7082            CalcAccountsHashKind::Full,
7083        );
7084        let AccountsHashKind::Full(accounts_hash) = accounts_hash else {
7085            panic!("calculate_accounts_hash_from_storages must return a FullAccountsHash");
7086        };
7087        (accounts_hash, capitalization)
7088    }
7089
7090    /// Calculate the incremental accounts hash
7091    ///
7092    /// This calculation is intended to be used by incremental snapshots, and thus differs from a
7093    /// "full" accounts hash in a few ways:
7094    /// - Zero-lamport accounts are *included* in the hash because zero-lamport accounts are also
7095    ///   included in the incremental snapshot.  This ensures reconstructing the AccountsDb is
7096    ///   still correct when using this incremental accounts hash.
7097    /// - `storages` must be the same as the ones going into the incremental snapshot.
7098    pub fn calculate_incremental_accounts_hash(
7099        &self,
7100        config: &CalcAccountsHashConfig<'_>,
7101        storages: &SortedStorages<'_>,
7102        stats: HashStats,
7103    ) -> (IncrementalAccountsHash, /* capitalization */ u64) {
7104        let (accounts_hash, capitalization) = self.calculate_accounts_hash_from_storages(
7105            config,
7106            storages,
7107            stats,
7108            CalcAccountsHashKind::Incremental,
7109        );
7110        let AccountsHashKind::Incremental(incremental_accounts_hash) = accounts_hash else {
7111            panic!("calculate_incremental_accounts_hash must return an IncrementalAccountsHash");
7112        };
7113        (incremental_accounts_hash, capitalization)
7114    }
7115
7116    /// The shared code for calculating accounts hash from storages.
7117    /// Used for both full accounts hash and incremental accounts hash calculation.
7118    fn calculate_accounts_hash_from_storages(
7119        &self,
7120        config: &CalcAccountsHashConfig<'_>,
7121        storages: &SortedStorages<'_>,
7122        mut stats: HashStats,
7123        kind: CalcAccountsHashKind,
7124    ) -> (AccountsHashKind, u64) {
7125        let total_time = Measure::start("");
7126        let _guard = self.active_stats.activate(ActiveStatItem::Hash);
7127        let storages_start_slot = storages.range().start;
7128        stats.oldest_root = storages_start_slot;
7129
7130        let slot = storages.max_slot_inclusive();
7131        let use_bg_thread_pool = config.use_bg_thread_pool;
7132        let accounts_hash_cache_path = self.accounts_hash_cache_path.clone();
7133        let transient_accounts_hash_cache_dir = TempDir::new_in(&accounts_hash_cache_path)
7134            .expect("create transient accounts hash cache dir");
7135        let transient_accounts_hash_cache_path =
7136            transient_accounts_hash_cache_dir.path().to_path_buf();
7137        let scan_and_hash = || {
7138            let (cache_hash_data, cache_hash_data_us) = measure_us!(Self::get_cache_hash_data(
7139                accounts_hash_cache_path,
7140                config,
7141                kind,
7142                slot,
7143                storages_start_slot,
7144            ));
7145            stats.cache_hash_data_us += cache_hash_data_us;
7146
7147            let bounds = Range {
7148                start: 0,
7149                end: PUBKEY_BINS_FOR_CALCULATING_HASHES,
7150            };
7151
7152            let accounts_hasher = AccountsHasher {
7153                zero_lamport_accounts: kind.zero_lamport_accounts(),
7154                dir_for_temp_cache_files: transient_accounts_hash_cache_path,
7155                active_stats: &self.active_stats,
7156            };
7157
7158            // get raw data by scanning
7159            let cache_hash_data_file_references = self.scan_snapshot_stores_with_cache(
7160                &cache_hash_data,
7161                storages,
7162                &mut stats,
7163                PUBKEY_BINS_FOR_CALCULATING_HASHES,
7164                &bounds,
7165                config,
7166            );
7167
7168            let cache_hash_data_files = cache_hash_data_file_references
7169                .iter()
7170                .map(|d| d.map())
7171                .collect::<Vec<_>>();
7172
7173            if let Some(err) = cache_hash_data_files
7174                .iter()
7175                .filter_map(|r| r.as_ref().err())
7176                .next()
7177            {
7178                panic!("failed generating accounts hash files: {:?}", err);
7179            }
7180
7181            // convert mmapped cache files into slices of data
7182            let cache_hash_intermediates = cache_hash_data_files
7183                .iter()
7184                .map(|d| d.as_ref().unwrap().get_cache_hash_data())
7185                .collect::<Vec<_>>();
7186
7187            // turn raw data into merkle tree hashes and sum of lamports
7188            let (accounts_hash, capitalization) =
7189                accounts_hasher.rest_of_hash_calculation(&cache_hash_intermediates, &mut stats);
7190            let accounts_hash = match kind {
7191                CalcAccountsHashKind::Full => AccountsHashKind::Full(AccountsHash(accounts_hash)),
7192                CalcAccountsHashKind::Incremental => {
7193                    AccountsHashKind::Incremental(IncrementalAccountsHash(accounts_hash))
7194                }
7195            };
7196            info!(
7197                "calculate_accounts_hash_from_storages: slot: {slot}, {accounts_hash:?}, \
7198                 capitalization: {capitalization}"
7199            );
7200            (accounts_hash, capitalization)
7201        };
7202
7203        let result = if use_bg_thread_pool {
7204            self.thread_pool_hash.install(scan_and_hash)
7205        } else {
7206            scan_and_hash()
7207        };
7208        stats.total_us = total_time.end_as_us();
7209        stats.log();
7210        result
7211    }
7212
7213    /// Verify accounts hash at startup (or tests)
7214    ///
7215    /// Calculate accounts hash(es) and compare them to the values set at startup.
7216    /// If `base` is `None`, only calculates the full accounts hash for `[0, slot]`.
7217    /// If `base` is `Some`, calculate the full accounts hash for `[0, base slot]`
7218    /// and then calculate the incremental accounts hash for `(base slot, slot]`.
7219    pub fn verify_accounts_hash_and_lamports(
7220        &self,
7221        snapshot_storages_and_slots: (&[Arc<AccountStorageEntry>], &[Slot]),
7222        slot: Slot,
7223        total_lamports: u64,
7224        base: Option<(Slot, /*capitalization*/ u64)>,
7225        config: VerifyAccountsHashAndLamportsConfig,
7226    ) -> Result<(), AccountsHashVerificationError> {
7227        let calc_config = CalcAccountsHashConfig {
7228            use_bg_thread_pool: config.use_bg_thread_pool,
7229            ancestors: Some(config.ancestors),
7230            epoch_schedule: config.epoch_schedule,
7231            rent_collector: config.rent_collector,
7232            store_detailed_debug_info_on_failure: config.store_detailed_debug_info,
7233        };
7234        let hash_mismatch_is_error = !config.ignore_mismatch;
7235
7236        if let Some((base_slot, base_capitalization)) = base {
7237            self.verify_accounts_hash_and_lamports(
7238                snapshot_storages_and_slots,
7239                base_slot,
7240                base_capitalization,
7241                None,
7242                config,
7243            )?;
7244
7245            let storages_and_slots = snapshot_storages_and_slots
7246                .0
7247                .iter()
7248                .zip(snapshot_storages_and_slots.1.iter())
7249                .filter(|storage_and_slot| *storage_and_slot.1 > base_slot)
7250                .map(|(storage, slot)| (storage, *slot));
7251            let sorted_storages = SortedStorages::new_with_slots(storages_and_slots, None, None);
7252            let calculated_incremental_accounts_hash = self.calculate_incremental_accounts_hash(
7253                &calc_config,
7254                &sorted_storages,
7255                HashStats::default(),
7256            );
7257            let found_incremental_accounts_hash = self
7258                .get_incremental_accounts_hash(slot)
7259                .ok_or(AccountsHashVerificationError::MissingAccountsHash)?;
7260            if calculated_incremental_accounts_hash != found_incremental_accounts_hash {
7261                warn!(
7262                    "mismatched incremental accounts hash for slot {slot}: \
7263                     {calculated_incremental_accounts_hash:?} (calculated) != \
7264                     {found_incremental_accounts_hash:?} (expected)"
7265                );
7266                if hash_mismatch_is_error {
7267                    return Err(AccountsHashVerificationError::MismatchedAccountsHash);
7268                }
7269            }
7270        } else {
7271            let storages_and_slots = snapshot_storages_and_slots
7272                .0
7273                .iter()
7274                .zip(snapshot_storages_and_slots.1.iter())
7275                .filter(|storage_and_slot| *storage_and_slot.1 <= slot)
7276                .map(|(storage, slot)| (storage, *slot));
7277            let sorted_storages = SortedStorages::new_with_slots(storages_and_slots, None, None);
7278            let (calculated_accounts_hash, calculated_lamports) =
7279                self.calculate_accounts_hash(&calc_config, &sorted_storages, HashStats::default());
7280            if calculated_lamports != total_lamports {
7281                warn!(
7282                    "Mismatched total lamports: {} calculated: {}",
7283                    total_lamports, calculated_lamports
7284                );
7285                return Err(AccountsHashVerificationError::MismatchedTotalLamports(
7286                    calculated_lamports,
7287                    total_lamports,
7288                ));
7289            }
7290            let (found_accounts_hash, _) = self
7291                .get_accounts_hash(slot)
7292                .ok_or(AccountsHashVerificationError::MissingAccountsHash)?;
7293            if calculated_accounts_hash != found_accounts_hash {
7294                warn!(
7295                    "Mismatched accounts hash for slot {slot}: {calculated_accounts_hash:?} \
7296                     (calculated) != {found_accounts_hash:?} (expected)"
7297                );
7298                if hash_mismatch_is_error {
7299                    return Err(AccountsHashVerificationError::MismatchedAccountsHash);
7300                }
7301            }
7302        }
7303
7304        Ok(())
7305    }
7306
7307    /// helper to return
7308    /// 1. pubkey, hash pairs for the slot
7309    /// 2. us spent scanning
7310    /// 3. Measure started when we began accumulating
7311    pub fn get_pubkey_hash_for_slot(
7312        &self,
7313        slot: Slot,
7314    ) -> (Vec<(Pubkey, AccountHash)>, u64, Measure) {
7315        let mut scan = Measure::start("scan");
7316        let scan_result: ScanStorageResult<(Pubkey, AccountHash), DashMap<Pubkey, AccountHash>> =
7317            self.scan_account_storage(
7318                slot,
7319                |loaded_account: &LoadedAccount| {
7320                    // Cache only has one version per key, don't need to worry about versioning
7321                    Some((*loaded_account.pubkey(), loaded_account.loaded_hash()))
7322                },
7323                |accum: &DashMap<Pubkey, AccountHash>, loaded_account: &LoadedAccount, _data| {
7324                    let mut loaded_hash = loaded_account.loaded_hash();
7325                    if loaded_hash == AccountHash(Hash::default()) {
7326                        loaded_hash = Self::hash_account(loaded_account, loaded_account.pubkey())
7327                    }
7328                    accum.insert(*loaded_account.pubkey(), loaded_hash);
7329                },
7330                ScanAccountStorageData::NoData,
7331            );
7332        scan.stop();
7333
7334        let accumulate = Measure::start("accumulate");
7335        let hashes: Vec<_> = match scan_result {
7336            ScanStorageResult::Cached(cached_result) => cached_result,
7337            ScanStorageResult::Stored(stored_result) => stored_result.into_iter().collect(),
7338        };
7339
7340        (hashes, scan.as_us(), accumulate)
7341    }
7342
7343    /// Return all of the accounts for a given slot
7344    pub fn get_pubkey_hash_account_for_slot(&self, slot: Slot) -> Vec<PubkeyHashAccount> {
7345        type ScanResult =
7346            ScanStorageResult<PubkeyHashAccount, DashMap<Pubkey, (AccountHash, AccountSharedData)>>;
7347        let scan_result: ScanResult = self.scan_account_storage(
7348            slot,
7349            |loaded_account: &LoadedAccount| {
7350                // Cache only has one version per key, don't need to worry about versioning
7351                Some(PubkeyHashAccount {
7352                    pubkey: *loaded_account.pubkey(),
7353                    hash: loaded_account.loaded_hash(),
7354                    account: loaded_account.take_account(),
7355                })
7356            },
7357            |accum: &DashMap<Pubkey, (AccountHash, AccountSharedData)>,
7358             loaded_account: &LoadedAccount,
7359             _data| {
7360                // Storage may have duplicates so only keep the latest version for each key
7361                let mut loaded_hash = loaded_account.loaded_hash();
7362                let key = *loaded_account.pubkey();
7363                let account = loaded_account.take_account();
7364                if loaded_hash == AccountHash(Hash::default()) {
7365                    loaded_hash = Self::hash_account(&account, &key)
7366                }
7367                accum.insert(key, (loaded_hash, account));
7368            },
7369            ScanAccountStorageData::NoData,
7370        );
7371
7372        match scan_result {
7373            ScanStorageResult::Cached(cached_result) => cached_result,
7374            ScanStorageResult::Stored(stored_result) => stored_result
7375                .into_iter()
7376                .map(|(pubkey, (hash, account))| PubkeyHashAccount {
7377                    pubkey,
7378                    hash,
7379                    account,
7380                })
7381                .collect(),
7382        }
7383    }
7384
7385    /// Wrapper function to calculate accounts delta hash for `slot` (only used for testing and benchmarking.)
7386    ///
7387    /// As part of calculating the accounts delta hash, get a list of accounts modified this slot
7388    /// (aka dirty pubkeys) and add them to `self.uncleaned_pubkeys` for future cleaning.
7389    #[cfg(feature = "dev-context-only-utils")]
7390    pub fn calculate_accounts_delta_hash(&self, slot: Slot) -> AccountsDeltaHash {
7391        self.calculate_accounts_delta_hash_internal(slot, None, HashMap::default())
7392    }
7393
7394    /// Calculate accounts delta hash for `slot`
7395    ///
7396    /// As part of calculating the accounts delta hash, get a list of accounts modified this slot
7397    /// (aka dirty pubkeys) and add them to `self.uncleaned_pubkeys` for future cleaning.
7398    pub fn calculate_accounts_delta_hash_internal(
7399        &self,
7400        slot: Slot,
7401        ignore: Option<Pubkey>,
7402        mut skipped_rewrites: HashMap<Pubkey, AccountHash>,
7403    ) -> AccountsDeltaHash {
7404        let (mut hashes, scan_us, mut accumulate) = self.get_pubkey_hash_for_slot(slot);
7405        let dirty_keys = hashes.iter().map(|(pubkey, _hash)| *pubkey).collect();
7406
7407        hashes.iter().for_each(|(k, _h)| {
7408            skipped_rewrites.remove(k);
7409        });
7410
7411        let num_skipped_rewrites = skipped_rewrites.len();
7412        hashes.extend(skipped_rewrites);
7413
7414        info!("skipped rewrite hashes {} {}", slot, num_skipped_rewrites);
7415
7416        if let Some(ignore) = ignore {
7417            hashes.retain(|k| k.0 != ignore);
7418        }
7419
7420        let accounts_delta_hash = self
7421            .thread_pool
7422            .install(|| AccountsDeltaHash(AccountsHasher::accumulate_account_hashes(hashes)));
7423        accumulate.stop();
7424        let mut uncleaned_time = Measure::start("uncleaned_index");
7425        self.uncleaned_pubkeys.insert(slot, dirty_keys);
7426        uncleaned_time.stop();
7427
7428        self.set_accounts_delta_hash(slot, accounts_delta_hash);
7429
7430        self.stats
7431            .store_uncleaned_update
7432            .fetch_add(uncleaned_time.as_us(), Ordering::Relaxed);
7433        self.stats
7434            .delta_hash_scan_time_total_us
7435            .fetch_add(scan_us, Ordering::Relaxed);
7436        self.stats
7437            .delta_hash_accumulate_time_total_us
7438            .fetch_add(accumulate.as_us(), Ordering::Relaxed);
7439        self.stats.delta_hash_num.fetch_add(1, Ordering::Relaxed);
7440        self.stats
7441            .skipped_rewrites_num
7442            .fetch_add(num_skipped_rewrites, Ordering::Relaxed);
7443
7444        accounts_delta_hash
7445    }
7446
7447    /// Set the accounts delta hash for `slot` in the `accounts_delta_hashes` map
7448    ///
7449    /// returns the previous accounts delta hash for `slot`
7450    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
7451    fn set_accounts_delta_hash(
7452        &self,
7453        slot: Slot,
7454        accounts_delta_hash: AccountsDeltaHash,
7455    ) -> Option<AccountsDeltaHash> {
7456        self.accounts_delta_hashes
7457            .lock()
7458            .unwrap()
7459            .insert(slot, accounts_delta_hash)
7460    }
7461
7462    /// After deserializing a snapshot, set the accounts delta hash for the new AccountsDb
7463    pub fn set_accounts_delta_hash_from_snapshot(
7464        &mut self,
7465        slot: Slot,
7466        accounts_delta_hash: SerdeAccountsDeltaHash,
7467    ) -> Option<AccountsDeltaHash> {
7468        self.set_accounts_delta_hash(slot, accounts_delta_hash.into())
7469    }
7470
7471    /// Get the accounts delta hash for `slot` in the `accounts_delta_hashes` map
7472    pub fn get_accounts_delta_hash(&self, slot: Slot) -> Option<AccountsDeltaHash> {
7473        self.accounts_delta_hashes
7474            .lock()
7475            .unwrap()
7476            .get(&slot)
7477            .cloned()
7478    }
7479
7480    /// When reconstructing AccountsDb from a snapshot, insert the `bank_hash_stats` into the
7481    /// internal bank hash stats map.
7482    ///
7483    /// This fn is only called when loading from a snapshot, which means AccountsDb is new and its
7484    /// bank hash stats map is unpopulated.  Except for slot 0.
7485    ///
7486    /// Slot 0 is a special case.  When a new AccountsDb is created--like when loading from a
7487    /// snapshot--the bank hash stats map is populated with a default entry at slot 0.  Remove the
7488    /// default entry at slot 0, and then insert the new value at `slot`.
7489    pub fn update_bank_hash_stats_from_snapshot(
7490        &mut self,
7491        slot: Slot,
7492        stats: BankHashStats,
7493    ) -> Option<BankHashStats> {
7494        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
7495        bank_hash_stats.remove(&0);
7496        bank_hash_stats.insert(slot, stats)
7497    }
7498
7499    /// Get the bank hash stats for `slot` in the `bank_hash_stats` map
7500    pub fn get_bank_hash_stats(&self, slot: Slot) -> Option<BankHashStats> {
7501        self.bank_hash_stats.lock().unwrap().get(&slot).cloned()
7502    }
7503
7504    fn update_index<'a>(
7505        &self,
7506        infos: Vec<AccountInfo>,
7507        accounts: &impl StorableAccounts<'a>,
7508        reclaim: UpsertReclaim,
7509        update_index_thread_selection: UpdateIndexThreadSelection,
7510    ) -> SlotList<AccountInfo> {
7511        let target_slot = accounts.target_slot();
7512        // using a thread pool here results in deadlock panics from bank_hashes.write()
7513        // so, instead we limit how many threads will be created to the same size as the bg thread pool
7514        let len = std::cmp::min(accounts.len(), infos.len());
7515        let threshold = 1;
7516        let update = |start, end| {
7517            let mut reclaims = Vec::with_capacity((end - start) / 2);
7518
7519            (start..end).for_each(|i| {
7520                let info = infos[i];
7521                accounts.account(i, |account| {
7522                    let old_slot = accounts.slot(i);
7523                    self.accounts_index.upsert(
7524                        target_slot,
7525                        old_slot,
7526                        account.pubkey(),
7527                        &account,
7528                        &self.account_indexes,
7529                        info,
7530                        &mut reclaims,
7531                        reclaim,
7532                    );
7533                });
7534            });
7535            reclaims
7536        };
7537        if matches!(
7538            update_index_thread_selection,
7539            UpdateIndexThreadSelection::PoolWithThreshold,
7540        ) && len > threshold
7541        {
7542            let chunk_size = std::cmp::max(1, len / quarter_thread_count()); // # pubkeys/thread
7543            let batches = 1 + len / chunk_size;
7544            (0..batches)
7545                .into_par_iter()
7546                .map(|batch| {
7547                    let start = batch * chunk_size;
7548                    let end = std::cmp::min(start + chunk_size, len);
7549                    update(start, end)
7550                })
7551                .flatten()
7552                .collect::<Vec<_>>()
7553        } else {
7554            update(0, len)
7555        }
7556    }
7557
7558    fn should_not_shrink(alive_bytes: u64, total_bytes: u64) -> bool {
7559        alive_bytes >= total_bytes
7560    }
7561
7562    fn is_shrinking_productive(store: &AccountStorageEntry) -> bool {
7563        let alive_count = store.count();
7564        let alive_bytes = store.alive_bytes() as u64;
7565        let total_bytes = store.capacity();
7566
7567        if Self::should_not_shrink(alive_bytes, total_bytes) {
7568            trace!(
7569                "shrink_slot_forced ({}): not able to shrink at all: num alive: {}, bytes alive: \
7570                 {}, bytes total: {}, bytes saved: {}",
7571                store.slot(),
7572                alive_count,
7573                alive_bytes,
7574                total_bytes,
7575                total_bytes.saturating_sub(alive_bytes),
7576            );
7577            return false;
7578        }
7579
7580        true
7581    }
7582
7583    /// Determines whether a given AccountStorageEntry instance is a
7584    /// candidate for shrinking.
7585    pub(crate) fn is_candidate_for_shrink(&self, store: &AccountStorageEntry) -> bool {
7586        // appended ancient append vecs should not be shrunk by the normal shrink codepath.
7587        // It is not possible to identify ancient append vecs when we pack, so no check for ancient when we are not appending.
7588        let total_bytes = if self.create_ancient_storage == CreateAncientStorage::Append
7589            && is_ancient(&store.accounts)
7590            && store.accounts.can_append()
7591        {
7592            store.written_bytes()
7593        } else {
7594            store.capacity()
7595        };
7596        match self.shrink_ratio {
7597            AccountShrinkThreshold::TotalSpace { shrink_ratio: _ } => {
7598                (store.alive_bytes() as u64) < total_bytes
7599            }
7600            AccountShrinkThreshold::IndividualStore { shrink_ratio } => {
7601                (store.alive_bytes() as f64 / total_bytes as f64) < shrink_ratio
7602            }
7603        }
7604    }
7605
7606    /// returns (dead slots, reclaimed_offsets)
7607    fn remove_dead_accounts<'a, I>(
7608        &'a self,
7609        reclaims: I,
7610        expected_slot: Option<Slot>,
7611        reset_accounts: bool,
7612    ) -> (IntSet<Slot>, SlotOffsets)
7613    where
7614        I: Iterator<Item = &'a (Slot, AccountInfo)>,
7615    {
7616        let mut reclaimed_offsets = SlotOffsets::default();
7617
7618        assert!(self.storage.no_shrink_in_progress());
7619
7620        let mut dead_slots = IntSet::default();
7621        let mut new_shrink_candidates = ShrinkCandidates::default();
7622        let mut measure = Measure::start("remove");
7623        for (slot, account_info) in reclaims {
7624            // No cached accounts should make it here
7625            assert!(!account_info.is_cached());
7626            reclaimed_offsets
7627                .entry(*slot)
7628                .or_default()
7629                .insert(account_info.offset());
7630        }
7631        if let Some(expected_slot) = expected_slot {
7632            assert_eq!(reclaimed_offsets.len(), 1);
7633            assert!(reclaimed_offsets.contains_key(&expected_slot));
7634        }
7635
7636        self.clean_accounts_stats
7637            .slots_cleaned
7638            .fetch_add(reclaimed_offsets.len() as u64, Ordering::Relaxed);
7639
7640        reclaimed_offsets.iter().for_each(|(slot, offsets)| {
7641            if let Some(store) = self.storage.get_slot_storage_entry(*slot) {
7642                assert_eq!(
7643                    *slot,
7644                    store.slot(),
7645                    "AccountsDB::accounts_index corrupted. Storage pointed to: {}, expected: {}, \
7646                     should only point to one slot",
7647                    store.slot(),
7648                    *slot
7649                );
7650                if offsets.len() == store.count() {
7651                    // all remaining alive accounts in the storage are being removed, so the entire storage/slot is dead
7652                    store.remove_accounts(store.alive_bytes(), reset_accounts, offsets.len());
7653                    self.dirty_stores.insert(*slot, store.clone());
7654                    dead_slots.insert(*slot);
7655                } else {
7656                    // not all accounts are being removed, so figure out sizes of accounts we are removing and update the alive bytes and alive account count
7657                    let (_, us) = measure_us!({
7658                        let mut offsets = offsets.iter().cloned().collect::<Vec<_>>();
7659                        // sort so offsets are in order. This improves efficiency of loading the accounts.
7660                        offsets.sort_unstable();
7661                        let dead_bytes = store.accounts.get_account_sizes(&offsets).iter().sum();
7662                        store.remove_accounts(dead_bytes, reset_accounts, offsets.len());
7663                        if Self::is_shrinking_productive(&store)
7664                            && self.is_candidate_for_shrink(&store)
7665                        {
7666                            // Checking that this single storage entry is ready for shrinking,
7667                            // should be a sufficient indication that the slot is ready to be shrunk
7668                            // because slots should only have one storage entry, namely the one that was
7669                            // created by `flush_slot_cache()`.
7670                            new_shrink_candidates.insert(*slot);
7671                        }
7672                    });
7673                    self.clean_accounts_stats
7674                        .get_account_sizes_us
7675                        .fetch_add(us, Ordering::Relaxed);
7676                }
7677            }
7678        });
7679        measure.stop();
7680        self.clean_accounts_stats
7681            .remove_dead_accounts_remove_us
7682            .fetch_add(measure.as_us(), Ordering::Relaxed);
7683
7684        let mut measure = Measure::start("shrink");
7685        let mut shrink_candidate_slots = self.shrink_candidate_slots.lock().unwrap();
7686        for slot in new_shrink_candidates {
7687            shrink_candidate_slots.insert(slot);
7688        }
7689        drop(shrink_candidate_slots);
7690        measure.stop();
7691        self.clean_accounts_stats
7692            .remove_dead_accounts_shrink_us
7693            .fetch_add(measure.as_us(), Ordering::Relaxed);
7694
7695        dead_slots.retain(|slot| {
7696            if let Some(slot_store) = self.storage.get_slot_storage_entry(*slot) {
7697                if slot_store.count() != 0 {
7698                    return false;
7699                }
7700            }
7701            true
7702        });
7703
7704        (dead_slots, reclaimed_offsets)
7705    }
7706
7707    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7708    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7709    fn remove_dead_slots_metadata<'a>(
7710        &'a self,
7711        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
7712        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7713        // Should only be `Some` for non-cached slots
7714        purged_stored_account_slots: Option<&mut AccountSlots>,
7715        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7716    ) {
7717        let mut measure = Measure::start("remove_dead_slots_metadata-ms");
7718        self.clean_dead_slots_from_accounts_index(
7719            dead_slots_iter.clone(),
7720            purged_slot_pubkeys,
7721            purged_stored_account_slots,
7722            pubkeys_removed_from_accounts_index,
7723        );
7724
7725        let mut accounts_delta_hashes = self.accounts_delta_hashes.lock().unwrap();
7726        let mut bank_hash_stats = self.bank_hash_stats.lock().unwrap();
7727        for slot in dead_slots_iter {
7728            accounts_delta_hashes.remove(slot);
7729            bank_hash_stats.remove(slot);
7730        }
7731        drop(accounts_delta_hashes);
7732        drop(bank_hash_stats);
7733
7734        measure.stop();
7735        inc_new_counter_info!("remove_dead_slots_metadata-ms", measure.as_ms() as usize);
7736    }
7737
7738    /// lookup each pubkey in 'pubkeys' and unref it in the accounts index
7739    /// skip pubkeys that are in 'pubkeys_removed_from_accounts_index'
7740    fn unref_pubkeys<'a>(
7741        &'a self,
7742        pubkeys: impl Iterator<Item = &'a Pubkey> + Clone + Send + Sync,
7743        num_pubkeys: usize,
7744        pubkeys_removed_from_accounts_index: &'a PubkeysRemovedFromAccountsIndex,
7745    ) {
7746        let batches = 1 + (num_pubkeys / UNREF_ACCOUNTS_BATCH_SIZE);
7747        self.thread_pool_clean.install(|| {
7748            (0..batches).into_par_iter().for_each(|batch| {
7749                let skip = batch * UNREF_ACCOUNTS_BATCH_SIZE;
7750                self.accounts_index.scan(
7751                    pubkeys
7752                        .clone()
7753                        .skip(skip)
7754                        .take(UNREF_ACCOUNTS_BATCH_SIZE)
7755                        .filter(|pubkey| {
7756                            // filter out pubkeys that have already been removed from the accounts index in a previous step
7757                            let already_removed =
7758                                pubkeys_removed_from_accounts_index.contains(pubkey);
7759                            !already_removed
7760                        }),
7761                    |_pubkey, _slots_refs, _entry| {
7762                        /* unused */
7763                        AccountsIndexScanResult::Unref
7764                    },
7765                    Some(AccountsIndexScanResult::Unref),
7766                    false,
7767                    ScanFilter::All,
7768                )
7769            });
7770        });
7771    }
7772
7773    /// lookup each pubkey in 'purged_slot_pubkeys' and unref it in the accounts index
7774    /// populate 'purged_stored_account_slots' by grouping 'purged_slot_pubkeys' by pubkey
7775    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7776    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7777    fn unref_accounts(
7778        &self,
7779        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7780        purged_stored_account_slots: &mut AccountSlots,
7781        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7782    ) {
7783        self.unref_pubkeys(
7784            purged_slot_pubkeys.iter().map(|(_slot, pubkey)| pubkey),
7785            purged_slot_pubkeys.len(),
7786            pubkeys_removed_from_accounts_index,
7787        );
7788        for (slot, pubkey) in purged_slot_pubkeys {
7789            purged_stored_account_slots
7790                .entry(pubkey)
7791                .or_default()
7792                .insert(slot);
7793        }
7794    }
7795
7796    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7797    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7798    fn clean_dead_slots_from_accounts_index<'a>(
7799        &'a self,
7800        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
7801        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7802        // Should only be `Some` for non-cached slots
7803        purged_stored_account_slots: Option<&mut AccountSlots>,
7804        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7805    ) {
7806        let mut accounts_index_root_stats = AccountsIndexRootsStats::default();
7807        let mut measure = Measure::start("unref_from_storage");
7808        if let Some(purged_stored_account_slots) = purged_stored_account_slots {
7809            self.unref_accounts(
7810                purged_slot_pubkeys,
7811                purged_stored_account_slots,
7812                pubkeys_removed_from_accounts_index,
7813            );
7814        }
7815        measure.stop();
7816        accounts_index_root_stats.clean_unref_from_storage_us += measure.as_us();
7817
7818        let mut measure = Measure::start("clean_dead_slot");
7819        let mut rooted_cleaned_count = 0;
7820        let mut unrooted_cleaned_count = 0;
7821        let dead_slots: Vec<_> = dead_slots_iter
7822            .map(|slot| {
7823                if self.accounts_index.clean_dead_slot(*slot) {
7824                    rooted_cleaned_count += 1;
7825                } else {
7826                    unrooted_cleaned_count += 1;
7827                }
7828                *slot
7829            })
7830            .collect();
7831        measure.stop();
7832        accounts_index_root_stats.clean_dead_slot_us += measure.as_us();
7833        if self.log_dead_slots.load(Ordering::Relaxed) {
7834            info!(
7835                "remove_dead_slots_metadata: {} dead slots",
7836                dead_slots.len()
7837            );
7838            trace!("remove_dead_slots_metadata: dead_slots: {:?}", dead_slots);
7839        }
7840        self.accounts_index
7841            .update_roots_stats(&mut accounts_index_root_stats);
7842        accounts_index_root_stats.rooted_cleaned_count += rooted_cleaned_count;
7843        accounts_index_root_stats.unrooted_cleaned_count += unrooted_cleaned_count;
7844
7845        self.clean_accounts_stats
7846            .latest_accounts_index_roots_stats
7847            .update(&accounts_index_root_stats);
7848    }
7849
7850    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7851    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7852    fn clean_stored_dead_slots(
7853        &self,
7854        dead_slots: &IntSet<Slot>,
7855        purged_account_slots: Option<&mut AccountSlots>,
7856        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7857    ) {
7858        let mut measure = Measure::start("clean_stored_dead_slots-ms");
7859        let mut stores = vec![];
7860        // get all stores in a vec so we can iterate in parallel
7861        for slot in dead_slots.iter() {
7862            if let Some(slot_storage) = self.storage.get_slot_storage_entry(*slot) {
7863                stores.push(slot_storage);
7864            }
7865        }
7866        // get all pubkeys in all dead slots
7867        let purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = {
7868            self.thread_pool_clean.install(|| {
7869                stores
7870                    .into_par_iter()
7871                    .map(|store| {
7872                        let slot = store.slot();
7873                        let mut pubkeys = Vec::with_capacity(store.count());
7874                        store.accounts.scan_pubkeys(|pubkey| {
7875                            pubkeys.push((slot, *pubkey));
7876                        });
7877                        pubkeys
7878                    })
7879                    .flatten()
7880                    .collect::<HashSet<_>>()
7881            })
7882        };
7883        self.remove_dead_slots_metadata(
7884            dead_slots.iter(),
7885            purged_slot_pubkeys,
7886            purged_account_slots,
7887            pubkeys_removed_from_accounts_index,
7888        );
7889        measure.stop();
7890        self.clean_accounts_stats
7891            .clean_stored_dead_slots_us
7892            .fetch_add(measure.as_us(), Ordering::Relaxed);
7893    }
7894
7895    pub fn store_cached<'a>(
7896        &self,
7897        accounts: impl StorableAccounts<'a>,
7898        transactions: Option<&'a [&'a SanitizedTransaction]>,
7899    ) {
7900        self.store(
7901            accounts,
7902            &StoreTo::Cache,
7903            transactions,
7904            StoreReclaims::Default,
7905            UpdateIndexThreadSelection::PoolWithThreshold,
7906        );
7907    }
7908
7909    pub(crate) fn store_cached_inline_update_index<'a>(
7910        &self,
7911        accounts: impl StorableAccounts<'a>,
7912        transactions: Option<&'a [&'a SanitizedTransaction]>,
7913    ) {
7914        self.store(
7915            accounts,
7916            &StoreTo::Cache,
7917            transactions,
7918            StoreReclaims::Default,
7919            UpdateIndexThreadSelection::Inline,
7920        );
7921    }
7922
7923    /// Store the account update.
7924    /// only called by tests
7925    pub fn store_uncached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
7926        let storage = self.find_storage_candidate(slot);
7927        self.store(
7928            (slot, accounts),
7929            &StoreTo::Storage(&storage),
7930            None,
7931            StoreReclaims::Default,
7932            UpdateIndexThreadSelection::PoolWithThreshold,
7933        );
7934    }
7935
7936    fn store<'a>(
7937        &self,
7938        accounts: impl StorableAccounts<'a>,
7939        store_to: &StoreTo,
7940        transactions: Option<&'a [&'a SanitizedTransaction]>,
7941        reclaim: StoreReclaims,
7942        update_index_thread_selection: UpdateIndexThreadSelection,
7943    ) {
7944        // If all transactions in a batch are errored,
7945        // it's possible to get a store with no accounts.
7946        if accounts.is_empty() {
7947            return;
7948        }
7949
7950        let mut stats = BankHashStats::default();
7951        let mut total_data = 0;
7952        (0..accounts.len()).for_each(|index| {
7953            accounts.account(index, |account| {
7954                total_data += account.data().len();
7955                stats.update(&account);
7956            })
7957        });
7958
7959        self.stats
7960            .store_total_data
7961            .fetch_add(total_data as u64, Ordering::Relaxed);
7962
7963        {
7964            // we need to drop the bank_hash_stats lock to prevent deadlocks
7965            self.bank_hash_stats
7966                .lock()
7967                .unwrap()
7968                .entry(accounts.target_slot())
7969                .or_default()
7970                .accumulate(&stats);
7971        }
7972
7973        // we use default hashes for now since the same account may be stored to the cache multiple times
7974        self.store_accounts_unfrozen(
7975            accounts,
7976            store_to,
7977            transactions,
7978            reclaim,
7979            update_index_thread_selection,
7980        );
7981        self.report_store_timings();
7982    }
7983
7984    fn report_store_timings(&self) {
7985        if self.stats.last_store_report.should_update(1000) {
7986            let read_cache_stats = self.read_only_accounts_cache.get_and_reset_stats();
7987            datapoint_info!(
7988                "accounts_db_store_timings",
7989                (
7990                    "hash_accounts",
7991                    self.stats.store_hash_accounts.swap(0, Ordering::Relaxed),
7992                    i64
7993                ),
7994                (
7995                    "store_accounts",
7996                    self.stats.store_accounts.swap(0, Ordering::Relaxed),
7997                    i64
7998                ),
7999                (
8000                    "update_index",
8001                    self.stats.store_update_index.swap(0, Ordering::Relaxed),
8002                    i64
8003                ),
8004                (
8005                    "handle_reclaims",
8006                    self.stats.store_handle_reclaims.swap(0, Ordering::Relaxed),
8007                    i64
8008                ),
8009                (
8010                    "append_accounts",
8011                    self.stats.store_append_accounts.swap(0, Ordering::Relaxed),
8012                    i64
8013                ),
8014                (
8015                    "stakes_cache_check_and_store_us",
8016                    self.stats
8017                        .stakes_cache_check_and_store_us
8018                        .swap(0, Ordering::Relaxed),
8019                    i64
8020                ),
8021                (
8022                    "num_accounts",
8023                    self.stats.store_num_accounts.swap(0, Ordering::Relaxed),
8024                    i64
8025                ),
8026                (
8027                    "total_data",
8028                    self.stats.store_total_data.swap(0, Ordering::Relaxed),
8029                    i64
8030                ),
8031                (
8032                    "read_only_accounts_cache_entries",
8033                    self.read_only_accounts_cache.cache_len(),
8034                    i64
8035                ),
8036                (
8037                    "read_only_accounts_cache_data_size",
8038                    self.read_only_accounts_cache.data_size(),
8039                    i64
8040                ),
8041                ("read_only_accounts_cache_hits", read_cache_stats.hits, i64),
8042                (
8043                    "read_only_accounts_cache_misses",
8044                    read_cache_stats.misses,
8045                    i64
8046                ),
8047                (
8048                    "read_only_accounts_cache_evicts",
8049                    read_cache_stats.evicts,
8050                    i64
8051                ),
8052                (
8053                    "read_only_accounts_cache_load_us",
8054                    read_cache_stats.load_us,
8055                    i64
8056                ),
8057                (
8058                    "read_only_accounts_cache_store_us",
8059                    read_cache_stats.store_us,
8060                    i64
8061                ),
8062                (
8063                    "read_only_accounts_cache_evict_us",
8064                    read_cache_stats.evict_us,
8065                    i64
8066                ),
8067                (
8068                    "read_only_accounts_cache_evictor_wakeup_count_all",
8069                    read_cache_stats.evictor_wakeup_count_all,
8070                    i64
8071                ),
8072                (
8073                    "read_only_accounts_cache_evictor_wakeup_count_productive",
8074                    read_cache_stats.evictor_wakeup_count_productive,
8075                    i64
8076                ),
8077                (
8078                    "calc_stored_meta_us",
8079                    self.stats.calc_stored_meta.swap(0, Ordering::Relaxed),
8080                    i64
8081                ),
8082                (
8083                    "handle_dead_keys_us",
8084                    self.stats.handle_dead_keys_us.swap(0, Ordering::Relaxed),
8085                    i64
8086                ),
8087                (
8088                    "purge_exact_us",
8089                    self.stats.purge_exact_us.swap(0, Ordering::Relaxed),
8090                    i64
8091                ),
8092                (
8093                    "purge_exact_count",
8094                    self.stats.purge_exact_count.swap(0, Ordering::Relaxed),
8095                    i64
8096                ),
8097            );
8098
8099            datapoint_info!(
8100                "accounts_db_store_timings2",
8101                (
8102                    "create_store_count",
8103                    self.stats.create_store_count.swap(0, Ordering::Relaxed),
8104                    i64
8105                ),
8106                (
8107                    "store_get_slot_store",
8108                    self.stats.store_get_slot_store.swap(0, Ordering::Relaxed),
8109                    i64
8110                ),
8111                (
8112                    "store_find_existing",
8113                    self.stats.store_find_existing.swap(0, Ordering::Relaxed),
8114                    i64
8115                ),
8116                (
8117                    "dropped_stores",
8118                    self.stats.dropped_stores.swap(0, Ordering::Relaxed),
8119                    i64
8120                ),
8121            );
8122        }
8123    }
8124
8125    fn store_accounts_unfrozen<'a>(
8126        &self,
8127        accounts: impl StorableAccounts<'a>,
8128        store_to: &StoreTo,
8129        transactions: Option<&'a [&'a SanitizedTransaction]>,
8130        reclaim: StoreReclaims,
8131        update_index_thread_selection: UpdateIndexThreadSelection,
8132    ) {
8133        // This path comes from a store to a non-frozen slot.
8134        // If a store is dead here, then a newer update for
8135        // each pubkey in the store must exist in another
8136        // store in the slot. Thus it is safe to reset the store and
8137        // re-use it for a future store op. The pubkey ref counts should still
8138        // hold just 1 ref from this slot.
8139        let reset_accounts = true;
8140
8141        self.store_accounts_custom(
8142            accounts,
8143            store_to,
8144            reset_accounts,
8145            transactions,
8146            reclaim,
8147            update_index_thread_selection,
8148        );
8149    }
8150
8151    pub fn store_accounts_frozen<'a>(
8152        &self,
8153        accounts: impl StorableAccounts<'a>,
8154        storage: &Arc<AccountStorageEntry>,
8155    ) -> StoreAccountsTiming {
8156        // stores on a frozen slot should not reset
8157        // the append vec so that hashing could happen on the store
8158        // and accounts in the append_vec can be unrefed correctly
8159        let reset_accounts = false;
8160        self.store_accounts_custom(
8161            accounts,
8162            &StoreTo::Storage(storage),
8163            reset_accounts,
8164            None,
8165            StoreReclaims::Ignore,
8166            UpdateIndexThreadSelection::PoolWithThreshold,
8167        )
8168    }
8169
8170    fn store_accounts_custom<'a>(
8171        &self,
8172        accounts: impl StorableAccounts<'a>,
8173        store_to: &StoreTo,
8174        reset_accounts: bool,
8175        transactions: Option<&'a [&'a SanitizedTransaction]>,
8176        reclaim: StoreReclaims,
8177        update_index_thread_selection: UpdateIndexThreadSelection,
8178    ) -> StoreAccountsTiming {
8179        self.stats
8180            .store_num_accounts
8181            .fetch_add(accounts.len() as u64, Ordering::Relaxed);
8182        let mut store_accounts_time = Measure::start("store_accounts");
8183        let infos = self.store_accounts_to(&accounts, store_to, transactions);
8184        store_accounts_time.stop();
8185        self.stats
8186            .store_accounts
8187            .fetch_add(store_accounts_time.as_us(), Ordering::Relaxed);
8188        let mut update_index_time = Measure::start("update_index");
8189
8190        let reclaim = if matches!(reclaim, StoreReclaims::Ignore) {
8191            UpsertReclaim::IgnoreReclaims
8192        } else if store_to.is_cached() {
8193            UpsertReclaim::PreviousSlotEntryWasCached
8194        } else {
8195            UpsertReclaim::PopulateReclaims
8196        };
8197
8198        // if we are squashing a single slot, then we can expect a single dead slot
8199        let expected_single_dead_slot =
8200            (!accounts.contains_multiple_slots()).then(|| accounts.target_slot());
8201
8202        // If the cache was flushed, then because `update_index` occurs
8203        // after the account are stored by the above `store_accounts_to`
8204        // call and all the accounts are stored, all reads after this point
8205        // will know to not check the cache anymore
8206        let mut reclaims =
8207            self.update_index(infos, &accounts, reclaim, update_index_thread_selection);
8208
8209        // For each updated account, `reclaims` should only have at most one
8210        // item (if the account was previously updated in this slot).
8211        // filter out the cached reclaims as those don't actually map
8212        // to anything that needs to be cleaned in the backing storage
8213        // entries
8214        reclaims.retain(|(_, r)| !r.is_cached());
8215
8216        if store_to.is_cached() {
8217            assert!(reclaims.is_empty());
8218        }
8219
8220        update_index_time.stop();
8221        self.stats
8222            .store_update_index
8223            .fetch_add(update_index_time.as_us(), Ordering::Relaxed);
8224
8225        // A store for a single slot should:
8226        // 1) Only make "reclaims" for the same slot
8227        // 2) Should not cause any slots to be removed from the storage
8228        // database because
8229        //    a) this slot  has at least one account (the one being stored),
8230        //    b)From 1) we know no other slots are included in the "reclaims"
8231        //
8232        // From 1) and 2) we guarantee passing `no_purge_stats` == None, which is
8233        // equivalent to asserting there will be no dead slots, is safe.
8234        let mut handle_reclaims_elapsed = 0;
8235        if reclaim == UpsertReclaim::PopulateReclaims {
8236            let mut handle_reclaims_time = Measure::start("handle_reclaims");
8237            self.handle_reclaims(
8238                (!reclaims.is_empty()).then(|| reclaims.iter()),
8239                expected_single_dead_slot,
8240                reset_accounts,
8241                &HashSet::default(),
8242                // this callsite does NOT process dead slots
8243                HandleReclaims::DoNotProcessDeadSlots,
8244            );
8245            handle_reclaims_time.stop();
8246            handle_reclaims_elapsed = handle_reclaims_time.as_us();
8247            self.stats
8248                .store_handle_reclaims
8249                .fetch_add(handle_reclaims_elapsed, Ordering::Relaxed);
8250        } else {
8251            assert!(reclaims.is_empty());
8252        }
8253
8254        StoreAccountsTiming {
8255            store_accounts_elapsed: store_accounts_time.as_us(),
8256            update_index_elapsed: update_index_time.as_us(),
8257            handle_reclaims_elapsed,
8258        }
8259    }
8260
8261    pub fn add_root(&self, slot: Slot) -> AccountsAddRootTiming {
8262        let mut index_time = Measure::start("index_add_root");
8263        self.accounts_index.add_root(slot);
8264        index_time.stop();
8265        let mut cache_time = Measure::start("cache_add_root");
8266        self.accounts_cache.add_root(slot);
8267        cache_time.stop();
8268        let mut store_time = Measure::start("store_add_root");
8269        // We would not expect this slot to be shrinking right now, but other slots may be.
8270        // But, even if it was, we would just mark a store id as dirty unnecessarily and that is ok.
8271        // So, allow shrinking to be in progress.
8272        if let Some(store) = self
8273            .storage
8274            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
8275        {
8276            self.dirty_stores.insert(slot, store);
8277        }
8278        store_time.stop();
8279
8280        AccountsAddRootTiming {
8281            index_us: index_time.as_us(),
8282            cache_us: cache_time.as_us(),
8283            store_us: store_time.as_us(),
8284        }
8285    }
8286
8287    /// Get storages to use for snapshots, for the requested slots
8288    pub fn get_snapshot_storages(
8289        &self,
8290        requested_slots: impl RangeBounds<Slot> + Sync,
8291    ) -> (Vec<Arc<AccountStorageEntry>>, Vec<Slot>) {
8292        let start = Instant::now();
8293        let max_alive_root_exclusive = self
8294            .accounts_index
8295            .roots_tracker
8296            .read()
8297            .unwrap()
8298            .alive_roots
8299            .max_exclusive();
8300        let (slots, storages) = self
8301            .storage
8302            .get_if(|slot, storage| {
8303                (*slot < max_alive_root_exclusive)
8304                    && requested_slots.contains(slot)
8305                    && storage.has_accounts()
8306            })
8307            .into_vec()
8308            .into_iter()
8309            .unzip();
8310        let duration = start.elapsed();
8311        debug!("get_snapshot_storages: {duration:?}");
8312        (storages, slots)
8313    }
8314
8315    /// Returns the latest full snapshot slot
8316    pub fn latest_full_snapshot_slot(&self) -> Option<Slot> {
8317        self.latest_full_snapshot_slot.read()
8318    }
8319
8320    /// Sets the latest full snapshot slot to `slot`
8321    pub fn set_latest_full_snapshot_slot(&self, slot: Slot) {
8322        *self.latest_full_snapshot_slot.lock_write() = Some(slot);
8323    }
8324
8325    /// return Some(lamports_to_top_off) if 'account' would collect rent
8326    fn stats_for_rent_payers(
8327        pubkey: &Pubkey,
8328        lamports: u64,
8329        account_data_len: usize,
8330        account_rent_epoch: Epoch,
8331        executable: bool,
8332        rent_collector: &RentCollector,
8333    ) -> Option<u64> {
8334        if lamports == 0 {
8335            return None;
8336        }
8337        (rent_collector.should_collect_rent(pubkey, executable)
8338            && !rent_collector
8339                .get_rent_due(lamports, account_data_len, account_rent_epoch)
8340                .is_exempt())
8341        .then(|| {
8342            let min_balance = rent_collector.rent.minimum_balance(account_data_len);
8343            // return lamports required to top off this account to make it rent exempt
8344            min_balance.saturating_sub(lamports)
8345        })
8346    }
8347
8348    fn generate_index_for_slot(
8349        &self,
8350        storage: &AccountStorageEntry,
8351        slot: Slot,
8352        store_id: AccountsFileId,
8353        rent_collector: &RentCollector,
8354        storage_info: &StorageSizeAndCountMap,
8355    ) -> SlotIndexGenerationInfo {
8356        if storage.accounts.get_account_sizes(&[0]).is_empty() {
8357            return SlotIndexGenerationInfo::default();
8358        }
8359        let secondary = !self.account_indexes.is_empty();
8360
8361        let mut rent_paying_accounts_by_partition = Vec::default();
8362        let mut accounts_data_len = 0;
8363        let mut num_accounts_rent_paying = 0;
8364        let mut amount_to_top_off_rent = 0;
8365        let mut stored_size_alive = 0;
8366        let mut all_accounts_are_zero_lamports = true;
8367
8368        let (dirty_pubkeys, insert_time_us, mut generate_index_results) = {
8369            let mut items_local = Vec::default();
8370            storage.accounts.scan_index(|info| {
8371                stored_size_alive += info.stored_size_aligned;
8372                if info.index_info.lamports > 0 {
8373                    accounts_data_len += info.index_info.data_len;
8374                    all_accounts_are_zero_lamports = false;
8375                }
8376                items_local.push(info.index_info);
8377            });
8378
8379            let items_len = items_local.len();
8380            let items = items_local.into_iter().map(|info| {
8381                if let Some(amount_to_top_off_rent_this_account) = Self::stats_for_rent_payers(
8382                    &info.pubkey,
8383                    info.lamports,
8384                    info.data_len as usize,
8385                    info.rent_epoch,
8386                    info.executable,
8387                    rent_collector,
8388                ) {
8389                    amount_to_top_off_rent += amount_to_top_off_rent_this_account;
8390                    num_accounts_rent_paying += 1;
8391                    // remember this rent-paying account pubkey
8392                    rent_paying_accounts_by_partition.push(info.pubkey);
8393                }
8394
8395                (
8396                    info.pubkey,
8397                    AccountInfo::new(
8398                        StorageLocation::AppendVec(store_id, info.offset), // will never be cached
8399                        info.lamports,
8400                    ),
8401                )
8402            });
8403            self.accounts_index
8404                .insert_new_if_missing_into_primary_index(slot, items_len, items)
8405        };
8406        if secondary {
8407            // scan storage a second time to update the secondary index
8408            storage.accounts.scan_accounts(|stored_account| {
8409                stored_size_alive += stored_account.stored_size();
8410                let pubkey = stored_account.pubkey();
8411                self.accounts_index.update_secondary_indexes(
8412                    pubkey,
8413                    &stored_account,
8414                    &self.account_indexes,
8415                );
8416            });
8417        }
8418
8419        if let Some(duplicates_this_slot) = std::mem::take(&mut generate_index_results.duplicates) {
8420            // there were duplicate pubkeys in this same slot
8421            // Some were not inserted. This means some info like stored data is off.
8422            duplicates_this_slot
8423                .into_iter()
8424                .for_each(|(pubkey, (_slot, info))| {
8425                    storage
8426                        .accounts
8427                        .get_stored_account_meta_callback(info.offset(), |duplicate| {
8428                            assert_eq!(&pubkey, duplicate.pubkey());
8429                            stored_size_alive =
8430                                stored_size_alive.saturating_sub(duplicate.stored_size());
8431                            if !duplicate.is_zero_lamport() {
8432                                accounts_data_len =
8433                                    accounts_data_len.saturating_sub(duplicate.data().len() as u64);
8434                            }
8435                        });
8436                });
8437        }
8438
8439        {
8440            // second, collect into the shared DashMap once we've figured out all the info per store_id
8441            let mut info = storage_info.entry(store_id).or_default();
8442            info.stored_size += stored_size_alive;
8443            info.count += generate_index_results.count;
8444        }
8445
8446        // dirty_pubkeys will contain a pubkey if an item has multiple rooted entries for
8447        // a given pubkey. If there is just a single item, there is no cleaning to
8448        // be done on that pubkey. Use only those pubkeys with multiple updates.
8449        if !dirty_pubkeys.is_empty() {
8450            self.uncleaned_pubkeys.insert(slot, dirty_pubkeys);
8451        }
8452        SlotIndexGenerationInfo {
8453            insert_time_us,
8454            num_accounts: generate_index_results.count as u64,
8455            num_accounts_rent_paying,
8456            accounts_data_len,
8457            amount_to_top_off_rent,
8458            rent_paying_accounts_by_partition,
8459            all_accounts_are_zero_lamports,
8460        }
8461    }
8462
8463    pub fn generate_index(
8464        &self,
8465        limit_load_slot_count_from_snapshot: Option<usize>,
8466        verify: bool,
8467        genesis_config: &GenesisConfig,
8468    ) -> IndexGenerationInfo {
8469        let mut total_time = Measure::start("generate_index");
8470        let mut slots = self.storage.all_slots();
8471        slots.sort_unstable();
8472        if let Some(limit) = limit_load_slot_count_from_snapshot {
8473            slots.truncate(limit); // get rid of the newer slots and keep just the older
8474        }
8475        let max_slot = slots.last().cloned().unwrap_or_default();
8476        let schedule = &genesis_config.epoch_schedule;
8477        let rent_collector = RentCollector::new(
8478            schedule.get_epoch(max_slot),
8479            schedule.clone(),
8480            genesis_config.slots_per_year(),
8481            genesis_config.rent.clone(),
8482        );
8483        let accounts_data_len = AtomicU64::new(0);
8484
8485        let rent_paying_accounts_by_partition =
8486            Mutex::new(RentPayingAccountsByPartition::new(schedule));
8487        let mut outer_duplicates_lt_hash = None;
8488
8489        // pass == 0 always runs and generates the index
8490        // pass == 1 only runs if verify == true.
8491        // verify checks that all the expected items are in the accounts index and measures how long it takes to look them all up
8492        let passes = if verify { 2 } else { 1 };
8493        for pass in 0..passes {
8494            if pass == 0 {
8495                self.accounts_index
8496                    .set_startup(Startup::StartupWithExtraThreads);
8497            }
8498            let storage_info = StorageSizeAndCountMap::default();
8499            let total_processed_slots_across_all_threads = AtomicU64::new(0);
8500            let outer_slots_len = slots.len();
8501            let threads = if self.accounts_index.is_disk_index_enabled() {
8502                // these write directly to disk, so the more threads, the better
8503                num_cpus::get()
8504            } else {
8505                // seems to be a good heuristic given varying # cpus for in-mem disk index
8506                8
8507            };
8508            let chunk_size = (outer_slots_len / (std::cmp::max(1, threads.saturating_sub(1)))) + 1; // approximately 400k slots in a snapshot
8509            let mut index_time = Measure::start("index");
8510            let insertion_time_us = AtomicU64::new(0);
8511            let rent_paying = AtomicUsize::new(0);
8512            let amount_to_top_off_rent = AtomicU64::new(0);
8513            let total_including_duplicates = AtomicU64::new(0);
8514            let all_accounts_are_zero_lamports_slots = AtomicU64::new(0);
8515            let mut all_zeros_slots = Mutex::new(Vec::<(Slot, Arc<AccountStorageEntry>)>::new());
8516            let scan_time: u64 = slots
8517                .par_chunks(chunk_size)
8518                .map(|slots| {
8519                    let mut log_status = MultiThreadProgress::new(
8520                        &total_processed_slots_across_all_threads,
8521                        2,
8522                        outer_slots_len as u64,
8523                    );
8524                    let mut scan_time_sum = 0;
8525                    let mut all_accounts_are_zero_lamports_slots_inner = 0;
8526                    let mut all_zeros_slots_inner = vec![];
8527                    let mut insert_time_sum = 0;
8528                    let mut total_including_duplicates_sum = 0;
8529                    let mut accounts_data_len_sum = 0;
8530                    for (index, slot) in slots.iter().enumerate() {
8531                        let mut scan_time = Measure::start("scan");
8532                        log_status.report(index as u64);
8533                        let Some(storage) = self.storage.get_slot_storage_entry(*slot) else {
8534                            // no storage at this slot, no information to pull out
8535                            continue;
8536                        };
8537                        let store_id = storage.id();
8538
8539                        scan_time.stop();
8540                        scan_time_sum += scan_time.as_us();
8541
8542                        let insert_us = if pass == 0 {
8543                            // generate index
8544                            self.maybe_throttle_index_generation();
8545                            let SlotIndexGenerationInfo {
8546                                insert_time_us: insert_us,
8547                                num_accounts: total_this_slot,
8548                                num_accounts_rent_paying: rent_paying_this_slot,
8549                                accounts_data_len: accounts_data_len_this_slot,
8550                                amount_to_top_off_rent: amount_to_top_off_rent_this_slot,
8551                                rent_paying_accounts_by_partition:
8552                                    rent_paying_accounts_by_partition_this_slot,
8553                                all_accounts_are_zero_lamports,
8554                            } = self.generate_index_for_slot(
8555                                &storage,
8556                                *slot,
8557                                store_id,
8558                                &rent_collector,
8559                                &storage_info,
8560                            );
8561
8562                            if rent_paying_this_slot > 0 {
8563                                // We don't have any rent paying accounts on mainnet, so this code should never be hit.
8564                                rent_paying.fetch_add(rent_paying_this_slot, Ordering::Relaxed);
8565                                amount_to_top_off_rent
8566                                    .fetch_add(amount_to_top_off_rent_this_slot, Ordering::Relaxed);
8567                                let mut rent_paying_accounts_by_partition =
8568                                    rent_paying_accounts_by_partition.lock().unwrap();
8569                                rent_paying_accounts_by_partition_this_slot
8570                                    .iter()
8571                                    .for_each(|k| {
8572                                        rent_paying_accounts_by_partition.add_account(k);
8573                                    });
8574                            }
8575                            total_including_duplicates_sum += total_this_slot;
8576                            accounts_data_len_sum += accounts_data_len_this_slot;
8577                            if all_accounts_are_zero_lamports {
8578                                all_accounts_are_zero_lamports_slots_inner += 1;
8579                                all_zeros_slots_inner.push((*slot, Arc::clone(&storage)));
8580                            }
8581
8582                            insert_us
8583                        } else {
8584                            // verify index matches expected and measure the time to get all items
8585                            assert!(verify);
8586                            let mut lookup_time = Measure::start("lookup_time");
8587                            storage.accounts.scan_accounts(|account_info| {
8588                                let key = account_info.pubkey();
8589                                let index_entry = self.accounts_index.get_cloned(key).unwrap();
8590                                let slot_list = index_entry.slot_list.read().unwrap();
8591                                let mut count = 0;
8592                                for (slot2, account_info2) in slot_list.iter() {
8593                                    if slot2 == slot {
8594                                        count += 1;
8595                                        let ai = AccountInfo::new(
8596                                            StorageLocation::AppendVec(
8597                                                store_id,
8598                                                account_info.offset(),
8599                                            ), // will never be cached
8600                                            account_info.lamports(),
8601                                        );
8602                                        assert_eq!(&ai, account_info2);
8603                                    }
8604                                }
8605                                assert_eq!(1, count);
8606                            });
8607                            lookup_time.stop();
8608                            lookup_time.as_us()
8609                        };
8610                        insert_time_sum += insert_us;
8611                    }
8612                    all_accounts_are_zero_lamports_slots.fetch_add(
8613                        all_accounts_are_zero_lamports_slots_inner,
8614                        Ordering::Relaxed,
8615                    );
8616                    all_zeros_slots
8617                        .lock()
8618                        .unwrap()
8619                        .append(&mut all_zeros_slots_inner);
8620                    insertion_time_us.fetch_add(insert_time_sum, Ordering::Relaxed);
8621                    total_including_duplicates
8622                        .fetch_add(total_including_duplicates_sum, Ordering::Relaxed);
8623                    accounts_data_len.fetch_add(accounts_data_len_sum, Ordering::Relaxed);
8624                    scan_time_sum
8625                })
8626                .sum();
8627            index_time.stop();
8628
8629            info!("rent_collector: {:?}", rent_collector);
8630
8631            let mut index_flush_us = 0;
8632            let total_duplicate_slot_keys = AtomicU64::default();
8633            let mut populate_duplicate_keys_us = 0;
8634            let mut total_items_in_mem = 0;
8635            let mut min_bin_size_in_mem = 0;
8636            let mut max_bin_size_in_mem = 0;
8637            let total_num_unique_duplicate_keys = AtomicU64::default();
8638
8639            // outer vec is accounts index bin (determined by pubkey value)
8640            // inner vec is the pubkeys within that bin that are present in > 1 slot
8641            let unique_pubkeys_by_bin = Mutex::new(Vec::<Vec<Pubkey>>::default());
8642            if pass == 0 {
8643                // tell accounts index we are done adding the initial accounts at startup
8644                let mut m = Measure::start("accounts_index_idle_us");
8645                self.accounts_index.set_startup(Startup::Normal);
8646                m.stop();
8647                index_flush_us = m.as_us();
8648
8649                populate_duplicate_keys_us = measure_us!({
8650                    // this has to happen before visit_duplicate_pubkeys_during_startup below
8651                    // get duplicate keys from acct idx. We have to wait until we've finished flushing.
8652                    self.accounts_index
8653                        .populate_and_retrieve_duplicate_keys_from_startup(|slot_keys| {
8654                            total_duplicate_slot_keys
8655                                .fetch_add(slot_keys.len() as u64, Ordering::Relaxed);
8656                            let unique_keys =
8657                                HashSet::<Pubkey>::from_iter(slot_keys.iter().map(|(_, key)| *key));
8658                            for (slot, key) in slot_keys {
8659                                self.uncleaned_pubkeys.entry(slot).or_default().push(key);
8660                            }
8661                            let unique_pubkeys_by_bin_inner =
8662                                unique_keys.into_iter().collect::<Vec<_>>();
8663                            total_num_unique_duplicate_keys.fetch_add(
8664                                unique_pubkeys_by_bin_inner.len() as u64,
8665                                Ordering::Relaxed,
8666                            );
8667                            // does not matter that this is not ordered by slot
8668                            unique_pubkeys_by_bin
8669                                .lock()
8670                                .unwrap()
8671                                .push(unique_pubkeys_by_bin_inner);
8672                        });
8673                })
8674                .1;
8675
8676                (total_items_in_mem, min_bin_size_in_mem, max_bin_size_in_mem) = self
8677                    .accounts_index
8678                    .account_maps
8679                    .iter()
8680                    .map(|map_bin| map_bin.len_for_stats())
8681                    .fold((0, usize::MAX, usize::MIN), |acc, len| {
8682                        (
8683                            acc.0 + len,
8684                            std::cmp::min(acc.1, len),
8685                            std::cmp::max(acc.2, len),
8686                        )
8687                    });
8688            }
8689            let unique_pubkeys_by_bin = unique_pubkeys_by_bin.into_inner().unwrap();
8690
8691            let mut timings = GenerateIndexTimings {
8692                index_flush_us,
8693                scan_time,
8694                index_time: index_time.as_us(),
8695                insertion_time_us: insertion_time_us.load(Ordering::Relaxed),
8696                min_bin_size_in_mem,
8697                max_bin_size_in_mem,
8698                total_items_in_mem,
8699                rent_paying,
8700                amount_to_top_off_rent,
8701                total_duplicate_slot_keys: total_duplicate_slot_keys.load(Ordering::Relaxed),
8702                total_num_unique_duplicate_keys: total_num_unique_duplicate_keys
8703                    .load(Ordering::Relaxed),
8704                populate_duplicate_keys_us,
8705                total_including_duplicates: total_including_duplicates.load(Ordering::Relaxed),
8706                total_slots: slots.len() as u64,
8707                all_accounts_are_zero_lamports_slots: all_accounts_are_zero_lamports_slots
8708                    .load(Ordering::Relaxed),
8709                ..GenerateIndexTimings::default()
8710            };
8711
8712            if pass == 0 {
8713                #[derive(Debug, Default)]
8714                struct DuplicatePubkeysVisitedInfo {
8715                    accounts_data_len_from_duplicates: u64,
8716                    num_duplicate_accounts: u64,
8717                    uncleaned_roots: IntSet<Slot>,
8718                    duplicates_lt_hash: Box<DuplicatesLtHash>,
8719                }
8720                impl DuplicatePubkeysVisitedInfo {
8721                    fn reduce(mut a: Self, mut b: Self) -> Self {
8722                        if a.uncleaned_roots.len() >= b.uncleaned_roots.len() {
8723                            a.merge(b);
8724                            a
8725                        } else {
8726                            b.merge(a);
8727                            b
8728                        }
8729                    }
8730                    fn merge(&mut self, other: Self) {
8731                        self.accounts_data_len_from_duplicates +=
8732                            other.accounts_data_len_from_duplicates;
8733                        self.num_duplicate_accounts += other.num_duplicate_accounts;
8734                        self.uncleaned_roots.extend(other.uncleaned_roots);
8735                        self.duplicates_lt_hash
8736                            .0
8737                            .mix_in(&other.duplicates_lt_hash.0);
8738                    }
8739                }
8740
8741                // subtract data.len() from accounts_data_len for all old accounts that are in the index twice
8742                let mut accounts_data_len_dedup_timer =
8743                    Measure::start("handle accounts data len duplicates");
8744                let DuplicatePubkeysVisitedInfo {
8745                    accounts_data_len_from_duplicates,
8746                    num_duplicate_accounts,
8747                    uncleaned_roots,
8748                    duplicates_lt_hash,
8749                } = unique_pubkeys_by_bin
8750                    .par_iter()
8751                    .fold(
8752                        DuplicatePubkeysVisitedInfo::default,
8753                        |accum, pubkeys_by_bin| {
8754                            let intermediate = pubkeys_by_bin
8755                                .par_chunks(4096)
8756                                .fold(DuplicatePubkeysVisitedInfo::default, |accum, pubkeys| {
8757                                    let (
8758                                        accounts_data_len_from_duplicates,
8759                                        accounts_duplicates_num,
8760                                        uncleaned_roots,
8761                                        duplicates_lt_hash,
8762                                    ) = self.visit_duplicate_pubkeys_during_startup(
8763                                        pubkeys,
8764                                        &rent_collector,
8765                                        &timings,
8766                                    );
8767                                    let intermediate = DuplicatePubkeysVisitedInfo {
8768                                        accounts_data_len_from_duplicates,
8769                                        num_duplicate_accounts: accounts_duplicates_num,
8770                                        uncleaned_roots,
8771                                        duplicates_lt_hash,
8772                                    };
8773                                    DuplicatePubkeysVisitedInfo::reduce(accum, intermediate)
8774                                })
8775                                .reduce(
8776                                    DuplicatePubkeysVisitedInfo::default,
8777                                    DuplicatePubkeysVisitedInfo::reduce,
8778                                );
8779                            DuplicatePubkeysVisitedInfo::reduce(accum, intermediate)
8780                        },
8781                    )
8782                    .reduce(
8783                        DuplicatePubkeysVisitedInfo::default,
8784                        DuplicatePubkeysVisitedInfo::reduce,
8785                    );
8786                accounts_data_len_dedup_timer.stop();
8787                timings.accounts_data_len_dedup_time_us = accounts_data_len_dedup_timer.as_us();
8788                timings.slots_to_clean = uncleaned_roots.len() as u64;
8789                timings.num_duplicate_accounts = num_duplicate_accounts;
8790
8791                self.accounts_index
8792                    .add_uncleaned_roots(uncleaned_roots.into_iter());
8793                accounts_data_len.fetch_sub(accounts_data_len_from_duplicates, Ordering::Relaxed);
8794                if self.is_experimental_accumulator_hash_enabled() {
8795                    let old_val = outer_duplicates_lt_hash.replace(duplicates_lt_hash);
8796                    assert!(old_val.is_none());
8797                }
8798                info!(
8799                    "accounts data len: {}",
8800                    accounts_data_len.load(Ordering::Relaxed)
8801                );
8802
8803                // insert all zero lamport account storage into the dirty stores and add them into the uncleaned roots for clean to pick up
8804                let all_zero_slots_to_clean = std::mem::take(all_zeros_slots.get_mut().unwrap());
8805                info!(
8806                    "insert all zero slots to clean at startup {}",
8807                    all_zero_slots_to_clean.len()
8808                );
8809                for (slot, storage) in all_zero_slots_to_clean {
8810                    self.dirty_stores.insert(slot, storage);
8811                    self.accounts_index.add_uncleaned_roots([slot]);
8812                }
8813            }
8814
8815            if pass == 0 {
8816                // Need to add these last, otherwise older updates will be cleaned
8817                for root in &slots {
8818                    self.accounts_index.add_root(*root);
8819                }
8820
8821                self.set_storage_count_and_alive_bytes(storage_info, &mut timings);
8822            }
8823            total_time.stop();
8824            timings.total_time_us = total_time.as_us();
8825            timings.report(self.accounts_index.get_startup_stats());
8826        }
8827
8828        self.accounts_index.log_secondary_indexes();
8829
8830        IndexGenerationInfo {
8831            accounts_data_len: accounts_data_len.load(Ordering::Relaxed),
8832            rent_paying_accounts_by_partition: rent_paying_accounts_by_partition
8833                .into_inner()
8834                .unwrap(),
8835            duplicates_lt_hash: outer_duplicates_lt_hash,
8836        }
8837    }
8838
8839    /// Startup processes can consume large amounts of memory while inserting accounts into the index as fast as possible.
8840    /// Calling this can slow down the insertion process to allow flushing to disk to keep pace.
8841    fn maybe_throttle_index_generation(&self) {
8842        // This number is chosen to keep the initial ram usage sufficiently small
8843        // The process of generating the index is goverened entirely by how fast the disk index can be populated.
8844        // 10M accounts is sufficiently small that it will never have memory usage. It seems sufficiently large that it will provide sufficient performance.
8845        // Performance is measured by total time to generate the index.
8846        // 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.
8847        // 550M was straining a 384G machine at startup.
8848        // This is a tunable parameter that just needs to be small enough to keep the generation threads from overwhelming RAM and oom at startup.
8849        const LIMIT: usize = 10_000_000;
8850        while self
8851            .accounts_index
8852            .get_startup_remaining_items_to_flush_estimate()
8853            > LIMIT
8854        {
8855            // 10 ms is long enough to allow some flushing to occur before insertion is resumed.
8856            // callers of this are typically run in parallel, so many threads will be sleeping at different starting intervals, waiting to resume insertion.
8857            sleep(Duration::from_millis(10));
8858        }
8859    }
8860
8861    /// Used during generate_index() to:
8862    /// 1. get the _duplicate_ accounts data len from the given pubkeys
8863    /// 2. get the slots that contained duplicate pubkeys
8864    /// 3. update rent stats
8865    /// 4. build up the duplicates lt hash
8866    ///
8867    /// Note this should only be used when ALL entries in the accounts index are roots.
8868    ///
8869    /// returns tuple of:
8870    /// - data len sum of all older duplicates
8871    /// - number of duplicate accounts
8872    /// - slots that contained duplicate pubkeys
8873    /// - lt hash of duplicates
8874    fn visit_duplicate_pubkeys_during_startup(
8875        &self,
8876        pubkeys: &[Pubkey],
8877        rent_collector: &RentCollector,
8878        timings: &GenerateIndexTimings,
8879    ) -> (u64, u64, IntSet<Slot>, Box<DuplicatesLtHash>) {
8880        let mut accounts_data_len_from_duplicates = 0;
8881        let mut num_duplicate_accounts = 0_u64;
8882        let mut uncleaned_slots = IntSet::default();
8883        let mut duplicates_lt_hash = Box::new(DuplicatesLtHash::default());
8884        let mut removed_rent_paying = 0;
8885        let mut removed_top_off = 0;
8886        let mut lt_hash_time = Duration::default();
8887        self.accounts_index.scan(
8888            pubkeys.iter(),
8889            |pubkey, slots_refs, _entry| {
8890                if let Some((slot_list, _ref_count)) = slots_refs {
8891                    if slot_list.len() > 1 {
8892                        // Only the account data len in the highest slot should be used, and the rest are
8893                        // duplicates.  So find the max slot to keep.
8894                        // Then sum up the remaining data len, which are the duplicates.
8895                        // All of the slots need to go in the 'uncleaned_slots' list. For clean to work properly,
8896                        // the slot where duplicate accounts are found in the index need to be in 'uncleaned_slots' list, too.
8897                        let max = slot_list.iter().map(|(slot, _)| slot).max().unwrap();
8898                        slot_list.iter().for_each(|(slot, account_info)| {
8899                            uncleaned_slots.insert(*slot);
8900                            if slot == max {
8901                                // the info in 'max' is the most recent, current info for this pubkey
8902                                return;
8903                            }
8904                            let maybe_storage_entry = self
8905                                .storage
8906                                .get_account_storage_entry(*slot, account_info.store_id());
8907                            let mut accessor = LoadedAccountAccessor::Stored(
8908                                maybe_storage_entry.map(|entry| (entry, account_info.offset())),
8909                            );
8910                            accessor.check_and_get_loaded_account(|loaded_account| {
8911                                let data_len = loaded_account.data_len();
8912                                if loaded_account.lamports() > 0 {
8913                                    accounts_data_len_from_duplicates += data_len;
8914                                }
8915                                num_duplicate_accounts += 1;
8916                                if let Some(lamports_to_top_off) = Self::stats_for_rent_payers(
8917                                    pubkey,
8918                                    loaded_account.lamports(),
8919                                    data_len,
8920                                    loaded_account.rent_epoch(),
8921                                    loaded_account.executable(),
8922                                    rent_collector,
8923                                ) {
8924                                    removed_rent_paying += 1;
8925                                    removed_top_off += lamports_to_top_off;
8926                                }
8927                                if self.is_experimental_accumulator_hash_enabled() {
8928                                    let (_, duration) = meas_dur!({
8929                                        let account_lt_hash =
8930                                            Self::lt_hash_account(&loaded_account, pubkey);
8931                                        duplicates_lt_hash.0.mix_in(&account_lt_hash.0);
8932                                    });
8933                                    lt_hash_time += duration;
8934                                }
8935                            });
8936                        });
8937                    }
8938                }
8939                AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
8940            },
8941            None,
8942            false,
8943            ScanFilter::All,
8944        );
8945        timings
8946            .rent_paying
8947            .fetch_sub(removed_rent_paying, Ordering::Relaxed);
8948        timings
8949            .amount_to_top_off_rent
8950            .fetch_sub(removed_top_off, Ordering::Relaxed);
8951        timings
8952            .par_duplicates_lt_hash_us
8953            .fetch_add(lt_hash_time.as_micros() as u64, Ordering::Relaxed);
8954        (
8955            accounts_data_len_from_duplicates as u64,
8956            num_duplicate_accounts,
8957            uncleaned_slots,
8958            duplicates_lt_hash,
8959        )
8960    }
8961
8962    fn set_storage_count_and_alive_bytes(
8963        &self,
8964        stored_sizes_and_counts: StorageSizeAndCountMap,
8965        timings: &mut GenerateIndexTimings,
8966    ) {
8967        // store count and size for each storage
8968        let mut storage_size_storages_time = Measure::start("storage_size_storages");
8969        for (_slot, store) in self.storage.iter() {
8970            let id = store.id();
8971            // Should be default at this point
8972            assert_eq!(store.alive_bytes(), 0);
8973            if let Some(entry) = stored_sizes_and_counts.get(&id) {
8974                trace!(
8975                    "id: {} setting count: {} cur: {}",
8976                    id,
8977                    entry.count,
8978                    store.count(),
8979                );
8980                {
8981                    let mut count_and_status = store.count_and_status.lock_write();
8982                    assert_eq!(count_and_status.0, 0);
8983                    count_and_status.0 = entry.count;
8984                }
8985                store
8986                    .alive_bytes
8987                    .store(entry.stored_size, Ordering::Release);
8988            } else {
8989                trace!("id: {} clearing count", id);
8990                store.count_and_status.lock_write().0 = 0;
8991            }
8992        }
8993        storage_size_storages_time.stop();
8994        timings.storage_size_storages_us = storage_size_storages_time.as_us();
8995    }
8996
8997    pub fn print_accounts_stats(&self, label: &str) {
8998        self.print_index(label);
8999        self.print_count_and_status(label);
9000    }
9001
9002    fn print_index(&self, label: &str) {
9003        let mut alive_roots: Vec<_> = self.accounts_index.all_alive_roots();
9004        #[allow(clippy::stable_sort_primitive)]
9005        alive_roots.sort();
9006        info!("{}: accounts_index alive_roots: {:?}", label, alive_roots,);
9007        let full_pubkey_range = Pubkey::from([0; 32])..=Pubkey::from([0xff; 32]);
9008
9009        self.accounts_index.account_maps.iter().for_each(|map| {
9010            for (pubkey, account_entry) in map.items(&full_pubkey_range) {
9011                info!("  key: {} ref_count: {}", pubkey, account_entry.ref_count(),);
9012                info!(
9013                    "      slots: {:?}",
9014                    *account_entry.slot_list.read().unwrap()
9015                );
9016            }
9017        });
9018    }
9019
9020    pub fn print_count_and_status(&self, label: &str) {
9021        let mut slots: Vec<_> = self.storage.all_slots();
9022        #[allow(clippy::stable_sort_primitive)]
9023        slots.sort();
9024        info!("{}: count_and status for {} slots:", label, slots.len());
9025        for slot in &slots {
9026            let entry = self.storage.get_slot_storage_entry(*slot).unwrap();
9027            info!(
9028                "  slot: {} id: {} count_and_status: {:?} len: {} capacity: {}",
9029                slot,
9030                entry.id(),
9031                entry.count_and_status.read(),
9032                entry.accounts.len(),
9033                entry.accounts.capacity(),
9034            );
9035        }
9036    }
9037}
9038
9039/// Specify the source of the accounts data when calculating the accounts hash
9040///
9041/// Using the Index is meant for testing the hash calculation itself and debugging;
9042/// not intended during normal validator operation.
9043#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9044pub enum CalcAccountsHashDataSource {
9045    IndexForTests,
9046    Storages,
9047}
9048
9049#[derive(Debug, Copy, Clone)]
9050enum HandleReclaims<'a> {
9051    ProcessDeadSlots(&'a PurgeStats),
9052    DoNotProcessDeadSlots,
9053}
9054
9055/// Which accounts hash calculation is being performed?
9056#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9057pub enum CalcAccountsHashKind {
9058    Full,
9059    Incremental,
9060}
9061
9062impl CalcAccountsHashKind {
9063    /// How should zero-lamport accounts be handled by this accounts hash calculation?
9064    fn zero_lamport_accounts(&self) -> ZeroLamportAccounts {
9065        match self {
9066            CalcAccountsHashKind::Full => ZeroLamportAccounts::Excluded,
9067            CalcAccountsHashKind::Incremental => ZeroLamportAccounts::Included,
9068        }
9069    }
9070}
9071
9072pub(crate) enum UpdateIndexThreadSelection {
9073    /// Use current thread only
9074    Inline,
9075    /// Use a thread-pool if the number of updates exceeds a threshold
9076    PoolWithThreshold,
9077}
9078
9079/// How should old storages be handled in clean_accounts()?
9080#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9081pub enum OldStoragesPolicy {
9082    /// Clean all old storages, even if they were not explictly marked as dirty.
9083    ///
9084    /// This is the default behavior when not skipping rewrites.
9085    Clean,
9086    /// Leave all old storages.
9087    ///
9088    /// When skipping rewrites, we intentionally will have ancient storages.
9089    /// Do not clean them up automatically in clean_accounts().
9090    Leave,
9091}
9092
9093// These functions/fields are only usable from a dev context (i.e. tests and benches)
9094#[cfg(feature = "dev-context-only-utils")]
9095impl AccountStorageEntry {
9096    fn accounts_count(&self) -> usize {
9097        let mut count = 0;
9098        self.accounts.scan_pubkeys(|_| {
9099            count += 1;
9100        });
9101        count
9102    }
9103}
9104
9105// These functions/fields are only usable from a dev context (i.e. tests and benches)
9106#[cfg(feature = "dev-context-only-utils")]
9107impl AccountsDb {
9108    pub fn load_without_fixed_root(
9109        &self,
9110        ancestors: &Ancestors,
9111        pubkey: &Pubkey,
9112    ) -> Option<(AccountSharedData, Slot)> {
9113        self.do_load(
9114            ancestors,
9115            pubkey,
9116            None,
9117            LoadHint::Unspecified,
9118            // callers of this expect zero lamport accounts that exist in the index to be returned as Some(empty)
9119            LoadZeroLamports::SomeWithZeroLamportAccountForTests,
9120        )
9121    }
9122
9123    pub fn accounts_delta_hashes(&self) -> &Mutex<HashMap<Slot, AccountsDeltaHash>> {
9124        &self.accounts_delta_hashes
9125    }
9126
9127    pub fn accounts_hashes(&self) -> &Mutex<HashMap<Slot, (AccountsHash, /*capitalization*/ u64)>> {
9128        &self.accounts_hashes
9129    }
9130
9131    pub fn assert_load_account(&self, slot: Slot, pubkey: Pubkey, expected_lamports: u64) {
9132        let ancestors = vec![(slot, 0)].into_iter().collect();
9133        let (account, slot) = self.load_without_fixed_root(&ancestors, &pubkey).unwrap();
9134        assert_eq!((account.lamports(), slot), (expected_lamports, slot));
9135    }
9136
9137    pub fn assert_not_load_account(&self, slot: Slot, pubkey: Pubkey) {
9138        let ancestors = vec![(slot, 0)].into_iter().collect();
9139        let load = self.load_without_fixed_root(&ancestors, &pubkey);
9140        assert!(load.is_none(), "{load:?}");
9141    }
9142
9143    pub fn check_accounts(&self, pubkeys: &[Pubkey], slot: Slot, num: usize, count: usize) {
9144        let ancestors = vec![(slot, 0)].into_iter().collect();
9145        for _ in 0..num {
9146            let idx = thread_rng().gen_range(0..num);
9147            let account = self.load_without_fixed_root(&ancestors, &pubkeys[idx]);
9148            let account1 = Some((
9149                AccountSharedData::new(
9150                    (idx + count) as u64,
9151                    0,
9152                    AccountSharedData::default().owner(),
9153                ),
9154                slot,
9155            ));
9156            assert_eq!(account, account1);
9157        }
9158    }
9159
9160    /// callers used to call store_uncached. But, this is not allowed anymore.
9161    pub fn store_for_tests(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
9162        self.store(
9163            (slot, accounts),
9164            &StoreTo::Cache,
9165            None,
9166            StoreReclaims::Default,
9167            UpdateIndexThreadSelection::PoolWithThreshold,
9168        );
9169    }
9170
9171    #[allow(clippy::needless_range_loop)]
9172    pub fn modify_accounts(&self, pubkeys: &[Pubkey], slot: Slot, num: usize, count: usize) {
9173        for idx in 0..num {
9174            let account = AccountSharedData::new(
9175                (idx + count) as u64,
9176                0,
9177                AccountSharedData::default().owner(),
9178            );
9179            self.store_for_tests(slot, &[(&pubkeys[idx], &account)]);
9180        }
9181    }
9182
9183    pub fn check_storage(&self, slot: Slot, alive_count: usize, total_count: usize) {
9184        let store = self.storage.get_slot_storage_entry(slot).unwrap();
9185        assert_eq!(store.status(), AccountStorageStatus::Available);
9186        assert_eq!(store.count(), alive_count);
9187        assert_eq!(store.accounts_count(), total_count);
9188    }
9189
9190    pub fn create_account(
9191        &self,
9192        pubkeys: &mut Vec<Pubkey>,
9193        slot: Slot,
9194        num: usize,
9195        space: usize,
9196        num_vote: usize,
9197    ) {
9198        let ancestors = vec![(slot, 0)].into_iter().collect();
9199        for t in 0..num {
9200            let pubkey = solana_sdk::pubkey::new_rand();
9201            let account =
9202                AccountSharedData::new((t + 1) as u64, space, AccountSharedData::default().owner());
9203            pubkeys.push(pubkey);
9204            assert!(self.load_without_fixed_root(&ancestors, &pubkey).is_none());
9205            self.store_for_tests(slot, &[(&pubkey, &account)]);
9206        }
9207        for t in 0..num_vote {
9208            let pubkey = solana_sdk::pubkey::new_rand();
9209            let account =
9210                AccountSharedData::new((num + t + 1) as u64, space, &solana_vote_program::id());
9211            pubkeys.push(pubkey);
9212            let ancestors = vec![(slot, 0)].into_iter().collect();
9213            assert!(self.load_without_fixed_root(&ancestors, &pubkey).is_none());
9214            self.store_for_tests(slot, &[(&pubkey, &account)]);
9215        }
9216    }
9217
9218    pub fn sizes_of_accounts_in_storage_for_tests(&self, slot: Slot) -> Vec<usize> {
9219        let mut sizes = Vec::default();
9220        if let Some(storage) = self.storage.get_slot_storage_entry(slot) {
9221            storage.accounts.scan_accounts(|account| {
9222                sizes.push(account.stored_size());
9223            });
9224        }
9225        sizes
9226    }
9227
9228    pub fn ref_count_for_pubkey(&self, pubkey: &Pubkey) -> RefCount {
9229        self.accounts_index.ref_count_from_storage(pubkey)
9230    }
9231
9232    pub fn alive_account_count_in_slot(&self, slot: Slot) -> usize {
9233        self.storage
9234            .get_slot_storage_entry(slot)
9235            .map(|storage| storage.count())
9236            .unwrap_or(0)
9237            .saturating_add(
9238                self.accounts_cache
9239                    .slot_cache(slot)
9240                    .map(|slot_cache| slot_cache.len())
9241                    .unwrap_or_default(),
9242            )
9243    }
9244
9245    /// useful to adapt tests written prior to introduction of the write cache
9246    /// to use the write cache
9247    pub fn add_root_and_flush_write_cache(&self, slot: Slot) {
9248        self.add_root(slot);
9249        self.flush_root_write_cache(slot);
9250    }
9251
9252    /// useful to adapt tests written prior to introduction of the write cache
9253    /// to use the write cache
9254    pub fn flush_root_write_cache(&self, root: Slot) {
9255        assert!(
9256            self.accounts_index
9257                .roots_tracker
9258                .read()
9259                .unwrap()
9260                .alive_roots
9261                .contains(&root),
9262            "slot: {root}"
9263        );
9264        self.flush_accounts_cache(true, Some(root));
9265    }
9266
9267    pub fn all_account_count_in_accounts_file(&self, slot: Slot) -> usize {
9268        let store = self.storage.get_slot_storage_entry(slot);
9269        if let Some(store) = store {
9270            store.accounts_count()
9271        } else {
9272            0
9273        }
9274    }
9275
9276    pub fn verify_accounts_hash_and_lamports_for_tests(
9277        &self,
9278        slot: Slot,
9279        total_lamports: u64,
9280        config: VerifyAccountsHashAndLamportsConfig,
9281    ) -> Result<(), AccountsHashVerificationError> {
9282        let snapshot_storages = self.get_snapshot_storages(..);
9283        let snapshot_storages_and_slots = (
9284            snapshot_storages.0.as_slice(),
9285            snapshot_storages.1.as_slice(),
9286        );
9287        self.verify_accounts_hash_and_lamports(
9288            snapshot_storages_and_slots,
9289            slot,
9290            total_lamports,
9291            None,
9292            config,
9293        )
9294    }
9295}
9296
9297// These functions/fields are only usable from a dev context (i.e. tests and benches)
9298#[cfg(feature = "dev-context-only-utils")]
9299impl<'a> VerifyAccountsHashAndLamportsConfig<'a> {
9300    pub fn new_for_test(
9301        ancestors: &'a Ancestors,
9302        epoch_schedule: &'a EpochSchedule,
9303        rent_collector: &'a RentCollector,
9304    ) -> Self {
9305        Self {
9306            ancestors,
9307            test_hash_calculation: true,
9308            epoch_schedule,
9309            rent_collector,
9310            ignore_mismatch: false,
9311            store_detailed_debug_info: false,
9312            use_bg_thread_pool: false,
9313        }
9314    }
9315}
9316
9317/// A set of utility functions used for testing and benchmarking
9318#[cfg(feature = "dev-context-only-utils")]
9319pub mod test_utils {
9320    use {
9321        super::*,
9322        crate::{accounts::Accounts, append_vec::aligned_stored_size},
9323    };
9324
9325    pub fn create_test_accounts(
9326        accounts: &Accounts,
9327        pubkeys: &mut Vec<Pubkey>,
9328        num: usize,
9329        slot: Slot,
9330    ) {
9331        let data_size = 0;
9332        if accounts
9333            .accounts_db
9334            .storage
9335            .get_slot_storage_entry(slot)
9336            .is_none()
9337        {
9338            // Some callers relied on old behavior where the the file size was rounded up to the
9339            // next page size because they append to the storage file after it was written.
9340            // This behavior is not supported by a normal running validator.  Since this function
9341            // is only called by tests/benches, add some extra capacity to the file to not break
9342            // the tests/benches.  Those tests/benches should be updated though!  Bypassing the
9343            // write cache in general is not supported.
9344            let bytes_required = num * aligned_stored_size(data_size) + 4096;
9345            // 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.
9346            _ = accounts.accounts_db.create_and_insert_store(
9347                slot,
9348                bytes_required as u64,
9349                "create_test_accounts",
9350            );
9351        }
9352
9353        for t in 0..num {
9354            let pubkey = solana_sdk::pubkey::new_rand();
9355            let account = AccountSharedData::new(
9356                (t + 1) as u64,
9357                data_size,
9358                AccountSharedData::default().owner(),
9359            );
9360            accounts.store_slow_uncached(slot, &pubkey, &account);
9361            pubkeys.push(pubkey);
9362        }
9363    }
9364
9365    // Only used by bench, not safe to call otherwise accounts can conflict with the
9366    // accounts cache!
9367    pub fn update_accounts_bench(accounts: &Accounts, pubkeys: &[Pubkey], slot: u64) {
9368        for pubkey in pubkeys {
9369            let amount = thread_rng().gen_range(0..10);
9370            let account = AccountSharedData::new(amount, 0, AccountSharedData::default().owner());
9371            accounts.store_slow_uncached(slot, pubkey, &account);
9372        }
9373    }
9374}
9375
9376#[cfg(test)]
9377pub mod tests {
9378    use {
9379        super::*,
9380        crate::{
9381            account_info::StoredSize,
9382            account_storage::meta::{AccountMeta, StoredMeta},
9383            accounts_file::AccountsFileProvider,
9384            accounts_hash::MERKLE_FANOUT,
9385            accounts_index::{tests::*, AccountSecondaryIndexesIncludeExclude},
9386            ancient_append_vecs,
9387            append_vec::{
9388                aligned_stored_size, test_utils::TempFile, AppendVec, AppendVecStoredAccountMeta,
9389            },
9390            storable_accounts::AccountForStorage,
9391        },
9392        assert_matches::assert_matches,
9393        itertools::Itertools,
9394        rand::{prelude::SliceRandom, thread_rng, Rng},
9395        solana_sdk::{
9396            account::{
9397                accounts_equal, Account, AccountSharedData, ReadableAccount, WritableAccount,
9398            },
9399            hash::HASH_BYTES,
9400            pubkey::PUBKEY_BYTES,
9401        },
9402        std::{
9403            hash::DefaultHasher,
9404            iter::{self, FromIterator},
9405            str::FromStr,
9406            sync::{atomic::AtomicBool, RwLock},
9407            thread::{self, Builder, JoinHandle},
9408        },
9409        test_case::test_case,
9410    };
9411
9412    fn linear_ancestors(end_slot: u64) -> Ancestors {
9413        let mut ancestors: Ancestors = vec![(0, 0)].into_iter().collect();
9414        for i in 1..end_slot {
9415            ancestors.insert(i, (i - 1) as usize);
9416        }
9417        ancestors
9418    }
9419
9420    impl AccountsDb {
9421        fn get_storage_for_slot(&self, slot: Slot) -> Option<Arc<AccountStorageEntry>> {
9422            self.storage.get_slot_storage_entry(slot)
9423        }
9424    }
9425
9426    /// this tuple contains slot info PER account
9427    impl<'a, T: ReadableAccount + Sync> StorableAccounts<'a> for (Slot, &'a [(&'a Pubkey, &'a T, Slot)])
9428    where
9429        AccountForStorage<'a>: From<&'a T>,
9430    {
9431        fn account<Ret>(
9432            &self,
9433            index: usize,
9434            mut callback: impl for<'local> FnMut(AccountForStorage<'local>) -> Ret,
9435        ) -> Ret {
9436            callback(self.1[index].1.into())
9437        }
9438        fn slot(&self, index: usize) -> Slot {
9439            // note that this could be different than 'target_slot()' PER account
9440            self.1[index].2
9441        }
9442        fn target_slot(&self) -> Slot {
9443            self.0
9444        }
9445        fn len(&self) -> usize {
9446            self.1.len()
9447        }
9448        fn contains_multiple_slots(&self) -> bool {
9449            let len = self.len();
9450            if len > 0 {
9451                let slot = self.slot(0);
9452                // true if any item has a different slot than the first item
9453                (1..len).any(|i| slot != self.slot(i))
9454            } else {
9455                false
9456            }
9457        }
9458    }
9459
9460    impl AccountStorageEntry {
9461        fn add_account(&self, num_bytes: usize) {
9462            self.add_accounts(1, num_bytes)
9463        }
9464    }
9465
9466    impl CurrentAncientAccountsFile {
9467        /// note this requires that 'slot_and_accounts_file' is Some
9468        fn id(&self) -> AccountsFileId {
9469            self.accounts_file().id()
9470        }
9471    }
9472
9473    /// Helper macro to define accounts_db_test for both `AppendVec` and `HotStorage`.
9474    /// This macro supports creating both regular tests and tests that should panic.
9475    /// Usage:
9476    ///   For regular test, use the following syntax.
9477    ///     define_accounts_db_test!(TEST_NAME, |accounts_db| { TEST_BODY }); // regular test
9478    ///   For test that should panic, use the following syntax.
9479    ///     define_accounts_db_test!(TEST_NAME, panic = "PANIC_MSG", |accounts_db| { TEST_BODY });
9480    macro_rules! define_accounts_db_test {
9481        (@testfn $name:ident, $accounts_file_provider: ident, |$accounts_db:ident| $inner: tt) => {
9482                fn run_test($accounts_db: AccountsDb) {
9483                    $inner
9484                }
9485                let accounts_db =
9486                    AccountsDb::new_single_for_tests_with_provider($accounts_file_provider);
9487                run_test(accounts_db);
9488
9489        };
9490        ($name:ident, |$accounts_db:ident| $inner: tt) => {
9491            #[test_case(AccountsFileProvider::AppendVec; "append_vec")]
9492            #[test_case(AccountsFileProvider::HotStorage; "hot_storage")]
9493            fn $name(accounts_file_provider: AccountsFileProvider) {
9494                define_accounts_db_test!(@testfn $name, accounts_file_provider, |$accounts_db| $inner);
9495            }
9496        };
9497        ($name:ident, panic = $panic_message:literal, |$accounts_db:ident| $inner: tt) => {
9498            #[test_case(AccountsFileProvider::AppendVec; "append_vec")]
9499            #[test_case(AccountsFileProvider::HotStorage; "hot_storage")]
9500            #[should_panic(expected = $panic_message)]
9501            fn $name(accounts_file_provider: AccountsFileProvider) {
9502                define_accounts_db_test!(@testfn $name, accounts_file_provider, |$accounts_db| $inner);
9503            }
9504        };
9505    }
9506    pub(crate) use define_accounts_db_test;
9507
9508    fn run_generate_index_duplicates_within_slot_test(db: AccountsDb, reverse: bool) {
9509        let slot0 = 0;
9510
9511        let pubkey = Pubkey::from([1; 32]);
9512
9513        let append_vec = db.create_and_insert_store(slot0, 1000, "test");
9514
9515        let mut account_small = AccountSharedData::default();
9516        account_small.set_data(vec![1]);
9517        account_small.set_lamports(1);
9518        let mut account_big = AccountSharedData::default();
9519        account_big.set_data(vec![5; 10]);
9520        account_big.set_lamports(2);
9521        assert_ne!(
9522            aligned_stored_size(account_big.data().len()),
9523            aligned_stored_size(account_small.data().len())
9524        );
9525        // same account twice with different data lens
9526        // Rules are the last one of each pubkey is the one that ends up in the index.
9527        let mut data = vec![(&pubkey, &account_big), (&pubkey, &account_small)];
9528        if reverse {
9529            data = data.into_iter().rev().collect();
9530        }
9531        let expected_accounts_data_len = data.last().unwrap().1.data().len();
9532        let expected_alive_bytes = aligned_stored_size(expected_accounts_data_len);
9533        let storable_accounts = (slot0, &data[..]);
9534
9535        // construct append vec with account to generate an index from
9536        append_vec.accounts.append_accounts(&storable_accounts, 0);
9537
9538        let genesis_config = GenesisConfig::default();
9539        assert!(!db.accounts_index.contains(&pubkey));
9540        let result = db.generate_index(None, false, &genesis_config);
9541        // index entry should only contain a single entry for the pubkey since index cannot hold more than 1 entry per slot
9542        let entry = db.accounts_index.get_cloned(&pubkey).unwrap();
9543        assert_eq!(entry.slot_list.read().unwrap().len(), 1);
9544        if db.accounts_file_provider == AccountsFileProvider::AppendVec {
9545            // alive bytes doesn't match account size for tiered storage
9546            assert_eq!(append_vec.alive_bytes(), expected_alive_bytes);
9547        }
9548        // total # accounts in append vec
9549        assert_eq!(append_vec.accounts_count(), 2);
9550        // # alive accounts
9551        assert_eq!(append_vec.count(), 1);
9552        // all account data alive
9553        assert_eq!(
9554            result.accounts_data_len as usize, expected_accounts_data_len,
9555            "reverse: {reverse}"
9556        );
9557    }
9558
9559    define_accounts_db_test!(test_generate_index_duplicates_within_slot, |db| {
9560        run_generate_index_duplicates_within_slot_test(db, false);
9561    });
9562
9563    define_accounts_db_test!(test_generate_index_duplicates_within_slot_reverse, |db| {
9564        run_generate_index_duplicates_within_slot_test(db, true);
9565    });
9566
9567    fn generate_sample_account_from_storage(i: u8) -> AccountFromStorage {
9568        // offset has to be 8 byte aligned
9569        let offset = (i as usize) * std::mem::size_of::<u64>();
9570        AccountFromStorage {
9571            index_info: AccountInfo::new(StorageLocation::AppendVec(i as u32, offset), i as u64),
9572            data_len: i as u64,
9573            pubkey: Pubkey::new_from_array([i; 32]),
9574        }
9575    }
9576
9577    /// Reserve ancient storage size is not supported for TiredStorage
9578    #[test]
9579    fn test_sort_and_remove_dups() {
9580        // empty
9581        let mut test1 = vec![];
9582        let expected = test1.clone();
9583        AccountsDb::sort_and_remove_dups(&mut test1);
9584        assert_eq!(test1, expected);
9585        assert_eq!(test1, expected);
9586        // just 0
9587        let mut test1 = vec![generate_sample_account_from_storage(0)];
9588        let expected = test1.clone();
9589        AccountsDb::sort_and_remove_dups(&mut test1);
9590        assert_eq!(test1, expected);
9591        assert_eq!(test1, expected);
9592        // 0, 1
9593        let mut test1 = vec![
9594            generate_sample_account_from_storage(0),
9595            generate_sample_account_from_storage(1),
9596        ];
9597        let expected = test1.clone();
9598        AccountsDb::sort_and_remove_dups(&mut test1);
9599        assert_eq!(test1, expected);
9600        assert_eq!(test1, expected);
9601        // 1, 0. sort should reverse
9602        let mut test2 = vec![
9603            generate_sample_account_from_storage(1),
9604            generate_sample_account_from_storage(0),
9605        ];
9606        AccountsDb::sort_and_remove_dups(&mut test2);
9607        assert_eq!(test2, expected);
9608        assert_eq!(test2, expected);
9609
9610        for insert_other_good in 0..2 {
9611            // 0 twice so it gets removed
9612            let mut test1 = vec![
9613                generate_sample_account_from_storage(0),
9614                generate_sample_account_from_storage(0),
9615            ];
9616            let mut expected = test1.clone();
9617            expected.truncate(1); // get rid of 1st duplicate
9618            test1.first_mut().unwrap().data_len = 2342342; // this one should be ignored, so modify the data_len so it will fail the compare below if it is used
9619            if insert_other_good < 2 {
9620                // insert another good one before or after the 2 bad ones
9621                test1.insert(insert_other_good, generate_sample_account_from_storage(1));
9622                // other good one should always be last since it is sorted after
9623                expected.push(generate_sample_account_from_storage(1));
9624            }
9625            AccountsDb::sort_and_remove_dups(&mut test1);
9626            assert_eq!(test1, expected);
9627            assert_eq!(test1, expected);
9628        }
9629
9630        let mut test1 = [1, 0, 1, 0, 1u8]
9631            .into_iter()
9632            .map(generate_sample_account_from_storage)
9633            .collect::<Vec<_>>();
9634        test1.iter_mut().take(3).for_each(|entry| {
9635            entry.data_len = 2342342; // this one should be ignored, so modify the data_len so it will fail the compare below if it is used
9636            entry.index_info = AccountInfo::new(StorageLocation::Cached, 23434);
9637        });
9638
9639        let expected = [0, 1u8]
9640            .into_iter()
9641            .map(generate_sample_account_from_storage)
9642            .collect::<Vec<_>>();
9643        AccountsDb::sort_and_remove_dups(&mut test1);
9644        assert_eq!(test1, expected);
9645        assert_eq!(test1, expected);
9646    }
9647
9648    #[test]
9649    fn test_sort_and_remove_dups_random() {
9650        use rand::prelude::*;
9651        let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(1234);
9652        let accounts: Vec<_> =
9653            std::iter::repeat_with(|| generate_sample_account_from_storage(rng.gen::<u8>()))
9654                .take(1000)
9655                .collect();
9656
9657        let mut accounts1 = accounts.clone();
9658        let num_dups1 = AccountsDb::sort_and_remove_dups(&mut accounts1);
9659
9660        // Use BTreeMap to calculate sort and remove dups alternatively.
9661        let mut map = std::collections::BTreeMap::default();
9662        let mut num_dups2 = 0;
9663        for account in accounts.iter() {
9664            if map.insert(*account.pubkey(), *account).is_some() {
9665                num_dups2 += 1;
9666            }
9667        }
9668        let accounts2: Vec<_> = map.into_values().collect();
9669        assert_eq!(accounts1, accounts2);
9670        assert_eq!(num_dups1, num_dups2);
9671    }
9672
9673    /// Reserve ancient storage size is not supported for TiredStorage
9674    #[test]
9675    fn test_create_ancient_accounts_file() {
9676        let ancient_append_vec_size = ancient_append_vecs::get_ancient_append_vec_capacity();
9677        let db = AccountsDb::new_single_for_tests();
9678
9679        {
9680            // create an ancient appendvec from a small appendvec, the size of
9681            // the ancient appendvec should be the size of the ideal ancient
9682            // appendvec size.
9683            let mut current_ancient = CurrentAncientAccountsFile::default();
9684            let slot0 = 0;
9685
9686            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
9687            let _existing_append_vec = db.create_and_insert_store(slot0, 1000, "test");
9688            let _ = current_ancient.create_ancient_accounts_file(slot0, &db, 0);
9689            assert_eq!(
9690                current_ancient.accounts_file().capacity(),
9691                ancient_append_vec_size
9692            );
9693        }
9694
9695        {
9696            // create an ancient appendvec from a large appendvec (bigger than
9697            // current ancient_append_vec_size), the ancient appendvec should be
9698            // the size of the bigger ancient appendvec size.
9699            let mut current_ancient = CurrentAncientAccountsFile::default();
9700            let slot1 = 1;
9701            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
9702            let _existing_append_vec = db.create_and_insert_store(slot1, 1000, "test");
9703            let _ = current_ancient.create_ancient_accounts_file(
9704                slot1,
9705                &db,
9706                2 * ancient_append_vec_size as usize,
9707            );
9708            assert_eq!(
9709                current_ancient.accounts_file().capacity(),
9710                2 * ancient_append_vec_size
9711            );
9712        }
9713    }
9714
9715    define_accounts_db_test!(test_maybe_unref_accounts_already_in_ancient, |db| {
9716        let slot0 = 0;
9717        let slot1 = 1;
9718        let available_bytes = 1_000_000;
9719        let mut current_ancient = CurrentAncientAccountsFile::default();
9720
9721        // setup 'to_store'
9722        let pubkey = Pubkey::from([1; 32]);
9723        let account_size = 3;
9724
9725        let account = AccountSharedData::default();
9726
9727        let account_meta = AccountMeta {
9728            lamports: 1,
9729            owner: Pubkey::from([2; 32]),
9730            executable: false,
9731            rent_epoch: 0,
9732        };
9733        let offset = 3 * std::mem::size_of::<u64>();
9734        let hash = AccountHash(Hash::new(&[2; 32]));
9735        let stored_meta = StoredMeta {
9736            // global write version
9737            write_version_obsolete: 0,
9738            // key for the account
9739            pubkey,
9740            data_len: 43,
9741        };
9742        let account = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
9743            meta: &stored_meta,
9744            // account data
9745            account_meta: &account_meta,
9746            data: account.data(),
9747            offset,
9748            stored_size: account_size,
9749            hash: &hash,
9750        });
9751        let account_from_storage = AccountFromStorage::new(&account);
9752        let map_from_storage = vec![&account_from_storage];
9753        let alive_total_bytes = account.stored_size();
9754        let to_store =
9755            AccountsToStore::new(available_bytes, &map_from_storage, alive_total_bytes, slot0);
9756        // Done: setup 'to_store'
9757
9758        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
9759        let _existing_append_vec = db.create_and_insert_store(slot0, 1000, "test");
9760        {
9761            let _shrink_in_progress = current_ancient.create_ancient_accounts_file(slot0, &db, 0);
9762        }
9763        let mut ancient_slot_pubkeys = AncientSlotPubkeys::default();
9764        assert!(ancient_slot_pubkeys.inner.is_none());
9765        // same slot as current_ancient, so no-op
9766        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
9767            slot0,
9768            &db,
9769            &current_ancient,
9770            &to_store,
9771        );
9772        assert!(ancient_slot_pubkeys.inner.is_none());
9773        // different slot than current_ancient, so update 'ancient_slot_pubkeys'
9774        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
9775        let _existing_append_vec = db.create_and_insert_store(slot1, 1000, "test");
9776        let _shrink_in_progress = current_ancient.create_ancient_accounts_file(slot1, &db, 0);
9777        let slot2 = 2;
9778        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
9779            slot2,
9780            &db,
9781            &current_ancient,
9782            &to_store,
9783        );
9784        assert!(ancient_slot_pubkeys.inner.is_some());
9785        assert_eq!(ancient_slot_pubkeys.inner.as_ref().unwrap().slot, slot1);
9786        assert!(ancient_slot_pubkeys
9787            .inner
9788            .as_ref()
9789            .unwrap()
9790            .pubkeys
9791            .contains(&pubkey));
9792        assert_eq!(
9793            ancient_slot_pubkeys.inner.as_ref().unwrap().pubkeys.len(),
9794            1
9795        );
9796    });
9797
9798    #[test]
9799    fn test_get_keys_to_unref_ancient() {
9800        let rent_epoch = 0;
9801        let lamports = 0;
9802        let executable = false;
9803        let owner = Pubkey::default();
9804        let data = Vec::new();
9805
9806        let pubkey = solana_sdk::pubkey::new_rand();
9807        let pubkey2 = solana_sdk::pubkey::new_rand();
9808        let pubkey3 = solana_sdk::pubkey::new_rand();
9809        let pubkey4 = solana_sdk::pubkey::new_rand();
9810
9811        let meta = StoredMeta {
9812            write_version_obsolete: 5,
9813            pubkey,
9814            data_len: 7,
9815        };
9816        let meta2 = StoredMeta {
9817            write_version_obsolete: 5,
9818            pubkey: pubkey2,
9819            data_len: 7,
9820        };
9821        let meta3 = StoredMeta {
9822            write_version_obsolete: 5,
9823            pubkey: pubkey3,
9824            data_len: 7,
9825        };
9826        let meta4 = StoredMeta {
9827            write_version_obsolete: 5,
9828            pubkey: pubkey4,
9829            data_len: 7,
9830        };
9831        let account_meta = AccountMeta {
9832            lamports,
9833            owner,
9834            executable,
9835            rent_epoch,
9836        };
9837        let offset = 99 * std::mem::size_of::<u64>(); // offset needs to be 8 byte aligned
9838        let stored_size = 101;
9839        let hash = AccountHash(Hash::new_unique());
9840        let stored_account = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
9841            meta: &meta,
9842            account_meta: &account_meta,
9843            data: &data,
9844            offset,
9845            stored_size,
9846            hash: &hash,
9847        });
9848        let stored_account2 = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
9849            meta: &meta2,
9850            account_meta: &account_meta,
9851            data: &data,
9852            offset,
9853            stored_size,
9854            hash: &hash,
9855        });
9856        let stored_account3 = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
9857            meta: &meta3,
9858            account_meta: &account_meta,
9859            data: &data,
9860            offset,
9861            stored_size,
9862            hash: &hash,
9863        });
9864        let stored_account4 = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
9865            meta: &meta4,
9866            account_meta: &account_meta,
9867            data: &data,
9868            offset,
9869            stored_size,
9870            hash: &hash,
9871        });
9872        let mut existing_ancient_pubkeys = HashSet::default();
9873        let account_from_storage = AccountFromStorage::new(&stored_account);
9874        let accounts_from_storage = [&account_from_storage];
9875        // pubkey NOT in existing_ancient_pubkeys, so do NOT unref, but add to existing_ancient_pubkeys
9876        let unrefs = AccountsDb::get_keys_to_unref_ancient(
9877            &accounts_from_storage,
9878            &mut existing_ancient_pubkeys,
9879        );
9880        assert!(unrefs.is_empty());
9881        assert_eq!(
9882            existing_ancient_pubkeys.iter().collect::<Vec<_>>(),
9883            vec![&pubkey]
9884        );
9885        // pubkey already in existing_ancient_pubkeys, so DO unref
9886        let unrefs = AccountsDb::get_keys_to_unref_ancient(
9887            &accounts_from_storage,
9888            &mut existing_ancient_pubkeys,
9889        );
9890        assert_eq!(
9891            existing_ancient_pubkeys.iter().collect::<Vec<_>>(),
9892            vec![&pubkey]
9893        );
9894        assert_eq!(unrefs.iter().cloned().collect::<Vec<_>>(), vec![&pubkey]);
9895        // pubkey2 NOT in existing_ancient_pubkeys, so do NOT unref, but add to existing_ancient_pubkeys
9896        let account_from_storage2 = AccountFromStorage::new(&stored_account2);
9897        let accounts_from_storage = [&account_from_storage2];
9898        let unrefs = AccountsDb::get_keys_to_unref_ancient(
9899            &accounts_from_storage,
9900            &mut existing_ancient_pubkeys,
9901        );
9902        assert!(unrefs.is_empty());
9903        assert_eq!(
9904            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9905            vec![&pubkey, &pubkey2]
9906                .into_iter()
9907                .sorted()
9908                .collect::<Vec<_>>()
9909        );
9910        // pubkey2 already in existing_ancient_pubkeys, so DO unref
9911        let unrefs = AccountsDb::get_keys_to_unref_ancient(
9912            &accounts_from_storage,
9913            &mut existing_ancient_pubkeys,
9914        );
9915        assert_eq!(
9916            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9917            vec![&pubkey, &pubkey2]
9918                .into_iter()
9919                .sorted()
9920                .collect::<Vec<_>>()
9921        );
9922        assert_eq!(unrefs.iter().cloned().collect::<Vec<_>>(), vec![&pubkey2]);
9923        // pubkey3/4 NOT in existing_ancient_pubkeys, so do NOT unref, but add to existing_ancient_pubkeys
9924        let account_from_storage3 = AccountFromStorage::new(&stored_account3);
9925        let account_from_storage4 = AccountFromStorage::new(&stored_account4);
9926        let accounts_from_storage = [&account_from_storage3, &account_from_storage4];
9927        let unrefs = AccountsDb::get_keys_to_unref_ancient(
9928            &accounts_from_storage,
9929            &mut existing_ancient_pubkeys,
9930        );
9931        assert!(unrefs.is_empty());
9932        assert_eq!(
9933            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9934            vec![&pubkey, &pubkey2, &pubkey3, &pubkey4]
9935                .into_iter()
9936                .sorted()
9937                .collect::<Vec<_>>()
9938        );
9939        // pubkey3/4 already in existing_ancient_pubkeys, so DO unref
9940        let unrefs = AccountsDb::get_keys_to_unref_ancient(
9941            &accounts_from_storage,
9942            &mut existing_ancient_pubkeys,
9943        );
9944        assert_eq!(
9945            existing_ancient_pubkeys.iter().sorted().collect::<Vec<_>>(),
9946            vec![&pubkey, &pubkey2, &pubkey3, &pubkey4]
9947                .into_iter()
9948                .sorted()
9949                .collect::<Vec<_>>()
9950        );
9951        assert_eq!(
9952            unrefs.iter().cloned().sorted().collect::<Vec<_>>(),
9953            vec![&pubkey3, &pubkey4]
9954                .into_iter()
9955                .sorted()
9956                .collect::<Vec<_>>()
9957        );
9958    }
9959
9960    pub(crate) fn sample_storages_and_account_in_slot(
9961        slot: Slot,
9962        accounts: &AccountsDb,
9963    ) -> (
9964        Vec<Arc<AccountStorageEntry>>,
9965        Vec<CalculateHashIntermediate>,
9966    ) {
9967        let pubkey0 = Pubkey::from([0u8; 32]);
9968        let pubkey127 = Pubkey::from([0x7fu8; 32]);
9969        let pubkey128 = Pubkey::from([0x80u8; 32]);
9970        let pubkey255 = Pubkey::from([0xffu8; 32]);
9971
9972        let mut raw_expected = vec![
9973            CalculateHashIntermediate {
9974                hash: AccountHash(Hash::default()),
9975                lamports: 1,
9976                pubkey: pubkey0,
9977            },
9978            CalculateHashIntermediate {
9979                hash: AccountHash(Hash::default()),
9980                lamports: 128,
9981                pubkey: pubkey127,
9982            },
9983            CalculateHashIntermediate {
9984                hash: AccountHash(Hash::default()),
9985                lamports: 129,
9986                pubkey: pubkey128,
9987            },
9988            CalculateHashIntermediate {
9989                hash: AccountHash(Hash::default()),
9990                lamports: 256,
9991                pubkey: pubkey255,
9992            },
9993        ];
9994
9995        let expected_hashes = [
9996            AccountHash(Hash::from_str("EkyjPt4oL7KpRMEoAdygngnkhtVwCxqJ2MkwaGV4kUU4").unwrap()),
9997            AccountHash(Hash::from_str("4N7T4C2MK3GbHudqhfGsCyi2GpUU3roN6nhwViA41LYL").unwrap()),
9998            AccountHash(Hash::from_str("HzWMbUEnSfkrPiMdZeM6zSTdU5czEvGkvDcWBApToGC9").unwrap()),
9999            AccountHash(Hash::from_str("AsWzo1HphgrrgQ6V2zFUVDssmfaBipx2XfwGZRqcJjir").unwrap()),
10000        ];
10001
10002        let mut raw_accounts = Vec::default();
10003
10004        for i in 0..raw_expected.len() {
10005            raw_accounts.push(AccountSharedData::new(
10006                raw_expected[i].lamports,
10007                1,
10008                AccountSharedData::default().owner(),
10009            ));
10010            let hash = AccountsDb::hash_account(&raw_accounts[i], &raw_expected[i].pubkey);
10011            assert_eq!(hash, expected_hashes[i]);
10012            raw_expected[i].hash = hash;
10013        }
10014
10015        let to_store = raw_accounts
10016            .iter()
10017            .zip(raw_expected.iter())
10018            .map(|(account, intermediate)| (&intermediate.pubkey, account))
10019            .collect::<Vec<_>>();
10020
10021        accounts.store_for_tests(slot, &to_store[..]);
10022        accounts.add_root_and_flush_write_cache(slot);
10023
10024        let (storages, slots) = accounts.get_snapshot_storages(..=slot);
10025        assert_eq!(storages.len(), slots.len());
10026        storages
10027            .iter()
10028            .zip(slots.iter())
10029            .for_each(|(storage, slot)| {
10030                assert_eq!(&storage.slot(), slot);
10031            });
10032        (storages, raw_expected)
10033    }
10034
10035    pub(crate) fn sample_storages_and_accounts(
10036        accounts: &AccountsDb,
10037    ) -> (
10038        Vec<Arc<AccountStorageEntry>>,
10039        Vec<CalculateHashIntermediate>,
10040    ) {
10041        sample_storages_and_account_in_slot(1, accounts)
10042    }
10043
10044    pub(crate) fn get_storage_refs(input: &[Arc<AccountStorageEntry>]) -> SortedStorages {
10045        SortedStorages::new(input)
10046    }
10047
10048    define_accounts_db_test!(
10049        test_accountsdb_calculate_accounts_hash_from_storages_simple,
10050        |db| {
10051            let (storages, _size, _slot_expected) = sample_storage();
10052
10053            let result = db.calculate_accounts_hash(
10054                &CalcAccountsHashConfig::default(),
10055                &get_storage_refs(&storages),
10056                HashStats::default(),
10057            );
10058            let expected_hash =
10059                Hash::from_str("GKot5hBsd81kMupNCXHaqbhv3huEbxAFMLnpcX2hniwn").unwrap();
10060            let expected_accounts_hash = AccountsHash(expected_hash);
10061            assert_eq!(result, (expected_accounts_hash, 0));
10062        }
10063    );
10064
10065    define_accounts_db_test!(
10066        test_accountsdb_calculate_accounts_hash_from_storages,
10067        |db| {
10068            let (storages, raw_expected) = sample_storages_and_accounts(&db);
10069            let expected_hash = AccountsHasher::compute_merkle_root_loop(
10070                raw_expected.clone(),
10071                MERKLE_FANOUT,
10072                |item| &item.hash.0,
10073            );
10074            let sum = raw_expected.iter().map(|item| item.lamports).sum();
10075            let result = db.calculate_accounts_hash(
10076                &CalcAccountsHashConfig::default(),
10077                &get_storage_refs(&storages),
10078                HashStats::default(),
10079            );
10080
10081            let expected_accounts_hash = AccountsHash(expected_hash);
10082            assert_eq!(result, (expected_accounts_hash, sum));
10083        }
10084    );
10085
10086    fn sample_storage() -> (Vec<Arc<AccountStorageEntry>>, usize, Slot) {
10087        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10088        let slot_expected: Slot = 0;
10089        let size: usize = 123;
10090        let data = AccountStorageEntry::new(
10091            &paths[0],
10092            slot_expected,
10093            0,
10094            size as u64,
10095            AccountsFileProvider::AppendVec,
10096        );
10097
10098        let arc = Arc::new(data);
10099        let storages = vec![arc];
10100        (storages, size, slot_expected)
10101    }
10102
10103    pub(crate) fn append_single_account_with_default_hash(
10104        storage: &AccountStorageEntry,
10105        pubkey: &Pubkey,
10106        account: &AccountSharedData,
10107        mark_alive: bool,
10108        add_to_index: Option<&AccountInfoAccountsIndex>,
10109    ) {
10110        let slot = storage.slot();
10111        let accounts = [(pubkey, account)];
10112        let slice = &accounts[..];
10113        let storable_accounts = (slot, slice);
10114        let stored_accounts_info = storage
10115            .accounts
10116            .append_accounts(&storable_accounts, 0)
10117            .unwrap();
10118        if mark_alive {
10119            // updates 'alive_bytes' on the storage
10120            storage.add_account(stored_accounts_info.size);
10121        }
10122
10123        if let Some(index) = add_to_index {
10124            let account_info = AccountInfo::new(
10125                StorageLocation::AppendVec(storage.id(), stored_accounts_info.offsets[0]),
10126                account.lamports(),
10127            );
10128            index.upsert(
10129                slot,
10130                slot,
10131                pubkey,
10132                account,
10133                &AccountSecondaryIndexes::default(),
10134                account_info,
10135                &mut Vec::default(),
10136                UpsertReclaim::IgnoreReclaims,
10137            );
10138        }
10139    }
10140
10141    fn append_sample_data_to_storage(
10142        storage: &AccountStorageEntry,
10143        pubkey: &Pubkey,
10144        mark_alive: bool,
10145        account_data_size: Option<u64>,
10146    ) {
10147        let acc = AccountSharedData::new(
10148            1,
10149            account_data_size.unwrap_or(48) as usize,
10150            AccountSharedData::default().owner(),
10151        );
10152        append_single_account_with_default_hash(storage, pubkey, &acc, mark_alive, None);
10153    }
10154
10155    pub(crate) fn sample_storage_with_entries(
10156        tf: &TempFile,
10157        slot: Slot,
10158        pubkey: &Pubkey,
10159        mark_alive: bool,
10160    ) -> Arc<AccountStorageEntry> {
10161        sample_storage_with_entries_id(tf, slot, pubkey, 0, mark_alive, None)
10162    }
10163
10164    fn sample_storage_with_entries_id_fill_percentage(
10165        tf: &TempFile,
10166        slot: Slot,
10167        pubkey: &Pubkey,
10168        id: AccountsFileId,
10169        mark_alive: bool,
10170        account_data_size: Option<u64>,
10171        fill_percentage: u64,
10172    ) -> Arc<AccountStorageEntry> {
10173        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10174        let file_size = account_data_size.unwrap_or(123) * 100 / fill_percentage;
10175        let size_aligned: usize = aligned_stored_size(file_size as usize);
10176        let mut data = AccountStorageEntry::new(
10177            &paths[0],
10178            slot,
10179            id,
10180            size_aligned as u64,
10181            AccountsFileProvider::AppendVec,
10182        );
10183        let av = AccountsFile::AppendVec(AppendVec::new(
10184            &tf.path,
10185            true,
10186            (1024 * 1024).max(size_aligned),
10187        ));
10188        data.accounts = av;
10189
10190        let arc = Arc::new(data);
10191        append_sample_data_to_storage(&arc, pubkey, mark_alive, account_data_size);
10192        arc
10193    }
10194
10195    fn sample_storage_with_entries_id(
10196        tf: &TempFile,
10197        slot: Slot,
10198        pubkey: &Pubkey,
10199        id: AccountsFileId,
10200        mark_alive: bool,
10201        account_data_size: Option<u64>,
10202    ) -> Arc<AccountStorageEntry> {
10203        sample_storage_with_entries_id_fill_percentage(
10204            tf,
10205            slot,
10206            pubkey,
10207            id,
10208            mark_alive,
10209            account_data_size,
10210            100,
10211        )
10212    }
10213
10214    define_accounts_db_test!(test_accountsdb_add_root, |db| {
10215        let key = Pubkey::default();
10216        let account0 = AccountSharedData::new(1, 0, &key);
10217
10218        db.store_for_tests(0, &[(&key, &account0)]);
10219        db.add_root(0);
10220        let ancestors = vec![(1, 1)].into_iter().collect();
10221        assert_eq!(
10222            db.load_without_fixed_root(&ancestors, &key),
10223            Some((account0, 0))
10224        );
10225    });
10226
10227    define_accounts_db_test!(test_accountsdb_latest_ancestor, |db| {
10228        let key = Pubkey::default();
10229        let account0 = AccountSharedData::new(1, 0, &key);
10230
10231        db.store_for_tests(0, &[(&key, &account0)]);
10232
10233        let account1 = AccountSharedData::new(0, 0, &key);
10234        db.store_for_tests(1, &[(&key, &account1)]);
10235
10236        let ancestors = vec![(1, 1)].into_iter().collect();
10237        assert_eq!(
10238            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10239            &account1
10240        );
10241
10242        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
10243        assert_eq!(
10244            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10245            &account1
10246        );
10247
10248        let mut accounts = Vec::new();
10249        db.unchecked_scan_accounts(
10250            "",
10251            &ancestors,
10252            |_, account, _| {
10253                accounts.push(account.take_account());
10254            },
10255            &ScanConfig::default(),
10256        );
10257        assert_eq!(accounts, vec![account1]);
10258    });
10259
10260    define_accounts_db_test!(test_accountsdb_latest_ancestor_with_root, |db| {
10261        let key = Pubkey::default();
10262        let account0 = AccountSharedData::new(1, 0, &key);
10263
10264        db.store_for_tests(0, &[(&key, &account0)]);
10265
10266        let account1 = AccountSharedData::new(0, 0, &key);
10267        db.store_for_tests(1, &[(&key, &account1)]);
10268        db.add_root(0);
10269
10270        let ancestors = vec![(1, 1)].into_iter().collect();
10271        assert_eq!(
10272            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10273            &account1
10274        );
10275
10276        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
10277        assert_eq!(
10278            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10279            &account1
10280        );
10281    });
10282
10283    define_accounts_db_test!(test_accountsdb_root_one_slot, |db| {
10284        let key = Pubkey::default();
10285        let account0 = AccountSharedData::new(1, 0, &key);
10286
10287        // store value 1 in the "root", i.e. db zero
10288        db.store_for_tests(0, &[(&key, &account0)]);
10289
10290        // now we have:
10291        //
10292        //                       root0 -> key.lamports==1
10293        //                        / \
10294        //                       /   \
10295        //  key.lamports==0 <- slot1    \
10296        //                             slot2 -> key.lamports==1
10297        //                                       (via root0)
10298
10299        // store value 0 in one child
10300        let account1 = AccountSharedData::new(0, 0, &key);
10301        db.store_for_tests(1, &[(&key, &account1)]);
10302
10303        // masking accounts is done at the Accounts level, at accountsDB we see
10304        // original account (but could also accept "None", which is implemented
10305        // at the Accounts level)
10306        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
10307        assert_eq!(
10308            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10309            &account1
10310        );
10311
10312        // we should see 1 token in slot 2
10313        let ancestors = vec![(0, 0), (2, 2)].into_iter().collect();
10314        assert_eq!(
10315            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
10316            &account0
10317        );
10318
10319        db.add_root(0);
10320
10321        let ancestors = vec![(1, 1)].into_iter().collect();
10322        assert_eq!(
10323            db.load_without_fixed_root(&ancestors, &key),
10324            Some((account1, 1))
10325        );
10326        let ancestors = vec![(2, 2)].into_iter().collect();
10327        assert_eq!(
10328            db.load_without_fixed_root(&ancestors, &key),
10329            Some((account0, 0))
10330        ); // original value
10331    });
10332
10333    define_accounts_db_test!(test_accountsdb_add_root_many, |db| {
10334        let mut pubkeys: Vec<Pubkey> = vec![];
10335        db.create_account(&mut pubkeys, 0, 100, 0, 0);
10336        for _ in 1..100 {
10337            let idx = thread_rng().gen_range(0..99);
10338            let ancestors = vec![(0, 0)].into_iter().collect();
10339            let account = db
10340                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10341                .unwrap();
10342            let default_account = AccountSharedData::from(Account {
10343                lamports: (idx + 1) as u64,
10344                ..Account::default()
10345            });
10346            assert_eq!((default_account, 0), account);
10347        }
10348
10349        db.add_root(0);
10350
10351        // check that all the accounts appear with a new root
10352        for _ in 1..100 {
10353            let idx = thread_rng().gen_range(0..99);
10354            let ancestors = vec![(0, 0)].into_iter().collect();
10355            let account0 = db
10356                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10357                .unwrap();
10358            let ancestors = vec![(1, 1)].into_iter().collect();
10359            let account1 = db
10360                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10361                .unwrap();
10362            let default_account = AccountSharedData::from(Account {
10363                lamports: (idx + 1) as u64,
10364                ..Account::default()
10365            });
10366            assert_eq!(&default_account, &account0.0);
10367            assert_eq!(&default_account, &account1.0);
10368        }
10369    });
10370
10371    define_accounts_db_test!(test_accountsdb_count_stores, |db| {
10372        let mut pubkeys: Vec<Pubkey> = vec![];
10373        db.create_account(&mut pubkeys, 0, 2, DEFAULT_FILE_SIZE as usize / 3, 0);
10374        db.add_root_and_flush_write_cache(0);
10375        db.check_storage(0, 2, 2);
10376
10377        let pubkey = solana_sdk::pubkey::new_rand();
10378        let account = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 3, &pubkey);
10379        db.store_for_tests(1, &[(&pubkey, &account)]);
10380        db.store_for_tests(1, &[(&pubkeys[0], &account)]);
10381        // adding root doesn't change anything
10382        db.calculate_accounts_delta_hash(1);
10383        db.add_root_and_flush_write_cache(1);
10384        {
10385            let slot_0_store = &db.storage.get_slot_storage_entry(0).unwrap();
10386            let slot_1_store = &db.storage.get_slot_storage_entry(1).unwrap();
10387            assert_eq!(slot_0_store.count(), 2);
10388            assert_eq!(slot_1_store.count(), 2);
10389            assert_eq!(slot_0_store.accounts_count(), 2);
10390            assert_eq!(slot_1_store.accounts_count(), 2);
10391        }
10392
10393        // overwrite old rooted account version; only the r_slot_0_stores.count() should be
10394        // decremented
10395        // slot 2 is not a root and should be ignored by clean
10396        db.store_for_tests(2, &[(&pubkeys[0], &account)]);
10397        db.clean_accounts_for_tests();
10398        {
10399            let slot_0_store = &db.storage.get_slot_storage_entry(0).unwrap();
10400            let slot_1_store = &db.storage.get_slot_storage_entry(1).unwrap();
10401            assert_eq!(slot_0_store.count(), 1);
10402            assert_eq!(slot_1_store.count(), 2);
10403            assert_eq!(slot_0_store.accounts_count(), 2);
10404            assert_eq!(slot_1_store.accounts_count(), 2);
10405        }
10406    });
10407
10408    define_accounts_db_test!(test_accounts_unsquashed, |db0| {
10409        let key = Pubkey::default();
10410
10411        // 1 token in the "root", i.e. db zero
10412        let account0 = AccountSharedData::new(1, 0, &key);
10413        db0.store_for_tests(0, &[(&key, &account0)]);
10414
10415        // 0 lamports in the child
10416        let account1 = AccountSharedData::new(0, 0, &key);
10417        db0.store_for_tests(1, &[(&key, &account1)]);
10418
10419        // masking accounts is done at the Accounts level, at accountsDB we see
10420        // original account
10421        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
10422        assert_eq!(
10423            db0.load_without_fixed_root(&ancestors, &key),
10424            Some((account1, 1))
10425        );
10426        let ancestors = vec![(0, 0)].into_iter().collect();
10427        assert_eq!(
10428            db0.load_without_fixed_root(&ancestors, &key),
10429            Some((account0, 0))
10430        );
10431    });
10432
10433    fn run_test_remove_unrooted_slot(is_cached: bool, db: AccountsDb) {
10434        let unrooted_slot = 9;
10435        let unrooted_bank_id = 9;
10436        let key = Pubkey::default();
10437        let account0 = AccountSharedData::new(1, 0, &key);
10438        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
10439        assert!(!db.accounts_index.contains(&key));
10440        if is_cached {
10441            db.store_cached((unrooted_slot, &[(&key, &account0)][..]), None);
10442        } else {
10443            db.store_for_tests(unrooted_slot, &[(&key, &account0)]);
10444        }
10445        assert!(db.get_bank_hash_stats(unrooted_slot).is_some());
10446        assert!(db.accounts_index.contains(&key));
10447        db.assert_load_account(unrooted_slot, key, 1);
10448
10449        // Purge the slot
10450        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
10451        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
10452        assert!(db.get_bank_hash_stats(unrooted_slot).is_none());
10453        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_none());
10454        assert!(db.storage.get_slot_storage_entry(unrooted_slot).is_none());
10455        assert!(!db.accounts_index.contains(&key));
10456
10457        // Test we can store for the same slot again and get the right information
10458        let account0 = AccountSharedData::new(2, 0, &key);
10459        db.store_for_tests(unrooted_slot, &[(&key, &account0)]);
10460        db.assert_load_account(unrooted_slot, key, 2);
10461    }
10462
10463    define_accounts_db_test!(test_remove_unrooted_slot_cached, |db| {
10464        run_test_remove_unrooted_slot(true, db);
10465    });
10466
10467    define_accounts_db_test!(test_remove_unrooted_slot_storage, |db| {
10468        run_test_remove_unrooted_slot(false, db);
10469    });
10470
10471    fn update_accounts(accounts: &AccountsDb, pubkeys: &[Pubkey], slot: Slot, range: usize) {
10472        for _ in 1..1000 {
10473            let idx = thread_rng().gen_range(0..range);
10474            let ancestors = vec![(slot, 0)].into_iter().collect();
10475            if let Some((mut account, _)) =
10476                accounts.load_without_fixed_root(&ancestors, &pubkeys[idx])
10477            {
10478                account.checked_add_lamports(1).unwrap();
10479                accounts.store_for_tests(slot, &[(&pubkeys[idx], &account)]);
10480                if account.is_zero_lamport() {
10481                    let ancestors = vec![(slot, 0)].into_iter().collect();
10482                    assert!(accounts
10483                        .load_without_fixed_root(&ancestors, &pubkeys[idx])
10484                        .is_none());
10485                } else {
10486                    let default_account = AccountSharedData::from(Account {
10487                        lamports: account.lamports(),
10488                        ..Account::default()
10489                    });
10490                    assert_eq!(default_account, account);
10491                }
10492            }
10493        }
10494    }
10495
10496    #[test]
10497    fn test_account_one() {
10498        let (_accounts_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10499        let db = AccountsDb::new_for_tests(paths);
10500        let mut pubkeys: Vec<Pubkey> = vec![];
10501        db.create_account(&mut pubkeys, 0, 1, 0, 0);
10502        let ancestors = vec![(0, 0)].into_iter().collect();
10503        let account = db.load_without_fixed_root(&ancestors, &pubkeys[0]).unwrap();
10504        let default_account = AccountSharedData::from(Account {
10505            lamports: 1,
10506            ..Account::default()
10507        });
10508        assert_eq!((default_account, 0), account);
10509    }
10510
10511    #[test]
10512    fn test_account_many() {
10513        let (_accounts_dirs, paths) = get_temp_accounts_paths(2).unwrap();
10514        let db = AccountsDb::new_for_tests(paths);
10515        let mut pubkeys: Vec<Pubkey> = vec![];
10516        db.create_account(&mut pubkeys, 0, 100, 0, 0);
10517        db.check_accounts(&pubkeys, 0, 100, 1);
10518    }
10519
10520    #[test]
10521    fn test_account_update() {
10522        let accounts = AccountsDb::new_single_for_tests();
10523        let mut pubkeys: Vec<Pubkey> = vec![];
10524        accounts.create_account(&mut pubkeys, 0, 100, 0, 0);
10525        update_accounts(&accounts, &pubkeys, 0, 99);
10526        accounts.add_root_and_flush_write_cache(0);
10527        accounts.check_storage(0, 100, 100);
10528    }
10529
10530    #[test]
10531    fn test_account_grow_many() {
10532        let (_accounts_dir, paths) = get_temp_accounts_paths(2).unwrap();
10533        let size = 4096;
10534        let accounts = AccountsDb {
10535            file_size: size,
10536            ..AccountsDb::new_for_tests(paths)
10537        };
10538        let mut keys = vec![];
10539        for i in 0..9 {
10540            let key = solana_sdk::pubkey::new_rand();
10541            let account = AccountSharedData::new(i + 1, size as usize / 4, &key);
10542            accounts.store_for_tests(0, &[(&key, &account)]);
10543            keys.push(key);
10544        }
10545        let ancestors = vec![(0, 0)].into_iter().collect();
10546        for (i, key) in keys.iter().enumerate() {
10547            assert_eq!(
10548                accounts
10549                    .load_without_fixed_root(&ancestors, key)
10550                    .unwrap()
10551                    .0
10552                    .lamports(),
10553                (i as u64) + 1
10554            );
10555        }
10556
10557        let mut append_vec_histogram = HashMap::new();
10558        let mut all_slots = vec![];
10559        for slot_storage in accounts.storage.iter() {
10560            all_slots.push(slot_storage.0)
10561        }
10562        for slot in all_slots {
10563            *append_vec_histogram.entry(slot).or_insert(0) += 1;
10564        }
10565        for count in append_vec_histogram.values() {
10566            assert!(*count >= 2);
10567        }
10568    }
10569
10570    #[test]
10571    fn test_account_grow() {
10572        for pass in 0..27 {
10573            let accounts = AccountsDb::new_single_for_tests();
10574
10575            let status = [AccountStorageStatus::Available, AccountStorageStatus::Full];
10576            let pubkey1 = solana_sdk::pubkey::new_rand();
10577            let account1 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey1);
10578            accounts.store_for_tests(0, &[(&pubkey1, &account1)]);
10579            if pass == 0 {
10580                accounts.add_root_and_flush_write_cache(0);
10581                let store = &accounts.storage.get_slot_storage_entry(0).unwrap();
10582                assert_eq!(store.count(), 1);
10583                assert_eq!(store.status(), AccountStorageStatus::Available);
10584                continue;
10585            }
10586
10587            let pubkey2 = solana_sdk::pubkey::new_rand();
10588            let account2 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey2);
10589            accounts.store_for_tests(0, &[(&pubkey2, &account2)]);
10590
10591            if pass == 1 {
10592                accounts.add_root_and_flush_write_cache(0);
10593                assert_eq!(accounts.storage.len(), 1);
10594                let store = &accounts.storage.get_slot_storage_entry(0).unwrap();
10595                assert_eq!(store.count(), 2);
10596                assert_eq!(store.status(), AccountStorageStatus::Available);
10597                continue;
10598            }
10599            let ancestors = vec![(0, 0)].into_iter().collect();
10600            assert_eq!(
10601                accounts
10602                    .load_without_fixed_root(&ancestors, &pubkey1)
10603                    .unwrap()
10604                    .0,
10605                account1
10606            );
10607            assert_eq!(
10608                accounts
10609                    .load_without_fixed_root(&ancestors, &pubkey2)
10610                    .unwrap()
10611                    .0,
10612                account2
10613            );
10614
10615            // lots of writes, but they are all duplicates
10616            for i in 0..25 {
10617                accounts.store_for_tests(0, &[(&pubkey1, &account1)]);
10618                let flush = pass == i + 2;
10619                if flush {
10620                    accounts.add_root_and_flush_write_cache(0);
10621                    assert_eq!(accounts.storage.len(), 1);
10622                    let store = &accounts.storage.get_slot_storage_entry(0).unwrap();
10623                    assert_eq!(store.status(), status[0]);
10624                }
10625                let ancestors = vec![(0, 0)].into_iter().collect();
10626                assert_eq!(
10627                    accounts
10628                        .load_without_fixed_root(&ancestors, &pubkey1)
10629                        .unwrap()
10630                        .0,
10631                    account1
10632                );
10633                assert_eq!(
10634                    accounts
10635                        .load_without_fixed_root(&ancestors, &pubkey2)
10636                        .unwrap()
10637                        .0,
10638                    account2
10639                );
10640                if flush {
10641                    break;
10642                }
10643            }
10644        }
10645    }
10646
10647    #[test]
10648    fn test_lazy_gc_slot() {
10649        solana_logger::setup();
10650        //This test is pedantic
10651        //A slot is purged when a non root bank is cleaned up.  If a slot is behind root but it is
10652        //not root, it means we are retaining dead banks.
10653        let accounts = AccountsDb::new_single_for_tests();
10654        let pubkey = solana_sdk::pubkey::new_rand();
10655        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10656        //store an account
10657        accounts.store_for_tests(0, &[(&pubkey, &account)]);
10658        accounts.add_root_and_flush_write_cache(0);
10659
10660        let ancestors = vec![(0, 0)].into_iter().collect();
10661        let id = accounts
10662            .accounts_index
10663            .get_with_and_then(
10664                &pubkey,
10665                Some(&ancestors),
10666                None,
10667                false,
10668                |(_slot, account_info)| account_info.store_id(),
10669            )
10670            .unwrap();
10671        accounts.calculate_accounts_delta_hash(0);
10672
10673        //slot is still there, since gc is lazy
10674        assert_eq!(accounts.storage.get_slot_storage_entry(0).unwrap().id(), id);
10675
10676        //store causes clean
10677        accounts.store_for_tests(1, &[(&pubkey, &account)]);
10678
10679        // generate delta state for slot 1, so clean operates on it.
10680        accounts.calculate_accounts_delta_hash(1);
10681
10682        //slot is gone
10683        accounts.print_accounts_stats("pre-clean");
10684        accounts.add_root_and_flush_write_cache(1);
10685        assert!(accounts.storage.get_slot_storage_entry(0).is_some());
10686        accounts.clean_accounts_for_tests();
10687        assert!(accounts.storage.get_slot_storage_entry(0).is_none());
10688
10689        //new value is there
10690        let ancestors = vec![(1, 1)].into_iter().collect();
10691        assert_eq!(
10692            accounts.load_without_fixed_root(&ancestors, &pubkey),
10693            Some((account, 1))
10694        );
10695    }
10696
10697    #[test]
10698    fn test_clean_zero_lamport_and_dead_slot() {
10699        solana_logger::setup();
10700
10701        let accounts = AccountsDb::new_single_for_tests();
10702        let pubkey1 = solana_sdk::pubkey::new_rand();
10703        let pubkey2 = solana_sdk::pubkey::new_rand();
10704        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
10705        let zero_lamport_account =
10706            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10707
10708        // Store two accounts
10709        accounts.store_for_tests(0, &[(&pubkey1, &account)]);
10710        accounts.store_for_tests(0, &[(&pubkey2, &account)]);
10711
10712        // Make sure both accounts are in the same AppendVec in slot 0, which
10713        // will prevent pubkey1 from being cleaned up later even when it's a
10714        // zero-lamport account
10715        let ancestors = vec![(0, 1)].into_iter().collect();
10716        let (slot1, account_info1) = accounts
10717            .accounts_index
10718            .get_with_and_then(
10719                &pubkey1,
10720                Some(&ancestors),
10721                None,
10722                false,
10723                |(slot, account_info)| (slot, account_info),
10724            )
10725            .unwrap();
10726        let (slot2, account_info2) = accounts
10727            .accounts_index
10728            .get_with_and_then(
10729                &pubkey2,
10730                Some(&ancestors),
10731                None,
10732                false,
10733                |(slot, account_info)| (slot, account_info),
10734            )
10735            .unwrap();
10736        assert_eq!(slot1, 0);
10737        assert_eq!(slot1, slot2);
10738        assert_eq!(account_info1.storage_location(), StorageLocation::Cached);
10739        assert_eq!(
10740            account_info1.storage_location(),
10741            account_info2.storage_location()
10742        );
10743
10744        // Update account 1 in slot 1
10745        accounts.store_for_tests(1, &[(&pubkey1, &account)]);
10746
10747        // Update account 1 as  zero lamports account
10748        accounts.store_for_tests(2, &[(&pubkey1, &zero_lamport_account)]);
10749
10750        // Pubkey 1 was the only account in slot 1, and it was updated in slot 2, so
10751        // slot 1 should be purged
10752        accounts.calculate_accounts_delta_hash(0);
10753        accounts.add_root_and_flush_write_cache(0);
10754        accounts.calculate_accounts_delta_hash(1);
10755        accounts.add_root_and_flush_write_cache(1);
10756        accounts.calculate_accounts_delta_hash(2);
10757        accounts.add_root_and_flush_write_cache(2);
10758
10759        // Slot 1 should be removed, slot 0 cannot be removed because it still has
10760        // the latest update for pubkey 2
10761        accounts.clean_accounts_for_tests();
10762        assert!(accounts.storage.get_slot_storage_entry(0).is_some());
10763        assert!(accounts.storage.get_slot_storage_entry(1).is_none());
10764
10765        // Slot 1 should be cleaned because all it's accounts are
10766        // zero lamports, and are not present in any other slot's
10767        // storage entries
10768        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
10769    }
10770
10771    #[test]
10772    #[should_panic(expected = "ref count expected to be zero")]
10773    fn test_remove_zero_lamport_multi_ref_accounts_panic() {
10774        let accounts = AccountsDb::new_single_for_tests();
10775        let pubkey_zero = Pubkey::from([1; 32]);
10776        let one_lamport_account =
10777            AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10778
10779        let zero_lamport_account =
10780            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10781        let slot = 1;
10782
10783        accounts.store_for_tests(slot, &[(&pubkey_zero, &one_lamport_account)]);
10784        accounts.calculate_accounts_delta_hash(slot);
10785        accounts.add_root_and_flush_write_cache(slot);
10786
10787        accounts.store_for_tests(slot + 1, &[(&pubkey_zero, &zero_lamport_account)]);
10788        accounts.calculate_accounts_delta_hash(slot + 1);
10789        accounts.add_root_and_flush_write_cache(slot + 1);
10790
10791        // This should panic because there are 2 refs for pubkey_zero.
10792        accounts.remove_zero_lamport_single_ref_accounts_after_shrink(
10793            &[&pubkey_zero],
10794            slot,
10795            &ShrinkStats::default(),
10796            true,
10797        );
10798    }
10799
10800    #[test]
10801    fn test_remove_zero_lamport_single_ref_accounts_after_shrink() {
10802        for pass in 0..3 {
10803            let accounts = AccountsDb::new_single_for_tests();
10804            let pubkey_zero = Pubkey::from([1; 32]);
10805            let pubkey2 = Pubkey::from([2; 32]);
10806            let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10807            let zero_lamport_account =
10808                AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10809            let slot = 1;
10810
10811            accounts.store_for_tests(
10812                slot,
10813                &[(&pubkey_zero, &zero_lamport_account), (&pubkey2, &account)],
10814            );
10815
10816            // Simulate rooting the zero-lamport account, writes it to storage
10817            accounts.calculate_accounts_delta_hash(slot);
10818            accounts.add_root_and_flush_write_cache(slot);
10819
10820            if pass > 0 {
10821                // store in write cache
10822                accounts.store_for_tests(slot + 1, &[(&pubkey_zero, &zero_lamport_account)]);
10823                if pass == 2 {
10824                    // move to a storage (causing ref count to increase)
10825                    accounts.calculate_accounts_delta_hash(slot + 1);
10826                    accounts.add_root_and_flush_write_cache(slot + 1);
10827                }
10828            }
10829
10830            accounts.accounts_index.get_and_then(&pubkey_zero, |entry| {
10831                let expected_ref_count = if pass < 2 { 1 } else { 2 };
10832                assert_eq!(entry.unwrap().ref_count(), expected_ref_count, "{pass}");
10833                let expected_slot_list = if pass < 1 { 1 } else { 2 };
10834                assert_eq!(
10835                    entry.unwrap().slot_list.read().unwrap().len(),
10836                    expected_slot_list
10837                );
10838                (false, ())
10839            });
10840            accounts.accounts_index.get_and_then(&pubkey2, |entry| {
10841                assert!(entry.is_some());
10842                (false, ())
10843            });
10844
10845            let zero_lamport_single_ref_pubkeys =
10846                if pass < 2 { vec![&pubkey_zero] } else { vec![] };
10847            accounts.remove_zero_lamport_single_ref_accounts_after_shrink(
10848                &zero_lamport_single_ref_pubkeys,
10849                slot,
10850                &ShrinkStats::default(),
10851                true,
10852            );
10853
10854            accounts.accounts_index.get_and_then(&pubkey_zero, |entry| {
10855                match pass {
10856                    0 => {
10857                        // should not exist in index at all
10858                        assert!(entry.is_none(), "{pass}");
10859                    }
10860                    1 => {
10861                        // alive only in slot + 1
10862                        assert_eq!(entry.unwrap().slot_list.read().unwrap().len(), 1);
10863                        assert_eq!(
10864                            entry
10865                                .unwrap()
10866                                .slot_list
10867                                .read()
10868                                .unwrap()
10869                                .first()
10870                                .map(|(s, _)| s)
10871                                .cloned()
10872                                .unwrap(),
10873                            slot + 1
10874                        );
10875                        let expected_ref_count = 0;
10876                        assert_eq!(
10877                            entry.map(|e| e.ref_count()),
10878                            Some(expected_ref_count),
10879                            "{pass}"
10880                        );
10881                    }
10882                    2 => {
10883                        // alive in both slot, slot + 1
10884                        assert_eq!(entry.unwrap().slot_list.read().unwrap().len(), 2);
10885
10886                        let slots = entry
10887                            .unwrap()
10888                            .slot_list
10889                            .read()
10890                            .unwrap()
10891                            .iter()
10892                            .map(|(s, _)| s)
10893                            .cloned()
10894                            .collect::<Vec<_>>();
10895                        assert_eq!(slots, vec![slot, slot + 1]);
10896                        let expected_ref_count = 2;
10897                        assert_eq!(
10898                            entry.map(|e| e.ref_count()),
10899                            Some(expected_ref_count),
10900                            "{pass}"
10901                        );
10902                    }
10903                    _ => {
10904                        unreachable!("Shouldn't reach here.")
10905                    }
10906                }
10907                (false, ())
10908            });
10909
10910            accounts.accounts_index.get_and_then(&pubkey2, |entry| {
10911                assert!(entry.is_some(), "{pass}");
10912                (false, ())
10913            });
10914        }
10915    }
10916
10917    #[test]
10918    fn test_shrink_zero_lamport_single_ref_account() {
10919        solana_logger::setup();
10920        // note that 'None' checks the case based on the default value of `latest_full_snapshot_slot` in `AccountsDb`
10921        for latest_full_snapshot_slot in [None, Some(0), Some(1), Some(2)] {
10922            // store a zero and non-zero lamport account
10923            // make sure clean marks the ref_count=1, zero lamport account dead and removes pubkey from index completely
10924            let accounts = AccountsDb::new_single_for_tests();
10925            let pubkey_zero = Pubkey::from([1; 32]);
10926            let pubkey2 = Pubkey::from([2; 32]);
10927            let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10928            let zero_lamport_account =
10929                AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10930            let slot = 1;
10931            // Store a zero-lamport account and a non-zero lamport account
10932            accounts.store_for_tests(
10933                slot,
10934                &[(&pubkey_zero, &zero_lamport_account), (&pubkey2, &account)],
10935            );
10936
10937            // Simulate rooting the zero-lamport account, should be a
10938            // candidate for cleaning
10939            accounts.calculate_accounts_delta_hash(slot);
10940            accounts.add_root_and_flush_write_cache(slot);
10941
10942            // for testing, we need to cause shrink to think this will be productive.
10943            // The zero lamport account isn't dead, but it can become dead inside shrink.
10944            accounts
10945                .storage
10946                .get_slot_storage_entry(slot)
10947                .unwrap()
10948                .alive_bytes
10949                .fetch_sub(aligned_stored_size(0), Ordering::Release);
10950
10951            if let Some(latest_full_snapshot_slot) = latest_full_snapshot_slot {
10952                accounts.set_latest_full_snapshot_slot(latest_full_snapshot_slot);
10953            }
10954
10955            // Shrink the slot. The behavior on the zero lamport account will depend on `latest_full_snapshot_slot`.
10956            accounts.shrink_slot_forced(slot);
10957
10958            assert!(
10959                accounts.storage.get_slot_storage_entry(1).is_some(),
10960                "{latest_full_snapshot_slot:?}"
10961            );
10962
10963            let expected_alive_count = if latest_full_snapshot_slot.unwrap_or(Slot::MAX) < slot {
10964                // zero lamport account should NOT be dead in the index
10965                assert!(
10966                    accounts
10967                        .accounts_index
10968                        .contains_with(&pubkey_zero, None, None),
10969                    "{latest_full_snapshot_slot:?}"
10970                );
10971                2
10972            } else {
10973                // zero lamport account should be dead in the index
10974                assert!(
10975                    !accounts
10976                        .accounts_index
10977                        .contains_with(&pubkey_zero, None, None),
10978                    "{latest_full_snapshot_slot:?}"
10979                );
10980                // the zero lamport account should be marked as dead
10981                1
10982            };
10983
10984            assert_eq!(
10985                accounts.alive_account_count_in_slot(slot),
10986                expected_alive_count,
10987                "{latest_full_snapshot_slot:?}"
10988            );
10989
10990            // other account should still be alive
10991            assert!(
10992                accounts.accounts_index.contains_with(&pubkey2, None, None),
10993                "{latest_full_snapshot_slot:?}"
10994            );
10995            assert!(
10996                accounts.storage.get_slot_storage_entry(slot).is_some(),
10997                "{latest_full_snapshot_slot:?}"
10998            );
10999        }
11000    }
11001
11002    #[test]
11003    fn test_clean_multiple_zero_lamport_decrements_index_ref_count() {
11004        solana_logger::setup();
11005
11006        let accounts = AccountsDb::new_single_for_tests();
11007        let pubkey1 = solana_sdk::pubkey::new_rand();
11008        let pubkey2 = solana_sdk::pubkey::new_rand();
11009        let zero_lamport_account =
11010            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11011
11012        // Store 2 accounts in slot 0, then update account 1 in two more slots
11013        accounts.store_for_tests(0, &[(&pubkey1, &zero_lamport_account)]);
11014        accounts.store_for_tests(0, &[(&pubkey2, &zero_lamport_account)]);
11015        accounts.store_for_tests(1, &[(&pubkey1, &zero_lamport_account)]);
11016        accounts.store_for_tests(2, &[(&pubkey1, &zero_lamport_account)]);
11017        // Root all slots
11018        accounts.calculate_accounts_delta_hash(0);
11019        accounts.add_root_and_flush_write_cache(0);
11020        accounts.calculate_accounts_delta_hash(1);
11021        accounts.add_root_and_flush_write_cache(1);
11022        accounts.calculate_accounts_delta_hash(2);
11023        accounts.add_root_and_flush_write_cache(2);
11024
11025        // Account ref counts should match how many slots they were stored in
11026        // Account 1 = 3 slots; account 2 = 1 slot
11027        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 3);
11028        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 1);
11029
11030        accounts.clean_accounts_for_tests();
11031        // Slots 0 and 1 should each have been cleaned because all of their
11032        // accounts are zero lamports
11033        assert!(accounts.storage.get_slot_storage_entry(0).is_none());
11034        assert!(accounts.storage.get_slot_storage_entry(1).is_none());
11035        // Slot 2 only has a zero lamport account as well. But, calc_delete_dependencies()
11036        // should exclude slot 2 from the clean due to changes in other slots
11037        assert!(accounts.storage.get_slot_storage_entry(2).is_some());
11038        // Index ref counts should be consistent with the slot stores. Account 1 ref count
11039        // should be 1 since slot 2 is the only alive slot; account 2 should have a ref
11040        // count of 0 due to slot 0 being dead
11041        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 1);
11042        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 0);
11043
11044        accounts.clean_accounts_for_tests();
11045        // Slot 2 will now be cleaned, which will leave account 1 with a ref count of 0
11046        assert!(accounts.storage.get_slot_storage_entry(2).is_none());
11047        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
11048    }
11049
11050    #[test]
11051    fn test_clean_zero_lamport_and_old_roots() {
11052        solana_logger::setup();
11053
11054        let accounts = AccountsDb::new_single_for_tests();
11055        let pubkey = solana_sdk::pubkey::new_rand();
11056        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11057        let zero_lamport_account =
11058            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11059
11060        // Store a zero-lamport account
11061        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11062        accounts.store_for_tests(1, &[(&pubkey, &zero_lamport_account)]);
11063
11064        // Simulate rooting the zero-lamport account, should be a
11065        // candidate for cleaning
11066        accounts.calculate_accounts_delta_hash(0);
11067        accounts.add_root_and_flush_write_cache(0);
11068        accounts.calculate_accounts_delta_hash(1);
11069        accounts.add_root_and_flush_write_cache(1);
11070
11071        // Slot 0 should be removed, and
11072        // zero-lamport account should be cleaned
11073        accounts.clean_accounts_for_tests();
11074
11075        assert!(accounts.storage.get_slot_storage_entry(0).is_none());
11076        assert!(accounts.storage.get_slot_storage_entry(1).is_none());
11077
11078        // Slot 0 should be cleaned because all it's accounts have been
11079        // updated in the rooted slot 1
11080        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11081
11082        // Slot 1 should be cleaned because all it's accounts are
11083        // zero lamports, and are not present in any other slot's
11084        // storage entries
11085        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11086
11087        // zero lamport account, should no longer exist in accounts index
11088        // because it has been removed
11089        assert!(!accounts.accounts_index.contains_with(&pubkey, None, None));
11090    }
11091
11092    #[test]
11093    fn test_clean_old_with_normal_account() {
11094        solana_logger::setup();
11095
11096        let accounts = AccountsDb::new_single_for_tests();
11097        let pubkey = solana_sdk::pubkey::new_rand();
11098        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11099        //store an account
11100        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11101        accounts.store_for_tests(1, &[(&pubkey, &account)]);
11102
11103        // simulate slots are rooted after while
11104        accounts.calculate_accounts_delta_hash(0);
11105        accounts.add_root_and_flush_write_cache(0);
11106        accounts.calculate_accounts_delta_hash(1);
11107        accounts.add_root_and_flush_write_cache(1);
11108
11109        //even if rooted, old state isn't cleaned up
11110        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
11111        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11112
11113        accounts.clean_accounts_for_tests();
11114
11115        //now old state is cleaned up
11116        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11117        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11118    }
11119
11120    #[test]
11121    fn test_clean_old_with_zero_lamport_account() {
11122        solana_logger::setup();
11123
11124        let accounts = AccountsDb::new_single_for_tests();
11125        let pubkey1 = solana_sdk::pubkey::new_rand();
11126        let pubkey2 = solana_sdk::pubkey::new_rand();
11127        let normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11128        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11129        //store an account
11130        accounts.store_for_tests(0, &[(&pubkey1, &normal_account)]);
11131        accounts.store_for_tests(1, &[(&pubkey1, &zero_account)]);
11132        accounts.store_for_tests(0, &[(&pubkey2, &normal_account)]);
11133        accounts.store_for_tests(1, &[(&pubkey2, &normal_account)]);
11134
11135        //simulate slots are rooted after while
11136        accounts.calculate_accounts_delta_hash(0);
11137        accounts.add_root_and_flush_write_cache(0);
11138        accounts.calculate_accounts_delta_hash(1);
11139        accounts.add_root_and_flush_write_cache(1);
11140
11141        //even if rooted, old state isn't cleaned up
11142        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
11143        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
11144
11145        accounts.print_accounts_stats("");
11146
11147        accounts.clean_accounts_for_tests();
11148
11149        //Old state behind zero-lamport account is cleaned up
11150        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11151        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
11152    }
11153
11154    #[test]
11155    fn test_clean_old_with_both_normal_and_zero_lamport_accounts() {
11156        solana_logger::setup();
11157
11158        let mut accounts = AccountsDb {
11159            account_indexes: spl_token_mint_index_enabled(),
11160            ..AccountsDb::new_single_for_tests()
11161        };
11162        let pubkey1 = solana_sdk::pubkey::new_rand();
11163        let pubkey2 = solana_sdk::pubkey::new_rand();
11164
11165        // Set up account to be added to secondary index
11166        let mint_key = Pubkey::new_unique();
11167        let mut account_data_with_mint =
11168            vec![0; solana_inline_spl::token::Account::get_packed_len()];
11169        account_data_with_mint[..PUBKEY_BYTES].clone_from_slice(&(mint_key.to_bytes()));
11170
11171        let mut normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11172        normal_account.set_owner(solana_inline_spl::token::id());
11173        normal_account.set_data(account_data_with_mint.clone());
11174        let mut zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11175        zero_account.set_owner(solana_inline_spl::token::id());
11176        zero_account.set_data(account_data_with_mint);
11177
11178        //store an account
11179        accounts.store_for_tests(0, &[(&pubkey1, &normal_account)]);
11180        accounts.store_for_tests(0, &[(&pubkey1, &normal_account)]);
11181        accounts.store_for_tests(1, &[(&pubkey1, &zero_account)]);
11182        accounts.store_for_tests(0, &[(&pubkey2, &normal_account)]);
11183        accounts.store_for_tests(2, &[(&pubkey2, &normal_account)]);
11184
11185        //simulate slots are rooted after while
11186        accounts.calculate_accounts_delta_hash(0);
11187        accounts.add_root_and_flush_write_cache(0);
11188        accounts.calculate_accounts_delta_hash(1);
11189        accounts.add_root_and_flush_write_cache(1);
11190        accounts.calculate_accounts_delta_hash(2);
11191        accounts.add_root_and_flush_write_cache(2);
11192
11193        //even if rooted, old state isn't cleaned up
11194        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
11195        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11196        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
11197
11198        // Secondary index should still find both pubkeys
11199        let mut found_accounts = HashSet::new();
11200        let index_key = IndexKey::SplTokenMint(mint_key);
11201        let bank_id = 0;
11202        accounts
11203            .accounts_index
11204            .index_scan_accounts(
11205                &Ancestors::default(),
11206                bank_id,
11207                index_key,
11208                |key, _| {
11209                    found_accounts.insert(*key);
11210                },
11211                &ScanConfig::default(),
11212            )
11213            .unwrap();
11214        assert_eq!(found_accounts.len(), 2);
11215        assert!(found_accounts.contains(&pubkey1));
11216        assert!(found_accounts.contains(&pubkey2));
11217
11218        {
11219            accounts.account_indexes.keys = Some(AccountSecondaryIndexesIncludeExclude {
11220                exclude: true,
11221                keys: [mint_key].iter().cloned().collect::<HashSet<Pubkey>>(),
11222            });
11223            // Secondary index can't be used - do normal scan: should still find both pubkeys
11224            let mut found_accounts = HashSet::new();
11225            let used_index = accounts
11226                .index_scan_accounts(
11227                    &Ancestors::default(),
11228                    bank_id,
11229                    index_key,
11230                    |account| {
11231                        found_accounts.insert(*account.unwrap().0);
11232                    },
11233                    &ScanConfig::default(),
11234                )
11235                .unwrap();
11236            assert!(!used_index);
11237            assert_eq!(found_accounts.len(), 2);
11238            assert!(found_accounts.contains(&pubkey1));
11239            assert!(found_accounts.contains(&pubkey2));
11240
11241            accounts.account_indexes.keys = None;
11242
11243            // Secondary index can now be used since it isn't marked as excluded
11244            let mut found_accounts = HashSet::new();
11245            let used_index = accounts
11246                .index_scan_accounts(
11247                    &Ancestors::default(),
11248                    bank_id,
11249                    index_key,
11250                    |account| {
11251                        found_accounts.insert(*account.unwrap().0);
11252                    },
11253                    &ScanConfig::default(),
11254                )
11255                .unwrap();
11256            assert!(used_index);
11257            assert_eq!(found_accounts.len(), 2);
11258            assert!(found_accounts.contains(&pubkey1));
11259            assert!(found_accounts.contains(&pubkey2));
11260
11261            accounts.account_indexes.keys = None;
11262        }
11263
11264        accounts.clean_accounts_for_tests();
11265
11266        //both zero lamport and normal accounts are cleaned up
11267        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11268        // The only store to slot 1 was a zero lamport account, should
11269        // be purged by zero-lamport cleaning logic because slot 1 is
11270        // rooted
11271        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11272        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
11273
11274        // `pubkey1`, a zero lamport account, should no longer exist in accounts index
11275        // because it has been removed by the clean
11276        assert!(!accounts.accounts_index.contains_with(&pubkey1, None, None));
11277
11278        // Secondary index should have purged `pubkey1` as well
11279        let mut found_accounts = vec![];
11280        accounts
11281            .accounts_index
11282            .index_scan_accounts(
11283                &Ancestors::default(),
11284                bank_id,
11285                IndexKey::SplTokenMint(mint_key),
11286                |key, _| found_accounts.push(*key),
11287                &ScanConfig::default(),
11288            )
11289            .unwrap();
11290        assert_eq!(found_accounts, vec![pubkey2]);
11291    }
11292
11293    #[test]
11294    fn test_clean_max_slot_zero_lamport_account() {
11295        solana_logger::setup();
11296
11297        let accounts = AccountsDb::new_single_for_tests();
11298        let pubkey = solana_sdk::pubkey::new_rand();
11299        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11300        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11301
11302        // store an account, make it a zero lamport account
11303        // in slot 1
11304        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11305        accounts.store_for_tests(1, &[(&pubkey, &zero_account)]);
11306
11307        // simulate slots are rooted after while
11308        accounts.calculate_accounts_delta_hash(0);
11309        accounts.add_root_and_flush_write_cache(0);
11310        accounts.calculate_accounts_delta_hash(1);
11311        accounts.add_root_and_flush_write_cache(1);
11312
11313        // Only clean up to account 0, should not purge slot 0 based on
11314        // updates in later slots in slot 1
11315        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
11316        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11317        accounts.clean_accounts(
11318            Some(0),
11319            false,
11320            &EpochSchedule::default(),
11321            OldStoragesPolicy::Leave,
11322        );
11323        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
11324        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
11325        assert!(accounts.accounts_index.contains_with(&pubkey, None, None));
11326
11327        // Now the account can be cleaned up
11328        accounts.clean_accounts(
11329            Some(1),
11330            false,
11331            &EpochSchedule::default(),
11332            OldStoragesPolicy::Leave,
11333        );
11334        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
11335        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
11336
11337        // The zero lamport account, should no longer exist in accounts index
11338        // because it has been removed
11339        assert!(!accounts.accounts_index.contains_with(&pubkey, None, None));
11340    }
11341
11342    #[test]
11343    fn test_uncleaned_roots_with_account() {
11344        solana_logger::setup();
11345
11346        let accounts = AccountsDb::new_single_for_tests();
11347        let pubkey = solana_sdk::pubkey::new_rand();
11348        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11349        //store an account
11350        accounts.store_for_tests(0, &[(&pubkey, &account)]);
11351        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11352
11353        // simulate slots are rooted after while
11354        accounts.add_root_and_flush_write_cache(0);
11355        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
11356
11357        //now uncleaned roots are cleaned up
11358        accounts.clean_accounts_for_tests();
11359        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11360    }
11361
11362    #[test]
11363    fn test_uncleaned_roots_with_no_account() {
11364        solana_logger::setup();
11365
11366        let accounts = AccountsDb::new_single_for_tests();
11367
11368        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11369
11370        // simulate slots are rooted after while
11371        accounts.add_root_and_flush_write_cache(0);
11372        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
11373
11374        //now uncleaned roots are cleaned up
11375        accounts.clean_accounts_for_tests();
11376        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
11377    }
11378
11379    fn assert_no_stores(accounts: &AccountsDb, slot: Slot) {
11380        let store = accounts.storage.get_slot_storage_entry(slot);
11381        assert!(store.is_none());
11382    }
11383
11384    #[test]
11385    fn test_accounts_db_purge_keep_live() {
11386        solana_logger::setup();
11387        let some_lamport = 223;
11388        let zero_lamport = 0;
11389        let no_data = 0;
11390        let owner = *AccountSharedData::default().owner();
11391
11392        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11393        let pubkey = solana_sdk::pubkey::new_rand();
11394
11395        let account2 = AccountSharedData::new(some_lamport, no_data, &owner);
11396        let pubkey2 = solana_sdk::pubkey::new_rand();
11397
11398        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11399
11400        let accounts = AccountsDb::new_single_for_tests();
11401        accounts.calculate_accounts_delta_hash(0);
11402        accounts.add_root_and_flush_write_cache(0);
11403
11404        // Step A
11405        let mut current_slot = 1;
11406        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
11407        // Store another live account to slot 1 which will prevent any purge
11408        // since the store count will not be zero
11409        accounts.store_for_tests(current_slot, &[(&pubkey2, &account2)]);
11410        accounts.calculate_accounts_delta_hash(current_slot);
11411        accounts.add_root_and_flush_write_cache(current_slot);
11412        let (slot1, account_info1) = accounts
11413            .accounts_index
11414            .get_with_and_then(&pubkey, None, None, false, |(slot, account_info)| {
11415                (slot, account_info)
11416            })
11417            .unwrap();
11418        let (slot2, account_info2) = accounts
11419            .accounts_index
11420            .get_with_and_then(&pubkey2, None, None, false, |(slot, account_info)| {
11421                (slot, account_info)
11422            })
11423            .unwrap();
11424        assert_eq!(slot1, current_slot);
11425        assert_eq!(slot1, slot2);
11426        assert_eq!(account_info1.store_id(), account_info2.store_id());
11427
11428        // Step B
11429        current_slot += 1;
11430        let zero_lamport_slot = current_slot;
11431        accounts.store_for_tests(current_slot, &[(&pubkey, &zero_lamport_account)]);
11432        accounts.calculate_accounts_delta_hash(current_slot);
11433        accounts.add_root_and_flush_write_cache(current_slot);
11434
11435        accounts.assert_load_account(current_slot, pubkey, zero_lamport);
11436
11437        current_slot += 1;
11438        accounts.calculate_accounts_delta_hash(current_slot);
11439        accounts.add_root_and_flush_write_cache(current_slot);
11440
11441        accounts.print_accounts_stats("pre_purge");
11442
11443        accounts.clean_accounts_for_tests();
11444
11445        accounts.print_accounts_stats("post_purge");
11446
11447        // The earlier entry for pubkey in the account index is purged,
11448        let (slot_list_len, index_slot) = {
11449            let account_entry = accounts.accounts_index.get_cloned(&pubkey).unwrap();
11450            let slot_list = account_entry.slot_list.read().unwrap();
11451            (slot_list.len(), slot_list[0].0)
11452        };
11453        assert_eq!(slot_list_len, 1);
11454        // Zero lamport entry was not the one purged
11455        assert_eq!(index_slot, zero_lamport_slot);
11456        // The ref count should still be 2 because no slots were purged
11457        assert_eq!(accounts.ref_count_for_pubkey(&pubkey), 2);
11458
11459        // storage for slot 1 had 2 accounts, now has 1 after pubkey 1
11460        // was reclaimed
11461        accounts.check_storage(1, 1, 2);
11462        // storage for slot 2 had 1 accounts, now has 1
11463        accounts.check_storage(2, 1, 1);
11464    }
11465
11466    #[test]
11467    fn test_accounts_db_purge1() {
11468        solana_logger::setup();
11469        let some_lamport = 223;
11470        let zero_lamport = 0;
11471        let no_data = 0;
11472        let owner = *AccountSharedData::default().owner();
11473
11474        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11475        let pubkey = solana_sdk::pubkey::new_rand();
11476
11477        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11478
11479        let accounts = AccountsDb::new_single_for_tests();
11480        accounts.add_root(0);
11481
11482        let mut current_slot = 1;
11483        accounts.store_for_tests(current_slot, &[(&pubkey, &account)]);
11484        accounts.calculate_accounts_delta_hash(current_slot);
11485        accounts.add_root_and_flush_write_cache(current_slot);
11486
11487        current_slot += 1;
11488        accounts.store_for_tests(current_slot, &[(&pubkey, &zero_lamport_account)]);
11489        accounts.calculate_accounts_delta_hash(current_slot);
11490        accounts.add_root_and_flush_write_cache(current_slot);
11491
11492        accounts.assert_load_account(current_slot, pubkey, zero_lamport);
11493
11494        // Otherwise slot 2 will not be removed
11495        current_slot += 1;
11496        accounts.calculate_accounts_delta_hash(current_slot);
11497        accounts.add_root_and_flush_write_cache(current_slot);
11498
11499        accounts.print_accounts_stats("pre_purge");
11500
11501        let ancestors = linear_ancestors(current_slot);
11502        info!("ancestors: {:?}", ancestors);
11503        let hash = accounts.update_accounts_hash_for_tests(current_slot, &ancestors, true, true);
11504
11505        accounts.clean_accounts_for_tests();
11506
11507        assert_eq!(
11508            accounts.update_accounts_hash_for_tests(current_slot, &ancestors, true, true),
11509            hash
11510        );
11511
11512        accounts.print_accounts_stats("post_purge");
11513
11514        // Make sure the index is for pubkey cleared
11515        assert!(!accounts.accounts_index.contains(&pubkey));
11516
11517        // slot 1 & 2 should not have any stores
11518        assert_no_stores(&accounts, 1);
11519        assert_no_stores(&accounts, 2);
11520    }
11521
11522    #[test]
11523    #[ignore]
11524    fn test_store_account_stress() {
11525        let slot = 42;
11526        let num_threads = 2;
11527
11528        let min_file_bytes = std::mem::size_of::<StoredMeta>() + std::mem::size_of::<AccountMeta>();
11529
11530        let db = Arc::new(AccountsDb {
11531            file_size: min_file_bytes as u64,
11532            ..AccountsDb::new_single_for_tests()
11533        });
11534
11535        db.add_root(slot);
11536        let thread_hdls: Vec<_> = (0..num_threads)
11537            .map(|_| {
11538                let db = db.clone();
11539                std::thread::Builder::new()
11540                    .name("account-writers".to_string())
11541                    .spawn(move || {
11542                        let pubkey = solana_sdk::pubkey::new_rand();
11543                        let mut account = AccountSharedData::new(1, 0, &pubkey);
11544                        let mut i = 0;
11545                        loop {
11546                            let account_bal = thread_rng().gen_range(1..99);
11547                            account.set_lamports(account_bal);
11548                            db.store_for_tests(slot, &[(&pubkey, &account)]);
11549
11550                            let (account, slot) = db
11551                                .load_without_fixed_root(&Ancestors::default(), &pubkey)
11552                                .unwrap_or_else(|| {
11553                                    panic!("Could not fetch stored account {pubkey}, iter {i}")
11554                                });
11555                            assert_eq!(slot, slot);
11556                            assert_eq!(account.lamports(), account_bal);
11557                            i += 1;
11558                        }
11559                    })
11560                    .unwrap()
11561            })
11562            .collect();
11563
11564        for t in thread_hdls {
11565            t.join().unwrap();
11566        }
11567    }
11568
11569    #[test]
11570    fn test_accountsdb_scan_accounts() {
11571        solana_logger::setup();
11572        let db = AccountsDb::new_single_for_tests();
11573        let key = Pubkey::default();
11574        let key0 = solana_sdk::pubkey::new_rand();
11575        let account0 = AccountSharedData::new(1, 0, &key);
11576
11577        db.store_for_tests(0, &[(&key0, &account0)]);
11578
11579        let key1 = solana_sdk::pubkey::new_rand();
11580        let account1 = AccountSharedData::new(2, 0, &key);
11581        db.store_for_tests(1, &[(&key1, &account1)]);
11582
11583        let ancestors = vec![(0, 0)].into_iter().collect();
11584        let mut accounts = Vec::new();
11585        db.unchecked_scan_accounts(
11586            "",
11587            &ancestors,
11588            |_, account, _| {
11589                accounts.push(account.take_account());
11590            },
11591            &ScanConfig::default(),
11592        );
11593        assert_eq!(accounts, vec![account0]);
11594
11595        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
11596        let mut accounts = Vec::new();
11597        db.unchecked_scan_accounts(
11598            "",
11599            &ancestors,
11600            |_, account, _| {
11601                accounts.push(account.take_account());
11602            },
11603            &ScanConfig::default(),
11604        );
11605        assert_eq!(accounts.len(), 2);
11606    }
11607
11608    #[test]
11609    fn test_cleanup_key_not_removed() {
11610        solana_logger::setup();
11611        let db = AccountsDb::new_single_for_tests();
11612
11613        let key = Pubkey::default();
11614        let key0 = solana_sdk::pubkey::new_rand();
11615        let account0 = AccountSharedData::new(1, 0, &key);
11616
11617        db.store_for_tests(0, &[(&key0, &account0)]);
11618
11619        let key1 = solana_sdk::pubkey::new_rand();
11620        let account1 = AccountSharedData::new(2, 0, &key);
11621        db.store_for_tests(1, &[(&key1, &account1)]);
11622
11623        db.print_accounts_stats("pre");
11624
11625        let slots: HashSet<Slot> = vec![1].into_iter().collect();
11626        let purge_keys = [(key1, slots)];
11627        let _ = db.purge_keys_exact(purge_keys.iter());
11628
11629        let account2 = AccountSharedData::new(3, 0, &key);
11630        db.store_for_tests(2, &[(&key1, &account2)]);
11631
11632        db.print_accounts_stats("post");
11633        let ancestors = vec![(2, 0)].into_iter().collect();
11634        assert_eq!(
11635            db.load_without_fixed_root(&ancestors, &key1)
11636                .unwrap()
11637                .0
11638                .lamports(),
11639            3
11640        );
11641    }
11642
11643    #[test]
11644    fn test_store_large_account() {
11645        solana_logger::setup();
11646        let db = AccountsDb::new_single_for_tests();
11647
11648        let key = Pubkey::default();
11649        let data_len = DEFAULT_FILE_SIZE as usize + 7;
11650        let account = AccountSharedData::new(1, data_len, &key);
11651
11652        db.store_for_tests(0, &[(&key, &account)]);
11653
11654        let ancestors = vec![(0, 0)].into_iter().collect();
11655        let ret = db.load_without_fixed_root(&ancestors, &key).unwrap();
11656        assert_eq!(ret.0.data().len(), data_len);
11657    }
11658
11659    #[test]
11660    fn test_stored_readable_account() {
11661        let lamports = 1;
11662        let owner = Pubkey::new_unique();
11663        let executable = true;
11664        let rent_epoch = 2;
11665        let meta = StoredMeta {
11666            write_version_obsolete: 5,
11667            pubkey: Pubkey::new_unique(),
11668            data_len: 7,
11669        };
11670        let account_meta = AccountMeta {
11671            lamports,
11672            owner,
11673            executable,
11674            rent_epoch,
11675        };
11676        let data = Vec::new();
11677        let account = Account {
11678            lamports,
11679            owner,
11680            executable,
11681            rent_epoch,
11682            data: data.clone(),
11683        };
11684        let offset = 99 * std::mem::size_of::<u64>(); // offset needs to be 8 byte aligned
11685        let stored_size = 101;
11686        let hash = AccountHash(Hash::new_unique());
11687        let stored_account = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
11688            meta: &meta,
11689            account_meta: &account_meta,
11690            data: &data,
11691            offset,
11692            stored_size,
11693            hash: &hash,
11694        });
11695        assert!(accounts_equal(&account, &stored_account));
11696    }
11697
11698    /// A place holder stored size for a cached entry. We don't need to store the size for cached entries, but we have to pass something.
11699    /// stored size is only used for shrinking. We don't shrink items in the write cache.
11700    const CACHE_VIRTUAL_STORED_SIZE: StoredSize = 0;
11701
11702    #[test]
11703    fn test_hash_stored_account() {
11704        // Number are just sequential.
11705        let meta = StoredMeta {
11706            write_version_obsolete: 0x09_0a_0b_0c_0d_0e_0f_10,
11707            data_len: 0x11_12_13_14_15_16_17_18,
11708            pubkey: Pubkey::from([
11709                0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26,
11710                0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34,
11711                0x35, 0x36, 0x37, 0x38,
11712            ]),
11713        };
11714        let account_meta = AccountMeta {
11715            lamports: 0x39_3a_3b_3c_3d_3e_3f_40,
11716            rent_epoch: 0x41_42_43_44_45_46_47_48,
11717            owner: Pubkey::from([
11718                0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56,
11719                0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64,
11720                0x65, 0x66, 0x67, 0x68,
11721            ]),
11722            executable: false,
11723        };
11724        const ACCOUNT_DATA_LEN: usize = 3;
11725        let data: [u8; ACCOUNT_DATA_LEN] = [0x69, 0x6a, 0x6b];
11726        let offset: usize = 0x6c_6d_6e_6f_70_71_72_73;
11727        let hash = AccountHash(Hash::from([
11728            0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x81,
11729            0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f,
11730            0x90, 0x91, 0x92, 0x93,
11731        ]));
11732
11733        let stored_account = StoredAccountMeta::AppendVec(AppendVecStoredAccountMeta {
11734            meta: &meta,
11735            account_meta: &account_meta,
11736            data: &data,
11737            offset,
11738            stored_size: CACHE_VIRTUAL_STORED_SIZE as usize,
11739            hash: &hash,
11740        });
11741        let account = stored_account.to_account_shared_data();
11742
11743        let expected_account_hash =
11744            AccountHash(Hash::from_str("4xuaE8UfH8EYsPyDZvJXUScoZSyxUJf2BpzVMLTFh497").unwrap());
11745
11746        assert_eq!(
11747            AccountsDb::hash_account(&stored_account, stored_account.pubkey(),),
11748            expected_account_hash,
11749            "StoredAccountMeta's data layout might be changed; update hashing if needed."
11750        );
11751        assert_eq!(
11752            AccountsDb::hash_account(&account, stored_account.pubkey(),),
11753            expected_account_hash,
11754            "Account-based hashing must be consistent with StoredAccountMeta-based one."
11755        );
11756    }
11757
11758    #[test]
11759    fn test_bank_hash_stats() {
11760        solana_logger::setup();
11761        let db = AccountsDb::new_single_for_tests();
11762
11763        let key = Pubkey::default();
11764        let some_data_len = 5;
11765        let some_slot: Slot = 0;
11766        let account = AccountSharedData::new(1, some_data_len, &key);
11767        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11768
11769        db.store_for_tests(some_slot, &[(&key, &account)]);
11770        let mut account = db.load_without_fixed_root(&ancestors, &key).unwrap().0;
11771        account.checked_sub_lamports(1).unwrap();
11772        account.set_executable(true);
11773        db.store_for_tests(some_slot, &[(&key, &account)]);
11774        db.add_root(some_slot);
11775
11776        let stats = db.get_bank_hash_stats(some_slot).unwrap();
11777        assert_eq!(stats.num_updated_accounts, 1);
11778        assert_eq!(stats.num_removed_accounts, 1);
11779        assert_eq!(stats.num_lamports_stored, 1);
11780        assert_eq!(stats.total_data_len, 2 * some_data_len as u64);
11781        assert_eq!(stats.num_executable_accounts, 1);
11782    }
11783
11784    // something we can get a ref to
11785    lazy_static! {
11786        pub static ref EPOCH_SCHEDULE: EpochSchedule = EpochSchedule::default();
11787        pub static ref RENT_COLLECTOR: RentCollector = RentCollector::default();
11788    }
11789
11790    impl<'a> CalcAccountsHashConfig<'a> {
11791        pub(crate) fn default() -> Self {
11792            Self {
11793                use_bg_thread_pool: false,
11794                ancestors: None,
11795                epoch_schedule: &EPOCH_SCHEDULE,
11796                rent_collector: &RENT_COLLECTOR,
11797                store_detailed_debug_info_on_failure: false,
11798            }
11799        }
11800    }
11801
11802    #[test]
11803    fn test_verify_accounts_hash() {
11804        solana_logger::setup();
11805        let db = AccountsDb::new_single_for_tests();
11806
11807        let key = solana_sdk::pubkey::new_rand();
11808        let some_data_len = 0;
11809        let some_slot: Slot = 0;
11810        let account = AccountSharedData::new(1, some_data_len, &key);
11811        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11812        let epoch_schedule = EpochSchedule::default();
11813        let rent_collector = RentCollector::default();
11814
11815        db.store_for_tests(some_slot, &[(&key, &account)]);
11816        db.add_root_and_flush_write_cache(some_slot);
11817        let (_, capitalization) =
11818            db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
11819
11820        let config = VerifyAccountsHashAndLamportsConfig::new_for_test(
11821            &ancestors,
11822            &epoch_schedule,
11823            &rent_collector,
11824        );
11825
11826        assert_matches!(
11827            db.verify_accounts_hash_and_lamports_for_tests(some_slot, 1, config.clone()),
11828            Ok(_)
11829        );
11830
11831        db.accounts_hashes.lock().unwrap().remove(&some_slot);
11832
11833        assert_matches!(
11834            db.verify_accounts_hash_and_lamports_for_tests(some_slot, 1, config.clone()),
11835            Err(AccountsHashVerificationError::MissingAccountsHash)
11836        );
11837
11838        db.set_accounts_hash(
11839            some_slot,
11840            (AccountsHash(Hash::new(&[0xca; HASH_BYTES])), capitalization),
11841        );
11842
11843        assert_matches!(
11844            db.verify_accounts_hash_and_lamports_for_tests(some_slot, 1, config),
11845            Err(AccountsHashVerificationError::MismatchedAccountsHash)
11846        );
11847    }
11848
11849    #[test]
11850    fn test_verify_bank_capitalization() {
11851        for pass in 0..2 {
11852            solana_logger::setup();
11853            let db = AccountsDb::new_single_for_tests();
11854
11855            let key = solana_sdk::pubkey::new_rand();
11856            let some_data_len = 0;
11857            let some_slot: Slot = 0;
11858            let account = AccountSharedData::new(1, some_data_len, &key);
11859            let ancestors = vec![(some_slot, 0)].into_iter().collect();
11860            let epoch_schedule = EpochSchedule::default();
11861            let rent_collector = RentCollector::default();
11862            let config = VerifyAccountsHashAndLamportsConfig::new_for_test(
11863                &ancestors,
11864                &epoch_schedule,
11865                &rent_collector,
11866            );
11867
11868            db.store_for_tests(some_slot, &[(&key, &account)]);
11869            if pass == 0 {
11870                db.add_root_and_flush_write_cache(some_slot);
11871                db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
11872
11873                assert_matches!(
11874                    db.verify_accounts_hash_and_lamports_for_tests(some_slot, 1, config.clone()),
11875                    Ok(_)
11876                );
11877                continue;
11878            }
11879
11880            let native_account_pubkey = solana_sdk::pubkey::new_rand();
11881            db.store_for_tests(
11882                some_slot,
11883                &[(
11884                    &native_account_pubkey,
11885                    &solana_sdk::native_loader::create_loadable_account_for_test("foo"),
11886                )],
11887            );
11888            db.add_root_and_flush_write_cache(some_slot);
11889            db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
11890
11891            assert_matches!(
11892                db.verify_accounts_hash_and_lamports_for_tests(some_slot, 2, config.clone()),
11893                Ok(_)
11894            );
11895
11896            assert_matches!(
11897                db.verify_accounts_hash_and_lamports_for_tests(some_slot, 10, config),
11898                Err(AccountsHashVerificationError::MismatchedTotalLamports(expected, actual)) if expected == 2 && actual == 10
11899            );
11900        }
11901    }
11902
11903    #[test]
11904    fn test_verify_accounts_hash_no_account() {
11905        solana_logger::setup();
11906        let db = AccountsDb::new_single_for_tests();
11907
11908        let some_slot: Slot = 0;
11909        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11910
11911        db.add_root(some_slot);
11912        db.update_accounts_hash_for_tests(some_slot, &ancestors, true, true);
11913
11914        let epoch_schedule = EpochSchedule::default();
11915        let rent_collector = RentCollector::default();
11916        let config = VerifyAccountsHashAndLamportsConfig::new_for_test(
11917            &ancestors,
11918            &epoch_schedule,
11919            &rent_collector,
11920        );
11921
11922        assert_matches!(
11923            db.verify_accounts_hash_and_lamports_for_tests(some_slot, 0, config),
11924            Ok(_)
11925        );
11926    }
11927
11928    #[test]
11929    fn test_verify_accounts_hash_bad_account_hash() {
11930        solana_logger::setup();
11931        let db = AccountsDb::new_single_for_tests();
11932
11933        let key = Pubkey::default();
11934        let some_data_len = 0;
11935        let some_slot: Slot = 0;
11936        let account = AccountSharedData::new(1, some_data_len, &key);
11937        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11938
11939        let accounts = &[(&key, &account)][..];
11940        db.update_accounts_hash_for_tests(some_slot, &ancestors, false, false);
11941
11942        // provide bogus account hashes
11943        db.store_accounts_unfrozen(
11944            (some_slot, accounts),
11945            &StoreTo::Storage(&db.find_storage_candidate(some_slot)),
11946            None,
11947            StoreReclaims::Default,
11948            UpdateIndexThreadSelection::PoolWithThreshold,
11949        );
11950        db.add_root(some_slot);
11951
11952        let epoch_schedule = EpochSchedule::default();
11953        let rent_collector = RentCollector::default();
11954        let config = VerifyAccountsHashAndLamportsConfig::new_for_test(
11955            &ancestors,
11956            &epoch_schedule,
11957            &rent_collector,
11958        );
11959
11960        assert_matches!(
11961            db.verify_accounts_hash_and_lamports_for_tests(some_slot, 1, config),
11962            Err(AccountsHashVerificationError::MismatchedAccountsHash)
11963        );
11964    }
11965
11966    #[test]
11967    fn test_storage_finder() {
11968        solana_logger::setup();
11969        let db = AccountsDb {
11970            file_size: 16 * 1024,
11971            ..AccountsDb::new_single_for_tests()
11972        };
11973        let key = solana_sdk::pubkey::new_rand();
11974        let lamports = 100;
11975        let data_len = 8190;
11976        let account = AccountSharedData::new(lamports, data_len, &solana_sdk::pubkey::new_rand());
11977        // pre-populate with a smaller empty store
11978        db.create_and_insert_store(1, 8192, "test_storage_finder");
11979        db.store_for_tests(1, &[(&key, &account)]);
11980    }
11981
11982    #[test]
11983    fn test_get_snapshot_storages_empty() {
11984        let db = AccountsDb::new_single_for_tests();
11985        assert!(db.get_snapshot_storages(..=0).0.is_empty());
11986    }
11987
11988    #[test]
11989    fn test_get_snapshot_storages_only_older_than_or_equal_to_snapshot_slot() {
11990        let db = AccountsDb::new_single_for_tests();
11991
11992        let key = Pubkey::default();
11993        let account = AccountSharedData::new(1, 0, &key);
11994        let before_slot = 0;
11995        let base_slot = before_slot + 1;
11996        let after_slot = base_slot + 1;
11997
11998        db.store_for_tests(base_slot, &[(&key, &account)]);
11999        db.add_root_and_flush_write_cache(base_slot);
12000        assert!(db.get_snapshot_storages(..=before_slot).0.is_empty());
12001
12002        assert_eq!(1, db.get_snapshot_storages(..=base_slot).0.len());
12003        assert_eq!(1, db.get_snapshot_storages(..=after_slot).0.len());
12004    }
12005
12006    #[test]
12007    fn test_get_snapshot_storages_only_non_empty() {
12008        for pass in 0..2 {
12009            let db = AccountsDb::new_single_for_tests();
12010
12011            let key = Pubkey::default();
12012            let account = AccountSharedData::new(1, 0, &key);
12013            let base_slot = 0;
12014            let after_slot = base_slot + 1;
12015
12016            db.store_for_tests(base_slot, &[(&key, &account)]);
12017            if pass == 0 {
12018                db.add_root_and_flush_write_cache(base_slot);
12019                db.storage.remove(&base_slot, false);
12020                assert!(db.get_snapshot_storages(..=after_slot).0.is_empty());
12021                continue;
12022            }
12023
12024            db.store_for_tests(base_slot, &[(&key, &account)]);
12025            db.add_root_and_flush_write_cache(base_slot);
12026            assert_eq!(1, db.get_snapshot_storages(..=after_slot).0.len());
12027        }
12028    }
12029
12030    #[test]
12031    fn test_get_snapshot_storages_only_roots() {
12032        let db = AccountsDb::new_single_for_tests();
12033
12034        let key = Pubkey::default();
12035        let account = AccountSharedData::new(1, 0, &key);
12036        let base_slot = 0;
12037        let after_slot = base_slot + 1;
12038
12039        db.store_for_tests(base_slot, &[(&key, &account)]);
12040        assert!(db.get_snapshot_storages(..=after_slot).0.is_empty());
12041
12042        db.add_root_and_flush_write_cache(base_slot);
12043        assert_eq!(1, db.get_snapshot_storages(..=after_slot).0.len());
12044    }
12045
12046    #[test]
12047    fn test_get_snapshot_storages_exclude_empty() {
12048        let db = AccountsDb::new_single_for_tests();
12049
12050        let key = Pubkey::default();
12051        let account = AccountSharedData::new(1, 0, &key);
12052        let base_slot = 0;
12053        let after_slot = base_slot + 1;
12054
12055        db.store_for_tests(base_slot, &[(&key, &account)]);
12056        db.add_root_and_flush_write_cache(base_slot);
12057        assert_eq!(1, db.get_snapshot_storages(..=after_slot).0.len());
12058
12059        db.storage
12060            .get_slot_storage_entry(0)
12061            .unwrap()
12062            .remove_accounts(0, true, 1);
12063        assert!(db.get_snapshot_storages(..=after_slot).0.is_empty());
12064    }
12065
12066    #[test]
12067    fn test_get_snapshot_storages_with_base_slot() {
12068        let db = AccountsDb::new_single_for_tests();
12069
12070        let key = Pubkey::default();
12071        let account = AccountSharedData::new(1, 0, &key);
12072
12073        let slot = 10;
12074        db.store_for_tests(slot, &[(&key, &account)]);
12075        db.add_root_and_flush_write_cache(slot);
12076        assert_eq!(0, db.get_snapshot_storages(slot + 1..=slot + 1).0.len());
12077        assert_eq!(1, db.get_snapshot_storages(slot..=slot + 1).0.len());
12078    }
12079
12080    define_accounts_db_test!(
12081        test_storage_remove_account_double_remove,
12082        panic = "double remove of account in slot: 0/store: 0!!",
12083        |accounts| {
12084            let pubkey = solana_sdk::pubkey::new_rand();
12085            let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12086            accounts.store_for_tests(0, &[(&pubkey, &account)]);
12087            accounts.add_root_and_flush_write_cache(0);
12088            let storage_entry = accounts.storage.get_slot_storage_entry(0).unwrap();
12089            storage_entry.remove_accounts(0, true, 1);
12090            storage_entry.remove_accounts(0, true, 1);
12091        }
12092    );
12093
12094    fn do_full_clean_refcount(mut accounts: AccountsDb, store1_first: bool, store_size: u64) {
12095        let pubkey1 = Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
12096        let pubkey2 = Pubkey::from_str("My22211111111111111111111111111111111111111").unwrap();
12097        let pubkey3 = Pubkey::from_str("My33311111111111111111111111111111111111111").unwrap();
12098
12099        let old_lamport = 223;
12100        let zero_lamport = 0;
12101        let dummy_lamport = 999_999;
12102
12103        // size data so only 1 fits in a 4k store
12104        let data_size = 2200;
12105
12106        let owner = *AccountSharedData::default().owner();
12107
12108        let account = AccountSharedData::new(old_lamport, data_size, &owner);
12109        let account2 = AccountSharedData::new(old_lamport + 100_001, data_size, &owner);
12110        let account3 = AccountSharedData::new(old_lamport + 100_002, data_size, &owner);
12111        let account4 = AccountSharedData::new(dummy_lamport, data_size, &owner);
12112        let zero_lamport_account = AccountSharedData::new(zero_lamport, data_size, &owner);
12113
12114        let mut current_slot = 0;
12115        accounts.file_size = store_size;
12116
12117        // A: Initialize AccountsDb with pubkey1 and pubkey2
12118        current_slot += 1;
12119        if store1_first {
12120            accounts.store_for_tests(current_slot, &[(&pubkey1, &account)]);
12121            accounts.store_for_tests(current_slot, &[(&pubkey2, &account)]);
12122        } else {
12123            accounts.store_for_tests(current_slot, &[(&pubkey2, &account)]);
12124            accounts.store_for_tests(current_slot, &[(&pubkey1, &account)]);
12125        }
12126        accounts.calculate_accounts_delta_hash(current_slot);
12127        accounts.add_root_and_flush_write_cache(current_slot);
12128
12129        info!("post A");
12130        accounts.print_accounts_stats("Post-A");
12131
12132        // B: Test multiple updates to pubkey1 in a single slot/storage
12133        current_slot += 1;
12134        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
12135        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
12136        accounts.store_for_tests(current_slot, &[(&pubkey1, &account2)]);
12137        accounts.store_for_tests(current_slot, &[(&pubkey1, &account2)]);
12138        accounts.add_root_and_flush_write_cache(current_slot);
12139        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
12140        // Stores to same pubkey, same slot only count once towards the
12141        // ref count
12142        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12143        accounts.calculate_accounts_delta_hash(current_slot);
12144        accounts.add_root_and_flush_write_cache(current_slot);
12145
12146        accounts.print_accounts_stats("Post-B pre-clean");
12147
12148        accounts.clean_accounts_for_tests();
12149
12150        info!("post B");
12151        accounts.print_accounts_stats("Post-B");
12152
12153        // C: more updates to trigger clean of previous updates
12154        current_slot += 1;
12155        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12156        accounts.store_for_tests(current_slot, &[(&pubkey1, &account3)]);
12157        accounts.store_for_tests(current_slot, &[(&pubkey2, &account3)]);
12158        accounts.store_for_tests(current_slot, &[(&pubkey3, &account4)]);
12159        accounts.add_root_and_flush_write_cache(current_slot);
12160        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12161        accounts.calculate_accounts_delta_hash(current_slot);
12162
12163        info!("post C");
12164
12165        accounts.print_accounts_stats("Post-C");
12166
12167        // D: Make all keys 0-lamport, cleans all keys
12168        current_slot += 1;
12169        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12170        accounts.store_for_tests(current_slot, &[(&pubkey1, &zero_lamport_account)]);
12171        accounts.store_for_tests(current_slot, &[(&pubkey2, &zero_lamport_account)]);
12172        accounts.store_for_tests(current_slot, &[(&pubkey3, &zero_lamport_account)]);
12173
12174        let snapshot_stores = accounts.get_snapshot_storages(..=current_slot).0;
12175        let total_accounts: usize = snapshot_stores.iter().map(|s| s.accounts_count()).sum();
12176        assert!(!snapshot_stores.is_empty());
12177        assert!(total_accounts > 0);
12178
12179        info!("post D");
12180        accounts.print_accounts_stats("Post-D");
12181
12182        accounts.calculate_accounts_delta_hash(current_slot);
12183        accounts.add_root_and_flush_write_cache(current_slot);
12184        accounts.clean_accounts_for_tests();
12185
12186        accounts.print_accounts_stats("Post-D clean");
12187
12188        let total_accounts_post_clean: usize =
12189            snapshot_stores.iter().map(|s| s.accounts_count()).sum();
12190        assert_eq!(total_accounts, total_accounts_post_clean);
12191
12192        // should clean all 3 pubkeys
12193        assert_eq!(accounts.ref_count_for_pubkey(&pubkey1), 0);
12194        assert_eq!(accounts.ref_count_for_pubkey(&pubkey2), 0);
12195        assert_eq!(accounts.ref_count_for_pubkey(&pubkey3), 0);
12196    }
12197
12198    // Setup 3 scenarios which try to differentiate between pubkey1 being in an
12199    // Available slot or a Full slot which would cause a different reset behavior
12200    // when pubkey1 is cleaned and therefore cause the ref count to be incorrect
12201    // preventing a removal of that key.
12202    //
12203    // do stores with a 4mb size so only 1 store is created per slot
12204    define_accounts_db_test!(test_full_clean_refcount_no_first_4m, |accounts| {
12205        do_full_clean_refcount(accounts, false, 4 * 1024 * 1024);
12206    });
12207
12208    // do stores with a 4k size and store pubkey1 first
12209    define_accounts_db_test!(test_full_clean_refcount_no_first_4k, |accounts| {
12210        do_full_clean_refcount(accounts, false, 4 * 1024);
12211    });
12212
12213    // do stores with a 4k size and store pubkey1 2nd
12214    define_accounts_db_test!(test_full_clean_refcount_first_4k, |accounts| {
12215        do_full_clean_refcount(accounts, true, 4 * 1024);
12216    });
12217
12218    #[test]
12219    fn test_clean_stored_dead_slots_empty() {
12220        let accounts = AccountsDb::new_single_for_tests();
12221        let mut dead_slots = IntSet::default();
12222        dead_slots.insert(10);
12223        accounts.clean_stored_dead_slots(&dead_slots, None, &HashSet::default());
12224    }
12225
12226    #[test]
12227    fn test_shrink_all_slots_none() {
12228        let epoch_schedule = EpochSchedule::default();
12229        for startup in &[false, true] {
12230            let accounts = AccountsDb::new_single_for_tests();
12231
12232            for _ in 0..10 {
12233                accounts.shrink_candidate_slots(&epoch_schedule);
12234            }
12235
12236            accounts.shrink_all_slots(*startup, &EpochSchedule::default(), None);
12237        }
12238    }
12239
12240    #[test]
12241    fn test_shrink_candidate_slots() {
12242        solana_logger::setup();
12243
12244        let mut accounts = AccountsDb::new_single_for_tests();
12245
12246        let pubkey_count = 30000;
12247        let pubkeys: Vec<_> = (0..pubkey_count)
12248            .map(|_| solana_sdk::pubkey::new_rand())
12249            .collect();
12250
12251        let some_lamport = 223;
12252        let no_data = 0;
12253        let owner = *AccountSharedData::default().owner();
12254
12255        let account = AccountSharedData::new(some_lamport, no_data, &owner);
12256
12257        let mut current_slot = 0;
12258
12259        current_slot += 1;
12260        for pubkey in &pubkeys {
12261            accounts.store_for_tests(current_slot, &[(pubkey, &account)]);
12262        }
12263        let shrink_slot = current_slot;
12264        accounts.calculate_accounts_delta_hash(current_slot);
12265        accounts.add_root_and_flush_write_cache(current_slot);
12266
12267        current_slot += 1;
12268        let pubkey_count_after_shrink = 25000;
12269        let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
12270
12271        for pubkey in updated_pubkeys {
12272            accounts.store_for_tests(current_slot, &[(pubkey, &account)]);
12273        }
12274        accounts.calculate_accounts_delta_hash(current_slot);
12275        accounts.add_root_and_flush_write_cache(current_slot);
12276        accounts.clean_accounts_for_tests();
12277
12278        assert_eq!(
12279            pubkey_count,
12280            accounts.all_account_count_in_accounts_file(shrink_slot)
12281        );
12282
12283        // Only, try to shrink stale slots, nothing happens because shrink ratio
12284        // is not small enough to do a shrink
12285        // Note this shrink ratio had to change because we are WAY over-allocating append vecs when we flush the write cache at the moment.
12286        accounts.shrink_ratio = AccountShrinkThreshold::TotalSpace { shrink_ratio: 0.4 };
12287        accounts.shrink_candidate_slots(&EpochSchedule::default());
12288        assert_eq!(
12289            pubkey_count,
12290            accounts.all_account_count_in_accounts_file(shrink_slot)
12291        );
12292
12293        // Now, do full-shrink.
12294        accounts.shrink_all_slots(false, &EpochSchedule::default(), None);
12295        assert_eq!(
12296            pubkey_count_after_shrink,
12297            accounts.all_account_count_in_accounts_file(shrink_slot)
12298        );
12299    }
12300
12301    /// This test creates an ancient storage with three alive accounts
12302    /// of various sizes. It then simulates killing one of the
12303    /// accounts in a more recent (non-ancient) slot by overwriting
12304    /// the account that has the smallest data size.  The dead account
12305    /// is expected to be deleted from its ancient storage in the
12306    /// process of shrinking candidate slots.  The capacity of the
12307    /// storage after shrinking is expected to be the sum of alive
12308    /// bytes of the two remaining alive ancient accounts.
12309    #[test]
12310    fn test_shrink_candidate_slots_with_dead_ancient_account() {
12311        solana_logger::setup();
12312        let epoch_schedule = EpochSchedule::default();
12313        let num_ancient_slots = 3;
12314        // Prepare 3 append vecs to combine [medium, big, small]
12315        let account_data_sizes = vec![1000, 2000, 150];
12316        let (db, starting_ancient_slot) =
12317            create_db_with_storages_and_index_with_customized_account_size_per_slot(
12318                true,
12319                num_ancient_slots,
12320                account_data_sizes,
12321            );
12322        db.add_root(starting_ancient_slot);
12323        let slots_to_combine: Vec<Slot> =
12324            (starting_ancient_slot..starting_ancient_slot + num_ancient_slots as Slot).collect();
12325        db.combine_ancient_slots(slots_to_combine, CAN_RANDOMLY_SHRINK_FALSE);
12326        let storage = db.get_storage_for_slot(starting_ancient_slot).unwrap();
12327        let ancient_accounts = db.get_unique_accounts_from_storage(&storage);
12328        // Check that three accounts are indeed present in the combined storage.
12329        assert_eq!(ancient_accounts.stored_accounts.len(), 3);
12330        // Find an ancient account with smallest data length.
12331        // This will be a dead account, overwritten in the current slot.
12332        let modified_account_pubkey = ancient_accounts
12333            .stored_accounts
12334            .iter()
12335            .min_by(|a, b| a.data_len.cmp(&b.data_len))
12336            .unwrap()
12337            .pubkey;
12338        let modified_account_owner = *AccountSharedData::default().owner();
12339        let modified_account = AccountSharedData::new(223, 0, &modified_account_owner);
12340        let ancient_append_vec_offset = db.ancient_append_vec_offset.unwrap().abs();
12341        let current_slot = epoch_schedule.slots_per_epoch + ancient_append_vec_offset as u64 + 1;
12342        // Simulate killing of the ancient account by overwriting it in the current slot.
12343        db.store_for_tests(
12344            current_slot,
12345            &[(&modified_account_pubkey, &modified_account)],
12346        );
12347        db.calculate_accounts_delta_hash(current_slot);
12348        db.add_root_and_flush_write_cache(current_slot);
12349        // This should remove the dead ancient account from the index.
12350        db.clean_accounts_for_tests();
12351        db.shrink_ancient_slots(&epoch_schedule);
12352        let storage = db.get_storage_for_slot(starting_ancient_slot).unwrap();
12353        let created_accounts = db.get_unique_accounts_from_storage(&storage);
12354        // The dead account should still be in the ancient storage,
12355        // because the storage wouldn't be shrunk with normal alive to
12356        // capacity ratio.
12357        assert_eq!(created_accounts.stored_accounts.len(), 3);
12358        db.shrink_candidate_slots(&epoch_schedule);
12359        let storage = db.get_storage_for_slot(starting_ancient_slot).unwrap();
12360        let created_accounts = db.get_unique_accounts_from_storage(&storage);
12361        // At this point the dead ancient account should be removed
12362        // and storage capacity shrunk to the sum of alive bytes of
12363        // accounts it holds.  This is the data lengths of the
12364        // accounts plus the length of their metadata.
12365        assert_eq!(
12366            created_accounts.capacity as usize,
12367            aligned_stored_size(1000) + aligned_stored_size(2000)
12368        );
12369        // The above check works only when the AppendVec storage is
12370        // used. More generally the pubkey of the smallest account
12371        // shouldn't be present in the shrunk storage, which is
12372        // validated by the following scan of the storage accounts.
12373        storage.accounts.scan_pubkeys(|pubkey| {
12374            assert_ne!(pubkey, &modified_account_pubkey);
12375        });
12376    }
12377
12378    #[test]
12379    fn test_select_candidates_by_total_usage_no_candidates() {
12380        // no input candidates -- none should be selected
12381        solana_logger::setup();
12382        let candidates = ShrinkCandidates::default();
12383        let db = AccountsDb::new_single_for_tests();
12384
12385        let (selected_candidates, next_candidates) =
12386            db.select_candidates_by_total_usage(&candidates, DEFAULT_ACCOUNTS_SHRINK_RATIO);
12387
12388        assert_eq!(0, selected_candidates.len());
12389        assert_eq!(0, next_candidates.len());
12390    }
12391
12392    #[test]
12393    fn test_select_candidates_by_total_usage_3_way_split_condition() {
12394        // three candidates, one selected for shrink, one is put back to the candidate list and one is ignored
12395        solana_logger::setup();
12396        let mut candidates = ShrinkCandidates::default();
12397        let db = AccountsDb::new_single_for_tests();
12398
12399        let common_store_path = Path::new("");
12400        let store_file_size = 100;
12401
12402        let store1_slot = 11;
12403        let store1 = Arc::new(AccountStorageEntry::new(
12404            common_store_path,
12405            store1_slot,
12406            store1_slot as AccountsFileId,
12407            store_file_size,
12408            AccountsFileProvider::AppendVec,
12409        ));
12410        db.storage.insert(store1_slot, Arc::clone(&store1));
12411        store1.alive_bytes.store(0, Ordering::Release);
12412        candidates.insert(store1_slot);
12413
12414        let store2_slot = 22;
12415        let store2 = Arc::new(AccountStorageEntry::new(
12416            common_store_path,
12417            store2_slot,
12418            store2_slot as AccountsFileId,
12419            store_file_size,
12420            AccountsFileProvider::AppendVec,
12421        ));
12422        db.storage.insert(store2_slot, Arc::clone(&store2));
12423        store2
12424            .alive_bytes
12425            .store(store_file_size as usize / 2, Ordering::Release);
12426        candidates.insert(store2_slot);
12427
12428        let store3_slot = 33;
12429        let store3 = Arc::new(AccountStorageEntry::new(
12430            common_store_path,
12431            store3_slot,
12432            store3_slot as AccountsFileId,
12433            store_file_size,
12434            AccountsFileProvider::AppendVec,
12435        ));
12436        db.storage.insert(store3_slot, Arc::clone(&store3));
12437        store3
12438            .alive_bytes
12439            .store(store_file_size as usize, Ordering::Release);
12440        candidates.insert(store3_slot);
12441
12442        // Set the target alive ratio to 0.6 so that we can just get rid of store1, the remaining two stores
12443        // alive ratio can be > the target ratio: the actual ratio is 0.75 because of 150 alive bytes / 200 total bytes.
12444        // The target ratio is also set to larger than store2's alive ratio: 0.5 so that it would be added
12445        // to the candidates list for next round.
12446        let target_alive_ratio = 0.6;
12447        let (selected_candidates, next_candidates) =
12448            db.select_candidates_by_total_usage(&candidates, target_alive_ratio);
12449        assert_eq!(1, selected_candidates.len());
12450        assert!(selected_candidates.contains(&store1_slot));
12451        assert_eq!(1, next_candidates.len());
12452        assert!(next_candidates.contains(&store2_slot));
12453    }
12454
12455    #[test]
12456    fn test_select_candidates_by_total_usage_2_way_split_condition() {
12457        // three candidates, 2 are selected for shrink, one is ignored
12458        solana_logger::setup();
12459        let db = AccountsDb::new_single_for_tests();
12460        let mut candidates = ShrinkCandidates::default();
12461
12462        let common_store_path = Path::new("");
12463        let store_file_size = 100;
12464
12465        let store1_slot = 11;
12466        let store1 = Arc::new(AccountStorageEntry::new(
12467            common_store_path,
12468            store1_slot,
12469            store1_slot as AccountsFileId,
12470            store_file_size,
12471            AccountsFileProvider::AppendVec,
12472        ));
12473        db.storage.insert(store1_slot, Arc::clone(&store1));
12474        store1.alive_bytes.store(0, Ordering::Release);
12475        candidates.insert(store1_slot);
12476
12477        let store2_slot = 22;
12478        let store2 = Arc::new(AccountStorageEntry::new(
12479            common_store_path,
12480            store2_slot,
12481            store2_slot as AccountsFileId,
12482            store_file_size,
12483            AccountsFileProvider::AppendVec,
12484        ));
12485        db.storage.insert(store2_slot, Arc::clone(&store2));
12486        store2
12487            .alive_bytes
12488            .store(store_file_size as usize / 2, Ordering::Release);
12489        candidates.insert(store2_slot);
12490
12491        let store3_slot = 33;
12492        let store3 = Arc::new(AccountStorageEntry::new(
12493            common_store_path,
12494            store3_slot,
12495            store3_slot as AccountsFileId,
12496            store_file_size,
12497            AccountsFileProvider::AppendVec,
12498        ));
12499        db.storage.insert(store3_slot, Arc::clone(&store3));
12500        store3
12501            .alive_bytes
12502            .store(store_file_size as usize, Ordering::Release);
12503        candidates.insert(store3_slot);
12504
12505        // Set the target ratio to default (0.8), both store1 and store2 must be selected and store3 is ignored.
12506        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
12507        let (selected_candidates, next_candidates) =
12508            db.select_candidates_by_total_usage(&candidates, target_alive_ratio);
12509        assert_eq!(2, selected_candidates.len());
12510        assert!(selected_candidates.contains(&store1_slot));
12511        assert!(selected_candidates.contains(&store2_slot));
12512        assert_eq!(0, next_candidates.len());
12513    }
12514
12515    #[test]
12516    fn test_select_candidates_by_total_usage_all_clean() {
12517        // 2 candidates, they must be selected to achieve the target alive ratio
12518        solana_logger::setup();
12519        let db = AccountsDb::new_single_for_tests();
12520        let mut candidates = ShrinkCandidates::default();
12521
12522        let common_store_path = Path::new("");
12523        let store_file_size = 100;
12524
12525        let store1_slot = 11;
12526        let store1 = Arc::new(AccountStorageEntry::new(
12527            common_store_path,
12528            store1_slot,
12529            store1_slot as AccountsFileId,
12530            store_file_size,
12531            AccountsFileProvider::AppendVec,
12532        ));
12533        db.storage.insert(store1_slot, Arc::clone(&store1));
12534        store1
12535            .alive_bytes
12536            .store(store_file_size as usize / 4, Ordering::Release);
12537        candidates.insert(store1_slot);
12538
12539        let store2_slot = 22;
12540        let store2 = Arc::new(AccountStorageEntry::new(
12541            common_store_path,
12542            store2_slot,
12543            store2_slot as AccountsFileId,
12544            store_file_size,
12545            AccountsFileProvider::AppendVec,
12546        ));
12547        db.storage.insert(store2_slot, Arc::clone(&store2));
12548        store2
12549            .alive_bytes
12550            .store(store_file_size as usize / 2, Ordering::Release);
12551        candidates.insert(store2_slot);
12552
12553        // Set the target ratio to default (0.8), both stores from the two different slots must be selected.
12554        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
12555        let (selected_candidates, next_candidates) =
12556            db.select_candidates_by_total_usage(&candidates, target_alive_ratio);
12557        assert_eq!(2, selected_candidates.len());
12558        assert!(selected_candidates.contains(&store1_slot));
12559        assert!(selected_candidates.contains(&store2_slot));
12560        assert_eq!(0, next_candidates.len());
12561    }
12562
12563    const UPSERT_POPULATE_RECLAIMS: UpsertReclaim = UpsertReclaim::PopulateReclaims;
12564
12565    #[test]
12566    fn test_delete_dependencies() {
12567        solana_logger::setup();
12568        let accounts_index = AccountsIndex::<AccountInfo, AccountInfo>::default_for_tests();
12569        let key0 = Pubkey::new_from_array([0u8; 32]);
12570        let key1 = Pubkey::new_from_array([1u8; 32]);
12571        let key2 = Pubkey::new_from_array([2u8; 32]);
12572        let info0 = AccountInfo::new(StorageLocation::AppendVec(0, 0), 0);
12573        let info1 = AccountInfo::new(StorageLocation::AppendVec(1, 0), 0);
12574        let info2 = AccountInfo::new(StorageLocation::AppendVec(2, 0), 0);
12575        let info3 = AccountInfo::new(StorageLocation::AppendVec(3, 0), 0);
12576        let mut reclaims = vec![];
12577        accounts_index.upsert(
12578            0,
12579            0,
12580            &key0,
12581            &AccountSharedData::default(),
12582            &AccountSecondaryIndexes::default(),
12583            info0,
12584            &mut reclaims,
12585            UPSERT_POPULATE_RECLAIMS,
12586        );
12587        accounts_index.upsert(
12588            1,
12589            1,
12590            &key0,
12591            &AccountSharedData::default(),
12592            &AccountSecondaryIndexes::default(),
12593            info1,
12594            &mut reclaims,
12595            UPSERT_POPULATE_RECLAIMS,
12596        );
12597        accounts_index.upsert(
12598            1,
12599            1,
12600            &key1,
12601            &AccountSharedData::default(),
12602            &AccountSecondaryIndexes::default(),
12603            info1,
12604            &mut reclaims,
12605            UPSERT_POPULATE_RECLAIMS,
12606        );
12607        accounts_index.upsert(
12608            2,
12609            2,
12610            &key1,
12611            &AccountSharedData::default(),
12612            &AccountSecondaryIndexes::default(),
12613            info2,
12614            &mut reclaims,
12615            UPSERT_POPULATE_RECLAIMS,
12616        );
12617        accounts_index.upsert(
12618            2,
12619            2,
12620            &key2,
12621            &AccountSharedData::default(),
12622            &AccountSecondaryIndexes::default(),
12623            info2,
12624            &mut reclaims,
12625            UPSERT_POPULATE_RECLAIMS,
12626        );
12627        accounts_index.upsert(
12628            3,
12629            3,
12630            &key2,
12631            &AccountSharedData::default(),
12632            &AccountSecondaryIndexes::default(),
12633            info3,
12634            &mut reclaims,
12635            UPSERT_POPULATE_RECLAIMS,
12636        );
12637        accounts_index.add_root(0);
12638        accounts_index.add_root(1);
12639        accounts_index.add_root(2);
12640        accounts_index.add_root(3);
12641        let num_bins = accounts_index.bins();
12642        let candidates: Box<_> =
12643            std::iter::repeat_with(|| RwLock::new(HashMap::<Pubkey, CleaningInfo>::new()))
12644                .take(num_bins)
12645                .collect();
12646        for key in [&key0, &key1, &key2] {
12647            let index_entry = accounts_index.get_cloned(key).unwrap();
12648            let rooted_entries = accounts_index
12649                .get_rooted_entries(index_entry.slot_list.read().unwrap().as_slice(), None);
12650            let ref_count = index_entry.ref_count();
12651            let index = accounts_index.bin_calculator.bin_from_pubkey(key);
12652            let mut candidates_bin = candidates[index].write().unwrap();
12653            candidates_bin.insert(
12654                *key,
12655                CleaningInfo {
12656                    slot_list: rooted_entries,
12657                    ref_count,
12658                    ..Default::default()
12659                },
12660            );
12661        }
12662        for candidates_bin in candidates.iter() {
12663            let candidates_bin = candidates_bin.read().unwrap();
12664            for (
12665                key,
12666                CleaningInfo {
12667                    slot_list: list,
12668                    ref_count,
12669                    ..
12670                },
12671            ) in candidates_bin.iter()
12672            {
12673                info!(" purge {} ref_count {} =>", key, ref_count);
12674                for x in list {
12675                    info!("  {:?}", x);
12676                }
12677            }
12678        }
12679
12680        let mut store_counts = HashMap::new();
12681        store_counts.insert(0, (0, HashSet::from_iter(vec![key0])));
12682        store_counts.insert(1, (0, HashSet::from_iter(vec![key0, key1])));
12683        store_counts.insert(2, (0, HashSet::from_iter(vec![key1, key2])));
12684        store_counts.insert(3, (1, HashSet::from_iter(vec![key2])));
12685        let accounts = AccountsDb::new_single_for_tests();
12686        accounts.calc_delete_dependencies(&candidates, &mut store_counts, None);
12687        let mut stores: Vec<_> = store_counts.keys().cloned().collect();
12688        stores.sort_unstable();
12689        for store in &stores {
12690            info!(
12691                "store: {:?} : {:?}",
12692                store,
12693                store_counts.get(store).unwrap()
12694            );
12695        }
12696        for x in 0..3 {
12697            // if the store count doesn't exist for this id, then it is implied to be > 0
12698            assert!(store_counts
12699                .get(&x)
12700                .map(|entry| entry.0 >= 1)
12701                .unwrap_or(true));
12702        }
12703    }
12704
12705    #[test]
12706    fn test_account_balance_for_capitalization_sysvar() {
12707        let normal_sysvar = solana_sdk::account::create_account_for_test(
12708            &solana_sdk::slot_history::SlotHistory::default(),
12709        );
12710        assert_eq!(normal_sysvar.lamports(), 1);
12711    }
12712
12713    #[test]
12714    fn test_account_balance_for_capitalization_native_program() {
12715        let normal_native_program =
12716            solana_sdk::native_loader::create_loadable_account_for_test("foo");
12717        assert_eq!(normal_native_program.lamports(), 1);
12718    }
12719
12720    #[test]
12721    fn test_checked_sum_for_capitalization_normal() {
12722        assert_eq!(
12723            AccountsDb::checked_sum_for_capitalization(vec![1, 2].into_iter()),
12724            3
12725        );
12726    }
12727
12728    #[test]
12729    #[should_panic(expected = "overflow is detected while summing capitalization")]
12730    fn test_checked_sum_for_capitalization_overflow() {
12731        assert_eq!(
12732            AccountsDb::checked_sum_for_capitalization(vec![1, u64::MAX].into_iter()),
12733            3
12734        );
12735    }
12736
12737    #[test]
12738    fn test_store_overhead() {
12739        solana_logger::setup();
12740        let accounts = AccountsDb::new_single_for_tests();
12741        let account = AccountSharedData::default();
12742        let pubkey = solana_sdk::pubkey::new_rand();
12743        accounts.store_for_tests(0, &[(&pubkey, &account)]);
12744        accounts.add_root_and_flush_write_cache(0);
12745        let store = accounts.storage.get_slot_storage_entry(0).unwrap();
12746        let total_len = store.accounts.len();
12747        info!("total: {}", total_len);
12748        assert_eq!(total_len, STORE_META_OVERHEAD);
12749    }
12750
12751    #[test]
12752    fn test_store_clean_after_shrink() {
12753        solana_logger::setup();
12754        let accounts = AccountsDb::new_single_for_tests();
12755        let epoch_schedule = EpochSchedule::default();
12756
12757        let account = AccountSharedData::new(1, 16 * 4096, &Pubkey::default());
12758        let pubkey1 = solana_sdk::pubkey::new_rand();
12759        accounts.store_cached((0, &[(&pubkey1, &account)][..]), None);
12760
12761        let pubkey2 = solana_sdk::pubkey::new_rand();
12762        accounts.store_cached((0, &[(&pubkey2, &account)][..]), None);
12763
12764        let zero_account = AccountSharedData::new(0, 1, &Pubkey::default());
12765        accounts.store_cached((1, &[(&pubkey1, &zero_account)][..]), None);
12766
12767        // Add root 0 and flush separately
12768        accounts.calculate_accounts_delta_hash(0);
12769        accounts.add_root(0);
12770        accounts.flush_accounts_cache(true, None);
12771
12772        // clear out the dirty keys
12773        accounts.clean_accounts_for_tests();
12774
12775        // flush 1
12776        accounts.calculate_accounts_delta_hash(1);
12777        accounts.add_root(1);
12778        accounts.flush_accounts_cache(true, None);
12779
12780        accounts.print_accounts_stats("pre-clean");
12781
12782        // clean to remove pubkey1 from 0,
12783        // shrink to shrink pubkey1 from 0
12784        // then another clean to remove pubkey1 from slot 1
12785        accounts.clean_accounts_for_tests();
12786
12787        accounts.shrink_candidate_slots(&epoch_schedule);
12788
12789        accounts.clean_accounts_for_tests();
12790
12791        accounts.print_accounts_stats("post-clean");
12792        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
12793    }
12794
12795    #[test]
12796    #[should_panic(expected = "We've run out of storage ids!")]
12797    fn test_wrapping_storage_id() {
12798        let db = AccountsDb::new_single_for_tests();
12799
12800        let zero_lamport_account =
12801            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
12802
12803        // set 'next' id to the max possible value
12804        db.next_id.store(AccountsFileId::MAX, Ordering::Release);
12805        let slots = 3;
12806        let keys = (0..slots).map(|_| Pubkey::new_unique()).collect::<Vec<_>>();
12807        // write unique keys to successive slots
12808        keys.iter().enumerate().for_each(|(slot, key)| {
12809            let slot = slot as Slot;
12810            db.store_for_tests(slot, &[(key, &zero_lamport_account)]);
12811            db.calculate_accounts_delta_hash(slot);
12812            db.add_root_and_flush_write_cache(slot);
12813        });
12814        assert_eq!(slots - 1, db.next_id.load(Ordering::Acquire));
12815        let ancestors = Ancestors::default();
12816        keys.iter().for_each(|key| {
12817            assert!(db.load_without_fixed_root(&ancestors, key).is_some());
12818        });
12819    }
12820
12821    #[test]
12822    #[should_panic(expected = "We've run out of storage ids!")]
12823    fn test_reuse_storage_id() {
12824        solana_logger::setup();
12825        let db = AccountsDb::new_single_for_tests();
12826
12827        let zero_lamport_account =
12828            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
12829
12830        // set 'next' id to the max possible value
12831        db.next_id.store(AccountsFileId::MAX, Ordering::Release);
12832        let slots = 3;
12833        let keys = (0..slots).map(|_| Pubkey::new_unique()).collect::<Vec<_>>();
12834        // write unique keys to successive slots
12835        keys.iter().enumerate().for_each(|(slot, key)| {
12836            let slot = slot as Slot;
12837            db.store_for_tests(slot, &[(key, &zero_lamport_account)]);
12838            db.calculate_accounts_delta_hash(slot);
12839            db.add_root_and_flush_write_cache(slot);
12840            // reset next_id to what it was previously to cause us to re-use the same id
12841            db.next_id.store(AccountsFileId::MAX, Ordering::Release);
12842        });
12843        let ancestors = Ancestors::default();
12844        keys.iter().for_each(|key| {
12845            assert!(db.load_without_fixed_root(&ancestors, key).is_some());
12846        });
12847    }
12848
12849    #[test]
12850    fn test_zero_lamport_new_root_not_cleaned() {
12851        let db = AccountsDb::new_single_for_tests();
12852        let account_key = Pubkey::new_unique();
12853        let zero_lamport_account =
12854            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
12855
12856        // Store zero lamport account into slots 0 and 1, root both slots
12857        db.store_for_tests(0, &[(&account_key, &zero_lamport_account)]);
12858        db.store_for_tests(1, &[(&account_key, &zero_lamport_account)]);
12859        db.calculate_accounts_delta_hash(0);
12860        db.add_root_and_flush_write_cache(0);
12861        db.calculate_accounts_delta_hash(1);
12862        db.add_root_and_flush_write_cache(1);
12863
12864        // Only clean zero lamport accounts up to slot 0
12865        db.clean_accounts(
12866            Some(0),
12867            false,
12868            &EpochSchedule::default(),
12869            OldStoragesPolicy::Leave,
12870        );
12871
12872        // Should still be able to find zero lamport account in slot 1
12873        assert_eq!(
12874            db.load_without_fixed_root(&Ancestors::default(), &account_key),
12875            Some((zero_lamport_account, 1))
12876        );
12877    }
12878
12879    #[test]
12880    fn test_store_load_cached() {
12881        let db = AccountsDb::new_single_for_tests();
12882        let key = Pubkey::default();
12883        let account0 = AccountSharedData::new(1, 0, &key);
12884        let slot = 0;
12885        db.store_cached((slot, &[(&key, &account0)][..]), None);
12886
12887        // Load with no ancestors and no root will return nothing
12888        assert!(db
12889            .load_without_fixed_root(&Ancestors::default(), &key)
12890            .is_none());
12891
12892        // Load with ancestors not equal to `slot` will return nothing
12893        let ancestors = vec![(slot + 1, 1)].into_iter().collect();
12894        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
12895
12896        // Load with ancestors equal to `slot` will return the account
12897        let ancestors = vec![(slot, 1)].into_iter().collect();
12898        assert_eq!(
12899            db.load_without_fixed_root(&ancestors, &key),
12900            Some((account0.clone(), slot))
12901        );
12902
12903        // Adding root will return the account even without ancestors
12904        db.add_root(slot);
12905        assert_eq!(
12906            db.load_without_fixed_root(&Ancestors::default(), &key),
12907            Some((account0, slot))
12908        );
12909    }
12910
12911    #[test]
12912    fn test_store_flush_load_cached() {
12913        let db = AccountsDb::new_single_for_tests();
12914        let key = Pubkey::default();
12915        let account0 = AccountSharedData::new(1, 0, &key);
12916        let slot = 0;
12917        db.store_cached((slot, &[(&key, &account0)][..]), None);
12918        db.mark_slot_frozen(slot);
12919
12920        // No root was added yet, requires an ancestor to find
12921        // the account
12922        db.flush_accounts_cache(true, None);
12923        let ancestors = vec![(slot, 1)].into_iter().collect();
12924        assert_eq!(
12925            db.load_without_fixed_root(&ancestors, &key),
12926            Some((account0.clone(), slot))
12927        );
12928
12929        // Add root then flush
12930        db.add_root(slot);
12931        db.flush_accounts_cache(true, None);
12932        assert_eq!(
12933            db.load_without_fixed_root(&Ancestors::default(), &key),
12934            Some((account0, slot))
12935        );
12936    }
12937
12938    #[test]
12939    fn test_flush_accounts_cache() {
12940        let db = AccountsDb::new_single_for_tests();
12941        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
12942
12943        let unrooted_slot = 4;
12944        let root5 = 5;
12945        let root6 = 6;
12946        let unrooted_key = solana_sdk::pubkey::new_rand();
12947        let key5 = solana_sdk::pubkey::new_rand();
12948        let key6 = solana_sdk::pubkey::new_rand();
12949        db.store_cached((unrooted_slot, &[(&unrooted_key, &account0)][..]), None);
12950        db.store_cached((root5, &[(&key5, &account0)][..]), None);
12951        db.store_cached((root6, &[(&key6, &account0)][..]), None);
12952        for slot in &[unrooted_slot, root5, root6] {
12953            db.mark_slot_frozen(*slot);
12954        }
12955        db.add_root(root5);
12956        db.add_root(root6);
12957
12958        // Unrooted slot should be able to be fetched before the flush
12959        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
12960        assert_eq!(
12961            db.load_without_fixed_root(&ancestors, &unrooted_key),
12962            Some((account0.clone(), unrooted_slot))
12963        );
12964        db.flush_accounts_cache(true, None);
12965
12966        // After the flush, the unrooted slot is still in the cache
12967        assert!(db
12968            .load_without_fixed_root(&ancestors, &unrooted_key)
12969            .is_some());
12970        assert!(db.accounts_index.contains(&unrooted_key));
12971        assert_eq!(db.accounts_cache.num_slots(), 1);
12972        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_some());
12973        assert_eq!(
12974            db.load_without_fixed_root(&Ancestors::default(), &key5),
12975            Some((account0.clone(), root5))
12976        );
12977        assert_eq!(
12978            db.load_without_fixed_root(&Ancestors::default(), &key6),
12979            Some((account0, root6))
12980        );
12981    }
12982
12983    fn max_cache_slots() -> usize {
12984        // this used to be the limiting factor - used here to facilitate tests.
12985        200
12986    }
12987
12988    #[test]
12989    fn test_flush_accounts_cache_if_needed() {
12990        run_test_flush_accounts_cache_if_needed(0, 2 * max_cache_slots());
12991        run_test_flush_accounts_cache_if_needed(2 * max_cache_slots(), 0);
12992        run_test_flush_accounts_cache_if_needed(max_cache_slots() - 1, 0);
12993        run_test_flush_accounts_cache_if_needed(0, max_cache_slots() - 1);
12994        run_test_flush_accounts_cache_if_needed(max_cache_slots(), 0);
12995        run_test_flush_accounts_cache_if_needed(0, max_cache_slots());
12996        run_test_flush_accounts_cache_if_needed(2 * max_cache_slots(), 2 * max_cache_slots());
12997        run_test_flush_accounts_cache_if_needed(max_cache_slots() - 1, max_cache_slots() - 1);
12998        run_test_flush_accounts_cache_if_needed(max_cache_slots(), max_cache_slots());
12999    }
13000
13001    fn run_test_flush_accounts_cache_if_needed(num_roots: usize, num_unrooted: usize) {
13002        let mut db = AccountsDb::new_single_for_tests();
13003        db.write_cache_limit_bytes = Some(max_cache_slots() as u64);
13004        let space = 1; // # data bytes per account. write cache counts data len
13005        let account0 = AccountSharedData::new(1, space, &Pubkey::default());
13006        let mut keys = vec![];
13007        let num_slots = 2 * max_cache_slots();
13008        for i in 0..num_roots + num_unrooted {
13009            let key = Pubkey::new_unique();
13010            db.store_cached((i as Slot, &[(&key, &account0)][..]), None);
13011            keys.push(key);
13012            db.mark_slot_frozen(i as Slot);
13013            if i < num_roots {
13014                db.add_root(i as Slot);
13015            }
13016        }
13017
13018        db.flush_accounts_cache(false, None);
13019
13020        let total_slots = num_roots + num_unrooted;
13021        // If there's <= the max size, then nothing will be flushed from the slot
13022        if total_slots <= max_cache_slots() {
13023            assert_eq!(db.accounts_cache.num_slots(), total_slots);
13024        } else {
13025            // Otherwise, all the roots are flushed, and only at most max_cache_slots()
13026            // of the unrooted slots are kept in the cache
13027            let expected_size = std::cmp::min(num_unrooted, max_cache_slots());
13028            if expected_size > 0 {
13029                // +1: slot is 1-based. slot 1 has 1 byte of data
13030                for unrooted_slot in (total_slots - expected_size + 1)..total_slots {
13031                    assert!(
13032                        db.accounts_cache
13033                            .slot_cache(unrooted_slot as Slot)
13034                            .is_some(),
13035                        "unrooted_slot: {unrooted_slot}, total_slots: {total_slots}, \
13036                         expected_size: {expected_size}"
13037                    );
13038                }
13039            }
13040        }
13041
13042        // Should still be able to fetch all the accounts after flush
13043        for (slot, key) in (0..num_slots as Slot).zip(keys) {
13044            let ancestors = if slot < num_roots as Slot {
13045                Ancestors::default()
13046            } else {
13047                vec![(slot, 1)].into_iter().collect()
13048            };
13049            assert_eq!(
13050                db.load_without_fixed_root(&ancestors, &key),
13051                Some((account0.clone(), slot))
13052            );
13053        }
13054    }
13055
13056    #[test]
13057    fn test_read_only_accounts_cache() {
13058        let db = Arc::new(AccountsDb::new_single_for_tests());
13059
13060        let account_key = Pubkey::new_unique();
13061        let zero_lamport_account =
13062            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13063        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13064        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13065        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13066
13067        db.add_root(0);
13068        db.add_root(1);
13069        db.clean_accounts_for_tests();
13070        db.flush_accounts_cache(true, None);
13071        db.clean_accounts_for_tests();
13072        db.add_root(2);
13073
13074        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13075        let account = db
13076            .load_with_fixed_root(&Ancestors::default(), &account_key)
13077            .map(|(account, _)| account)
13078            .unwrap();
13079        assert_eq!(account.lamports(), 1);
13080        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13081        let account = db
13082            .load_with_fixed_root(&Ancestors::default(), &account_key)
13083            .map(|(account, _)| account)
13084            .unwrap();
13085        assert_eq!(account.lamports(), 1);
13086        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13087        db.store_cached((2, &[(&account_key, &zero_lamport_account)][..]), None);
13088        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13089        let account = db
13090            .load_with_fixed_root(&Ancestors::default(), &account_key)
13091            .map(|(account, _)| account);
13092        assert!(account.is_none());
13093        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13094    }
13095
13096    #[test]
13097    fn test_load_with_read_only_accounts_cache() {
13098        let db = Arc::new(AccountsDb::new_single_for_tests());
13099
13100        let account_key = Pubkey::new_unique();
13101        let zero_lamport_account =
13102            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13103        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13104        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13105        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13106
13107        db.add_root(0);
13108        db.add_root(1);
13109        db.clean_accounts_for_tests();
13110        db.flush_accounts_cache(true, None);
13111        db.clean_accounts_for_tests();
13112        db.add_root(2);
13113
13114        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13115        let (account, slot) = db
13116            .load_account_with(&Ancestors::default(), &account_key, |_| false)
13117            .unwrap();
13118        assert_eq!(account.lamports(), 1);
13119        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13120        assert_eq!(slot, 1);
13121
13122        let (account, slot) = db
13123            .load_account_with(&Ancestors::default(), &account_key, |_| true)
13124            .unwrap();
13125        assert_eq!(account.lamports(), 1);
13126        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13127        assert_eq!(slot, 1);
13128
13129        db.store_cached((2, &[(&account_key, &zero_lamport_account)][..]), None);
13130        let account = db.load_account_with(&Ancestors::default(), &account_key, |_| false);
13131        assert!(account.is_none());
13132        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13133
13134        db.read_only_accounts_cache.reset_for_tests();
13135        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13136        let account = db.load_account_with(&Ancestors::default(), &account_key, |_| true);
13137        assert!(account.is_none());
13138        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13139
13140        let slot2_account = AccountSharedData::new(2, 1, AccountSharedData::default().owner());
13141        db.store_cached((2, &[(&account_key, &slot2_account)][..]), None);
13142        let (account, slot) = db
13143            .load_account_with(&Ancestors::default(), &account_key, |_| false)
13144            .unwrap();
13145        assert_eq!(account.lamports(), 2);
13146        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13147        assert_eq!(slot, 2);
13148
13149        let slot2_account = AccountSharedData::new(2, 1, AccountSharedData::default().owner());
13150        db.store_cached((2, &[(&account_key, &slot2_account)][..]), None);
13151        let (account, slot) = db
13152            .load_account_with(&Ancestors::default(), &account_key, |_| true)
13153            .unwrap();
13154        assert_eq!(account.lamports(), 2);
13155        // The account shouldn't be added to read_only_cache because it is in write_cache.
13156        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13157        assert_eq!(slot, 2);
13158    }
13159
13160    #[test]
13161    fn test_account_matches_owners() {
13162        let db = Arc::new(AccountsDb::new_single_for_tests());
13163
13164        let owners: Vec<Pubkey> = (0..2).map(|_| Pubkey::new_unique()).collect();
13165
13166        let account1_key = Pubkey::new_unique();
13167        let account1 = AccountSharedData::new(321, 10, &owners[0]);
13168
13169        let account2_key = Pubkey::new_unique();
13170        let account2 = AccountSharedData::new(1, 1, &owners[1]);
13171
13172        let account3_key = Pubkey::new_unique();
13173        let account3 = AccountSharedData::new(1, 1, &Pubkey::new_unique());
13174
13175        // Account with 0 lamports
13176        let account4_key = Pubkey::new_unique();
13177        let account4 = AccountSharedData::new(0, 1, &owners[1]);
13178
13179        db.store_cached((0, &[(&account1_key, &account1)][..]), None);
13180        db.store_cached((1, &[(&account2_key, &account2)][..]), None);
13181        db.store_cached((2, &[(&account3_key, &account3)][..]), None);
13182        db.store_cached((3, &[(&account4_key, &account4)][..]), None);
13183
13184        db.add_root(0);
13185        db.add_root(1);
13186        db.add_root(2);
13187        db.add_root(3);
13188
13189        // Flush the cache so that the account meta will be read from the storage
13190        db.flush_accounts_cache(true, None);
13191        db.clean_accounts_for_tests();
13192
13193        assert_eq!(
13194            db.account_matches_owners(&Ancestors::default(), &account1_key, &owners),
13195            Ok(0)
13196        );
13197        assert_eq!(
13198            db.account_matches_owners(&Ancestors::default(), &account2_key, &owners),
13199            Ok(1)
13200        );
13201        assert_eq!(
13202            db.account_matches_owners(&Ancestors::default(), &account3_key, &owners),
13203            Err(MatchAccountOwnerError::NoMatch)
13204        );
13205        assert_eq!(
13206            db.account_matches_owners(&Ancestors::default(), &account4_key, &owners),
13207            Err(MatchAccountOwnerError::NoMatch)
13208        );
13209        assert_eq!(
13210            db.account_matches_owners(&Ancestors::default(), &Pubkey::new_unique(), &owners),
13211            Err(MatchAccountOwnerError::UnableToLoad)
13212        );
13213
13214        // Flush the cache and load account1 (so that it's in the cache)
13215        db.flush_accounts_cache(true, None);
13216        db.clean_accounts_for_tests();
13217        let _ = db
13218            .do_load(
13219                &Ancestors::default(),
13220                &account1_key,
13221                Some(0),
13222                LoadHint::Unspecified,
13223                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
13224            )
13225            .unwrap();
13226
13227        assert_eq!(
13228            db.account_matches_owners(&Ancestors::default(), &account1_key, &owners),
13229            Ok(0)
13230        );
13231        assert_eq!(
13232            db.account_matches_owners(&Ancestors::default(), &account2_key, &owners),
13233            Ok(1)
13234        );
13235        assert_eq!(
13236            db.account_matches_owners(&Ancestors::default(), &account3_key, &owners),
13237            Err(MatchAccountOwnerError::NoMatch)
13238        );
13239        assert_eq!(
13240            db.account_matches_owners(&Ancestors::default(), &account4_key, &owners),
13241            Err(MatchAccountOwnerError::NoMatch)
13242        );
13243        assert_eq!(
13244            db.account_matches_owners(&Ancestors::default(), &Pubkey::new_unique(), &owners),
13245            Err(MatchAccountOwnerError::UnableToLoad)
13246        );
13247    }
13248
13249    /// a test that will accept either answer
13250    const LOAD_ZERO_LAMPORTS_ANY_TESTS: LoadZeroLamports = LoadZeroLamports::None;
13251
13252    #[test]
13253    fn test_flush_cache_clean() {
13254        let db = Arc::new(AccountsDb::new_single_for_tests());
13255
13256        let account_key = Pubkey::new_unique();
13257        let zero_lamport_account =
13258            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13259        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13260        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13261        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13262
13263        db.add_root(0);
13264        db.add_root(1);
13265
13266        // Clean should not remove anything yet as nothing has been flushed
13267        db.clean_accounts_for_tests();
13268        let account = db
13269            .do_load(
13270                &Ancestors::default(),
13271                &account_key,
13272                Some(0),
13273                LoadHint::Unspecified,
13274                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
13275            )
13276            .unwrap();
13277        assert_eq!(account.0.lamports(), 0);
13278        // since this item is in the cache, it should not be in the read only cache
13279        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13280
13281        // Flush, then clean again. Should not need another root to initiate the cleaning
13282        // because `accounts_index.uncleaned_roots` should be correct
13283        db.flush_accounts_cache(true, None);
13284        db.clean_accounts_for_tests();
13285        assert!(db
13286            .do_load(
13287                &Ancestors::default(),
13288                &account_key,
13289                Some(0),
13290                LoadHint::Unspecified,
13291                LOAD_ZERO_LAMPORTS_ANY_TESTS
13292            )
13293            .is_none());
13294    }
13295
13296    #[test]
13297    fn test_flush_cache_dont_clean_zero_lamport_account() {
13298        let db = Arc::new(AccountsDb::new_single_for_tests());
13299
13300        let zero_lamport_account_key = Pubkey::new_unique();
13301        let other_account_key = Pubkey::new_unique();
13302
13303        let original_lamports = 1;
13304        let slot0_account =
13305            AccountSharedData::new(original_lamports, 1, AccountSharedData::default().owner());
13306        let zero_lamport_account =
13307            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13308
13309        // Store into slot 0, and then flush the slot to storage
13310        db.store_cached(
13311            (0, &[(&zero_lamport_account_key, &slot0_account)][..]),
13312            None,
13313        );
13314        // Second key keeps other lamport account entry for slot 0 alive,
13315        // preventing clean of the zero_lamport_account in slot 1.
13316        db.store_cached((0, &[(&other_account_key, &slot0_account)][..]), None);
13317        db.add_root(0);
13318        db.flush_accounts_cache(true, None);
13319        assert!(db.storage.get_slot_storage_entry(0).is_some());
13320
13321        // Store into slot 1, a dummy slot that will be dead and purged before flush
13322        db.store_cached(
13323            (1, &[(&zero_lamport_account_key, &zero_lamport_account)][..]),
13324            None,
13325        );
13326
13327        // Store into slot 2, which makes all updates from slot 1 outdated.
13328        // This means slot 1 is a dead slot. Later, slot 1 will be cleaned/purged
13329        // before it even reaches storage, but this purge of slot 1should not affect
13330        // the refcount of `zero_lamport_account_key` because cached keys do not bump
13331        // the refcount in the index. This means clean should *not* remove
13332        // `zero_lamport_account_key` from slot 2
13333        db.store_cached(
13334            (2, &[(&zero_lamport_account_key, &zero_lamport_account)][..]),
13335            None,
13336        );
13337        db.add_root(1);
13338        db.add_root(2);
13339
13340        // Flush, then clean. Should not need another root to initiate the cleaning
13341        // because `accounts_index.uncleaned_roots` should be correct
13342        db.flush_accounts_cache(true, None);
13343        db.clean_accounts_for_tests();
13344
13345        // The `zero_lamport_account_key` is still alive in slot 1, so refcount for the
13346        // pubkey should be 2
13347        assert_eq!(
13348            db.accounts_index
13349                .ref_count_from_storage(&zero_lamport_account_key),
13350            2
13351        );
13352        assert_eq!(
13353            db.accounts_index.ref_count_from_storage(&other_account_key),
13354            1
13355        );
13356
13357        // The zero-lamport account in slot 2 should not be purged yet, because the
13358        // entry in slot 1 is blocking cleanup of the zero-lamport account.
13359        let max_root = None;
13360        // Fine to simulate a transaction load since we are not doing any out of band
13361        // removals, only using clean_accounts
13362        let load_hint = LoadHint::FixedMaxRoot;
13363        assert_eq!(
13364            db.do_load(
13365                &Ancestors::default(),
13366                &zero_lamport_account_key,
13367                max_root,
13368                load_hint,
13369                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
13370            )
13371            .unwrap()
13372            .0
13373            .lamports(),
13374            0
13375        );
13376    }
13377
13378    struct ScanTracker {
13379        t_scan: JoinHandle<()>,
13380        exit: Arc<AtomicBool>,
13381    }
13382
13383    impl ScanTracker {
13384        fn exit(self) -> thread::Result<()> {
13385            self.exit.store(true, Ordering::Relaxed);
13386            self.t_scan.join()
13387        }
13388    }
13389
13390    fn setup_scan(
13391        db: Arc<AccountsDb>,
13392        scan_ancestors: Arc<Ancestors>,
13393        bank_id: BankId,
13394        stall_key: Pubkey,
13395    ) -> ScanTracker {
13396        let exit = Arc::new(AtomicBool::new(false));
13397        let exit_ = exit.clone();
13398        let ready = Arc::new(AtomicBool::new(false));
13399        let ready_ = ready.clone();
13400
13401        let t_scan = Builder::new()
13402            .name("scan".to_string())
13403            .spawn(move || {
13404                db.scan_accounts(
13405                    &scan_ancestors,
13406                    bank_id,
13407                    |maybe_account| {
13408                        ready_.store(true, Ordering::Relaxed);
13409                        if let Some((pubkey, _, _)) = maybe_account {
13410                            if *pubkey == stall_key {
13411                                loop {
13412                                    if exit_.load(Ordering::Relaxed) {
13413                                        break;
13414                                    } else {
13415                                        sleep(Duration::from_millis(10));
13416                                    }
13417                                }
13418                            }
13419                        }
13420                    },
13421                    &ScanConfig::default(),
13422                )
13423                .unwrap();
13424            })
13425            .unwrap();
13426
13427        // Wait for scan to start
13428        while !ready.load(Ordering::Relaxed) {
13429            sleep(Duration::from_millis(10));
13430        }
13431
13432        ScanTracker { t_scan, exit }
13433    }
13434
13435    #[test]
13436    fn test_scan_flush_accounts_cache_then_clean_drop() {
13437        let db = Arc::new(AccountsDb::new_single_for_tests());
13438        let account_key = Pubkey::new_unique();
13439        let account_key2 = Pubkey::new_unique();
13440        let zero_lamport_account =
13441            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13442        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13443        let slot2_account = AccountSharedData::new(2, 1, AccountSharedData::default().owner());
13444
13445        /*
13446            Store zero lamport account into slots 0, 1, 2 where
13447            root slots are 0, 2, and slot 1 is unrooted.
13448                                    0 (root)
13449                                /        \
13450                              1            2 (root)
13451        */
13452        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13453        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13454        // Fodder for the scan so that the lock on `account_key` is not held
13455        db.store_cached((1, &[(&account_key2, &slot1_account)][..]), None);
13456        db.store_cached((2, &[(&account_key, &slot2_account)][..]), None);
13457        db.calculate_accounts_delta_hash(0);
13458
13459        let max_scan_root = 0;
13460        db.add_root(max_scan_root);
13461        let scan_ancestors: Arc<Ancestors> = Arc::new(vec![(0, 1), (1, 1)].into_iter().collect());
13462        let bank_id = 0;
13463        let scan_tracker = setup_scan(db.clone(), scan_ancestors.clone(), bank_id, account_key2);
13464
13465        // Add a new root 2
13466        let new_root = 2;
13467        db.calculate_accounts_delta_hash(new_root);
13468        db.add_root(new_root);
13469
13470        // Check that the scan is properly set up
13471        assert_eq!(
13472            db.accounts_index.min_ongoing_scan_root().unwrap(),
13473            max_scan_root
13474        );
13475
13476        // If we specify a requested_flush_root == 2, then `slot 2 <= max_flush_slot` will
13477        // be flushed even though `slot 2 > max_scan_root`. The unrooted slot 1 should
13478        // remain in the cache
13479        db.flush_accounts_cache(true, Some(new_root));
13480        assert_eq!(db.accounts_cache.num_slots(), 1);
13481        assert!(db.accounts_cache.slot_cache(1).is_some());
13482
13483        // Intra cache cleaning should not clean the entry for `account_key` from slot 0,
13484        // even though it was updated in slot `2` because of the ongoing scan
13485        let account = db
13486            .do_load(
13487                &Ancestors::default(),
13488                &account_key,
13489                Some(0),
13490                LoadHint::Unspecified,
13491                LoadZeroLamports::SomeWithZeroLamportAccountForTests,
13492            )
13493            .unwrap();
13494        assert_eq!(account.0.lamports(), zero_lamport_account.lamports());
13495
13496        // Run clean, unrooted slot 1 should not be purged, and still readable from the cache,
13497        // because we're still doing a scan on it.
13498        db.clean_accounts_for_tests();
13499        let account = db
13500            .do_load(
13501                &scan_ancestors,
13502                &account_key,
13503                Some(max_scan_root),
13504                LoadHint::Unspecified,
13505                LOAD_ZERO_LAMPORTS_ANY_TESTS,
13506            )
13507            .unwrap();
13508        assert_eq!(account.0.lamports(), slot1_account.lamports());
13509
13510        // When the scan is over, clean should not panic and should not purge something
13511        // still in the cache.
13512        scan_tracker.exit().unwrap();
13513        db.clean_accounts_for_tests();
13514        let account = db
13515            .do_load(
13516                &scan_ancestors,
13517                &account_key,
13518                Some(max_scan_root),
13519                LoadHint::Unspecified,
13520                LOAD_ZERO_LAMPORTS_ANY_TESTS,
13521            )
13522            .unwrap();
13523        assert_eq!(account.0.lamports(), slot1_account.lamports());
13524
13525        // Simulate dropping the bank, which finally removes the slot from the cache
13526        let bank_id = 1;
13527        db.purge_slot(1, bank_id, false);
13528        assert!(db
13529            .do_load(
13530                &scan_ancestors,
13531                &account_key,
13532                Some(max_scan_root),
13533                LoadHint::Unspecified,
13534                LOAD_ZERO_LAMPORTS_ANY_TESTS
13535            )
13536            .is_none());
13537    }
13538
13539    impl AccountsDb {
13540        fn get_and_assert_single_storage(&self, slot: Slot) -> Arc<AccountStorageEntry> {
13541            self.storage.get_slot_storage_entry(slot).unwrap()
13542        }
13543    }
13544
13545    define_accounts_db_test!(test_alive_bytes, |accounts_db| {
13546        let slot: Slot = 0;
13547        let num_keys = 10;
13548
13549        for data_size in 0..num_keys {
13550            let account = AccountSharedData::new(1, data_size, &Pubkey::default());
13551            accounts_db.store_cached((slot, &[(&Pubkey::new_unique(), &account)][..]), None);
13552        }
13553
13554        accounts_db.add_root(slot);
13555        accounts_db.flush_accounts_cache(true, None);
13556
13557        // Flushing cache should only create one storage entry
13558        let storage0 = accounts_db.get_and_assert_single_storage(slot);
13559
13560        storage0.accounts.scan_accounts(|account| {
13561            let before_size = storage0.alive_bytes();
13562            let account_info = accounts_db
13563                .accounts_index
13564                .get_cloned(account.pubkey())
13565                .unwrap()
13566                .slot_list
13567                .read()
13568                .unwrap()
13569                // Should only be one entry per key, since every key was only stored to slot 0
13570                [0];
13571            assert_eq!(account_info.0, slot);
13572            let reclaims = [account_info];
13573            accounts_db.remove_dead_accounts(reclaims.iter(), None, true);
13574            let after_size = storage0.alive_bytes();
13575            if storage0.count() == 0
13576                && AccountsFileProvider::HotStorage == accounts_db.accounts_file_provider
13577            {
13578                // when `remove_dead_accounts` reaches 0 accounts, all bytes are marked as dead
13579                assert_eq!(after_size, 0);
13580            } else {
13581                assert_eq!(before_size, after_size + account.stored_size());
13582            }
13583        });
13584    });
13585
13586    fn setup_accounts_db_cache_clean(
13587        num_slots: usize,
13588        scan_slot: Option<Slot>,
13589        write_cache_limit_bytes: Option<u64>,
13590    ) -> (Arc<AccountsDb>, Vec<Pubkey>, Vec<Slot>, Option<ScanTracker>) {
13591        let mut accounts_db = AccountsDb::new_single_for_tests();
13592        accounts_db.write_cache_limit_bytes = write_cache_limit_bytes;
13593        let accounts_db = Arc::new(accounts_db);
13594
13595        let slots: Vec<_> = (0..num_slots as Slot).collect();
13596        let stall_slot = num_slots as Slot;
13597        let scan_stall_key = Pubkey::new_unique();
13598        let keys: Vec<Pubkey> = std::iter::repeat_with(Pubkey::new_unique)
13599            .take(num_slots)
13600            .collect();
13601        if scan_slot.is_some() {
13602            accounts_db.store_cached(
13603                // Store it in a slot that isn't returned in `slots`
13604                (
13605                    stall_slot,
13606                    &[(
13607                        &scan_stall_key,
13608                        &AccountSharedData::new(1, 0, &Pubkey::default()),
13609                    )][..],
13610                ),
13611                None,
13612            );
13613        }
13614
13615        // Store some subset of the keys in slots 0..num_slots
13616        let mut scan_tracker = None;
13617        for slot in &slots {
13618            for key in &keys[*slot as usize..] {
13619                let space = 1; // 1 byte allows us to track by size
13620                accounts_db.store_cached(
13621                    (
13622                        *slot,
13623                        &[(key, &AccountSharedData::new(1, space, &Pubkey::default()))][..],
13624                    ),
13625                    None,
13626                );
13627            }
13628            accounts_db.add_root(*slot as Slot);
13629            if Some(*slot) == scan_slot {
13630                let ancestors = Arc::new(vec![(stall_slot, 1), (*slot, 1)].into_iter().collect());
13631                let bank_id = 0;
13632                scan_tracker = Some(setup_scan(
13633                    accounts_db.clone(),
13634                    ancestors,
13635                    bank_id,
13636                    scan_stall_key,
13637                ));
13638                assert_eq!(
13639                    accounts_db.accounts_index.min_ongoing_scan_root().unwrap(),
13640                    *slot
13641                );
13642            }
13643        }
13644
13645        accounts_db.accounts_cache.remove_slot(stall_slot);
13646
13647        // If there's <= max_cache_slots(), no slots should be flushed
13648        if accounts_db.accounts_cache.num_slots() <= max_cache_slots() {
13649            accounts_db.flush_accounts_cache(false, None);
13650            assert_eq!(accounts_db.accounts_cache.num_slots(), num_slots);
13651        }
13652
13653        (accounts_db, keys, slots, scan_tracker)
13654    }
13655
13656    #[test]
13657    fn test_accounts_db_cache_clean_dead_slots() {
13658        let num_slots = 10;
13659        let (accounts_db, keys, mut slots, _) =
13660            setup_accounts_db_cache_clean(num_slots, None, None);
13661        let last_dead_slot = (num_slots - 1) as Slot;
13662        assert_eq!(*slots.last().unwrap(), last_dead_slot);
13663        let alive_slot = last_dead_slot as Slot + 1;
13664        slots.push(alive_slot);
13665        for key in &keys {
13666            // Store a slot that overwrites all previous keys, rendering all previous keys dead
13667            accounts_db.store_cached(
13668                (
13669                    alive_slot,
13670                    &[(key, &AccountSharedData::new(1, 0, &Pubkey::default()))][..],
13671                ),
13672                None,
13673            );
13674            accounts_db.add_root(alive_slot);
13675        }
13676
13677        // Before the flush, we can find entries in the database for slots < alive_slot if we specify
13678        // a smaller max root
13679        for key in &keys {
13680            assert!(accounts_db
13681                .do_load(
13682                    &Ancestors::default(),
13683                    key,
13684                    Some(last_dead_slot),
13685                    LoadHint::Unspecified,
13686                    LOAD_ZERO_LAMPORTS_ANY_TESTS
13687                )
13688                .is_some());
13689        }
13690
13691        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
13692        accounts_db.flush_accounts_cache(true, None);
13693        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
13694        let mut uncleaned_roots = accounts_db
13695            .accounts_index
13696            .clear_uncleaned_roots(None)
13697            .into_iter()
13698            .collect::<Vec<_>>();
13699        uncleaned_roots.sort_unstable();
13700        assert_eq!(uncleaned_roots, slots);
13701        assert_eq!(
13702            accounts_db.accounts_cache.fetch_max_flush_root(),
13703            alive_slot,
13704        );
13705
13706        // Specifying a max_root < alive_slot, should not return any more entries,
13707        // as those have been purged from the accounts index for the dead slots.
13708        for key in &keys {
13709            assert!(accounts_db
13710                .do_load(
13711                    &Ancestors::default(),
13712                    key,
13713                    Some(last_dead_slot),
13714                    LoadHint::Unspecified,
13715                    LOAD_ZERO_LAMPORTS_ANY_TESTS
13716                )
13717                .is_none());
13718        }
13719        // Each slot should only have one entry in the storage, since all other accounts were
13720        // cleaned due to later updates
13721        for slot in &slots {
13722            if let ScanStorageResult::Stored(slot_accounts) = accounts_db.scan_account_storage(
13723                *slot as Slot,
13724                |_| Some(0),
13725                |slot_accounts: &DashSet<Pubkey>, loaded_account: &LoadedAccount, _data| {
13726                    slot_accounts.insert(*loaded_account.pubkey());
13727                },
13728                ScanAccountStorageData::NoData,
13729            ) {
13730                if *slot == alive_slot {
13731                    assert_eq!(slot_accounts.len(), keys.len());
13732                } else {
13733                    assert!(slot_accounts.is_empty());
13734                }
13735            } else {
13736                panic!("Expected slot to be in storage, not cache");
13737            }
13738        }
13739    }
13740
13741    #[test]
13742    fn test_accounts_db_cache_clean() {
13743        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(10, None, None);
13744
13745        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
13746        accounts_db.flush_accounts_cache(true, None);
13747        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
13748        let mut uncleaned_roots = accounts_db
13749            .accounts_index
13750            .clear_uncleaned_roots(None)
13751            .into_iter()
13752            .collect::<Vec<_>>();
13753        uncleaned_roots.sort_unstable();
13754        assert_eq!(uncleaned_roots, slots);
13755        assert_eq!(
13756            accounts_db.accounts_cache.fetch_max_flush_root(),
13757            *slots.last().unwrap()
13758        );
13759
13760        // Each slot should only have one entry in the storage, since all other accounts were
13761        // cleaned due to later updates
13762        for slot in &slots {
13763            if let ScanStorageResult::Stored(slot_account) = accounts_db.scan_account_storage(
13764                *slot as Slot,
13765                |_| Some(0),
13766                |slot_account: &RwLock<Pubkey>, loaded_account: &LoadedAccount, _data| {
13767                    *slot_account.write().unwrap() = *loaded_account.pubkey();
13768                },
13769                ScanAccountStorageData::NoData,
13770            ) {
13771                assert_eq!(*slot_account.read().unwrap(), keys[*slot as usize]);
13772            } else {
13773                panic!("Everything should have been flushed")
13774            }
13775        }
13776    }
13777
13778    fn run_test_accounts_db_cache_clean_max_root(
13779        num_slots: usize,
13780        requested_flush_root: Slot,
13781        scan_root: Option<Slot>,
13782    ) {
13783        assert!(requested_flush_root < (num_slots as Slot));
13784        let (accounts_db, keys, slots, scan_tracker) =
13785            setup_accounts_db_cache_clean(num_slots, scan_root, Some(max_cache_slots() as u64));
13786        let is_cache_at_limit = num_slots - requested_flush_root as usize - 1 > max_cache_slots();
13787
13788        // If:
13789        // 1) `requested_flush_root` is specified,
13790        // 2) not at the cache limit, i.e. `is_cache_at_limit == false`, then
13791        // `flush_accounts_cache()` should clean and flush only slots <= requested_flush_root,
13792        accounts_db.flush_accounts_cache(true, Some(requested_flush_root));
13793
13794        if !is_cache_at_limit {
13795            // Should flush all slots between 0..=requested_flush_root
13796            assert_eq!(
13797                accounts_db.accounts_cache.num_slots(),
13798                slots.len() - requested_flush_root as usize - 1
13799            );
13800        } else {
13801            // Otherwise, if we are at the cache limit, all roots will be flushed
13802            assert_eq!(accounts_db.accounts_cache.num_slots(), 0,);
13803        }
13804
13805        let mut uncleaned_roots = accounts_db
13806            .accounts_index
13807            .clear_uncleaned_roots(None)
13808            .into_iter()
13809            .collect::<Vec<_>>();
13810        uncleaned_roots.sort_unstable();
13811
13812        let expected_max_flushed_root = if !is_cache_at_limit {
13813            // Should flush all slots between 0..=requested_flush_root
13814            requested_flush_root
13815        } else {
13816            // Otherwise, if we are at the cache limit, all roots will be flushed
13817            num_slots as Slot - 1
13818        };
13819
13820        assert_eq!(
13821            uncleaned_roots,
13822            slots[0..=expected_max_flushed_root as usize].to_vec()
13823        );
13824        assert_eq!(
13825            accounts_db.accounts_cache.fetch_max_flush_root(),
13826            expected_max_flushed_root,
13827        );
13828
13829        for slot in &slots {
13830            let slot_accounts = accounts_db.scan_account_storage(
13831                *slot as Slot,
13832                |loaded_account: &LoadedAccount| {
13833                    assert!(
13834                        !is_cache_at_limit,
13835                        "When cache is at limit, all roots should have been flushed to storage"
13836                    );
13837                    // All slots <= requested_flush_root should have been flushed, regardless
13838                    // of ongoing scans
13839                    assert!(*slot > requested_flush_root);
13840                    Some(*loaded_account.pubkey())
13841                },
13842                |slot_accounts: &DashSet<Pubkey>, loaded_account: &LoadedAccount, _data| {
13843                    slot_accounts.insert(*loaded_account.pubkey());
13844                    if !is_cache_at_limit {
13845                        // Only true when the limit hasn't been reached and there are still
13846                        // slots left in the cache
13847                        assert!(*slot <= requested_flush_root);
13848                    }
13849                },
13850                ScanAccountStorageData::NoData,
13851            );
13852
13853            let slot_accounts = match slot_accounts {
13854                ScanStorageResult::Cached(slot_accounts) => {
13855                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
13856                }
13857                ScanStorageResult::Stored(slot_accounts) => {
13858                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
13859                }
13860            };
13861
13862            let expected_accounts =
13863                if *slot >= requested_flush_root || *slot >= scan_root.unwrap_or(Slot::MAX) {
13864                    // 1) If slot > `requested_flush_root`, then  either:
13865                    //   a) If `is_cache_at_limit == false`, still in the cache
13866                    //   b) if `is_cache_at_limit == true`, were not cleaned before being flushed to storage.
13867                    //
13868                    // In both cases all the *original* updates at index `slot` were uncleaned and thus
13869                    // should be discoverable by this scan.
13870                    //
13871                    // 2) If slot == `requested_flush_root`, the slot was not cleaned before being flushed to storage,
13872                    // so it also contains all the original updates.
13873                    //
13874                    // 3) If *slot >= scan_root, then we should not clean it either
13875                    keys[*slot as usize..]
13876                        .iter()
13877                        .cloned()
13878                        .collect::<HashSet<Pubkey>>()
13879                } else {
13880                    // Slots less than `requested_flush_root` and `scan_root` were cleaned in the cache before being flushed
13881                    // to storage, should only contain one account
13882                    std::iter::once(keys[*slot as usize]).collect::<HashSet<Pubkey>>()
13883                };
13884
13885            assert_eq!(slot_accounts, expected_accounts);
13886        }
13887
13888        if let Some(scan_tracker) = scan_tracker {
13889            scan_tracker.exit().unwrap();
13890        }
13891    }
13892
13893    #[test]
13894    fn test_accounts_db_cache_clean_max_root() {
13895        let requested_flush_root = 5;
13896        run_test_accounts_db_cache_clean_max_root(10, requested_flush_root, None);
13897    }
13898
13899    #[test]
13900    fn test_accounts_db_cache_clean_max_root_with_scan() {
13901        let requested_flush_root = 5;
13902        run_test_accounts_db_cache_clean_max_root(
13903            10,
13904            requested_flush_root,
13905            Some(requested_flush_root - 1),
13906        );
13907        run_test_accounts_db_cache_clean_max_root(
13908            10,
13909            requested_flush_root,
13910            Some(requested_flush_root + 1),
13911        );
13912    }
13913
13914    #[test]
13915    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit() {
13916        let requested_flush_root = 5;
13917        // Test that if there are > max_cache_slots() in the cache after flush, then more roots
13918        // will be flushed
13919        run_test_accounts_db_cache_clean_max_root(
13920            max_cache_slots() + requested_flush_root as usize + 2,
13921            requested_flush_root,
13922            None,
13923        );
13924    }
13925
13926    #[test]
13927    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit_and_scan() {
13928        let requested_flush_root = 5;
13929        // Test that if there are > max_cache_slots() in the cache after flush, then more roots
13930        // will be flushed
13931        run_test_accounts_db_cache_clean_max_root(
13932            max_cache_slots() + requested_flush_root as usize + 2,
13933            requested_flush_root,
13934            Some(requested_flush_root - 1),
13935        );
13936        run_test_accounts_db_cache_clean_max_root(
13937            max_cache_slots() + requested_flush_root as usize + 2,
13938            requested_flush_root,
13939            Some(requested_flush_root + 1),
13940        );
13941    }
13942
13943    fn run_flush_rooted_accounts_cache(should_clean: bool) {
13944        let num_slots = 10;
13945        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(num_slots, None, None);
13946        let mut cleaned_bytes = 0;
13947        let mut cleaned_accounts = 0;
13948        let should_clean_tracker = if should_clean {
13949            Some((&mut cleaned_bytes, &mut cleaned_accounts))
13950        } else {
13951            None
13952        };
13953
13954        // If no cleaning is specified, then flush everything
13955        accounts_db.flush_rooted_accounts_cache(None, should_clean_tracker);
13956        for slot in &slots {
13957            let slot_accounts = if let ScanStorageResult::Stored(slot_accounts) = accounts_db
13958                .scan_account_storage(
13959                    *slot as Slot,
13960                    |_| Some(0),
13961                    |slot_account: &DashSet<Pubkey>, loaded_account: &LoadedAccount, _data| {
13962                        slot_account.insert(*loaded_account.pubkey());
13963                    },
13964                    ScanAccountStorageData::NoData,
13965                ) {
13966                slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
13967            } else {
13968                panic!("All roots should have been flushed to storage");
13969            };
13970            let expected_accounts = if !should_clean || slot == slots.last().unwrap() {
13971                // The slot was not cleaned before being flushed to storage,
13972                // so it also contains all the original updates.
13973                keys[*slot as usize..]
13974                    .iter()
13975                    .cloned()
13976                    .collect::<HashSet<Pubkey>>()
13977            } else {
13978                // If clean was specified, only the latest slot should have all the updates.
13979                // All these other slots have been cleaned before flush
13980                std::iter::once(keys[*slot as usize]).collect::<HashSet<Pubkey>>()
13981            };
13982            assert_eq!(slot_accounts, expected_accounts);
13983        }
13984    }
13985
13986    #[test]
13987    fn test_flush_rooted_accounts_cache_with_clean() {
13988        run_flush_rooted_accounts_cache(true);
13989    }
13990
13991    #[test]
13992    fn test_flush_rooted_accounts_cache_without_clean() {
13993        run_flush_rooted_accounts_cache(false);
13994    }
13995
13996    fn run_test_shrink_unref(do_intra_cache_clean: bool) {
13997        let db = AccountsDb::new_single_for_tests();
13998        let epoch_schedule = EpochSchedule::default();
13999        let account_key1 = Pubkey::new_unique();
14000        let account_key2 = Pubkey::new_unique();
14001        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14002
14003        // Store into slot 0
14004        // This has to be done uncached since we are trying to add another account to the append vec AFTER it has been flushed.
14005        // This doesn't work if the flush creates an append vec of exactly the right size.
14006        // Normal operations NEVER write the same account to the same append vec twice during a write cache flush.
14007        db.store_uncached(0, &[(&account_key1, &account1)][..]);
14008        db.store_uncached(0, &[(&account_key2, &account1)][..]);
14009        db.add_root(0);
14010        if !do_intra_cache_clean {
14011            // Add an additional ref within the same slot to pubkey 1
14012            db.store_uncached(0, &[(&account_key1, &account1)]);
14013        }
14014
14015        // Make account_key1 in slot 0 outdated by updating in rooted slot 1
14016        db.store_cached((1, &[(&account_key1, &account1)][..]), None);
14017        db.add_root(1);
14018        // Flushes all roots
14019        db.flush_accounts_cache(true, None);
14020        db.calculate_accounts_delta_hash(0);
14021        db.calculate_accounts_delta_hash(1);
14022
14023        // Clean to remove outdated entry from slot 0
14024        db.clean_accounts(
14025            Some(1),
14026            false,
14027            &EpochSchedule::default(),
14028            OldStoragesPolicy::Leave,
14029        );
14030
14031        // Shrink Slot 0
14032        {
14033            let mut shrink_candidate_slots = db.shrink_candidate_slots.lock().unwrap();
14034            shrink_candidate_slots.insert(0);
14035        }
14036        db.shrink_candidate_slots(&epoch_schedule);
14037
14038        // Make slot 0 dead by updating the remaining key
14039        db.store_cached((2, &[(&account_key2, &account1)][..]), None);
14040        db.add_root(2);
14041
14042        // Flushes all roots
14043        db.flush_accounts_cache(true, None);
14044
14045        // Should be one store before clean for slot 0
14046        db.get_and_assert_single_storage(0);
14047        db.calculate_accounts_delta_hash(2);
14048        db.clean_accounts(
14049            Some(2),
14050            false,
14051            &EpochSchedule::default(),
14052            OldStoragesPolicy::Leave,
14053        );
14054
14055        // No stores should exist for slot 0 after clean
14056        assert_no_storages_at_slot(&db, 0);
14057
14058        // Ref count for `account_key1` (account removed earlier by shrink)
14059        // should be 1, since it was only stored in slot 0 and 1, and slot 0
14060        // is now dead
14061        assert_eq!(db.accounts_index.ref_count_from_storage(&account_key1), 1);
14062    }
14063
14064    #[test]
14065    fn test_shrink_unref() {
14066        run_test_shrink_unref(false)
14067    }
14068
14069    #[test]
14070    fn test_shrink_unref_with_intra_slot_cleaning() {
14071        run_test_shrink_unref(true)
14072    }
14073
14074    define_accounts_db_test!(test_partial_clean, |db| {
14075        let account_key1 = Pubkey::new_unique();
14076        let account_key2 = Pubkey::new_unique();
14077        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14078        let account2 = AccountSharedData::new(2, 0, AccountSharedData::default().owner());
14079        let account3 = AccountSharedData::new(3, 0, AccountSharedData::default().owner());
14080        let account4 = AccountSharedData::new(4, 0, AccountSharedData::default().owner());
14081
14082        // Store accounts into slots 0 and 1
14083        db.store_uncached(0, &[(&account_key1, &account1), (&account_key2, &account1)]);
14084        db.store_uncached(1, &[(&account_key1, &account2)]);
14085        db.calculate_accounts_delta_hash(0);
14086        db.calculate_accounts_delta_hash(1);
14087        db.print_accounts_stats("pre-clean1");
14088
14089        // clean accounts - no accounts should be cleaned, since no rooted slots
14090        //
14091        // Checking that the uncleaned_pubkeys are not pre-maturely removed
14092        // such that when the slots are rooted, and can actually be cleaned, then the
14093        // delta keys are still there.
14094        db.clean_accounts_for_tests();
14095
14096        db.print_accounts_stats("post-clean1");
14097        // Check stores > 0
14098        assert!(!db.storage.is_empty_entry(0));
14099        assert!(!db.storage.is_empty_entry(1));
14100
14101        // root slot 0
14102        db.add_root_and_flush_write_cache(0);
14103
14104        // store into slot 2
14105        db.store_uncached(2, &[(&account_key2, &account3), (&account_key1, &account3)]);
14106        db.calculate_accounts_delta_hash(2);
14107        db.clean_accounts_for_tests();
14108        db.print_accounts_stats("post-clean2");
14109
14110        // root slots 1
14111        db.add_root_and_flush_write_cache(1);
14112        db.clean_accounts_for_tests();
14113
14114        db.print_accounts_stats("post-clean3");
14115
14116        db.store_uncached(3, &[(&account_key2, &account4)]);
14117        db.calculate_accounts_delta_hash(3);
14118        db.add_root_and_flush_write_cache(3);
14119
14120        // Check that we can clean where max_root=3 and slot=2 is not rooted
14121        db.clean_accounts_for_tests();
14122
14123        assert!(db.uncleaned_pubkeys.is_empty());
14124
14125        db.print_accounts_stats("post-clean4");
14126
14127        assert!(db.storage.is_empty_entry(0));
14128        assert!(!db.storage.is_empty_entry(1));
14129    });
14130
14131    const RACY_SLEEP_MS: u64 = 10;
14132    const RACE_TIME: u64 = 5;
14133
14134    fn start_load_thread(
14135        with_retry: bool,
14136        ancestors: Ancestors,
14137        db: Arc<AccountsDb>,
14138        exit: Arc<AtomicBool>,
14139        pubkey: Arc<Pubkey>,
14140        expected_lamports: impl Fn(&(AccountSharedData, Slot)) -> u64 + Send + 'static,
14141    ) -> JoinHandle<()> {
14142        let load_hint = if with_retry {
14143            LoadHint::FixedMaxRoot
14144        } else {
14145            LoadHint::Unspecified
14146        };
14147
14148        std::thread::Builder::new()
14149            .name("account-do-load".to_string())
14150            .spawn(move || {
14151                loop {
14152                    if exit.load(Ordering::Relaxed) {
14153                        return;
14154                    }
14155                    // Meddle load_limit to cover all branches of implementation.
14156                    // There should absolutely no behaviorial difference; the load_limit triggered
14157                    // slow branch should only affect the performance.
14158                    // Ordering::Relaxed is ok because of no data dependencies; the modified field is
14159                    // completely free-standing cfg(test) control-flow knob.
14160                    db.load_limit
14161                        .store(thread_rng().gen_range(0..10) as u64, Ordering::Relaxed);
14162
14163                    // Load should never be unable to find this key
14164                    let loaded_account = db
14165                        .do_load(
14166                            &ancestors,
14167                            &pubkey,
14168                            None,
14169                            load_hint,
14170                            LOAD_ZERO_LAMPORTS_ANY_TESTS,
14171                        )
14172                        .unwrap();
14173                    // slot + 1 == account.lamports because of the account-cache-flush thread
14174                    assert_eq!(
14175                        loaded_account.0.lamports(),
14176                        expected_lamports(&loaded_account)
14177                    );
14178                }
14179            })
14180            .unwrap()
14181    }
14182
14183    fn do_test_load_account_and_cache_flush_race(with_retry: bool) {
14184        solana_logger::setup();
14185
14186        let mut db = AccountsDb::new_single_for_tests();
14187        db.load_delay = RACY_SLEEP_MS;
14188        let db = Arc::new(db);
14189        let pubkey = Arc::new(Pubkey::new_unique());
14190        let exit = Arc::new(AtomicBool::new(false));
14191        db.store_cached(
14192            (
14193                0,
14194                &[(
14195                    pubkey.as_ref(),
14196                    &AccountSharedData::new(1, 0, AccountSharedData::default().owner()),
14197                )][..],
14198            ),
14199            None,
14200        );
14201        db.add_root(0);
14202        db.flush_accounts_cache(true, None);
14203
14204        let t_flush_accounts_cache = {
14205            let db = db.clone();
14206            let exit = exit.clone();
14207            let pubkey = pubkey.clone();
14208            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14209            std::thread::Builder::new()
14210                .name("account-cache-flush".to_string())
14211                .spawn(move || {
14212                    let mut slot: Slot = 1;
14213                    loop {
14214                        if exit.load(Ordering::Relaxed) {
14215                            return;
14216                        }
14217                        account.set_lamports(slot + 1);
14218                        db.store_cached((slot, &[(pubkey.as_ref(), &account)][..]), None);
14219                        db.add_root(slot);
14220                        sleep(Duration::from_millis(RACY_SLEEP_MS));
14221                        db.flush_accounts_cache(true, None);
14222                        slot += 1;
14223                    }
14224                })
14225                .unwrap()
14226        };
14227
14228        let t_do_load = start_load_thread(
14229            with_retry,
14230            Ancestors::default(),
14231            db,
14232            exit.clone(),
14233            pubkey,
14234            |(_, slot)| slot + 1,
14235        );
14236
14237        sleep(Duration::from_secs(RACE_TIME));
14238        exit.store(true, Ordering::Relaxed);
14239        t_flush_accounts_cache.join().unwrap();
14240        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
14241    }
14242
14243    #[test]
14244    fn test_load_account_and_cache_flush_race_with_retry() {
14245        do_test_load_account_and_cache_flush_race(true);
14246    }
14247
14248    #[test]
14249    fn test_load_account_and_cache_flush_race_without_retry() {
14250        do_test_load_account_and_cache_flush_race(false);
14251    }
14252
14253    fn do_test_load_account_and_shrink_race(with_retry: bool) {
14254        let mut db = AccountsDb::new_single_for_tests();
14255        let epoch_schedule = EpochSchedule::default();
14256        db.load_delay = RACY_SLEEP_MS;
14257        let db = Arc::new(db);
14258        let pubkey = Arc::new(Pubkey::new_unique());
14259        let exit = Arc::new(AtomicBool::new(false));
14260        let slot = 1;
14261
14262        // Store an account
14263        let lamports = 42;
14264        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14265        account.set_lamports(lamports);
14266        db.store_uncached(slot, &[(&pubkey, &account)]);
14267
14268        // Set the slot as a root so account loads will see the contents of this slot
14269        db.add_root(slot);
14270
14271        let t_shrink_accounts = {
14272            let db = db.clone();
14273            let exit = exit.clone();
14274
14275            std::thread::Builder::new()
14276                .name("account-shrink".to_string())
14277                .spawn(move || loop {
14278                    if exit.load(Ordering::Relaxed) {
14279                        return;
14280                    }
14281                    // Simulate adding shrink candidates from clean_accounts()
14282                    db.shrink_candidate_slots.lock().unwrap().insert(slot);
14283                    db.shrink_candidate_slots(&epoch_schedule);
14284                })
14285                .unwrap()
14286        };
14287
14288        let t_do_load = start_load_thread(
14289            with_retry,
14290            Ancestors::default(),
14291            db,
14292            exit.clone(),
14293            pubkey,
14294            move |_| lamports,
14295        );
14296
14297        sleep(Duration::from_secs(RACE_TIME));
14298        exit.store(true, Ordering::Relaxed);
14299        t_shrink_accounts.join().unwrap();
14300        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
14301    }
14302
14303    #[test]
14304    fn test_load_account_and_shrink_race_with_retry() {
14305        do_test_load_account_and_shrink_race(true);
14306    }
14307
14308    #[test]
14309    fn test_load_account_and_shrink_race_without_retry() {
14310        do_test_load_account_and_shrink_race(false);
14311    }
14312
14313    #[test]
14314    fn test_cache_flush_delayed_remove_unrooted_race() {
14315        let mut db = AccountsDb::new_single_for_tests();
14316        db.load_delay = RACY_SLEEP_MS;
14317        let db = Arc::new(db);
14318        let slot = 10;
14319        let bank_id = 10;
14320
14321        let lamports = 42;
14322        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14323        account.set_lamports(lamports);
14324
14325        // Start up a thread to flush the accounts cache
14326        let (flush_trial_start_sender, flush_trial_start_receiver) = unbounded();
14327        let (flush_done_sender, flush_done_receiver) = unbounded();
14328        let t_flush_cache = {
14329            let db = db.clone();
14330            std::thread::Builder::new()
14331                .name("account-cache-flush".to_string())
14332                .spawn(move || loop {
14333                    // Wait for the signal to start a trial
14334                    if flush_trial_start_receiver.recv().is_err() {
14335                        return;
14336                    }
14337                    db.flush_slot_cache(10);
14338                    flush_done_sender.send(()).unwrap();
14339                })
14340                .unwrap()
14341        };
14342
14343        // Start up a thread remove the slot
14344        let (remove_trial_start_sender, remove_trial_start_receiver) = unbounded();
14345        let (remove_done_sender, remove_done_receiver) = unbounded();
14346        let t_remove = {
14347            let db = db.clone();
14348            std::thread::Builder::new()
14349                .name("account-remove".to_string())
14350                .spawn(move || loop {
14351                    // Wait for the signal to start a trial
14352                    if remove_trial_start_receiver.recv().is_err() {
14353                        return;
14354                    }
14355                    db.remove_unrooted_slots(&[(slot, bank_id)]);
14356                    remove_done_sender.send(()).unwrap();
14357                })
14358                .unwrap()
14359        };
14360
14361        let num_trials = 10;
14362        for _ in 0..num_trials {
14363            let pubkey = Pubkey::new_unique();
14364            db.store_cached((slot, &[(&pubkey, &account)][..]), None);
14365            // Wait for both threads to finish
14366            flush_trial_start_sender.send(()).unwrap();
14367            remove_trial_start_sender.send(()).unwrap();
14368            let _ = flush_done_receiver.recv();
14369            let _ = remove_done_receiver.recv();
14370        }
14371
14372        drop(flush_trial_start_sender);
14373        drop(remove_trial_start_sender);
14374        t_flush_cache.join().unwrap();
14375        t_remove.join().unwrap();
14376    }
14377
14378    #[test]
14379    fn test_cache_flush_remove_unrooted_race_multiple_slots() {
14380        let db = AccountsDb::new_single_for_tests();
14381        let db = Arc::new(db);
14382        let num_cached_slots = 100;
14383
14384        let num_trials = 100;
14385        let (new_trial_start_sender, new_trial_start_receiver) = unbounded();
14386        let (flush_done_sender, flush_done_receiver) = unbounded();
14387        // Start up a thread to flush the accounts cache
14388        let t_flush_cache = {
14389            let db = db.clone();
14390
14391            std::thread::Builder::new()
14392                .name("account-cache-flush".to_string())
14393                .spawn(move || loop {
14394                    // Wait for the signal to start a trial
14395                    if new_trial_start_receiver.recv().is_err() {
14396                        return;
14397                    }
14398                    for slot in 0..num_cached_slots {
14399                        db.flush_slot_cache(slot);
14400                    }
14401                    flush_done_sender.send(()).unwrap();
14402                })
14403                .unwrap()
14404        };
14405
14406        let exit = Arc::new(AtomicBool::new(false));
14407
14408        let t_spurious_signal = {
14409            let db = db.clone();
14410            let exit = exit.clone();
14411            std::thread::Builder::new()
14412                .name("account-cache-flush".to_string())
14413                .spawn(move || loop {
14414                    if exit.load(Ordering::Relaxed) {
14415                        return;
14416                    }
14417                    // Simulate spurious wake-up that can happen, but is too rare to
14418                    // otherwise depend on in tests.
14419                    db.remove_unrooted_slots_synchronization.signal.notify_all();
14420                })
14421                .unwrap()
14422        };
14423
14424        // Run multiple trials. Has the added benefit of rewriting the same slots after we've
14425        // dumped them in previous trials.
14426        for _ in 0..num_trials {
14427            // Store an account
14428            let lamports = 42;
14429            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14430            account.set_lamports(lamports);
14431
14432            // Pick random 50% of the slots to pass to `remove_unrooted_slots()`
14433            let mut all_slots: Vec<(Slot, BankId)> = (0..num_cached_slots)
14434                .map(|slot| {
14435                    let bank_id = slot + 1;
14436                    (slot, bank_id)
14437                })
14438                .collect();
14439            all_slots.shuffle(&mut rand::thread_rng());
14440            let slots_to_dump = &all_slots[0..num_cached_slots as usize / 2];
14441            let slots_to_keep = &all_slots[num_cached_slots as usize / 2..];
14442
14443            // Set up a one account per slot across many different slots, track which
14444            // pubkey was stored in each slot.
14445            let slot_to_pubkey_map: HashMap<Slot, Pubkey> = (0..num_cached_slots)
14446                .map(|slot| {
14447                    let pubkey = Pubkey::new_unique();
14448                    db.store_cached((slot, &[(&pubkey, &account)][..]), None);
14449                    (slot, pubkey)
14450                })
14451                .collect();
14452
14453            // Signal the flushing shred to start flushing
14454            new_trial_start_sender.send(()).unwrap();
14455
14456            // Here we want to test both:
14457            // 1) Flush thread starts flushing a slot before we try dumping it.
14458            // 2) Flushing thread trying to flush while/after we're trying to dump the slot,
14459            // in which case flush should ignore/move past the slot to be dumped
14460            //
14461            // Hence, we split into chunks to get the dumping of each chunk to race with the
14462            // flushes. If we were to dump the entire chunk at once, then this reduces the possibility
14463            // of the flush occurring first since the dumping logic reserves all the slots it's about
14464            // to dump immediately.
14465
14466            for chunks in slots_to_dump.chunks(slots_to_dump.len() / 2) {
14467                db.remove_unrooted_slots(chunks);
14468            }
14469
14470            // Check that all the slots in `slots_to_dump` were completely removed from the
14471            // cache, storage, and index
14472
14473            for (slot, _) in slots_to_dump {
14474                assert_no_storages_at_slot(&db, *slot);
14475                assert!(db.accounts_cache.slot_cache(*slot).is_none());
14476                let account_in_slot = slot_to_pubkey_map[slot];
14477                assert!(!db.accounts_index.contains(&account_in_slot));
14478            }
14479
14480            // Wait for flush to finish before starting next trial
14481
14482            flush_done_receiver.recv().unwrap();
14483
14484            for (slot, bank_id) in slots_to_keep {
14485                let account_in_slot = slot_to_pubkey_map[slot];
14486                assert!(db
14487                    .load(
14488                        &Ancestors::from(vec![(*slot, 0)]),
14489                        &account_in_slot,
14490                        LoadHint::FixedMaxRoot
14491                    )
14492                    .is_some());
14493                // Clear for next iteration so that `assert!(self.storage.get_slot_storage_entry(purged_slot).is_none());`
14494                // in `purge_slot_pubkeys()` doesn't trigger
14495                db.remove_unrooted_slots(&[(*slot, *bank_id)]);
14496            }
14497        }
14498
14499        exit.store(true, Ordering::Relaxed);
14500        drop(new_trial_start_sender);
14501        t_flush_cache.join().unwrap();
14502
14503        t_spurious_signal.join().unwrap();
14504    }
14505
14506    #[test]
14507    fn test_collect_uncleaned_slots_up_to_slot() {
14508        solana_logger::setup();
14509        let db = AccountsDb::new_single_for_tests();
14510
14511        let slot1 = 11;
14512        let slot2 = 222;
14513        let slot3 = 3333;
14514
14515        let pubkey1 = Pubkey::new_unique();
14516        let pubkey2 = Pubkey::new_unique();
14517        let pubkey3 = Pubkey::new_unique();
14518
14519        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
14520        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
14521        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
14522
14523        let mut uncleaned_slots1 = db.collect_uncleaned_slots_up_to_slot(slot1);
14524        let mut uncleaned_slots2 = db.collect_uncleaned_slots_up_to_slot(slot2);
14525        let mut uncleaned_slots3 = db.collect_uncleaned_slots_up_to_slot(slot3);
14526
14527        uncleaned_slots1.sort_unstable();
14528        uncleaned_slots2.sort_unstable();
14529        uncleaned_slots3.sort_unstable();
14530
14531        assert_eq!(uncleaned_slots1, [slot1]);
14532        assert_eq!(uncleaned_slots2, [slot1, slot2]);
14533        assert_eq!(uncleaned_slots3, [slot1, slot2, slot3]);
14534    }
14535
14536    #[test]
14537    fn test_remove_uncleaned_slots_and_collect_pubkeys_up_to_slot() {
14538        solana_logger::setup();
14539        let db = AccountsDb::new_single_for_tests();
14540
14541        let slot1 = 11;
14542        let slot2 = 222;
14543        let slot3 = 3333;
14544
14545        let pubkey1 = Pubkey::new_unique();
14546        let pubkey2 = Pubkey::new_unique();
14547        let pubkey3 = Pubkey::new_unique();
14548
14549        let account1 = AccountSharedData::new(0, 0, &pubkey1);
14550        let account2 = AccountSharedData::new(0, 0, &pubkey2);
14551        let account3 = AccountSharedData::new(0, 0, &pubkey3);
14552
14553        db.store_for_tests(slot1, &[(&pubkey1, &account1)]);
14554        db.store_for_tests(slot2, &[(&pubkey2, &account2)]);
14555        db.store_for_tests(slot3, &[(&pubkey3, &account3)]);
14556
14557        // slot 1 is _not_ a root on purpose
14558        db.add_root(slot2);
14559        db.add_root(slot3);
14560
14561        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
14562        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
14563        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
14564
14565        let num_bins = db.accounts_index.bins();
14566        let candidates: Box<_> =
14567            std::iter::repeat_with(|| RwLock::new(HashMap::<Pubkey, CleaningInfo>::new()))
14568                .take(num_bins)
14569                .collect();
14570        db.remove_uncleaned_slots_up_to_slot_and_move_pubkeys(slot3, &candidates);
14571
14572        let candidates_contain = |pubkey: &Pubkey| {
14573            candidates
14574                .iter()
14575                .any(|bin| bin.read().unwrap().contains(pubkey))
14576        };
14577        assert!(candidates_contain(&pubkey1));
14578        assert!(candidates_contain(&pubkey2));
14579        assert!(candidates_contain(&pubkey3));
14580    }
14581
14582    #[test]
14583    fn test_shrink_productive() {
14584        solana_logger::setup();
14585        let path = Path::new("");
14586        let file_size = 100;
14587        let slot = 11;
14588
14589        let store = Arc::new(AccountStorageEntry::new(
14590            path,
14591            slot,
14592            slot as AccountsFileId,
14593            file_size,
14594            AccountsFileProvider::AppendVec,
14595        ));
14596        store.add_account(file_size as usize);
14597        assert!(!AccountsDb::is_shrinking_productive(&store));
14598
14599        let store = Arc::new(AccountStorageEntry::new(
14600            path,
14601            slot,
14602            slot as AccountsFileId,
14603            file_size,
14604            AccountsFileProvider::AppendVec,
14605        ));
14606        store.add_account(file_size as usize / 2);
14607        store.add_account(file_size as usize / 4);
14608        store.remove_accounts(file_size as usize / 4, false, 1);
14609        assert!(AccountsDb::is_shrinking_productive(&store));
14610
14611        store.add_account(file_size as usize / 2);
14612        assert!(!AccountsDb::is_shrinking_productive(&store));
14613    }
14614
14615    #[test]
14616    fn test_is_candidate_for_shrink() {
14617        solana_logger::setup();
14618
14619        let mut accounts = AccountsDb::new_single_for_tests();
14620        let common_store_path = Path::new("");
14621        let store_file_size = 100_000;
14622        let entry = Arc::new(AccountStorageEntry::new(
14623            common_store_path,
14624            0,
14625            1,
14626            store_file_size,
14627            AccountsFileProvider::AppendVec,
14628        ));
14629        match accounts.shrink_ratio {
14630            AccountShrinkThreshold::TotalSpace { shrink_ratio } => {
14631                assert_eq!(
14632                    (DEFAULT_ACCOUNTS_SHRINK_RATIO * 100.) as u64,
14633                    (shrink_ratio * 100.) as u64
14634                )
14635            }
14636            AccountShrinkThreshold::IndividualStore { shrink_ratio: _ } => {
14637                panic!("Expect the default to be TotalSpace")
14638            }
14639        }
14640
14641        entry
14642            .alive_bytes
14643            .store(store_file_size as usize - 1, Ordering::Release);
14644        assert!(accounts.is_candidate_for_shrink(&entry));
14645        entry
14646            .alive_bytes
14647            .store(store_file_size as usize, Ordering::Release);
14648        assert!(!accounts.is_candidate_for_shrink(&entry));
14649
14650        let shrink_ratio = 0.3;
14651        let file_size_shrink_limit = (store_file_size as f64 * shrink_ratio) as usize;
14652        entry
14653            .alive_bytes
14654            .store(file_size_shrink_limit + 1, Ordering::Release);
14655        accounts.shrink_ratio = AccountShrinkThreshold::TotalSpace { shrink_ratio };
14656        assert!(accounts.is_candidate_for_shrink(&entry));
14657        accounts.shrink_ratio = AccountShrinkThreshold::IndividualStore { shrink_ratio };
14658        assert!(!accounts.is_candidate_for_shrink(&entry));
14659    }
14660
14661    define_accounts_db_test!(test_calculate_storage_count_and_alive_bytes, |accounts| {
14662        accounts.accounts_index.set_startup(Startup::Startup);
14663        let shared_key = solana_sdk::pubkey::new_rand();
14664        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14665        let slot0 = 0;
14666
14667        accounts.accounts_index.set_startup(Startup::Startup);
14668
14669        let storage = accounts.create_and_insert_store(slot0, 4_000, "flush_slot_cache");
14670        storage
14671            .accounts
14672            .append_accounts(&(slot0, &[(&shared_key, &account)][..]), 0);
14673
14674        let storage = accounts.storage.get_slot_storage_entry(slot0).unwrap();
14675        let storage_info = StorageSizeAndCountMap::default();
14676        accounts.generate_index_for_slot(
14677            &storage,
14678            slot0,
14679            0,
14680            &RentCollector::default(),
14681            &storage_info,
14682        );
14683        assert_eq!(storage_info.len(), 1);
14684        for entry in storage_info.iter() {
14685            let expected_stored_size =
14686                if accounts.accounts_file_provider == AccountsFileProvider::HotStorage {
14687                    33
14688                } else {
14689                    144
14690                };
14691            assert_eq!(
14692                (entry.key(), entry.value().count, entry.value().stored_size),
14693                (&0, 1, expected_stored_size)
14694            );
14695        }
14696        accounts.accounts_index.set_startup(Startup::Normal);
14697    });
14698
14699    define_accounts_db_test!(
14700        test_calculate_storage_count_and_alive_bytes_0_accounts,
14701        |accounts| {
14702            // empty store
14703            let storage = accounts.create_and_insert_store(0, 1, "test");
14704            let storage_info = StorageSizeAndCountMap::default();
14705            accounts.generate_index_for_slot(
14706                &storage,
14707                0,
14708                0,
14709                &RentCollector::default(),
14710                &storage_info,
14711            );
14712            assert!(storage_info.is_empty());
14713        }
14714    );
14715
14716    define_accounts_db_test!(
14717        test_calculate_storage_count_and_alive_bytes_2_accounts,
14718        |accounts| {
14719            let keys = [
14720                solana_sdk::pubkey::Pubkey::from([0; 32]),
14721                solana_sdk::pubkey::Pubkey::from([255; 32]),
14722            ];
14723            accounts.accounts_index.set_startup(Startup::Startup);
14724
14725            // make sure accounts are in 2 different bins
14726            assert!(
14727                (accounts.accounts_index.bins() == 1)
14728                    ^ (accounts
14729                        .accounts_index
14730                        .bin_calculator
14731                        .bin_from_pubkey(&keys[0])
14732                        != accounts
14733                            .accounts_index
14734                            .bin_calculator
14735                            .bin_from_pubkey(&keys[1]))
14736            );
14737            let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14738            let account_big = AccountSharedData::new(1, 1000, AccountSharedData::default().owner());
14739            let slot0 = 0;
14740            let storage = accounts.create_and_insert_store(slot0, 4_000, "flush_slot_cache");
14741            storage.accounts.append_accounts(
14742                &(slot0, &[(&keys[0], &account), (&keys[1], &account_big)][..]),
14743                0,
14744            );
14745
14746            let storage_info = StorageSizeAndCountMap::default();
14747            accounts.generate_index_for_slot(
14748                &storage,
14749                0,
14750                0,
14751                &RentCollector::default(),
14752                &storage_info,
14753            );
14754            assert_eq!(storage_info.len(), 1);
14755            for entry in storage_info.iter() {
14756                let expected_stored_size =
14757                    if accounts.accounts_file_provider == AccountsFileProvider::HotStorage {
14758                        1065
14759                    } else {
14760                        1280
14761                    };
14762                assert_eq!(
14763                    (entry.key(), entry.value().count, entry.value().stored_size),
14764                    (&0, 2, expected_stored_size)
14765                );
14766            }
14767            accounts.accounts_index.set_startup(Startup::Normal);
14768        }
14769    );
14770
14771    define_accounts_db_test!(test_set_storage_count_and_alive_bytes, |accounts| {
14772        // make sure we have storage 0
14773        let shared_key = solana_sdk::pubkey::new_rand();
14774        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14775        let slot0 = 0;
14776        accounts.store_for_tests(slot0, &[(&shared_key, &account)]);
14777        accounts.add_root_and_flush_write_cache(slot0);
14778
14779        // fake out the store count to avoid the assert
14780        for (_, store) in accounts.storage.iter() {
14781            store.alive_bytes.store(0, Ordering::Release);
14782            let mut count_and_status = store.count_and_status.lock_write();
14783            count_and_status.0 = 0;
14784        }
14785
14786        // count needs to be <= approx stored count in store.
14787        // approx stored count is 1 in store since we added a single account.
14788        let count = 1;
14789
14790        // populate based on made up hash data
14791        let dashmap = DashMap::default();
14792        dashmap.insert(
14793            0,
14794            StorageSizeAndCount {
14795                stored_size: 2,
14796                count,
14797            },
14798        );
14799
14800        for (_, store) in accounts.storage.iter() {
14801            assert_eq!(store.count_and_status.read().0, 0);
14802            assert_eq!(store.alive_bytes(), 0);
14803        }
14804        accounts.set_storage_count_and_alive_bytes(dashmap, &mut GenerateIndexTimings::default());
14805        assert_eq!(accounts.storage.len(), 1);
14806        for (_, store) in accounts.storage.iter() {
14807            assert_eq!(store.id(), 0);
14808            assert_eq!(store.count_and_status.read().0, count);
14809            assert_eq!(store.alive_bytes(), 2);
14810        }
14811    });
14812
14813    define_accounts_db_test!(test_purge_alive_unrooted_slots_after_clean, |accounts| {
14814        // Key shared between rooted and nonrooted slot
14815        let shared_key = solana_sdk::pubkey::new_rand();
14816        // Key to keep the storage entry for the unrooted slot alive
14817        let unrooted_key = solana_sdk::pubkey::new_rand();
14818        let slot0 = 0;
14819        let slot1 = 1;
14820
14821        // Store accounts with greater than 0 lamports
14822        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
14823        accounts.store_for_tests(slot0, &[(&shared_key, &account)]);
14824        accounts.store_for_tests(slot0, &[(&unrooted_key, &account)]);
14825
14826        // Simulate adding dirty pubkeys on bank freeze. Note this is
14827        // not a rooted slot
14828        accounts.calculate_accounts_delta_hash(slot0);
14829
14830        // On the next *rooted* slot, update the `shared_key` account to zero lamports
14831        let zero_lamport_account =
14832            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
14833        accounts.store_for_tests(slot1, &[(&shared_key, &zero_lamport_account)]);
14834
14835        // Simulate adding dirty pubkeys on bank freeze, set root
14836        accounts.calculate_accounts_delta_hash(slot1);
14837        accounts.add_root_and_flush_write_cache(slot1);
14838
14839        // The later rooted zero-lamport update to `shared_key` cannot be cleaned
14840        // because it is kept alive by the unrooted slot.
14841        accounts.clean_accounts_for_tests();
14842        assert!(accounts.accounts_index.contains(&shared_key));
14843
14844        // Simulate purge_slot() all from AccountsBackgroundService
14845        accounts.purge_slot(slot0, 0, true);
14846
14847        // Now clean should clean up the remaining key
14848        accounts.clean_accounts_for_tests();
14849        assert!(!accounts.accounts_index.contains(&shared_key));
14850        assert_no_storages_at_slot(&accounts, slot0);
14851    });
14852
14853    /// asserts that not only are there 0 append vecs, but there is not even an entry in the storage map for 'slot'
14854    fn assert_no_storages_at_slot(db: &AccountsDb, slot: Slot) {
14855        assert!(db.storage.get_slot_storage_entry(slot).is_none());
14856    }
14857
14858    // Test to make sure `clean_accounts()` works properly with `latest_full_snapshot_slot`
14859    //
14860    // Basically:
14861    //
14862    // - slot 1: set Account1's balance to non-zero
14863    // - slot 2: set Account1's balance to a different non-zero amount
14864    // - slot 3: set Account1's balance to zero
14865    // - call `clean_accounts()` with `max_clean_root` set to 2
14866    //     - ensure Account1 has *not* been purged
14867    //     - ensure the store from slot 1 is cleaned up
14868    // - call `clean_accounts()` with `latest_full_snapshot_slot` set to 2
14869    //     - ensure Account1 has *not* been purged
14870    // - call `clean_accounts()` with `latest_full_snapshot_slot` set to 3
14871    //     - ensure Account1 *has* been purged
14872    define_accounts_db_test!(
14873        test_clean_accounts_with_latest_full_snapshot_slot,
14874        |accounts_db| {
14875            let pubkey = solana_sdk::pubkey::new_rand();
14876            let owner = solana_sdk::pubkey::new_rand();
14877            let space = 0;
14878
14879            let slot1: Slot = 1;
14880            let account = AccountSharedData::new(111, space, &owner);
14881            accounts_db.store_cached((slot1, &[(&pubkey, &account)][..]), None);
14882            accounts_db.calculate_accounts_delta_hash(slot1);
14883            accounts_db.add_root_and_flush_write_cache(slot1);
14884
14885            let slot2: Slot = 2;
14886            let account = AccountSharedData::new(222, space, &owner);
14887            accounts_db.store_cached((slot2, &[(&pubkey, &account)][..]), None);
14888            accounts_db.calculate_accounts_delta_hash(slot2);
14889            accounts_db.add_root_and_flush_write_cache(slot2);
14890
14891            let slot3: Slot = 3;
14892            let account = AccountSharedData::new(0, space, &owner);
14893            accounts_db.store_cached((slot3, &[(&pubkey, &account)][..]), None);
14894            accounts_db.calculate_accounts_delta_hash(slot3);
14895            accounts_db.add_root_and_flush_write_cache(slot3);
14896
14897            assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 3);
14898
14899            accounts_db.set_latest_full_snapshot_slot(slot2);
14900            accounts_db.clean_accounts(
14901                Some(slot2),
14902                false,
14903                &EpochSchedule::default(),
14904                OldStoragesPolicy::Leave,
14905            );
14906            assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 2);
14907
14908            accounts_db.set_latest_full_snapshot_slot(slot2);
14909            accounts_db.clean_accounts(
14910                None,
14911                false,
14912                &EpochSchedule::default(),
14913                OldStoragesPolicy::Leave,
14914            );
14915            assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 1);
14916
14917            accounts_db.set_latest_full_snapshot_slot(slot3);
14918            accounts_db.clean_accounts(
14919                None,
14920                false,
14921                &EpochSchedule::default(),
14922                OldStoragesPolicy::Leave,
14923            );
14924            assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 0);
14925        }
14926    );
14927
14928    #[test]
14929    fn test_filter_zero_lamport_clean_for_incremental_snapshots() {
14930        solana_logger::setup();
14931        let slot = 10;
14932
14933        struct TestParameters {
14934            latest_full_snapshot_slot: Option<Slot>,
14935            max_clean_root: Option<Slot>,
14936            should_contain: bool,
14937        }
14938
14939        let do_test = |test_params: TestParameters| {
14940            let account_info = AccountInfo::new(StorageLocation::AppendVec(42, 128), 0);
14941            let pubkey = solana_sdk::pubkey::new_rand();
14942            let mut key_set = HashSet::default();
14943            key_set.insert(pubkey);
14944            let store_count = 0;
14945            let mut store_counts = HashMap::default();
14946            store_counts.insert(slot, (store_count, key_set));
14947            let candidates = [RwLock::new(HashMap::new())];
14948            candidates[0].write().unwrap().insert(
14949                pubkey,
14950                CleaningInfo {
14951                    slot_list: vec![(slot, account_info)],
14952                    ref_count: 1,
14953                    ..Default::default()
14954                },
14955            );
14956            let accounts_db = AccountsDb::new_single_for_tests();
14957            if let Some(latest_full_snapshot_slot) = test_params.latest_full_snapshot_slot {
14958                accounts_db.set_latest_full_snapshot_slot(latest_full_snapshot_slot);
14959            }
14960            accounts_db.filter_zero_lamport_clean_for_incremental_snapshots(
14961                test_params.max_clean_root,
14962                &store_counts,
14963                &candidates,
14964            );
14965
14966            assert_eq!(
14967                candidates[0].read().unwrap().contains_key(&pubkey),
14968                test_params.should_contain
14969            );
14970        };
14971
14972        // Scenario 1: last full snapshot is NONE
14973        // In this scenario incremental snapshots are OFF, so always purge
14974        {
14975            let latest_full_snapshot_slot = None;
14976
14977            do_test(TestParameters {
14978                latest_full_snapshot_slot,
14979                max_clean_root: Some(slot),
14980                should_contain: true,
14981            });
14982
14983            do_test(TestParameters {
14984                latest_full_snapshot_slot,
14985                max_clean_root: None,
14986                should_contain: true,
14987            });
14988        }
14989
14990        // Scenario 2: last full snapshot is GREATER THAN zero lamport account slot
14991        // In this scenario always purge, and just test the various permutations of
14992        // `should_filter_for_incremental_snapshots` based on `max_clean_root`.
14993        {
14994            let latest_full_snapshot_slot = Some(slot + 1);
14995
14996            do_test(TestParameters {
14997                latest_full_snapshot_slot,
14998                max_clean_root: latest_full_snapshot_slot,
14999                should_contain: true,
15000            });
15001
15002            do_test(TestParameters {
15003                latest_full_snapshot_slot,
15004                max_clean_root: latest_full_snapshot_slot.map(|s| s + 1),
15005                should_contain: true,
15006            });
15007
15008            do_test(TestParameters {
15009                latest_full_snapshot_slot,
15010                max_clean_root: None,
15011                should_contain: true,
15012            });
15013        }
15014
15015        // Scenario 3: last full snapshot is EQUAL TO zero lamport account slot
15016        // In this scenario always purge, as it's the same as Scenario 2.
15017        {
15018            let latest_full_snapshot_slot = Some(slot);
15019
15020            do_test(TestParameters {
15021                latest_full_snapshot_slot,
15022                max_clean_root: latest_full_snapshot_slot,
15023                should_contain: true,
15024            });
15025
15026            do_test(TestParameters {
15027                latest_full_snapshot_slot,
15028                max_clean_root: latest_full_snapshot_slot.map(|s| s + 1),
15029                should_contain: true,
15030            });
15031
15032            do_test(TestParameters {
15033                latest_full_snapshot_slot,
15034                max_clean_root: None,
15035                should_contain: true,
15036            });
15037        }
15038
15039        // Scenario 4: last full snapshot is LESS THAN zero lamport account slot
15040        // In this scenario do *not* purge, except when `should_filter_for_incremental_snapshots`
15041        // is false
15042        {
15043            let latest_full_snapshot_slot = Some(slot - 1);
15044
15045            do_test(TestParameters {
15046                latest_full_snapshot_slot,
15047                max_clean_root: latest_full_snapshot_slot,
15048                should_contain: true,
15049            });
15050
15051            do_test(TestParameters {
15052                latest_full_snapshot_slot,
15053                max_clean_root: latest_full_snapshot_slot.map(|s| s + 1),
15054                should_contain: false,
15055            });
15056
15057            do_test(TestParameters {
15058                latest_full_snapshot_slot,
15059                max_clean_root: None,
15060                should_contain: false,
15061            });
15062        }
15063    }
15064
15065    impl AccountsDb {
15066        /// helper function to test unref_accounts or clean_dead_slots_from_accounts_index
15067        fn test_unref(
15068            &self,
15069            call_unref: bool,
15070            purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
15071            purged_stored_account_slots: &mut AccountSlots,
15072            pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
15073        ) {
15074            if call_unref {
15075                self.unref_accounts(
15076                    purged_slot_pubkeys,
15077                    purged_stored_account_slots,
15078                    pubkeys_removed_from_accounts_index,
15079                );
15080            } else {
15081                let empty_vec = Vec::default();
15082                self.clean_dead_slots_from_accounts_index(
15083                    empty_vec.iter(),
15084                    purged_slot_pubkeys,
15085                    Some(purged_stored_account_slots),
15086                    pubkeys_removed_from_accounts_index,
15087                );
15088            }
15089        }
15090    }
15091
15092    #[test]
15093    /// test 'unref' parameter 'pubkeys_removed_from_accounts_index'
15094    fn test_unref_pubkeys_removed_from_accounts_index() {
15095        let slot1 = 1;
15096        let pk1 = Pubkey::from([1; 32]);
15097        for already_removed in [false, true] {
15098            let mut pubkeys_removed_from_accounts_index =
15099                PubkeysRemovedFromAccountsIndex::default();
15100            if already_removed {
15101                pubkeys_removed_from_accounts_index.insert(pk1);
15102            }
15103            // pk1 in slot1, purge it
15104            let db = AccountsDb::new_single_for_tests();
15105            let mut purged_slot_pubkeys = HashSet::default();
15106            purged_slot_pubkeys.insert((slot1, pk1));
15107            let mut reclaims = SlotList::default();
15108            db.accounts_index.upsert(
15109                slot1,
15110                slot1,
15111                &pk1,
15112                &AccountSharedData::default(),
15113                &AccountSecondaryIndexes::default(),
15114                AccountInfo::default(),
15115                &mut reclaims,
15116                UpsertReclaim::IgnoreReclaims,
15117            );
15118
15119            let mut purged_stored_account_slots = AccountSlots::default();
15120            db.test_unref(
15121                true,
15122                purged_slot_pubkeys,
15123                &mut purged_stored_account_slots,
15124                &pubkeys_removed_from_accounts_index,
15125            );
15126            assert_eq!(
15127                vec![(pk1, vec![slot1].into_iter().collect::<IntSet<_>>())],
15128                purged_stored_account_slots.into_iter().collect::<Vec<_>>()
15129            );
15130            let expected = u64::from(already_removed);
15131            assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), expected);
15132        }
15133    }
15134
15135    #[test]
15136    fn test_unref_accounts() {
15137        let pubkeys_removed_from_accounts_index = PubkeysRemovedFromAccountsIndex::default();
15138        for call_unref in [false, true] {
15139            {
15140                let db = AccountsDb::new_single_for_tests();
15141                let mut purged_stored_account_slots = AccountSlots::default();
15142
15143                db.test_unref(
15144                    call_unref,
15145                    HashSet::default(),
15146                    &mut purged_stored_account_slots,
15147                    &pubkeys_removed_from_accounts_index,
15148                );
15149                assert!(purged_stored_account_slots.is_empty());
15150            }
15151
15152            let slot1 = 1;
15153            let slot2 = 2;
15154            let pk1 = Pubkey::from([1; 32]);
15155            let pk2 = Pubkey::from([2; 32]);
15156            {
15157                // pk1 in slot1, purge it
15158                let db = AccountsDb::new_single_for_tests();
15159                let mut purged_slot_pubkeys = HashSet::default();
15160                purged_slot_pubkeys.insert((slot1, pk1));
15161                let mut reclaims = SlotList::default();
15162                db.accounts_index.upsert(
15163                    slot1,
15164                    slot1,
15165                    &pk1,
15166                    &AccountSharedData::default(),
15167                    &AccountSecondaryIndexes::default(),
15168                    AccountInfo::default(),
15169                    &mut reclaims,
15170                    UpsertReclaim::IgnoreReclaims,
15171                );
15172
15173                let mut purged_stored_account_slots = AccountSlots::default();
15174                db.test_unref(
15175                    call_unref,
15176                    purged_slot_pubkeys,
15177                    &mut purged_stored_account_slots,
15178                    &pubkeys_removed_from_accounts_index,
15179                );
15180                assert_eq!(
15181                    vec![(pk1, vec![slot1].into_iter().collect::<IntSet<_>>())],
15182                    purged_stored_account_slots.into_iter().collect::<Vec<_>>()
15183                );
15184                assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), 0);
15185            }
15186            {
15187                let db = AccountsDb::new_single_for_tests();
15188                let mut purged_stored_account_slots = AccountSlots::default();
15189                let mut purged_slot_pubkeys = HashSet::default();
15190                let mut reclaims = SlotList::default();
15191                // pk1 and pk2 both in slot1 and slot2, so each has refcount of 2
15192                for slot in [slot1, slot2] {
15193                    for pk in [pk1, pk2] {
15194                        db.accounts_index.upsert(
15195                            slot,
15196                            slot,
15197                            &pk,
15198                            &AccountSharedData::default(),
15199                            &AccountSecondaryIndexes::default(),
15200                            AccountInfo::default(),
15201                            &mut reclaims,
15202                            UpsertReclaim::IgnoreReclaims,
15203                        );
15204                    }
15205                }
15206                // purge pk1 from both 1 and 2 and pk2 from slot 1
15207                let purges = vec![(slot1, pk1), (slot1, pk2), (slot2, pk1)];
15208                purges.into_iter().for_each(|(slot, pk)| {
15209                    purged_slot_pubkeys.insert((slot, pk));
15210                });
15211                db.test_unref(
15212                    call_unref,
15213                    purged_slot_pubkeys,
15214                    &mut purged_stored_account_slots,
15215                    &pubkeys_removed_from_accounts_index,
15216                );
15217                for (pk, slots) in [(pk1, vec![slot1, slot2]), (pk2, vec![slot1])] {
15218                    let result = purged_stored_account_slots.remove(&pk).unwrap();
15219                    assert_eq!(result, slots.into_iter().collect::<IntSet<_>>());
15220                }
15221                assert!(purged_stored_account_slots.is_empty());
15222                assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), 0);
15223                assert_eq!(db.accounts_index.ref_count_from_storage(&pk2), 1);
15224            }
15225        }
15226    }
15227
15228    define_accounts_db_test!(test_many_unrefs, |db| {
15229        let mut purged_stored_account_slots = AccountSlots::default();
15230        let mut reclaims = SlotList::default();
15231        let pk1 = Pubkey::from([1; 32]);
15232        // make sure we have > 1 batch. Bigger numbers cost more in test time here.
15233        let n = (UNREF_ACCOUNTS_BATCH_SIZE + 1) as Slot;
15234        // put the pubkey into the acct idx in 'n' slots
15235        let purged_slot_pubkeys = (0..n)
15236            .map(|slot| {
15237                db.accounts_index.upsert(
15238                    slot,
15239                    slot,
15240                    &pk1,
15241                    &AccountSharedData::default(),
15242                    &AccountSecondaryIndexes::default(),
15243                    AccountInfo::default(),
15244                    &mut reclaims,
15245                    UpsertReclaim::IgnoreReclaims,
15246                );
15247                (slot, pk1)
15248            })
15249            .collect::<HashSet<_>>();
15250
15251        assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), n);
15252        // unref all 'n' slots
15253        db.unref_accounts(
15254            purged_slot_pubkeys,
15255            &mut purged_stored_account_slots,
15256            &HashSet::default(),
15257        );
15258        assert_eq!(db.accounts_index.ref_count_from_storage(&pk1), 0);
15259    });
15260
15261    #[test_case(CreateAncientStorage::Append; "append")]
15262    #[test_case(CreateAncientStorage::Pack; "pack")]
15263    fn test_get_oldest_non_ancient_slot_for_hash_calc_scan(
15264        create_ancient_storage: CreateAncientStorage,
15265    ) {
15266        let expected = |v| {
15267            if create_ancient_storage == CreateAncientStorage::Append {
15268                Some(v)
15269            } else {
15270                None
15271            }
15272        };
15273
15274        let mut db = AccountsDb::new_single_for_tests();
15275        db.create_ancient_storage = create_ancient_storage;
15276
15277        let config = CalcAccountsHashConfig::default();
15278        let slot = config.epoch_schedule.slots_per_epoch;
15279        let slots_per_epoch = config.epoch_schedule.slots_per_epoch;
15280        assert_ne!(slot, 0);
15281        let offset = 10;
15282        assert_eq!(
15283            db.get_oldest_non_ancient_slot_for_hash_calc_scan(slots_per_epoch + offset, &config),
15284            expected(db.ancient_append_vec_offset.unwrap() as u64 + offset + 1)
15285        );
15286        // ancient append vecs enabled (but at 0 offset), so can be non-zero
15287        db.ancient_append_vec_offset = Some(0);
15288        // 0..=(slots_per_epoch - 1) are all non-ancient
15289        assert_eq!(
15290            db.get_oldest_non_ancient_slot_for_hash_calc_scan(slots_per_epoch - 1, &config),
15291            expected(0)
15292        );
15293        // 1..=slots_per_epoch are all non-ancient, so 1 is oldest non ancient
15294        assert_eq!(
15295            db.get_oldest_non_ancient_slot_for_hash_calc_scan(slots_per_epoch, &config),
15296            expected(1)
15297        );
15298        assert_eq!(
15299            db.get_oldest_non_ancient_slot_for_hash_calc_scan(slots_per_epoch + offset, &config),
15300            expected(offset + 1)
15301        );
15302    }
15303
15304    define_accounts_db_test!(test_mark_dirty_dead_stores_empty, |db| {
15305        let slot = 0;
15306        for add_dirty_stores in [false, true] {
15307            let dead_storages = db.mark_dirty_dead_stores(slot, add_dirty_stores, None, false);
15308            assert!(dead_storages.is_empty());
15309            assert!(db.dirty_stores.is_empty());
15310        }
15311    });
15312
15313    #[test]
15314    fn test_mark_dirty_dead_stores_no_shrink_in_progress() {
15315        // None for shrink_in_progress, 1 existing store at the slot
15316        // There should be no more append vecs at that slot after the call to mark_dirty_dead_stores.
15317        // This tests the case where this slot was combined into an ancient append vec from an older slot and
15318        // there is no longer an append vec at this slot.
15319        for add_dirty_stores in [false, true] {
15320            let slot = 0;
15321            let db = AccountsDb::new_single_for_tests();
15322            let size = 1;
15323            let existing_store = db.create_and_insert_store(slot, size, "test");
15324            let old_id = existing_store.id();
15325            let dead_storages = db.mark_dirty_dead_stores(slot, add_dirty_stores, None, false);
15326            assert!(db.storage.get_slot_storage_entry(slot).is_none());
15327            assert_eq!(dead_storages.len(), 1);
15328            assert_eq!(dead_storages.first().unwrap().id(), old_id);
15329            if add_dirty_stores {
15330                assert_eq!(1, db.dirty_stores.len());
15331                let dirty_store = db.dirty_stores.get(&slot).unwrap();
15332                assert_eq!(dirty_store.id(), old_id);
15333            } else {
15334                assert!(db.dirty_stores.is_empty());
15335            }
15336            assert!(db.storage.is_empty_entry(slot));
15337        }
15338    }
15339
15340    #[test]
15341    fn test_mark_dirty_dead_stores() {
15342        let slot = 0;
15343
15344        // use shrink_in_progress to cause us to drop the initial store
15345        for add_dirty_stores in [false, true] {
15346            let db = AccountsDb::new_single_for_tests();
15347            let size = 1;
15348            let old_store = db.create_and_insert_store(slot, size, "test");
15349            let old_id = old_store.id();
15350            let shrink_in_progress = db.get_store_for_shrink(slot, 100);
15351            let dead_storages =
15352                db.mark_dirty_dead_stores(slot, add_dirty_stores, Some(shrink_in_progress), false);
15353            assert!(db.storage.get_slot_storage_entry(slot).is_some());
15354            assert_eq!(dead_storages.len(), 1);
15355            assert_eq!(dead_storages.first().unwrap().id(), old_id);
15356            if add_dirty_stores {
15357                assert_eq!(1, db.dirty_stores.len());
15358                let dirty_store = db.dirty_stores.get(&slot).unwrap();
15359                assert_eq!(dirty_store.id(), old_id);
15360            } else {
15361                assert!(db.dirty_stores.is_empty());
15362            }
15363            assert!(db.storage.get_slot_storage_entry(slot).is_some());
15364        }
15365    }
15366
15367    #[test]
15368    fn test_split_storages_ancient_chunks() {
15369        let storages = SortedStorages::empty();
15370        assert_eq!(storages.max_slot_inclusive(), 0);
15371        let result = SplitAncientStorages::new(Some(0), &storages);
15372        assert_eq!(result, SplitAncientStorages::default());
15373    }
15374
15375    /// get all the ranges the splitter produces
15376    fn get_all_slot_ranges(splitter: &SplitAncientStorages) -> Vec<Option<Range<Slot>>> {
15377        (0..splitter.chunk_count)
15378            .map(|chunk| {
15379                assert_eq!(
15380                    splitter.get_starting_slot_from_normal_chunk(chunk),
15381                    if chunk == 0 {
15382                        splitter.normal_slot_range.start
15383                    } else {
15384                        (splitter.first_chunk_start + ((chunk as Slot) - 1) * MAX_ITEMS_PER_CHUNK)
15385                            .max(splitter.normal_slot_range.start)
15386                    },
15387                    "chunk: {chunk}, num_chunks: {}, splitter: {:?}",
15388                    splitter.chunk_count,
15389                    splitter,
15390                );
15391                splitter.get_slot_range(chunk)
15392            })
15393            .collect::<Vec<_>>()
15394    }
15395
15396    /// test function to make sure the split range covers exactly every slot in the original range
15397    fn verify_all_slots_covered_exactly_once(
15398        splitter: &SplitAncientStorages,
15399        overall_range: &Range<Slot>,
15400    ) {
15401        // verify all slots covered exactly once
15402        let result = get_all_slot_ranges(splitter);
15403        let mut expected = overall_range.start;
15404        result.iter().for_each(|range| {
15405            if let Some(range) = range {
15406                assert!(
15407                    overall_range.start == range.start || range.start % MAX_ITEMS_PER_CHUNK == 0
15408                );
15409                for slot in range.clone() {
15410                    assert_eq!(slot, expected);
15411                    expected += 1;
15412                }
15413            }
15414        });
15415        assert_eq!(expected, overall_range.end);
15416    }
15417
15418    /// new splitter for test
15419    /// without any ancient append vecs
15420    fn new_splitter(range: &Range<Slot>) -> SplitAncientStorages {
15421        let splitter =
15422            SplitAncientStorages::new_with_ancient_info(range, Vec::default(), range.start);
15423
15424        verify_all_slots_covered_exactly_once(&splitter, range);
15425
15426        splitter
15427    }
15428
15429    /// new splitter for test
15430    /// without any ancient append vecs
15431    fn new_splitter2(start: Slot, count: Slot) -> SplitAncientStorages {
15432        new_splitter(&Range {
15433            start,
15434            end: start + count,
15435        })
15436    }
15437
15438    #[test]
15439    fn test_split_storages_splitter_simple() {
15440        let plus_1 = MAX_ITEMS_PER_CHUNK + 1;
15441        let plus_2 = plus_1 + 1;
15442
15443        // starting at 0 is aligned with beginning, so 1st chunk is unnecessary since beginning slot starts at boundary
15444        // second chunk is the final chunk, which is not full (does not have 2500 entries)
15445        let splitter = new_splitter2(0, 1);
15446        let result = get_all_slot_ranges(&splitter);
15447        assert_eq!(result, [Some(0..1), None]);
15448
15449        // 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
15450        let splitter = new_splitter2(1, 1);
15451        let result = get_all_slot_ranges(&splitter);
15452        assert_eq!(result, [Some(1..2), None]);
15453
15454        // 1 full chunk, aligned
15455        let splitter = new_splitter2(0, MAX_ITEMS_PER_CHUNK);
15456        let result = get_all_slot_ranges(&splitter);
15457        assert_eq!(result, [Some(0..MAX_ITEMS_PER_CHUNK), None, None]);
15458
15459        // 1 full chunk + 1, aligned
15460        let splitter = new_splitter2(0, plus_1);
15461        let result = get_all_slot_ranges(&splitter);
15462        assert_eq!(
15463            result,
15464            [
15465                Some(0..MAX_ITEMS_PER_CHUNK),
15466                Some(MAX_ITEMS_PER_CHUNK..plus_1),
15467                None
15468            ]
15469        );
15470
15471        // 1 full chunk + 2, aligned
15472        let splitter = new_splitter2(0, plus_2);
15473        let result = get_all_slot_ranges(&splitter);
15474        assert_eq!(
15475            result,
15476            [
15477                Some(0..MAX_ITEMS_PER_CHUNK),
15478                Some(MAX_ITEMS_PER_CHUNK..plus_2),
15479                None
15480            ]
15481        );
15482
15483        // 1 full chunk, mis-aligned by 1
15484        let offset = 1;
15485        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK);
15486        let result = get_all_slot_ranges(&splitter);
15487        assert_eq!(
15488            result,
15489            [
15490                Some(offset..MAX_ITEMS_PER_CHUNK),
15491                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK + offset),
15492                None
15493            ]
15494        );
15495
15496        // starting at 1 is not aligned with beginning
15497        let offset = 1;
15498        let splitter = new_splitter2(offset, plus_1);
15499        let result = get_all_slot_ranges(&splitter);
15500        assert_eq!(
15501            result,
15502            [
15503                Some(offset..MAX_ITEMS_PER_CHUNK),
15504                Some(MAX_ITEMS_PER_CHUNK..plus_1 + offset),
15505                None
15506            ],
15507            "{splitter:?}"
15508        );
15509
15510        // 2 full chunks, aligned
15511        let offset = 0;
15512        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK * 2);
15513        let result = get_all_slot_ranges(&splitter);
15514        assert_eq!(
15515            result,
15516            [
15517                Some(offset..MAX_ITEMS_PER_CHUNK),
15518                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK * 2),
15519                None,
15520                None
15521            ],
15522            "{splitter:?}"
15523        );
15524
15525        // 2 full chunks + 1, mis-aligned
15526        let offset = 1;
15527        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK * 2);
15528        let result = get_all_slot_ranges(&splitter);
15529        assert_eq!(
15530            result,
15531            [
15532                Some(offset..MAX_ITEMS_PER_CHUNK),
15533                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK * 2),
15534                Some(MAX_ITEMS_PER_CHUNK * 2..MAX_ITEMS_PER_CHUNK * 2 + offset),
15535                None,
15536            ],
15537            "{splitter:?}"
15538        );
15539
15540        // 3 full chunks - 1, mis-aligned by 2
15541        // we need ALL the chunks here
15542        let offset = 2;
15543        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK * 3 - 1);
15544        let result = get_all_slot_ranges(&splitter);
15545        assert_eq!(
15546            result,
15547            [
15548                Some(offset..MAX_ITEMS_PER_CHUNK),
15549                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK * 2),
15550                Some(MAX_ITEMS_PER_CHUNK * 2..MAX_ITEMS_PER_CHUNK * 3),
15551                Some(MAX_ITEMS_PER_CHUNK * 3..MAX_ITEMS_PER_CHUNK * 3 + 1),
15552            ],
15553            "{splitter:?}"
15554        );
15555
15556        // 1 full chunk - 1, mis-aligned by 2
15557        // we need ALL the chunks here
15558        let offset = 2;
15559        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
15560        let result = get_all_slot_ranges(&splitter);
15561        assert_eq!(
15562            result,
15563            [
15564                Some(offset..MAX_ITEMS_PER_CHUNK),
15565                Some(MAX_ITEMS_PER_CHUNK..MAX_ITEMS_PER_CHUNK + 1),
15566            ],
15567            "{splitter:?}"
15568        );
15569
15570        // 1 full chunk - 1, aligned at big offset
15571        // huge offset
15572        // we need ALL the chunks here
15573        let offset = MAX_ITEMS_PER_CHUNK * 100;
15574        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
15575        let result = get_all_slot_ranges(&splitter);
15576        assert_eq!(
15577            result,
15578            [Some(offset..MAX_ITEMS_PER_CHUNK * 101 - 1), None,],
15579            "{splitter:?}"
15580        );
15581
15582        // 1 full chunk - 1, mis-aligned by 2 at big offset
15583        // huge offset
15584        // we need ALL the chunks here
15585        let offset = MAX_ITEMS_PER_CHUNK * 100 + 2;
15586        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
15587        let result = get_all_slot_ranges(&splitter);
15588        assert_eq!(
15589            result,
15590            [
15591                Some(offset..MAX_ITEMS_PER_CHUNK * 101),
15592                Some(MAX_ITEMS_PER_CHUNK * 101..MAX_ITEMS_PER_CHUNK * 101 + 1),
15593            ],
15594            "{splitter:?}"
15595        );
15596    }
15597
15598    #[test]
15599    fn test_split_storages_splitter_large_offset() {
15600        solana_logger::setup();
15601        // 1 full chunk - 1, mis-aligned by 2 at big offset
15602        // huge offset
15603        // we need ALL the chunks here
15604        let offset = MAX_ITEMS_PER_CHUNK * 100 + 2;
15605        let splitter = new_splitter2(offset, MAX_ITEMS_PER_CHUNK - 1);
15606        let result = get_all_slot_ranges(&splitter);
15607        assert_eq!(
15608            result,
15609            [
15610                Some(offset..MAX_ITEMS_PER_CHUNK * 101),
15611                Some(MAX_ITEMS_PER_CHUNK * 101..MAX_ITEMS_PER_CHUNK * 101 + 1),
15612            ],
15613            "{splitter:?}"
15614        );
15615    }
15616
15617    #[test]
15618    fn test_split_storages_parametric_splitter() {
15619        for offset_multiplier in [1, 1000] {
15620            for offset in [
15621                0,
15622                1,
15623                2,
15624                MAX_ITEMS_PER_CHUNK - 2,
15625                MAX_ITEMS_PER_CHUNK - 1,
15626                MAX_ITEMS_PER_CHUNK,
15627                MAX_ITEMS_PER_CHUNK + 1,
15628            ] {
15629                for full_chunks in [0, 1, 2, 3] {
15630                    for reduced_items in [0, 1, 2] {
15631                        for added_items in [0, 1, 2] {
15632                            // this will verify the entire range correctly
15633                            _ = new_splitter2(
15634                                offset * offset_multiplier,
15635                                (full_chunks * MAX_ITEMS_PER_CHUNK + added_items)
15636                                    .saturating_sub(reduced_items),
15637                            );
15638                        }
15639                    }
15640                }
15641            }
15642        }
15643    }
15644
15645    define_accounts_db_test!(test_add_uncleaned_pubkeys_after_shrink, |db| {
15646        let slot = 0;
15647        let pubkey = Pubkey::from([1; 32]);
15648        db.add_uncleaned_pubkeys_after_shrink(slot, vec![pubkey].into_iter());
15649        assert_eq!(&*db.uncleaned_pubkeys.get(&slot).unwrap(), &vec![pubkey]);
15650    });
15651
15652    define_accounts_db_test!(test_get_ancient_slots, |db| {
15653        let slot1 = 1;
15654
15655        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
15656        let storages = (0..3)
15657            .map(|i| db.create_and_insert_store(slot1 + (i as Slot), 1000, "test"))
15658            .collect::<Vec<_>>();
15659
15660        for count in 1..4 {
15661            // use subset of storages
15662            let mut raw_storages = storages.clone();
15663            raw_storages.truncate(count);
15664            let snapshot_storages = SortedStorages::new(&raw_storages);
15665            // 0 = all storages are non-ancient
15666            // 1 = all storages are non-ancient
15667            // 2 = ancient slots: 1
15668            // 3 = ancient slots: 1, 2
15669            // 4 = ancient slots: 1, 2, 3
15670            // 5 = ...
15671            for all_are_large in [false, true] {
15672                for oldest_non_ancient_slot in 0..6 {
15673                    let ancient_slots = SplitAncientStorages::get_ancient_slots(
15674                        oldest_non_ancient_slot,
15675                        &snapshot_storages,
15676                        |_storage| all_are_large,
15677                    );
15678
15679                    if all_are_large {
15680                        assert_eq!(
15681                            raw_storages
15682                                .iter()
15683                                .filter_map(|storage| {
15684                                    let slot = storage.slot();
15685                                    (slot < oldest_non_ancient_slot).then_some(slot)
15686                                })
15687                                .collect::<Vec<_>>(),
15688                            ancient_slots,
15689                            "count: {count}"
15690                        );
15691                    } else {
15692                        // none are treated as ancient since none were deemed large enough append vecs.
15693                        assert!(ancient_slots.is_empty());
15694                    }
15695                }
15696            }
15697        }
15698    });
15699
15700    define_accounts_db_test!(test_get_ancient_slots_one_large, |db| {
15701        let slot1 = 1;
15702
15703        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
15704        let storages = (0..3)
15705            .map(|i| db.create_and_insert_store(slot1 + (i as Slot), 1000, "test"))
15706            .collect::<Vec<_>>();
15707
15708        for count in 1..4 {
15709            // use subset of storages
15710            let mut raw_storages = storages.clone();
15711            raw_storages.truncate(count);
15712            let snapshot_storages = SortedStorages::new(&raw_storages);
15713            // 0 = all storages are non-ancient
15714            // 1 = all storages are non-ancient
15715            // 2 = ancient slots: 1
15716            // 3 = ancient slots: 1, 2
15717            // 4 = ancient slots: 1, 2 (except 2 is large, 3 is not, so treat 3 as non-ancient)
15718            // 5 = ...
15719            for oldest_non_ancient_slot in 0..6 {
15720                let ancient_slots = SplitAncientStorages::get_ancient_slots(
15721                    oldest_non_ancient_slot,
15722                    &snapshot_storages,
15723                    |storage| storage.slot() == 2,
15724                );
15725                let mut expected = raw_storages
15726                    .iter()
15727                    .filter_map(|storage| {
15728                        let slot = storage.slot();
15729                        (slot < oldest_non_ancient_slot).then_some(slot)
15730                    })
15731                    .collect::<Vec<_>>();
15732                if expected.len() >= 2 {
15733                    // slot 3 is not considered ancient since slot 3 is a small append vec.
15734                    // slot 2 is the only large append vec, so 1 by itself is not ancient. [1, 2] is ancient, [1,2,3] becomes just [1,2]
15735                    expected.truncate(2);
15736                } else {
15737                    // we're not asking about the big append vec at 2, so nothing
15738                    expected.clear();
15739                }
15740                assert_eq!(expected, ancient_slots, "count: {count}");
15741            }
15742        }
15743    });
15744
15745    #[test]
15746    fn test_hash_storage_info() {
15747        {
15748            let hasher = DefaultHasher::new();
15749            let hash = hasher.finish();
15750            assert_eq!(15130871412783076140, hash);
15751        }
15752        {
15753            let mut hasher = DefaultHasher::new();
15754            let slot: Slot = 0;
15755            let tf = crate::append_vec::test_utils::get_append_vec_path("test_hash_storage_info");
15756            let pubkey1 = solana_sdk::pubkey::new_rand();
15757            let mark_alive = false;
15758            let storage = sample_storage_with_entries(&tf, slot, &pubkey1, mark_alive);
15759
15760            let load = AccountsDb::hash_storage_info(&mut hasher, &storage, slot);
15761            let hash = hasher.finish();
15762            // can't assert hash here - it is a function of mod date
15763            assert!(load);
15764            let slot = 2; // changed this
15765            let mut hasher = DefaultHasher::new();
15766            let load = AccountsDb::hash_storage_info(&mut hasher, &storage, slot);
15767            let hash2 = hasher.finish();
15768            assert_ne!(hash, hash2); // slot changed, these should be different
15769                                     // can't assert hash here - it is a function of mod date
15770            assert!(load);
15771            let mut hasher = DefaultHasher::new();
15772            append_sample_data_to_storage(&storage, &solana_sdk::pubkey::new_rand(), false, None);
15773            let load = AccountsDb::hash_storage_info(&mut hasher, &storage, slot);
15774            let hash3 = hasher.finish();
15775            assert_ne!(hash2, hash3); // moddate and written size changed
15776                                      // can't assert hash here - it is a function of mod date
15777            assert!(load);
15778            let mut hasher = DefaultHasher::new();
15779            let load = AccountsDb::hash_storage_info(&mut hasher, &storage, slot);
15780            let hash4 = hasher.finish();
15781            assert_eq!(hash4, hash3); // same
15782                                      // can't assert hash here - it is a function of mod date
15783            assert!(load);
15784        }
15785    }
15786
15787    #[test]
15788    fn test_sweep_get_oldest_non_ancient_slot_max() {
15789        let epoch_schedule = EpochSchedule::default();
15790        // way into future
15791        for ancient_append_vec_offset in [
15792            epoch_schedule.slots_per_epoch,
15793            epoch_schedule.slots_per_epoch + 1,
15794            epoch_schedule.slots_per_epoch * 2,
15795        ] {
15796            let db = AccountsDb::new_with_config(
15797                Vec::new(),
15798                Some(AccountsDbConfig {
15799                    ancient_append_vec_offset: Some(ancient_append_vec_offset as i64),
15800                    ..ACCOUNTS_DB_CONFIG_FOR_TESTING
15801                }),
15802                None,
15803                Arc::default(),
15804            );
15805            // before any roots are added, we expect the oldest non-ancient slot to be 0
15806            assert_eq!(0, db.get_oldest_non_ancient_slot(&epoch_schedule));
15807            for max_root_inclusive in [
15808                0,
15809                epoch_schedule.slots_per_epoch,
15810                epoch_schedule.slots_per_epoch * 2,
15811                epoch_schedule.slots_per_epoch * 10,
15812            ] {
15813                db.add_root(max_root_inclusive);
15814                // oldest non-ancient will never exceed max_root_inclusive, even if the offset is so large it would mathematically move ancient PAST the newest root
15815                assert_eq!(
15816                    max_root_inclusive,
15817                    db.get_oldest_non_ancient_slot(&epoch_schedule)
15818                );
15819            }
15820        }
15821    }
15822
15823    #[test]
15824    fn test_sweep_get_oldest_non_ancient_slot() {
15825        let epoch_schedule = EpochSchedule::default();
15826        let ancient_append_vec_offset = 50_000;
15827        let db = AccountsDb::new_with_config(
15828            Vec::new(),
15829            Some(AccountsDbConfig {
15830                ancient_append_vec_offset: Some(ancient_append_vec_offset),
15831                ..ACCOUNTS_DB_CONFIG_FOR_TESTING
15832            }),
15833            None,
15834            Arc::default(),
15835        );
15836        // before any roots are added, we expect the oldest non-ancient slot to be 0
15837        assert_eq!(0, db.get_oldest_non_ancient_slot(&epoch_schedule));
15838        // adding roots until slots_per_epoch +/- ancient_append_vec_offset should still saturate to 0 as oldest non ancient slot
15839        let max_root_inclusive = AccountsDb::apply_offset_to_slot(0, ancient_append_vec_offset - 1);
15840        db.add_root(max_root_inclusive);
15841        // oldest non-ancient will never exceed max_root_inclusive
15842        assert_eq!(0, db.get_oldest_non_ancient_slot(&epoch_schedule));
15843        for offset in 0..3u64 {
15844            let max_root_inclusive = ancient_append_vec_offset as u64 + offset;
15845            db.add_root(max_root_inclusive);
15846            assert_eq!(
15847                0,
15848                db.get_oldest_non_ancient_slot(&epoch_schedule),
15849                "offset: {offset}"
15850            );
15851        }
15852        for offset in 0..3u64 {
15853            let max_root_inclusive = AccountsDb::apply_offset_to_slot(
15854                epoch_schedule.slots_per_epoch - 1,
15855                -ancient_append_vec_offset,
15856            ) + offset;
15857            db.add_root(max_root_inclusive);
15858            assert_eq!(
15859                offset,
15860                db.get_oldest_non_ancient_slot(&epoch_schedule),
15861                "offset: {offset}, max_root_inclusive: {max_root_inclusive}"
15862            );
15863        }
15864    }
15865
15866    #[test]
15867    fn test_sweep_get_oldest_non_ancient_slot2() {
15868        // note that this test has to worry about saturation at 0 as we subtract `slots_per_epoch` and `ancient_append_vec_offset`
15869        let epoch_schedule = EpochSchedule::default();
15870        for ancient_append_vec_offset in [-10_000i64, 50_000] {
15871            // at `starting_slot_offset`=0, with a negative `ancient_append_vec_offset`, we expect saturation to 0
15872            // big enough to avoid all saturation issues.
15873            let avoid_saturation = 1_000_000;
15874            assert!(
15875                avoid_saturation
15876                    > epoch_schedule.slots_per_epoch + ancient_append_vec_offset.unsigned_abs()
15877            );
15878            for starting_slot_offset in [0, avoid_saturation] {
15879                let db = AccountsDb::new_with_config(
15880                    Vec::new(),
15881                    Some(AccountsDbConfig {
15882                        ancient_append_vec_offset: Some(ancient_append_vec_offset),
15883                        ..ACCOUNTS_DB_CONFIG_FOR_TESTING
15884                    }),
15885                    None,
15886                    Arc::default(),
15887                );
15888                // before any roots are added, we expect the oldest non-ancient slot to be 0
15889                assert_eq!(0, db.get_oldest_non_ancient_slot(&epoch_schedule));
15890
15891                let ancient_append_vec_offset = db.ancient_append_vec_offset.unwrap();
15892                assert_ne!(ancient_append_vec_offset, 0);
15893                // try a few values to simulate a real validator
15894                for inc in [0, 1, 2, 3, 4, 5, 8, 10, 10, 11, 200, 201, 1_000] {
15895                    // oldest non-ancient slot is 1 greater than first ancient slot
15896                    let completed_slot =
15897                        epoch_schedule.slots_per_epoch + inc + starting_slot_offset;
15898
15899                    // test get_oldest_non_ancient_slot, which is based off the largest root
15900                    db.add_root(completed_slot);
15901                    let expected_oldest_non_ancient_slot = AccountsDb::apply_offset_to_slot(
15902                        AccountsDb::apply_offset_to_slot(
15903                            completed_slot,
15904                            -((epoch_schedule.slots_per_epoch as i64).saturating_sub(1)),
15905                        ),
15906                        ancient_append_vec_offset,
15907                    );
15908                    assert_eq!(
15909                        expected_oldest_non_ancient_slot,
15910                        db.get_oldest_non_ancient_slot(&epoch_schedule)
15911                    );
15912                }
15913            }
15914        }
15915    }
15916
15917    #[test]
15918    #[should_panic(expected = "called `Option::unwrap()` on a `None` value")]
15919    fn test_current_ancient_slot_assert() {
15920        let current_ancient = CurrentAncientAccountsFile::default();
15921        _ = current_ancient.slot();
15922    }
15923
15924    #[test]
15925    #[should_panic(expected = "called `Option::unwrap()` on a `None` value")]
15926    fn test_current_ancient_append_vec_assert() {
15927        let current_ancient = CurrentAncientAccountsFile::default();
15928        _ = current_ancient.accounts_file();
15929    }
15930
15931    #[test]
15932    fn test_current_ancient_simple() {
15933        let slot = 1;
15934        let slot2 = 2;
15935        let slot3 = 3;
15936        {
15937            // new
15938            let db = AccountsDb::new_single_for_tests();
15939            let size = 1000;
15940            let append_vec = db.create_and_insert_store(slot, size, "test");
15941            let mut current_ancient = CurrentAncientAccountsFile::new(slot, append_vec.clone());
15942            assert_eq!(current_ancient.slot(), slot);
15943            assert_eq!(current_ancient.id(), append_vec.id());
15944            assert_eq!(current_ancient.accounts_file().id(), append_vec.id());
15945
15946            let _shrink_in_progress = current_ancient.create_if_necessary(slot2, &db, 0);
15947            assert_eq!(current_ancient.slot(), slot);
15948            assert_eq!(current_ancient.id(), append_vec.id());
15949        }
15950
15951        {
15952            // create_if_necessary
15953            let db = AccountsDb::new_single_for_tests();
15954            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
15955            let _existing_append_vec = db.create_and_insert_store(slot2, 1000, "test");
15956
15957            let mut current_ancient = CurrentAncientAccountsFile::default();
15958            let mut _shrink_in_progress = current_ancient.create_if_necessary(slot2, &db, 0);
15959            let id = current_ancient.id();
15960            assert_eq!(current_ancient.slot(), slot2);
15961            assert!(is_ancient(&current_ancient.accounts_file().accounts));
15962            let slot3 = 3;
15963            // should do nothing
15964            let _shrink_in_progress = current_ancient.create_if_necessary(slot3, &db, 0);
15965            assert_eq!(current_ancient.slot(), slot2);
15966            assert_eq!(current_ancient.id(), id);
15967            assert!(is_ancient(&current_ancient.accounts_file().accounts));
15968        }
15969
15970        {
15971            // create_ancient_append_vec
15972            let db = AccountsDb::new_single_for_tests();
15973            let mut current_ancient = CurrentAncientAccountsFile::default();
15974            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
15975            let _existing_append_vec = db.create_and_insert_store(slot2, 1000, "test");
15976
15977            {
15978                let _shrink_in_progress =
15979                    current_ancient.create_ancient_accounts_file(slot2, &db, 0);
15980            }
15981            let id = current_ancient.id();
15982            assert_eq!(current_ancient.slot(), slot2);
15983            assert!(is_ancient(&current_ancient.accounts_file().accounts));
15984
15985            // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
15986            let _existing_append_vec = db.create_and_insert_store(slot3, 1000, "test");
15987
15988            let mut _shrink_in_progress =
15989                current_ancient.create_ancient_accounts_file(slot3, &db, 0);
15990            assert_eq!(current_ancient.slot(), slot3);
15991            assert!(is_ancient(&current_ancient.accounts_file().accounts));
15992            assert_ne!(current_ancient.id(), id);
15993        }
15994    }
15995
15996    define_accounts_db_test!(test_get_sorted_potential_ancient_slots, |db| {
15997        let ancient_append_vec_offset = db.ancient_append_vec_offset.unwrap();
15998        let epoch_schedule = EpochSchedule::default();
15999        let oldest_non_ancient_slot = db.get_oldest_non_ancient_slot(&epoch_schedule);
16000        assert!(db
16001            .get_sorted_potential_ancient_slots(oldest_non_ancient_slot)
16002            .is_empty());
16003        let root1 = MAX_ANCIENT_SLOTS_DEFAULT as u64 + ancient_append_vec_offset as u64 + 1;
16004        db.add_root(root1);
16005        let root2 = root1 + 1;
16006        db.add_root(root2);
16007        let oldest_non_ancient_slot = db.get_oldest_non_ancient_slot(&epoch_schedule);
16008        assert!(db
16009            .get_sorted_potential_ancient_slots(oldest_non_ancient_slot)
16010            .is_empty());
16011        let completed_slot = epoch_schedule.slots_per_epoch;
16012        db.accounts_index.add_root(AccountsDb::apply_offset_to_slot(
16013            completed_slot,
16014            ancient_append_vec_offset,
16015        ));
16016        let oldest_non_ancient_slot = db.get_oldest_non_ancient_slot(&epoch_schedule);
16017        // get_sorted_potential_ancient_slots uses 'less than' as opposed to 'less or equal'
16018        // so, we need to get more than an epoch away to get the first valid root
16019        assert!(db
16020            .get_sorted_potential_ancient_slots(oldest_non_ancient_slot)
16021            .is_empty());
16022        let completed_slot = epoch_schedule.slots_per_epoch + root1;
16023        db.accounts_index.add_root(AccountsDb::apply_offset_to_slot(
16024            completed_slot,
16025            ancient_append_vec_offset,
16026        ));
16027        let oldest_non_ancient_slot = db.get_oldest_non_ancient_slot(&epoch_schedule);
16028        assert_eq!(
16029            db.get_sorted_potential_ancient_slots(oldest_non_ancient_slot),
16030            vec![root1, root2]
16031        );
16032        let completed_slot = epoch_schedule.slots_per_epoch + root2;
16033        db.accounts_index.add_root(AccountsDb::apply_offset_to_slot(
16034            completed_slot,
16035            ancient_append_vec_offset,
16036        ));
16037        let oldest_non_ancient_slot = db.get_oldest_non_ancient_slot(&epoch_schedule);
16038        assert_eq!(
16039            db.get_sorted_potential_ancient_slots(oldest_non_ancient_slot),
16040            vec![root1, root2]
16041        );
16042        db.accounts_index
16043            .roots_tracker
16044            .write()
16045            .unwrap()
16046            .alive_roots
16047            .remove(&root1);
16048        let oldest_non_ancient_slot = db.get_oldest_non_ancient_slot(&epoch_schedule);
16049        assert_eq!(
16050            db.get_sorted_potential_ancient_slots(oldest_non_ancient_slot),
16051            vec![root2]
16052        );
16053    });
16054
16055    #[test]
16056    fn test_shrink_collect_simple() {
16057        solana_logger::setup();
16058        let account_counts = [
16059            1,
16060            SHRINK_COLLECT_CHUNK_SIZE,
16061            SHRINK_COLLECT_CHUNK_SIZE + 1,
16062            SHRINK_COLLECT_CHUNK_SIZE * 2,
16063        ];
16064        // 2 = append_opposite_alive_account + append_opposite_zero_lamport_account
16065        let max_appended_accounts = 2;
16066        let max_num_accounts = *account_counts.iter().max().unwrap();
16067        let pubkeys = (0..(max_num_accounts + max_appended_accounts))
16068            .map(|_| solana_sdk::pubkey::new_rand())
16069            .collect::<Vec<_>>();
16070        // write accounts, maybe remove from index
16071        // check shrink_collect results
16072        for lamports in [0, 1] {
16073            for space in [0, 8] {
16074                if lamports == 0 && space != 0 {
16075                    // illegal - zero lamport accounts are written with 0 space
16076                    continue;
16077                }
16078                for alive in [false, true] {
16079                    for append_opposite_alive_account in [false, true] {
16080                        for append_opposite_zero_lamport_account in [true, false] {
16081                            for mut account_count in account_counts {
16082                                let mut normal_account_count = account_count;
16083                                let mut pubkey_opposite_zero_lamports = None;
16084                                if append_opposite_zero_lamport_account {
16085                                    pubkey_opposite_zero_lamports = Some(&pubkeys[account_count]);
16086                                    normal_account_count += 1;
16087                                    account_count += 1;
16088                                }
16089                                let mut pubkey_opposite_alive = None;
16090                                if append_opposite_alive_account {
16091                                    // this needs to happen AFTER append_opposite_zero_lamport_account
16092                                    pubkey_opposite_alive = Some(&pubkeys[account_count]);
16093                                    account_count += 1;
16094                                }
16095                                debug!(
16096                                    "space: {space}, lamports: {lamports}, alive: {alive}, \
16097                                     account_count: {account_count}, \
16098                                     append_opposite_alive_account: \
16099                                     {append_opposite_alive_account}, \
16100                                     append_opposite_zero_lamport_account: \
16101                                     {append_opposite_zero_lamport_account}, \
16102                                     normal_account_count: {normal_account_count}"
16103                                );
16104                                let db = AccountsDb::new_single_for_tests();
16105                                let slot5 = 5;
16106                                // don't do special zero lamport account handling
16107                                db.set_latest_full_snapshot_slot(0);
16108                                let mut account = AccountSharedData::new(
16109                                    lamports,
16110                                    space,
16111                                    AccountSharedData::default().owner(),
16112                                );
16113                                let mut to_purge = Vec::default();
16114                                for pubkey in pubkeys.iter().take(account_count) {
16115                                    // store in append vec and index
16116                                    let old_lamports = account.lamports();
16117                                    if Some(pubkey) == pubkey_opposite_zero_lamports {
16118                                        account.set_lamports(u64::from(old_lamports == 0));
16119                                    }
16120
16121                                    db.store_for_tests(slot5, &[(pubkey, &account)]);
16122                                    account.set_lamports(old_lamports);
16123                                    let mut alive = alive;
16124                                    if append_opposite_alive_account
16125                                        && Some(pubkey) == pubkey_opposite_alive
16126                                    {
16127                                        // invert this for one special pubkey
16128                                        alive = !alive;
16129                                    }
16130                                    if !alive {
16131                                        // remove from index so pubkey is 'dead'
16132                                        to_purge.push(*pubkey);
16133                                    }
16134                                }
16135                                db.add_root_and_flush_write_cache(slot5);
16136                                to_purge.iter().for_each(|pubkey| {
16137                                    db.accounts_index.purge_exact(
16138                                        pubkey,
16139                                        &([slot5].into_iter().collect::<HashSet<_>>()),
16140                                        &mut Vec::default(),
16141                                    );
16142                                });
16143
16144                                let storage = db.get_storage_for_slot(slot5).unwrap();
16145                                let unique_accounts = db
16146                                    .get_unique_accounts_from_storage_for_shrink(
16147                                        &storage,
16148                                        &ShrinkStats::default(),
16149                                    );
16150
16151                                let shrink_collect = db.shrink_collect::<AliveAccounts<'_>>(
16152                                    &storage,
16153                                    &unique_accounts,
16154                                    &ShrinkStats::default(),
16155                                );
16156                                let expect_single_opposite_alive_account =
16157                                    if append_opposite_alive_account {
16158                                        vec![*pubkey_opposite_alive.unwrap()]
16159                                    } else {
16160                                        vec![]
16161                                    };
16162
16163                                let expected_alive_accounts = if alive {
16164                                    pubkeys[..normal_account_count]
16165                                        .iter()
16166                                        .filter(|p| Some(p) != pubkey_opposite_alive.as_ref())
16167                                        .sorted()
16168                                        .cloned()
16169                                        .collect::<Vec<_>>()
16170                                } else {
16171                                    expect_single_opposite_alive_account.clone()
16172                                };
16173
16174                                let expected_unrefed = if alive {
16175                                    expect_single_opposite_alive_account.clone()
16176                                } else {
16177                                    pubkeys[..normal_account_count]
16178                                        .iter()
16179                                        .sorted()
16180                                        .cloned()
16181                                        .collect::<Vec<_>>()
16182                                };
16183
16184                                assert_eq!(shrink_collect.slot, slot5);
16185
16186                                assert_eq!(
16187                                    shrink_collect
16188                                        .alive_accounts
16189                                        .accounts
16190                                        .iter()
16191                                        .map(|account| *account.pubkey())
16192                                        .sorted()
16193                                        .collect::<Vec<_>>(),
16194                                    expected_alive_accounts
16195                                );
16196                                assert_eq!(
16197                                    shrink_collect
16198                                        .pubkeys_to_unref
16199                                        .iter()
16200                                        .sorted()
16201                                        .cloned()
16202                                        .cloned()
16203                                        .collect::<Vec<_>>(),
16204                                    expected_unrefed
16205                                );
16206
16207                                let alive_total_one_account = 136 + space;
16208                                if alive {
16209                                    let mut expected_alive_total_bytes =
16210                                        alive_total_one_account * normal_account_count;
16211                                    if append_opposite_zero_lamport_account {
16212                                        // zero lamport accounts store size=0 data
16213                                        expected_alive_total_bytes -= space;
16214                                    }
16215                                    assert_eq!(
16216                                        shrink_collect.alive_total_bytes,
16217                                        expected_alive_total_bytes
16218                                    );
16219                                } else if append_opposite_alive_account {
16220                                    assert_eq!(
16221                                        shrink_collect.alive_total_bytes,
16222                                        alive_total_one_account
16223                                    );
16224                                } else {
16225                                    assert_eq!(shrink_collect.alive_total_bytes, 0);
16226                                }
16227                                // expected_capacity is determined by what size append vec gets created when the write cache is flushed to an append vec.
16228                                let mut expected_capacity =
16229                                    (account_count * aligned_stored_size(space)) as u64;
16230                                if append_opposite_zero_lamport_account && space != 0 {
16231                                    // zero lamport accounts always write space = 0
16232                                    expected_capacity -= space as u64;
16233                                }
16234
16235                                assert_eq!(shrink_collect.capacity, expected_capacity);
16236                                assert_eq!(shrink_collect.total_starting_accounts, account_count);
16237                                let mut expected_all_are_zero_lamports = lamports == 0;
16238                                if !append_opposite_alive_account {
16239                                    expected_all_are_zero_lamports |= !alive;
16240                                }
16241                                if append_opposite_zero_lamport_account && lamports == 0 && alive {
16242                                    expected_all_are_zero_lamports =
16243                                        !expected_all_are_zero_lamports;
16244                                }
16245                                assert_eq!(
16246                                    shrink_collect.all_are_zero_lamports,
16247                                    expected_all_are_zero_lamports
16248                                );
16249                            }
16250                        }
16251                    }
16252                }
16253            }
16254        }
16255    }
16256
16257    pub(crate) const CAN_RANDOMLY_SHRINK_FALSE: bool = false;
16258
16259    define_accounts_db_test!(test_combine_ancient_slots_empty, |db| {
16260        // empty slots
16261        db.combine_ancient_slots(Vec::default(), CAN_RANDOMLY_SHRINK_FALSE);
16262    });
16263
16264    #[test]
16265    fn test_combine_ancient_slots_simple() {
16266        // We used to test 'alive = false' with the old shrinking algorithm, but
16267        // not any more with the new shrinking algorithm. 'alive = false' means
16268        // that we will have account entries that's in the storages but not in
16269        // accounts-db index. This violate the assumption in accounts-db, which
16270        // the new shrinking algorithm now depends on. Therefore, we don't test
16271        // 'alive = false'.
16272        _ = get_one_ancient_append_vec_and_others(true, 0);
16273    }
16274
16275    fn get_all_accounts_from_storages<'a>(
16276        storages: impl Iterator<Item = &'a Arc<AccountStorageEntry>>,
16277    ) -> Vec<(Pubkey, AccountSharedData)> {
16278        storages
16279            .flat_map(|storage| {
16280                let mut vec = Vec::default();
16281                storage.accounts.scan_accounts(|account| {
16282                    vec.push((*account.pubkey(), account.to_account_shared_data()));
16283                });
16284                // make sure scan_pubkeys results match
16285                // Note that we assume traversals are both in the same order, but this doesn't have to be true.
16286                let mut compare = Vec::default();
16287                storage.accounts.scan_pubkeys(|k| {
16288                    compare.push(*k);
16289                });
16290                assert_eq!(compare, vec.iter().map(|(k, _)| *k).collect::<Vec<_>>());
16291                vec
16292            })
16293            .collect::<Vec<_>>()
16294    }
16295
16296    pub(crate) fn get_all_accounts(
16297        db: &AccountsDb,
16298        slots: impl Iterator<Item = Slot>,
16299    ) -> Vec<(Pubkey, AccountSharedData)> {
16300        slots
16301            .filter_map(|slot| {
16302                let storage = db.storage.get_slot_storage_entry(slot);
16303                storage.map(|storage| get_all_accounts_from_storages(std::iter::once(&storage)))
16304            })
16305            .flatten()
16306            .collect::<Vec<_>>()
16307    }
16308
16309    pub(crate) fn compare_all_accounts(
16310        one: &[(Pubkey, AccountSharedData)],
16311        two: &[(Pubkey, AccountSharedData)],
16312    ) {
16313        let mut failures = 0;
16314        let mut two_indexes = (0..two.len()).collect::<Vec<_>>();
16315        one.iter().for_each(|(pubkey, account)| {
16316            for i in 0..two_indexes.len() {
16317                let pubkey2 = two[two_indexes[i]].0;
16318                if pubkey2 == *pubkey {
16319                    if !accounts_equal(account, &two[two_indexes[i]].1) {
16320                        failures += 1;
16321                    }
16322                    two_indexes.remove(i);
16323                    break;
16324                }
16325            }
16326        });
16327        // helper method to reduce the volume of logged data to help identify differences
16328        // modify this when you hit a failure
16329        let clean = |accounts: &[(Pubkey, AccountSharedData)]| {
16330            accounts
16331                .iter()
16332                .map(|(_pubkey, account)| account.lamports())
16333                .collect::<Vec<_>>()
16334        };
16335        assert_eq!(
16336            failures,
16337            0,
16338            "one: {:?}, two: {:?}, two_indexes: {:?}",
16339            clean(one),
16340            clean(two),
16341            two_indexes,
16342        );
16343        assert!(
16344            two_indexes.is_empty(),
16345            "one: {one:?}, two: {two:?}, two_indexes: {two_indexes:?}"
16346        );
16347    }
16348
16349    #[test]
16350    fn test_shrink_ancient_overflow_with_min_size() {
16351        solana_logger::setup();
16352
16353        let ideal_av_size = ancient_append_vecs::get_ancient_append_vec_capacity();
16354        let num_normal_slots = 2;
16355
16356        // build an ancient append vec at slot 'ancient_slot' with one `fat`
16357        // account that's larger than the ideal size of ancient append vec to
16358        // simulate the *oversized* append vec for shrinking.
16359        let account_size = (1.5 * ideal_av_size as f64) as u64;
16360        let (db, ancient_slot) = get_one_ancient_append_vec_and_others_with_account_size(
16361            true,
16362            num_normal_slots,
16363            Some(account_size),
16364        );
16365
16366        let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
16367        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
16368
16369        let ancient = db.storage.get_slot_storage_entry(ancient_slot).unwrap();
16370
16371        // assert that the min_size, which about 1.5 * ideal_av_size, kicked in
16372        // and result that the ancient append vec capacity exceeds the ideal_av_size
16373        assert!(ancient.capacity() > ideal_av_size);
16374
16375        // combine 1 normal append vec into existing oversize ancient append vec.
16376        db.combine_ancient_slots(
16377            (ancient_slot..max_slot_inclusive).collect(),
16378            CAN_RANDOMLY_SHRINK_FALSE,
16379        );
16380
16381        compare_all_accounts(
16382            &initial_accounts,
16383            &get_all_accounts(&db, ancient_slot..max_slot_inclusive),
16384        );
16385
16386        // the append vec at max_slot_inclusive-1 should NOT have been removed
16387        // since the append vec is already oversized and we created an ancient
16388        // append vec there.
16389        let ancient2 = db
16390            .storage
16391            .get_slot_storage_entry(max_slot_inclusive - 1)
16392            .unwrap();
16393        assert!(is_ancient(&ancient2.accounts));
16394        assert!(ancient2.capacity() > ideal_av_size); // min_size kicked in, which cause the appendvec to be larger than the ideal_av_size
16395
16396        // Combine normal append vec(s) into existing ancient append vec this
16397        // will overflow the original ancient append vec because of the oversized
16398        // ancient append vec is full.
16399        db.combine_ancient_slots(
16400            (ancient_slot..=max_slot_inclusive).collect(),
16401            CAN_RANDOMLY_SHRINK_FALSE,
16402        );
16403
16404        compare_all_accounts(
16405            &initial_accounts,
16406            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16407        );
16408
16409        // Nothing should be combined because the append vec are oversized.
16410        // min_size kicked in, which cause the appendvecs to be larger than the ideal_av_size.
16411        let ancient = db.storage.get_slot_storage_entry(ancient_slot).unwrap();
16412        assert!(is_ancient(&ancient.accounts));
16413        assert!(ancient.capacity() > ideal_av_size);
16414
16415        let ancient2 = db
16416            .storage
16417            .get_slot_storage_entry(max_slot_inclusive - 1)
16418            .unwrap();
16419        assert!(is_ancient(&ancient2.accounts));
16420        assert!(ancient2.capacity() > ideal_av_size);
16421
16422        let ancient3 = db
16423            .storage
16424            .get_slot_storage_entry(max_slot_inclusive)
16425            .unwrap();
16426        assert!(is_ancient(&ancient3.accounts));
16427        assert!(ancient3.capacity() > ideal_av_size);
16428    }
16429
16430    #[test]
16431    fn test_shink_overflow_too_much() {
16432        let num_normal_slots = 2;
16433        let ideal_av_size = ancient_append_vecs::get_ancient_append_vec_capacity();
16434        let fat_account_size = (1.5 * ideal_av_size as f64) as u64;
16435
16436        // Prepare 3 append vecs to combine [small, big, small]
16437        let account_data_sizes = vec![100, fat_account_size, 100];
16438        let (db, slot1) = create_db_with_storages_and_index_with_customized_account_size_per_slot(
16439            true,
16440            num_normal_slots + 1,
16441            account_data_sizes,
16442        );
16443        let storage = db.get_storage_for_slot(slot1).unwrap();
16444        let created_accounts = db.get_unique_accounts_from_storage(&storage);
16445
16446        // Adjust alive_ratio for slot2 to test it is shrinkable and is a
16447        // candidate for squashing into the previous ancient append vec.
16448        // However, due to the fact that this append vec is `oversized`, it can't
16449        // be squashed into the ancient append vec at previous slot (exceeds the
16450        // size limit). Therefore, a new "oversized" ancient append vec is
16451        // created at slot2 as the overflow. This is where the "min_bytes" in
16452        // `fn create_ancient_append_vec` is used.
16453        let slot2 = slot1 + 1;
16454        let storage2 = db.storage.get_slot_storage_entry(slot2).unwrap();
16455        let original_cap_slot2 = storage2.accounts.capacity();
16456        storage2
16457            .accounts
16458            .set_current_len_for_tests(original_cap_slot2 as usize);
16459
16460        // Combine append vec into ancient append vec.
16461        let slots_to_combine: Vec<Slot> = (slot1..slot1 + (num_normal_slots + 1) as Slot).collect();
16462        db.combine_ancient_slots(slots_to_combine, CAN_RANDOMLY_SHRINK_FALSE);
16463
16464        // slot2 is too big to fit into ideal ancient append vec at slot1. So slot2 won't be merged into slot1.
16465        // slot1 will have its own ancient append vec.
16466        assert!(db.storage.get_slot_storage_entry(slot1).is_some());
16467        let ancient = db.get_storage_for_slot(slot1).unwrap();
16468        assert!(is_ancient(&ancient.accounts));
16469        assert_eq!(ancient.capacity(), ideal_av_size);
16470
16471        let after_store = db.get_storage_for_slot(slot1).unwrap();
16472        let GetUniqueAccountsResult {
16473            stored_accounts: after_stored_accounts,
16474            capacity: after_capacity,
16475            ..
16476        } = db.get_unique_accounts_from_storage(&after_store);
16477        assert!(created_accounts.capacity <= after_capacity);
16478        assert_eq!(created_accounts.stored_accounts.len(), 1);
16479        assert_eq!(after_stored_accounts.len(), 1);
16480
16481        // slot2, even after shrinking, is still oversized. Therefore, slot 2
16482        // exists as an ancient append vec.
16483        let storage2_after = db.storage.get_slot_storage_entry(slot2).unwrap();
16484        assert!(is_ancient(&storage2_after.accounts));
16485        assert!(storage2_after.capacity() > ideal_av_size);
16486        let after_store = db.get_storage_for_slot(slot2).unwrap();
16487        let GetUniqueAccountsResult {
16488            stored_accounts: after_stored_accounts,
16489            capacity: after_capacity,
16490            ..
16491        } = db.get_unique_accounts_from_storage(&after_store);
16492        assert!(created_accounts.capacity <= after_capacity);
16493        assert_eq!(created_accounts.stored_accounts.len(), 1);
16494        assert_eq!(after_stored_accounts.len(), 1);
16495    }
16496
16497    #[test]
16498    fn test_shrink_ancient_overflow() {
16499        solana_logger::setup();
16500
16501        let num_normal_slots = 2;
16502        // build an ancient append vec at slot 'ancient_slot'
16503        let (db, ancient_slot) = get_one_ancient_append_vec_and_others(true, num_normal_slots);
16504
16505        let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
16506        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
16507
16508        let ancient = db.storage.get_slot_storage_entry(ancient_slot).unwrap();
16509        let initial_len = ancient.alive_bytes();
16510        // set size of ancient to be 'full'
16511        adjust_append_vec_len_for_tests(&ancient, ancient.accounts.capacity() as usize);
16512
16513        // combine 1 normal append vec into existing ancient append vec
16514        // this will overflow the original ancient append vec because of the marking full above
16515        db.combine_ancient_slots(
16516            (ancient_slot..max_slot_inclusive).collect(),
16517            CAN_RANDOMLY_SHRINK_FALSE,
16518        );
16519
16520        // Restore size of ancient so we don't read garbage accounts when comparing. Now that we have created a second ancient append vec,
16521        // This first one is happy to be quite empty.
16522        adjust_append_vec_len_for_tests(&ancient, initial_len);
16523
16524        compare_all_accounts(
16525            &initial_accounts,
16526            &get_all_accounts(&db, ancient_slot..max_slot_inclusive),
16527        );
16528
16529        // the append vec at max_slot_inclusive-1 should NOT have been removed since we created an ancient append vec there
16530        assert!(is_ancient(
16531            &db.storage
16532                .get_slot_storage_entry(max_slot_inclusive - 1)
16533                .unwrap()
16534                .accounts
16535        ));
16536
16537        // combine normal append vec(s) into existing ancient append vec
16538        // this will overflow the original ancient append vec because of the marking full above
16539        db.combine_ancient_slots(
16540            (ancient_slot..=max_slot_inclusive).collect(),
16541            CAN_RANDOMLY_SHRINK_FALSE,
16542        );
16543
16544        // now, combine the next slot into the one that was just overflow
16545        compare_all_accounts(
16546            &initial_accounts,
16547            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16548        );
16549
16550        // 2 ancients and then missing (because combined into 2nd ancient)
16551        assert!(is_ancient(
16552            &db.storage
16553                .get_slot_storage_entry(ancient_slot)
16554                .unwrap()
16555                .accounts
16556        ));
16557        assert!(is_ancient(
16558            &db.storage
16559                .get_slot_storage_entry(max_slot_inclusive - 1)
16560                .unwrap()
16561                .accounts
16562        ));
16563        assert!(db
16564            .storage
16565            .get_slot_storage_entry(max_slot_inclusive)
16566            .is_none());
16567    }
16568
16569    #[test]
16570    fn test_shrink_ancient() {
16571        solana_logger::setup();
16572
16573        let num_normal_slots = 1;
16574        // build an ancient append vec at slot 'ancient_slot'
16575        let (db, ancient_slot) = get_one_ancient_append_vec_and_others(true, num_normal_slots);
16576
16577        let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
16578        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
16579        compare_all_accounts(
16580            &initial_accounts,
16581            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16582        );
16583
16584        // combine normal append vec(s) into existing ancient append vec
16585        db.combine_ancient_slots(
16586            (ancient_slot..=max_slot_inclusive).collect(),
16587            CAN_RANDOMLY_SHRINK_FALSE,
16588        );
16589
16590        compare_all_accounts(
16591            &initial_accounts,
16592            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16593        );
16594
16595        // create a 2nd ancient append vec at 'next_slot'
16596        let next_slot = max_slot_inclusive + 1;
16597        create_storages_and_update_index(&db, None, next_slot, num_normal_slots, true, None);
16598        let max_slot_inclusive = next_slot + (num_normal_slots as Slot);
16599
16600        let initial_accounts = get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1));
16601        compare_all_accounts(
16602            &initial_accounts,
16603            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16604        );
16605
16606        db.combine_ancient_slots(
16607            (next_slot..=max_slot_inclusive).collect(),
16608            CAN_RANDOMLY_SHRINK_FALSE,
16609        );
16610
16611        compare_all_accounts(
16612            &initial_accounts,
16613            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16614        );
16615
16616        // now, shrink the second ancient append vec into the first one
16617        let mut current_ancient = CurrentAncientAccountsFile::new(
16618            ancient_slot,
16619            db.get_storage_for_slot(ancient_slot).unwrap(),
16620        );
16621        let mut dropped_roots = Vec::default();
16622        db.combine_one_store_into_ancient(
16623            next_slot,
16624            &db.get_storage_for_slot(next_slot).unwrap(),
16625            &mut current_ancient,
16626            &mut AncientSlotPubkeys::default(),
16627            &mut dropped_roots,
16628        );
16629        assert!(db.storage.is_empty_entry(next_slot));
16630        // this removes the storages entry completely from the hashmap for 'next_slot'.
16631        // Otherwise, we have a zero length vec in that hashmap
16632        db.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
16633        assert!(db.storage.get_slot_storage_entry(next_slot).is_none());
16634
16635        // include all the slots we put into the ancient append vec - they should contain nothing
16636        compare_all_accounts(
16637            &initial_accounts,
16638            &get_all_accounts(&db, ancient_slot..(max_slot_inclusive + 1)),
16639        );
16640        // look at just the ancient append vec
16641        compare_all_accounts(
16642            &initial_accounts,
16643            &get_all_accounts(&db, ancient_slot..(ancient_slot + 1)),
16644        );
16645        // make sure there is only 1 ancient append vec at the ancient slot
16646        assert!(db.storage.get_slot_storage_entry(ancient_slot).is_some());
16647        assert!(is_ancient(
16648            &db.storage
16649                .get_slot_storage_entry(ancient_slot)
16650                .unwrap()
16651                .accounts
16652        ));
16653        ((ancient_slot + 1)..=max_slot_inclusive)
16654            .for_each(|slot| assert!(db.storage.get_slot_storage_entry(slot).is_none()));
16655    }
16656
16657    pub fn get_account_from_account_from_storage(
16658        account: &AccountFromStorage,
16659        db: &AccountsDb,
16660        slot: Slot,
16661    ) -> AccountSharedData {
16662        let storage = db
16663            .storage
16664            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
16665            .unwrap();
16666        storage
16667            .accounts
16668            .get_account_shared_data(account.index_info.offset())
16669            .unwrap()
16670    }
16671
16672    #[test]
16673    fn test_combine_ancient_slots_append() {
16674        solana_logger::setup();
16675        // combine 2-4 slots into a single ancient append vec
16676        for num_normal_slots in 1..3 {
16677            // We used to test dead_accounts for [0..=num_normal_slots]. This
16678            // works with old shrinking algorithm, but no longer works with the
16679            // new shrinking algorithm. The new shrinking algorithm requires
16680            // that there should be no accounts entries, which are in the
16681            // storage but not in the accounts-db index. And we expect that this
16682            // assumption to be held by accounts-db. Therefore, we don't test
16683            // dead_accounts anymore.  By setting dead_accounts to 0, we
16684            // effectively skip dead_accounts removal in this test.
16685            for dead_accounts in [0] {
16686                let mut originals = Vec::default();
16687                // ancient_slot: contains ancient append vec
16688                // ancient_slot + 1: contains normal append vec with 1 alive account
16689                let (db, ancient_slot) =
16690                    get_one_ancient_append_vec_and_others(true, num_normal_slots);
16691
16692                let max_slot_inclusive = ancient_slot + (num_normal_slots as Slot);
16693
16694                for slot in ancient_slot..=max_slot_inclusive {
16695                    originals.push(db.get_storage_for_slot(slot).unwrap());
16696                }
16697
16698                {
16699                    // remove the intended dead slots from the index so they look dead
16700                    for (count_marked_dead, original) in originals.iter().skip(1).enumerate() {
16701                        // skip the ancient one
16702                        if count_marked_dead >= dead_accounts {
16703                            break;
16704                        }
16705                        let original_pubkey = original
16706                            .accounts
16707                            .get_stored_account_meta_callback(0, |account| *account.pubkey())
16708                            .unwrap();
16709                        let slot = ancient_slot + 1 + (count_marked_dead as Slot);
16710                        _ = db.purge_keys_exact(
16711                            [(
16712                                original_pubkey,
16713                                vec![slot].into_iter().collect::<HashSet<_>>(),
16714                            )]
16715                            .iter(),
16716                        );
16717                    }
16718                    // the entries from these original append vecs should not expect to be in the final ancient append vec
16719                    for _ in 0..dead_accounts {
16720                        originals.remove(1); // remove the first non-ancient original entry each time
16721                    }
16722                }
16723
16724                // combine normal append vec(s) into existing ancient append vec
16725                db.combine_ancient_slots(
16726                    (ancient_slot..=max_slot_inclusive).collect(),
16727                    CAN_RANDOMLY_SHRINK_FALSE,
16728                );
16729
16730                // normal slots should have been appended to the ancient append vec in the first slot
16731                assert!(db.storage.get_slot_storage_entry(ancient_slot).is_some());
16732                let ancient = db.get_storage_for_slot(ancient_slot).unwrap();
16733                assert!(is_ancient(&ancient.accounts));
16734                let first_alive = ancient_slot + 1 + (dead_accounts as Slot);
16735                for slot in first_alive..=max_slot_inclusive {
16736                    assert!(db.storage.get_slot_storage_entry(slot).is_none());
16737                }
16738
16739                let GetUniqueAccountsResult {
16740                    stored_accounts: mut after_stored_accounts,
16741                    ..
16742                } = db.get_unique_accounts_from_storage(&ancient);
16743                assert_eq!(
16744                    after_stored_accounts.len(),
16745                    num_normal_slots + 1 - dead_accounts,
16746                    "normal_slots: {num_normal_slots}, dead_accounts: {dead_accounts}"
16747                );
16748                for original in &originals {
16749                    let i = original
16750                        .accounts
16751                        .get_stored_account_meta_callback(0, |original| {
16752                            after_stored_accounts
16753                                .iter()
16754                                .enumerate()
16755                                .find_map(|(i, stored_ancient)| {
16756                                    (stored_ancient.pubkey() == original.pubkey()).then_some({
16757                                        assert!(accounts_equal(
16758                                            &get_account_from_account_from_storage(
16759                                                stored_ancient,
16760                                                &db,
16761                                                ancient_slot
16762                                            ),
16763                                            &original
16764                                        ));
16765                                        i
16766                                    })
16767                                })
16768                                .expect("did not find account")
16769                        })
16770                        .expect("did not find account");
16771                    after_stored_accounts.remove(i);
16772                }
16773                assert!(
16774                    after_stored_accounts.is_empty(),
16775                    "originals: {}, num_normal_slots: {}",
16776                    originals.len(),
16777                    num_normal_slots
16778                );
16779            }
16780        }
16781    }
16782
16783    fn populate_index(db: &AccountsDb, slots: Range<Slot>) {
16784        slots.into_iter().for_each(|slot| {
16785            if let Some(storage) = db.get_storage_for_slot(slot) {
16786                storage.accounts.scan_accounts(|account| {
16787                    let info = AccountInfo::new(
16788                        StorageLocation::AppendVec(storage.id(), account.offset()),
16789                        account.lamports(),
16790                    );
16791                    db.accounts_index.upsert(
16792                        slot,
16793                        slot,
16794                        account.pubkey(),
16795                        &account,
16796                        &AccountSecondaryIndexes::default(),
16797                        info,
16798                        &mut Vec::default(),
16799                        UpsertReclaim::IgnoreReclaims,
16800                    );
16801                })
16802            }
16803        })
16804    }
16805
16806    pub(crate) fn remove_account_for_tests(
16807        storage: &AccountStorageEntry,
16808        num_bytes: usize,
16809        reset_accounts: bool,
16810    ) {
16811        storage.remove_accounts(num_bytes, reset_accounts, 1);
16812    }
16813
16814    pub(crate) fn create_storages_and_update_index_with_customized_account_size_per_slot(
16815        db: &AccountsDb,
16816        tf: Option<&TempFile>,
16817        starting_slot: Slot,
16818        num_slots: usize,
16819        alive: bool,
16820        account_data_sizes: Vec<u64>,
16821    ) {
16822        if num_slots == 0 {
16823            return;
16824        }
16825        assert!(account_data_sizes.len() == num_slots);
16826        let local_tf = (tf.is_none()).then(|| {
16827            crate::append_vec::test_utils::get_append_vec_path("create_storages_and_update_index")
16828        });
16829        let tf = tf.unwrap_or_else(|| local_tf.as_ref().unwrap());
16830
16831        let starting_id = db
16832            .storage
16833            .iter()
16834            .map(|storage| storage.1.id())
16835            .max()
16836            .unwrap_or(999);
16837        for (i, account_data_size) in account_data_sizes.iter().enumerate().take(num_slots) {
16838            let id = starting_id + (i as AccountsFileId);
16839            let pubkey1 = solana_sdk::pubkey::new_rand();
16840            let storage = sample_storage_with_entries_id_fill_percentage(
16841                tf,
16842                starting_slot + (i as Slot),
16843                &pubkey1,
16844                id,
16845                alive,
16846                Some(*account_data_size),
16847                50,
16848            );
16849            insert_store(db, Arc::clone(&storage));
16850        }
16851
16852        let storage = db.get_storage_for_slot(starting_slot).unwrap();
16853        let created_accounts = db.get_unique_accounts_from_storage(&storage);
16854        assert_eq!(created_accounts.stored_accounts.len(), 1);
16855
16856        if alive {
16857            populate_index(db, starting_slot..(starting_slot + (num_slots as Slot) + 1));
16858        }
16859    }
16860
16861    pub(crate) fn create_storages_and_update_index(
16862        db: &AccountsDb,
16863        tf: Option<&TempFile>,
16864        starting_slot: Slot,
16865        num_slots: usize,
16866        alive: bool,
16867        account_data_size: Option<u64>,
16868    ) {
16869        if num_slots == 0 {
16870            return;
16871        }
16872
16873        let local_tf = (tf.is_none()).then(|| {
16874            crate::append_vec::test_utils::get_append_vec_path("create_storages_and_update_index")
16875        });
16876        let tf = tf.unwrap_or_else(|| local_tf.as_ref().unwrap());
16877
16878        let starting_id = db
16879            .storage
16880            .iter()
16881            .map(|storage| storage.1.id())
16882            .max()
16883            .unwrap_or(999);
16884        for i in 0..num_slots {
16885            let id = starting_id + (i as AccountsFileId);
16886            let pubkey1 = solana_sdk::pubkey::new_rand();
16887            let storage = sample_storage_with_entries_id(
16888                tf,
16889                starting_slot + (i as Slot),
16890                &pubkey1,
16891                id,
16892                alive,
16893                account_data_size,
16894            );
16895            insert_store(db, Arc::clone(&storage));
16896        }
16897
16898        let storage = db.get_storage_for_slot(starting_slot).unwrap();
16899        let created_accounts = db.get_unique_accounts_from_storage(&storage);
16900        assert_eq!(created_accounts.stored_accounts.len(), 1);
16901
16902        if alive {
16903            populate_index(db, starting_slot..(starting_slot + (num_slots as Slot) + 1));
16904        }
16905    }
16906
16907    pub(crate) fn create_db_with_storages_and_index(
16908        alive: bool,
16909        num_slots: usize,
16910        account_data_size: Option<u64>,
16911    ) -> (AccountsDb, Slot) {
16912        solana_logger::setup();
16913
16914        let db = AccountsDb::new_single_for_tests();
16915
16916        // create a single append vec with a single account in a slot
16917        // add the pubkey to index if alive
16918        // call combine_ancient_slots with the slot
16919        // verify we create an ancient appendvec that has alive accounts and does not have dead accounts
16920
16921        let slot1 = 1;
16922        create_storages_and_update_index(&db, None, slot1, num_slots, alive, account_data_size);
16923
16924        let slot1 = slot1 as Slot;
16925        (db, slot1)
16926    }
16927
16928    pub(crate) fn create_db_with_storages_and_index_with_customized_account_size_per_slot(
16929        alive: bool,
16930        num_slots: usize,
16931        account_data_size: Vec<u64>,
16932    ) -> (AccountsDb, Slot) {
16933        solana_logger::setup();
16934
16935        let db = AccountsDb::new_single_for_tests();
16936
16937        // create a single append vec with a single account in a slot
16938        // add the pubkey to index if alive
16939        // call combine_ancient_slots with the slot
16940        // verify we create an ancient appendvec that has alive accounts and does not have dead accounts
16941
16942        let slot1 = 1;
16943        create_storages_and_update_index_with_customized_account_size_per_slot(
16944            &db,
16945            None,
16946            slot1,
16947            num_slots,
16948            alive,
16949            account_data_size,
16950        );
16951
16952        let slot1 = slot1 as Slot;
16953        (db, slot1)
16954    }
16955
16956    fn get_one_ancient_append_vec_and_others_with_account_size(
16957        alive: bool,
16958        num_normal_slots: usize,
16959        account_data_size: Option<u64>,
16960    ) -> (AccountsDb, Slot) {
16961        let (db, slot1) =
16962            create_db_with_storages_and_index(alive, num_normal_slots + 1, account_data_size);
16963        let storage = db.get_storage_for_slot(slot1).unwrap();
16964        let created_accounts = db.get_unique_accounts_from_storage(&storage);
16965
16966        db.combine_ancient_slots(vec![slot1], CAN_RANDOMLY_SHRINK_FALSE);
16967        assert!(db.storage.get_slot_storage_entry(slot1).is_some());
16968        let ancient = db.get_storage_for_slot(slot1).unwrap();
16969        assert_eq!(alive, is_ancient(&ancient.accounts));
16970        let after_store = db.get_storage_for_slot(slot1).unwrap();
16971        let GetUniqueAccountsResult {
16972            stored_accounts: after_stored_accounts,
16973            capacity: after_capacity,
16974            ..
16975        } = db.get_unique_accounts_from_storage(&after_store);
16976        if alive {
16977            assert!(created_accounts.capacity <= after_capacity);
16978        } else {
16979            assert_eq!(created_accounts.capacity, after_capacity);
16980        }
16981        assert_eq!(created_accounts.stored_accounts.len(), 1);
16982        // always 1 account: either we leave the append vec alone if it is all dead
16983        // or we create a new one and copy into it if account is alive
16984        assert_eq!(after_stored_accounts.len(), 1);
16985        (db, slot1)
16986    }
16987
16988    fn get_one_ancient_append_vec_and_others(
16989        alive: bool,
16990        num_normal_slots: usize,
16991    ) -> (AccountsDb, Slot) {
16992        get_one_ancient_append_vec_and_others_with_account_size(alive, num_normal_slots, None)
16993    }
16994
16995    #[test]
16996    fn test_handle_dropped_roots_for_ancient() {
16997        solana_logger::setup();
16998        let db = AccountsDb::new_single_for_tests();
16999        db.handle_dropped_roots_for_ancient(std::iter::empty::<Slot>());
17000        let slot0 = 0;
17001        let dropped_roots = vec![slot0];
17002        db.accounts_index.add_root(slot0);
17003        db.accounts_index.add_uncleaned_roots([slot0]);
17004        assert!(db.accounts_index.is_uncleaned_root(slot0));
17005        assert!(db.accounts_index.is_alive_root(slot0));
17006        db.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
17007        assert!(!db.accounts_index.is_uncleaned_root(slot0));
17008        assert!(!db.accounts_index.is_alive_root(slot0));
17009    }
17010
17011    fn insert_store(db: &AccountsDb, append_vec: Arc<AccountStorageEntry>) {
17012        db.storage.insert(append_vec.slot(), append_vec);
17013    }
17014
17015    #[test]
17016    #[should_panic(expected = "self.storage.remove")]
17017    fn test_handle_dropped_roots_for_ancient_assert() {
17018        solana_logger::setup();
17019        let common_store_path = Path::new("");
17020        let store_file_size = 10_000;
17021        let entry = Arc::new(AccountStorageEntry::new(
17022            common_store_path,
17023            0,
17024            1,
17025            store_file_size,
17026            AccountsFileProvider::AppendVec,
17027        ));
17028        let db = AccountsDb::new_single_for_tests();
17029        let slot0 = 0;
17030        let dropped_roots = vec![slot0];
17031        insert_store(&db, entry);
17032        db.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
17033    }
17034
17035    #[test]
17036    fn test_should_move_to_ancient_accounts_file() {
17037        solana_logger::setup();
17038        let db = AccountsDb::new_single_for_tests();
17039        let slot5 = 5;
17040        let tf = crate::append_vec::test_utils::get_append_vec_path(
17041            "test_should_move_to_ancient_append_vec",
17042        );
17043        let pubkey1 = solana_sdk::pubkey::new_rand();
17044        let storage = sample_storage_with_entries(&tf, slot5, &pubkey1, false);
17045        let mut current_ancient = CurrentAncientAccountsFile::default();
17046
17047        let should_move = db.should_move_to_ancient_accounts_file(
17048            &storage,
17049            &mut current_ancient,
17050            slot5,
17051            CAN_RANDOMLY_SHRINK_FALSE,
17052        );
17053        assert!(current_ancient.slot_and_accounts_file.is_none());
17054        // slot is not ancient, so it is good to move
17055        assert!(should_move);
17056
17057        current_ancient = CurrentAncientAccountsFile::new(slot5, Arc::clone(&storage)); // just 'some', contents don't matter
17058        let should_move = db.should_move_to_ancient_accounts_file(
17059            &storage,
17060            &mut current_ancient,
17061            slot5,
17062            CAN_RANDOMLY_SHRINK_FALSE,
17063        );
17064        // should have kept the same 'current_ancient'
17065        assert_eq!(current_ancient.slot(), slot5);
17066        assert_eq!(current_ancient.accounts_file().slot(), slot5);
17067        assert_eq!(current_ancient.id(), storage.id());
17068
17069        // slot is not ancient, so it is good to move
17070        assert!(should_move);
17071
17072        // 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
17073        let mut current_ancient = CurrentAncientAccountsFile::default();
17074        let slot1_ancient = 1;
17075        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17076        let _existing_append_vec = db.create_and_insert_store(slot1_ancient, 1000, "test");
17077        let ancient1 = db
17078            .get_store_for_shrink(slot1_ancient, get_ancient_append_vec_capacity())
17079            .new_storage()
17080            .clone();
17081        let should_move = db.should_move_to_ancient_accounts_file(
17082            &ancient1,
17083            &mut current_ancient,
17084            slot1_ancient,
17085            CAN_RANDOMLY_SHRINK_FALSE,
17086        );
17087        assert!(!should_move);
17088        assert_eq!(current_ancient.id(), ancient1.id());
17089        assert_eq!(current_ancient.slot(), slot1_ancient);
17090
17091        // current is ancient1
17092        // try to move ancient2
17093        // current should become ancient2
17094        let slot2_ancient = 2;
17095        let mut current_ancient = CurrentAncientAccountsFile::new(slot1_ancient, ancient1.clone());
17096        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17097        let _existing_append_vec = db.create_and_insert_store(slot2_ancient, 1000, "test");
17098        let ancient2 = db
17099            .get_store_for_shrink(slot2_ancient, get_ancient_append_vec_capacity())
17100            .new_storage()
17101            .clone();
17102        let should_move = db.should_move_to_ancient_accounts_file(
17103            &ancient2,
17104            &mut current_ancient,
17105            slot2_ancient,
17106            CAN_RANDOMLY_SHRINK_FALSE,
17107        );
17108        assert!(!should_move);
17109        assert_eq!(current_ancient.id(), ancient2.id());
17110        assert_eq!(current_ancient.slot(), slot2_ancient);
17111
17112        // now try a full ancient append vec
17113        // current is None
17114        let slot3_full_ancient = 3;
17115        let mut current_ancient = CurrentAncientAccountsFile::default();
17116        // there has to be an existing append vec at this slot for a new current ancient at the slot to make sense
17117        let _existing_append_vec = db.create_and_insert_store(slot3_full_ancient, 1000, "test");
17118        let full_ancient_3 = make_full_ancient_accounts_file(&db, slot3_full_ancient, false);
17119        let should_move = db.should_move_to_ancient_accounts_file(
17120            &full_ancient_3.new_storage().clone(),
17121            &mut current_ancient,
17122            slot3_full_ancient,
17123            CAN_RANDOMLY_SHRINK_FALSE,
17124        );
17125        assert!(!should_move);
17126        assert_eq!(current_ancient.id(), full_ancient_3.new_storage().id());
17127        assert_eq!(current_ancient.slot(), slot3_full_ancient);
17128
17129        // now set current_ancient to something
17130        let mut current_ancient = CurrentAncientAccountsFile::new(slot1_ancient, ancient1.clone());
17131        let should_move = db.should_move_to_ancient_accounts_file(
17132            &full_ancient_3.new_storage().clone(),
17133            &mut current_ancient,
17134            slot3_full_ancient,
17135            CAN_RANDOMLY_SHRINK_FALSE,
17136        );
17137        assert!(!should_move);
17138        assert_eq!(current_ancient.id(), full_ancient_3.new_storage().id());
17139        assert_eq!(current_ancient.slot(), slot3_full_ancient);
17140
17141        // now mark the full ancient as candidate for shrink
17142        adjust_alive_bytes(full_ancient_3.new_storage(), 0);
17143
17144        // should shrink here, returning none for current
17145        let mut current_ancient = CurrentAncientAccountsFile::default();
17146        let should_move = db.should_move_to_ancient_accounts_file(
17147            &full_ancient_3.new_storage().clone(),
17148            &mut current_ancient,
17149            slot3_full_ancient,
17150            CAN_RANDOMLY_SHRINK_FALSE,
17151        );
17152        assert!(should_move);
17153        assert!(current_ancient.slot_and_accounts_file.is_none());
17154
17155        // should return true here, returning current from prior
17156        // now set current_ancient to something and see if it still goes to None
17157        let mut current_ancient = CurrentAncientAccountsFile::new(slot1_ancient, ancient1.clone());
17158        let should_move = db.should_move_to_ancient_accounts_file(
17159            &Arc::clone(full_ancient_3.new_storage()),
17160            &mut current_ancient,
17161            slot3_full_ancient,
17162            CAN_RANDOMLY_SHRINK_FALSE,
17163        );
17164        assert!(should_move);
17165        assert_eq!(current_ancient.id(), ancient1.id());
17166        assert_eq!(current_ancient.slot(), slot1_ancient);
17167    }
17168
17169    fn adjust_alive_bytes(storage: &AccountStorageEntry, alive_bytes: usize) {
17170        storage.alive_bytes.store(alive_bytes, Ordering::Release);
17171    }
17172
17173    /// cause 'ancient' to appear to contain 'len' bytes
17174    fn adjust_append_vec_len_for_tests(ancient: &AccountStorageEntry, len: usize) {
17175        assert!(is_ancient(&ancient.accounts));
17176        ancient.accounts.set_current_len_for_tests(len);
17177        adjust_alive_bytes(ancient, len);
17178    }
17179
17180    fn make_ancient_append_vec_full(ancient: &AccountStorageEntry, mark_alive: bool) {
17181        for _ in 0..100 {
17182            append_sample_data_to_storage(ancient, &Pubkey::default(), mark_alive, None);
17183        }
17184        // since we're not adding to the index, this is how we specify that all these accounts are alive
17185        adjust_alive_bytes(ancient, ancient.capacity() as usize);
17186    }
17187
17188    fn make_full_ancient_accounts_file(
17189        db: &AccountsDb,
17190        slot: Slot,
17191        mark_alive: bool,
17192    ) -> ShrinkInProgress<'_> {
17193        let full = db.get_store_for_shrink(slot, get_ancient_append_vec_capacity());
17194        make_ancient_append_vec_full(full.new_storage(), mark_alive);
17195        full
17196    }
17197
17198    define_accounts_db_test!(test_calculate_incremental_accounts_hash, |accounts_db| {
17199        let owner = Pubkey::new_unique();
17200        let mut accounts: Vec<_> = (0..10)
17201            .map(|_| (Pubkey::new_unique(), AccountSharedData::new(0, 0, &owner)))
17202            .collect();
17203
17204        // store some accounts into slot 0
17205        let slot = 0;
17206        {
17207            accounts[0].1.set_lamports(0);
17208            accounts[1].1.set_lamports(1);
17209            accounts[2].1.set_lamports(10);
17210            accounts[3].1.set_lamports(100);
17211            //accounts[4].1.set_lamports(1_000); <-- will be added next slot
17212
17213            let accounts = vec![
17214                (&accounts[0].0, &accounts[0].1),
17215                (&accounts[1].0, &accounts[1].1),
17216                (&accounts[2].0, &accounts[2].1),
17217                (&accounts[3].0, &accounts[3].1),
17218            ];
17219            accounts_db.store_cached((slot, accounts.as_slice()), None);
17220            accounts_db.add_root_and_flush_write_cache(slot);
17221        }
17222
17223        // store some accounts into slot 1
17224        let slot = slot + 1;
17225        {
17226            //accounts[0].1.set_lamports(0);      <-- unchanged
17227            accounts[1].1.set_lamports(0); /*     <-- drain account */
17228            //accounts[2].1.set_lamports(10);     <-- unchanged
17229            //accounts[3].1.set_lamports(100);    <-- unchanged
17230            accounts[4].1.set_lamports(1_000); /* <-- add account */
17231
17232            let accounts = vec![
17233                (&accounts[1].0, &accounts[1].1),
17234                (&accounts[4].0, &accounts[4].1),
17235            ];
17236            accounts_db.store_cached((slot, accounts.as_slice()), None);
17237            accounts_db.add_root_and_flush_write_cache(slot);
17238        }
17239
17240        // calculate the full accounts hash
17241        let full_accounts_hash = {
17242            accounts_db.clean_accounts(
17243                Some(slot - 1),
17244                false,
17245                &EpochSchedule::default(),
17246                OldStoragesPolicy::Leave,
17247            );
17248            let (storages, _) = accounts_db.get_snapshot_storages(..=slot);
17249            let storages = SortedStorages::new(&storages);
17250            accounts_db.calculate_accounts_hash(
17251                &CalcAccountsHashConfig::default(),
17252                &storages,
17253                HashStats::default(),
17254            )
17255        };
17256        assert_eq!(full_accounts_hash.1, 1_110);
17257        let full_accounts_hash_slot = slot;
17258
17259        // Calculate the expected full accounts hash here and ensure it matches.
17260        // Ensure the zero-lamport accounts are NOT included in the full accounts hash.
17261        let full_account_hashes = [(2, 0), (3, 0), (4, 1)].into_iter().map(|(index, _slot)| {
17262            let (pubkey, account) = &accounts[index];
17263            AccountsDb::hash_account(account, pubkey).0
17264        });
17265        let expected_accounts_hash = AccountsHash(compute_merkle_root(full_account_hashes));
17266        assert_eq!(full_accounts_hash.0, expected_accounts_hash);
17267
17268        // store accounts into slot 2
17269        let slot = slot + 1;
17270        {
17271            //accounts[0].1.set_lamports(0);         <-- unchanged
17272            //accounts[1].1.set_lamports(0);         <-- unchanged
17273            accounts[2].1.set_lamports(0); /*        <-- drain account */
17274            //accounts[3].1.set_lamports(100);       <-- unchanged
17275            //accounts[4].1.set_lamports(1_000);     <-- unchanged
17276            accounts[5].1.set_lamports(10_000); /*   <-- add account */
17277            accounts[6].1.set_lamports(100_000); /*  <-- add account */
17278            //accounts[7].1.set_lamports(1_000_000); <-- will be added next slot
17279
17280            let accounts = vec![
17281                (&accounts[2].0, &accounts[2].1),
17282                (&accounts[5].0, &accounts[5].1),
17283                (&accounts[6].0, &accounts[6].1),
17284            ];
17285            accounts_db.store_cached((slot, accounts.as_slice()), None);
17286            accounts_db.add_root_and_flush_write_cache(slot);
17287        }
17288
17289        // store accounts into slot 3
17290        let slot = slot + 1;
17291        {
17292            //accounts[0].1.set_lamports(0);          <-- unchanged
17293            //accounts[1].1.set_lamports(0);          <-- unchanged
17294            //accounts[2].1.set_lamports(0);          <-- unchanged
17295            accounts[3].1.set_lamports(0); /*         <-- drain account */
17296            //accounts[4].1.set_lamports(1_000);      <-- unchanged
17297            accounts[5].1.set_lamports(0); /*         <-- drain account */
17298            //accounts[6].1.set_lamports(100_000);    <-- unchanged
17299            accounts[7].1.set_lamports(1_000_000); /* <-- add account */
17300
17301            let accounts = vec![
17302                (&accounts[3].0, &accounts[3].1),
17303                (&accounts[5].0, &accounts[5].1),
17304                (&accounts[7].0, &accounts[7].1),
17305            ];
17306            accounts_db.store_cached((slot, accounts.as_slice()), None);
17307            accounts_db.add_root_and_flush_write_cache(slot);
17308        }
17309
17310        // calculate the incremental accounts hash
17311        let incremental_accounts_hash = {
17312            accounts_db.set_latest_full_snapshot_slot(full_accounts_hash_slot);
17313            accounts_db.clean_accounts(
17314                Some(slot - 1),
17315                false,
17316                &EpochSchedule::default(),
17317                OldStoragesPolicy::Leave,
17318            );
17319            let (storages, _) =
17320                accounts_db.get_snapshot_storages(full_accounts_hash_slot + 1..=slot);
17321            let storages = SortedStorages::new(&storages);
17322            accounts_db.calculate_incremental_accounts_hash(
17323                &CalcAccountsHashConfig::default(),
17324                &storages,
17325                HashStats::default(),
17326            )
17327        };
17328        assert_eq!(incremental_accounts_hash.1, 1_100_000);
17329
17330        // Ensure the zero-lamport accounts are included in the IAH.
17331        // Accounts 2, 3, and 5 are all zero-lamports.
17332        let incremental_account_hashes =
17333            [(2, 2), (3, 3), (5, 3), (6, 2), (7, 3)]
17334                .into_iter()
17335                .map(|(index, _slot)| {
17336                    let (pubkey, account) = &accounts[index];
17337                    if account.is_zero_lamport() {
17338                        // For incremental accounts hash, the hash of a zero lamport account is the hash of its pubkey.
17339                        // Ensure this implementation detail remains in sync with AccountsHasher::de_dup_in_parallel().
17340                        let hash = blake3::hash(bytemuck::bytes_of(pubkey));
17341                        Hash::new_from_array(hash.into())
17342                    } else {
17343                        AccountsDb::hash_account(account, pubkey).0
17344                    }
17345                });
17346        let expected_accounts_hash =
17347            IncrementalAccountsHash(compute_merkle_root(incremental_account_hashes));
17348        assert_eq!(incremental_accounts_hash.0, expected_accounts_hash);
17349    });
17350
17351    fn compute_merkle_root(hashes: impl IntoIterator<Item = Hash>) -> Hash {
17352        let hashes = hashes.into_iter().collect();
17353        AccountsHasher::compute_merkle_root_recurse(hashes, MERKLE_FANOUT)
17354    }
17355
17356    /// Test that `clean` reclaims old accounts when cleaning old storages
17357    ///
17358    /// When `clean` constructs candidates from old storages, pubkeys in these storages may have other
17359    /// newer versions of the accounts in other newer storages *not* explicitly marked to be visited by
17360    /// `clean`.  In this case, `clean` should still reclaim the old versions of these accounts.
17361    #[test]
17362    fn test_clean_old_storages_with_reclaims_rooted() {
17363        let accounts_db = AccountsDb::new_single_for_tests();
17364        let pubkey = Pubkey::new_unique();
17365        let old_slot = 11;
17366        let new_slot = 22;
17367        let slots = [old_slot, new_slot];
17368        for &slot in &slots {
17369            let account = AccountSharedData::new(slot, 0, &Pubkey::new_unique());
17370            // store `pubkey` into multiple slots, and also store another unique pubkey
17371            // to prevent the whole storage from being marked as dead by `clean`.
17372            accounts_db.store_for_tests(
17373                slot,
17374                &[(&pubkey, &account), (&Pubkey::new_unique(), &account)],
17375            );
17376            accounts_db.add_root_and_flush_write_cache(slot);
17377            // ensure this slot is *not* in the dirty_stores or uncleaned_pubkeys, because we want to
17378            // test cleaning *old* storages, i.e. when they aren't explicitly marked for cleaning
17379            assert!(!accounts_db.dirty_stores.contains_key(&slot));
17380            assert!(!accounts_db.uncleaned_pubkeys.contains_key(&slot));
17381        }
17382
17383        // add `old_slot` to the dirty stores list to mimic it being picked up as old
17384        let old_storage = accounts_db
17385            .storage
17386            .get_slot_storage_entry_shrinking_in_progress_ok(old_slot)
17387            .unwrap();
17388        accounts_db.dirty_stores.insert(old_slot, old_storage);
17389
17390        // ensure the slot list for `pubkey` has both the old and new slots
17391        let slot_list = accounts_db
17392            .accounts_index
17393            .get_bin(&pubkey)
17394            .slot_list_mut(&pubkey, |slot_list| slot_list.clone())
17395            .unwrap();
17396        assert_eq!(slot_list.len(), slots.len());
17397        assert!(slot_list.iter().map(|(slot, _)| slot).eq(slots.iter()));
17398
17399        // `clean` should now reclaim the account in `old_slot`, even though `new_slot` is not
17400        // explicitly being cleaned
17401        accounts_db.clean_accounts_for_tests();
17402
17403        // ensure we've reclaimed the account in `old_slot`
17404        let slot_list = accounts_db
17405            .accounts_index
17406            .get_bin(&pubkey)
17407            .slot_list_mut(&pubkey, |slot_list| slot_list.clone())
17408            .unwrap();
17409        assert_eq!(slot_list.len(), 1);
17410        assert!(slot_list
17411            .iter()
17412            .map(|(slot, _)| slot)
17413            .eq(iter::once(&new_slot)));
17414    }
17415
17416    /// Test that `clean` respects rooted vs unrooted slots w.r.t. reclaims
17417    ///
17418    /// When an account is in multiple slots, and the latest is unrooted, `clean` should *not* reclaim
17419    /// all the rooted versions.
17420    #[test]
17421    fn test_clean_old_storages_with_reclaims_unrooted() {
17422        let accounts_db = AccountsDb::new_single_for_tests();
17423        let pubkey = Pubkey::new_unique();
17424        let old_slot = 11;
17425        let new_slot = 22;
17426        let slots = [old_slot, new_slot];
17427        for &slot in &slots {
17428            let account = AccountSharedData::new(slot, 0, &Pubkey::new_unique());
17429            // store `pubkey` into multiple slots, and also store another unique pubkey
17430            // to prevent the whole storage from being marked as dead by `clean`.
17431            accounts_db.store_for_tests(
17432                slot,
17433                &[(&pubkey, &account), (&Pubkey::new_unique(), &account)],
17434            );
17435            accounts_db.calculate_accounts_delta_hash(slot);
17436            // ensure this slot is in uncleaned_pubkeys (but not dirty_stores) so it'll be cleaned
17437            assert!(!accounts_db.dirty_stores.contains_key(&slot));
17438            assert!(accounts_db.uncleaned_pubkeys.contains_key(&slot));
17439        }
17440
17441        // only `old_slot` should be rooted, not `new_slot`
17442        accounts_db.add_root_and_flush_write_cache(old_slot);
17443        assert!(accounts_db.accounts_index.is_alive_root(old_slot));
17444        assert!(!accounts_db.accounts_index.is_alive_root(new_slot));
17445
17446        // ensure the slot list for `pubkey` has both the old and new slots
17447        let slot_list = accounts_db
17448            .accounts_index
17449            .get_bin(&pubkey)
17450            .slot_list_mut(&pubkey, |slot_list| slot_list.clone())
17451            .unwrap();
17452        assert_eq!(slot_list.len(), slots.len());
17453        assert!(slot_list.iter().map(|(slot, _)| slot).eq(slots.iter()));
17454
17455        // `clean` should *not* reclaim the account in `old_slot` because `new_slot` is not a root
17456        accounts_db.clean_accounts_for_tests();
17457
17458        // ensure we have NOT reclaimed the account in `old_slot`
17459        let slot_list = accounts_db
17460            .accounts_index
17461            .get_bin(&pubkey)
17462            .slot_list_mut(&pubkey, |slot_list| slot_list.clone())
17463            .unwrap();
17464        assert_eq!(slot_list.len(), slots.len());
17465        assert!(slot_list.iter().map(|(slot, _)| slot).eq(slots.iter()));
17466    }
17467}