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;
24pub mod tests;
25
26#[cfg(feature = "dev-context-only-utils")]
27use qualifier_attr::qualifiers;
28use {
29    crate::{
30        account_info::{AccountInfo, Offset, StorageLocation},
31        account_storage::{
32            meta::StoredAccountMeta, AccountStorage, AccountStorageStatus, ShrinkInProgress,
33        },
34        accounts_cache::{AccountsCache, CachedAccount, SlotCache},
35        accounts_db::stats::{
36            AccountsStats, CleanAccountsStats, FlushStats, PurgeStats, ShrinkAncientStats,
37            ShrinkStats, ShrinkStatsSub, StoreAccountsTiming,
38        },
39        accounts_file::{
40            AccountsFile, AccountsFileError, AccountsFileProvider, MatchAccountOwnerError,
41            StorageAccess, ALIGN_BOUNDARY_OFFSET,
42        },
43        accounts_hash::{
44            AccountHash, AccountLtHash, AccountsDeltaHash, AccountsHash, AccountsHashKind,
45            AccountsHasher, AccountsLtHash, CalcAccountsHashConfig, CalculateHashIntermediate,
46            HashStats, IncrementalAccountsHash, SerdeAccountsDeltaHash, SerdeAccountsHash,
47            SerdeIncrementalAccountsHash, ZeroLamportAccounts, ZERO_LAMPORT_ACCOUNT_HASH,
48            ZERO_LAMPORT_ACCOUNT_LT_HASH,
49        },
50        accounts_index::{
51            in_mem_accounts_index::StartupStats, AccountSecondaryIndexes, AccountsIndex,
52            AccountsIndexConfig, AccountsIndexRootsStats, AccountsIndexScanResult, DiskIndexValue,
53            IndexKey, IndexValue, IsCached, RefCount, ScanConfig, ScanFilter, ScanResult, SlotList,
54            UpsertReclaim, ZeroLamport, ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS,
55            ACCOUNTS_INDEX_CONFIG_FOR_TESTING,
56        },
57        accounts_index_storage::Startup,
58        accounts_partition::RentPayingAccountsByPartition,
59        accounts_update_notifier_interface::AccountsUpdateNotifier,
60        active_stats::{ActiveStatItem, ActiveStats},
61        ancestors::Ancestors,
62        ancient_append_vecs::{
63            get_ancient_append_vec_capacity, is_ancient, AccountsToStore, StorageSelector,
64        },
65        append_vec::{aligned_stored_size, STORE_META_OVERHEAD},
66        cache_hash_data::{CacheHashData, DeletionPolicy as CacheHashDeletionPolicy},
67        contains::Contains,
68        epoch_accounts_hash::EpochAccountsHashManager,
69        partitioned_rewards::{
70            PartitionedEpochRewardsConfig, DEFAULT_PARTITIONED_EPOCH_REWARDS_CONFIG,
71        },
72        read_only_accounts_cache::ReadOnlyAccountsCache,
73        sorted_storages::SortedStorages,
74        storable_accounts::{StorableAccounts, StorableAccountsBySlot},
75        u64_align, utils,
76        verify_accounts_hash_in_background::VerifyAccountsHashInBackground,
77    },
78    crossbeam_channel::{unbounded, Receiver, Sender, TryRecvError},
79    dashmap::{DashMap, DashSet},
80    log::*,
81    rand::{thread_rng, Rng},
82    rayon::{prelude::*, ThreadPool},
83    seqlock::SeqLock,
84    smallvec::SmallVec,
85    solana_lattice_hash::lt_hash::LtHash,
86    solana_measure::{meas_dur, measure::Measure, measure_us},
87    solana_nohash_hasher::{BuildNoHashHasher, IntMap, IntSet},
88    solana_pubkey::Pubkey,
89    solana_rayon_threadlimit::get_thread_count,
90    solana_sdk::{
91        account::{Account, AccountSharedData, ReadableAccount},
92        clock::{BankId, Epoch, Slot},
93        epoch_schedule::EpochSchedule,
94        genesis_config::GenesisConfig,
95        hash::Hash,
96        rent_collector::RentCollector,
97        saturating_add_assign,
98        transaction::SanitizedTransaction,
99    },
100    std::{
101        borrow::Cow,
102        boxed::Box,
103        collections::{BTreeSet, HashMap, HashSet, VecDeque},
104        fs,
105        hash::{Hash as StdHash, Hasher as StdHasher},
106        io::Result as IoResult,
107        iter,
108        num::{NonZeroUsize, Saturating},
109        ops::{Range, RangeBounds},
110        path::{Path, PathBuf},
111        sync::{
112            atomic::{AtomicBool, AtomicU32, AtomicU64, AtomicUsize, Ordering},
113            Arc, Condvar, Mutex, RwLock,
114        },
115        thread::{sleep, Builder},
116        time::{Duration, Instant},
117    },
118    tempfile::TempDir,
119};
120
121// when the accounts write cache exceeds this many bytes, we will flush it
122// this can be specified on the command line, too (--accounts-db-cache-limit-mb)
123const WRITE_CACHE_LIMIT_BYTES_DEFAULT: u64 = 15_000_000_000;
124const SCAN_SLOT_PAR_ITER_THRESHOLD: usize = 4000;
125
126const UNREF_ACCOUNTS_BATCH_SIZE: usize = 10_000;
127
128const DEFAULT_FILE_SIZE: u64 = 4 * 1024 * 1024;
129const DEFAULT_NUM_DIRS: u32 = 4;
130
131// When calculating hashes, it is helpful to break the pubkeys found into bins based on the pubkey value.
132// More bins means smaller vectors to sort, copy, etc.
133pub const DEFAULT_HASH_CALCULATION_PUBKEY_BINS: usize = 65536;
134
135// Without chunks, we end up with 1 output vec for each outer snapshot storage.
136// This results in too many vectors to be efficient.
137// Chunks when scanning storages to calculate hashes.
138// If this is too big, we don't get enough parallelism of scanning storages.
139// If this is too small, then we produce too many output vectors to iterate.
140// Metrics indicate a sweet spot in the 2.5k-5k range for mnb.
141const MAX_ITEMS_PER_CHUNK: Slot = 2_500;
142
143// When getting accounts for shrinking from the index, this is the # of accounts to lookup per thread.
144// This allows us to split up accounts index accesses across multiple threads.
145const SHRINK_COLLECT_CHUNK_SIZE: usize = 50;
146
147/// The number of shrink candidate slots that is small enough so that
148/// additional storages from ancient slots can be added to the
149/// candidates for shrinking.
150const SHRINK_INSERT_ANCIENT_THRESHOLD: usize = 10;
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 StoreTo<'_> {
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    read_cache_evict_sample_size: None,
506    write_cache_limit_bytes: None,
507    ancient_append_vec_offset: None,
508    ancient_storage_ideal_size: None,
509    max_ancient_storages: None,
510    skip_initial_hash_calc: false,
511    exhaustively_verify_refcounts: false,
512    create_ancient_storage: CreateAncientStorage::Pack,
513    partitioned_epoch_rewards_config: DEFAULT_PARTITIONED_EPOCH_REWARDS_CONFIG,
514    test_skip_rewrites_but_include_in_bank_hash: false,
515    storage_access: StorageAccess::File,
516    scan_filter_for_shrinking: ScanFilter::OnlyAbnormalWithVerify,
517    enable_experimental_accumulator_hash: false,
518    verify_experimental_accumulator_hash: false,
519    snapshots_use_experimental_accumulator_hash: false,
520    num_clean_threads: None,
521    num_foreground_threads: None,
522    num_hash_threads: None,
523    hash_calculation_pubkey_bins: Some(4),
524};
525pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig {
526    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS),
527    account_indexes: None,
528    base_working_path: None,
529    accounts_hash_cache_path: None,
530    shrink_paths: None,
531    shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION,
532    read_cache_limit_bytes: None,
533    read_cache_evict_sample_size: None,
534    write_cache_limit_bytes: None,
535    ancient_append_vec_offset: None,
536    ancient_storage_ideal_size: None,
537    max_ancient_storages: None,
538    skip_initial_hash_calc: false,
539    exhaustively_verify_refcounts: false,
540    create_ancient_storage: CreateAncientStorage::Pack,
541    partitioned_epoch_rewards_config: DEFAULT_PARTITIONED_EPOCH_REWARDS_CONFIG,
542    test_skip_rewrites_but_include_in_bank_hash: false,
543    storage_access: StorageAccess::File,
544    scan_filter_for_shrinking: ScanFilter::OnlyAbnormalWithVerify,
545    enable_experimental_accumulator_hash: false,
546    verify_experimental_accumulator_hash: false,
547    snapshots_use_experimental_accumulator_hash: false,
548    num_clean_threads: None,
549    num_foreground_threads: None,
550    num_hash_threads: None,
551    hash_calculation_pubkey_bins: None,
552};
553
554pub type BinnedHashData = Vec<Vec<CalculateHashIntermediate>>;
555
556struct LoadAccountsIndexForShrink<'a, T: ShrinkCollectRefs<'a>> {
557    /// all alive accounts
558    alive_accounts: T,
559    /// pubkeys that are going to be unref'd in the accounts index after we are
560    /// done with shrinking, because they are dead
561    pubkeys_to_unref: Vec<&'a Pubkey>,
562    /// pubkeys that are the last remaining zero lamport instance of an account
563    zero_lamport_single_ref_pubkeys: Vec<&'a Pubkey>,
564    /// true if all alive accounts are zero lamport accounts
565    all_are_zero_lamports: bool,
566}
567
568/// reference an account found during scanning a storage. This is a byval struct to replace
569/// `StoredAccountMeta`
570#[derive(Debug, PartialEq, Copy, Clone)]
571pub struct AccountFromStorage {
572    pub index_info: AccountInfo,
573    pub data_len: u64,
574    pub pubkey: Pubkey,
575}
576
577impl ZeroLamport for AccountFromStorage {
578    fn is_zero_lamport(&self) -> bool {
579        self.index_info.is_zero_lamport()
580    }
581}
582
583impl AccountFromStorage {
584    pub fn pubkey(&self) -> &Pubkey {
585        &self.pubkey
586    }
587    pub fn stored_size(&self) -> usize {
588        aligned_stored_size(self.data_len as usize)
589    }
590    pub fn data_len(&self) -> usize {
591        self.data_len as usize
592    }
593    pub fn new(account: &StoredAccountMeta) -> Self {
594        // the id is irrelevant in this account info. This structure is only used DURING shrink operations.
595        // In those cases, there is only 1 append vec id per slot when we read the accounts.
596        // Any value of storage id in account info works fine when we want the 'normal' storage.
597        let storage_id = 0;
598        AccountFromStorage {
599            index_info: AccountInfo::new(
600                StorageLocation::AppendVec(storage_id, account.offset()),
601                account.lamports(),
602            ),
603            pubkey: *account.pubkey(),
604            data_len: account.data_len() as u64,
605        }
606    }
607}
608
609pub struct GetUniqueAccountsResult {
610    pub stored_accounts: Vec<AccountFromStorage>,
611    pub capacity: u64,
612    pub num_duplicated_accounts: usize,
613}
614
615pub struct AccountsAddRootTiming {
616    pub index_us: u64,
617    pub cache_us: u64,
618    pub store_us: u64,
619}
620
621/// Slots older the "number of slots in an epoch minus this number"
622/// than max root are treated as ancient and subject to packing.
623/// |  older  |<-          slots in an epoch          ->| max root
624/// |  older  |<-    offset   ->|                       |
625/// |          ancient          |        modern         |
626///
627/// If this is negative, this many slots older than the number of
628/// slots in epoch are still treated as modern (ie. non-ancient).
629/// |  older  |<- abs(offset) ->|<- slots in an epoch ->| max root
630/// | ancient |                 modern                  |
631///
632/// Note that another constant DEFAULT_MAX_ANCIENT_STORAGES sets a
633/// threshold for combining ancient storages so that their overall
634/// number is under a certain limit, whereas this constant establishes
635/// the distance from the max root slot beyond which storages holding
636/// the account data for the slots are considered ancient by the
637/// shrinking algorithm.
638const ANCIENT_APPEND_VEC_DEFAULT_OFFSET: Option<i64> = Some(100_000);
639/// The smallest size of ideal ancient storage.
640/// The setting can be overridden on the command line
641/// with --accounts-db-ancient-ideal-storage-size option.
642const DEFAULT_ANCIENT_STORAGE_IDEAL_SIZE: u64 = 100_000;
643/// Default value for the number of ancient storages the ancient slot
644/// combining should converge to.
645pub const DEFAULT_MAX_ANCIENT_STORAGES: usize = 100_000;
646
647#[derive(Debug, Default, Clone)]
648pub struct AccountsDbConfig {
649    pub index: Option<AccountsIndexConfig>,
650    pub account_indexes: Option<AccountSecondaryIndexes>,
651    /// Base directory for various necessary files
652    pub base_working_path: Option<PathBuf>,
653    pub accounts_hash_cache_path: Option<PathBuf>,
654    pub shrink_paths: Option<Vec<PathBuf>>,
655    pub shrink_ratio: AccountShrinkThreshold,
656    /// The low and high watermark sizes for the read cache, in bytes.
657    /// If None, defaults will be used.
658    pub read_cache_limit_bytes: Option<(usize, usize)>,
659    /// The number of elements that will be randomly sampled at eviction time,
660    /// the oldest of which will get evicted.
661    pub read_cache_evict_sample_size: Option<usize>,
662    pub write_cache_limit_bytes: Option<u64>,
663    /// if None, ancient append vecs are set to ANCIENT_APPEND_VEC_DEFAULT_OFFSET
664    /// Some(offset) means include slots up to (max_slot - (slots_per_epoch - 'offset'))
665    pub ancient_append_vec_offset: Option<i64>,
666    pub ancient_storage_ideal_size: Option<u64>,
667    pub max_ancient_storages: Option<usize>,
668    pub hash_calculation_pubkey_bins: Option<usize>,
669    pub test_skip_rewrites_but_include_in_bank_hash: bool,
670    pub skip_initial_hash_calc: bool,
671    pub exhaustively_verify_refcounts: bool,
672    /// how to create ancient storages
673    pub create_ancient_storage: CreateAncientStorage,
674    pub partitioned_epoch_rewards_config: PartitionedEpochRewardsConfig,
675    pub storage_access: StorageAccess,
676    pub scan_filter_for_shrinking: ScanFilter,
677    pub enable_experimental_accumulator_hash: bool,
678    pub verify_experimental_accumulator_hash: bool,
679    pub snapshots_use_experimental_accumulator_hash: bool,
680    /// Number of threads for background cleaning operations (`thread_pool_clean')
681    pub num_clean_threads: Option<NonZeroUsize>,
682    /// Number of threads for foreground operations (`thread_pool`)
683    pub num_foreground_threads: Option<NonZeroUsize>,
684    /// Number of threads for background accounts hashing (`thread_pool_hash`)
685    pub num_hash_threads: Option<NonZeroUsize>,
686}
687
688#[cfg(not(test))]
689const ABSURD_CONSECUTIVE_FAILED_ITERATIONS: usize = 100;
690
691#[derive(Debug, Clone, Copy)]
692pub enum AccountShrinkThreshold {
693    /// Measure the total space sparseness across all candidates
694    /// And select the candidates by using the top sparse account storage entries to shrink.
695    /// The value is the overall shrink threshold measured as ratio of the total live bytes
696    /// over the total bytes.
697    TotalSpace { shrink_ratio: f64 },
698    /// Use the following option to shrink all stores whose alive ratio is below
699    /// the specified threshold.
700    IndividualStore { shrink_ratio: f64 },
701}
702pub const DEFAULT_ACCOUNTS_SHRINK_OPTIMIZE_TOTAL_SPACE: bool = true;
703pub const DEFAULT_ACCOUNTS_SHRINK_RATIO: f64 = 0.80;
704// The default extra account space in percentage from the ideal target
705const DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION: AccountShrinkThreshold =
706    AccountShrinkThreshold::TotalSpace {
707        shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_RATIO,
708    };
709
710impl Default for AccountShrinkThreshold {
711    fn default() -> AccountShrinkThreshold {
712        DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION
713    }
714}
715
716pub enum ScanStorageResult<R, B> {
717    Cached(Vec<R>),
718    Stored(B),
719}
720
721#[derive(Debug, Default)]
722pub struct IndexGenerationInfo {
723    pub accounts_data_len: u64,
724    pub rent_paying_accounts_by_partition: RentPayingAccountsByPartition,
725    /// The lt hash of the old/duplicate accounts identified during index generation.
726    /// Will be used when verifying the accounts lt hash, after rebuilding a Bank.
727    pub duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
728}
729
730#[derive(Debug, Default)]
731struct SlotIndexGenerationInfo {
732    insert_time_us: u64,
733    num_accounts: u64,
734    num_accounts_rent_paying: usize,
735    accounts_data_len: u64,
736    amount_to_top_off_rent: u64,
737    rent_paying_accounts_by_partition: Vec<Pubkey>,
738    zero_lamport_pubkeys: Vec<Pubkey>,
739    all_accounts_are_zero_lamports: bool,
740}
741
742/// The lt hash of old/duplicate accounts
743///
744/// Accumulation of all the duplicate accounts found during index generation.
745/// These accounts need to have their lt hashes mixed *out*.
746/// This is the final value, that when applied to all the storages at startup,
747/// will produce the correct accounts lt hash.
748#[derive(Debug, Clone)]
749pub struct DuplicatesLtHash(pub LtHash);
750
751impl Default for DuplicatesLtHash {
752    fn default() -> Self {
753        Self(LtHash::identity())
754    }
755}
756
757#[derive(Default, Debug)]
758struct GenerateIndexTimings {
759    pub total_time_us: u64,
760    pub index_time: u64,
761    pub scan_time: u64,
762    pub insertion_time_us: u64,
763    pub min_bin_size_in_mem: usize,
764    pub max_bin_size_in_mem: usize,
765    pub total_items_in_mem: usize,
766    pub storage_size_storages_us: u64,
767    pub index_flush_us: u64,
768    pub rent_paying: AtomicUsize,
769    pub amount_to_top_off_rent: AtomicU64,
770    pub total_including_duplicates: u64,
771    pub accounts_data_len_dedup_time_us: u64,
772    pub total_duplicate_slot_keys: u64,
773    pub total_num_unique_duplicate_keys: u64,
774    pub num_duplicate_accounts: u64,
775    pub populate_duplicate_keys_us: u64,
776    pub total_slots: u64,
777    pub slots_to_clean: u64,
778    pub par_duplicates_lt_hash_us: AtomicU64,
779    pub visit_zero_lamports_us: u64,
780    pub num_zero_lamport_single_refs: u64,
781    pub all_accounts_are_zero_lamports_slots: u64,
782}
783
784#[derive(Default, Debug, PartialEq, Eq)]
785struct StorageSizeAndCount {
786    /// total size stored, including both alive and dead bytes
787    pub stored_size: usize,
788    /// number of accounts in the storage including both alive and dead accounts
789    pub count: usize,
790}
791type StorageSizeAndCountMap =
792    DashMap<AccountsFileId, StorageSizeAndCount, BuildNoHashHasher<AccountsFileId>>;
793
794impl GenerateIndexTimings {
795    pub fn report(&self, startup_stats: &StartupStats) {
796        datapoint_info!(
797            "generate_index",
798            ("overall_us", self.total_time_us, i64),
799            // we cannot accurately measure index insertion time because of many threads and lock contention
800            ("total_us", self.index_time, i64),
801            ("scan_stores_us", self.scan_time, i64),
802            ("insertion_time_us", self.insertion_time_us, i64),
803            ("min_bin_size_in_mem", self.min_bin_size_in_mem, i64),
804            ("max_bin_size_in_mem", self.max_bin_size_in_mem, i64),
805            (
806                "storage_size_storages_us",
807                self.storage_size_storages_us,
808                i64
809            ),
810            ("index_flush_us", self.index_flush_us, i64),
811            (
812                "total_rent_paying",
813                self.rent_paying.load(Ordering::Relaxed),
814                i64
815            ),
816            (
817                "amount_to_top_off_rent",
818                self.amount_to_top_off_rent.load(Ordering::Relaxed),
819                i64
820            ),
821            (
822                "total_items_including_duplicates",
823                self.total_including_duplicates,
824                i64
825            ),
826            ("total_items_in_mem", self.total_items_in_mem, i64),
827            (
828                "accounts_data_len_dedup_time_us",
829                self.accounts_data_len_dedup_time_us,
830                i64
831            ),
832            (
833                "total_duplicate_slot_keys",
834                self.total_duplicate_slot_keys,
835                i64
836            ),
837            (
838                "total_num_unique_duplicate_keys",
839                self.total_num_unique_duplicate_keys,
840                i64
841            ),
842            ("num_duplicate_accounts", self.num_duplicate_accounts, i64),
843            (
844                "populate_duplicate_keys_us",
845                self.populate_duplicate_keys_us,
846                i64
847            ),
848            ("total_slots", self.total_slots, i64),
849            ("slots_to_clean", self.slots_to_clean, i64),
850            (
851                "copy_data_us",
852                startup_stats.copy_data_us.swap(0, Ordering::Relaxed),
853                i64
854            ),
855            (
856                "par_duplicates_lt_hash_us",
857                self.par_duplicates_lt_hash_us.load(Ordering::Relaxed),
858                i64
859            ),
860            (
861                "num_zero_lamport_single_refs",
862                self.num_zero_lamport_single_refs,
863                i64
864            ),
865            ("visit_zero_lamports_us", self.visit_zero_lamports_us, i64),
866            (
867                "all_accounts_are_zero_lamports_slots",
868                self.all_accounts_are_zero_lamports_slots,
869                i64
870            ),
871        );
872    }
873}
874
875impl IndexValue for AccountInfo {}
876impl DiskIndexValue for AccountInfo {}
877
878impl ZeroLamport for AccountSharedData {
879    fn is_zero_lamport(&self) -> bool {
880        self.lamports() == 0
881    }
882}
883
884impl ZeroLamport for Account {
885    fn is_zero_lamport(&self) -> bool {
886        self.lamports() == 0
887    }
888}
889
890struct MultiThreadProgress<'a> {
891    last_update: Instant,
892    my_last_report_count: u64,
893    total_count: &'a AtomicU64,
894    report_delay_secs: u64,
895    first_caller: bool,
896    ultimate_count: u64,
897    start_time: Instant,
898}
899
900impl<'a> MultiThreadProgress<'a> {
901    fn new(total_count: &'a AtomicU64, report_delay_secs: u64, ultimate_count: u64) -> Self {
902        Self {
903            last_update: Instant::now(),
904            my_last_report_count: 0,
905            total_count,
906            report_delay_secs,
907            first_caller: false,
908            ultimate_count,
909            start_time: Instant::now(),
910        }
911    }
912    fn report(&mut self, my_current_count: u64) {
913        let now = Instant::now();
914        if now.duration_since(self.last_update).as_secs() >= self.report_delay_secs {
915            let my_total_newly_processed_slots_since_last_report =
916                my_current_count - self.my_last_report_count;
917
918            self.my_last_report_count = my_current_count;
919            let previous_total_processed_slots_across_all_threads = self.total_count.fetch_add(
920                my_total_newly_processed_slots_since_last_report,
921                Ordering::Relaxed,
922            );
923            self.first_caller =
924                self.first_caller || 0 == previous_total_processed_slots_across_all_threads;
925            if self.first_caller {
926                let total = previous_total_processed_slots_across_all_threads
927                    + my_total_newly_processed_slots_since_last_report;
928                info!(
929                    "generating index: {}/{} slots... ({}/s)",
930                    total,
931                    self.ultimate_count,
932                    total / self.start_time.elapsed().as_secs().max(1),
933                );
934            }
935            self.last_update = now;
936        }
937    }
938}
939
940/// An offset into the AccountsDb::storage vector
941pub type AtomicAccountsFileId = AtomicU32;
942pub type AccountsFileId = u32;
943
944type AccountSlots = HashMap<Pubkey, IntSet<Slot>>;
945type SlotOffsets = IntMap<Slot, IntSet<Offset>>;
946type ReclaimResult = (AccountSlots, SlotOffsets);
947type PubkeysRemovedFromAccountsIndex = HashSet<Pubkey>;
948type ShrinkCandidates = IntSet<Slot>;
949
950// Some hints for applicability of additional sanity checks for the do_load fast-path;
951// Slower fallback code path will be taken if the fast path has failed over the retry
952// threshold, regardless of these hints. Also, load cannot fail not-deterministically
953// even under very rare circumstances, unlike previously did allow.
954#[derive(Clone, Copy, Debug, PartialEq, Eq)]
955pub enum LoadHint {
956    // Caller hints that it's loading transactions for a block which is
957    // descended from the current root, and at the tip of its fork.
958    // Thereby, further this assumes AccountIndex::max_root should not increase
959    // during this load, meaning there should be no squash.
960    // Overall, this enables us to assert!() strictly while running the fast-path for
961    // account loading, while maintaining the determinism of account loading and resultant
962    // transaction execution thereof.
963    FixedMaxRoot,
964    /// same as `FixedMaxRoot`, except do not populate the read cache on load
965    FixedMaxRootDoNotPopulateReadCache,
966    // Caller can't hint the above safety assumption. Generally RPC and miscellaneous
967    // other call-site falls into this category. The likelihood of slower path is slightly
968    // increased as well.
969    Unspecified,
970}
971
972#[derive(Debug)]
973pub enum LoadedAccountAccessor<'a> {
974    // StoredAccountMeta can't be held directly here due to its lifetime dependency to
975    // AccountStorageEntry
976    Stored(Option<(Arc<AccountStorageEntry>, usize)>),
977    // None value in Cached variant means the cache was flushed
978    Cached(Option<Cow<'a, CachedAccount>>),
979}
980
981impl<'a> LoadedAccountAccessor<'a> {
982    fn check_and_get_loaded_account_shared_data(&mut self) -> AccountSharedData {
983        // all of these following .expect() and .unwrap() are like serious logic errors,
984        // ideal for representing this as rust type system....
985
986        match self {
987            LoadedAccountAccessor::Stored(Some((maybe_storage_entry, offset))) => {
988                // If we do find the storage entry, we can guarantee that the storage entry is
989                // safe to read from because we grabbed a reference to the storage entry while it
990                // was still in the storage map. This means even if the storage entry is removed
991                // from the storage map after we grabbed the storage entry, the recycler should not
992                // reset the storage entry until we drop the reference to the storage entry.
993                maybe_storage_entry.get_account_shared_data(*offset).expect(
994                    "If a storage entry was found in the storage map, it must not have been reset \
995                     yet",
996                )
997            }
998            _ => self.check_and_get_loaded_account(|loaded_account| loaded_account.take_account()),
999        }
1000    }
1001
1002    fn check_and_get_loaded_account<T>(
1003        &mut self,
1004        callback: impl for<'local> FnMut(LoadedAccount<'local>) -> T,
1005    ) -> T {
1006        // all of these following .expect() and .unwrap() are like serious logic errors,
1007        // ideal for representing this as rust type system....
1008
1009        match self {
1010            LoadedAccountAccessor::Cached(None) | LoadedAccountAccessor::Stored(None) => {
1011                panic!(
1012                    "Should have already been taken care of when creating this \
1013                     LoadedAccountAccessor"
1014                );
1015            }
1016            LoadedAccountAccessor::Cached(Some(_cached_account)) => {
1017                // Cached(Some(x)) variant always produces `Some` for get_loaded_account() since
1018                // it just returns the inner `x` without additional fetches
1019                self.get_loaded_account(callback).unwrap()
1020            }
1021            LoadedAccountAccessor::Stored(Some(_maybe_storage_entry)) => {
1022                // If we do find the storage entry, we can guarantee that the storage entry is
1023                // safe to read from because we grabbed a reference to the storage entry while it
1024                // was still in the storage map. This means even if the storage entry is removed
1025                // from the storage map after we grabbed the storage entry, the recycler should not
1026                // reset the storage entry until we drop the reference to the storage entry.
1027                self.get_loaded_account(callback).expect(
1028                    "If a storage entry was found in the storage map, it must not have been reset \
1029                     yet",
1030                )
1031            }
1032        }
1033    }
1034
1035    fn get_loaded_account<T>(
1036        &mut self,
1037        mut callback: impl for<'local> FnMut(LoadedAccount<'local>) -> T,
1038    ) -> Option<T> {
1039        match self {
1040            LoadedAccountAccessor::Cached(cached_account) => {
1041                let cached_account: Cow<'a, CachedAccount> = cached_account.take().expect(
1042                    "Cache flushed/purged should be handled before trying to fetch account",
1043                );
1044                Some(callback(LoadedAccount::Cached(cached_account)))
1045            }
1046            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
1047                // storage entry may not be present if slot was cleaned up in
1048                // between reading the accounts index and calling this function to
1049                // get account meta from the storage entry here
1050                maybe_storage_entry
1051                    .as_ref()
1052                    .and_then(|(storage_entry, offset)| {
1053                        storage_entry
1054                            .accounts
1055                            .get_stored_account_meta_callback(*offset, |account| {
1056                                callback(LoadedAccount::Stored(account))
1057                            })
1058                    })
1059            }
1060        }
1061    }
1062
1063    fn account_matches_owners(&self, owners: &[Pubkey]) -> Result<usize, MatchAccountOwnerError> {
1064        match self {
1065            LoadedAccountAccessor::Cached(cached_account) => cached_account
1066                .as_ref()
1067                .and_then(|cached_account| {
1068                    if cached_account.account.is_zero_lamport() {
1069                        None
1070                    } else {
1071                        owners
1072                            .iter()
1073                            .position(|entry| cached_account.account.owner() == entry)
1074                    }
1075                })
1076                .ok_or(MatchAccountOwnerError::NoMatch),
1077            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
1078                // storage entry may not be present if slot was cleaned up in
1079                // between reading the accounts index and calling this function to
1080                // get account meta from the storage entry here
1081                maybe_storage_entry
1082                    .as_ref()
1083                    .map(|(storage_entry, offset)| {
1084                        storage_entry
1085                            .accounts
1086                            .account_matches_owners(*offset, owners)
1087                    })
1088                    .unwrap_or(Err(MatchAccountOwnerError::UnableToLoad))
1089            }
1090        }
1091    }
1092}
1093
1094pub enum LoadedAccount<'a> {
1095    Stored(StoredAccountMeta<'a>),
1096    Cached(Cow<'a, CachedAccount>),
1097}
1098
1099impl LoadedAccount<'_> {
1100    pub fn loaded_hash(&self) -> AccountHash {
1101        match self {
1102            LoadedAccount::Stored(stored_account_meta) => *stored_account_meta.hash(),
1103            LoadedAccount::Cached(cached_account) => cached_account.hash(),
1104        }
1105    }
1106
1107    pub fn pubkey(&self) -> &Pubkey {
1108        match self {
1109            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.pubkey(),
1110            LoadedAccount::Cached(cached_account) => cached_account.pubkey(),
1111        }
1112    }
1113
1114    pub fn take_account(&self) -> AccountSharedData {
1115        match self {
1116            LoadedAccount::Stored(stored_account_meta) => {
1117                stored_account_meta.to_account_shared_data()
1118            }
1119            LoadedAccount::Cached(cached_account) => match cached_account {
1120                Cow::Owned(cached_account) => cached_account.account.clone(),
1121                Cow::Borrowed(cached_account) => cached_account.account.clone(),
1122            },
1123        }
1124    }
1125
1126    pub fn is_cached(&self) -> bool {
1127        match self {
1128            LoadedAccount::Stored(_) => false,
1129            LoadedAccount::Cached(_) => true,
1130        }
1131    }
1132
1133    /// data_len can be calculated without having access to `&data` in future implementations
1134    pub fn data_len(&self) -> usize {
1135        self.data().len()
1136    }
1137}
1138
1139impl ReadableAccount for LoadedAccount<'_> {
1140    fn lamports(&self) -> u64 {
1141        match self {
1142            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.lamports(),
1143            LoadedAccount::Cached(cached_account) => cached_account.account.lamports(),
1144        }
1145    }
1146    fn data(&self) -> &[u8] {
1147        match self {
1148            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.data(),
1149            LoadedAccount::Cached(cached_account) => cached_account.account.data(),
1150        }
1151    }
1152    fn owner(&self) -> &Pubkey {
1153        match self {
1154            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.owner(),
1155            LoadedAccount::Cached(cached_account) => cached_account.account.owner(),
1156        }
1157    }
1158    fn executable(&self) -> bool {
1159        match self {
1160            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.executable(),
1161            LoadedAccount::Cached(cached_account) => cached_account.account.executable(),
1162        }
1163    }
1164    fn rent_epoch(&self) -> Epoch {
1165        match self {
1166            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.rent_epoch(),
1167            LoadedAccount::Cached(cached_account) => cached_account.account.rent_epoch(),
1168        }
1169    }
1170    fn to_account_shared_data(&self) -> AccountSharedData {
1171        self.take_account()
1172    }
1173}
1174
1175#[derive(Debug)]
1176pub enum AccountsHashVerificationError {
1177    MissingAccountsHash,
1178    MismatchedAccountsHash,
1179    MismatchedTotalLamports(u64, u64),
1180}
1181
1182#[derive(Default)]
1183struct CleanKeyTimings {
1184    collect_delta_keys_us: u64,
1185    delta_insert_us: u64,
1186    dirty_store_processing_us: u64,
1187    delta_key_count: u64,
1188    dirty_pubkeys_count: u64,
1189    oldest_dirty_slot: Slot,
1190    /// number of ancient append vecs that were scanned because they were dirty when clean started
1191    dirty_ancient_stores: usize,
1192}
1193
1194/// Persistent storage structure holding the accounts
1195#[derive(Debug)]
1196pub struct AccountStorageEntry {
1197    pub(crate) id: AccountsFileId,
1198
1199    pub(crate) slot: Slot,
1200
1201    /// storage holding the accounts
1202    pub accounts: AccountsFile,
1203
1204    /// Keeps track of the number of accounts stored in a specific AppendVec.
1205    ///  This is periodically checked to reuse the stores that do not have
1206    ///  any accounts in it
1207    /// status corresponding to the storage, lets us know that
1208    ///  the append_vec, once maxed out, then emptied, can be reclaimed
1209    count_and_status: SeqLock<(usize, AccountStorageStatus)>,
1210
1211    alive_bytes: AtomicUsize,
1212
1213    /// offsets to accounts that are zero lamport single ref stored in this
1214    /// storage. These are still alive. But, shrink will be able to remove them.
1215    ///
1216    /// NOTE: It's possible that one of these zero lamport single ref accounts
1217    /// could be written in a new transaction (and later rooted & flushed) and a
1218    /// later clean runs and marks this account dead before this storage gets a
1219    /// chance to be shrunk, thus making the account dead in both "alive_bytes"
1220    /// and as a zero lamport single ref. If this happens, we will count this
1221    /// account as "dead" twice. However, this should be fine. It just makes
1222    /// shrink more likely to visit this storage.
1223    zero_lamport_single_ref_offsets: RwLock<IntSet<Offset>>,
1224}
1225
1226impl AccountStorageEntry {
1227    pub fn new(
1228        path: &Path,
1229        slot: Slot,
1230        id: AccountsFileId,
1231        file_size: u64,
1232        provider: AccountsFileProvider,
1233    ) -> Self {
1234        let tail = AccountsFile::file_name(slot, id);
1235        let path = Path::new(path).join(tail);
1236        let accounts = provider.new_writable(path, file_size);
1237
1238        Self {
1239            id,
1240            slot,
1241            accounts,
1242            count_and_status: SeqLock::new((0, AccountStorageStatus::Available)),
1243            alive_bytes: AtomicUsize::new(0),
1244            zero_lamport_single_ref_offsets: RwLock::default(),
1245        }
1246    }
1247
1248    /// open a new instance of the storage that is readonly
1249    fn reopen_as_readonly(&self, storage_access: StorageAccess) -> Option<Self> {
1250        if storage_access != StorageAccess::File {
1251            // if we are only using mmap, then no reason to re-open
1252            return None;
1253        }
1254
1255        let count_and_status = self.count_and_status.lock_write();
1256        self.accounts.reopen_as_readonly().map(|accounts| Self {
1257            id: self.id,
1258            slot: self.slot,
1259            count_and_status: SeqLock::new(*count_and_status),
1260            alive_bytes: AtomicUsize::new(self.alive_bytes()),
1261            accounts,
1262            zero_lamport_single_ref_offsets: RwLock::default(),
1263        })
1264    }
1265
1266    pub fn new_existing(
1267        slot: Slot,
1268        id: AccountsFileId,
1269        accounts: AccountsFile,
1270        _num_accounts: usize,
1271    ) -> Self {
1272        Self {
1273            id,
1274            slot,
1275            accounts,
1276            count_and_status: SeqLock::new((0, AccountStorageStatus::Available)),
1277            alive_bytes: AtomicUsize::new(0),
1278            zero_lamport_single_ref_offsets: RwLock::default(),
1279        }
1280    }
1281
1282    pub fn set_status(&self, mut status: AccountStorageStatus) {
1283        let mut count_and_status = self.count_and_status.lock_write();
1284
1285        let count = count_and_status.0;
1286
1287        if status == AccountStorageStatus::Full && count == 0 {
1288            // this case arises when the append_vec is full (store_ptrs fails),
1289            //  but all accounts have already been removed from the storage
1290            //
1291            // the only time it's safe to call reset() on an append_vec is when
1292            //  every account has been removed
1293            //          **and**
1294            //  the append_vec has previously been completely full
1295            //
1296            self.accounts.reset();
1297            status = AccountStorageStatus::Available;
1298        }
1299
1300        *count_and_status = (count, status);
1301    }
1302
1303    pub fn status(&self) -> AccountStorageStatus {
1304        self.count_and_status.read().1
1305    }
1306
1307    pub fn count(&self) -> usize {
1308        self.count_and_status.read().0
1309    }
1310
1311    pub fn alive_bytes(&self) -> usize {
1312        self.alive_bytes.load(Ordering::Acquire)
1313    }
1314
1315    /// Return true if offset is "new" and inserted successfully. Otherwise,
1316    /// return false if the offset exists already.
1317    fn insert_zero_lamport_single_ref_account_offset(&self, offset: usize) -> bool {
1318        let mut zero_lamport_single_ref_offsets =
1319            self.zero_lamport_single_ref_offsets.write().unwrap();
1320        zero_lamport_single_ref_offsets.insert(offset)
1321    }
1322
1323    /// Return the number of zero_lamport_single_ref accounts in the storage.
1324    fn num_zero_lamport_single_ref_accounts(&self) -> usize {
1325        self.zero_lamport_single_ref_offsets.read().unwrap().len()
1326    }
1327
1328    /// Return the "alive_bytes" minus "zero_lamport_single_ref_accounts bytes".
1329    fn alive_bytes_exclude_zero_lamport_single_ref_accounts(&self) -> usize {
1330        let zero_lamport_dead_bytes = self
1331            .accounts
1332            .dead_bytes_due_to_zero_lamport_single_ref(self.num_zero_lamport_single_ref_accounts());
1333        self.alive_bytes().saturating_sub(zero_lamport_dead_bytes)
1334    }
1335
1336    pub fn written_bytes(&self) -> u64 {
1337        self.accounts.len() as u64
1338    }
1339
1340    pub fn capacity(&self) -> u64 {
1341        self.accounts.capacity()
1342    }
1343
1344    pub fn has_accounts(&self) -> bool {
1345        self.count() > 0
1346    }
1347
1348    pub fn slot(&self) -> Slot {
1349        self.slot
1350    }
1351
1352    pub fn id(&self) -> AccountsFileId {
1353        self.id
1354    }
1355
1356    pub fn flush(&self) -> Result<(), AccountsFileError> {
1357        self.accounts.flush()
1358    }
1359
1360    fn get_account_shared_data(&self, offset: usize) -> Option<AccountSharedData> {
1361        self.accounts.get_account_shared_data(offset)
1362    }
1363
1364    fn add_accounts(&self, num_accounts: usize, num_bytes: usize) {
1365        let mut count_and_status = self.count_and_status.lock_write();
1366        *count_and_status = (count_and_status.0 + num_accounts, count_and_status.1);
1367        self.alive_bytes.fetch_add(num_bytes, Ordering::Release);
1368    }
1369
1370    fn try_available(&self) -> bool {
1371        let mut count_and_status = self.count_and_status.lock_write();
1372        let (count, status) = *count_and_status;
1373
1374        if status == AccountStorageStatus::Available {
1375            *count_and_status = (count, AccountStorageStatus::Candidate);
1376            true
1377        } else {
1378            false
1379        }
1380    }
1381
1382    /// returns # of accounts remaining in the storage
1383    fn remove_accounts(
1384        &self,
1385        num_bytes: usize,
1386        reset_accounts: bool,
1387        num_accounts: usize,
1388    ) -> usize {
1389        let mut count_and_status = self.count_and_status.lock_write();
1390        let (mut count, mut status) = *count_and_status;
1391
1392        if count == num_accounts && status == AccountStorageStatus::Full && reset_accounts {
1393            // this case arises when we remove the last account from the
1394            //  storage, but we've learned from previous write attempts that
1395            //  the storage is full
1396            //
1397            // the only time it's safe to call reset() on an append_vec is when
1398            //  every account has been removed
1399            //          **and**
1400            //  the append_vec has previously been completely full
1401            //
1402            // otherwise, the storage may be in flight with a store()
1403            //   call
1404            self.accounts.reset();
1405            status = AccountStorageStatus::Available;
1406        }
1407
1408        // Some code path is removing accounts too many; this may result in an
1409        // unintended reveal of old state for unrelated accounts.
1410        assert!(
1411            count >= num_accounts,
1412            "double remove of account in slot: {}/store: {}!!",
1413            self.slot(),
1414            self.id(),
1415        );
1416
1417        self.alive_bytes.fetch_sub(num_bytes, Ordering::Release);
1418        count = count.saturating_sub(num_accounts);
1419        *count_and_status = (count, status);
1420        count
1421    }
1422
1423    /// Returns the path to the underlying accounts storage file
1424    pub fn path(&self) -> &Path {
1425        self.accounts.path()
1426    }
1427}
1428
1429pub fn get_temp_accounts_paths(count: u32) -> IoResult<(Vec<TempDir>, Vec<PathBuf>)> {
1430    let temp_dirs: IoResult<Vec<TempDir>> = (0..count).map(|_| TempDir::new()).collect();
1431    let temp_dirs = temp_dirs?;
1432
1433    let paths: IoResult<Vec<_>> = temp_dirs
1434        .iter()
1435        .map(|temp_dir| {
1436            utils::create_accounts_run_and_snapshot_dirs(temp_dir)
1437                .map(|(run_dir, _snapshot_dir)| run_dir)
1438        })
1439        .collect();
1440    let paths = paths?;
1441    Ok((temp_dirs, paths))
1442}
1443
1444#[derive(Default, Debug)]
1445struct CleaningInfo {
1446    slot_list: SlotList<AccountInfo>,
1447    ref_count: u64,
1448    /// Indicates if this account might have a zero lamport index entry.
1449    /// If false, the account *shall* not have zero lamport index entries.
1450    /// If true, the account *might* have zero lamport index entries.
1451    might_contain_zero_lamport_entry: bool,
1452}
1453
1454/// This is the return type of AccountsDb::construct_candidate_clean_keys.
1455/// It's a collection of pubkeys with associated information to
1456/// facilitate the decision making about which accounts can be removed
1457/// from the accounts index. In addition, the minimal dirty slot is
1458/// included in the returned value.
1459type CleaningCandidates = (Box<[RwLock<HashMap<Pubkey, CleaningInfo>>]>, Option<Slot>);
1460
1461/// Removing unrooted slots in Accounts Background Service needs to be synchronized with flushing
1462/// slots from the Accounts Cache.  This keeps track of those slots and the Mutex + Condvar for
1463/// synchronization.
1464#[derive(Debug, Default)]
1465struct RemoveUnrootedSlotsSynchronization {
1466    // slots being flushed from the cache or being purged
1467    slots_under_contention: Mutex<IntSet<Slot>>,
1468    signal: Condvar,
1469}
1470
1471type AccountInfoAccountsIndex = AccountsIndex<AccountInfo, AccountInfo>;
1472
1473// This structure handles the load/store of the accounts
1474#[derive(Debug)]
1475pub struct AccountsDb {
1476    /// Keeps tracks of index into AppendVec on a per slot basis
1477    pub accounts_index: AccountInfoAccountsIndex,
1478
1479    /// Some(offset) iff we want to squash old append vecs together into 'ancient append vecs'
1480    /// Some(offset) means for slots up to (max_slot - (slots_per_epoch - 'offset')), put them in ancient append vecs
1481    pub ancient_append_vec_offset: Option<i64>,
1482    pub ancient_storage_ideal_size: u64,
1483    pub max_ancient_storages: usize,
1484    /// true iff we want to skip the initial hash calculation on startup
1485    pub skip_initial_hash_calc: bool,
1486
1487    /// The number of pubkey bins used for accounts hash calculation
1488    pub hash_calculation_pubkey_bins: usize,
1489
1490    pub storage: AccountStorage,
1491
1492    /// from AccountsDbConfig
1493    create_ancient_storage: CreateAncientStorage,
1494
1495    /// true if this client should skip rewrites but still include those rewrites in the bank hash as if rewrites had occurred.
1496    pub test_skip_rewrites_but_include_in_bank_hash: bool,
1497
1498    pub accounts_cache: AccountsCache,
1499
1500    write_cache_limit_bytes: Option<u64>,
1501
1502    sender_bg_hasher: RwLock<Option<Sender<Vec<CachedAccount>>>>,
1503    read_only_accounts_cache: ReadOnlyAccountsCache,
1504
1505    /// distribute the accounts across storage lists
1506    pub next_id: AtomicAccountsFileId,
1507
1508    /// Set of shrinkable stores organized by map of slot to storage id
1509    pub shrink_candidate_slots: Mutex<ShrinkCandidates>,
1510
1511    pub write_version: AtomicU64,
1512
1513    /// Set of storage paths to pick from
1514    pub paths: Vec<PathBuf>,
1515
1516    /// Base directory for various necessary files
1517    base_working_path: PathBuf,
1518    // used by tests - held until we are dropped
1519    #[allow(dead_code)]
1520    base_working_temp_dir: Option<TempDir>,
1521
1522    accounts_hash_cache_path: PathBuf,
1523
1524    shrink_paths: Vec<PathBuf>,
1525
1526    /// Directory of paths this accounts_db needs to hold/remove
1527    #[allow(dead_code)]
1528    pub temp_paths: Option<Vec<TempDir>>,
1529
1530    /// Starting file size of appendvecs
1531    file_size: u64,
1532
1533    /// Foreground thread pool used for par_iter
1534    pub thread_pool: ThreadPool,
1535    /// Thread pool for AccountsBackgroundServices
1536    pub thread_pool_clean: ThreadPool,
1537    /// Thread pool for AccountsHashVerifier
1538    pub thread_pool_hash: ThreadPool,
1539
1540    accounts_delta_hashes: Mutex<HashMap<Slot, AccountsDeltaHash>>,
1541    accounts_hashes: Mutex<HashMap<Slot, (AccountsHash, /*capitalization*/ u64)>>,
1542    incremental_accounts_hashes:
1543        Mutex<HashMap<Slot, (IncrementalAccountsHash, /*capitalization*/ u64)>>,
1544
1545    pub stats: AccountsStats,
1546
1547    clean_accounts_stats: CleanAccountsStats,
1548
1549    // Stats for purges called outside of clean_accounts()
1550    external_purge_slots_stats: PurgeStats,
1551
1552    pub shrink_stats: ShrinkStats,
1553
1554    pub(crate) shrink_ancient_stats: ShrinkAncientStats,
1555
1556    pub account_indexes: AccountSecondaryIndexes,
1557
1558    /// Set of unique keys per slot which is used
1559    /// to drive clean_accounts
1560    /// Populated when flushing the accounts write cache
1561    uncleaned_pubkeys: DashMap<Slot, Vec<Pubkey>, BuildNoHashHasher<Slot>>,
1562
1563    #[cfg(test)]
1564    load_delay: u64,
1565
1566    #[cfg(test)]
1567    load_limit: AtomicU64,
1568
1569    /// true if drop_callback is attached to the bank.
1570    is_bank_drop_callback_enabled: AtomicBool,
1571
1572    /// Set of slots currently being flushed by `flush_slot_cache()` or removed
1573    /// by `remove_unrooted_slot()`. Used to ensure `remove_unrooted_slots(slots)`
1574    /// can safely clear the set of unrooted slots `slots`.
1575    remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization,
1576
1577    shrink_ratio: AccountShrinkThreshold,
1578
1579    /// Set of stores which are recently rooted or had accounts removed
1580    /// such that potentially a 0-lamport account update could be present which
1581    /// means we can remove the account from the index entirely.
1582    dirty_stores: DashMap<Slot, Arc<AccountStorageEntry>, BuildNoHashHasher<Slot>>,
1583
1584    /// Zero-lamport accounts that are *not* purged during clean because they need to stay alive
1585    /// for incremental snapshot support.
1586    zero_lamport_accounts_to_purge_after_full_snapshot: DashSet<(Slot, Pubkey)>,
1587
1588    /// GeyserPlugin accounts update notifier
1589    accounts_update_notifier: Option<AccountsUpdateNotifier>,
1590
1591    pub(crate) active_stats: ActiveStats,
1592
1593    pub verify_accounts_hash_in_bg: VerifyAccountsHashInBackground,
1594
1595    /// Used to disable logging dead slots during removal.
1596    /// allow disabling noisy log
1597    pub log_dead_slots: AtomicBool,
1598
1599    /// debug feature to scan every append vec and verify refcounts are equal
1600    exhaustively_verify_refcounts: bool,
1601
1602    /// storage format to use for new storages
1603    accounts_file_provider: AccountsFileProvider,
1604
1605    /// method to use for accessing storages
1606    storage_access: StorageAccess,
1607
1608    /// index scan filtering for shrinking
1609    scan_filter_for_shrinking: ScanFilter,
1610
1611    /// this will live here until the feature for partitioned epoch rewards is activated.
1612    /// At that point, this and other code can be deleted.
1613    pub partitioned_epoch_rewards_config: PartitionedEpochRewardsConfig,
1614
1615    /// the full accounts hash calculation as of a predetermined block height 'N'
1616    /// to be included in the bank hash at a predetermined block height 'M'
1617    /// The cadence is once per epoch, all nodes calculate a full accounts hash as of a known slot calculated using 'N'
1618    /// Some time later (to allow for slow calculation time), the bank hash at a slot calculated using 'M' includes the full accounts hash.
1619    /// Thus, the state of all accounts on a validator is known to be correct at least once per epoch.
1620    pub epoch_accounts_hash_manager: EpochAccountsHashManager,
1621
1622    /// The latest full snapshot slot dictates how to handle zero lamport accounts
1623    /// Note, this is None if we're told to *not* take snapshots
1624    latest_full_snapshot_slot: SeqLock<Option<Slot>>,
1625
1626    /// Flag to indicate if the experimental accounts lattice hash is enabled.
1627    /// (For R&D only; a feature-gate also exists to turn this on and make it a part of consensus.)
1628    pub is_experimental_accumulator_hash_enabled: AtomicBool,
1629
1630    /// Flag to indicate if the experimental accounts lattice hash should be verified.
1631    /// (For R&D only)
1632    pub verify_experimental_accumulator_hash: bool,
1633
1634    /// Flag to indicate if the experimental accounts lattice hash is used for snapshots.
1635    /// (For R&D only; a feature-gate also exists to turn this on.)
1636    pub snapshots_use_experimental_accumulator_hash: AtomicBool,
1637
1638    /// These are the ancient storages that could be valuable to
1639    /// shrink, sorted by amount of dead bytes.  The elements
1640    /// are sorted from the largest dead bytes to the smallest.
1641    /// Members are Slot and capacity. If capacity is smaller, then
1642    /// that means the storage was already shrunk.
1643    pub(crate) best_ancient_slots_to_shrink: RwLock<VecDeque<(Slot, u64)>>,
1644}
1645
1646/// results from 'split_storages_ancient'
1647#[derive(Debug, Default, PartialEq)]
1648struct SplitAncientStorages {
1649    /// # ancient slots
1650    ancient_slot_count: usize,
1651    /// the specific ancient slots
1652    ancient_slots: Vec<Slot>,
1653    /// lowest slot that is not an ancient append vec
1654    first_non_ancient_slot: Slot,
1655    /// slot # of beginning of first aligned chunk starting from the first non ancient slot
1656    first_chunk_start: Slot,
1657    /// # non-ancient slots to scan
1658    non_ancient_slot_count: usize,
1659    /// # chunks to use to iterate the storages
1660    /// all ancient chunks, the special 0 and last chunks for non-full chunks, and all the 'full' chunks of normal slots
1661    chunk_count: usize,
1662    /// start and end(exclusive) of normal (non-ancient) slots to be scanned
1663    normal_slot_range: Range<Slot>,
1664}
1665
1666impl SplitAncientStorages {
1667    /// When calculating accounts hash, we break the slots/storages into chunks that remain the same during an entire epoch.
1668    /// a slot is in this chunk of slots:
1669    /// start:         (slot / MAX_ITEMS_PER_CHUNK) * MAX_ITEMS_PER_CHUNK
1670    /// end_exclusive: start + MAX_ITEMS_PER_CHUNK
1671    /// So a slot remains in the same chunk whenever it is included in the accounts hash.
1672    /// When the slot gets deleted or gets consumed in an ancient append vec, it will no longer be in its chunk.
1673    /// The results of scanning a chunk of appendvecs can be cached to avoid scanning large amounts of data over and over.
1674    fn new(oldest_non_ancient_slot: Option<Slot>, snapshot_storages: &SortedStorages) -> Self {
1675        let range = snapshot_storages.range();
1676
1677        let (ancient_slots, first_non_ancient_slot) = if let Some(oldest_non_ancient_slot) =
1678            oldest_non_ancient_slot
1679        {
1680            // any ancient append vecs should definitely be cached
1681            // We need to break the ranges into:
1682            // 1. individual ancient append vecs (may be empty)
1683            // 2. first unevenly divided chunk starting at 1 epoch old slot (may be empty)
1684            // 3. evenly divided full chunks in the middle
1685            // 4. unevenly divided chunk of most recent slots (may be empty)
1686            let ancient_slots =
1687                Self::get_ancient_slots(oldest_non_ancient_slot, snapshot_storages, |storage| {
1688                    storage.capacity() > get_ancient_append_vec_capacity() * 50 / 100
1689                });
1690
1691            let first_non_ancient_slot = ancient_slots
1692                .last()
1693                .map(|last_ancient_slot| last_ancient_slot.saturating_add(1))
1694                .unwrap_or(range.start);
1695
1696            (ancient_slots, first_non_ancient_slot)
1697        } else {
1698            (vec![], range.start)
1699        };
1700
1701        Self::new_with_ancient_info(range, ancient_slots, first_non_ancient_slot)
1702    }
1703
1704    /// return all ancient append vec slots from the early slots referenced by 'snapshot_storages'
1705    /// `treat_as_ancient` returns true if the storage at this slot is large and should be treated individually by accounts hash calculation.
1706    /// `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.
1707    fn get_ancient_slots(
1708        oldest_non_ancient_slot: Slot,
1709        snapshot_storages: &SortedStorages,
1710        treat_as_ancient: impl Fn(&AccountStorageEntry) -> bool,
1711    ) -> Vec<Slot> {
1712        let range = snapshot_storages.range();
1713        let mut i = 0;
1714        let mut len_truncate = 0;
1715        let mut possible_ancient_slots = snapshot_storages
1716            .iter_range(&(range.start..oldest_non_ancient_slot))
1717            .filter_map(|(slot, storage)| {
1718                storage.map(|storage| {
1719                    i += 1;
1720                    if treat_as_ancient(storage) {
1721                        // even though the slot is in range of being an ancient append vec, if it isn't actually a large append vec,
1722                        // then we are better off treating all these slots as normally cacheable to reduce work in dedup.
1723                        // Since this one is large, for the moment, this one becomes the highest slot where we want to individually cache files.
1724                        len_truncate = i;
1725                    }
1726                    slot
1727                })
1728            })
1729            .collect::<Vec<_>>();
1730        possible_ancient_slots.truncate(len_truncate);
1731        possible_ancient_slots
1732    }
1733
1734    /// create once ancient slots have been identified
1735    /// This is easier to test, removing SortedStorages as a type to deal with here.
1736    fn new_with_ancient_info(
1737        range: &Range<Slot>,
1738        ancient_slots: Vec<Slot>,
1739        first_non_ancient_slot: Slot,
1740    ) -> Self {
1741        if range.is_empty() {
1742            // Corner case mainly for tests, but gives us a consistent base case. Makes more sense to return default here than anything else.
1743            // caller is asking to split for empty set of slots
1744            return SplitAncientStorages::default();
1745        }
1746
1747        let max_slot_inclusive = range.end.saturating_sub(1);
1748        let ancient_slot_count = ancient_slots.len();
1749        let first_chunk_start = ((first_non_ancient_slot + MAX_ITEMS_PER_CHUNK)
1750            / MAX_ITEMS_PER_CHUNK)
1751            * MAX_ITEMS_PER_CHUNK;
1752
1753        let non_ancient_slot_count = (max_slot_inclusive - first_non_ancient_slot + 1) as usize;
1754
1755        let normal_slot_range = Range {
1756            start: first_non_ancient_slot,
1757            end: range.end,
1758        };
1759
1760        // 2 is for 2 special chunks - unaligned slots at the beginning and end
1761        let chunk_count =
1762            ancient_slot_count + 2 + non_ancient_slot_count / (MAX_ITEMS_PER_CHUNK as usize);
1763
1764        SplitAncientStorages {
1765            ancient_slot_count,
1766            ancient_slots,
1767            first_non_ancient_slot,
1768            first_chunk_start,
1769            non_ancient_slot_count,
1770            chunk_count,
1771            normal_slot_range,
1772        }
1773    }
1774
1775    /// given 'normal_chunk', return the starting slot of that chunk in the normal/non-ancient range
1776    /// a normal_chunk is 0<=normal_chunk<=non_ancient_chunk_count
1777    /// non_ancient_chunk_count is chunk_count-ancient_slot_count
1778    fn get_starting_slot_from_normal_chunk(&self, normal_chunk: usize) -> Slot {
1779        if normal_chunk == 0 {
1780            self.normal_slot_range.start
1781        } else {
1782            assert!(
1783                normal_chunk.saturating_add(self.ancient_slot_count) < self.chunk_count,
1784                "out of bounds: {}, {}",
1785                normal_chunk,
1786                self.chunk_count
1787            );
1788
1789            let normal_chunk = normal_chunk.saturating_sub(1);
1790            (self.first_chunk_start + MAX_ITEMS_PER_CHUNK * (normal_chunk as Slot))
1791                .max(self.normal_slot_range.start)
1792        }
1793    }
1794
1795    /// ancient slots are the first chunks
1796    fn is_chunk_ancient(&self, chunk: usize) -> bool {
1797        chunk < self.ancient_slot_count
1798    }
1799
1800    /// given chunk in 0<=chunk<self.chunk_count
1801    /// return the range of slots in that chunk
1802    /// None indicates the range is empty for that chunk.
1803    fn get_slot_range(&self, chunk: usize) -> Option<Range<Slot>> {
1804        let range = if self.is_chunk_ancient(chunk) {
1805            // ancient append vecs are handled individually
1806            let slot = self.ancient_slots[chunk];
1807            Range {
1808                start: slot,
1809                end: slot + 1,
1810            }
1811        } else {
1812            // normal chunks are after ancient chunks
1813            let normal_chunk = chunk - self.ancient_slot_count;
1814            if normal_chunk == 0 {
1815                // first slot
1816                Range {
1817                    start: self.normal_slot_range.start,
1818                    end: self.first_chunk_start.min(self.normal_slot_range.end),
1819                }
1820            } else {
1821                // normal full chunk or the last chunk
1822                let first_slot = self.get_starting_slot_from_normal_chunk(normal_chunk);
1823                Range {
1824                    start: first_slot,
1825                    end: (first_slot + MAX_ITEMS_PER_CHUNK).min(self.normal_slot_range.end),
1826                }
1827            }
1828        };
1829        // return empty range as None
1830        (!range.is_empty()).then_some(range)
1831    }
1832}
1833
1834pub fn quarter_thread_count() -> usize {
1835    std::cmp::max(2, num_cpus::get() / 4)
1836}
1837
1838pub fn make_min_priority_thread_pool() -> ThreadPool {
1839    // Use lower thread count to reduce priority.
1840    let num_threads = quarter_thread_count();
1841    rayon::ThreadPoolBuilder::new()
1842        .thread_name(|i| format!("solAccountsLo{i:02}"))
1843        .num_threads(num_threads)
1844        .build()
1845        .unwrap()
1846}
1847
1848/// Returns the default number of threads to use for background accounts hashing
1849pub fn default_num_hash_threads() -> NonZeroUsize {
1850    // 1/8 of the number of cpus and up to 6 threads gives good balance for the system.
1851    let num_threads = (num_cpus::get() / 8).clamp(2, 6);
1852    NonZeroUsize::new(num_threads).unwrap()
1853}
1854
1855pub fn make_hash_thread_pool(num_threads: Option<NonZeroUsize>) -> ThreadPool {
1856    let num_threads = num_threads.unwrap_or_else(default_num_hash_threads).get();
1857    rayon::ThreadPoolBuilder::new()
1858        .thread_name(|i| format!("solAcctHash{i:02}"))
1859        .num_threads(num_threads)
1860        .build()
1861        .unwrap()
1862}
1863
1864pub fn default_num_foreground_threads() -> usize {
1865    get_thread_count()
1866}
1867
1868#[cfg(feature = "frozen-abi")]
1869impl solana_frozen_abi::abi_example::AbiExample for AccountsDb {
1870    fn example() -> Self {
1871        let accounts_db = AccountsDb::new_single_for_tests();
1872        let key = Pubkey::default();
1873        let some_data_len = 5;
1874        let some_slot: Slot = 0;
1875        let account = AccountSharedData::new(1, some_data_len, &key);
1876        accounts_db.store_uncached(some_slot, &[(&key, &account)]);
1877        accounts_db.add_root(0);
1878
1879        accounts_db
1880    }
1881}
1882
1883impl ZeroLamport for StoredAccountMeta<'_> {
1884    fn is_zero_lamport(&self) -> bool {
1885        self.lamports() == 0
1886    }
1887}
1888
1889#[derive(Clone, Debug, Eq, PartialEq)]
1890pub struct PubkeyHashAccount {
1891    pub pubkey: Pubkey,
1892    pub hash: AccountHash,
1893    pub account: AccountSharedData,
1894}
1895
1896impl AccountsDb {
1897    pub const DEFAULT_ACCOUNTS_HASH_CACHE_DIR: &'static str = "accounts_hash_cache";
1898
1899    // The default high and low watermark sizes for the accounts read cache.
1900    // If the cache size exceeds MAX_SIZE_HI, it'll evict entries until the size is <= MAX_SIZE_LO.
1901    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
1902    const DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_LO: usize = 400 * 1024 * 1024;
1903    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
1904    const DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_HI: usize = 410 * 1024 * 1024;
1905
1906    // See AccountsDbConfig::read_cache_evict_sample_size.
1907    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
1908    const DEFAULT_READ_ONLY_CACHE_EVICT_SAMPLE_SIZE: usize = 8;
1909
1910    pub fn default_for_tests() -> Self {
1911        Self::new_single_for_tests()
1912    }
1913
1914    pub fn new_single_for_tests() -> Self {
1915        AccountsDb::new_for_tests(Vec::new())
1916    }
1917
1918    pub fn new_single_for_tests_with_provider(file_provider: AccountsFileProvider) -> Self {
1919        AccountsDb::new_for_tests_with_provider(Vec::new(), file_provider)
1920    }
1921
1922    pub fn new_for_tests(paths: Vec<PathBuf>) -> Self {
1923        Self::new_for_tests_with_provider(paths, AccountsFileProvider::default())
1924    }
1925
1926    fn new_for_tests_with_provider(
1927        paths: Vec<PathBuf>,
1928        accounts_file_provider: AccountsFileProvider,
1929    ) -> Self {
1930        let mut db = AccountsDb::new_with_config(
1931            paths,
1932            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
1933            None,
1934            Arc::default(),
1935        );
1936        db.accounts_file_provider = accounts_file_provider;
1937        db
1938    }
1939
1940    pub fn new_with_config(
1941        paths: Vec<PathBuf>,
1942        accounts_db_config: Option<AccountsDbConfig>,
1943        accounts_update_notifier: Option<AccountsUpdateNotifier>,
1944        exit: Arc<AtomicBool>,
1945    ) -> Self {
1946        let accounts_db_config = accounts_db_config.unwrap_or_default();
1947        let accounts_index = AccountsIndex::new(accounts_db_config.index.clone(), exit);
1948
1949        let base_working_path = accounts_db_config.base_working_path.clone();
1950        let (base_working_path, base_working_temp_dir) =
1951            if let Some(base_working_path) = base_working_path {
1952                (base_working_path, None)
1953            } else {
1954                let base_working_temp_dir = TempDir::new().unwrap();
1955                let base_working_path = base_working_temp_dir.path().to_path_buf();
1956                (base_working_path, Some(base_working_temp_dir))
1957            };
1958
1959        let (paths, temp_paths) = if paths.is_empty() {
1960            // Create a temporary set of accounts directories, used primarily
1961            // for testing
1962            let (temp_dirs, temp_paths) = get_temp_accounts_paths(DEFAULT_NUM_DIRS).unwrap();
1963            (temp_paths, Some(temp_dirs))
1964        } else {
1965            (paths, None)
1966        };
1967
1968        let shrink_paths = accounts_db_config
1969            .shrink_paths
1970            .clone()
1971            .unwrap_or_else(|| paths.clone());
1972
1973        let accounts_hash_cache_path = accounts_db_config.accounts_hash_cache_path.clone();
1974        let accounts_hash_cache_path = accounts_hash_cache_path.unwrap_or_else(|| {
1975            let accounts_hash_cache_path =
1976                base_working_path.join(Self::DEFAULT_ACCOUNTS_HASH_CACHE_DIR);
1977            if !accounts_hash_cache_path.exists() {
1978                fs::create_dir(&accounts_hash_cache_path).expect("create accounts hash cache dir");
1979            }
1980            accounts_hash_cache_path
1981        });
1982
1983        let read_cache_size = accounts_db_config.read_cache_limit_bytes.unwrap_or((
1984            Self::DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_LO,
1985            Self::DEFAULT_MAX_READ_ONLY_CACHE_DATA_SIZE_HI,
1986        ));
1987        let read_cache_evict_sample_size = accounts_db_config
1988            .read_cache_evict_sample_size
1989            .unwrap_or(Self::DEFAULT_READ_ONLY_CACHE_EVICT_SAMPLE_SIZE);
1990
1991        // Increase the stack for foreground threads
1992        // rayon needs a lot of stack
1993        const ACCOUNTS_STACK_SIZE: usize = 8 * 1024 * 1024;
1994        let num_foreground_threads = accounts_db_config
1995            .num_foreground_threads
1996            .map(Into::into)
1997            .unwrap_or_else(default_num_foreground_threads);
1998        let thread_pool = rayon::ThreadPoolBuilder::new()
1999            .num_threads(num_foreground_threads)
2000            .thread_name(|i| format!("solAccounts{i:02}"))
2001            .stack_size(ACCOUNTS_STACK_SIZE)
2002            .build()
2003            .expect("new rayon threadpool");
2004
2005        let num_clean_threads = accounts_db_config
2006            .num_clean_threads
2007            .map(Into::into)
2008            .unwrap_or_else(quarter_thread_count);
2009        let thread_pool_clean = rayon::ThreadPoolBuilder::new()
2010            .thread_name(|i| format!("solAccountsLo{i:02}"))
2011            .num_threads(num_clean_threads)
2012            .build()
2013            .expect("new rayon threadpool");
2014
2015        let thread_pool_hash = make_hash_thread_pool(accounts_db_config.num_hash_threads);
2016
2017        let new = Self {
2018            accounts_index,
2019            paths,
2020            base_working_path,
2021            base_working_temp_dir,
2022            accounts_hash_cache_path,
2023            temp_paths,
2024            shrink_paths,
2025            skip_initial_hash_calc: accounts_db_config.skip_initial_hash_calc,
2026            ancient_append_vec_offset: accounts_db_config
2027                .ancient_append_vec_offset
2028                .or(ANCIENT_APPEND_VEC_DEFAULT_OFFSET),
2029            ancient_storage_ideal_size: accounts_db_config
2030                .ancient_storage_ideal_size
2031                .unwrap_or(DEFAULT_ANCIENT_STORAGE_IDEAL_SIZE),
2032            max_ancient_storages: accounts_db_config
2033                .max_ancient_storages
2034                .unwrap_or(DEFAULT_MAX_ANCIENT_STORAGES),
2035            hash_calculation_pubkey_bins: accounts_db_config
2036                .hash_calculation_pubkey_bins
2037                .unwrap_or(DEFAULT_HASH_CALCULATION_PUBKEY_BINS),
2038            account_indexes: accounts_db_config.account_indexes.unwrap_or_default(),
2039            shrink_ratio: accounts_db_config.shrink_ratio,
2040            accounts_update_notifier,
2041            create_ancient_storage: accounts_db_config.create_ancient_storage,
2042            read_only_accounts_cache: ReadOnlyAccountsCache::new(
2043                read_cache_size.0,
2044                read_cache_size.1,
2045                read_cache_evict_sample_size,
2046            ),
2047            write_cache_limit_bytes: accounts_db_config.write_cache_limit_bytes,
2048            partitioned_epoch_rewards_config: accounts_db_config.partitioned_epoch_rewards_config,
2049            exhaustively_verify_refcounts: accounts_db_config.exhaustively_verify_refcounts,
2050            test_skip_rewrites_but_include_in_bank_hash: accounts_db_config
2051                .test_skip_rewrites_but_include_in_bank_hash,
2052            storage_access: accounts_db_config.storage_access,
2053            scan_filter_for_shrinking: accounts_db_config.scan_filter_for_shrinking,
2054            is_experimental_accumulator_hash_enabled: accounts_db_config
2055                .enable_experimental_accumulator_hash
2056                .into(),
2057            verify_experimental_accumulator_hash: accounts_db_config
2058                .verify_experimental_accumulator_hash,
2059            snapshots_use_experimental_accumulator_hash: accounts_db_config
2060                .snapshots_use_experimental_accumulator_hash
2061                .into(),
2062            thread_pool,
2063            thread_pool_clean,
2064            thread_pool_hash,
2065            verify_accounts_hash_in_bg: VerifyAccountsHashInBackground::default(),
2066            active_stats: ActiveStats::default(),
2067            storage: AccountStorage::default(),
2068            accounts_cache: AccountsCache::default(),
2069            sender_bg_hasher: RwLock::new(None),
2070            uncleaned_pubkeys: DashMap::default(),
2071            next_id: AtomicAccountsFileId::new(0),
2072            shrink_candidate_slots: Mutex::new(ShrinkCandidates::default()),
2073            write_version: AtomicU64::new(0),
2074            file_size: DEFAULT_FILE_SIZE,
2075            accounts_delta_hashes: Mutex::new(HashMap::new()),
2076            accounts_hashes: Mutex::new(HashMap::new()),
2077            incremental_accounts_hashes: Mutex::new(HashMap::new()),
2078            external_purge_slots_stats: PurgeStats::default(),
2079            clean_accounts_stats: CleanAccountsStats::default(),
2080            shrink_stats: ShrinkStats::default(),
2081            shrink_ancient_stats: ShrinkAncientStats::default(),
2082            stats: AccountsStats::default(),
2083            #[cfg(test)]
2084            load_delay: u64::default(),
2085            #[cfg(test)]
2086            load_limit: AtomicU64::default(),
2087            is_bank_drop_callback_enabled: AtomicBool::default(),
2088            remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization::default(),
2089            dirty_stores: DashMap::default(),
2090            zero_lamport_accounts_to_purge_after_full_snapshot: DashSet::default(),
2091            log_dead_slots: AtomicBool::new(true),
2092            accounts_file_provider: AccountsFileProvider::default(),
2093            epoch_accounts_hash_manager: EpochAccountsHashManager::new_invalid(),
2094            latest_full_snapshot_slot: SeqLock::new(None),
2095            best_ancient_slots_to_shrink: RwLock::default(),
2096        };
2097
2098        {
2099            for path in new.paths.iter() {
2100                std::fs::create_dir_all(path).expect("Create directory failed.");
2101            }
2102        }
2103        new
2104    }
2105
2106    pub fn file_size(&self) -> u64 {
2107        self.file_size
2108    }
2109
2110    /// Get the base working directory
2111    pub fn get_base_working_path(&self) -> PathBuf {
2112        self.base_working_path.clone()
2113    }
2114
2115    /// Returns true if there is an accounts update notifier.
2116    pub fn has_accounts_update_notifier(&self) -> bool {
2117        self.accounts_update_notifier.is_some()
2118    }
2119
2120    fn next_id(&self) -> AccountsFileId {
2121        let next_id = self.next_id.fetch_add(1, Ordering::AcqRel);
2122        assert!(
2123            next_id != AccountsFileId::MAX,
2124            "We've run out of storage ids!"
2125        );
2126        next_id
2127    }
2128
2129    fn new_storage_entry(&self, slot: Slot, path: &Path, size: u64) -> AccountStorageEntry {
2130        AccountStorageEntry::new(
2131            path,
2132            slot,
2133            self.next_id(),
2134            size,
2135            self.accounts_file_provider,
2136        )
2137    }
2138
2139    /// Returns if the experimental accounts lattice hash is enabled
2140    pub fn is_experimental_accumulator_hash_enabled(&self) -> bool {
2141        self.is_experimental_accumulator_hash_enabled
2142            .load(Ordering::Acquire)
2143    }
2144
2145    /// Sets if the experimental accounts lattice hash is enabled
2146    pub fn set_is_experimental_accumulator_hash_enabled(&self, is_enabled: bool) {
2147        self.is_experimental_accumulator_hash_enabled
2148            .store(is_enabled, Ordering::Release);
2149    }
2150
2151    /// Returns if snapshots use the experimental accounts lattice hash
2152    pub fn snapshots_use_experimental_accumulator_hash(&self) -> bool {
2153        self.snapshots_use_experimental_accumulator_hash
2154            .load(Ordering::Acquire)
2155    }
2156
2157    /// Sets if snapshots use the experimental accounts lattice hash
2158    pub fn set_snapshots_use_experimental_accumulator_hash(&self, is_enabled: bool) {
2159        self.snapshots_use_experimental_accumulator_hash
2160            .store(is_enabled, Ordering::Release);
2161    }
2162
2163    /// While scanning cleaning candidates obtain slots that can be
2164    /// reclaimed for each pubkey. In addition, if the pubkey is
2165    /// removed from the index, insert in pubkeys_removed_from_accounts_index.
2166    fn collect_reclaims(
2167        &self,
2168        pubkey: &Pubkey,
2169        max_clean_root_inclusive: Option<Slot>,
2170        ancient_account_cleans: &AtomicU64,
2171        epoch_schedule: &EpochSchedule,
2172        pubkeys_removed_from_accounts_index: &Mutex<PubkeysRemovedFromAccountsIndex>,
2173    ) -> SlotList<AccountInfo> {
2174        let one_epoch_old = self.get_oldest_non_ancient_slot(epoch_schedule);
2175        let mut clean_rooted = Measure::start("clean_old_root-ms");
2176        let mut reclaims = Vec::new();
2177        let removed_from_index = self.accounts_index.clean_rooted_entries(
2178            pubkey,
2179            &mut reclaims,
2180            max_clean_root_inclusive,
2181        );
2182        if removed_from_index {
2183            pubkeys_removed_from_accounts_index
2184                .lock()
2185                .unwrap()
2186                .insert(*pubkey);
2187        }
2188        if !reclaims.is_empty() {
2189            // figure out how many ancient accounts have been reclaimed
2190            let old_reclaims = reclaims
2191                .iter()
2192                .filter_map(|(slot, _)| (slot < &one_epoch_old).then_some(1))
2193                .sum();
2194            ancient_account_cleans.fetch_add(old_reclaims, Ordering::Relaxed);
2195        }
2196        clean_rooted.stop();
2197        self.clean_accounts_stats
2198            .clean_old_root_us
2199            .fetch_add(clean_rooted.as_us(), Ordering::Relaxed);
2200        reclaims
2201    }
2202
2203    /// Reclaim older states of accounts older than max_clean_root_inclusive for AccountsDb bloat mitigation.
2204    /// Any accounts which are removed from the accounts index are returned in PubkeysRemovedFromAccountsIndex.
2205    /// These should NOT be unref'd later from the accounts index.
2206    fn clean_accounts_older_than_root(
2207        &self,
2208        reclaims: &SlotList<AccountInfo>,
2209        pubkeys_removed_from_accounts_index: &HashSet<Pubkey>,
2210    ) -> ReclaimResult {
2211        let mut measure = Measure::start("clean_old_root_reclaims");
2212
2213        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
2214        // and those stores may be used for background hashing.
2215        let reset_accounts = false;
2216
2217        let reclaim_result = self.handle_reclaims(
2218            (!reclaims.is_empty()).then(|| reclaims.iter()),
2219            None,
2220            reset_accounts,
2221            pubkeys_removed_from_accounts_index,
2222            HandleReclaims::ProcessDeadSlots(&self.clean_accounts_stats.purge_stats),
2223        );
2224        measure.stop();
2225        debug!("{}", measure);
2226        self.clean_accounts_stats
2227            .clean_old_root_reclaim_us
2228            .fetch_add(measure.as_us(), Ordering::Relaxed);
2229        reclaim_result
2230    }
2231
2232    /// increment store_counts to non-zero for all stores that can not be deleted.
2233    /// a store cannot be deleted if:
2234    /// 1. one of the pubkeys in the store has account info to a store whose store count is not going to zero
2235    /// 2. a pubkey we were planning to remove is not removing all stores that contain the account
2236    fn calc_delete_dependencies(
2237        &self,
2238        candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>],
2239        store_counts: &mut HashMap<Slot, (usize, HashSet<Pubkey>)>,
2240        min_slot: Option<Slot>,
2241    ) {
2242        // Another pass to check if there are some filtered accounts which
2243        // do not match the criteria of deleting all appendvecs which contain them
2244        // then increment their storage count.
2245        let mut already_counted = IntSet::default();
2246        for (bin_index, bin) in candidates.iter().enumerate() {
2247            let bin = bin.read().unwrap();
2248            for (
2249                pubkey,
2250                CleaningInfo {
2251                    slot_list,
2252                    ref_count,
2253                    ..
2254                },
2255            ) in bin.iter()
2256            {
2257                let mut failed_slot = None;
2258                let all_stores_being_deleted = slot_list.len() as RefCount == *ref_count;
2259                if all_stores_being_deleted {
2260                    let mut delete = true;
2261                    for (slot, _account_info) in slot_list {
2262                        if let Some(count) = store_counts.get(slot).map(|s| s.0) {
2263                            debug!("calc_delete_dependencies() slot: {slot}, count len: {count}");
2264                            if count == 0 {
2265                                // this store CAN be removed
2266                                continue;
2267                            }
2268                        }
2269                        // One of the pubkeys in the store has account info to a store whose store count is not going to zero.
2270                        // If the store cannot be found, that also means store isn't being deleted.
2271                        failed_slot = Some(*slot);
2272                        delete = false;
2273                        break;
2274                    }
2275                    if delete {
2276                        // this pubkey can be deleted from all stores it is in
2277                        continue;
2278                    }
2279                } else {
2280                    // a pubkey we were planning to remove is not removing all stores that contain the account
2281                    debug!(
2282                        "calc_delete_dependencies(), pubkey: {pubkey}, slot list len: {}, \
2283                         ref count: {ref_count}, slot list: {slot_list:?}",
2284                        slot_list.len(),
2285                    );
2286                }
2287
2288                // increment store_counts to non-zero for all stores that can not be deleted.
2289                let mut pending_stores = IntSet::default();
2290                for (slot, _account_info) in slot_list {
2291                    if !already_counted.contains(slot) {
2292                        pending_stores.insert(*slot);
2293                    }
2294                }
2295                while !pending_stores.is_empty() {
2296                    let slot = pending_stores.iter().next().cloned().unwrap();
2297                    if Some(slot) == min_slot {
2298                        if let Some(failed_slot) = failed_slot.take() {
2299                            info!(
2300                                "calc_delete_dependencies, oldest slot is not able to be deleted \
2301                                 because of {pubkey} in slot {failed_slot}"
2302                            );
2303                        } else {
2304                            info!(
2305                                "calc_delete_dependencies, oldest slot is not able to be deleted \
2306                                 because of {pubkey}, slot list len: {}, ref count: {ref_count}",
2307                                slot_list.len()
2308                            );
2309                        }
2310                    }
2311
2312                    pending_stores.remove(&slot);
2313                    if !already_counted.insert(slot) {
2314                        continue;
2315                    }
2316                    // the point of all this code: remove the store count for all stores we cannot remove
2317                    if let Some(store_count) = store_counts.remove(&slot) {
2318                        // all pubkeys in this store also cannot be removed from all stores they are in
2319                        let affected_pubkeys = &store_count.1;
2320                        for key in affected_pubkeys {
2321                            let candidates_bin_index =
2322                                self.accounts_index.bin_calculator.bin_from_pubkey(key);
2323                            let mut update_pending_stores =
2324                                |bin: &HashMap<Pubkey, CleaningInfo>| {
2325                                    for (slot, _account_info) in &bin.get(key).unwrap().slot_list {
2326                                        if !already_counted.contains(slot) {
2327                                            pending_stores.insert(*slot);
2328                                        }
2329                                    }
2330                                };
2331                            if candidates_bin_index == bin_index {
2332                                update_pending_stores(&bin);
2333                            } else {
2334                                update_pending_stores(
2335                                    &candidates[candidates_bin_index].read().unwrap(),
2336                                );
2337                            }
2338                        }
2339                    }
2340                }
2341            }
2342        }
2343    }
2344
2345    fn background_hasher(receiver: Receiver<Vec<CachedAccount>>) {
2346        info!("Background account hasher has started");
2347        loop {
2348            let result = receiver.try_recv();
2349            match result {
2350                Ok(accounts) => {
2351                    for account in accounts {
2352                        // if we hold the only ref, then this account doesn't need to be hashed, we ignore this account and it will disappear
2353                        if Arc::strong_count(&account) > 1 {
2354                            // this will cause the hash to be calculated and store inside account if it needs to be calculated
2355                            let _ = (*account).hash();
2356                        };
2357                    }
2358                }
2359                Err(TryRecvError::Empty) => {
2360                    sleep(Duration::from_millis(5));
2361                }
2362                Err(err @ TryRecvError::Disconnected) => {
2363                    info!("Background account hasher is stopping because: {err}");
2364                    break;
2365                }
2366            }
2367        }
2368        info!("Background account hasher has stopped");
2369    }
2370
2371    pub fn start_background_hasher(&self) {
2372        if self.is_background_hasher_running() {
2373            return;
2374        }
2375
2376        let (sender, receiver) = unbounded();
2377        Builder::new()
2378            .name("solDbStoreHashr".to_string())
2379            .spawn(move || {
2380                Self::background_hasher(receiver);
2381            })
2382            .unwrap();
2383        *self.sender_bg_hasher.write().unwrap() = Some(sender);
2384    }
2385
2386    pub fn stop_background_hasher(&self) {
2387        drop(self.sender_bg_hasher.write().unwrap().take())
2388    }
2389
2390    pub fn is_background_hasher_running(&self) -> bool {
2391        self.sender_bg_hasher.read().unwrap().is_some()
2392    }
2393
2394    #[must_use]
2395    pub fn purge_keys_exact<'a, C>(
2396        &'a self,
2397        pubkey_to_slot_set: impl Iterator<Item = &'a (Pubkey, C)>,
2398    ) -> (Vec<(Slot, AccountInfo)>, PubkeysRemovedFromAccountsIndex)
2399    where
2400        C: Contains<'a, Slot> + 'a,
2401    {
2402        let mut reclaims = Vec::new();
2403        let mut dead_keys = Vec::new();
2404
2405        let mut purge_exact_count = 0;
2406        let (_, purge_exact_us) = measure_us!(for (pubkey, slots_set) in pubkey_to_slot_set {
2407            purge_exact_count += 1;
2408            let is_empty = self
2409                .accounts_index
2410                .purge_exact(pubkey, slots_set, &mut reclaims);
2411            if is_empty {
2412                dead_keys.push(pubkey);
2413            }
2414        });
2415
2416        let (pubkeys_removed_from_accounts_index, handle_dead_keys_us) = measure_us!(self
2417            .accounts_index
2418            .handle_dead_keys(&dead_keys, &self.account_indexes));
2419
2420        self.stats
2421            .purge_exact_count
2422            .fetch_add(purge_exact_count, Ordering::Relaxed);
2423        self.stats
2424            .handle_dead_keys_us
2425            .fetch_add(handle_dead_keys_us, Ordering::Relaxed);
2426        self.stats
2427            .purge_exact_us
2428            .fetch_add(purge_exact_us, Ordering::Relaxed);
2429        (reclaims, pubkeys_removed_from_accounts_index)
2430    }
2431
2432    fn max_clean_root(&self, proposed_clean_root: Option<Slot>) -> Option<Slot> {
2433        match (
2434            self.accounts_index.min_ongoing_scan_root(),
2435            proposed_clean_root,
2436        ) {
2437            (None, None) => None,
2438            (Some(min_scan_root), None) => Some(min_scan_root),
2439            (None, Some(proposed_clean_root)) => Some(proposed_clean_root),
2440            (Some(min_scan_root), Some(proposed_clean_root)) => {
2441                Some(std::cmp::min(min_scan_root, proposed_clean_root))
2442            }
2443        }
2444    }
2445
2446    /// get the oldest slot that is within one epoch of the highest known root.
2447    /// The slot will have been offset by `self.ancient_append_vec_offset`
2448    fn get_oldest_non_ancient_slot(&self, epoch_schedule: &EpochSchedule) -> Slot {
2449        self.get_oldest_non_ancient_slot_from_slot(
2450            epoch_schedule,
2451            self.accounts_index.max_root_inclusive(),
2452        )
2453    }
2454
2455    /// get the oldest slot that is within one epoch of `max_root_inclusive`.
2456    /// The slot will have been offset by `self.ancient_append_vec_offset`
2457    fn get_oldest_non_ancient_slot_from_slot(
2458        &self,
2459        epoch_schedule: &EpochSchedule,
2460        max_root_inclusive: Slot,
2461    ) -> Slot {
2462        let mut result = max_root_inclusive;
2463        if let Some(offset) = self.ancient_append_vec_offset {
2464            result = Self::apply_offset_to_slot(result, offset);
2465        }
2466        result = Self::apply_offset_to_slot(
2467            result,
2468            -((epoch_schedule.slots_per_epoch as i64).saturating_sub(1)),
2469        );
2470        result.min(max_root_inclusive)
2471    }
2472
2473    /// Collect all the uncleaned slots, up to a max slot
2474    ///
2475    /// Search through the uncleaned Pubkeys and return all the slots, up to a maximum slot.
2476    fn collect_uncleaned_slots_up_to_slot(&self, max_slot_inclusive: Slot) -> Vec<Slot> {
2477        self.uncleaned_pubkeys
2478            .iter()
2479            .filter_map(|entry| {
2480                let slot = *entry.key();
2481                (slot <= max_slot_inclusive).then_some(slot)
2482            })
2483            .collect()
2484    }
2485
2486    /// For each slot in the list of uncleaned slots, up to a maximum
2487    /// slot, remove it from the `uncleaned_pubkeys` and move all the
2488    /// pubkeys to `candidates` for cleaning.
2489    fn remove_uncleaned_slots_up_to_slot_and_move_pubkeys(
2490        &self,
2491        max_slot_inclusive: Slot,
2492        candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>],
2493    ) {
2494        let uncleaned_slots = self.collect_uncleaned_slots_up_to_slot(max_slot_inclusive);
2495        for uncleaned_slot in uncleaned_slots.into_iter() {
2496            if let Some((_removed_slot, mut removed_pubkeys)) =
2497                self.uncleaned_pubkeys.remove(&uncleaned_slot)
2498            {
2499                // Sort all keys by bin index so that we can insert
2500                // them in `candidates` more efficiently.
2501                removed_pubkeys.sort_by(|a, b| {
2502                    self.accounts_index
2503                        .bin_calculator
2504                        .bin_from_pubkey(a)
2505                        .cmp(&self.accounts_index.bin_calculator.bin_from_pubkey(b))
2506                });
2507                if let Some(first_removed_pubkey) = removed_pubkeys.first() {
2508                    let mut prev_bin = self
2509                        .accounts_index
2510                        .bin_calculator
2511                        .bin_from_pubkey(first_removed_pubkey);
2512                    let mut candidates_bin = candidates[prev_bin].write().unwrap();
2513                    for removed_pubkey in removed_pubkeys {
2514                        let curr_bin = self
2515                            .accounts_index
2516                            .bin_calculator
2517                            .bin_from_pubkey(&removed_pubkey);
2518                        if curr_bin != prev_bin {
2519                            candidates_bin = candidates[curr_bin].write().unwrap();
2520                            prev_bin = curr_bin;
2521                        }
2522                        // Conservatively mark the candidate might have a zero lamport entry for
2523                        // correctness so that scan WILL try to look in disk if it is
2524                        // not in-mem. These keys are from 1) recently processed
2525                        // slots, 2) zero lamports found in shrink. Therefore, they are very likely
2526                        // to be in-memory, and seldomly do we need to look them up in disk.
2527                        candidates_bin.insert(
2528                            removed_pubkey,
2529                            CleaningInfo {
2530                                might_contain_zero_lamport_entry: true,
2531                                ..Default::default()
2532                            },
2533                        );
2534                    }
2535                }
2536            }
2537        }
2538    }
2539
2540    fn count_pubkeys(candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>]) -> u64 {
2541        candidates
2542            .iter()
2543            .map(|x| x.read().unwrap().len())
2544            .sum::<usize>() as u64
2545    }
2546
2547    /// Construct a list of candidates for cleaning from:
2548    /// - dirty_stores      -- set of stores which had accounts
2549    ///                        removed or recently rooted;
2550    /// - uncleaned_pubkeys -- the delta set of updated pubkeys in
2551    ///                        rooted slots from the last clean.
2552    ///
2553    /// The function also returns the minimum slot we encountered.
2554    fn construct_candidate_clean_keys(
2555        &self,
2556        max_clean_root_inclusive: Option<Slot>,
2557        is_startup: bool,
2558        timings: &mut CleanKeyTimings,
2559        epoch_schedule: &EpochSchedule,
2560        old_storages_policy: OldStoragesPolicy,
2561    ) -> CleaningCandidates {
2562        let oldest_non_ancient_slot = self.get_oldest_non_ancient_slot(epoch_schedule);
2563        let mut dirty_store_processing_time = Measure::start("dirty_store_processing");
2564        let max_root_inclusive = self.accounts_index.max_root_inclusive();
2565        let max_slot_inclusive = max_clean_root_inclusive.unwrap_or(max_root_inclusive);
2566
2567        if old_storages_policy == OldStoragesPolicy::Clean {
2568            let slot_one_epoch_old =
2569                max_root_inclusive.saturating_sub(epoch_schedule.slots_per_epoch);
2570            // do nothing special for these 100 old storages that will likely get cleaned up shortly
2571            let acceptable_straggler_slot_count = 100;
2572            let old_slot_cutoff =
2573                slot_one_epoch_old.saturating_sub(acceptable_straggler_slot_count);
2574            let (old_storages, old_slots) = self.get_storages(..old_slot_cutoff);
2575            let num_old_storages = old_storages.len();
2576            for (old_slot, old_storage) in std::iter::zip(old_slots, old_storages) {
2577                self.dirty_stores.entry(old_slot).or_insert(old_storage);
2578            }
2579            info!("Marked {num_old_storages} old storages as dirty");
2580        }
2581
2582        let mut dirty_stores = Vec::with_capacity(self.dirty_stores.len());
2583        // find the oldest dirty slot
2584        // we'll add logging if that append vec cannot be marked dead
2585        let mut min_dirty_slot = None::<u64>;
2586        self.dirty_stores.retain(|slot, store| {
2587            if *slot > max_slot_inclusive {
2588                true
2589            } else {
2590                min_dirty_slot = min_dirty_slot.map(|min| min.min(*slot)).or(Some(*slot));
2591                dirty_stores.push((*slot, store.clone()));
2592                false
2593            }
2594        });
2595        let dirty_stores_len = dirty_stores.len();
2596        let num_bins = self.accounts_index.bins();
2597        let candidates: Box<_> =
2598            std::iter::repeat_with(|| RwLock::new(HashMap::<Pubkey, CleaningInfo>::new()))
2599                .take(num_bins)
2600                .collect();
2601
2602        let insert_candidate = |pubkey, is_zero_lamport| {
2603            let index = self.accounts_index.bin_calculator.bin_from_pubkey(&pubkey);
2604            let mut candidates_bin = candidates[index].write().unwrap();
2605            candidates_bin
2606                .entry(pubkey)
2607                .or_default()
2608                .might_contain_zero_lamport_entry |= is_zero_lamport;
2609        };
2610
2611        let dirty_ancient_stores = AtomicUsize::default();
2612        let mut dirty_store_routine = || {
2613            let chunk_size = 1.max(dirty_stores_len.saturating_div(rayon::current_num_threads()));
2614            let oldest_dirty_slots: Vec<u64> = dirty_stores
2615                .par_chunks(chunk_size)
2616                .map(|dirty_store_chunk| {
2617                    let mut oldest_dirty_slot = max_slot_inclusive.saturating_add(1);
2618                    dirty_store_chunk.iter().for_each(|(slot, store)| {
2619                        if *slot < oldest_non_ancient_slot {
2620                            dirty_ancient_stores.fetch_add(1, Ordering::Relaxed);
2621                        }
2622                        oldest_dirty_slot = oldest_dirty_slot.min(*slot);
2623
2624                        store.accounts.scan_index(|index| {
2625                            let pubkey = index.index_info.pubkey;
2626                            let is_zero_lamport = index.index_info.lamports == 0;
2627                            insert_candidate(pubkey, is_zero_lamport);
2628                        });
2629                    });
2630                    oldest_dirty_slot
2631                })
2632                .collect();
2633            timings.oldest_dirty_slot = *oldest_dirty_slots
2634                .iter()
2635                .min()
2636                .unwrap_or(&max_slot_inclusive.saturating_add(1));
2637        };
2638
2639        if is_startup {
2640            // Free to consume all the cores during startup
2641            dirty_store_routine();
2642        } else {
2643            self.thread_pool_clean.install(|| {
2644                dirty_store_routine();
2645            });
2646        }
2647        timings.dirty_pubkeys_count = Self::count_pubkeys(&candidates);
2648        trace!(
2649            "dirty_stores.len: {} pubkeys.len: {}",
2650            dirty_stores_len,
2651            timings.dirty_pubkeys_count,
2652        );
2653        dirty_store_processing_time.stop();
2654        timings.dirty_store_processing_us += dirty_store_processing_time.as_us();
2655        timings.dirty_ancient_stores = dirty_ancient_stores.load(Ordering::Relaxed);
2656
2657        let mut collect_delta_keys = Measure::start("key_create");
2658        self.remove_uncleaned_slots_up_to_slot_and_move_pubkeys(max_slot_inclusive, &candidates);
2659        collect_delta_keys.stop();
2660        timings.collect_delta_keys_us += collect_delta_keys.as_us();
2661
2662        timings.delta_key_count = Self::count_pubkeys(&candidates);
2663
2664        // Check if we should purge any of the
2665        // zero_lamport_accounts_to_purge_later, based on the
2666        // latest_full_snapshot_slot.
2667        let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
2668        assert!(
2669            latest_full_snapshot_slot.is_some()
2670                || self
2671                    .zero_lamport_accounts_to_purge_after_full_snapshot
2672                    .is_empty(),
2673            "if snapshots are disabled, then zero_lamport_accounts_to_purge_later should always \
2674             be empty"
2675        );
2676        if let Some(latest_full_snapshot_slot) = latest_full_snapshot_slot {
2677            self.zero_lamport_accounts_to_purge_after_full_snapshot
2678                .retain(|(slot, pubkey)| {
2679                    let is_candidate_for_clean =
2680                        max_slot_inclusive >= *slot && latest_full_snapshot_slot >= *slot;
2681                    if is_candidate_for_clean {
2682                        insert_candidate(*pubkey, true);
2683                    }
2684                    !is_candidate_for_clean
2685                });
2686        }
2687
2688        (candidates, min_dirty_slot)
2689    }
2690
2691    /// Call clean_accounts() with the common parameters that tests/benches use.
2692    pub fn clean_accounts_for_tests(&self) {
2693        self.clean_accounts(
2694            None,
2695            false,
2696            &EpochSchedule::default(),
2697            if self.ancient_append_vec_offset.is_some() {
2698                OldStoragesPolicy::Leave
2699            } else {
2700                OldStoragesPolicy::Clean
2701            },
2702        )
2703    }
2704
2705    /// called with cli argument to verify refcounts are correct on all accounts
2706    /// this is very slow
2707    /// this function will call Rayon par_iter, so you will want to have thread pool installed if
2708    /// you want to call this without consuming all the cores on the CPU.
2709    fn exhaustively_verify_refcounts(&self, max_slot_inclusive: Option<Slot>) {
2710        let max_slot_inclusive =
2711            max_slot_inclusive.unwrap_or_else(|| self.accounts_index.max_root_inclusive());
2712        info!("exhaustively verifying refcounts as of slot: {max_slot_inclusive}");
2713        let pubkey_refcount = DashMap::<Pubkey, Vec<Slot>>::default();
2714        let slots = self.storage.all_slots();
2715        // populate
2716        slots.into_par_iter().for_each(|slot| {
2717            if slot > max_slot_inclusive {
2718                return;
2719            }
2720            if let Some(storage) = self.storage.get_slot_storage_entry(slot) {
2721                storage.accounts.scan_accounts(|account| {
2722                    let pk = account.pubkey();
2723                    match pubkey_refcount.entry(*pk) {
2724                        dashmap::mapref::entry::Entry::Occupied(mut occupied_entry) => {
2725                            if !occupied_entry.get().iter().any(|s| s == &slot) {
2726                                occupied_entry.get_mut().push(slot);
2727                            }
2728                        }
2729                        dashmap::mapref::entry::Entry::Vacant(vacant_entry) => {
2730                            vacant_entry.insert(vec![slot]);
2731                        }
2732                    }
2733                });
2734            }
2735        });
2736        let total = pubkey_refcount.len();
2737        let failed = AtomicBool::default();
2738        let threads = quarter_thread_count();
2739        let per_batch = total / threads;
2740        (0..=threads).into_par_iter().for_each(|attempt| {
2741            pubkey_refcount
2742                .iter()
2743                .skip(attempt * per_batch)
2744                .take(per_batch)
2745                .for_each(|entry| {
2746                    if failed.load(Ordering::Relaxed) {
2747                        return;
2748                    }
2749
2750                    self.accounts_index
2751                        .get_and_then(entry.key(), |index_entry| {
2752                            if let Some(index_entry) = index_entry {
2753                                match (index_entry.ref_count() as usize).cmp(&entry.value().len()) {
2754                                    std::cmp::Ordering::Equal => {
2755                                        // ref counts match, nothing to do here
2756                                    }
2757                                    std::cmp::Ordering::Greater => {
2758                                        let slot_list = index_entry.slot_list.read().unwrap();
2759                                        let num_too_new = slot_list
2760                                            .iter()
2761                                            .filter(|(slot, _)| slot > &max_slot_inclusive)
2762                                            .count();
2763
2764                                        if ((index_entry.ref_count() as usize) - num_too_new)
2765                                            > entry.value().len()
2766                                        {
2767                                            failed.store(true, Ordering::Relaxed);
2768                                            error!(
2769                                                "exhaustively_verify_refcounts: {} refcount too \
2770                                                 large: {}, should be: {}, {:?}, {:?}, too_new: \
2771                                                 {num_too_new}",
2772                                                entry.key(),
2773                                                index_entry.ref_count(),
2774                                                entry.value().len(),
2775                                                *entry.value(),
2776                                                slot_list
2777                                            );
2778                                        }
2779                                    }
2780                                    std::cmp::Ordering::Less => {
2781                                        error!(
2782                                            "exhaustively_verify_refcounts: {} refcount too \
2783                                             small: {}, should be: {}, {:?}, {:?}",
2784                                            entry.key(),
2785                                            index_entry.ref_count(),
2786                                            entry.value().len(),
2787                                            *entry.value(),
2788                                            index_entry.slot_list.read().unwrap()
2789                                        );
2790                                    }
2791                                }
2792                            };
2793                            (false, ())
2794                        });
2795                });
2796        });
2797        if failed.load(Ordering::Relaxed) {
2798            panic!("exhaustively_verify_refcounts failed");
2799        }
2800    }
2801
2802    // Purge zero lamport accounts and older rooted account states as garbage
2803    // collection
2804    // Only remove those accounts where the entire rooted history of the account
2805    // can be purged because there are no live append vecs in the ancestors
2806    pub fn clean_accounts(
2807        &self,
2808        max_clean_root_inclusive: Option<Slot>,
2809        is_startup: bool,
2810        epoch_schedule: &EpochSchedule,
2811        old_storages_policy: OldStoragesPolicy,
2812    ) {
2813        if self.exhaustively_verify_refcounts {
2814            //at startup use all cores to verify refcounts
2815            if is_startup {
2816                self.exhaustively_verify_refcounts(max_clean_root_inclusive);
2817            } else {
2818                // otherwise, use the cleaning thread pool
2819                self.thread_pool_clean
2820                    .install(|| self.exhaustively_verify_refcounts(max_clean_root_inclusive));
2821            }
2822        }
2823
2824        let _guard = self.active_stats.activate(ActiveStatItem::Clean);
2825
2826        let ancient_account_cleans = AtomicU64::default();
2827        let purges_old_accounts_count = AtomicU64::default();
2828
2829        let mut measure_all = Measure::start("clean_accounts");
2830        let max_clean_root_inclusive = self.max_clean_root(max_clean_root_inclusive);
2831
2832        self.report_store_stats();
2833
2834        let active_guard = self
2835            .active_stats
2836            .activate(ActiveStatItem::CleanConstructCandidates);
2837        let mut measure_construct_candidates = Measure::start("construct_candidates");
2838        let mut key_timings = CleanKeyTimings::default();
2839        let (candidates, min_dirty_slot) = self.construct_candidate_clean_keys(
2840            max_clean_root_inclusive,
2841            is_startup,
2842            &mut key_timings,
2843            epoch_schedule,
2844            old_storages_policy,
2845        );
2846        measure_construct_candidates.stop();
2847        drop(active_guard);
2848
2849        let num_candidates = Self::count_pubkeys(&candidates);
2850        let found_not_zero_accum = AtomicU64::new(0);
2851        let not_found_on_fork_accum = AtomicU64::new(0);
2852        let missing_accum = AtomicU64::new(0);
2853        let useful_accum = AtomicU64::new(0);
2854        let reclaims: SlotList<AccountInfo> = Vec::with_capacity(num_candidates as usize);
2855        let reclaims = Mutex::new(reclaims);
2856        let pubkeys_removed_from_accounts_index: PubkeysRemovedFromAccountsIndex = HashSet::new();
2857        let pubkeys_removed_from_accounts_index = Mutex::new(pubkeys_removed_from_accounts_index);
2858        // parallel scan the index.
2859        let do_clean_scan = || {
2860            candidates.par_iter().for_each(|candidates_bin| {
2861                let mut found_not_zero = 0;
2862                let mut not_found_on_fork = 0;
2863                let mut missing = 0;
2864                let mut useful = 0;
2865                let mut purges_old_accounts_local = 0;
2866                let mut candidates_bin = candidates_bin.write().unwrap();
2867                // Iterate over each HashMap entry to
2868                // avoid capturing the HashMap in the
2869                // closure passed to scan thus making
2870                // conflicting read and write borrows.
2871                candidates_bin.retain(|candidate_pubkey, candidate_info| {
2872                    let mut should_collect_reclaims = false;
2873                    self.accounts_index.scan(
2874                        iter::once(candidate_pubkey),
2875                        |_candidate_pubkey, slot_list_and_ref_count, _entry| {
2876                            let mut useless = true;
2877                            if let Some((slot_list, ref_count)) = slot_list_and_ref_count {
2878                                // find the highest rooted slot in the slot list
2879                                let index_in_slot_list = self.accounts_index.latest_slot(
2880                                    None,
2881                                    slot_list,
2882                                    max_clean_root_inclusive,
2883                                );
2884
2885                                match index_in_slot_list {
2886                                    Some(index_in_slot_list) => {
2887                                        // found info relative to max_clean_root
2888                                        let (slot, account_info) = &slot_list[index_in_slot_list];
2889                                        if account_info.is_zero_lamport() {
2890                                            useless = false;
2891                                            // The latest one is zero lamports. We may be able to purge it.
2892                                            // Add all the rooted entries that contain this pubkey.
2893                                            // We know the highest rooted entry is zero lamports.
2894                                            candidate_info.slot_list =
2895                                                self.accounts_index.get_rooted_entries(
2896                                                    slot_list,
2897                                                    max_clean_root_inclusive,
2898                                                );
2899                                            candidate_info.ref_count = ref_count;
2900                                        } else {
2901                                            found_not_zero += 1;
2902                                        }
2903
2904                                        // If this candidate has multiple rooted slot list entries,
2905                                        // we should reclaim the older ones.
2906                                        if slot_list.len() > 1
2907                                            && *slot
2908                                                <= max_clean_root_inclusive.unwrap_or(Slot::MAX)
2909                                        {
2910                                            should_collect_reclaims = true;
2911                                            purges_old_accounts_local += 1;
2912                                            useless = false;
2913                                        }
2914                                    }
2915                                    None => {
2916                                        // This pubkey is in the index but not in a root slot, so clean
2917                                        // it up by adding it to the to-be-purged list.
2918                                        //
2919                                        // Also, this pubkey must have been touched by some slot since
2920                                        // it was in the dirty list, so we assume that the slot it was
2921                                        // touched in must be unrooted.
2922                                        not_found_on_fork += 1;
2923                                        should_collect_reclaims = true;
2924                                        purges_old_accounts_local += 1;
2925                                        useless = false;
2926                                    }
2927                                }
2928                            } else {
2929                                missing += 1;
2930                            }
2931                            if !useless {
2932                                useful += 1;
2933                            }
2934                            AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
2935                        },
2936                        None,
2937                        false,
2938                        if candidate_info.might_contain_zero_lamport_entry {
2939                            ScanFilter::All
2940                        } else {
2941                            self.scan_filter_for_shrinking
2942                        },
2943                    );
2944                    if should_collect_reclaims {
2945                        let reclaims_new = self.collect_reclaims(
2946                            candidate_pubkey,
2947                            max_clean_root_inclusive,
2948                            &ancient_account_cleans,
2949                            epoch_schedule,
2950                            &pubkeys_removed_from_accounts_index,
2951                        );
2952                        if !reclaims_new.is_empty() {
2953                            reclaims.lock().unwrap().extend(reclaims_new);
2954                        }
2955                    }
2956                    !candidate_info.slot_list.is_empty()
2957                });
2958                found_not_zero_accum.fetch_add(found_not_zero, Ordering::Relaxed);
2959                not_found_on_fork_accum.fetch_add(not_found_on_fork, Ordering::Relaxed);
2960                missing_accum.fetch_add(missing, Ordering::Relaxed);
2961                useful_accum.fetch_add(useful, Ordering::Relaxed);
2962                purges_old_accounts_count.fetch_add(purges_old_accounts_local, Ordering::Relaxed);
2963            });
2964        };
2965        let active_guard = self
2966            .active_stats
2967            .activate(ActiveStatItem::CleanScanCandidates);
2968        let mut accounts_scan = Measure::start("accounts_scan");
2969        if is_startup {
2970            do_clean_scan();
2971        } else {
2972            self.thread_pool_clean.install(do_clean_scan);
2973        }
2974        accounts_scan.stop();
2975        drop(active_guard);
2976
2977        let retained_keys_count = Self::count_pubkeys(&candidates);
2978        let reclaims = reclaims.into_inner().unwrap();
2979        let mut pubkeys_removed_from_accounts_index =
2980            pubkeys_removed_from_accounts_index.into_inner().unwrap();
2981
2982        let active_guard = self.active_stats.activate(ActiveStatItem::CleanOldAccounts);
2983        let mut clean_old_rooted = Measure::start("clean_old_roots");
2984        let (purged_account_slots, removed_accounts) =
2985            self.clean_accounts_older_than_root(&reclaims, &pubkeys_removed_from_accounts_index);
2986        clean_old_rooted.stop();
2987        drop(active_guard);
2988
2989        // Calculate store counts as if everything was purged
2990        // Then purge if we can
2991        let active_guard = self
2992            .active_stats
2993            .activate(ActiveStatItem::CleanCollectStoreCounts);
2994        let mut store_counts_time = Measure::start("store_counts");
2995        let mut store_counts: HashMap<Slot, (usize, HashSet<Pubkey>)> = HashMap::new();
2996        for candidates_bin in candidates.iter() {
2997            for (
2998                pubkey,
2999                CleaningInfo {
3000                    slot_list,
3001                    ref_count,
3002                    ..
3003                },
3004            ) in candidates_bin.write().unwrap().iter_mut()
3005            {
3006                debug_assert!(!slot_list.is_empty(), "candidate slot_list can't be empty");
3007                if purged_account_slots.contains_key(pubkey) {
3008                    *ref_count = self.accounts_index.ref_count_from_storage(pubkey);
3009                }
3010                slot_list.retain(|(slot, account_info)| {
3011                    let was_slot_purged = purged_account_slots
3012                        .get(pubkey)
3013                        .map(|slots_removed| slots_removed.contains(slot))
3014                        .unwrap_or(false);
3015                    if was_slot_purged {
3016                        // No need to look up the slot storage below if the entire
3017                        // slot was purged
3018                        return false;
3019                    }
3020                    // Check if this update in `slot` to the account with `key` was reclaimed earlier by
3021                    // `clean_accounts_older_than_root()`
3022                    let was_reclaimed = removed_accounts
3023                        .get(slot)
3024                        .map(|store_removed| store_removed.contains(&account_info.offset()))
3025                        .unwrap_or(false);
3026                    if was_reclaimed {
3027                        return false;
3028                    }
3029                    if let Some(store_count) = store_counts.get_mut(slot) {
3030                        store_count.0 -= 1;
3031                        store_count.1.insert(*pubkey);
3032                    } else {
3033                        let mut key_set = HashSet::new();
3034                        key_set.insert(*pubkey);
3035                        assert!(
3036                            !account_info.is_cached(),
3037                            "The Accounts Cache must be flushed first for this account info. \
3038                             pubkey: {}, slot: {}",
3039                            *pubkey,
3040                            *slot
3041                        );
3042                        let count = self
3043                            .storage
3044                            .get_account_storage_entry(*slot, account_info.store_id())
3045                            .map(|store| store.count())
3046                            .unwrap()
3047                            - 1;
3048                        debug!(
3049                            "store_counts, inserting slot: {}, store id: {}, count: {}",
3050                            slot,
3051                            account_info.store_id(),
3052                            count
3053                        );
3054                        store_counts.insert(*slot, (count, key_set));
3055                    }
3056                    true
3057                });
3058            }
3059        }
3060        store_counts_time.stop();
3061        drop(active_guard);
3062
3063        let active_guard = self
3064            .active_stats
3065            .activate(ActiveStatItem::CleanCalcDeleteDeps);
3066        let mut calc_deps_time = Measure::start("calc_deps");
3067        self.calc_delete_dependencies(&candidates, &mut store_counts, min_dirty_slot);
3068        calc_deps_time.stop();
3069        drop(active_guard);
3070
3071        let active_guard = self
3072            .active_stats
3073            .activate(ActiveStatItem::CleanFilterZeroLamport);
3074        let mut purge_filter = Measure::start("purge_filter");
3075        self.filter_zero_lamport_clean_for_incremental_snapshots(
3076            max_clean_root_inclusive,
3077            &store_counts,
3078            &candidates,
3079        );
3080        purge_filter.stop();
3081        drop(active_guard);
3082
3083        let active_guard = self.active_stats.activate(ActiveStatItem::CleanReclaims);
3084        let mut reclaims_time = Measure::start("reclaims");
3085        // Recalculate reclaims with new purge set
3086        let mut pubkey_to_slot_set = Vec::new();
3087        for candidates_bin in candidates.iter() {
3088            let candidates_bin = candidates_bin.read().unwrap();
3089            let mut bin_set = candidates_bin
3090                .iter()
3091                .filter_map(|(pubkey, cleaning_info)| {
3092                    let CleaningInfo {
3093                        slot_list,
3094                        ref_count: _,
3095                        ..
3096                    } = cleaning_info;
3097                    (!slot_list.is_empty()).then_some((
3098                        *pubkey,
3099                        slot_list
3100                            .iter()
3101                            .map(|(slot, _)| *slot)
3102                            .collect::<HashSet<Slot>>(),
3103                    ))
3104                })
3105                .collect::<Vec<_>>();
3106            pubkey_to_slot_set.append(&mut bin_set);
3107        }
3108
3109        let (reclaims, pubkeys_removed_from_accounts_index2) =
3110            self.purge_keys_exact(pubkey_to_slot_set.iter());
3111        pubkeys_removed_from_accounts_index.extend(pubkeys_removed_from_accounts_index2);
3112
3113        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
3114        // and those stores may be used for background hashing.
3115        let reset_accounts = false;
3116        self.handle_reclaims(
3117            (!reclaims.is_empty()).then(|| reclaims.iter()),
3118            None,
3119            reset_accounts,
3120            &pubkeys_removed_from_accounts_index,
3121            HandleReclaims::ProcessDeadSlots(&self.clean_accounts_stats.purge_stats),
3122        );
3123
3124        reclaims_time.stop();
3125        drop(active_guard);
3126
3127        measure_all.stop();
3128
3129        self.clean_accounts_stats.report();
3130        datapoint_info!(
3131            "clean_accounts",
3132            ("max_clean_root", max_clean_root_inclusive, Option<i64>),
3133            ("total_us", measure_all.as_us(), i64),
3134            (
3135                "collect_delta_keys_us",
3136                key_timings.collect_delta_keys_us,
3137                i64
3138            ),
3139            ("oldest_dirty_slot", key_timings.oldest_dirty_slot, i64),
3140            (
3141                "pubkeys_removed_from_accounts_index",
3142                pubkeys_removed_from_accounts_index.len(),
3143                i64
3144            ),
3145            (
3146                "dirty_ancient_stores",
3147                key_timings.dirty_ancient_stores,
3148                i64
3149            ),
3150            (
3151                "dirty_store_processing_us",
3152                key_timings.dirty_store_processing_us,
3153                i64
3154            ),
3155            ("construct_candidates_us", measure_construct_candidates.as_us(), i64),
3156            ("accounts_scan", accounts_scan.as_us(), i64),
3157            ("clean_old_rooted", clean_old_rooted.as_us(), i64),
3158            ("store_counts", store_counts_time.as_us(), i64),
3159            ("purge_filter", purge_filter.as_us(), i64),
3160            ("calc_deps", calc_deps_time.as_us(), i64),
3161            ("reclaims", reclaims_time.as_us(), i64),
3162            ("delta_insert_us", key_timings.delta_insert_us, i64),
3163            ("delta_key_count", key_timings.delta_key_count, i64),
3164            ("dirty_pubkeys_count", key_timings.dirty_pubkeys_count, i64),
3165            ("useful_keys", useful_accum.load(Ordering::Relaxed), i64),
3166            ("total_keys_count", num_candidates, i64),
3167            ("retained_keys_count", retained_keys_count, i64),
3168            (
3169                "scan_found_not_zero",
3170                found_not_zero_accum.load(Ordering::Relaxed),
3171                i64
3172            ),
3173            (
3174                "scan_not_found_on_fork",
3175                not_found_on_fork_accum.load(Ordering::Relaxed),
3176                i64
3177            ),
3178            ("scan_missing", missing_accum.load(Ordering::Relaxed), i64),
3179            (
3180                "get_account_sizes_us",
3181                self.clean_accounts_stats
3182                    .get_account_sizes_us
3183                    .swap(0, Ordering::Relaxed),
3184                i64
3185            ),
3186            (
3187                "slots_cleaned",
3188                self.clean_accounts_stats
3189                    .slots_cleaned
3190                    .swap(0, Ordering::Relaxed),
3191                i64
3192            ),
3193            (
3194                "clean_old_root_us",
3195                self.clean_accounts_stats
3196                    .clean_old_root_us
3197                    .swap(0, Ordering::Relaxed),
3198                i64
3199            ),
3200            (
3201                "clean_old_root_reclaim_us",
3202                self.clean_accounts_stats
3203                    .clean_old_root_reclaim_us
3204                    .swap(0, Ordering::Relaxed),
3205                i64
3206            ),
3207            (
3208                "remove_dead_accounts_remove_us",
3209                self.clean_accounts_stats
3210                    .remove_dead_accounts_remove_us
3211                    .swap(0, Ordering::Relaxed),
3212                i64
3213            ),
3214            (
3215                "remove_dead_accounts_shrink_us",
3216                self.clean_accounts_stats
3217                    .remove_dead_accounts_shrink_us
3218                    .swap(0, Ordering::Relaxed),
3219                i64
3220            ),
3221            (
3222                "clean_stored_dead_slots_us",
3223                self.clean_accounts_stats
3224                    .clean_stored_dead_slots_us
3225                    .swap(0, Ordering::Relaxed),
3226                i64
3227            ),
3228            (
3229                "roots_added",
3230                self.accounts_index.roots_added.swap(0, Ordering::Relaxed),
3231                i64
3232            ),
3233            (
3234                "purge_older_root_entries_one_slot_list",
3235                self.accounts_index
3236                    .purge_older_root_entries_one_slot_list
3237                    .swap(0, Ordering::Relaxed),
3238                i64
3239            ),
3240            (
3241                "roots_removed",
3242                self.accounts_index.roots_removed.swap(0, Ordering::Relaxed),
3243                i64
3244            ),
3245            (
3246                "active_scans",
3247                self.accounts_index.active_scans.load(Ordering::Relaxed),
3248                i64
3249            ),
3250            (
3251                "max_distance_to_min_scan_slot",
3252                self.accounts_index
3253                    .max_distance_to_min_scan_slot
3254                    .swap(0, Ordering::Relaxed),
3255                i64
3256            ),
3257            (
3258                "unref_zero_count",
3259                self.accounts_index
3260                    .unref_zero_count
3261                    .swap(0, Ordering::Relaxed),
3262                i64
3263            ),
3264            (
3265                "ancient_account_cleans",
3266                ancient_account_cleans.load(Ordering::Relaxed),
3267                i64
3268            ),
3269            (
3270                "purges_old_accounts_count",
3271                purges_old_accounts_count.load(Ordering::Relaxed),
3272                i64
3273            ),
3274            ("next_store_id", self.next_id.load(Ordering::Relaxed), i64),
3275        );
3276    }
3277
3278    /// Removes the accounts in the input `reclaims` from the tracked "count" of
3279    /// their corresponding  storage entries. Note this does not actually free
3280    /// the memory from the storage entries until all the storage entries for
3281    /// a given slot `S` are empty, at which point `process_dead_slots` will
3282    /// remove all the storage entries for `S`.
3283    ///
3284    /// # Arguments
3285    /// * `reclaims` - The accounts to remove from storage entries' "count". Note here
3286    ///    that we should not remove cache entries, only entries for accounts actually
3287    ///    stored in a storage entry.
3288    ///
3289    /// * `expected_single_dead_slot` - A correctness assertion. If this is equal to `Some(S)`,
3290    ///    then the function will check that the only slot being cleaned up in `reclaims`
3291    ///    is the slot == `S`. This is true for instance when `handle_reclaims` is called
3292    ///    from store or slot shrinking, as those should only touch the slot they are
3293    ///    currently storing to or shrinking.
3294    ///
3295    /// * `reset_accounts` - Reset the append_vec store when the store is dead (count==0)
3296    ///    From the clean and shrink paths it should be false since there may be an in-progress
3297    ///    hash operation and the stores may hold accounts that need to be unref'ed.
3298    /// * `pubkeys_removed_from_accounts_index` - These keys have already been removed from the accounts index
3299    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
3300    /// * `handle_reclaims`. `purge_stats` are stats used to track performance of purging dead slots if
3301    ///    value is `ProcessDeadSlots`.
3302    ///    Otherwise, there can be no dead slots
3303    ///    that happen as a result of this call, and the function will check that no slots are
3304    ///    cleaned up/removed via `process_dead_slots`. For instance, on store, no slots should
3305    ///    be cleaned up, but during the background clean accounts purges accounts from old rooted
3306    ///    slots, so outdated slots may be removed.
3307    fn handle_reclaims<'a, I>(
3308        &'a self,
3309        reclaims: Option<I>,
3310        expected_single_dead_slot: Option<Slot>,
3311        reset_accounts: bool,
3312        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3313        handle_reclaims: HandleReclaims<'a>,
3314    ) -> ReclaimResult
3315    where
3316        I: Iterator<Item = &'a (Slot, AccountInfo)>,
3317    {
3318        let mut reclaim_result = ReclaimResult::default();
3319        if let Some(reclaims) = reclaims {
3320            let (dead_slots, reclaimed_offsets) =
3321                self.remove_dead_accounts(reclaims, expected_single_dead_slot, reset_accounts);
3322            reclaim_result.1 = reclaimed_offsets;
3323
3324            if let HandleReclaims::ProcessDeadSlots(purge_stats) = handle_reclaims {
3325                if let Some(expected_single_dead_slot) = expected_single_dead_slot {
3326                    assert!(dead_slots.len() <= 1);
3327                    if dead_slots.len() == 1 {
3328                        assert!(dead_slots.contains(&expected_single_dead_slot));
3329                    }
3330                }
3331
3332                self.process_dead_slots(
3333                    &dead_slots,
3334                    Some(&mut reclaim_result.0),
3335                    purge_stats,
3336                    pubkeys_removed_from_accounts_index,
3337                );
3338            } else {
3339                assert!(dead_slots.is_empty());
3340            }
3341        }
3342        reclaim_result
3343    }
3344
3345    /// During clean, some zero-lamport accounts that are marked for purge should *not* actually
3346    /// get purged.  Filter out those accounts here by removing them from 'candidates'.
3347    /// Candidates may contain entries with empty slots list in CleaningInfo.
3348    /// The function removes such entries from 'candidates'.
3349    ///
3350    /// When using incremental snapshots, do not purge zero-lamport accounts if the slot is higher
3351    /// than the latest full snapshot slot.  This is to protect against the following scenario:
3352    ///
3353    ///   ```text
3354    ///   A full snapshot is taken, including account 'alpha' with a non-zero balance.  In a later slot,
3355    ///   alpha's lamports go to zero.  Eventually, cleaning runs.  Without this change,
3356    ///   alpha would be cleaned up and removed completely. Finally, an incremental snapshot is taken.
3357    ///
3358    ///   Later, the incremental and full snapshots are used to rebuild the bank and accounts
3359    ///   database (e.x. if the node restarts).  The full snapshot _does_ contain alpha
3360    ///   and its balance is non-zero.  However, since alpha was cleaned up in a slot after the full
3361    ///   snapshot slot (due to having zero lamports), the incremental snapshot would not contain alpha.
3362    ///   Thus, the accounts database will contain the old, incorrect info for alpha with a non-zero
3363    ///   balance.  Very bad!
3364    ///   ```
3365    ///
3366    /// This filtering step can be skipped if there is no `latest_full_snapshot_slot`, or if the
3367    /// `max_clean_root_inclusive` is less-than-or-equal-to the `latest_full_snapshot_slot`.
3368    fn filter_zero_lamport_clean_for_incremental_snapshots(
3369        &self,
3370        max_clean_root_inclusive: Option<Slot>,
3371        store_counts: &HashMap<Slot, (usize, HashSet<Pubkey>)>,
3372        candidates: &[RwLock<HashMap<Pubkey, CleaningInfo>>],
3373    ) {
3374        let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
3375        let should_filter_for_incremental_snapshots = max_clean_root_inclusive.unwrap_or(Slot::MAX)
3376            > latest_full_snapshot_slot.unwrap_or(Slot::MAX);
3377        assert!(
3378            latest_full_snapshot_slot.is_some() || !should_filter_for_incremental_snapshots,
3379            "if filtering for incremental snapshots, then snapshots should be enabled",
3380        );
3381
3382        for bin in candidates {
3383            let mut bin = bin.write().unwrap();
3384            bin.retain(|pubkey, cleaning_info| {
3385                let CleaningInfo {
3386                    slot_list,
3387                    ref_count: _,
3388                    ..
3389                } = cleaning_info;
3390                debug_assert!(!slot_list.is_empty(), "candidate slot_list can't be empty");
3391                // Only keep candidates where the entire history of the account in the root set
3392                // can be purged. All AppendVecs for those updates are dead.
3393                for (slot, _account_info) in slot_list.iter() {
3394                    if let Some(store_count) = store_counts.get(slot) {
3395                        if store_count.0 != 0 {
3396                            // one store this pubkey is in is not being removed, so this pubkey cannot be removed at all
3397                            return false;
3398                        }
3399                    } else {
3400                        // store is not being removed, so this pubkey cannot be removed at all
3401                        return false;
3402                    }
3403                }
3404
3405                // Exit early if not filtering more for incremental snapshots
3406                if !should_filter_for_incremental_snapshots {
3407                    return true;
3408                }
3409
3410                // Safety: We exited early if the slot list was empty,
3411                // so we're guaranteed here that `.max_by_key()` returns Some.
3412                let (slot, account_info) = slot_list
3413                    .iter()
3414                    .max_by_key(|(slot, _account_info)| slot)
3415                    .unwrap();
3416
3417                // Do *not* purge zero-lamport accounts if the slot is greater than the last full
3418                // snapshot slot.  Since we're `retain`ing the accounts-to-purge, I felt creating
3419                // the `cannot_purge` variable made this easier to understand.  Accounts that do
3420                // not get purged here are added to a list so they be considered for purging later
3421                // (i.e. after the next full snapshot).
3422                assert!(account_info.is_zero_lamport());
3423                let cannot_purge = *slot > latest_full_snapshot_slot.unwrap();
3424                if cannot_purge {
3425                    self.zero_lamport_accounts_to_purge_after_full_snapshot
3426                        .insert((*slot, *pubkey));
3427                }
3428                !cannot_purge
3429            });
3430        }
3431    }
3432
3433    // Must be kept private!, does sensitive cleanup that should only be called from
3434    // supported pipelines in AccountsDb
3435    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
3436    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
3437    fn process_dead_slots(
3438        &self,
3439        dead_slots: &IntSet<Slot>,
3440        purged_account_slots: Option<&mut AccountSlots>,
3441        purge_stats: &PurgeStats,
3442        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3443    ) {
3444        if dead_slots.is_empty() {
3445            return;
3446        }
3447        let mut clean_dead_slots = Measure::start("reclaims::clean_dead_slots");
3448        self.clean_stored_dead_slots(
3449            dead_slots,
3450            purged_account_slots,
3451            pubkeys_removed_from_accounts_index,
3452        );
3453        clean_dead_slots.stop();
3454
3455        let mut purge_removed_slots = Measure::start("reclaims::purge_removed_slots");
3456        self.purge_dead_slots_from_storage(dead_slots.iter(), purge_stats);
3457        purge_removed_slots.stop();
3458
3459        // If the slot is dead, remove the need to shrink the storages as
3460        // the storage entries will be purged.
3461        {
3462            let mut list = self.shrink_candidate_slots.lock().unwrap();
3463            for slot in dead_slots {
3464                list.remove(slot);
3465            }
3466        }
3467
3468        debug!(
3469            "process_dead_slots({}): {} {} {:?}",
3470            dead_slots.len(),
3471            clean_dead_slots,
3472            purge_removed_slots,
3473            dead_slots,
3474        );
3475    }
3476
3477    /// load the account index entry for the first `count` items in `accounts`
3478    /// store a reference to all alive accounts in `alive_accounts`
3479    /// store all pubkeys dead in `slot_to_shrink` in `pubkeys_to_unref`
3480    /// return sum of account size for all alive accounts
3481    fn load_accounts_index_for_shrink<'a, T: ShrinkCollectRefs<'a>>(
3482        &self,
3483        accounts: &'a [AccountFromStorage],
3484        stats: &ShrinkStats,
3485        slot_to_shrink: Slot,
3486    ) -> LoadAccountsIndexForShrink<'a, T> {
3487        let count = accounts.len();
3488        let mut alive_accounts = T::with_capacity(count, slot_to_shrink);
3489        let mut pubkeys_to_unref = Vec::with_capacity(count);
3490        let mut zero_lamport_single_ref_pubkeys = Vec::with_capacity(count);
3491
3492        let mut alive = 0;
3493        let mut dead = 0;
3494        let mut index = 0;
3495        let mut index_scan_returned_some_count = 0;
3496        let mut index_scan_returned_none_count = 0;
3497        let mut all_are_zero_lamports = true;
3498        let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
3499        self.accounts_index.scan(
3500            accounts.iter().map(|account| account.pubkey()),
3501            |pubkey, slots_refs, _entry| {
3502                let stored_account = &accounts[index];
3503                let mut do_populate_accounts_for_shrink = |ref_count, slot_list| {
3504                    if stored_account.is_zero_lamport()
3505                        && ref_count == 1
3506                        && latest_full_snapshot_slot
3507                            .map(|latest_full_snapshot_slot| {
3508                                latest_full_snapshot_slot >= slot_to_shrink
3509                            })
3510                            .unwrap_or(true)
3511                    {
3512                        // only do this if our slot is prior to the latest full snapshot
3513                        // we found a zero lamport account that is the only instance of this account. We can delete it completely.
3514                        zero_lamport_single_ref_pubkeys.push(pubkey);
3515                        self.add_uncleaned_pubkeys_after_shrink(
3516                            slot_to_shrink,
3517                            [*pubkey].into_iter(),
3518                        );
3519                    } else {
3520                        all_are_zero_lamports &= stored_account.is_zero_lamport();
3521                        alive_accounts.add(ref_count, stored_account, slot_list);
3522                        alive += 1;
3523                    }
3524                };
3525                if let Some((slot_list, ref_count)) = slots_refs {
3526                    index_scan_returned_some_count += 1;
3527                    let is_alive = slot_list.iter().any(|(slot, _acct_info)| {
3528                        // 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
3529                        *slot == slot_to_shrink
3530                    });
3531
3532                    if !is_alive {
3533                        // This pubkey was found in the storage, but no longer exists in the index.
3534                        // It would have had a ref to the storage from the initial store, but it will
3535                        // not exist in the re-written slot. Unref it to keep the index consistent with
3536                        // rewriting the storage entries.
3537                        pubkeys_to_unref.push(pubkey);
3538                        dead += 1;
3539                    } else {
3540                        do_populate_accounts_for_shrink(ref_count, slot_list);
3541                    }
3542                } else {
3543                    index_scan_returned_none_count += 1;
3544                    // getting None here means the account is 'normal' and was written to disk. This means it must have ref_count=1 and
3545                    // slot_list.len() = 1. This means it must be alive in this slot. This is by far the most common case.
3546                    // Note that we could get Some(...) here if the account is in the in mem index because it is hot.
3547                    // Note this could also mean the account isn't on disk either. That would indicate a bug in accounts db.
3548                    // Account is alive.
3549                    let ref_count = 1;
3550                    let slot_list = [(slot_to_shrink, AccountInfo::default())];
3551                    do_populate_accounts_for_shrink(ref_count, &slot_list);
3552                }
3553                index += 1;
3554                AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
3555            },
3556            None,
3557            false,
3558            self.scan_filter_for_shrinking,
3559        );
3560        assert_eq!(index, std::cmp::min(accounts.len(), count));
3561        stats
3562            .index_scan_returned_some
3563            .fetch_add(index_scan_returned_some_count, Ordering::Relaxed);
3564        stats
3565            .index_scan_returned_none
3566            .fetch_add(index_scan_returned_none_count, Ordering::Relaxed);
3567        stats.alive_accounts.fetch_add(alive, Ordering::Relaxed);
3568        stats.dead_accounts.fetch_add(dead, Ordering::Relaxed);
3569
3570        LoadAccountsIndexForShrink {
3571            alive_accounts,
3572            pubkeys_to_unref,
3573            zero_lamport_single_ref_pubkeys,
3574            all_are_zero_lamports,
3575        }
3576    }
3577
3578    /// get all accounts in all the storages passed in
3579    /// for duplicate pubkeys, the account with the highest write_value is returned
3580    pub fn get_unique_accounts_from_storage(
3581        &self,
3582        store: &AccountStorageEntry,
3583    ) -> GetUniqueAccountsResult {
3584        let capacity = store.capacity();
3585        let mut stored_accounts = Vec::with_capacity(store.count());
3586        store.accounts.scan_index(|info| {
3587            // file_id is unused and can be anything. We will always be loading whatever storage is in the slot.
3588            let file_id = 0;
3589            stored_accounts.push(AccountFromStorage {
3590                index_info: AccountInfo::new(
3591                    StorageLocation::AppendVec(file_id, info.index_info.offset),
3592                    info.index_info.lamports,
3593                ),
3594                pubkey: info.index_info.pubkey,
3595                data_len: info.index_info.data_len,
3596            });
3597        });
3598
3599        // sort by pubkey to keep account index lookups close
3600        let num_duplicated_accounts = Self::sort_and_remove_dups(&mut stored_accounts);
3601
3602        GetUniqueAccountsResult {
3603            stored_accounts,
3604            capacity,
3605            num_duplicated_accounts,
3606        }
3607    }
3608
3609    #[cfg(feature = "dev-context-only-utils")]
3610    pub fn set_storage_access(&mut self, storage_access: StorageAccess) {
3611        self.storage_access = storage_access;
3612    }
3613
3614    /// Sort `accounts` by pubkey and removes all but the *last* of consecutive
3615    /// accounts in the vector with the same pubkey.
3616    ///
3617    /// Return the number of duplicated elements in the vector.
3618    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
3619    fn sort_and_remove_dups(accounts: &mut Vec<AccountFromStorage>) -> usize {
3620        // stable sort because we want the most recent only
3621        accounts.sort_by(|a, b| a.pubkey().cmp(b.pubkey()));
3622        let len0 = accounts.len();
3623        if accounts.len() > 1 {
3624            let mut last = 0;
3625            let mut curr = 1;
3626
3627            while curr < accounts.len() {
3628                if accounts[curr].pubkey() == accounts[last].pubkey() {
3629                    accounts[last] = accounts[curr];
3630                } else {
3631                    last += 1;
3632                    accounts[last] = accounts[curr];
3633                }
3634                curr += 1;
3635            }
3636            accounts.truncate(last + 1);
3637        }
3638        len0 - accounts.len()
3639    }
3640
3641    pub(crate) fn get_unique_accounts_from_storage_for_shrink(
3642        &self,
3643        store: &AccountStorageEntry,
3644        stats: &ShrinkStats,
3645    ) -> GetUniqueAccountsResult {
3646        let (result, storage_read_elapsed_us) =
3647            measure_us!(self.get_unique_accounts_from_storage(store));
3648        stats
3649            .storage_read_elapsed
3650            .fetch_add(storage_read_elapsed_us, Ordering::Relaxed);
3651        stats
3652            .num_duplicated_accounts
3653            .fetch_add(result.num_duplicated_accounts as u64, Ordering::Relaxed);
3654        result
3655    }
3656
3657    /// shared code for shrinking normal slots and combining into ancient append vecs
3658    /// note 'unique_accounts' is passed by ref so we can return references to data within it, avoiding self-references
3659    pub(crate) fn shrink_collect<'a: 'b, 'b, T: ShrinkCollectRefs<'b>>(
3660        &self,
3661        store: &'a AccountStorageEntry,
3662        unique_accounts: &'b GetUniqueAccountsResult,
3663        stats: &ShrinkStats,
3664    ) -> ShrinkCollect<'b, T> {
3665        let slot = store.slot();
3666
3667        let GetUniqueAccountsResult {
3668            stored_accounts,
3669            capacity,
3670            num_duplicated_accounts,
3671        } = unique_accounts;
3672
3673        let mut index_read_elapsed = Measure::start("index_read_elapsed");
3674
3675        let len = stored_accounts.len();
3676        let alive_accounts_collect = Mutex::new(T::with_capacity(len, slot));
3677        let pubkeys_to_unref_collect = Mutex::new(Vec::with_capacity(len));
3678        let zero_lamport_single_ref_pubkeys_collect = Mutex::new(Vec::with_capacity(len));
3679        stats
3680            .accounts_loaded
3681            .fetch_add(len as u64, Ordering::Relaxed);
3682        stats
3683            .num_duplicated_accounts
3684            .fetch_add(*num_duplicated_accounts as u64, Ordering::Relaxed);
3685        let all_are_zero_lamports_collect = Mutex::new(true);
3686        self.thread_pool_clean.install(|| {
3687            stored_accounts
3688                .par_chunks(SHRINK_COLLECT_CHUNK_SIZE)
3689                .for_each(|stored_accounts| {
3690                    let LoadAccountsIndexForShrink {
3691                        alive_accounts,
3692                        mut pubkeys_to_unref,
3693                        all_are_zero_lamports,
3694                        mut zero_lamport_single_ref_pubkeys,
3695                    } = self.load_accounts_index_for_shrink(stored_accounts, stats, slot);
3696
3697                    // collect
3698                    alive_accounts_collect
3699                        .lock()
3700                        .unwrap()
3701                        .collect(alive_accounts);
3702                    pubkeys_to_unref_collect
3703                        .lock()
3704                        .unwrap()
3705                        .append(&mut pubkeys_to_unref);
3706                    zero_lamport_single_ref_pubkeys_collect
3707                        .lock()
3708                        .unwrap()
3709                        .append(&mut zero_lamport_single_ref_pubkeys);
3710                    if !all_are_zero_lamports {
3711                        *all_are_zero_lamports_collect.lock().unwrap() = false;
3712                    }
3713                });
3714        });
3715
3716        let alive_accounts = alive_accounts_collect.into_inner().unwrap();
3717        let pubkeys_to_unref = pubkeys_to_unref_collect.into_inner().unwrap();
3718        let zero_lamport_single_ref_pubkeys = zero_lamport_single_ref_pubkeys_collect
3719            .into_inner()
3720            .unwrap();
3721
3722        index_read_elapsed.stop();
3723        stats
3724            .index_read_elapsed
3725            .fetch_add(index_read_elapsed.as_us(), Ordering::Relaxed);
3726
3727        let alive_total_bytes = alive_accounts.alive_bytes();
3728
3729        stats
3730            .accounts_removed
3731            .fetch_add(len - alive_accounts.len(), Ordering::Relaxed);
3732        stats.bytes_removed.fetch_add(
3733            capacity.saturating_sub(alive_total_bytes as u64),
3734            Ordering::Relaxed,
3735        );
3736        stats
3737            .bytes_written
3738            .fetch_add(alive_total_bytes as u64, Ordering::Relaxed);
3739
3740        ShrinkCollect {
3741            slot,
3742            capacity: *capacity,
3743            pubkeys_to_unref,
3744            zero_lamport_single_ref_pubkeys,
3745            alive_accounts,
3746            alive_total_bytes,
3747            total_starting_accounts: len,
3748            all_are_zero_lamports: all_are_zero_lamports_collect.into_inner().unwrap(),
3749        }
3750    }
3751
3752    /// These accounts were found during shrink of `slot` to be slot_list=[slot] and ref_count == 1 and lamports = 0.
3753    /// This means this slot contained the only account data for this pubkey and it is zero lamport.
3754    /// Thus, we did NOT treat this as an alive account, so we did NOT copy the zero lamport account to the new
3755    /// storage. So, the account will no longer be alive or exist at `slot`.
3756    /// So, first, remove the ref count since this newly shrunk storage will no longer access it.
3757    /// Second, remove `slot` from the index entry's slot list. If the slot list is now empty, then the
3758    /// pubkey can be removed completely from the index.
3759    /// In parallel with this code (which is running in the bg), the same pubkey could be revived and written to
3760    /// as part of tx processing. In that case, the slot list will contain a slot in the write cache and the
3761    /// index entry will NOT be deleted.
3762    fn remove_zero_lamport_single_ref_accounts_after_shrink(
3763        &self,
3764        zero_lamport_single_ref_pubkeys: &[&Pubkey],
3765        slot: Slot,
3766        stats: &ShrinkStats,
3767        do_assert: bool,
3768    ) {
3769        stats.purged_zero_lamports.fetch_add(
3770            zero_lamport_single_ref_pubkeys.len() as u64,
3771            Ordering::Relaxed,
3772        );
3773
3774        // we have to unref before we `purge_keys_exact`. Otherwise, we could race with the foreground with tx processing
3775        // reviving this index entry and then we'd unref the revived version, which is a refcount bug.
3776
3777        self.accounts_index.scan(
3778            zero_lamport_single_ref_pubkeys.iter().cloned(),
3779            |_pubkey, _slots_refs, _entry| AccountsIndexScanResult::Unref,
3780            if do_assert {
3781                Some(AccountsIndexScanResult::UnrefAssert0)
3782            } else {
3783                Some(AccountsIndexScanResult::UnrefLog0)
3784            },
3785            false,
3786            ScanFilter::All,
3787        );
3788
3789        zero_lamport_single_ref_pubkeys.iter().for_each(|k| {
3790            _ = self.purge_keys_exact([&(**k, slot)].into_iter());
3791        });
3792    }
3793
3794    /// common code from shrink and combine_ancient_slots
3795    /// get rid of all original store_ids in the slot
3796    pub(crate) fn remove_old_stores_shrink<'a, T: ShrinkCollectRefs<'a>>(
3797        &self,
3798        shrink_collect: &ShrinkCollect<'a, T>,
3799        stats: &ShrinkStats,
3800        shrink_in_progress: Option<ShrinkInProgress>,
3801        shrink_can_be_active: bool,
3802    ) {
3803        let mut time = Measure::start("remove_old_stores_shrink");
3804
3805        // handle the zero lamport alive accounts before calling clean
3806        // We have to update the index entries for these zero lamport pubkeys before we remove the storage in `mark_dirty_dead_stores`
3807        // that contained the accounts.
3808        self.remove_zero_lamport_single_ref_accounts_after_shrink(
3809            &shrink_collect.zero_lamport_single_ref_pubkeys,
3810            shrink_collect.slot,
3811            stats,
3812            false,
3813        );
3814
3815        // Purge old, overwritten storage entries
3816        // This has the side effect of dropping `shrink_in_progress`, which removes the old storage completely. The
3817        // index has to be correct before we drop the old storage.
3818        let dead_storages = self.mark_dirty_dead_stores(
3819            shrink_collect.slot,
3820            // If all accounts are zero lamports, then we want to mark the entire OLD append vec as dirty.
3821            // otherwise, we'll call 'add_uncleaned_pubkeys_after_shrink' just on the unref'd keys below.
3822            shrink_collect.all_are_zero_lamports,
3823            shrink_in_progress,
3824            shrink_can_be_active,
3825        );
3826        let dead_storages_len = dead_storages.len();
3827
3828        if !shrink_collect.all_are_zero_lamports {
3829            self.add_uncleaned_pubkeys_after_shrink(
3830                shrink_collect.slot,
3831                shrink_collect.pubkeys_to_unref.iter().cloned().cloned(),
3832            );
3833        }
3834
3835        let (_, drop_storage_entries_elapsed) = measure_us!(drop(dead_storages));
3836        time.stop();
3837
3838        self.stats
3839            .dropped_stores
3840            .fetch_add(dead_storages_len as u64, Ordering::Relaxed);
3841        stats
3842            .drop_storage_entries_elapsed
3843            .fetch_add(drop_storage_entries_elapsed, Ordering::Relaxed);
3844        stats
3845            .remove_old_stores_shrink_us
3846            .fetch_add(time.as_us(), Ordering::Relaxed);
3847    }
3848
3849    pub(crate) fn unref_shrunk_dead_accounts<'a>(
3850        &self,
3851        pubkeys: impl Iterator<Item = &'a Pubkey>,
3852        slot: Slot,
3853    ) {
3854        self.accounts_index.scan(
3855            pubkeys,
3856            |pubkey, slot_refs, _entry| {
3857                match slot_refs {
3858                    Some((slot_list, ref_count)) => {
3859                        // Let's handle the special case - after unref, the result is a single ref zero lamport account.
3860                        if slot_list.len() == 1 && ref_count == 2 {
3861                            if let Some((slot_alive, acct_info)) = slot_list.first() {
3862                                if acct_info.is_zero_lamport() && !acct_info.is_cached() {
3863                                    self.zero_lamport_single_ref_found(
3864                                        *slot_alive,
3865                                        acct_info.offset(),
3866                                    );
3867                                }
3868                            }
3869                        }
3870                    }
3871                    None => {
3872                        // We also expect that the accounts index must contain an
3873                        // entry for `pubkey`. Log a warning for now. In future,
3874                        // we will panic when this happens.
3875                        warn!(
3876                        "pubkey {pubkey} in slot {slot} was NOT found in accounts index during \
3877                         shrink"
3878                    );
3879                        datapoint_warn!(
3880                            "accounts_db-shink_pubkey_missing_from_index",
3881                            ("store_slot", slot, i64),
3882                            ("pubkey", pubkey.to_string(), String),
3883                        );
3884                    }
3885                }
3886                AccountsIndexScanResult::Unref
3887            },
3888            None,
3889            false,
3890            ScanFilter::All,
3891        );
3892    }
3893
3894    /// This function handles the case when zero lamport single ref accounts are found during shrink.
3895    pub(crate) fn zero_lamport_single_ref_found(&self, slot: Slot, offset: Offset) {
3896        // This function can be called when a zero lamport single ref account is
3897        // found during shrink. Therefore, we can't use the safe version of
3898        // `get_slot_storage_entry` because shrink_in_progress map may not be
3899        // empty. We have to use the unsafe version to avoid to assert failure.
3900        // However, there is a possibility that the storage entry that we get is
3901        // an old one, which is being shrunk away, because multiple slots can be
3902        // shrunk away in parallel by thread pool. If this happens, any zero
3903        // lamport single ref offset marked on the storage will be lost when the
3904        // storage is dropped. However, this is not a problem, because after the
3905        // storage being shrunk, the new storage will not have any zero lamport
3906        // single ref account anyway. Therefore, we don't need to worry about
3907        // marking zero lamport single ref offset on the new storage.
3908        if let Some(store) = self
3909            .storage
3910            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
3911        {
3912            if store.insert_zero_lamport_single_ref_account_offset(offset) {
3913                // this wasn't previously marked as zero lamport single ref
3914                self.shrink_stats
3915                    .num_zero_lamport_single_ref_accounts_found
3916                    .fetch_add(1, Ordering::Relaxed);
3917
3918                if store.num_zero_lamport_single_ref_accounts() == store.count() {
3919                    // all accounts in this storage can be dead
3920                    self.dirty_stores.entry(slot).or_insert(store);
3921                    self.shrink_stats
3922                        .num_dead_slots_added_to_clean
3923                        .fetch_add(1, Ordering::Relaxed);
3924                } else if Self::is_shrinking_productive(&store)
3925                    && self.is_candidate_for_shrink(&store)
3926                {
3927                    // this store might be eligible for shrinking now
3928                    let is_new = self.shrink_candidate_slots.lock().unwrap().insert(slot);
3929                    if is_new {
3930                        self.shrink_stats
3931                            .num_slots_with_zero_lamport_accounts_added_to_shrink
3932                            .fetch_add(1, Ordering::Relaxed);
3933                    }
3934                } else {
3935                    self.shrink_stats
3936                        .marking_zero_dead_accounts_in_non_shrinkable_store
3937                        .fetch_add(1, Ordering::Relaxed);
3938                }
3939            }
3940        }
3941    }
3942
3943    /// Shrinks `store` by rewriting the alive accounts to a new storage
3944    fn shrink_storage(&self, store: Arc<AccountStorageEntry>) {
3945        let slot = store.slot();
3946        if self.accounts_cache.contains(slot) {
3947            // 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.
3948            // 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.
3949            // 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.
3950            // During the second window, once an append vec has been created for the slot, it could be possible to try to shrink that slot.
3951            // Shrink no-ops before this function if there is no store for the slot (notice this function requires 'store' to be passed).
3952            // So, if we enter this function but the slot is still in the write cache, reasonable behavior is to skip shrinking this slot.
3953            // Flush will ONLY write alive accounts to the append vec, which is what shrink does anyway.
3954            // Flush then adds the slot to 'uncleaned_roots', which causes clean to take a look at the slot.
3955            // Clean causes us to mark accounts as dead, which causes shrink to later take a look at the slot.
3956            // This could be an assert, but it could lead to intermittency in tests.
3957            // It is 'correct' to ignore calls to shrink when a slot is still in the write cache.
3958            return;
3959        }
3960        let unique_accounts =
3961            self.get_unique_accounts_from_storage_for_shrink(&store, &self.shrink_stats);
3962        debug!("do_shrink_slot_store: slot: {}", slot);
3963        let shrink_collect =
3964            self.shrink_collect::<AliveAccounts<'_>>(&store, &unique_accounts, &self.shrink_stats);
3965
3966        // This shouldn't happen if alive_bytes is accurate.
3967        // However, it is possible that the remaining alive bytes could be 0. In that case, the whole slot should be marked dead by clean.
3968        if Self::should_not_shrink(
3969            shrink_collect.alive_total_bytes as u64,
3970            shrink_collect.capacity,
3971        ) || shrink_collect.alive_total_bytes == 0
3972        {
3973            if shrink_collect.alive_total_bytes == 0 {
3974                // clean needs to take care of this dead slot
3975                self.dirty_stores.insert(slot, store.clone());
3976            }
3977
3978            if !shrink_collect.all_are_zero_lamports {
3979                // 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.
3980                info!(
3981                    "Unexpected shrink for slot {} alive {} capacity {}, likely caused by a bug \
3982                     for calculating alive bytes.",
3983                    slot, shrink_collect.alive_total_bytes, shrink_collect.capacity
3984                );
3985            }
3986
3987            self.shrink_stats
3988                .skipped_shrink
3989                .fetch_add(1, Ordering::Relaxed);
3990            return;
3991        }
3992
3993        self.unref_shrunk_dead_accounts(shrink_collect.pubkeys_to_unref.iter().cloned(), slot);
3994
3995        let total_accounts_after_shrink = shrink_collect.alive_accounts.len();
3996        debug!(
3997            "shrinking: slot: {}, accounts: ({} => {}) bytes: {} original: {}",
3998            slot,
3999            shrink_collect.total_starting_accounts,
4000            total_accounts_after_shrink,
4001            shrink_collect.alive_total_bytes,
4002            shrink_collect.capacity,
4003        );
4004
4005        let mut stats_sub = ShrinkStatsSub::default();
4006        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
4007        let (shrink_in_progress, time_us) =
4008            measure_us!(self.get_store_for_shrink(slot, shrink_collect.alive_total_bytes as u64));
4009        stats_sub.create_and_insert_store_elapsed_us = Saturating(time_us);
4010
4011        // here, we're writing back alive_accounts. That should be an atomic operation
4012        // without use of rather wide locks in this whole function, because we're
4013        // mutating rooted slots; There should be no writers to them.
4014        let accounts = [(slot, &shrink_collect.alive_accounts.alive_accounts()[..])];
4015        let storable_accounts = StorableAccountsBySlot::new(slot, &accounts, self);
4016        stats_sub.store_accounts_timing =
4017            self.store_accounts_frozen(storable_accounts, shrink_in_progress.new_storage());
4018
4019        rewrite_elapsed.stop();
4020        stats_sub.rewrite_elapsed_us = Saturating(rewrite_elapsed.as_us());
4021
4022        // `store_accounts_frozen()` above may have purged accounts from some
4023        // other storage entries (the ones that were just overwritten by this
4024        // new storage entry). This means some of those stores might have caused
4025        // this slot to be read to `self.shrink_candidate_slots`, so delete
4026        // those here
4027        self.shrink_candidate_slots.lock().unwrap().remove(&slot);
4028
4029        self.remove_old_stores_shrink(
4030            &shrink_collect,
4031            &self.shrink_stats,
4032            Some(shrink_in_progress),
4033            false,
4034        );
4035
4036        self.reopen_storage_as_readonly_shrinking_in_progress_ok(slot);
4037
4038        Self::update_shrink_stats(&self.shrink_stats, stats_sub, true);
4039        self.shrink_stats.report();
4040    }
4041
4042    pub(crate) fn update_shrink_stats(
4043        shrink_stats: &ShrinkStats,
4044        stats_sub: ShrinkStatsSub,
4045        increment_count: bool,
4046    ) {
4047        if increment_count {
4048            shrink_stats
4049                .num_slots_shrunk
4050                .fetch_add(1, Ordering::Relaxed);
4051        }
4052        shrink_stats.create_and_insert_store_elapsed.fetch_add(
4053            stats_sub.create_and_insert_store_elapsed_us.0,
4054            Ordering::Relaxed,
4055        );
4056        shrink_stats.store_accounts_elapsed.fetch_add(
4057            stats_sub.store_accounts_timing.store_accounts_elapsed,
4058            Ordering::Relaxed,
4059        );
4060        shrink_stats.update_index_elapsed.fetch_add(
4061            stats_sub.store_accounts_timing.update_index_elapsed,
4062            Ordering::Relaxed,
4063        );
4064        shrink_stats.handle_reclaims_elapsed.fetch_add(
4065            stats_sub.store_accounts_timing.handle_reclaims_elapsed,
4066            Ordering::Relaxed,
4067        );
4068        shrink_stats
4069            .rewrite_elapsed
4070            .fetch_add(stats_sub.rewrite_elapsed_us.0, Ordering::Relaxed);
4071        shrink_stats
4072            .unpackable_slots_count
4073            .fetch_add(stats_sub.unpackable_slots_count.0 as u64, Ordering::Relaxed);
4074        shrink_stats.newest_alive_packed_count.fetch_add(
4075            stats_sub.newest_alive_packed_count.0 as u64,
4076            Ordering::Relaxed,
4077        );
4078    }
4079
4080    /// get stores for 'slot'
4081    /// Drop 'shrink_in_progress', which will cause the old store to be removed from the storage map.
4082    /// For 'shrink_in_progress'.'old_storage' which is not retained, insert in 'dead_storages' and optionally 'dirty_stores'
4083    /// This is the end of the life cycle of `shrink_in_progress`.
4084    pub fn mark_dirty_dead_stores(
4085        &self,
4086        slot: Slot,
4087        add_dirty_stores: bool,
4088        shrink_in_progress: Option<ShrinkInProgress>,
4089        shrink_can_be_active: bool,
4090    ) -> Vec<Arc<AccountStorageEntry>> {
4091        let mut dead_storages = Vec::default();
4092
4093        let mut not_retaining_store = |store: &Arc<AccountStorageEntry>| {
4094            if add_dirty_stores {
4095                self.dirty_stores.insert(slot, store.clone());
4096            }
4097            dead_storages.push(store.clone());
4098        };
4099
4100        if let Some(shrink_in_progress) = shrink_in_progress {
4101            // shrink is in progress, so 1 new append vec to keep, 1 old one to throw away
4102            not_retaining_store(shrink_in_progress.old_storage());
4103            // dropping 'shrink_in_progress' removes the old append vec that was being shrunk from db's storage
4104        } else if let Some(store) = self.storage.remove(&slot, shrink_can_be_active) {
4105            // no shrink in progress, so all append vecs in this slot are dead
4106            not_retaining_store(&store);
4107        }
4108
4109        dead_storages
4110    }
4111
4112    /// we are done writing to the storage at `slot`. It can be re-opened as read-only if that would help
4113    /// system performance.
4114    pub(crate) fn reopen_storage_as_readonly_shrinking_in_progress_ok(&self, slot: Slot) {
4115        if let Some(storage) = self
4116            .storage
4117            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
4118        {
4119            if let Some(new_storage) = storage.reopen_as_readonly(self.storage_access) {
4120                // consider here the race condition of tx processing having looked up something in the index,
4121                // which could return (slot, append vec id). We want the lookup for the storage to get a storage
4122                // that works whether the lookup occurs before or after the replace call here.
4123                // So, the two storages have to be exactly equivalent wrt offsets, counts, len, id, etc.
4124                assert_eq!(storage.id(), new_storage.id());
4125                assert_eq!(storage.accounts.len(), new_storage.accounts.len());
4126                self.storage
4127                    .replace_storage_with_equivalent(slot, Arc::new(new_storage));
4128            }
4129        }
4130    }
4131
4132    /// return a store that can contain 'size' bytes
4133    pub fn get_store_for_shrink(&self, slot: Slot, size: u64) -> ShrinkInProgress<'_> {
4134        let shrunken_store = self.create_store(slot, size, "shrink", self.shrink_paths.as_slice());
4135        self.storage.shrinking_in_progress(slot, shrunken_store)
4136    }
4137
4138    // Reads all accounts in given slot's AppendVecs and filter only to alive,
4139    // then create a minimum AppendVec filled with the alive.
4140    fn shrink_slot_forced(&self, slot: Slot) {
4141        debug!("shrink_slot_forced: slot: {}", slot);
4142
4143        if let Some(store) = self
4144            .storage
4145            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
4146        {
4147            if Self::is_shrinking_productive(&store) {
4148                self.shrink_storage(store)
4149            }
4150        }
4151    }
4152
4153    fn all_slots_in_storage(&self) -> Vec<Slot> {
4154        self.storage.all_slots()
4155    }
4156
4157    /// Given the input `ShrinkCandidates`, this function sorts the stores by their alive ratio
4158    /// in increasing order with the most sparse entries in the front. It will then simulate the
4159    /// shrinking by working on the most sparse entries first and if the overall alive ratio is
4160    /// achieved, it will stop and return:
4161    /// first tuple element: the filtered-down candidates and
4162    /// second duple element: the candidates which
4163    /// are skipped in this round and might be eligible for the future shrink.
4164    fn select_candidates_by_total_usage(
4165        &self,
4166        shrink_slots: &ShrinkCandidates,
4167        shrink_ratio: f64,
4168    ) -> (IntMap<Slot, Arc<AccountStorageEntry>>, ShrinkCandidates) {
4169        struct StoreUsageInfo {
4170            slot: Slot,
4171            alive_ratio: f64,
4172            store: Arc<AccountStorageEntry>,
4173        }
4174        let mut store_usage: Vec<StoreUsageInfo> = Vec::with_capacity(shrink_slots.len());
4175        let mut total_alive_bytes: u64 = 0;
4176        let mut total_bytes: u64 = 0;
4177        for slot in shrink_slots {
4178            let Some(store) = self.storage.get_slot_storage_entry(*slot) else {
4179                continue;
4180            };
4181            let alive_bytes = store.alive_bytes();
4182            total_alive_bytes += alive_bytes as u64;
4183            total_bytes += store.capacity();
4184            let alive_ratio = alive_bytes as f64 / store.capacity() as f64;
4185            store_usage.push(StoreUsageInfo {
4186                slot: *slot,
4187                alive_ratio,
4188                store: store.clone(),
4189            });
4190        }
4191        store_usage.sort_by(|a, b| {
4192            a.alive_ratio
4193                .partial_cmp(&b.alive_ratio)
4194                .unwrap_or(std::cmp::Ordering::Equal)
4195        });
4196
4197        // Working from the beginning of store_usage which are the most sparse and see when we can stop
4198        // shrinking while still achieving the overall goals.
4199        let mut shrink_slots = IntMap::default();
4200        let mut shrink_slots_next_batch = ShrinkCandidates::default();
4201        for usage in &store_usage {
4202            let store = &usage.store;
4203            let alive_ratio = (total_alive_bytes as f64) / (total_bytes as f64);
4204            debug!(
4205                "alive_ratio: {:?} store_id: {:?}, store_ratio: {:?} requirement: {:?}, \
4206                 total_bytes: {:?} total_alive_bytes: {:?}",
4207                alive_ratio,
4208                usage.store.id(),
4209                usage.alive_ratio,
4210                shrink_ratio,
4211                total_bytes,
4212                total_alive_bytes
4213            );
4214            if alive_ratio > shrink_ratio {
4215                // we have reached our goal, stop
4216                debug!(
4217                    "Shrinking goal can be achieved at slot {:?}, total_alive_bytes: {:?} \
4218                     total_bytes: {:?}, alive_ratio: {:}, shrink_ratio: {:?}",
4219                    usage.slot, total_alive_bytes, total_bytes, alive_ratio, shrink_ratio
4220                );
4221                if usage.alive_ratio < shrink_ratio {
4222                    shrink_slots_next_batch.insert(usage.slot);
4223                } else {
4224                    break;
4225                }
4226            } else {
4227                let current_store_size = store.capacity();
4228                let after_shrink_size = store.alive_bytes() as u64;
4229                let bytes_saved = current_store_size.saturating_sub(after_shrink_size);
4230                total_bytes -= bytes_saved;
4231                shrink_slots.insert(usage.slot, Arc::clone(store));
4232            }
4233        }
4234        (shrink_slots, shrink_slots_next_batch)
4235    }
4236
4237    fn get_roots_less_than(&self, slot: Slot) -> Vec<Slot> {
4238        self.accounts_index
4239            .roots_tracker
4240            .read()
4241            .unwrap()
4242            .alive_roots
4243            .get_all_less_than(slot)
4244    }
4245
4246    /// return all slots that are more than one epoch old and thus could already be an ancient append vec
4247    /// or which could need to be combined into a new or existing ancient append vec
4248    /// offset is used to combine newer slots than we normally would. This is designed to be used for testing.
4249    fn get_sorted_potential_ancient_slots(&self, oldest_non_ancient_slot: Slot) -> Vec<Slot> {
4250        let mut ancient_slots = self.get_roots_less_than(oldest_non_ancient_slot);
4251        ancient_slots.sort_unstable();
4252        ancient_slots
4253    }
4254
4255    /// get a sorted list of slots older than an epoch
4256    /// squash those slots into ancient append vecs
4257    pub fn shrink_ancient_slots(&self, epoch_schedule: &EpochSchedule) {
4258        if self.ancient_append_vec_offset.is_none() {
4259            return;
4260        }
4261
4262        let oldest_non_ancient_slot = self.get_oldest_non_ancient_slot(epoch_schedule);
4263        let can_randomly_shrink = true;
4264        let sorted_slots = self.get_sorted_potential_ancient_slots(oldest_non_ancient_slot);
4265        if self.create_ancient_storage == CreateAncientStorage::Append {
4266            self.combine_ancient_slots(sorted_slots, can_randomly_shrink);
4267        } else {
4268            self.combine_ancient_slots_packed(sorted_slots, can_randomly_shrink);
4269        }
4270    }
4271
4272    /// 'accounts' that exist in the current slot we are combining into a different ancient slot
4273    /// 'existing_ancient_pubkeys': pubkeys that exist currently in the ancient append vec slot
4274    /// returns the pubkeys that are in 'accounts' that are already in 'existing_ancient_pubkeys'
4275    /// Also updated 'existing_ancient_pubkeys' to include all pubkeys in 'accounts' since they will soon be written into the ancient slot.
4276    fn get_keys_to_unref_ancient<'a>(
4277        accounts: &'a [&AccountFromStorage],
4278        existing_ancient_pubkeys: &mut HashSet<Pubkey>,
4279    ) -> HashSet<&'a Pubkey> {
4280        let mut unref = HashSet::<&Pubkey>::default();
4281        // 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.
4282        // 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.
4283        // The slot that currently references the account is going away, so unref to maintain # slots that reference the pubkey = refcount.
4284        accounts.iter().for_each(|account| {
4285            let key = account.pubkey();
4286            if !existing_ancient_pubkeys.insert(*key) {
4287                // this key exists BOTH in 'accounts' and already in the ancient append vec, so we need to unref it
4288                unref.insert(key);
4289            }
4290        });
4291        unref
4292    }
4293
4294    /// 'accounts' are about to be appended to an ancient append vec. That ancient append vec may already have some accounts.
4295    /// Unref each account in 'accounts' that already exists in 'existing_ancient_pubkeys'.
4296    /// As a side effect, on exit, 'existing_ancient_pubkeys' will now contain all pubkeys in 'accounts'.
4297    fn unref_accounts_already_in_storage(
4298        &self,
4299        accounts: &[&AccountFromStorage],
4300        existing_ancient_pubkeys: &mut HashSet<Pubkey>,
4301    ) {
4302        let unref = Self::get_keys_to_unref_ancient(accounts, existing_ancient_pubkeys);
4303
4304        self.unref_pubkeys(
4305            unref.iter().cloned(),
4306            unref.len(),
4307            &PubkeysRemovedFromAccountsIndex::default(),
4308        );
4309    }
4310
4311    /// get the storage from 'slot' to squash
4312    /// or None if this slot should be skipped
4313    /// side effect could be updating 'current_ancient'
4314    fn get_storage_to_move_to_ancient_accounts_file(
4315        &self,
4316        slot: Slot,
4317        current_ancient: &mut CurrentAncientAccountsFile,
4318        can_randomly_shrink: bool,
4319    ) -> Option<Arc<AccountStorageEntry>> {
4320        self.storage
4321            .get_slot_storage_entry(slot)
4322            .and_then(|storage| {
4323                self.should_move_to_ancient_accounts_file(
4324                    &storage,
4325                    current_ancient,
4326                    slot,
4327                    can_randomly_shrink,
4328                )
4329                .then_some(storage)
4330            })
4331    }
4332
4333    /// return true if the accounts in this slot should be moved to an ancient append vec
4334    /// otherwise, return false and the caller can skip this slot
4335    /// side effect could be updating 'current_ancient'
4336    /// can_randomly_shrink: true if ancient append vecs that otherwise don't qualify to be shrunk can be randomly shrunk
4337    ///  this is convenient for a running system
4338    ///  this is not useful for testing
4339    fn should_move_to_ancient_accounts_file(
4340        &self,
4341        storage: &Arc<AccountStorageEntry>,
4342        current_ancient: &mut CurrentAncientAccountsFile,
4343        slot: Slot,
4344        can_randomly_shrink: bool,
4345    ) -> bool {
4346        let accounts = &storage.accounts;
4347
4348        self.shrink_ancient_stats
4349            .slots_considered
4350            .fetch_add(1, Ordering::Relaxed);
4351
4352        // if an append vec is at least 80% of the ideal capacity of an ancient append vec, that's close enough.
4353        // If we packed, then we end up allocating exact size ancient append vecs. Those will likely never be exactly the ideal ancient capacity.
4354        if accounts.capacity() * 100 / get_ancient_append_vec_capacity() > 80 {
4355            self.shrink_ancient_stats
4356                .ancient_scanned
4357                .fetch_add(1, Ordering::Relaxed);
4358
4359            // randomly shrink ancient slots
4360            // this exercises the ancient shrink code more often
4361            let written_bytes = storage.written_bytes();
4362            let mut alive_ratio = 0;
4363            let is_candidate = if written_bytes > 0 {
4364                alive_ratio = (storage.alive_bytes() as u64) * 100 / written_bytes;
4365                alive_ratio < 90
4366            } else {
4367                false
4368            };
4369            if is_candidate || (can_randomly_shrink && thread_rng().gen_range(0..10000) == 0) {
4370                // we are a candidate for shrink, so either append us to the previous append vec
4371                // or recreate us as a new append vec and eliminate the dead accounts
4372                info!(
4373                    "ancient_append_vec: shrinking full ancient: {}, random: {}, alive_ratio: {}",
4374                    slot, !is_candidate, alive_ratio
4375                );
4376                if !is_candidate {
4377                    self.shrink_ancient_stats
4378                        .random_shrink
4379                        .fetch_add(1, Ordering::Relaxed);
4380                }
4381                self.shrink_ancient_stats
4382                    .ancient_append_vecs_shrunk
4383                    .fetch_add(1, Ordering::Relaxed);
4384                return true;
4385            }
4386            if storage.accounts.can_append() {
4387                // this slot is ancient and can become the 'current' ancient for other slots to be squashed into
4388                *current_ancient = CurrentAncientAccountsFile::new(slot, Arc::clone(storage));
4389            } else {
4390                *current_ancient = CurrentAncientAccountsFile::default();
4391            }
4392            return false; // we're done with this slot - this slot IS the ancient append vec
4393        }
4394
4395        // otherwise, yes, squash this slot into the current ancient append vec or create one at this slot
4396        true
4397    }
4398
4399    /// Combine all account data from storages in 'sorted_slots' into ancient append vecs.
4400    /// This keeps us from accumulating append vecs for each slot older than an epoch.
4401    fn combine_ancient_slots(&self, sorted_slots: Vec<Slot>, can_randomly_shrink: bool) {
4402        if sorted_slots.is_empty() {
4403            return;
4404        }
4405
4406        let mut total = Measure::start("combine_ancient_slots");
4407        let mut guard = None;
4408
4409        // the ancient append vec currently being written to
4410        let mut current_ancient = CurrentAncientAccountsFile::default();
4411        let mut dropped_roots = vec![];
4412
4413        // we have to keep track of what pubkeys exist in the current ancient append vec so we can unref correctly
4414        let mut ancient_slot_pubkeys = AncientSlotPubkeys::default();
4415
4416        let len = sorted_slots.len();
4417        for slot in sorted_slots {
4418            let Some(old_storage) = self.get_storage_to_move_to_ancient_accounts_file(
4419                slot,
4420                &mut current_ancient,
4421                can_randomly_shrink,
4422            ) else {
4423                // nothing to squash for this slot
4424                continue;
4425            };
4426
4427            if guard.is_none() {
4428                // we are now doing interesting work in squashing ancient
4429                guard = Some(self.active_stats.activate(ActiveStatItem::SquashAncient));
4430                info!(
4431                    "ancient_append_vec: combine_ancient_slots first slot: {}, num_roots: {}",
4432                    slot, len
4433                );
4434            }
4435
4436            self.combine_one_store_into_ancient(
4437                slot,
4438                &old_storage,
4439                &mut current_ancient,
4440                &mut ancient_slot_pubkeys,
4441                &mut dropped_roots,
4442            );
4443        }
4444
4445        self.handle_dropped_roots_for_ancient(dropped_roots.into_iter());
4446
4447        total.stop();
4448        self.shrink_ancient_stats
4449            .total_us
4450            .fetch_add(total.as_us(), Ordering::Relaxed);
4451
4452        // only log when we moved some accounts to ancient append vecs or we've exceeded 100ms
4453        // results will continue to accumulate otherwise
4454        if guard.is_some() || self.shrink_ancient_stats.total_us.load(Ordering::Relaxed) > 100_000 {
4455            self.shrink_ancient_stats.report();
4456        }
4457    }
4458
4459    /// put entire alive contents of 'old_storage' into the current ancient append vec or a newly created ancient append vec
4460    fn combine_one_store_into_ancient(
4461        &self,
4462        slot: Slot,
4463        old_storage: &Arc<AccountStorageEntry>,
4464        current_ancient: &mut CurrentAncientAccountsFile,
4465        ancient_slot_pubkeys: &mut AncientSlotPubkeys,
4466        dropped_roots: &mut Vec<Slot>,
4467    ) {
4468        let unique_accounts = self.get_unique_accounts_from_storage_for_shrink(
4469            old_storage,
4470            &self.shrink_ancient_stats.shrink_stats,
4471        );
4472        let shrink_collect = self.shrink_collect::<AliveAccounts<'_>>(
4473            old_storage,
4474            &unique_accounts,
4475            &self.shrink_ancient_stats.shrink_stats,
4476        );
4477
4478        // could follow what shrink does more closely
4479        if shrink_collect.total_starting_accounts == 0 || shrink_collect.alive_total_bytes == 0 {
4480            return; // skipping slot with no useful accounts to write
4481        }
4482
4483        let mut stats_sub = ShrinkStatsSub::default();
4484        let mut bytes_remaining_to_write = shrink_collect.alive_total_bytes;
4485        let (mut shrink_in_progress, create_and_insert_store_elapsed_us) = measure_us!(
4486            current_ancient.create_if_necessary(slot, self, shrink_collect.alive_total_bytes)
4487        );
4488        stats_sub.create_and_insert_store_elapsed_us =
4489            Saturating(create_and_insert_store_elapsed_us);
4490        let available_bytes = current_ancient.accounts_file().accounts.remaining_bytes();
4491        // split accounts in 'slot' into:
4492        // 'Primary', which can fit in 'current_ancient'
4493        // 'Overflow', which will have to go into a new ancient append vec at 'slot'
4494        let to_store = AccountsToStore::new(
4495            available_bytes,
4496            shrink_collect.alive_accounts.alive_accounts(),
4497            shrink_collect.alive_total_bytes,
4498            slot,
4499        );
4500
4501        ancient_slot_pubkeys.maybe_unref_accounts_already_in_ancient(
4502            slot,
4503            self,
4504            current_ancient,
4505            &to_store,
4506        );
4507
4508        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
4509        // write what we can to the current ancient storage
4510        let (store_accounts_timing, bytes_written) =
4511            current_ancient.store_ancient_accounts(self, &to_store, StorageSelector::Primary);
4512        stats_sub.store_accounts_timing = store_accounts_timing;
4513        bytes_remaining_to_write = bytes_remaining_to_write.saturating_sub(bytes_written as usize);
4514
4515        // handle accounts from 'slot' which did not fit into the current ancient append vec
4516        if to_store.has_overflow() {
4517            // We need a new ancient append vec at this slot.
4518            // Assert: it cannot be the case that we already had an ancient append vec at this slot and
4519            // yet that ancient append vec does not have room for the accounts stored at this slot currently
4520            assert_ne!(slot, current_ancient.slot());
4521
4522            // we filled one up
4523            self.reopen_storage_as_readonly_shrinking_in_progress_ok(current_ancient.slot());
4524
4525            // Now we create an ancient append vec at `slot` to store the overflows.
4526            let (shrink_in_progress_overflow, time_us) = measure_us!(current_ancient
4527                .create_ancient_accounts_file(
4528                    slot,
4529                    self,
4530                    to_store.get_bytes(StorageSelector::Overflow)
4531                ));
4532            stats_sub.create_and_insert_store_elapsed_us += time_us;
4533            // We cannot possibly be shrinking the original slot that created an ancient append vec
4534            // AND not have enough room in the ancient append vec at that slot
4535            // to hold all the contents of that slot.
4536            // We need this new 'shrink_in_progress' to be used in 'remove_old_stores_shrink' below.
4537            // All non-overflow accounts were put in a prior slot's ancient append vec. All overflow accounts
4538            // are essentially being shrunk into a new ancient append vec in 'slot'.
4539            assert!(shrink_in_progress.is_none());
4540            shrink_in_progress = Some(shrink_in_progress_overflow);
4541
4542            // write the overflow accounts to the next ancient storage
4543            let (store_accounts_timing, bytes_written) =
4544                current_ancient.store_ancient_accounts(self, &to_store, StorageSelector::Overflow);
4545            bytes_remaining_to_write =
4546                bytes_remaining_to_write.saturating_sub(bytes_written as usize);
4547
4548            stats_sub
4549                .store_accounts_timing
4550                .accumulate(&store_accounts_timing);
4551        }
4552        assert_eq!(bytes_remaining_to_write, 0);
4553        rewrite_elapsed.stop();
4554        stats_sub.rewrite_elapsed_us = Saturating(rewrite_elapsed.as_us());
4555
4556        if slot != current_ancient.slot() {
4557            // all append vecs in this slot have been combined into an ancient append vec
4558            dropped_roots.push(slot);
4559        }
4560
4561        self.remove_old_stores_shrink(
4562            &shrink_collect,
4563            &self.shrink_ancient_stats.shrink_stats,
4564            shrink_in_progress,
4565            false,
4566        );
4567
4568        // 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.
4569        self.shrink_candidate_slots.lock().unwrap().remove(&slot);
4570
4571        Self::update_shrink_stats(&self.shrink_ancient_stats.shrink_stats, stats_sub, true);
4572    }
4573
4574    /// each slot in 'dropped_roots' has been combined into an ancient append vec.
4575    /// We are done with the slot now forever.
4576    pub(crate) fn handle_dropped_roots_for_ancient(
4577        &self,
4578        dropped_roots: impl Iterator<Item = Slot>,
4579    ) {
4580        let mut accounts_delta_hashes = self.accounts_delta_hashes.lock().unwrap();
4581
4582        dropped_roots.for_each(|slot| {
4583            self.accounts_index.clean_dead_slot(slot);
4584            accounts_delta_hashes.remove(&slot);
4585            // the storage has been removed from this slot and recycled or dropped
4586            assert!(self.storage.remove(&slot, false).is_none());
4587            debug_assert!(
4588                !self
4589                    .accounts_index
4590                    .roots_tracker
4591                    .read()
4592                    .unwrap()
4593                    .alive_roots
4594                    .contains(&slot),
4595                "slot: {slot}"
4596            );
4597        });
4598    }
4599
4600    /// add all 'pubkeys' into the set of pubkeys that are 'uncleaned', associated with 'slot'
4601    /// clean will visit these pubkeys next time it runs
4602    fn add_uncleaned_pubkeys_after_shrink(
4603        &self,
4604        slot: Slot,
4605        pubkeys: impl Iterator<Item = Pubkey>,
4606    ) {
4607        /*
4608        This is only called during 'shrink'-type operations.
4609        Original accounts were separated into 'accounts' and 'pubkeys_to_unref'.
4610        These sets correspond to 'alive' and 'dead'.
4611        'alive' means this account in this slot is in the accounts index.
4612        'dead' means this account in this slot is NOT in the accounts index.
4613        If dead, nobody will care if this version of this account is not written into the newly shrunk append vec for this slot.
4614        For all dead accounts, they were already unrefed and are now absent in the new append vec.
4615        This means that another version of this pubkey could possibly now be cleaned since this one is now gone.
4616        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.
4617        So, for all unrefed accounts, send them to clean to be revisited next time clean runs.
4618        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.
4619        Clean doesn't care about alive accounts that remain alive.
4620        Except... A slightly different case is if ALL the alive accounts in this slot are zero lamport accounts, then it is possible that
4621        this slot can be marked dead. So, if all alive accounts are zero lamports, we send the entire OLD/pre-shrunk append vec
4622        to clean so that all the pubkeys are visited.
4623        It is a performance optimization to not send the ENTIRE old/pre-shrunk append vec to clean in the normal case.
4624        */
4625
4626        let mut uncleaned_pubkeys = self.uncleaned_pubkeys.entry(slot).or_default();
4627        uncleaned_pubkeys.extend(pubkeys);
4628    }
4629
4630    pub fn shrink_candidate_slots(&self, epoch_schedule: &EpochSchedule) -> usize {
4631        let oldest_non_ancient_slot = self.get_oldest_non_ancient_slot(epoch_schedule);
4632
4633        let shrink_candidates_slots =
4634            std::mem::take(&mut *self.shrink_candidate_slots.lock().unwrap());
4635        self.shrink_stats
4636            .initial_candidates_count
4637            .store(shrink_candidates_slots.len() as u64, Ordering::Relaxed);
4638
4639        let candidates_count = shrink_candidates_slots.len();
4640        let ((mut shrink_slots, shrink_slots_next_batch), select_time_us) = measure_us!({
4641            if let AccountShrinkThreshold::TotalSpace { shrink_ratio } = self.shrink_ratio {
4642                let (shrink_slots, shrink_slots_next_batch) =
4643                    self.select_candidates_by_total_usage(&shrink_candidates_slots, shrink_ratio);
4644                (shrink_slots, Some(shrink_slots_next_batch))
4645            } else {
4646                (
4647                    // lookup storage for each slot
4648                    shrink_candidates_slots
4649                        .into_iter()
4650                        .filter_map(|slot| {
4651                            self.storage
4652                                .get_slot_storage_entry(slot)
4653                                .map(|storage| (slot, storage))
4654                        })
4655                        .collect(),
4656                    None,
4657                )
4658            }
4659        });
4660
4661        // If there are too few slots to shrink, add an ancient slot
4662        // for shrinking.
4663        if shrink_slots.len() < SHRINK_INSERT_ANCIENT_THRESHOLD {
4664            let mut ancients = self.best_ancient_slots_to_shrink.write().unwrap();
4665            while let Some((slot, capacity)) = ancients.pop_front() {
4666                if let Some(store) = self.storage.get_slot_storage_entry(slot) {
4667                    if !shrink_slots.contains(&slot)
4668                        && capacity == store.capacity()
4669                        && Self::is_candidate_for_shrink(self, &store)
4670                    {
4671                        let ancient_bytes_added_to_shrink = store.alive_bytes() as u64;
4672                        shrink_slots.insert(slot, store);
4673                        self.shrink_stats
4674                            .ancient_bytes_added_to_shrink
4675                            .fetch_add(ancient_bytes_added_to_shrink, Ordering::Relaxed);
4676                        self.shrink_stats
4677                            .ancient_slots_added_to_shrink
4678                            .fetch_add(1, Ordering::Relaxed);
4679                        break;
4680                    }
4681                }
4682            }
4683        }
4684        if shrink_slots.is_empty()
4685            && shrink_slots_next_batch
4686                .as_ref()
4687                .map(|s| s.is_empty())
4688                .unwrap_or(true)
4689        {
4690            return 0;
4691        }
4692
4693        let _guard = (!shrink_slots.is_empty())
4694            .then_some(|| self.active_stats.activate(ActiveStatItem::Shrink));
4695
4696        let num_selected = shrink_slots.len();
4697        let (_, shrink_all_us) = measure_us!({
4698            self.thread_pool_clean.install(|| {
4699                shrink_slots
4700                    .into_par_iter()
4701                    .for_each(|(slot, slot_shrink_candidate)| {
4702                        if self.ancient_append_vec_offset.is_some()
4703                            && slot < oldest_non_ancient_slot
4704                        {
4705                            self.shrink_stats
4706                                .num_ancient_slots_shrunk
4707                                .fetch_add(1, Ordering::Relaxed);
4708                        }
4709                        self.shrink_storage(slot_shrink_candidate);
4710                    });
4711            })
4712        });
4713
4714        let mut pended_counts: usize = 0;
4715        if let Some(shrink_slots_next_batch) = shrink_slots_next_batch {
4716            let mut shrink_slots = self.shrink_candidate_slots.lock().unwrap();
4717            pended_counts = shrink_slots_next_batch.len();
4718            for slot in shrink_slots_next_batch {
4719                shrink_slots.insert(slot);
4720            }
4721        }
4722
4723        datapoint_info!(
4724            "shrink_candidate_slots",
4725            ("select_time_us", select_time_us, i64),
4726            ("shrink_all_us", shrink_all_us, i64),
4727            ("candidates_count", candidates_count, i64),
4728            ("selected_count", num_selected, i64),
4729            ("deferred_to_next_round_count", pended_counts, i64)
4730        );
4731
4732        num_selected
4733    }
4734
4735    /// This is only called at startup from bank when we are being extra careful such as when we downloaded a snapshot.
4736    /// Also called from tests.
4737    /// `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
4738    /// the bank hash calculation verification at startup.
4739    pub fn shrink_all_slots(
4740        &self,
4741        is_startup: bool,
4742        epoch_schedule: &EpochSchedule,
4743        newest_slot_skip_shrink_inclusive: Option<Slot>,
4744    ) {
4745        let _guard = self.active_stats.activate(ActiveStatItem::Shrink);
4746        const DIRTY_STORES_CLEANING_THRESHOLD: usize = 10_000;
4747        const OUTER_CHUNK_SIZE: usize = 2000;
4748        let mut slots = self.all_slots_in_storage();
4749        if let Some(newest_slot_skip_shrink_inclusive) = newest_slot_skip_shrink_inclusive {
4750            // at startup, we cannot shrink the slot that we're about to replay and recalculate bank hash for.
4751            // That storage's contents are used to verify the bank hash (and accounts delta hash) of the startup slot.
4752            slots.retain(|slot| slot < &newest_slot_skip_shrink_inclusive);
4753        }
4754
4755        // if we are restoring from incremental + full snapshot, then we cannot clean past latest_full_snapshot_slot.
4756        // If we were to clean past that, then we could mark accounts prior to latest_full_snapshot_slot as dead.
4757        // If we mark accounts prior to latest_full_snapshot_slot as dead, then we could shrink those accounts away.
4758        // If we shrink accounts away, then when we run the full hash of all accounts calculation up to latest_full_snapshot_slot,
4759        // then we will get the wrong answer, because some accounts may be GONE from the slot range up to latest_full_snapshot_slot.
4760        // So, we can only clean UP TO and including latest_full_snapshot_slot.
4761        // As long as we don't mark anything as dead at slots > latest_full_snapshot_slot, then shrink will have nothing to do for
4762        // slots > latest_full_snapshot_slot.
4763        let maybe_clean = || {
4764            if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
4765                let latest_full_snapshot_slot = self.latest_full_snapshot_slot();
4766                self.clean_accounts(
4767                    latest_full_snapshot_slot,
4768                    is_startup,
4769                    epoch_schedule,
4770                    // Leave any old storages alone for now.  Once the validator is running
4771                    // normal, calls to clean_accounts() will have the correct policy based
4772                    // on if ancient storages are enabled or not.
4773                    OldStoragesPolicy::Leave,
4774                );
4775            }
4776        };
4777
4778        if is_startup {
4779            let threads = num_cpus::get();
4780            let inner_chunk_size = std::cmp::max(OUTER_CHUNK_SIZE / threads, 1);
4781            slots.chunks(OUTER_CHUNK_SIZE).for_each(|chunk| {
4782                chunk.par_chunks(inner_chunk_size).for_each(|slots| {
4783                    for slot in slots {
4784                        self.shrink_slot_forced(*slot);
4785                    }
4786                });
4787                maybe_clean();
4788            });
4789        } else {
4790            for slot in slots {
4791                self.shrink_slot_forced(slot);
4792                maybe_clean();
4793            }
4794        }
4795    }
4796
4797    pub fn scan_accounts<F>(
4798        &self,
4799        ancestors: &Ancestors,
4800        bank_id: BankId,
4801        mut scan_func: F,
4802        config: &ScanConfig,
4803    ) -> ScanResult<()>
4804    where
4805        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4806    {
4807        // This can error out if the slots being scanned over are aborted
4808        self.accounts_index.scan_accounts(
4809            ancestors,
4810            bank_id,
4811            |pubkey, (account_info, slot)| {
4812                let account_slot = self
4813                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4814                    .get_loaded_account(|loaded_account| {
4815                        (pubkey, loaded_account.take_account(), slot)
4816                    });
4817                scan_func(account_slot)
4818            },
4819            config,
4820        )?;
4821
4822        Ok(())
4823    }
4824
4825    pub fn unchecked_scan_accounts<F>(
4826        &self,
4827        metric_name: &'static str,
4828        ancestors: &Ancestors,
4829        mut scan_func: F,
4830        config: &ScanConfig,
4831    ) where
4832        F: FnMut(&Pubkey, LoadedAccount, Slot),
4833    {
4834        self.accounts_index.unchecked_scan_accounts(
4835            metric_name,
4836            ancestors,
4837            |pubkey, (account_info, slot)| {
4838                self.get_account_accessor(slot, pubkey, &account_info.storage_location())
4839                    .get_loaded_account(|loaded_account| {
4840                        scan_func(pubkey, loaded_account, slot);
4841                    });
4842            },
4843            config,
4844        );
4845    }
4846
4847    /// Only guaranteed to be safe when called from rent collection
4848    pub fn range_scan_accounts<F, R>(
4849        &self,
4850        metric_name: &'static str,
4851        ancestors: &Ancestors,
4852        range: R,
4853        config: &ScanConfig,
4854        mut scan_func: F,
4855    ) where
4856        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4857        R: RangeBounds<Pubkey> + std::fmt::Debug,
4858    {
4859        self.accounts_index.range_scan_accounts(
4860            metric_name,
4861            ancestors,
4862            range,
4863            config,
4864            |pubkey, (account_info, slot)| {
4865                // unlike other scan fns, this is called from Bank::collect_rent_eagerly(),
4866                // which is on-consensus processing in the banking/replaying stage.
4867                // This requires infallible and consistent account loading.
4868                // So, we unwrap Option<LoadedAccount> from get_loaded_account() here.
4869                // This is safe because this closure is invoked with the account_info,
4870                // while we lock the index entry at AccountsIndex::do_scan_accounts() ultimately,
4871                // meaning no other subsystems can invalidate the account_info before making their
4872                // changes to the index entry.
4873                // For details, see the comment in retry_to_get_account_accessor()
4874                if let Some(account_slot) = self
4875                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4876                    .get_loaded_account(|loaded_account| {
4877                        (pubkey, loaded_account.take_account(), slot)
4878                    })
4879                {
4880                    scan_func(Some(account_slot))
4881                }
4882            },
4883        );
4884    }
4885
4886    pub fn index_scan_accounts<F>(
4887        &self,
4888        ancestors: &Ancestors,
4889        bank_id: BankId,
4890        index_key: IndexKey,
4891        mut scan_func: F,
4892        config: &ScanConfig,
4893    ) -> ScanResult<bool>
4894    where
4895        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4896    {
4897        let key = match &index_key {
4898            IndexKey::ProgramId(key) => key,
4899            IndexKey::SplTokenMint(key) => key,
4900            IndexKey::SplTokenOwner(key) => key,
4901        };
4902        if !self.account_indexes.include_key(key) {
4903            // the requested key was not indexed in the secondary index, so do a normal scan
4904            let used_index = false;
4905            self.scan_accounts(ancestors, bank_id, scan_func, config)?;
4906            return Ok(used_index);
4907        }
4908
4909        self.accounts_index.index_scan_accounts(
4910            ancestors,
4911            bank_id,
4912            index_key,
4913            |pubkey, (account_info, slot)| {
4914                let account_slot = self
4915                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4916                    .get_loaded_account(|loaded_account| {
4917                        (pubkey, loaded_account.take_account(), slot)
4918                    });
4919                scan_func(account_slot)
4920            },
4921            config,
4922        )?;
4923        let used_index = true;
4924        Ok(used_index)
4925    }
4926
4927    /// Scan a specific slot through all the account storage
4928    pub(crate) fn scan_account_storage<R, B>(
4929        &self,
4930        slot: Slot,
4931        cache_map_func: impl Fn(&LoadedAccount) -> Option<R> + Sync,
4932        storage_scan_func: impl Fn(&B, &LoadedAccount, Option<&[u8]>) + Sync,
4933        scan_account_storage_data: ScanAccountStorageData,
4934    ) -> ScanStorageResult<R, B>
4935    where
4936        R: Send,
4937        B: Send + Default + Sync,
4938    {
4939        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
4940            // If we see the slot in the cache, then all the account information
4941            // is in this cached slot
4942            if slot_cache.len() > SCAN_SLOT_PAR_ITER_THRESHOLD {
4943                ScanStorageResult::Cached(self.thread_pool.install(|| {
4944                    slot_cache
4945                        .par_iter()
4946                        .filter_map(|cached_account| {
4947                            cache_map_func(&LoadedAccount::Cached(Cow::Borrowed(
4948                                cached_account.value(),
4949                            )))
4950                        })
4951                        .collect()
4952                }))
4953            } else {
4954                ScanStorageResult::Cached(
4955                    slot_cache
4956                        .iter()
4957                        .filter_map(|cached_account| {
4958                            cache_map_func(&LoadedAccount::Cached(Cow::Borrowed(
4959                                cached_account.value(),
4960                            )))
4961                        })
4962                        .collect(),
4963                )
4964            }
4965        } else {
4966            let retval = B::default();
4967            // If the slot is not in the cache, then all the account information must have
4968            // been flushed. This is guaranteed because we only remove the rooted slot from
4969            // the cache *after* we've finished flushing in `flush_slot_cache`.
4970            // Regarding `shrinking_in_progress_ok`:
4971            // This fn could be running in the foreground, so shrinking could be running in the background, independently.
4972            // Even if shrinking is running, there will be 0-1 active storages to scan here at any point.
4973            // When a concurrent shrink completes, the active storage at this slot will
4974            // be replaced with an equivalent storage with only alive accounts in it.
4975            // A shrink on this slot could have completed anytime before the call here, a shrink could currently be in progress,
4976            // or the shrink could complete immediately or anytime after this call. This has always been true.
4977            // So, whether we get a never-shrunk, an about-to-be shrunk, or a will-be-shrunk-in-future storage here to scan,
4978            // all are correct and possible in a normally running system.
4979            if let Some(storage) = self
4980                .storage
4981                .get_slot_storage_entry_shrinking_in_progress_ok(slot)
4982            {
4983                storage.accounts.scan_accounts(|account| {
4984                    let loaded_account = LoadedAccount::Stored(account);
4985                    let data = (scan_account_storage_data
4986                        == ScanAccountStorageData::DataRefForStorage)
4987                        .then_some(loaded_account.data());
4988                    storage_scan_func(&retval, &loaded_account, data)
4989                });
4990            }
4991
4992            ScanStorageResult::Stored(retval)
4993        }
4994    }
4995
4996    pub fn load(
4997        &self,
4998        ancestors: &Ancestors,
4999        pubkey: &Pubkey,
5000        load_hint: LoadHint,
5001    ) -> Option<(AccountSharedData, Slot)> {
5002        self.do_load(ancestors, pubkey, None, load_hint, LoadZeroLamports::None)
5003    }
5004
5005    /// Return Ok(index_of_matching_owner) if the account owner at `offset` is one of the pubkeys in `owners`.
5006    /// Return Err(MatchAccountOwnerError::NoMatch) if the account has 0 lamports or the owner is not one of
5007    /// the pubkeys in `owners`.
5008    /// Return Err(MatchAccountOwnerError::UnableToLoad) if the account could not be accessed.
5009    pub fn account_matches_owners(
5010        &self,
5011        ancestors: &Ancestors,
5012        account: &Pubkey,
5013        owners: &[Pubkey],
5014    ) -> Result<usize, MatchAccountOwnerError> {
5015        let (slot, storage_location, _maybe_account_accesor) = self
5016            .read_index_for_accessor_or_load_slow(ancestors, account, None, false)
5017            .ok_or(MatchAccountOwnerError::UnableToLoad)?;
5018
5019        if !storage_location.is_cached() {
5020            let result = self.read_only_accounts_cache.load(*account, slot);
5021            if let Some(account) = result {
5022                return if account.is_zero_lamport() {
5023                    Err(MatchAccountOwnerError::NoMatch)
5024                } else {
5025                    owners
5026                        .iter()
5027                        .position(|entry| account.owner() == entry)
5028                        .ok_or(MatchAccountOwnerError::NoMatch)
5029                };
5030            }
5031        }
5032
5033        let (account_accessor, _slot) = self
5034            .retry_to_get_account_accessor(
5035                slot,
5036                storage_location,
5037                ancestors,
5038                account,
5039                None,
5040                LoadHint::Unspecified,
5041            )
5042            .ok_or(MatchAccountOwnerError::UnableToLoad)?;
5043        account_accessor.account_matches_owners(owners)
5044    }
5045
5046    /// load the account with `pubkey` into the read only accounts cache.
5047    /// The goal is to make subsequent loads (which caller expects to occur) to find the account quickly.
5048    pub fn load_account_into_read_cache(&self, ancestors: &Ancestors, pubkey: &Pubkey) {
5049        self.do_load_with_populate_read_cache(
5050            ancestors,
5051            pubkey,
5052            None,
5053            LoadHint::Unspecified,
5054            true,
5055            // no return from this function, so irrelevant
5056            LoadZeroLamports::None,
5057        );
5058    }
5059
5060    /// note this returns None for accounts with zero lamports
5061    pub fn load_with_fixed_root(
5062        &self,
5063        ancestors: &Ancestors,
5064        pubkey: &Pubkey,
5065    ) -> Option<(AccountSharedData, Slot)> {
5066        self.load(ancestors, pubkey, LoadHint::FixedMaxRoot)
5067    }
5068
5069    fn read_index_for_accessor_or_load_slow<'a>(
5070        &'a self,
5071        ancestors: &Ancestors,
5072        pubkey: &'a Pubkey,
5073        max_root: Option<Slot>,
5074        clone_in_lock: bool,
5075    ) -> Option<(Slot, StorageLocation, Option<LoadedAccountAccessor<'a>>)> {
5076        self.accounts_index.get_with_and_then(
5077            pubkey,
5078            Some(ancestors),
5079            max_root,
5080            true,
5081            |(slot, account_info)| {
5082                let storage_location = account_info.storage_location();
5083                let account_accessor = clone_in_lock
5084                    .then(|| self.get_account_accessor(slot, pubkey, &storage_location));
5085                (slot, storage_location, account_accessor)
5086            },
5087        )
5088    }
5089
5090    fn retry_to_get_account_accessor<'a>(
5091        &'a self,
5092        mut slot: Slot,
5093        mut storage_location: StorageLocation,
5094        ancestors: &'a Ancestors,
5095        pubkey: &'a Pubkey,
5096        max_root: Option<Slot>,
5097        load_hint: LoadHint,
5098    ) -> Option<(LoadedAccountAccessor<'a>, Slot)> {
5099        // Happy drawing time! :)
5100        //
5101        // Reader                               | Accessed data source for cached/stored
5102        // -------------------------------------+----------------------------------
5103        // R1 read_index_for_accessor_or_load_slow()| cached/stored: index
5104        //          |                           |
5105        //        <(store_id, offset, ..)>      |
5106        //          V                           |
5107        // R2 retry_to_get_account_accessor()/  | cached: map of caches & entry for (slot, pubkey)
5108        //        get_account_accessor()        | stored: map of stores
5109        //          |                           |
5110        //        <Accessor>                    |
5111        //          V                           |
5112        // R3 check_and_get_loaded_account()/   | cached: N/A (note: basically noop unwrap)
5113        //        get_loaded_account()          | stored: store's entry for slot
5114        //          |                           |
5115        //        <LoadedAccount>               |
5116        //          V                           |
5117        // R4 take_account()                    | cached/stored: entry of cache/storage for (slot, pubkey)
5118        //          |                           |
5119        //        <AccountSharedData>           |
5120        //          V                           |
5121        //    Account!!                         V
5122        //
5123        // Flusher                              | Accessed data source for cached/stored
5124        // -------------------------------------+----------------------------------
5125        // F1 flush_slot_cache()                | N/A
5126        //          |                           |
5127        //          V                           |
5128        // F2 store_accounts_frozen()/          | map of stores (creates new entry)
5129        //        write_accounts_to_storage()   |
5130        //          |                           |
5131        //          V                           |
5132        // F3 store_accounts_frozen()/          | index
5133        //        update_index()                | (replaces existing store_id, offset in caches)
5134        //          |                           |
5135        //          V                           |
5136        // F4 accounts_cache.remove_slot()      | map of caches (removes old entry)
5137        //                                      V
5138        //
5139        // Remarks for flusher: So, for any reading operations, it's a race condition where F4 happens
5140        // between R1 and R2. In that case, retrying from R1 is safu because F3 should have
5141        // been occurred.
5142        //
5143        // Shrinker                             | Accessed data source for stored
5144        // -------------------------------------+----------------------------------
5145        // S1 do_shrink_slot_store()            | N/A
5146        //          |                           |
5147        //          V                           |
5148        // S2 store_accounts_frozen()/          | map of stores (creates new entry)
5149        //        write_accounts_to_storage()   |
5150        //          |                           |
5151        //          V                           |
5152        // S3 store_accounts_frozen()/          | index
5153        //        update_index()                | (replaces existing store_id, offset in stores)
5154        //          |                           |
5155        //          V                           |
5156        // S4 do_shrink_slot_store()/           | map of stores (removes old entry)
5157        //        dead_storages
5158        //
5159        // Remarks for shrinker: So, for any reading operations, it's a race condition
5160        // where S4 happens between R1 and R2. In that case, retrying from R1 is safu because S3 should have
5161        // been occurred, and S3 atomically replaced the index accordingly.
5162        //
5163        // Cleaner                              | Accessed data source for stored
5164        // -------------------------------------+----------------------------------
5165        // C1 clean_accounts()                  | N/A
5166        //          |                           |
5167        //          V                           |
5168        // C2 clean_accounts()/                 | index
5169        //        purge_keys_exact()            | (removes existing store_id, offset for stores)
5170        //          |                           |
5171        //          V                           |
5172        // C3 clean_accounts()/                 | map of stores (removes old entry)
5173        //        handle_reclaims()             |
5174        //
5175        // Remarks for cleaner: So, for any reading operations, it's a race condition
5176        // where C3 happens between R1 and R2. In that case, retrying from R1 is safu.
5177        // In that case, None would be returned while bailing out at R1.
5178        //
5179        // Purger                                 | Accessed data source for cached/stored
5180        // ---------------------------------------+----------------------------------
5181        // P1 purge_slot()                        | N/A
5182        //          |                             |
5183        //          V                             |
5184        // P2 purge_slots_from_cache_and_store()  | map of caches/stores (removes old entry)
5185        //          |                             |
5186        //          V                             |
5187        // P3 purge_slots_from_cache_and_store()/ | index
5188        //       purge_slot_cache()/              |
5189        //          purge_slot_cache_pubkeys()    | (removes existing store_id, offset for cache)
5190        //       purge_slot_storage()/            |
5191        //          purge_keys_exact()            | (removes accounts index entries)
5192        //          handle_reclaims()             | (removes storage entries)
5193        //      OR                                |
5194        //    clean_accounts()/                   |
5195        //        clean_accounts_older_than_root()| (removes existing store_id, offset for stores)
5196        //                                        V
5197        //
5198        // Remarks for purger: So, for any reading operations, it's a race condition
5199        // where P2 happens between R1 and R2. In that case, retrying from R1 is safu.
5200        // In that case, we may bail at index read retry when P3 hasn't been run
5201
5202        #[cfg(test)]
5203        {
5204            // Give some time for cache flushing to occur here for unit tests
5205            sleep(Duration::from_millis(self.load_delay));
5206        }
5207
5208        // Failsafe for potential race conditions with other subsystems
5209        let mut num_acceptable_failed_iterations = 0;
5210        loop {
5211            let account_accessor = self.get_account_accessor(slot, pubkey, &storage_location);
5212            match account_accessor {
5213                LoadedAccountAccessor::Cached(Some(_)) | LoadedAccountAccessor::Stored(Some(_)) => {
5214                    // Great! There was no race, just return :) This is the most usual situation
5215                    return Some((account_accessor, slot));
5216                }
5217                LoadedAccountAccessor::Cached(None) => {
5218                    num_acceptable_failed_iterations += 1;
5219                    // Cache was flushed in between checking the index and retrieving from the cache,
5220                    // so retry. This works because in accounts cache flush, an account is written to
5221                    // storage *before* it is removed from the cache
5222                    match load_hint {
5223                        LoadHint::FixedMaxRootDoNotPopulateReadCache | LoadHint::FixedMaxRoot => {
5224                            // it's impossible for this to fail for transaction loads from
5225                            // replaying/banking more than once.
5226                            // This is because:
5227                            // 1) For a slot `X` that's being replayed, there is only one
5228                            // latest ancestor containing the latest update for the account, and this
5229                            // ancestor can only be flushed once.
5230                            // 2) The root cannot move while replaying, so the index cannot continually
5231                            // find more up to date entries than the current `slot`
5232                            assert!(num_acceptable_failed_iterations <= 1);
5233                        }
5234                        LoadHint::Unspecified => {
5235                            // Because newer root can be added to the index (= not fixed),
5236                            // multiple flush race conditions can be observed under very rare
5237                            // condition, at least theoretically
5238                        }
5239                    }
5240                }
5241                LoadedAccountAccessor::Stored(None) => {
5242                    match load_hint {
5243                        LoadHint::FixedMaxRootDoNotPopulateReadCache | LoadHint::FixedMaxRoot => {
5244                            // When running replay on the validator, or banking stage on the leader,
5245                            // it should be very rare that the storage entry doesn't exist if the
5246                            // entry in the accounts index is the latest version of this account.
5247                            //
5248                            // There are only a few places where the storage entry may not exist
5249                            // after reading the index:
5250                            // 1) Shrink has removed the old storage entry and rewritten to
5251                            // a newer storage entry
5252                            // 2) The `pubkey` asked for in this function is a zero-lamport account,
5253                            // and the storage entry holding this account qualified for zero-lamport clean.
5254                            //
5255                            // In both these cases, it should be safe to retry and recheck the accounts
5256                            // index indefinitely, without incrementing num_acceptable_failed_iterations.
5257                            // That's because if the root is fixed, there should be a bounded number
5258                            // of pending cleans/shrinks (depends how far behind the AccountsBackgroundService
5259                            // is), termination to the desired condition is guaranteed.
5260                            //
5261                            // Also note that in both cases, if we do find the storage entry,
5262                            // we can guarantee that the storage entry is safe to read from because
5263                            // we grabbed a reference to the storage entry while it was still in the
5264                            // storage map. This means even if the storage entry is removed from the storage
5265                            // map after we grabbed the storage entry, the recycler should not reset the
5266                            // storage entry until we drop the reference to the storage entry.
5267                            //
5268                            // eh, no code in this arm? yes!
5269                        }
5270                        LoadHint::Unspecified => {
5271                            // RPC get_account() may have fetched an old root from the index that was
5272                            // either:
5273                            // 1) Cleaned up by clean_accounts(), so the accounts index has been updated
5274                            // and the storage entries have been removed.
5275                            // 2) Dropped by purge_slots() because the slot was on a minor fork, which
5276                            // removes the slots' storage entries but doesn't purge from the accounts index
5277                            // (account index cleanup is left to clean for stored slots). Note that
5278                            // this generally is impossible to occur in the wild because the RPC
5279                            // should hold the slot's bank, preventing it from being purged() to
5280                            // begin with.
5281                            num_acceptable_failed_iterations += 1;
5282                        }
5283                    }
5284                }
5285            }
5286            #[cfg(not(test))]
5287            let load_limit = ABSURD_CONSECUTIVE_FAILED_ITERATIONS;
5288
5289            #[cfg(test)]
5290            let load_limit = self.load_limit.load(Ordering::Relaxed);
5291
5292            let fallback_to_slow_path = if num_acceptable_failed_iterations >= load_limit {
5293                // The latest version of the account existed in the index, but could not be
5294                // fetched from storage. This means a race occurred between this function and clean
5295                // accounts/purge_slots
5296                let message = format!(
5297                    "do_load() failed to get key: {pubkey} from storage, latest attempt was for \
5298                     slot: {slot}, storage_location: {storage_location:?}, load_hint: \
5299                     {load_hint:?}",
5300                );
5301                datapoint_warn!("accounts_db-do_load_warn", ("warn", message, String));
5302                true
5303            } else {
5304                false
5305            };
5306
5307            // Because reading from the cache/storage failed, retry from the index read
5308            let (new_slot, new_storage_location, maybe_account_accessor) = self
5309                .read_index_for_accessor_or_load_slow(
5310                    ancestors,
5311                    pubkey,
5312                    max_root,
5313                    fallback_to_slow_path,
5314                )?;
5315            // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5316
5317            if new_slot == slot && new_storage_location.is_store_id_equal(&storage_location) {
5318                let message = format!(
5319                    "Bad index entry detected ({}, {}, {:?}, {:?}, {:?}, {:?})",
5320                    pubkey,
5321                    slot,
5322                    storage_location,
5323                    load_hint,
5324                    new_storage_location,
5325                    self.accounts_index.get_cloned(pubkey)
5326                );
5327                // Considering that we've failed to get accessor above and further that
5328                // the index still returned the same (slot, store_id) tuple, offset must be same
5329                // too.
5330                assert!(
5331                    new_storage_location.is_offset_equal(&storage_location),
5332                    "{message}"
5333                );
5334
5335                // If the entry was missing from the cache, that means it must have been flushed,
5336                // and the accounts index is always updated before cache flush, so store_id must
5337                // not indicate being cached at this point.
5338                assert!(!new_storage_location.is_cached(), "{message}");
5339
5340                // If this is not a cache entry, then this was a minor fork slot
5341                // that had its storage entries cleaned up by purge_slots() but hasn't been
5342                // cleaned yet. That means this must be rpc access and not replay/banking at the
5343                // very least. Note that purge shouldn't occur even for RPC as caller must hold all
5344                // of ancestor slots..
5345                assert_eq!(load_hint, LoadHint::Unspecified, "{message}");
5346
5347                // Everything being assert!()-ed, let's panic!() here as it's an error condition
5348                // after all....
5349                // That reasoning is based on the fact all of code-path reaching this fn
5350                // retry_to_get_account_accessor() must outlive the Arc<Bank> (and its all
5351                // ancestors) over this fn invocation, guaranteeing the prevention of being purged,
5352                // first of all.
5353                // For details, see the comment in AccountIndex::do_checked_scan_accounts(),
5354                // which is referring back here.
5355                panic!("{message}");
5356            } else if fallback_to_slow_path {
5357                // the above bad-index-entry check must had been checked first to retain the same
5358                // behavior
5359                return Some((
5360                    maybe_account_accessor.expect("must be some if clone_in_lock=true"),
5361                    new_slot,
5362                ));
5363            }
5364
5365            slot = new_slot;
5366            storage_location = new_storage_location;
5367        }
5368    }
5369
5370    fn do_load(
5371        &self,
5372        ancestors: &Ancestors,
5373        pubkey: &Pubkey,
5374        max_root: Option<Slot>,
5375        load_hint: LoadHint,
5376        load_zero_lamports: LoadZeroLamports,
5377    ) -> Option<(AccountSharedData, Slot)> {
5378        self.do_load_with_populate_read_cache(
5379            ancestors,
5380            pubkey,
5381            max_root,
5382            load_hint,
5383            false,
5384            load_zero_lamports,
5385        )
5386    }
5387
5388    /// Load account with `pubkey` and maybe put into read cache.
5389    ///
5390    /// If the account is not already cached, invoke `should_put_in_read_cache_fn`.
5391    /// The caller can inspect the account and indicate if it should be put into the read cache or not.
5392    ///
5393    /// Return the account and the slot when the account was last stored.
5394    /// Return None for ZeroLamport accounts.
5395    pub fn load_account_with(
5396        &self,
5397        ancestors: &Ancestors,
5398        pubkey: &Pubkey,
5399        should_put_in_read_cache_fn: impl Fn(&AccountSharedData) -> bool,
5400    ) -> Option<(AccountSharedData, Slot)> {
5401        let (slot, storage_location, _maybe_account_accesor) =
5402            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, None, false)?;
5403        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5404
5405        let in_write_cache = storage_location.is_cached();
5406        if !in_write_cache {
5407            let result = self.read_only_accounts_cache.load(*pubkey, slot);
5408            if let Some(account) = result {
5409                if account.is_zero_lamport() {
5410                    return None;
5411                }
5412                return Some((account, slot));
5413            }
5414        }
5415
5416        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
5417            slot,
5418            storage_location,
5419            ancestors,
5420            pubkey,
5421            None,
5422            LoadHint::Unspecified,
5423        )?;
5424
5425        // note that the account being in the cache could be different now than it was previously
5426        // since the cache could be flushed in between the 2 calls.
5427        let in_write_cache = matches!(account_accessor, LoadedAccountAccessor::Cached(_));
5428        let account = account_accessor.check_and_get_loaded_account_shared_data();
5429        if account.is_zero_lamport() {
5430            return None;
5431        }
5432
5433        if !in_write_cache && should_put_in_read_cache_fn(&account) {
5434            /*
5435            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
5436            safe/reflect 'A''s latest state on this fork.
5437            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
5438            not the read-only cache, after it's been updated in replay of slot 'S'.
5439            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
5440            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
5441            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
5442            which means '(S', A)' does not exist in the write cache yet.
5443            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
5444            must exist in the write cache, which is a contradiction.
5445            */
5446            self.read_only_accounts_cache
5447                .store(*pubkey, slot, account.clone());
5448        }
5449        Some((account, slot))
5450    }
5451
5452    /// if 'load_into_read_cache_only', then return value is meaningless.
5453    ///   The goal is to get the account into the read-only cache.
5454    fn do_load_with_populate_read_cache(
5455        &self,
5456        ancestors: &Ancestors,
5457        pubkey: &Pubkey,
5458        max_root: Option<Slot>,
5459        load_hint: LoadHint,
5460        load_into_read_cache_only: bool,
5461        load_zero_lamports: LoadZeroLamports,
5462    ) -> Option<(AccountSharedData, Slot)> {
5463        #[cfg(not(test))]
5464        assert!(max_root.is_none());
5465
5466        let (slot, storage_location, _maybe_account_accesor) =
5467            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
5468        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5469
5470        let in_write_cache = storage_location.is_cached();
5471        if !load_into_read_cache_only {
5472            if !in_write_cache {
5473                let result = self.read_only_accounts_cache.load(*pubkey, slot);
5474                if let Some(account) = result {
5475                    if matches!(load_zero_lamports, LoadZeroLamports::None)
5476                        && account.is_zero_lamport()
5477                    {
5478                        return None;
5479                    }
5480                    return Some((account, slot));
5481                }
5482            }
5483        } else {
5484            // goal is to load into read cache
5485            if in_write_cache {
5486                // no reason to load in read cache. already in write cache
5487                return None;
5488            }
5489            if self.read_only_accounts_cache.in_cache(pubkey, slot) {
5490                // already in read cache
5491                return None;
5492            }
5493        }
5494
5495        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
5496            slot,
5497            storage_location,
5498            ancestors,
5499            pubkey,
5500            max_root,
5501            load_hint,
5502        )?;
5503        // note that the account being in the cache could be different now than it was previously
5504        // since the cache could be flushed in between the 2 calls.
5505        let in_write_cache = matches!(account_accessor, LoadedAccountAccessor::Cached(_));
5506        let account = account_accessor.check_and_get_loaded_account_shared_data();
5507        if matches!(load_zero_lamports, LoadZeroLamports::None) && account.is_zero_lamport() {
5508            return None;
5509        }
5510
5511        if !in_write_cache && load_hint != LoadHint::FixedMaxRootDoNotPopulateReadCache {
5512            /*
5513            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
5514            safe/reflect 'A''s latest state on this fork.
5515            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
5516            not the read-only cache, after it's been updated in replay of slot 'S'.
5517            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
5518            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
5519            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
5520            which means '(S', A)' does not exist in the write cache yet.
5521            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
5522            must exist in the write cache, which is a contradiction.
5523            */
5524            self.read_only_accounts_cache
5525                .store(*pubkey, slot, account.clone());
5526        }
5527        Some((account, slot))
5528    }
5529
5530    pub fn load_account_hash(
5531        &self,
5532        ancestors: &Ancestors,
5533        pubkey: &Pubkey,
5534        max_root: Option<Slot>,
5535        load_hint: LoadHint,
5536    ) -> Option<AccountHash> {
5537        let (slot, storage_location, _maybe_account_accesor) =
5538            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
5539        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
5540
5541        let (mut account_accessor, _) = self.retry_to_get_account_accessor(
5542            slot,
5543            storage_location,
5544            ancestors,
5545            pubkey,
5546            max_root,
5547            load_hint,
5548        )?;
5549        account_accessor
5550            .check_and_get_loaded_account(|loaded_account| Some(loaded_account.loaded_hash()))
5551    }
5552
5553    fn get_account_accessor<'a>(
5554        &'a self,
5555        slot: Slot,
5556        pubkey: &'a Pubkey,
5557        storage_location: &StorageLocation,
5558    ) -> LoadedAccountAccessor<'a> {
5559        match storage_location {
5560            StorageLocation::Cached => {
5561                let maybe_cached_account = self.accounts_cache.load(slot, pubkey).map(Cow::Owned);
5562                LoadedAccountAccessor::Cached(maybe_cached_account)
5563            }
5564            StorageLocation::AppendVec(store_id, offset) => {
5565                let maybe_storage_entry = self
5566                    .storage
5567                    .get_account_storage_entry(slot, *store_id)
5568                    .map(|account_storage_entry| (account_storage_entry, *offset));
5569                LoadedAccountAccessor::Stored(maybe_storage_entry)
5570            }
5571        }
5572    }
5573
5574    fn find_storage_candidate(&self, slot: Slot) -> Arc<AccountStorageEntry> {
5575        let mut get_slot_stores = Measure::start("get_slot_stores");
5576        let store = self.storage.get_slot_storage_entry(slot);
5577        get_slot_stores.stop();
5578        self.stats
5579            .store_get_slot_store
5580            .fetch_add(get_slot_stores.as_us(), Ordering::Relaxed);
5581        let mut find_existing = Measure::start("find_existing");
5582        if let Some(store) = store {
5583            if store.try_available() {
5584                let ret = store.clone();
5585                drop(store);
5586                find_existing.stop();
5587                self.stats
5588                    .store_find_existing
5589                    .fetch_add(find_existing.as_us(), Ordering::Relaxed);
5590                return ret;
5591            }
5592        }
5593        find_existing.stop();
5594        self.stats
5595            .store_find_existing
5596            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
5597
5598        let store = self.create_store(slot, self.file_size, "store", &self.paths);
5599
5600        // try_available is like taking a lock on the store,
5601        // preventing other threads from using it.
5602        // It must succeed here and happen before insert,
5603        // otherwise another thread could also grab it from the index.
5604        assert!(store.try_available());
5605        self.insert_store(slot, store.clone());
5606        store
5607    }
5608
5609    fn has_space_available(&self, slot: Slot, size: u64) -> bool {
5610        let store = self.storage.get_slot_storage_entry(slot).unwrap();
5611        if store.status() == AccountStorageStatus::Available
5612            && store.accounts.remaining_bytes() >= size
5613        {
5614            return true;
5615        }
5616        false
5617    }
5618
5619    fn create_store(
5620        &self,
5621        slot: Slot,
5622        size: u64,
5623        from: &str,
5624        paths: &[PathBuf],
5625    ) -> Arc<AccountStorageEntry> {
5626        self.stats
5627            .create_store_count
5628            .fetch_add(1, Ordering::Relaxed);
5629        let path_index = thread_rng().gen_range(0..paths.len());
5630        let store = Arc::new(self.new_storage_entry(slot, Path::new(&paths[path_index]), size));
5631
5632        debug!(
5633            "creating store: {} slot: {} len: {} size: {} from: {} path: {}",
5634            store.id(),
5635            slot,
5636            store.accounts.len(),
5637            store.accounts.capacity(),
5638            from,
5639            store.accounts.path().display(),
5640        );
5641
5642        store
5643    }
5644
5645    fn create_and_insert_store(
5646        &self,
5647        slot: Slot,
5648        size: u64,
5649        from: &str,
5650    ) -> Arc<AccountStorageEntry> {
5651        self.create_and_insert_store_with_paths(slot, size, from, &self.paths)
5652    }
5653
5654    fn create_and_insert_store_with_paths(
5655        &self,
5656        slot: Slot,
5657        size: u64,
5658        from: &str,
5659        paths: &[PathBuf],
5660    ) -> Arc<AccountStorageEntry> {
5661        let store = self.create_store(slot, size, from, paths);
5662        let store_for_index = store.clone();
5663
5664        self.insert_store(slot, store_for_index);
5665        store
5666    }
5667
5668    fn insert_store(&self, slot: Slot, store: Arc<AccountStorageEntry>) {
5669        self.storage.insert(slot, store)
5670    }
5671
5672    pub fn enable_bank_drop_callback(&self) {
5673        self.is_bank_drop_callback_enabled
5674            .store(true, Ordering::Release);
5675    }
5676
5677    /// This should only be called after the `Bank::drop()` runs in bank.rs, See BANK_DROP_SAFETY
5678    /// comment below for more explanation.
5679    ///   * `is_serialized_with_abs` - indicates whether this call runs sequentially with all other
5680    ///        accounts_db relevant calls, such as shrinking, purging etc., in account background
5681    ///        service.
5682    pub fn purge_slot(&self, slot: Slot, bank_id: BankId, is_serialized_with_abs: bool) {
5683        if self.is_bank_drop_callback_enabled.load(Ordering::Acquire) && !is_serialized_with_abs {
5684            panic!(
5685                "bad drop callpath detected; Bank::drop() must run serially with other logic in
5686                ABS like clean_accounts()"
5687            )
5688        }
5689
5690        // BANK_DROP_SAFETY: Because this function only runs once the bank is dropped,
5691        // we know that there are no longer any ongoing scans on this bank, because scans require
5692        // and hold a reference to the bank at the tip of the fork they're scanning. Hence it's
5693        // safe to remove this bank_id from the `removed_bank_ids` list at this point.
5694        if self
5695            .accounts_index
5696            .removed_bank_ids
5697            .lock()
5698            .unwrap()
5699            .remove(&bank_id)
5700        {
5701            // If this slot was already cleaned up, no need to do any further cleans
5702            return;
5703        }
5704
5705        self.purge_slots(std::iter::once(&slot));
5706    }
5707
5708    /// Purges every slot in `removed_slots` from both the cache and storage. This includes
5709    /// entries in the accounts index, cache entries, and any backing storage entries.
5710    pub fn purge_slots_from_cache_and_store<'a>(
5711        &self,
5712        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5713        purge_stats: &PurgeStats,
5714        log_accounts: bool,
5715    ) {
5716        let mut remove_cache_elapsed_across_slots = 0;
5717        let mut num_cached_slots_removed = 0;
5718        let mut total_removed_cached_bytes = 0;
5719        if log_accounts {
5720            if let Some(min) = removed_slots.clone().min() {
5721                info!(
5722                    "purge_slots_from_cache_and_store: {:?}",
5723                    self.get_pubkey_hash_for_slot(*min).0
5724                );
5725            }
5726        }
5727        for remove_slot in removed_slots {
5728            // This function is only currently safe with respect to `flush_slot_cache()` because
5729            // both functions run serially in AccountsBackgroundService.
5730            let mut remove_cache_elapsed = Measure::start("remove_cache_elapsed");
5731            // Note: we cannot remove this slot from the slot cache until we've removed its
5732            // entries from the accounts index first. This is because `scan_accounts()` relies on
5733            // holding the index lock, finding the index entry, and then looking up the entry
5734            // in the cache. If it fails to find that entry, it will panic in `get_loaded_account()`
5735            if let Some(slot_cache) = self.accounts_cache.slot_cache(*remove_slot) {
5736                // If the slot is still in the cache, remove the backing storages for
5737                // the slot and from the Accounts Index
5738                num_cached_slots_removed += 1;
5739                total_removed_cached_bytes += slot_cache.total_bytes();
5740                self.purge_slot_cache(*remove_slot, slot_cache);
5741                remove_cache_elapsed.stop();
5742                remove_cache_elapsed_across_slots += remove_cache_elapsed.as_us();
5743                // Nobody else should have removed the slot cache entry yet
5744                assert!(self.accounts_cache.remove_slot(*remove_slot).is_some());
5745            } else {
5746                self.purge_slot_storage(*remove_slot, purge_stats);
5747            }
5748            // It should not be possible that a slot is neither in the cache or storage. Even in
5749            // a slot with all ticks, `Bank::new_from_parent()` immediately stores some sysvars
5750            // on bank creation.
5751        }
5752
5753        purge_stats
5754            .remove_cache_elapsed
5755            .fetch_add(remove_cache_elapsed_across_slots, Ordering::Relaxed);
5756        purge_stats
5757            .num_cached_slots_removed
5758            .fetch_add(num_cached_slots_removed, Ordering::Relaxed);
5759        purge_stats
5760            .total_removed_cached_bytes
5761            .fetch_add(total_removed_cached_bytes, Ordering::Relaxed);
5762    }
5763
5764    /// Purge the backing storage entries for the given slot, does not purge from
5765    /// the cache!
5766    fn purge_dead_slots_from_storage<'a>(
5767        &'a self,
5768        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5769        purge_stats: &PurgeStats,
5770    ) {
5771        // Check all slots `removed_slots` are no longer "relevant" roots.
5772        // Note that the slots here could have been rooted slots, but if they're passed here
5773        // for removal it means:
5774        // 1) All updates in that old root have been outdated by updates in newer roots
5775        // 2) Those slots/roots should have already been purged from the accounts index root
5776        // tracking metadata via `accounts_index.clean_dead_slot()`.
5777        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5778        assert!(self
5779            .accounts_index
5780            .get_rooted_from_list(removed_slots.clone())
5781            .is_empty());
5782        safety_checks_elapsed.stop();
5783        purge_stats
5784            .safety_checks_elapsed
5785            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5786
5787        let mut total_removed_stored_bytes = 0;
5788        let mut all_removed_slot_storages = vec![];
5789
5790        let mut remove_storage_entries_elapsed = Measure::start("remove_storage_entries_elapsed");
5791        for remove_slot in removed_slots {
5792            // Remove the storage entries and collect some metrics
5793            if let Some(store) = self.storage.remove(remove_slot, false) {
5794                total_removed_stored_bytes += store.accounts.capacity();
5795                all_removed_slot_storages.push(store);
5796            }
5797        }
5798        remove_storage_entries_elapsed.stop();
5799        let num_stored_slots_removed = all_removed_slot_storages.len();
5800
5801        // Backing mmaps for removed storages entries explicitly dropped here outside
5802        // of any locks
5803        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
5804        drop(all_removed_slot_storages);
5805        drop_storage_entries_elapsed.stop();
5806
5807        purge_stats
5808            .remove_storage_entries_elapsed
5809            .fetch_add(remove_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5810        purge_stats
5811            .drop_storage_entries_elapsed
5812            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5813        purge_stats
5814            .num_stored_slots_removed
5815            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
5816        purge_stats
5817            .total_removed_storage_entries
5818            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
5819        purge_stats
5820            .total_removed_stored_bytes
5821            .fetch_add(total_removed_stored_bytes, Ordering::Relaxed);
5822        self.stats
5823            .dropped_stores
5824            .fetch_add(num_stored_slots_removed as u64, Ordering::Relaxed);
5825    }
5826
5827    fn purge_slot_cache(&self, purged_slot: Slot, slot_cache: SlotCache) {
5828        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
5829        let pubkey_to_slot_set: Vec<(Pubkey, Slot)> = slot_cache
5830            .iter()
5831            .map(|account| {
5832                purged_slot_pubkeys.insert((purged_slot, *account.key()));
5833                (*account.key(), purged_slot)
5834            })
5835            .collect();
5836        self.purge_slot_cache_pubkeys(
5837            purged_slot,
5838            purged_slot_pubkeys,
5839            pubkey_to_slot_set,
5840            true,
5841            &HashSet::default(),
5842        );
5843    }
5844
5845    fn purge_slot_cache_pubkeys(
5846        &self,
5847        purged_slot: Slot,
5848        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
5849        pubkey_to_slot_set: Vec<(Pubkey, Slot)>,
5850        is_dead: bool,
5851        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
5852    ) {
5853        // Slot purged from cache should not exist in the backing store
5854        assert!(self
5855            .storage
5856            .get_slot_storage_entry_shrinking_in_progress_ok(purged_slot)
5857            .is_none());
5858        let num_purged_keys = pubkey_to_slot_set.len();
5859        let (reclaims, _) = self.purge_keys_exact(pubkey_to_slot_set.iter());
5860        assert_eq!(reclaims.len(), num_purged_keys);
5861        if is_dead {
5862            self.remove_dead_slots_metadata(
5863                std::iter::once(&purged_slot),
5864                purged_slot_pubkeys,
5865                None,
5866                pubkeys_removed_from_accounts_index,
5867            );
5868        }
5869    }
5870
5871    fn purge_slot_storage(&self, remove_slot: Slot, purge_stats: &PurgeStats) {
5872        // Because AccountsBackgroundService synchronously flushes from the accounts cache
5873        // and handles all Bank::drop() (the cleanup function that leads to this
5874        // function call), then we don't need to worry above an overlapping cache flush
5875        // with this function call. This means, if we get into this case, we can be
5876        // confident that the entire state for this slot has been flushed to the storage
5877        // already.
5878        let mut scan_storages_elapsed = Measure::start("scan_storages_elapsed");
5879        let mut stored_keys = HashSet::new();
5880        if let Some(storage) = self
5881            .storage
5882            .get_slot_storage_entry_shrinking_in_progress_ok(remove_slot)
5883        {
5884            storage.accounts.scan_pubkeys(|pk| {
5885                stored_keys.insert((*pk, remove_slot));
5886            });
5887        }
5888        scan_storages_elapsed.stop();
5889        purge_stats
5890            .scan_storages_elapsed
5891            .fetch_add(scan_storages_elapsed.as_us(), Ordering::Relaxed);
5892
5893        let mut purge_accounts_index_elapsed = Measure::start("purge_accounts_index_elapsed");
5894        // Purge this slot from the accounts index
5895        let (reclaims, pubkeys_removed_from_accounts_index) =
5896            self.purge_keys_exact(stored_keys.iter());
5897        purge_accounts_index_elapsed.stop();
5898        purge_stats
5899            .purge_accounts_index_elapsed
5900            .fetch_add(purge_accounts_index_elapsed.as_us(), Ordering::Relaxed);
5901
5902        // `handle_reclaims()` should remove all the account index entries and
5903        // storage entries
5904        let mut handle_reclaims_elapsed = Measure::start("handle_reclaims_elapsed");
5905        // Slot should be dead after removing all its account entries
5906        let expected_dead_slot = Some(remove_slot);
5907        self.handle_reclaims(
5908            (!reclaims.is_empty()).then(|| reclaims.iter()),
5909            expected_dead_slot,
5910            false,
5911            &pubkeys_removed_from_accounts_index,
5912            HandleReclaims::ProcessDeadSlots(purge_stats),
5913        );
5914        handle_reclaims_elapsed.stop();
5915        purge_stats
5916            .handle_reclaims_elapsed
5917            .fetch_add(handle_reclaims_elapsed.as_us(), Ordering::Relaxed);
5918        // After handling the reclaimed entries, this slot's
5919        // storage entries should be purged from self.storage
5920        assert!(
5921            self.storage.get_slot_storage_entry(remove_slot).is_none(),
5922            "slot {remove_slot} is not none"
5923        );
5924    }
5925
5926    fn purge_slots<'a>(&self, slots: impl Iterator<Item = &'a Slot> + Clone) {
5927        // `add_root()` should be called first
5928        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5929        let non_roots = slots
5930            // Only safe to check when there are duplicate versions of a slot
5931            // because ReplayStage will not make new roots before dumping the
5932            // duplicate slots first. Thus we will not be in a case where we
5933            // root slot `S`, then try to dump some other version of slot `S`, the
5934            // dumping has to finish first
5935            //
5936            // Also note roots are never removed via `remove_unrooted_slot()`, so
5937            // it's safe to filter them out here as they won't need deletion from
5938            // self.accounts_index.removed_bank_ids in `purge_slots_from_cache_and_store()`.
5939            .filter(|slot| !self.accounts_index.is_alive_root(**slot));
5940        safety_checks_elapsed.stop();
5941        self.external_purge_slots_stats
5942            .safety_checks_elapsed
5943            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5944        self.purge_slots_from_cache_and_store(non_roots, &self.external_purge_slots_stats, false);
5945        self.external_purge_slots_stats
5946            .report("external_purge_slots_stats", Some(1000));
5947    }
5948
5949    pub fn remove_unrooted_slots(&self, remove_slots: &[(Slot, BankId)]) {
5950        let rooted_slots = self
5951            .accounts_index
5952            .get_rooted_from_list(remove_slots.iter().map(|(slot, _)| slot));
5953        assert!(
5954            rooted_slots.is_empty(),
5955            "Trying to remove accounts for rooted slots {rooted_slots:?}"
5956        );
5957
5958        let RemoveUnrootedSlotsSynchronization {
5959            slots_under_contention,
5960            signal,
5961        } = &self.remove_unrooted_slots_synchronization;
5962
5963        {
5964            // Slots that are currently being flushed by flush_slot_cache()
5965
5966            let mut currently_contended_slots = slots_under_contention.lock().unwrap();
5967
5968            // Slots that are currently being flushed by flush_slot_cache() AND
5969            // we want to remove in this function
5970            let mut remaining_contended_flush_slots: Vec<Slot> = remove_slots
5971                .iter()
5972                .filter_map(|(remove_slot, _)| {
5973                    // Reserve the slots that we want to purge that aren't currently
5974                    // being flushed to prevent cache from flushing those slots in
5975                    // the future.
5976                    //
5977                    // Note that the single replay thread has to remove a specific slot `N`
5978                    // before another version of the same slot can be replayed. This means
5979                    // multiple threads should not call `remove_unrooted_slots()` simultaneously
5980                    // with the same slot.
5981                    let is_being_flushed = !currently_contended_slots.insert(*remove_slot);
5982                    // If the cache is currently flushing this slot, add it to the list
5983                    is_being_flushed.then_some(remove_slot)
5984                })
5985                .cloned()
5986                .collect();
5987
5988            // Wait for cache flushes to finish
5989            loop {
5990                if !remaining_contended_flush_slots.is_empty() {
5991                    // Wait for the signal that the cache has finished flushing a slot
5992                    //
5993                    // Don't wait if the remaining_contended_flush_slots is empty, otherwise
5994                    // we may never get a signal since there's no cache flush thread to
5995                    // do the signaling
5996                    currently_contended_slots = signal.wait(currently_contended_slots).unwrap();
5997                } else {
5998                    // There are no slots being flushed to wait on, so it's safe to continue
5999                    // to purging the slots we want to purge!
6000                    break;
6001                }
6002
6003                // For each slot the cache flush has finished, mark that we're about to start
6004                // purging these slots by reserving it in `currently_contended_slots`.
6005                remaining_contended_flush_slots.retain(|flush_slot| {
6006                    // returns true if slot was already in set. This means slot is being flushed
6007                    !currently_contended_slots.insert(*flush_slot)
6008                });
6009            }
6010        }
6011
6012        // Mark down these slots are about to be purged so that new attempts to scan these
6013        // banks fail, and any ongoing scans over these slots will detect that they should abort
6014        // their results
6015        {
6016            let mut locked_removed_bank_ids = self.accounts_index.removed_bank_ids.lock().unwrap();
6017            for (_slot, remove_bank_id) in remove_slots.iter() {
6018                locked_removed_bank_ids.insert(*remove_bank_id);
6019            }
6020        }
6021
6022        let remove_unrooted_purge_stats = PurgeStats::default();
6023        self.purge_slots_from_cache_and_store(
6024            remove_slots.iter().map(|(slot, _)| slot),
6025            &remove_unrooted_purge_stats,
6026            true,
6027        );
6028        remove_unrooted_purge_stats.report("remove_unrooted_slots_purge_slots_stats", None);
6029
6030        let mut currently_contended_slots = slots_under_contention.lock().unwrap();
6031        for (remove_slot, _) in remove_slots {
6032            assert!(currently_contended_slots.remove(remove_slot));
6033        }
6034    }
6035
6036    /// Calculates the `AccountLtHash` of `account`
6037    pub fn lt_hash_account(account: &impl ReadableAccount, pubkey: &Pubkey) -> AccountLtHash {
6038        if account.lamports() == 0 {
6039            return ZERO_LAMPORT_ACCOUNT_LT_HASH;
6040        }
6041
6042        let hasher = Self::hash_account_helper(account, pubkey);
6043        let lt_hash = LtHash::with(&hasher);
6044        AccountLtHash(lt_hash)
6045    }
6046
6047    /// Calculates the `AccountHash` of `account`
6048    pub fn hash_account<T: ReadableAccount>(account: &T, pubkey: &Pubkey) -> AccountHash {
6049        if account.lamports() == 0 {
6050            return ZERO_LAMPORT_ACCOUNT_HASH;
6051        }
6052
6053        let hasher = Self::hash_account_helper(account, pubkey);
6054        let hash = Hash::new_from_array(hasher.finalize().into());
6055        AccountHash(hash)
6056    }
6057
6058    /// Hashes `account` and returns the underlying Hasher
6059    fn hash_account_helper(account: &impl ReadableAccount, pubkey: &Pubkey) -> blake3::Hasher {
6060        let mut hasher = blake3::Hasher::new();
6061
6062        // allocate a buffer on the stack that's big enough
6063        // to hold a token account or a stake account
6064        const META_SIZE: usize = 8 /* lamports */ + 8 /* rent_epoch */ + 1 /* executable */ + 32 /* owner */ + 32 /* pubkey */;
6065        const DATA_SIZE: usize = 200; // stake accounts are 200 B and token accounts are 165-182ish B
6066        const BUFFER_SIZE: usize = META_SIZE + DATA_SIZE;
6067        let mut buffer = SmallVec::<[u8; BUFFER_SIZE]>::new();
6068
6069        // collect lamports, rent_epoch into buffer to hash
6070        buffer.extend_from_slice(&account.lamports().to_le_bytes());
6071        buffer.extend_from_slice(&account.rent_epoch().to_le_bytes());
6072
6073        let data = account.data();
6074        if data.len() > DATA_SIZE {
6075            // For larger accounts whose data can't fit into the buffer, update the hash now.
6076            hasher.update(&buffer);
6077            buffer.clear();
6078
6079            // hash account's data
6080            hasher.update(data);
6081        } else {
6082            // For small accounts whose data can fit into the buffer, append it to the buffer.
6083            buffer.extend_from_slice(data);
6084        }
6085
6086        // collect exec_flag, owner, pubkey into buffer to hash
6087        buffer.push(account.executable().into());
6088        buffer.extend_from_slice(account.owner().as_ref());
6089        buffer.extend_from_slice(pubkey.as_ref());
6090        hasher.update(&buffer);
6091
6092        hasher
6093    }
6094
6095    fn write_accounts_to_storage<'a>(
6096        &self,
6097        slot: Slot,
6098        storage: &AccountStorageEntry,
6099        accounts_and_meta_to_store: &impl StorableAccounts<'a>,
6100    ) -> Vec<AccountInfo> {
6101        let mut infos: Vec<AccountInfo> = Vec::with_capacity(accounts_and_meta_to_store.len());
6102        let mut total_append_accounts_us = 0;
6103        while infos.len() < accounts_and_meta_to_store.len() {
6104            let mut append_accounts = Measure::start("append_accounts");
6105            let stored_accounts_info = storage
6106                .accounts
6107                .append_accounts(accounts_and_meta_to_store, infos.len());
6108            append_accounts.stop();
6109            total_append_accounts_us += append_accounts.as_us();
6110            let Some(stored_accounts_info) = stored_accounts_info else {
6111                storage.set_status(AccountStorageStatus::Full);
6112
6113                // See if an account overflows the append vecs in the slot.
6114                accounts_and_meta_to_store.account_default_if_zero_lamport(
6115                    infos.len(),
6116                    |account| {
6117                        let data_len = account.data().len();
6118                        let data_len = (data_len + STORE_META_OVERHEAD) as u64;
6119                        if !self.has_space_available(slot, data_len) {
6120                            info!(
6121                                "write_accounts_to_storage, no space: {}, {}, {}, {}, {}",
6122                                storage.accounts.capacity(),
6123                                storage.accounts.remaining_bytes(),
6124                                data_len,
6125                                infos.len(),
6126                                accounts_and_meta_to_store.len()
6127                            );
6128                            let special_store_size = std::cmp::max(data_len * 2, self.file_size);
6129                            self.create_and_insert_store(slot, special_store_size, "large create");
6130                        }
6131                    },
6132                );
6133                continue;
6134            };
6135
6136            let store_id = storage.id();
6137            for (i, offset) in stored_accounts_info.offsets.iter().enumerate() {
6138                infos.push(AccountInfo::new(
6139                    StorageLocation::AppendVec(store_id, *offset),
6140                    accounts_and_meta_to_store
6141                        .account_default_if_zero_lamport(i, |account| account.lamports()),
6142                ));
6143            }
6144            storage.add_accounts(
6145                stored_accounts_info.offsets.len(),
6146                stored_accounts_info.size,
6147            );
6148
6149            // restore the state to available
6150            storage.set_status(AccountStorageStatus::Available);
6151        }
6152
6153        self.stats
6154            .store_append_accounts
6155            .fetch_add(total_append_accounts_us, Ordering::Relaxed);
6156
6157        infos
6158    }
6159
6160    pub fn mark_slot_frozen(&self, slot: Slot) {
6161        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
6162            slot_cache.mark_slot_frozen();
6163            slot_cache.report_slot_store_metrics();
6164        }
6165        self.accounts_cache.report_size();
6166    }
6167
6168    // These functions/fields are only usable from a dev context (i.e. tests and benches)
6169    #[cfg(feature = "dev-context-only-utils")]
6170    pub fn flush_accounts_cache_slot_for_tests(&self, slot: Slot) {
6171        self.flush_slot_cache(slot);
6172    }
6173
6174    /// true if write cache is too big
6175    fn should_aggressively_flush_cache(&self) -> bool {
6176        self.write_cache_limit_bytes
6177            .unwrap_or(WRITE_CACHE_LIMIT_BYTES_DEFAULT)
6178            < self.accounts_cache.size()
6179    }
6180
6181    // `force_flush` flushes all the cached roots `<= requested_flush_root`. It also then
6182    // flushes:
6183    // 1) excess remaining roots or unrooted slots while 'should_aggressively_flush_cache' is true
6184    pub fn flush_accounts_cache(&self, force_flush: bool, requested_flush_root: Option<Slot>) {
6185        #[cfg(not(test))]
6186        assert!(requested_flush_root.is_some());
6187
6188        if !force_flush && !self.should_aggressively_flush_cache() {
6189            return;
6190        }
6191
6192        // Flush only the roots <= requested_flush_root, so that snapshotting has all
6193        // the relevant roots in storage.
6194        let mut flush_roots_elapsed = Measure::start("flush_roots_elapsed");
6195        let mut account_bytes_saved = 0;
6196        let mut num_accounts_saved = 0;
6197
6198        let _guard = self.active_stats.activate(ActiveStatItem::Flush);
6199
6200        // Note even if force_flush is false, we will still flush all roots <= the
6201        // given `requested_flush_root`, even if some of the later roots cannot be used for
6202        // cleaning due to an ongoing scan
6203        let (total_new_cleaned_roots, num_cleaned_roots_flushed, mut flush_stats) = self
6204            .flush_rooted_accounts_cache(
6205                requested_flush_root,
6206                Some((&mut account_bytes_saved, &mut num_accounts_saved)),
6207            );
6208        flush_roots_elapsed.stop();
6209
6210        // Note we don't purge unrooted slots here because there may be ongoing scans/references
6211        // for those slot, let the Bank::drop() implementation do cleanup instead on dead
6212        // banks
6213
6214        // If 'should_aggressively_flush_cache', then flush the excess ones to storage
6215        let (total_new_excess_roots, num_excess_roots_flushed, flush_stats_aggressively) =
6216            if self.should_aggressively_flush_cache() {
6217                // Start by flushing the roots
6218                //
6219                // Cannot do any cleaning on roots past `requested_flush_root` because future
6220                // snapshots may need updates from those later slots, hence we pass `None`
6221                // for `should_clean`.
6222                self.flush_rooted_accounts_cache(None, None)
6223            } else {
6224                (0, 0, FlushStats::default())
6225            };
6226        flush_stats.accumulate(&flush_stats_aggressively);
6227
6228        let mut excess_slot_count = 0;
6229        let mut unflushable_unrooted_slot_count = 0;
6230        let max_flushed_root = self.accounts_cache.fetch_max_flush_root();
6231        if self.should_aggressively_flush_cache() {
6232            let mut old_slots = self.accounts_cache.cached_frozen_slots();
6233            old_slots.sort_unstable();
6234            excess_slot_count = old_slots.len();
6235            let mut flush_stats = FlushStats::default();
6236            old_slots.into_iter().for_each(|old_slot| {
6237                // Don't flush slots that are known to be unrooted
6238                if old_slot > max_flushed_root {
6239                    if self.should_aggressively_flush_cache() {
6240                        if let Some(stats) = self.flush_slot_cache(old_slot) {
6241                            flush_stats.accumulate(&stats);
6242                        }
6243                    }
6244                } else {
6245                    unflushable_unrooted_slot_count += 1;
6246                }
6247            });
6248            datapoint_info!(
6249                "accounts_db-flush_accounts_cache_aggressively",
6250                ("num_flushed", flush_stats.num_flushed.0, i64),
6251                ("num_purged", flush_stats.num_purged.0, i64),
6252                ("total_flush_size", flush_stats.total_size.0, i64),
6253                ("total_cache_size", self.accounts_cache.size(), i64),
6254                ("total_frozen_slots", excess_slot_count, i64),
6255                ("total_slots", self.accounts_cache.num_slots(), i64),
6256            );
6257        }
6258
6259        datapoint_info!(
6260            "accounts_db-flush_accounts_cache",
6261            ("total_new_cleaned_roots", total_new_cleaned_roots, i64),
6262            ("num_cleaned_roots_flushed", num_cleaned_roots_flushed, i64),
6263            ("total_new_excess_roots", total_new_excess_roots, i64),
6264            ("num_excess_roots_flushed", num_excess_roots_flushed, i64),
6265            ("excess_slot_count", excess_slot_count, i64),
6266            (
6267                "unflushable_unrooted_slot_count",
6268                unflushable_unrooted_slot_count,
6269                i64
6270            ),
6271            ("flush_roots_elapsed", flush_roots_elapsed.as_us(), i64),
6272            ("account_bytes_saved", account_bytes_saved, i64),
6273            ("num_accounts_saved", num_accounts_saved, i64),
6274            (
6275                "store_accounts_total_us",
6276                flush_stats.store_accounts_total_us.0,
6277                i64
6278            ),
6279            (
6280                "update_index_us",
6281                flush_stats.store_accounts_timing.update_index_elapsed,
6282                i64
6283            ),
6284            (
6285                "store_accounts_elapsed_us",
6286                flush_stats.store_accounts_timing.store_accounts_elapsed,
6287                i64
6288            ),
6289            (
6290                "handle_reclaims_elapsed_us",
6291                flush_stats.store_accounts_timing.handle_reclaims_elapsed,
6292                i64
6293            ),
6294        );
6295    }
6296
6297    fn flush_rooted_accounts_cache(
6298        &self,
6299        requested_flush_root: Option<Slot>,
6300        should_clean: Option<(&mut usize, &mut usize)>,
6301    ) -> (usize, usize, FlushStats) {
6302        let max_clean_root = should_clean.as_ref().and_then(|_| {
6303            // If there is a long running scan going on, this could prevent any cleaning
6304            // based on updates from slots > `max_clean_root`.
6305            self.max_clean_root(requested_flush_root)
6306        });
6307
6308        let mut written_accounts = HashSet::new();
6309
6310        // If `should_clean` is None, then`should_flush_f` is also None, which will cause
6311        // `flush_slot_cache` to flush all accounts to storage without cleaning any accounts.
6312        let mut should_flush_f = should_clean.map(|(account_bytes_saved, num_accounts_saved)| {
6313            move |&pubkey: &Pubkey, account: &AccountSharedData| {
6314                // if not in hashset, then not flushed previously, so flush it
6315                let should_flush = written_accounts.insert(pubkey);
6316                if !should_flush {
6317                    *account_bytes_saved += account.data().len();
6318                    *num_accounts_saved += 1;
6319                    // If a later root already wrote this account, no point
6320                    // in flushing it
6321                }
6322                should_flush
6323            }
6324        });
6325
6326        // Always flush up to `requested_flush_root`, which is necessary for things like snapshotting.
6327        let flushed_roots: BTreeSet<Slot> = self.accounts_cache.clear_roots(requested_flush_root);
6328
6329        // Iterate from highest to lowest so that we don't need to flush earlier
6330        // outdated updates in earlier roots
6331        let mut num_roots_flushed = 0;
6332        let mut flush_stats = FlushStats::default();
6333        for &root in flushed_roots.iter().rev() {
6334            if let Some(stats) =
6335                self.flush_slot_cache_with_clean(root, should_flush_f.as_mut(), max_clean_root)
6336            {
6337                num_roots_flushed += 1;
6338                flush_stats.accumulate(&stats);
6339            }
6340        }
6341
6342        // Note that self.flush_slot_cache_with_clean() can return None if the
6343        // slot is already been flushed. This can happen if the cache is
6344        // overwhelmed and we flushed some yet to be rooted frozen slots.
6345        // However, independent of whether the last slot was actually flushed
6346        // from the cache by the above loop, we should always update the
6347        // `max_flush_root` to the max of the flushed roots, because that's
6348        // max_flushed_root tracks the logical last root that was flushed to
6349        // storage by snapshotting.
6350        if let Some(&root) = flushed_roots.last() {
6351            self.accounts_cache.set_max_flush_root(root);
6352        }
6353        let num_new_roots = flushed_roots.len();
6354        (num_new_roots, num_roots_flushed, flush_stats)
6355    }
6356
6357    fn do_flush_slot_cache(
6358        &self,
6359        slot: Slot,
6360        slot_cache: &SlotCache,
6361        mut should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
6362        max_clean_root: Option<Slot>,
6363    ) -> FlushStats {
6364        let mut flush_stats = FlushStats::default();
6365        let iter_items: Vec<_> = slot_cache.iter().collect();
6366        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
6367        let mut pubkey_to_slot_set: Vec<(Pubkey, Slot)> = vec![];
6368        if should_flush_f.is_some() {
6369            if let Some(max_clean_root) = max_clean_root {
6370                if slot > max_clean_root {
6371                    // Only if the root is greater than the `max_clean_root` do we
6372                    // have to prevent cleaning, otherwise, just default to `should_flush_f`
6373                    // for any slots <= `max_clean_root`
6374                    should_flush_f = None;
6375                }
6376            }
6377        }
6378
6379        let accounts: Vec<(&Pubkey, &AccountSharedData)> = iter_items
6380            .iter()
6381            .filter_map(|iter_item| {
6382                let key = iter_item.key();
6383                let account = &iter_item.value().account;
6384                let should_flush = should_flush_f
6385                    .as_mut()
6386                    .map(|should_flush_f| should_flush_f(key, account))
6387                    .unwrap_or(true);
6388                if should_flush {
6389                    flush_stats.total_size += aligned_stored_size(account.data().len()) as u64;
6390                    flush_stats.num_flushed += 1;
6391                    Some((key, account))
6392                } else {
6393                    // If we don't flush, we have to remove the entry from the
6394                    // index, since it's equivalent to purging
6395                    purged_slot_pubkeys.insert((slot, *key));
6396                    pubkey_to_slot_set.push((*key, slot));
6397                    flush_stats.num_purged += 1;
6398                    None
6399                }
6400            })
6401            .collect();
6402
6403        let is_dead_slot = accounts.is_empty();
6404        // Remove the account index entries from earlier roots that are outdated by later roots.
6405        // Safe because queries to the index will be reading updates from later roots.
6406        self.purge_slot_cache_pubkeys(
6407            slot,
6408            purged_slot_pubkeys,
6409            pubkey_to_slot_set,
6410            is_dead_slot,
6411            &HashSet::default(),
6412        );
6413
6414        if !is_dead_slot {
6415            // This ensures that all updates are written to an AppendVec, before any
6416            // updates to the index happen, so anybody that sees a real entry in the index,
6417            // will be able to find the account in storage
6418            let flushed_store =
6419                self.create_and_insert_store(slot, flush_stats.total_size.0, "flush_slot_cache");
6420            let (store_accounts_timing_inner, store_accounts_total_inner_us) =
6421                measure_us!(self.store_accounts_frozen((slot, &accounts[..]), &flushed_store,));
6422            flush_stats.store_accounts_timing = store_accounts_timing_inner;
6423            flush_stats.store_accounts_total_us = Saturating(store_accounts_total_inner_us);
6424
6425            // If the above sizing function is correct, just one AppendVec is enough to hold
6426            // all the data for the slot
6427            assert!(self.storage.get_slot_storage_entry(slot).is_some());
6428            self.reopen_storage_as_readonly_shrinking_in_progress_ok(slot);
6429        }
6430
6431        // Remove this slot from the cache, which will to AccountsDb's new readers should look like an
6432        // atomic switch from the cache to storage.
6433        // There is some racy condition for existing readers who just has read exactly while
6434        // flushing. That case is handled by retry_to_get_account_accessor()
6435        assert!(self.accounts_cache.remove_slot(slot).is_some());
6436
6437        // Add `accounts` to uncleaned_pubkeys since we know they were written
6438        // to a storage and should be visited by `clean`.
6439        self.uncleaned_pubkeys
6440            .entry(slot)
6441            .or_default()
6442            .extend(accounts.iter().map(|(pubkey, _account)| **pubkey));
6443
6444        flush_stats
6445    }
6446
6447    /// flush all accounts in this slot
6448    fn flush_slot_cache(&self, slot: Slot) -> Option<FlushStats> {
6449        self.flush_slot_cache_with_clean(slot, None::<&mut fn(&_, &_) -> bool>, None)
6450    }
6451
6452    /// `should_flush_f` is an optional closure that determines whether a given
6453    /// account should be flushed. Passing `None` will by default flush all
6454    /// accounts
6455    fn flush_slot_cache_with_clean(
6456        &self,
6457        slot: Slot,
6458        should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
6459        max_clean_root: Option<Slot>,
6460    ) -> Option<FlushStats> {
6461        if self
6462            .remove_unrooted_slots_synchronization
6463            .slots_under_contention
6464            .lock()
6465            .unwrap()
6466            .insert(slot)
6467        {
6468            // We have not seen this slot, flush it.
6469            let flush_stats = self.accounts_cache.slot_cache(slot).map(|slot_cache| {
6470                #[cfg(test)]
6471                {
6472                    // Give some time for cache flushing to occur here for unit tests
6473                    sleep(Duration::from_millis(self.load_delay));
6474                }
6475                // Since we added the slot to `slots_under_contention` AND this slot
6476                // still exists in the cache, we know the slot cannot be removed
6477                // by any other threads past this point. We are now responsible for
6478                // flushing this slot.
6479                self.do_flush_slot_cache(slot, &slot_cache, should_flush_f, max_clean_root)
6480            });
6481
6482            // Nobody else should have been purging this slot, so should not have been removed
6483            // from `self.remove_unrooted_slots_synchronization`.
6484            assert!(self
6485                .remove_unrooted_slots_synchronization
6486                .slots_under_contention
6487                .lock()
6488                .unwrap()
6489                .remove(&slot));
6490
6491            // Signal to any threads blocked on `remove_unrooted_slots(slot)` that we have finished
6492            // flushing
6493            self.remove_unrooted_slots_synchronization
6494                .signal
6495                .notify_all();
6496            flush_stats
6497        } else {
6498            // We have already seen this slot. It is already under flushing. Skip.
6499            None
6500        }
6501    }
6502
6503    fn write_accounts_to_cache<'a, 'b>(
6504        &self,
6505        slot: Slot,
6506        accounts_and_meta_to_store: &impl StorableAccounts<'b>,
6507        txs: Option<&[&SanitizedTransaction]>,
6508    ) -> Vec<AccountInfo> {
6509        let mut current_write_version = if self.accounts_update_notifier.is_some() {
6510            self.write_version
6511                .fetch_add(accounts_and_meta_to_store.len() as u64, Ordering::AcqRel)
6512        } else {
6513            0
6514        };
6515
6516        let (account_infos, cached_accounts) = (0..accounts_and_meta_to_store.len())
6517            .map(|index| {
6518                let txn = txs.map(|txs| *txs.get(index).expect("txs must be present if provided"));
6519                let mut account_info = AccountInfo::default();
6520                accounts_and_meta_to_store.account_default_if_zero_lamport(index, |account| {
6521                    let account_shared_data = account.to_account_shared_data();
6522                    let pubkey = account.pubkey();
6523                    account_info = AccountInfo::new(StorageLocation::Cached, account.lamports());
6524
6525                    self.notify_account_at_accounts_update(
6526                        slot,
6527                        &account_shared_data,
6528                        &txn,
6529                        pubkey,
6530                        current_write_version,
6531                    );
6532                    saturating_add_assign!(current_write_version, 1);
6533
6534                    let cached_account =
6535                        self.accounts_cache.store(slot, pubkey, account_shared_data);
6536                    (account_info, cached_account)
6537                })
6538            })
6539            .unzip();
6540
6541        // hash this accounts in bg
6542        if let Some(sender) = self.sender_bg_hasher.read().unwrap().as_ref() {
6543            let _ = sender.send(cached_accounts);
6544        };
6545
6546        account_infos
6547    }
6548
6549    fn store_accounts_to<'a: 'c, 'b, 'c>(
6550        &self,
6551        accounts: &'c impl StorableAccounts<'b>,
6552        store_to: &StoreTo,
6553        transactions: Option<&'a [&'a SanitizedTransaction]>,
6554    ) -> Vec<AccountInfo> {
6555        let mut calc_stored_meta_time = Measure::start("calc_stored_meta");
6556        let slot = accounts.target_slot();
6557        if self
6558            .read_only_accounts_cache
6559            .can_slot_be_in_cache(accounts.target_slot())
6560        {
6561            (0..accounts.len()).for_each(|index| {
6562                accounts.account(index, |account| {
6563                    // based on the patterns of how a validator writes accounts, it is almost always the case that there is no read only cache entry
6564                    // for this pubkey and slot. So, we can give that hint to the `remove` for performance.
6565                    self.read_only_accounts_cache
6566                        .remove_assume_not_present(*account.pubkey());
6567                })
6568            });
6569        }
6570        calc_stored_meta_time.stop();
6571        self.stats
6572            .calc_stored_meta
6573            .fetch_add(calc_stored_meta_time.as_us(), Ordering::Relaxed);
6574
6575        match store_to {
6576            StoreTo::Cache => self.write_accounts_to_cache(slot, accounts, transactions),
6577            StoreTo::Storage(storage) => self.write_accounts_to_storage(slot, storage, accounts),
6578        }
6579    }
6580
6581    fn report_store_stats(&self) {
6582        let mut total_count = 0;
6583        let mut newest_slot = 0;
6584        let mut oldest_slot = u64::MAX;
6585        let mut total_bytes = 0;
6586        let mut total_alive_bytes = 0;
6587        for (slot, store) in self.storage.iter() {
6588            total_count += 1;
6589            newest_slot = std::cmp::max(newest_slot, slot);
6590
6591            oldest_slot = std::cmp::min(oldest_slot, slot);
6592
6593            total_alive_bytes += store.alive_bytes();
6594            total_bytes += store.capacity();
6595        }
6596        info!(
6597            "total_stores: {total_count}, newest_slot: {newest_slot}, oldest_slot: {oldest_slot}"
6598        );
6599
6600        let total_alive_ratio = if total_bytes > 0 {
6601            total_alive_bytes as f64 / total_bytes as f64
6602        } else {
6603            0.
6604        };
6605
6606        datapoint_info!(
6607            "accounts_db-stores",
6608            ("total_count", total_count, i64),
6609            ("total_bytes", total_bytes, i64),
6610            ("total_alive_bytes", total_alive_bytes, i64),
6611            ("total_alive_ratio", total_alive_ratio, f64),
6612        );
6613        datapoint_info!(
6614            "accounts_db-perf-stats",
6615            (
6616                "delta_hash_num",
6617                self.stats.delta_hash_num.swap(0, Ordering::Relaxed),
6618                i64
6619            ),
6620            (
6621                "delta_hash_scan_us",
6622                self.stats
6623                    .delta_hash_scan_time_total_us
6624                    .swap(0, Ordering::Relaxed),
6625                i64
6626            ),
6627            (
6628                "delta_hash_accumulate_us",
6629                self.stats
6630                    .delta_hash_accumulate_time_total_us
6631                    .swap(0, Ordering::Relaxed),
6632                i64
6633            ),
6634            (
6635                "skipped_rewrites_num",
6636                self.stats.skipped_rewrites_num.swap(0, Ordering::Relaxed),
6637                i64
6638            ),
6639        );
6640    }
6641
6642    pub fn checked_sum_for_capitalization<T: Iterator<Item = u64>>(balances: T) -> u64 {
6643        AccountsHasher::checked_cast_for_capitalization(balances.map(|b| b as u128).sum::<u128>())
6644    }
6645
6646    pub fn calculate_accounts_hash_from_index(
6647        &self,
6648        max_slot: Slot,
6649        config: &CalcAccountsHashConfig<'_>,
6650    ) -> (AccountsHash, u64) {
6651        let mut collect = Measure::start("collect");
6652        let keys: Vec<_> = self
6653            .accounts_index
6654            .account_maps
6655            .iter()
6656            .flat_map(|map| {
6657                let mut keys = map.keys();
6658                keys.sort_unstable(); // hashmap is not ordered, but bins are relative to each other
6659                keys
6660            })
6661            .collect();
6662        collect.stop();
6663
6664        // Pick a chunk size big enough to allow us to produce output vectors that are smaller than the overall size.
6665        // We'll also accumulate the lamports within each chunk and fewer chunks results in less contention to accumulate the sum.
6666        let chunks = crate::accounts_hash::MERKLE_FANOUT.pow(4);
6667        let total_lamports = Mutex::<u64>::new(0);
6668
6669        let get_account_hashes = || {
6670            keys.par_chunks(chunks)
6671                .map(|pubkeys| {
6672                    let mut sum = 0u128;
6673                    let account_hashes: Vec<Hash> = pubkeys
6674                        .iter()
6675                        .filter_map(|pubkey| {
6676                            let index_entry = self.accounts_index.get_cloned(pubkey)?;
6677                            self.accounts_index
6678                                .get_account_info_with_and_then(
6679                                    &index_entry,
6680                                    config.ancestors,
6681                                    Some(max_slot),
6682                                    |(slot, account_info)| {
6683                                        if account_info.is_zero_lamport() {
6684                                            return None;
6685                                        }
6686                                        self.get_account_accessor(
6687                                            slot,
6688                                            pubkey,
6689                                            &account_info.storage_location(),
6690                                        )
6691                                        .get_loaded_account(|loaded_account| {
6692                                            let mut loaded_hash = loaded_account.loaded_hash();
6693                                            let balance = loaded_account.lamports();
6694                                            let hash_is_missing =
6695                                                loaded_hash == AccountHash(Hash::default());
6696                                            if hash_is_missing {
6697                                                let computed_hash = Self::hash_account(
6698                                                    &loaded_account,
6699                                                    loaded_account.pubkey(),
6700                                                );
6701                                                loaded_hash = computed_hash;
6702                                            }
6703                                            sum += balance as u128;
6704                                            loaded_hash.0
6705                                        })
6706                                    },
6707                                )
6708                                .flatten()
6709                        })
6710                        .collect();
6711                    let mut total = total_lamports.lock().unwrap();
6712                    *total = AccountsHasher::checked_cast_for_capitalization(*total as u128 + sum);
6713                    account_hashes
6714                })
6715                .collect()
6716        };
6717
6718        let mut scan = Measure::start("scan");
6719        let account_hashes: Vec<Vec<Hash>> = self.thread_pool_clean.install(get_account_hashes);
6720        scan.stop();
6721
6722        let total_lamports = *total_lamports.lock().unwrap();
6723
6724        let mut hash_time = Measure::start("hash");
6725        let (accumulated_hash, hash_total) = AccountsHasher::calculate_hash(account_hashes);
6726        hash_time.stop();
6727
6728        datapoint_info!(
6729            "calculate_accounts_hash_from_index",
6730            ("accounts_scan", scan.as_us(), i64),
6731            ("hash", hash_time.as_us(), i64),
6732            ("hash_total", hash_total, i64),
6733            ("collect", collect.as_us(), i64),
6734        );
6735
6736        let accounts_hash = AccountsHash(accumulated_hash);
6737        (accounts_hash, total_lamports)
6738    }
6739
6740    /// Calculates the accounts lt hash
6741    ///
6742    /// Only intended to be called at startup (or by tests).
6743    /// Only intended to be used while testing the experimental accumulator hash.
6744    pub fn calculate_accounts_lt_hash_at_startup_from_index(
6745        &self,
6746        ancestors: &Ancestors,
6747        startup_slot: Slot,
6748    ) -> AccountsLtHash {
6749        // This impl iterates over all the index bins in parallel, and computes the lt hash
6750        // sequentially per bin.  Then afterwards reduces to a single lt hash.
6751        // This implementation is quite fast.  Runtime is about 150 seconds on mnb as of 10/2/2024.
6752        // The sequential implementation took about 6,275 seconds!
6753        // A different parallel implementation that iterated over the bins *sequentially* and then
6754        // hashed the accounts *within* a bin in parallel took about 600 seconds.  That impl uses
6755        // less memory, as only a single index bin is loaded into mem at a time.
6756        let lt_hash = self
6757            .accounts_index
6758            .account_maps
6759            .par_iter()
6760            .fold(
6761                LtHash::identity,
6762                |mut accumulator_lt_hash, accounts_index_bin| {
6763                    for pubkey in accounts_index_bin.keys() {
6764                        let account_lt_hash = self
6765                            .accounts_index
6766                            .get_with_and_then(
6767                                &pubkey,
6768                                Some(ancestors),
6769                                Some(startup_slot),
6770                                false,
6771                                |(slot, account_info)| {
6772                                    (!account_info.is_zero_lamport()).then(|| {
6773                                        self.get_account_accessor(
6774                                            slot,
6775                                            &pubkey,
6776                                            &account_info.storage_location(),
6777                                        )
6778                                        .get_loaded_account(|loaded_account| {
6779                                            Self::lt_hash_account(&loaded_account, &pubkey)
6780                                        })
6781                                        // SAFETY: The index said this pubkey exists, so
6782                                        // there must be an account to load.
6783                                        .unwrap()
6784                                    })
6785                                },
6786                            )
6787                            .flatten();
6788                        if let Some(account_lt_hash) = account_lt_hash {
6789                            accumulator_lt_hash.mix_in(&account_lt_hash.0);
6790                        }
6791                    }
6792                    accumulator_lt_hash
6793                },
6794            )
6795            .reduce(LtHash::identity, |mut accum, elem| {
6796                accum.mix_in(&elem);
6797                accum
6798            });
6799
6800        AccountsLtHash(lt_hash)
6801    }
6802
6803    /// Calculates the accounts lt hash
6804    ///
6805    /// Intended to be used to verify the accounts lt hash at startup.
6806    ///
6807    /// The `duplicates_lt_hash` is the old/duplicate accounts to mix *out* of the storages.
6808    /// This value comes from index generation.
6809    pub fn calculate_accounts_lt_hash_at_startup_from_storages(
6810        &self,
6811        storages: &[Arc<AccountStorageEntry>],
6812        duplicates_lt_hash: &DuplicatesLtHash,
6813    ) -> AccountsLtHash {
6814        let mut lt_hash = storages
6815            .par_iter()
6816            .fold(LtHash::identity, |mut accum, storage| {
6817                storage.accounts.scan_accounts(|stored_account_meta| {
6818                    let account_lt_hash =
6819                        Self::lt_hash_account(&stored_account_meta, stored_account_meta.pubkey());
6820                    accum.mix_in(&account_lt_hash.0);
6821                });
6822                accum
6823            })
6824            .reduce(LtHash::identity, |mut accum, elem| {
6825                accum.mix_in(&elem);
6826                accum
6827            });
6828
6829        lt_hash.mix_out(&duplicates_lt_hash.0);
6830
6831        AccountsLtHash(lt_hash)
6832    }
6833
6834    /// This is only valid to call from tests.
6835    /// run the accounts hash calculation and store the results
6836    pub fn update_accounts_hash_for_tests(
6837        &self,
6838        slot: Slot,
6839        ancestors: &Ancestors,
6840        debug_verify: bool,
6841        is_startup: bool,
6842    ) -> (AccountsHash, u64) {
6843        self.update_accounts_hash_with_verify_from(
6844            CalcAccountsHashDataSource::IndexForTests,
6845            debug_verify,
6846            slot,
6847            ancestors,
6848            None,
6849            &EpochSchedule::default(),
6850            &RentCollector::default(),
6851            is_startup,
6852        )
6853    }
6854
6855    fn update_old_slot_stats(&self, stats: &HashStats, storage: Option<&Arc<AccountStorageEntry>>) {
6856        if let Some(storage) = storage {
6857            stats.roots_older_than_epoch.fetch_add(1, Ordering::Relaxed);
6858            let num_accounts = storage.count();
6859            let sizes = storage.capacity();
6860            stats
6861                .append_vec_sizes_older_than_epoch
6862                .fetch_add(sizes as usize, Ordering::Relaxed);
6863            stats
6864                .accounts_in_roots_older_than_epoch
6865                .fetch_add(num_accounts, Ordering::Relaxed);
6866        }
6867    }
6868
6869    /// return slot + offset, where offset can be +/-
6870    fn apply_offset_to_slot(slot: Slot, offset: i64) -> Slot {
6871        if offset > 0 {
6872            slot.saturating_add(offset as u64)
6873        } else {
6874            slot.saturating_sub(offset.unsigned_abs())
6875        }
6876    }
6877
6878    /// `oldest_non_ancient_slot` is only applicable when `Append` is used for ancient append vec packing.
6879    /// If `Pack` is used for ancient append vec packing, return None.
6880    /// Otherwise, return a slot 'max_slot_inclusive' - (slots_per_epoch - `self.ancient_append_vec_offset`)
6881    /// If ancient append vecs are not enabled, return 0.
6882    fn get_oldest_non_ancient_slot_for_hash_calc_scan(
6883        &self,
6884        max_slot_inclusive: Slot,
6885        config: &CalcAccountsHashConfig<'_>,
6886    ) -> Option<Slot> {
6887        if self.create_ancient_storage == CreateAncientStorage::Pack {
6888            // oldest_non_ancient_slot is only applicable when ancient storages are created with `Append`. When ancient storages are created with `Pack`, ancient storages
6889            // can be created in between non-ancient storages. Return None, because oldest_non_ancient_slot is not applicable here.
6890            None
6891        } else if self.ancient_append_vec_offset.is_some() {
6892            // For performance, this is required when ancient appendvecs are enabled
6893            Some(
6894                self.get_oldest_non_ancient_slot_from_slot(
6895                    config.epoch_schedule,
6896                    max_slot_inclusive,
6897                ),
6898            )
6899        } else {
6900            // This causes the entire range to be chunked together, treating older append vecs just like new ones.
6901            // This performs well if there are many old append vecs that haven't been cleaned yet.
6902            // 0 will have the effect of causing ALL older append vecs to be chunked together, just like every other append vec.
6903            Some(0)
6904        }
6905    }
6906
6907    /// hash info about 'storage' into 'hasher'
6908    /// return true iff storage is valid for loading from cache
6909    fn hash_storage_info(
6910        hasher: &mut impl StdHasher,
6911        storage: &AccountStorageEntry,
6912        slot: Slot,
6913    ) -> bool {
6914        // hash info about this storage
6915        storage.written_bytes().hash(hasher);
6916        slot.hash(hasher);
6917        let storage_file = storage.accounts.path();
6918        storage_file.hash(hasher);
6919        let Ok(metadata) = std::fs::metadata(storage_file) else {
6920            return false;
6921        };
6922        let Ok(amod) = metadata.modified() else {
6923            return false;
6924        };
6925        let amod = amod
6926            .duration_since(std::time::UNIX_EPOCH)
6927            .unwrap()
6928            .as_secs();
6929        amod.hash(hasher);
6930
6931        // if we made it here, we have hashed info and we should try to load from the cache
6932        true
6933    }
6934
6935    pub fn calculate_accounts_hash_from(
6936        &self,
6937        data_source: CalcAccountsHashDataSource,
6938        slot: Slot,
6939        config: &CalcAccountsHashConfig<'_>,
6940    ) -> (AccountsHash, u64) {
6941        match data_source {
6942            CalcAccountsHashDataSource::Storages => {
6943                if self.accounts_cache.contains_any_slots(slot) {
6944                    // this indicates a race condition
6945                    inc_new_counter_info!("accounts_hash_items_in_write_cache", 1);
6946                }
6947
6948                let mut collect_time = Measure::start("collect");
6949                let (combined_maps, slots) = self.get_storages(..=slot);
6950                collect_time.stop();
6951
6952                let mut sort_time = Measure::start("sort_storages");
6953                let min_root = self.accounts_index.min_alive_root();
6954                let storages = SortedStorages::new_with_slots(
6955                    combined_maps.iter().zip(slots),
6956                    min_root,
6957                    Some(slot),
6958                );
6959                sort_time.stop();
6960
6961                let mut timings = HashStats {
6962                    collect_snapshots_us: collect_time.as_us(),
6963                    storage_sort_us: sort_time.as_us(),
6964                    ..HashStats::default()
6965                };
6966                timings.calc_storage_size_quartiles(&combined_maps);
6967
6968                self.calculate_accounts_hash(config, &storages, timings)
6969            }
6970            CalcAccountsHashDataSource::IndexForTests => {
6971                self.calculate_accounts_hash_from_index(slot, config)
6972            }
6973        }
6974    }
6975
6976    fn calculate_accounts_hash_with_verify_from(
6977        &self,
6978        data_source: CalcAccountsHashDataSource,
6979        debug_verify: bool,
6980        slot: Slot,
6981        config: CalcAccountsHashConfig<'_>,
6982        expected_capitalization: Option<u64>,
6983    ) -> (AccountsHash, u64) {
6984        let (accounts_hash, total_lamports) =
6985            self.calculate_accounts_hash_from(data_source, slot, &config);
6986        if debug_verify {
6987            // calculate the other way (store or non-store) and verify results match.
6988            let data_source_other = match data_source {
6989                CalcAccountsHashDataSource::IndexForTests => CalcAccountsHashDataSource::Storages,
6990                CalcAccountsHashDataSource::Storages => CalcAccountsHashDataSource::IndexForTests,
6991            };
6992            let (accounts_hash_other, total_lamports_other) =
6993                self.calculate_accounts_hash_from(data_source_other, slot, &config);
6994
6995            let success = accounts_hash == accounts_hash_other
6996                && total_lamports == total_lamports_other
6997                && total_lamports == expected_capitalization.unwrap_or(total_lamports);
6998            assert!(
6999                success,
7000                "calculate_accounts_hash_with_verify mismatch. hashes: {}, {}; lamports: {}, {}; \
7001                 expected lamports: {:?}, data source: {:?}, slot: {}",
7002                accounts_hash.0,
7003                accounts_hash_other.0,
7004                total_lamports,
7005                total_lamports_other,
7006                expected_capitalization,
7007                data_source,
7008                slot
7009            );
7010        }
7011        (accounts_hash, total_lamports)
7012    }
7013
7014    /// run the accounts hash calculation and store the results
7015    #[allow(clippy::too_many_arguments)]
7016    pub fn update_accounts_hash_with_verify_from(
7017        &self,
7018        data_source: CalcAccountsHashDataSource,
7019        debug_verify: bool,
7020        slot: Slot,
7021        ancestors: &Ancestors,
7022        expected_capitalization: Option<u64>,
7023        epoch_schedule: &EpochSchedule,
7024        rent_collector: &RentCollector,
7025        is_startup: bool,
7026    ) -> (AccountsHash, u64) {
7027        let (accounts_hash, total_lamports) = self.calculate_accounts_hash_with_verify_from(
7028            data_source,
7029            debug_verify,
7030            slot,
7031            CalcAccountsHashConfig {
7032                use_bg_thread_pool: !is_startup,
7033                ancestors: Some(ancestors),
7034                epoch_schedule,
7035                rent_collector,
7036                store_detailed_debug_info_on_failure: false,
7037            },
7038            expected_capitalization,
7039        );
7040        self.set_accounts_hash(slot, (accounts_hash, total_lamports));
7041        (accounts_hash, total_lamports)
7042    }
7043
7044    /// Calculate the full accounts hash for `storages` and save the results at `slot`
7045    pub fn update_accounts_hash(
7046        &self,
7047        config: &CalcAccountsHashConfig<'_>,
7048        storages: &SortedStorages<'_>,
7049        slot: Slot,
7050        stats: HashStats,
7051    ) -> (AccountsHash, /*capitalization*/ u64) {
7052        let accounts_hash = self.calculate_accounts_hash(config, storages, stats);
7053        let old_accounts_hash = self.set_accounts_hash(slot, accounts_hash);
7054        if let Some(old_accounts_hash) = old_accounts_hash {
7055            warn!(
7056                "Accounts hash was already set for slot {slot}! old: {old_accounts_hash:?}, new: \
7057                 {accounts_hash:?}"
7058            );
7059        }
7060        accounts_hash
7061    }
7062
7063    /// Calculate the incremental accounts hash for `storages` and save the results at `slot`
7064    pub fn update_incremental_accounts_hash(
7065        &self,
7066        config: &CalcAccountsHashConfig<'_>,
7067        storages: &SortedStorages<'_>,
7068        slot: Slot,
7069        stats: HashStats,
7070    ) -> (IncrementalAccountsHash, /*capitalization*/ u64) {
7071        let incremental_accounts_hash =
7072            self.calculate_incremental_accounts_hash(config, storages, stats);
7073        let old_incremental_accounts_hash =
7074            self.set_incremental_accounts_hash(slot, incremental_accounts_hash);
7075        if let Some(old_incremental_accounts_hash) = old_incremental_accounts_hash {
7076            warn!(
7077                "Incremental accounts hash was already set for slot {slot}! old: \
7078                 {old_incremental_accounts_hash:?}, new: {incremental_accounts_hash:?}"
7079            );
7080        }
7081        incremental_accounts_hash
7082    }
7083
7084    /// Set the accounts hash for `slot`
7085    ///
7086    /// returns the previous accounts hash for `slot`
7087    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
7088    fn set_accounts_hash(
7089        &self,
7090        slot: Slot,
7091        accounts_hash: (AccountsHash, /*capitalization*/ u64),
7092    ) -> Option<(AccountsHash, /*capitalization*/ u64)> {
7093        self.accounts_hashes
7094            .lock()
7095            .unwrap()
7096            .insert(slot, accounts_hash)
7097    }
7098
7099    /// After deserializing a snapshot, set the accounts hash for the new AccountsDb
7100    pub fn set_accounts_hash_from_snapshot(
7101        &mut self,
7102        slot: Slot,
7103        accounts_hash: SerdeAccountsHash,
7104        capitalization: u64,
7105    ) -> Option<(AccountsHash, /*capitalization*/ u64)> {
7106        self.set_accounts_hash(slot, (accounts_hash.into(), capitalization))
7107    }
7108
7109    /// Get the accounts hash for `slot`
7110    pub fn get_accounts_hash(&self, slot: Slot) -> Option<(AccountsHash, /*capitalization*/ u64)> {
7111        self.accounts_hashes.lock().unwrap().get(&slot).cloned()
7112    }
7113
7114    /// Get all accounts hashes
7115    pub fn get_accounts_hashes(&self) -> HashMap<Slot, (AccountsHash, /*capitalization*/ u64)> {
7116        self.accounts_hashes.lock().unwrap().clone()
7117    }
7118
7119    /// Set the incremental accounts hash for `slot`
7120    ///
7121    /// returns the previous incremental accounts hash for `slot`
7122    pub fn set_incremental_accounts_hash(
7123        &self,
7124        slot: Slot,
7125        incremental_accounts_hash: (IncrementalAccountsHash, /*capitalization*/ u64),
7126    ) -> Option<(IncrementalAccountsHash, /*capitalization*/ u64)> {
7127        self.incremental_accounts_hashes
7128            .lock()
7129            .unwrap()
7130            .insert(slot, incremental_accounts_hash)
7131    }
7132
7133    /// After deserializing a snapshot, set the incremental accounts hash for the new AccountsDb
7134    pub fn set_incremental_accounts_hash_from_snapshot(
7135        &mut self,
7136        slot: Slot,
7137        incremental_accounts_hash: SerdeIncrementalAccountsHash,
7138        capitalization: u64,
7139    ) -> Option<(IncrementalAccountsHash, /*capitalization*/ u64)> {
7140        self.set_incremental_accounts_hash(slot, (incremental_accounts_hash.into(), capitalization))
7141    }
7142
7143    /// Get the incremental accounts hash for `slot`
7144    pub fn get_incremental_accounts_hash(
7145        &self,
7146        slot: Slot,
7147    ) -> Option<(IncrementalAccountsHash, /*capitalization*/ u64)> {
7148        self.incremental_accounts_hashes
7149            .lock()
7150            .unwrap()
7151            .get(&slot)
7152            .cloned()
7153    }
7154
7155    /// Get all incremental accounts hashes
7156    pub fn get_incremental_accounts_hashes(
7157        &self,
7158    ) -> HashMap<Slot, (IncrementalAccountsHash, /*capitalization*/ u64)> {
7159        self.incremental_accounts_hashes.lock().unwrap().clone()
7160    }
7161
7162    /// Purge accounts hashes that are older than `latest_full_snapshot_slot`
7163    ///
7164    /// Should only be called by AccountsHashVerifier, since it consumes the accounts hashes and
7165    /// knows which ones are still needed.
7166    pub fn purge_old_accounts_hashes(&self, latest_full_snapshot_slot: Slot) {
7167        self.accounts_hashes
7168            .lock()
7169            .unwrap()
7170            .retain(|&slot, _| slot >= latest_full_snapshot_slot);
7171        self.incremental_accounts_hashes
7172            .lock()
7173            .unwrap()
7174            .retain(|&slot, _| slot >= latest_full_snapshot_slot);
7175    }
7176
7177    fn sort_slot_storage_scan(accum: &mut BinnedHashData) -> u64 {
7178        let (_, sort_time) = measure_us!(accum.iter_mut().for_each(|items| {
7179            // sort_by vs unstable because slot and write_version are already in order
7180            items.sort_by(AccountsHasher::compare_two_hash_entries);
7181        }));
7182        sort_time
7183    }
7184
7185    /// normal code path returns the common cache path
7186    /// when called after a failure has been detected, redirect the cache storage to a separate folder for debugging later
7187    fn get_cache_hash_data(
7188        accounts_hash_cache_path: PathBuf,
7189        config: &CalcAccountsHashConfig<'_>,
7190        kind: CalcAccountsHashKind,
7191        slot: Slot,
7192        storages_start_slot: Slot,
7193    ) -> CacheHashData {
7194        let accounts_hash_cache_path = if !config.store_detailed_debug_info_on_failure {
7195            accounts_hash_cache_path
7196        } else {
7197            // this path executes when we are failing with a hash mismatch
7198            let failed_dir = accounts_hash_cache_path
7199                .join("failed_calculate_accounts_hash_cache")
7200                .join(slot.to_string());
7201            _ = std::fs::remove_dir_all(&failed_dir);
7202            failed_dir
7203        };
7204        let deletion_policy = match kind {
7205            CalcAccountsHashKind::Full => CacheHashDeletionPolicy::AllUnused,
7206            CalcAccountsHashKind::Incremental => {
7207                CacheHashDeletionPolicy::UnusedAtLeast(storages_start_slot)
7208            }
7209        };
7210        CacheHashData::new(accounts_hash_cache_path, deletion_policy)
7211    }
7212
7213    /// Calculate the full accounts hash
7214    ///
7215    /// This is intended to be used by startup verification, and also AccountsHashVerifier.
7216    /// Uses account storage files as the data source for the calculation.
7217    pub fn calculate_accounts_hash(
7218        &self,
7219        config: &CalcAccountsHashConfig<'_>,
7220        storages: &SortedStorages<'_>,
7221        stats: HashStats,
7222    ) -> (AccountsHash, u64) {
7223        let (accounts_hash, capitalization) = self.calculate_accounts_hash_from_storages(
7224            config,
7225            storages,
7226            stats,
7227            CalcAccountsHashKind::Full,
7228        );
7229        let AccountsHashKind::Full(accounts_hash) = accounts_hash else {
7230            panic!("calculate_accounts_hash_from_storages must return a FullAccountsHash");
7231        };
7232        (accounts_hash, capitalization)
7233    }
7234
7235    /// Calculate the incremental accounts hash
7236    ///
7237    /// This calculation is intended to be used by incremental snapshots, and thus differs from a
7238    /// "full" accounts hash in a few ways:
7239    /// - Zero-lamport accounts are *included* in the hash because zero-lamport accounts are also
7240    ///   included in the incremental snapshot.  This ensures reconstructing the AccountsDb is
7241    ///   still correct when using this incremental accounts hash.
7242    /// - `storages` must be the same as the ones going into the incremental snapshot.
7243    pub fn calculate_incremental_accounts_hash(
7244        &self,
7245        config: &CalcAccountsHashConfig<'_>,
7246        storages: &SortedStorages<'_>,
7247        stats: HashStats,
7248    ) -> (IncrementalAccountsHash, /* capitalization */ u64) {
7249        let (accounts_hash, capitalization) = self.calculate_accounts_hash_from_storages(
7250            config,
7251            storages,
7252            stats,
7253            CalcAccountsHashKind::Incremental,
7254        );
7255        let AccountsHashKind::Incremental(incremental_accounts_hash) = accounts_hash else {
7256            panic!("calculate_incremental_accounts_hash must return an IncrementalAccountsHash");
7257        };
7258        (incremental_accounts_hash, capitalization)
7259    }
7260
7261    /// The shared code for calculating accounts hash from storages.
7262    /// Used for both full accounts hash and incremental accounts hash calculation.
7263    fn calculate_accounts_hash_from_storages(
7264        &self,
7265        config: &CalcAccountsHashConfig<'_>,
7266        storages: &SortedStorages<'_>,
7267        mut stats: HashStats,
7268        kind: CalcAccountsHashKind,
7269    ) -> (AccountsHashKind, u64) {
7270        let total_time = Measure::start("");
7271        let _guard = self.active_stats.activate(ActiveStatItem::Hash);
7272        let storages_start_slot = storages.range().start;
7273        stats.oldest_root = storages_start_slot;
7274
7275        let slot = storages.max_slot_inclusive();
7276        let use_bg_thread_pool = config.use_bg_thread_pool;
7277        let accounts_hash_cache_path = self.accounts_hash_cache_path.clone();
7278        let transient_accounts_hash_cache_dir = TempDir::new_in(&accounts_hash_cache_path)
7279            .expect("create transient accounts hash cache dir");
7280        let transient_accounts_hash_cache_path =
7281            transient_accounts_hash_cache_dir.path().to_path_buf();
7282        let scan_and_hash = || {
7283            let (cache_hash_data, cache_hash_data_us) = measure_us!(Self::get_cache_hash_data(
7284                accounts_hash_cache_path,
7285                config,
7286                kind,
7287                slot,
7288                storages_start_slot,
7289            ));
7290            stats.cache_hash_data_us += cache_hash_data_us;
7291
7292            let bounds = Range {
7293                start: 0,
7294                end: self.hash_calculation_pubkey_bins,
7295            };
7296
7297            let accounts_hasher = AccountsHasher {
7298                zero_lamport_accounts: kind.zero_lamport_accounts(),
7299                dir_for_temp_cache_files: transient_accounts_hash_cache_path,
7300                active_stats: &self.active_stats,
7301            };
7302
7303            // get raw data by scanning
7304            let cache_hash_data_file_references = self.scan_snapshot_stores_with_cache(
7305                &cache_hash_data,
7306                storages,
7307                &mut stats,
7308                self.hash_calculation_pubkey_bins,
7309                &bounds,
7310                config,
7311            );
7312
7313            let cache_hash_data_files = cache_hash_data_file_references
7314                .iter()
7315                .map(|d| d.map())
7316                .collect::<Vec<_>>();
7317
7318            if let Some(err) = cache_hash_data_files
7319                .iter()
7320                .filter_map(|r| r.as_ref().err())
7321                .next()
7322            {
7323                panic!("failed generating accounts hash files: {:?}", err);
7324            }
7325
7326            // convert mmapped cache files into slices of data
7327            let cache_hash_intermediates = cache_hash_data_files
7328                .iter()
7329                .map(|d| d.as_ref().unwrap().get_cache_hash_data())
7330                .collect::<Vec<_>>();
7331
7332            // turn raw data into merkle tree hashes and sum of lamports
7333            let (accounts_hash, capitalization) = accounts_hasher.rest_of_hash_calculation(
7334                &cache_hash_intermediates,
7335                self.hash_calculation_pubkey_bins,
7336                &mut stats,
7337            );
7338            let accounts_hash = match kind {
7339                CalcAccountsHashKind::Full => AccountsHashKind::Full(AccountsHash(accounts_hash)),
7340                CalcAccountsHashKind::Incremental => {
7341                    AccountsHashKind::Incremental(IncrementalAccountsHash(accounts_hash))
7342                }
7343            };
7344            info!(
7345                "calculate_accounts_hash_from_storages: slot: {slot}, {accounts_hash:?}, \
7346                 capitalization: {capitalization}"
7347            );
7348            (accounts_hash, capitalization)
7349        };
7350
7351        let result = if use_bg_thread_pool {
7352            self.thread_pool_hash.install(scan_and_hash)
7353        } else {
7354            scan_and_hash()
7355        };
7356        stats.total_us = total_time.end_as_us();
7357        stats.log();
7358        result
7359    }
7360
7361    /// Verify accounts hash at startup (or tests)
7362    ///
7363    /// Calculate accounts hash(es) and compare them to the values set at startup.
7364    /// If `base` is `None`, only calculates the full accounts hash for `[0, slot]`.
7365    /// If `base` is `Some`, calculate the full accounts hash for `[0, base slot]`
7366    /// and then calculate the incremental accounts hash for `(base slot, slot]`.
7367    pub fn verify_accounts_hash_and_lamports(
7368        &self,
7369        snapshot_storages_and_slots: (&[Arc<AccountStorageEntry>], &[Slot]),
7370        slot: Slot,
7371        total_lamports: u64,
7372        base: Option<(Slot, /*capitalization*/ u64)>,
7373        config: VerifyAccountsHashAndLamportsConfig,
7374    ) -> Result<(), AccountsHashVerificationError> {
7375        let calc_config = CalcAccountsHashConfig {
7376            use_bg_thread_pool: config.use_bg_thread_pool,
7377            ancestors: Some(config.ancestors),
7378            epoch_schedule: config.epoch_schedule,
7379            rent_collector: config.rent_collector,
7380            store_detailed_debug_info_on_failure: config.store_detailed_debug_info,
7381        };
7382        let hash_mismatch_is_error = !config.ignore_mismatch;
7383
7384        if let Some((base_slot, base_capitalization)) = base {
7385            self.verify_accounts_hash_and_lamports(
7386                snapshot_storages_and_slots,
7387                base_slot,
7388                base_capitalization,
7389                None,
7390                config,
7391            )?;
7392
7393            let storages_and_slots = snapshot_storages_and_slots
7394                .0
7395                .iter()
7396                .zip(snapshot_storages_and_slots.1.iter())
7397                .filter(|storage_and_slot| *storage_and_slot.1 > base_slot)
7398                .map(|(storage, slot)| (storage, *slot));
7399            let sorted_storages = SortedStorages::new_with_slots(storages_and_slots, None, None);
7400            let calculated_incremental_accounts_hash = self.calculate_incremental_accounts_hash(
7401                &calc_config,
7402                &sorted_storages,
7403                HashStats::default(),
7404            );
7405            let found_incremental_accounts_hash = self
7406                .get_incremental_accounts_hash(slot)
7407                .ok_or(AccountsHashVerificationError::MissingAccountsHash)?;
7408            if calculated_incremental_accounts_hash != found_incremental_accounts_hash {
7409                warn!(
7410                    "mismatched incremental accounts hash for slot {slot}: \
7411                     {calculated_incremental_accounts_hash:?} (calculated) != \
7412                     {found_incremental_accounts_hash:?} (expected)"
7413                );
7414                if hash_mismatch_is_error {
7415                    return Err(AccountsHashVerificationError::MismatchedAccountsHash);
7416                }
7417            }
7418        } else {
7419            let storages_and_slots = snapshot_storages_and_slots
7420                .0
7421                .iter()
7422                .zip(snapshot_storages_and_slots.1.iter())
7423                .filter(|storage_and_slot| *storage_and_slot.1 <= slot)
7424                .map(|(storage, slot)| (storage, *slot));
7425            let sorted_storages = SortedStorages::new_with_slots(storages_and_slots, None, None);
7426            let (calculated_accounts_hash, calculated_lamports) =
7427                self.calculate_accounts_hash(&calc_config, &sorted_storages, HashStats::default());
7428            if calculated_lamports != total_lamports {
7429                warn!(
7430                    "Mismatched total lamports: {} calculated: {}",
7431                    total_lamports, calculated_lamports
7432                );
7433                return Err(AccountsHashVerificationError::MismatchedTotalLamports(
7434                    calculated_lamports,
7435                    total_lamports,
7436                ));
7437            }
7438            let (found_accounts_hash, _) = self
7439                .get_accounts_hash(slot)
7440                .ok_or(AccountsHashVerificationError::MissingAccountsHash)?;
7441            if calculated_accounts_hash != found_accounts_hash {
7442                warn!(
7443                    "Mismatched accounts hash for slot {slot}: {calculated_accounts_hash:?} \
7444                     (calculated) != {found_accounts_hash:?} (expected)"
7445                );
7446                if hash_mismatch_is_error {
7447                    return Err(AccountsHashVerificationError::MismatchedAccountsHash);
7448                }
7449            }
7450        }
7451
7452        Ok(())
7453    }
7454
7455    /// Returns all of the accounts' pubkeys for a given slot
7456    pub fn get_pubkeys_for_slot(&self, slot: Slot) -> Vec<Pubkey> {
7457        let scan_result = self.scan_account_storage(
7458            slot,
7459            |loaded_account| Some(*loaded_account.pubkey()),
7460            |accum: &DashSet<_>, loaded_account, _data| {
7461                accum.insert(*loaded_account.pubkey());
7462            },
7463            ScanAccountStorageData::NoData,
7464        );
7465        match scan_result {
7466            ScanStorageResult::Cached(cached_result) => cached_result,
7467            ScanStorageResult::Stored(stored_result) => stored_result.into_iter().collect(),
7468        }
7469    }
7470    /// helper to return
7471    /// 1. pubkey, hash pairs for the slot
7472    /// 2. us spent scanning
7473    /// 3. Measure started when we began accumulating
7474    pub fn get_pubkey_hash_for_slot(
7475        &self,
7476        slot: Slot,
7477    ) -> (Vec<(Pubkey, AccountHash)>, u64, Measure) {
7478        let mut scan = Measure::start("scan");
7479        let scan_result: ScanStorageResult<(Pubkey, AccountHash), DashMap<Pubkey, AccountHash>> =
7480            self.scan_account_storage(
7481                slot,
7482                |loaded_account: &LoadedAccount| {
7483                    // Cache only has one version per key, don't need to worry about versioning
7484                    Some((*loaded_account.pubkey(), loaded_account.loaded_hash()))
7485                },
7486                |accum: &DashMap<Pubkey, AccountHash>, loaded_account: &LoadedAccount, _data| {
7487                    let mut loaded_hash = loaded_account.loaded_hash();
7488                    if loaded_hash == AccountHash(Hash::default()) {
7489                        loaded_hash = Self::hash_account(loaded_account, loaded_account.pubkey())
7490                    }
7491                    accum.insert(*loaded_account.pubkey(), loaded_hash);
7492                },
7493                ScanAccountStorageData::NoData,
7494            );
7495        scan.stop();
7496
7497        let accumulate = Measure::start("accumulate");
7498        let hashes: Vec<_> = match scan_result {
7499            ScanStorageResult::Cached(cached_result) => cached_result,
7500            ScanStorageResult::Stored(stored_result) => stored_result.into_iter().collect(),
7501        };
7502
7503        (hashes, scan.as_us(), accumulate)
7504    }
7505
7506    /// Return all of the accounts for a given slot
7507    pub fn get_pubkey_account_for_slot(&self, slot: Slot) -> Vec<(Pubkey, AccountSharedData)> {
7508        let scan_result = self.scan_account_storage(
7509            slot,
7510            |loaded_account| {
7511                // Cache only has one version per key, don't need to worry about versioning
7512                Some((*loaded_account.pubkey(), loaded_account.take_account()))
7513            },
7514            |accum: &DashMap<_, _>, loaded_account, _data| {
7515                // Storage may have duplicates so only keep the latest version for each key
7516                accum.insert(*loaded_account.pubkey(), loaded_account.take_account());
7517            },
7518            ScanAccountStorageData::NoData,
7519        );
7520
7521        match scan_result {
7522            ScanStorageResult::Cached(cached_result) => cached_result,
7523            ScanStorageResult::Stored(stored_result) => stored_result.into_iter().collect(),
7524        }
7525    }
7526
7527    /// Return all of the accounts for a given slot
7528    pub fn get_pubkey_hash_account_for_slot(&self, slot: Slot) -> Vec<PubkeyHashAccount> {
7529        type ScanResult =
7530            ScanStorageResult<PubkeyHashAccount, DashMap<Pubkey, (AccountHash, AccountSharedData)>>;
7531        let scan_result: ScanResult = self.scan_account_storage(
7532            slot,
7533            |loaded_account: &LoadedAccount| {
7534                // Cache only has one version per key, don't need to worry about versioning
7535                Some(PubkeyHashAccount {
7536                    pubkey: *loaded_account.pubkey(),
7537                    hash: loaded_account.loaded_hash(),
7538                    account: loaded_account.take_account(),
7539                })
7540            },
7541            |accum: &DashMap<Pubkey, (AccountHash, AccountSharedData)>,
7542             loaded_account: &LoadedAccount,
7543             _data| {
7544                // Storage may have duplicates so only keep the latest version for each key
7545                let mut loaded_hash = loaded_account.loaded_hash();
7546                let key = *loaded_account.pubkey();
7547                let account = loaded_account.take_account();
7548                if loaded_hash == AccountHash(Hash::default()) {
7549                    loaded_hash = Self::hash_account(&account, &key)
7550                }
7551                accum.insert(key, (loaded_hash, account));
7552            },
7553            ScanAccountStorageData::NoData,
7554        );
7555
7556        match scan_result {
7557            ScanStorageResult::Cached(cached_result) => cached_result,
7558            ScanStorageResult::Stored(stored_result) => stored_result
7559                .into_iter()
7560                .map(|(pubkey, (hash, account))| PubkeyHashAccount {
7561                    pubkey,
7562                    hash,
7563                    account,
7564                })
7565                .collect(),
7566        }
7567    }
7568
7569    /// Calculate accounts delta hash for `slot`
7570    pub fn calculate_accounts_delta_hash_internal(
7571        &self,
7572        slot: Slot,
7573        ignore: Option<Pubkey>,
7574        mut skipped_rewrites: HashMap<Pubkey, AccountHash>,
7575    ) -> AccountsDeltaHash {
7576        let (mut hashes, scan_us, mut accumulate) = self.get_pubkey_hash_for_slot(slot);
7577
7578        hashes.iter().for_each(|(k, _h)| {
7579            skipped_rewrites.remove(k);
7580        });
7581
7582        let num_skipped_rewrites = skipped_rewrites.len();
7583        hashes.extend(skipped_rewrites);
7584
7585        info!("skipped rewrite hashes {} {}", slot, num_skipped_rewrites);
7586
7587        if let Some(ignore) = ignore {
7588            hashes.retain(|k| k.0 != ignore);
7589        }
7590
7591        let accounts_delta_hash = self
7592            .thread_pool
7593            .install(|| AccountsDeltaHash(AccountsHasher::accumulate_account_hashes(hashes)));
7594        accumulate.stop();
7595
7596        self.set_accounts_delta_hash(slot, accounts_delta_hash);
7597
7598        self.stats
7599            .delta_hash_scan_time_total_us
7600            .fetch_add(scan_us, Ordering::Relaxed);
7601        self.stats
7602            .delta_hash_accumulate_time_total_us
7603            .fetch_add(accumulate.as_us(), Ordering::Relaxed);
7604        self.stats.delta_hash_num.fetch_add(1, Ordering::Relaxed);
7605        self.stats
7606            .skipped_rewrites_num
7607            .fetch_add(num_skipped_rewrites, Ordering::Relaxed);
7608
7609        accounts_delta_hash
7610    }
7611
7612    /// Set the accounts delta hash for `slot` in the `accounts_delta_hashes` map
7613    ///
7614    /// returns the previous accounts delta hash for `slot`
7615    #[cfg_attr(feature = "dev-context-only-utils", qualifiers(pub))]
7616    fn set_accounts_delta_hash(
7617        &self,
7618        slot: Slot,
7619        accounts_delta_hash: AccountsDeltaHash,
7620    ) -> Option<AccountsDeltaHash> {
7621        self.accounts_delta_hashes
7622            .lock()
7623            .unwrap()
7624            .insert(slot, accounts_delta_hash)
7625    }
7626
7627    /// After deserializing a snapshot, set the accounts delta hash for the new AccountsDb
7628    pub fn set_accounts_delta_hash_from_snapshot(
7629        &mut self,
7630        slot: Slot,
7631        accounts_delta_hash: SerdeAccountsDeltaHash,
7632    ) -> Option<AccountsDeltaHash> {
7633        self.set_accounts_delta_hash(slot, accounts_delta_hash.into())
7634    }
7635
7636    /// Get the accounts delta hash for `slot` in the `accounts_delta_hashes` map
7637    pub fn get_accounts_delta_hash(&self, slot: Slot) -> Option<AccountsDeltaHash> {
7638        self.accounts_delta_hashes
7639            .lock()
7640            .unwrap()
7641            .get(&slot)
7642            .cloned()
7643    }
7644
7645    fn update_index<'a>(
7646        &self,
7647        infos: Vec<AccountInfo>,
7648        accounts: &impl StorableAccounts<'a>,
7649        reclaim: UpsertReclaim,
7650        update_index_thread_selection: UpdateIndexThreadSelection,
7651        thread_pool: &ThreadPool,
7652    ) -> SlotList<AccountInfo> {
7653        let target_slot = accounts.target_slot();
7654        let len = std::cmp::min(accounts.len(), infos.len());
7655
7656        let update = |start, end| {
7657            let mut reclaims = Vec::with_capacity((end - start) / 2);
7658
7659            (start..end).for_each(|i| {
7660                let info = infos[i];
7661                accounts.account(i, |account| {
7662                    let old_slot = accounts.slot(i);
7663                    self.accounts_index.upsert(
7664                        target_slot,
7665                        old_slot,
7666                        account.pubkey(),
7667                        &account,
7668                        &self.account_indexes,
7669                        info,
7670                        &mut reclaims,
7671                        reclaim,
7672                    );
7673                });
7674            });
7675            reclaims
7676        };
7677
7678        let threshold = 1;
7679        if matches!(
7680            update_index_thread_selection,
7681            UpdateIndexThreadSelection::PoolWithThreshold,
7682        ) && len > threshold
7683        {
7684            let chunk_size = std::cmp::max(1, len / quarter_thread_count()); // # pubkeys/thread
7685            let batches = 1 + len / chunk_size;
7686            thread_pool.install(|| {
7687                (0..batches)
7688                    .into_par_iter()
7689                    .map(|batch| {
7690                        let start = batch * chunk_size;
7691                        let end = std::cmp::min(start + chunk_size, len);
7692                        update(start, end)
7693                    })
7694                    .flatten()
7695                    .collect::<Vec<_>>()
7696            })
7697        } else {
7698            update(0, len)
7699        }
7700    }
7701
7702    fn should_not_shrink(alive_bytes: u64, total_bytes: u64) -> bool {
7703        alive_bytes >= total_bytes
7704    }
7705
7706    fn is_shrinking_productive(store: &AccountStorageEntry) -> bool {
7707        let alive_count = store.count();
7708        let total_bytes = store.capacity();
7709        let alive_bytes = store.alive_bytes_exclude_zero_lamport_single_ref_accounts() as u64;
7710        if Self::should_not_shrink(alive_bytes, total_bytes) {
7711            trace!(
7712                "shrink_slot_forced ({}): not able to shrink at all: num alive: {}, bytes alive: \
7713                 {}, bytes total: {}, bytes saved: {}",
7714                store.slot(),
7715                alive_count,
7716                alive_bytes,
7717                total_bytes,
7718                total_bytes.saturating_sub(alive_bytes),
7719            );
7720            return false;
7721        }
7722
7723        true
7724    }
7725
7726    /// Determines whether a given AccountStorageEntry instance is a
7727    /// candidate for shrinking.
7728    pub(crate) fn is_candidate_for_shrink(&self, store: &AccountStorageEntry) -> bool {
7729        // appended ancient append vecs should not be shrunk by the normal shrink codepath.
7730        // It is not possible to identify ancient append vecs when we pack, so no check for ancient when we are not appending.
7731        let total_bytes = if self.create_ancient_storage == CreateAncientStorage::Append
7732            && is_ancient(&store.accounts)
7733            && store.accounts.can_append()
7734        {
7735            store.written_bytes()
7736        } else {
7737            store.capacity()
7738        };
7739
7740        let alive_bytes = store.alive_bytes_exclude_zero_lamport_single_ref_accounts() as u64;
7741        match self.shrink_ratio {
7742            AccountShrinkThreshold::TotalSpace { shrink_ratio: _ } => alive_bytes < total_bytes,
7743            AccountShrinkThreshold::IndividualStore { shrink_ratio } => {
7744                (alive_bytes as f64 / total_bytes as f64) < shrink_ratio
7745            }
7746        }
7747    }
7748
7749    /// returns (dead slots, reclaimed_offsets)
7750    fn remove_dead_accounts<'a, I>(
7751        &'a self,
7752        reclaims: I,
7753        expected_slot: Option<Slot>,
7754        reset_accounts: bool,
7755    ) -> (IntSet<Slot>, SlotOffsets)
7756    where
7757        I: Iterator<Item = &'a (Slot, AccountInfo)>,
7758    {
7759        let mut reclaimed_offsets = SlotOffsets::default();
7760
7761        assert!(self.storage.no_shrink_in_progress());
7762
7763        let mut dead_slots = IntSet::default();
7764        let mut new_shrink_candidates = ShrinkCandidates::default();
7765        let mut measure = Measure::start("remove");
7766        for (slot, account_info) in reclaims {
7767            // No cached accounts should make it here
7768            assert!(!account_info.is_cached());
7769            reclaimed_offsets
7770                .entry(*slot)
7771                .or_default()
7772                .insert(account_info.offset());
7773        }
7774        if let Some(expected_slot) = expected_slot {
7775            assert_eq!(reclaimed_offsets.len(), 1);
7776            assert!(reclaimed_offsets.contains_key(&expected_slot));
7777        }
7778
7779        self.clean_accounts_stats
7780            .slots_cleaned
7781            .fetch_add(reclaimed_offsets.len() as u64, Ordering::Relaxed);
7782
7783        reclaimed_offsets.iter().for_each(|(slot, offsets)| {
7784            if let Some(store) = self.storage.get_slot_storage_entry(*slot) {
7785                assert_eq!(
7786                    *slot,
7787                    store.slot(),
7788                    "AccountsDB::accounts_index corrupted. Storage pointed to: {}, expected: {}, \
7789                     should only point to one slot",
7790                    store.slot(),
7791                    *slot
7792                );
7793                if offsets.len() == store.count() {
7794                    // all remaining alive accounts in the storage are being removed, so the entire storage/slot is dead
7795                    store.remove_accounts(store.alive_bytes(), reset_accounts, offsets.len());
7796                    self.dirty_stores.insert(*slot, store.clone());
7797                    dead_slots.insert(*slot);
7798                } else {
7799                    // not all accounts are being removed, so figure out sizes of accounts we are removing and update the alive bytes and alive account count
7800                    let (_, us) = measure_us!({
7801                        let mut offsets = offsets.iter().cloned().collect::<Vec<_>>();
7802                        // sort so offsets are in order. This improves efficiency of loading the accounts.
7803                        offsets.sort_unstable();
7804                        let dead_bytes = store.accounts.get_account_sizes(&offsets).iter().sum();
7805                        store.remove_accounts(dead_bytes, reset_accounts, offsets.len());
7806                        if Self::is_shrinking_productive(&store)
7807                            && self.is_candidate_for_shrink(&store)
7808                        {
7809                            // Checking that this single storage entry is ready for shrinking,
7810                            // should be a sufficient indication that the slot is ready to be shrunk
7811                            // because slots should only have one storage entry, namely the one that was
7812                            // created by `flush_slot_cache()`.
7813                            new_shrink_candidates.insert(*slot);
7814                        }
7815                    });
7816                    self.clean_accounts_stats
7817                        .get_account_sizes_us
7818                        .fetch_add(us, Ordering::Relaxed);
7819                }
7820            }
7821        });
7822        measure.stop();
7823        self.clean_accounts_stats
7824            .remove_dead_accounts_remove_us
7825            .fetch_add(measure.as_us(), Ordering::Relaxed);
7826
7827        let mut measure = Measure::start("shrink");
7828        let mut shrink_candidate_slots = self.shrink_candidate_slots.lock().unwrap();
7829        for slot in new_shrink_candidates {
7830            shrink_candidate_slots.insert(slot);
7831        }
7832        drop(shrink_candidate_slots);
7833        measure.stop();
7834        self.clean_accounts_stats
7835            .remove_dead_accounts_shrink_us
7836            .fetch_add(measure.as_us(), Ordering::Relaxed);
7837
7838        dead_slots.retain(|slot| {
7839            if let Some(slot_store) = self.storage.get_slot_storage_entry(*slot) {
7840                if slot_store.count() != 0 {
7841                    return false;
7842                }
7843            }
7844            true
7845        });
7846
7847        (dead_slots, reclaimed_offsets)
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 remove_dead_slots_metadata<'a>(
7853        &'a self,
7854        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
7855        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7856        // Should only be `Some` for non-cached slots
7857        purged_stored_account_slots: Option<&mut AccountSlots>,
7858        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7859    ) {
7860        let mut measure = Measure::start("remove_dead_slots_metadata-ms");
7861        self.clean_dead_slots_from_accounts_index(
7862            dead_slots_iter.clone(),
7863            purged_slot_pubkeys,
7864            purged_stored_account_slots,
7865            pubkeys_removed_from_accounts_index,
7866        );
7867
7868        let mut accounts_delta_hashes = self.accounts_delta_hashes.lock().unwrap();
7869        for slot in dead_slots_iter {
7870            accounts_delta_hashes.remove(slot);
7871        }
7872        drop(accounts_delta_hashes);
7873
7874        measure.stop();
7875        inc_new_counter_info!("remove_dead_slots_metadata-ms", measure.as_ms() as usize);
7876    }
7877
7878    /// lookup each pubkey in 'pubkeys' and unref it in the accounts index
7879    /// skip pubkeys that are in 'pubkeys_removed_from_accounts_index'
7880    fn unref_pubkeys<'a>(
7881        &'a self,
7882        pubkeys: impl Iterator<Item = &'a Pubkey> + Clone + Send + Sync,
7883        num_pubkeys: usize,
7884        pubkeys_removed_from_accounts_index: &'a PubkeysRemovedFromAccountsIndex,
7885    ) {
7886        let batches = 1 + (num_pubkeys / UNREF_ACCOUNTS_BATCH_SIZE);
7887        self.thread_pool_clean.install(|| {
7888            (0..batches).into_par_iter().for_each(|batch| {
7889                let skip = batch * UNREF_ACCOUNTS_BATCH_SIZE;
7890                self.accounts_index.scan(
7891                    pubkeys
7892                        .clone()
7893                        .skip(skip)
7894                        .take(UNREF_ACCOUNTS_BATCH_SIZE)
7895                        .filter(|pubkey| {
7896                            // filter out pubkeys that have already been removed from the accounts index in a previous step
7897                            let already_removed =
7898                                pubkeys_removed_from_accounts_index.contains(pubkey);
7899                            !already_removed
7900                        }),
7901                    |_pubkey, slots_refs, _entry| {
7902                        if let Some((slot_list, ref_count)) = slots_refs {
7903                            // Let's handle the special case - after unref, the result is a single ref zero lamport account.
7904                            if slot_list.len() == 1 && ref_count == 2 {
7905                                if let Some((slot_alive, acct_info)) = slot_list.first() {
7906                                    if acct_info.is_zero_lamport() && !acct_info.is_cached() {
7907                                        self.zero_lamport_single_ref_found(
7908                                            *slot_alive,
7909                                            acct_info.offset(),
7910                                        );
7911                                    }
7912                                }
7913                            }
7914                        }
7915                        AccountsIndexScanResult::Unref
7916                    },
7917                    None,
7918                    false,
7919                    ScanFilter::All,
7920                )
7921            });
7922        });
7923    }
7924
7925    /// lookup each pubkey in 'purged_slot_pubkeys' and unref it in the accounts index
7926    /// populate 'purged_stored_account_slots' by grouping 'purged_slot_pubkeys' by pubkey
7927    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7928    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7929    fn unref_accounts(
7930        &self,
7931        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7932        purged_stored_account_slots: &mut AccountSlots,
7933        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7934    ) {
7935        self.unref_pubkeys(
7936            purged_slot_pubkeys.iter().map(|(_slot, pubkey)| pubkey),
7937            purged_slot_pubkeys.len(),
7938            pubkeys_removed_from_accounts_index,
7939        );
7940        for (slot, pubkey) in purged_slot_pubkeys {
7941            purged_stored_account_slots
7942                .entry(pubkey)
7943                .or_default()
7944                .insert(slot);
7945        }
7946    }
7947
7948    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7949    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
7950    fn clean_dead_slots_from_accounts_index<'a>(
7951        &'a self,
7952        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
7953        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7954        // Should only be `Some` for non-cached slots
7955        purged_stored_account_slots: Option<&mut AccountSlots>,
7956        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7957    ) {
7958        let mut accounts_index_root_stats = AccountsIndexRootsStats::default();
7959        let mut measure = Measure::start("unref_from_storage");
7960        if let Some(purged_stored_account_slots) = purged_stored_account_slots {
7961            self.unref_accounts(
7962                purged_slot_pubkeys,
7963                purged_stored_account_slots,
7964                pubkeys_removed_from_accounts_index,
7965            );
7966        }
7967        measure.stop();
7968        accounts_index_root_stats.clean_unref_from_storage_us += measure.as_us();
7969
7970        let mut measure = Measure::start("clean_dead_slot");
7971        let mut rooted_cleaned_count = 0;
7972        let mut unrooted_cleaned_count = 0;
7973        let dead_slots: Vec<_> = dead_slots_iter
7974            .map(|slot| {
7975                if self.accounts_index.clean_dead_slot(*slot) {
7976                    rooted_cleaned_count += 1;
7977                } else {
7978                    unrooted_cleaned_count += 1;
7979                }
7980                *slot
7981            })
7982            .collect();
7983        measure.stop();
7984        accounts_index_root_stats.clean_dead_slot_us += measure.as_us();
7985        if self.log_dead_slots.load(Ordering::Relaxed) {
7986            info!(
7987                "remove_dead_slots_metadata: {} dead slots",
7988                dead_slots.len()
7989            );
7990            trace!("remove_dead_slots_metadata: dead_slots: {:?}", dead_slots);
7991        }
7992        self.accounts_index
7993            .update_roots_stats(&mut accounts_index_root_stats);
7994        accounts_index_root_stats.rooted_cleaned_count += rooted_cleaned_count;
7995        accounts_index_root_stats.unrooted_cleaned_count += unrooted_cleaned_count;
7996
7997        self.clean_accounts_stats
7998            .latest_accounts_index_roots_stats
7999            .update(&accounts_index_root_stats);
8000    }
8001
8002    /// pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
8003    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
8004    fn clean_stored_dead_slots(
8005        &self,
8006        dead_slots: &IntSet<Slot>,
8007        purged_account_slots: Option<&mut AccountSlots>,
8008        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
8009    ) {
8010        let mut measure = Measure::start("clean_stored_dead_slots-ms");
8011        let mut stores = vec![];
8012        // get all stores in a vec so we can iterate in parallel
8013        for slot in dead_slots.iter() {
8014            if let Some(slot_storage) = self.storage.get_slot_storage_entry(*slot) {
8015                stores.push(slot_storage);
8016            }
8017        }
8018        // get all pubkeys in all dead slots
8019        let purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = {
8020            self.thread_pool_clean.install(|| {
8021                stores
8022                    .into_par_iter()
8023                    .map(|store| {
8024                        let slot = store.slot();
8025                        let mut pubkeys = Vec::with_capacity(store.count());
8026                        store.accounts.scan_pubkeys(|pubkey| {
8027                            pubkeys.push((slot, *pubkey));
8028                        });
8029                        pubkeys
8030                    })
8031                    .flatten()
8032                    .collect::<HashSet<_>>()
8033            })
8034        };
8035        self.remove_dead_slots_metadata(
8036            dead_slots.iter(),
8037            purged_slot_pubkeys,
8038            purged_account_slots,
8039            pubkeys_removed_from_accounts_index,
8040        );
8041        measure.stop();
8042        self.clean_accounts_stats
8043            .clean_stored_dead_slots_us
8044            .fetch_add(measure.as_us(), Ordering::Relaxed);
8045    }
8046
8047    pub fn store_cached<'a>(
8048        &self,
8049        accounts: impl StorableAccounts<'a>,
8050        transactions: Option<&'a [&'a SanitizedTransaction]>,
8051    ) {
8052        self.store(
8053            accounts,
8054            &StoreTo::Cache,
8055            transactions,
8056            StoreReclaims::Default,
8057            UpdateIndexThreadSelection::PoolWithThreshold,
8058        );
8059    }
8060
8061    pub(crate) fn store_cached_inline_update_index<'a>(
8062        &self,
8063        accounts: impl StorableAccounts<'a>,
8064        transactions: Option<&'a [&'a SanitizedTransaction]>,
8065    ) {
8066        self.store(
8067            accounts,
8068            &StoreTo::Cache,
8069            transactions,
8070            StoreReclaims::Default,
8071            UpdateIndexThreadSelection::Inline,
8072        );
8073    }
8074
8075    /// Store the account update.
8076    /// only called by tests
8077    pub fn store_uncached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
8078        let storage = self.find_storage_candidate(slot);
8079        self.store(
8080            (slot, accounts),
8081            &StoreTo::Storage(&storage),
8082            None,
8083            StoreReclaims::Default,
8084            UpdateIndexThreadSelection::PoolWithThreshold,
8085        );
8086    }
8087
8088    fn store<'a>(
8089        &self,
8090        accounts: impl StorableAccounts<'a>,
8091        store_to: &StoreTo,
8092        transactions: Option<&'a [&'a SanitizedTransaction]>,
8093        reclaim: StoreReclaims,
8094        update_index_thread_selection: UpdateIndexThreadSelection,
8095    ) {
8096        // If all transactions in a batch are errored,
8097        // it's possible to get a store with no accounts.
8098        if accounts.is_empty() {
8099            return;
8100        }
8101
8102        let mut total_data = 0;
8103        (0..accounts.len()).for_each(|index| {
8104            accounts.account(index, |account| {
8105                total_data += account.data().len();
8106            })
8107        });
8108
8109        self.stats
8110            .store_total_data
8111            .fetch_add(total_data as u64, Ordering::Relaxed);
8112
8113        self.store_accounts_unfrozen(
8114            accounts,
8115            store_to,
8116            transactions,
8117            reclaim,
8118            update_index_thread_selection,
8119        );
8120        self.report_store_timings();
8121    }
8122
8123    fn report_store_timings(&self) {
8124        if self.stats.last_store_report.should_update(1000) {
8125            let read_cache_stats = self.read_only_accounts_cache.get_and_reset_stats();
8126            datapoint_info!(
8127                "accounts_db_store_timings",
8128                (
8129                    "hash_accounts",
8130                    self.stats.store_hash_accounts.swap(0, Ordering::Relaxed),
8131                    i64
8132                ),
8133                (
8134                    "store_accounts",
8135                    self.stats.store_accounts.swap(0, Ordering::Relaxed),
8136                    i64
8137                ),
8138                (
8139                    "update_index",
8140                    self.stats.store_update_index.swap(0, Ordering::Relaxed),
8141                    i64
8142                ),
8143                (
8144                    "handle_reclaims",
8145                    self.stats.store_handle_reclaims.swap(0, Ordering::Relaxed),
8146                    i64
8147                ),
8148                (
8149                    "append_accounts",
8150                    self.stats.store_append_accounts.swap(0, Ordering::Relaxed),
8151                    i64
8152                ),
8153                (
8154                    "stakes_cache_check_and_store_us",
8155                    self.stats
8156                        .stakes_cache_check_and_store_us
8157                        .swap(0, Ordering::Relaxed),
8158                    i64
8159                ),
8160                (
8161                    "num_accounts",
8162                    self.stats.store_num_accounts.swap(0, Ordering::Relaxed),
8163                    i64
8164                ),
8165                (
8166                    "total_data",
8167                    self.stats.store_total_data.swap(0, Ordering::Relaxed),
8168                    i64
8169                ),
8170                (
8171                    "read_only_accounts_cache_entries",
8172                    self.read_only_accounts_cache.cache_len(),
8173                    i64
8174                ),
8175                (
8176                    "read_only_accounts_cache_data_size",
8177                    self.read_only_accounts_cache.data_size(),
8178                    i64
8179                ),
8180                ("read_only_accounts_cache_hits", read_cache_stats.hits, i64),
8181                (
8182                    "read_only_accounts_cache_misses",
8183                    read_cache_stats.misses,
8184                    i64
8185                ),
8186                (
8187                    "read_only_accounts_cache_evicts",
8188                    read_cache_stats.evicts,
8189                    i64
8190                ),
8191                (
8192                    "read_only_accounts_cache_load_us",
8193                    read_cache_stats.load_us,
8194                    i64
8195                ),
8196                (
8197                    "read_only_accounts_cache_store_us",
8198                    read_cache_stats.store_us,
8199                    i64
8200                ),
8201                (
8202                    "read_only_accounts_cache_evict_us",
8203                    read_cache_stats.evict_us,
8204                    i64
8205                ),
8206                (
8207                    "read_only_accounts_cache_evictor_wakeup_count_all",
8208                    read_cache_stats.evictor_wakeup_count_all,
8209                    i64
8210                ),
8211                (
8212                    "read_only_accounts_cache_evictor_wakeup_count_productive",
8213                    read_cache_stats.evictor_wakeup_count_productive,
8214                    i64
8215                ),
8216                (
8217                    "calc_stored_meta_us",
8218                    self.stats.calc_stored_meta.swap(0, Ordering::Relaxed),
8219                    i64
8220                ),
8221                (
8222                    "handle_dead_keys_us",
8223                    self.stats.handle_dead_keys_us.swap(0, Ordering::Relaxed),
8224                    i64
8225                ),
8226                (
8227                    "purge_exact_us",
8228                    self.stats.purge_exact_us.swap(0, Ordering::Relaxed),
8229                    i64
8230                ),
8231                (
8232                    "purge_exact_count",
8233                    self.stats.purge_exact_count.swap(0, Ordering::Relaxed),
8234                    i64
8235                ),
8236            );
8237
8238            datapoint_info!(
8239                "accounts_db_store_timings2",
8240                (
8241                    "create_store_count",
8242                    self.stats.create_store_count.swap(0, Ordering::Relaxed),
8243                    i64
8244                ),
8245                (
8246                    "store_get_slot_store",
8247                    self.stats.store_get_slot_store.swap(0, Ordering::Relaxed),
8248                    i64
8249                ),
8250                (
8251                    "store_find_existing",
8252                    self.stats.store_find_existing.swap(0, Ordering::Relaxed),
8253                    i64
8254                ),
8255                (
8256                    "dropped_stores",
8257                    self.stats.dropped_stores.swap(0, Ordering::Relaxed),
8258                    i64
8259                ),
8260            );
8261        }
8262    }
8263
8264    fn store_accounts_unfrozen<'a>(
8265        &self,
8266        accounts: impl StorableAccounts<'a>,
8267        store_to: &StoreTo,
8268        transactions: Option<&'a [&'a SanitizedTransaction]>,
8269        reclaim: StoreReclaims,
8270        update_index_thread_selection: UpdateIndexThreadSelection,
8271    ) {
8272        // This path comes from a store to a non-frozen slot.
8273        // If a store is dead here, then a newer update for
8274        // each pubkey in the store must exist in another
8275        // store in the slot. Thus it is safe to reset the store and
8276        // re-use it for a future store op. The pubkey ref counts should still
8277        // hold just 1 ref from this slot.
8278        let reset_accounts = true;
8279
8280        self.store_accounts_custom(
8281            accounts,
8282            store_to,
8283            reset_accounts,
8284            transactions,
8285            reclaim,
8286            update_index_thread_selection,
8287            &self.thread_pool,
8288        );
8289    }
8290
8291    pub fn store_accounts_frozen<'a>(
8292        &self,
8293        accounts: impl StorableAccounts<'a>,
8294        storage: &Arc<AccountStorageEntry>,
8295    ) -> StoreAccountsTiming {
8296        // stores on a frozen slot should not reset
8297        // the append vec so that hashing could happen on the store
8298        // and accounts in the append_vec can be unrefed correctly
8299        let reset_accounts = false;
8300        self.store_accounts_custom(
8301            accounts,
8302            &StoreTo::Storage(storage),
8303            reset_accounts,
8304            None,
8305            StoreReclaims::Ignore,
8306            UpdateIndexThreadSelection::PoolWithThreshold,
8307            &self.thread_pool_clean,
8308        )
8309    }
8310
8311    fn store_accounts_custom<'a>(
8312        &self,
8313        accounts: impl StorableAccounts<'a>,
8314        store_to: &StoreTo,
8315        reset_accounts: bool,
8316        transactions: Option<&'a [&'a SanitizedTransaction]>,
8317        reclaim: StoreReclaims,
8318        update_index_thread_selection: UpdateIndexThreadSelection,
8319        thread_pool: &ThreadPool,
8320    ) -> StoreAccountsTiming {
8321        self.stats
8322            .store_num_accounts
8323            .fetch_add(accounts.len() as u64, Ordering::Relaxed);
8324        let mut store_accounts_time = Measure::start("store_accounts");
8325        let infos = self.store_accounts_to(&accounts, store_to, transactions);
8326        store_accounts_time.stop();
8327        self.stats
8328            .store_accounts
8329            .fetch_add(store_accounts_time.as_us(), Ordering::Relaxed);
8330        let mut update_index_time = Measure::start("update_index");
8331
8332        let reclaim = if matches!(reclaim, StoreReclaims::Ignore) {
8333            UpsertReclaim::IgnoreReclaims
8334        } else if store_to.is_cached() {
8335            UpsertReclaim::PreviousSlotEntryWasCached
8336        } else {
8337            UpsertReclaim::PopulateReclaims
8338        };
8339
8340        // if we are squashing a single slot, then we can expect a single dead slot
8341        let expected_single_dead_slot =
8342            (!accounts.contains_multiple_slots()).then(|| accounts.target_slot());
8343
8344        // If the cache was flushed, then because `update_index` occurs
8345        // after the account are stored by the above `store_accounts_to`
8346        // call and all the accounts are stored, all reads after this point
8347        // will know to not check the cache anymore
8348        let mut reclaims = self.update_index(
8349            infos,
8350            &accounts,
8351            reclaim,
8352            update_index_thread_selection,
8353            thread_pool,
8354        );
8355
8356        // For each updated account, `reclaims` should only have at most one
8357        // item (if the account was previously updated in this slot).
8358        // filter out the cached reclaims as those don't actually map
8359        // to anything that needs to be cleaned in the backing storage
8360        // entries
8361        reclaims.retain(|(_, r)| !r.is_cached());
8362
8363        if store_to.is_cached() {
8364            assert!(reclaims.is_empty());
8365        }
8366
8367        update_index_time.stop();
8368        self.stats
8369            .store_update_index
8370            .fetch_add(update_index_time.as_us(), Ordering::Relaxed);
8371
8372        // A store for a single slot should:
8373        // 1) Only make "reclaims" for the same slot
8374        // 2) Should not cause any slots to be removed from the storage
8375        // database because
8376        //    a) this slot  has at least one account (the one being stored),
8377        //    b)From 1) we know no other slots are included in the "reclaims"
8378        //
8379        // From 1) and 2) we guarantee passing `no_purge_stats` == None, which is
8380        // equivalent to asserting there will be no dead slots, is safe.
8381        let mut handle_reclaims_elapsed = 0;
8382        if reclaim == UpsertReclaim::PopulateReclaims {
8383            let mut handle_reclaims_time = Measure::start("handle_reclaims");
8384            self.handle_reclaims(
8385                (!reclaims.is_empty()).then(|| reclaims.iter()),
8386                expected_single_dead_slot,
8387                reset_accounts,
8388                &HashSet::default(),
8389                // this callsite does NOT process dead slots
8390                HandleReclaims::DoNotProcessDeadSlots,
8391            );
8392            handle_reclaims_time.stop();
8393            handle_reclaims_elapsed = handle_reclaims_time.as_us();
8394            self.stats
8395                .store_handle_reclaims
8396                .fetch_add(handle_reclaims_elapsed, Ordering::Relaxed);
8397        } else {
8398            assert!(reclaims.is_empty());
8399        }
8400
8401        StoreAccountsTiming {
8402            store_accounts_elapsed: store_accounts_time.as_us(),
8403            update_index_elapsed: update_index_time.as_us(),
8404            handle_reclaims_elapsed,
8405        }
8406    }
8407
8408    pub fn add_root(&self, slot: Slot) -> AccountsAddRootTiming {
8409        let mut index_time = Measure::start("index_add_root");
8410        self.accounts_index.add_root(slot);
8411        index_time.stop();
8412        let mut cache_time = Measure::start("cache_add_root");
8413        self.accounts_cache.add_root(slot);
8414        cache_time.stop();
8415        let mut store_time = Measure::start("store_add_root");
8416        // We would not expect this slot to be shrinking right now, but other slots may be.
8417        // But, even if it was, we would just mark a store id as dirty unnecessarily and that is ok.
8418        // So, allow shrinking to be in progress.
8419        if let Some(store) = self
8420            .storage
8421            .get_slot_storage_entry_shrinking_in_progress_ok(slot)
8422        {
8423            self.dirty_stores.insert(slot, store);
8424        }
8425        store_time.stop();
8426
8427        AccountsAddRootTiming {
8428            index_us: index_time.as_us(),
8429            cache_us: cache_time.as_us(),
8430            store_us: store_time.as_us(),
8431        }
8432    }
8433
8434    /// Returns storages for `requested_slots`
8435    pub fn get_storages(
8436        &self,
8437        requested_slots: impl RangeBounds<Slot> + Sync,
8438    ) -> (Vec<Arc<AccountStorageEntry>>, Vec<Slot>) {
8439        let start = Instant::now();
8440        let (slots, storages) = self
8441            .storage
8442            .get_if(|slot, storage| requested_slots.contains(slot) && storage.has_accounts())
8443            .into_vec()
8444            .into_iter()
8445            .unzip();
8446        let duration = start.elapsed();
8447        debug!("get_snapshot_storages: {duration:?}");
8448        (storages, slots)
8449    }
8450
8451    /// Returns the latest full snapshot slot
8452    pub fn latest_full_snapshot_slot(&self) -> Option<Slot> {
8453        self.latest_full_snapshot_slot.read()
8454    }
8455
8456    /// Sets the latest full snapshot slot to `slot`
8457    pub fn set_latest_full_snapshot_slot(&self, slot: Slot) {
8458        *self.latest_full_snapshot_slot.lock_write() = Some(slot);
8459    }
8460
8461    /// return Some(lamports_to_top_off) if 'account' would collect rent
8462    fn stats_for_rent_payers(
8463        pubkey: &Pubkey,
8464        lamports: u64,
8465        account_data_len: usize,
8466        account_rent_epoch: Epoch,
8467        executable: bool,
8468        rent_collector: &RentCollector,
8469    ) -> Option<u64> {
8470        if lamports == 0 {
8471            return None;
8472        }
8473        (rent_collector.should_collect_rent(pubkey, executable)
8474            && !rent_collector
8475                .get_rent_due(lamports, account_data_len, account_rent_epoch)
8476                .is_exempt())
8477        .then(|| {
8478            let min_balance = rent_collector.rent.minimum_balance(account_data_len);
8479            // return lamports required to top off this account to make it rent exempt
8480            min_balance.saturating_sub(lamports)
8481        })
8482    }
8483
8484    fn generate_index_for_slot(
8485        &self,
8486        storage: &AccountStorageEntry,
8487        slot: Slot,
8488        store_id: AccountsFileId,
8489        rent_collector: &RentCollector,
8490        storage_info: &StorageSizeAndCountMap,
8491    ) -> SlotIndexGenerationInfo {
8492        if storage.accounts.get_account_sizes(&[0]).is_empty() {
8493            return SlotIndexGenerationInfo::default();
8494        }
8495        let secondary = !self.account_indexes.is_empty();
8496
8497        let mut rent_paying_accounts_by_partition = Vec::default();
8498        let mut accounts_data_len = 0;
8499        let mut num_accounts_rent_paying = 0;
8500        let mut amount_to_top_off_rent = 0;
8501        let mut stored_size_alive = 0;
8502        let mut zero_lamport_pubkeys = vec![];
8503        let mut all_accounts_are_zero_lamports = true;
8504
8505        let (dirty_pubkeys, insert_time_us, mut generate_index_results) = {
8506            let mut items_local = Vec::default();
8507            storage.accounts.scan_index(|info| {
8508                stored_size_alive += info.stored_size_aligned;
8509                if info.index_info.lamports > 0 {
8510                    accounts_data_len += info.index_info.data_len;
8511                    all_accounts_are_zero_lamports = false;
8512                } else {
8513                    // zero lamport accounts
8514                    zero_lamport_pubkeys.push(info.index_info.pubkey);
8515                }
8516                items_local.push(info.index_info);
8517            });
8518
8519            let items_len = items_local.len();
8520            let items = items_local.into_iter().map(|info| {
8521                if let Some(amount_to_top_off_rent_this_account) = Self::stats_for_rent_payers(
8522                    &info.pubkey,
8523                    info.lamports,
8524                    info.data_len as usize,
8525                    info.rent_epoch,
8526                    info.executable,
8527                    rent_collector,
8528                ) {
8529                    amount_to_top_off_rent += amount_to_top_off_rent_this_account;
8530                    num_accounts_rent_paying += 1;
8531                    // remember this rent-paying account pubkey
8532                    rent_paying_accounts_by_partition.push(info.pubkey);
8533                }
8534
8535                (
8536                    info.pubkey,
8537                    AccountInfo::new(
8538                        StorageLocation::AppendVec(store_id, info.offset), // will never be cached
8539                        info.lamports,
8540                    ),
8541                )
8542            });
8543            self.accounts_index
8544                .insert_new_if_missing_into_primary_index(slot, items_len, items)
8545        };
8546        if secondary {
8547            // scan storage a second time to update the secondary index
8548            storage.accounts.scan_accounts(|stored_account| {
8549                stored_size_alive += stored_account.stored_size();
8550                let pubkey = stored_account.pubkey();
8551                self.accounts_index.update_secondary_indexes(
8552                    pubkey,
8553                    &stored_account,
8554                    &self.account_indexes,
8555                );
8556            });
8557        }
8558
8559        if let Some(duplicates_this_slot) = std::mem::take(&mut generate_index_results.duplicates) {
8560            // there were duplicate pubkeys in this same slot
8561            // Some were not inserted. This means some info like stored data is off.
8562            duplicates_this_slot
8563                .into_iter()
8564                .for_each(|(pubkey, (_slot, info))| {
8565                    storage
8566                        .accounts
8567                        .get_stored_account_meta_callback(info.offset(), |duplicate| {
8568                            assert_eq!(&pubkey, duplicate.pubkey());
8569                            stored_size_alive =
8570                                stored_size_alive.saturating_sub(duplicate.stored_size());
8571                            if !duplicate.is_zero_lamport() {
8572                                accounts_data_len =
8573                                    accounts_data_len.saturating_sub(duplicate.data().len() as u64);
8574                            }
8575                        });
8576                });
8577        }
8578
8579        {
8580            // second, collect into the shared DashMap once we've figured out all the info per store_id
8581            let mut info = storage_info.entry(store_id).or_default();
8582            info.stored_size += stored_size_alive;
8583            info.count += generate_index_results.count;
8584        }
8585
8586        // dirty_pubkeys will contain a pubkey if an item has multiple rooted entries for
8587        // a given pubkey. If there is just a single item, there is no cleaning to
8588        // be done on that pubkey. Use only those pubkeys with multiple updates.
8589        if !dirty_pubkeys.is_empty() {
8590            let old = self.uncleaned_pubkeys.insert(slot, dirty_pubkeys);
8591            assert!(old.is_none());
8592        }
8593        SlotIndexGenerationInfo {
8594            insert_time_us,
8595            num_accounts: generate_index_results.count as u64,
8596            num_accounts_rent_paying,
8597            accounts_data_len,
8598            amount_to_top_off_rent,
8599            rent_paying_accounts_by_partition,
8600            zero_lamport_pubkeys,
8601            all_accounts_are_zero_lamports,
8602        }
8603    }
8604
8605    pub fn generate_index(
8606        &self,
8607        limit_load_slot_count_from_snapshot: Option<usize>,
8608        verify: bool,
8609        genesis_config: &GenesisConfig,
8610        should_calculate_duplicates_lt_hash: bool,
8611    ) -> IndexGenerationInfo {
8612        let mut total_time = Measure::start("generate_index");
8613        let mut slots = self.storage.all_slots();
8614        slots.sort_unstable();
8615        if let Some(limit) = limit_load_slot_count_from_snapshot {
8616            slots.truncate(limit); // get rid of the newer slots and keep just the older
8617        }
8618        let max_slot = slots.last().cloned().unwrap_or_default();
8619        let schedule = &genesis_config.epoch_schedule;
8620        let rent_collector = RentCollector::new(
8621            schedule.get_epoch(max_slot),
8622            schedule.clone(),
8623            genesis_config.slots_per_year(),
8624            genesis_config.rent.clone(),
8625        );
8626        let accounts_data_len = AtomicU64::new(0);
8627
8628        let rent_paying_accounts_by_partition =
8629            Mutex::new(RentPayingAccountsByPartition::new(schedule));
8630        let zero_lamport_pubkeys = Mutex::new(HashSet::new());
8631        let mut outer_duplicates_lt_hash = None;
8632
8633        // pass == 0 always runs and generates the index
8634        // pass == 1 only runs if verify == true.
8635        // verify checks that all the expected items are in the accounts index and measures how long it takes to look them all up
8636        let passes = if verify { 2 } else { 1 };
8637        for pass in 0..passes {
8638            if pass == 0 {
8639                self.accounts_index
8640                    .set_startup(Startup::StartupWithExtraThreads);
8641            }
8642            let storage_info = StorageSizeAndCountMap::default();
8643            let total_processed_slots_across_all_threads = AtomicU64::new(0);
8644            let outer_slots_len = slots.len();
8645            let threads = if self.accounts_index.is_disk_index_enabled() {
8646                // these write directly to disk, so the more threads, the better
8647                num_cpus::get()
8648            } else {
8649                // seems to be a good heuristic given varying # cpus for in-mem disk index
8650                8
8651            };
8652            let chunk_size = (outer_slots_len / (std::cmp::max(1, threads.saturating_sub(1)))) + 1; // approximately 400k slots in a snapshot
8653            let mut index_time = Measure::start("index");
8654            let insertion_time_us = AtomicU64::new(0);
8655            let rent_paying = AtomicUsize::new(0);
8656            let amount_to_top_off_rent = AtomicU64::new(0);
8657            let total_including_duplicates = AtomicU64::new(0);
8658            let all_accounts_are_zero_lamports_slots = AtomicU64::new(0);
8659            let mut all_zeros_slots = Mutex::new(Vec::<(Slot, Arc<AccountStorageEntry>)>::new());
8660            let scan_time: u64 = slots
8661                .par_chunks(chunk_size)
8662                .map(|slots| {
8663                    let mut log_status = MultiThreadProgress::new(
8664                        &total_processed_slots_across_all_threads,
8665                        2,
8666                        outer_slots_len as u64,
8667                    );
8668                    let mut scan_time_sum = 0;
8669                    let mut all_accounts_are_zero_lamports_slots_inner = 0;
8670                    let mut all_zeros_slots_inner = vec![];
8671                    let mut insert_time_sum = 0;
8672                    let mut total_including_duplicates_sum = 0;
8673                    let mut accounts_data_len_sum = 0;
8674                    for (index, slot) in slots.iter().enumerate() {
8675                        let mut scan_time = Measure::start("scan");
8676                        log_status.report(index as u64);
8677                        let Some(storage) = self.storage.get_slot_storage_entry(*slot) else {
8678                            // no storage at this slot, no information to pull out
8679                            continue;
8680                        };
8681                        let store_id = storage.id();
8682
8683                        scan_time.stop();
8684                        scan_time_sum += scan_time.as_us();
8685
8686                        let insert_us = if pass == 0 {
8687                            // generate index
8688                            self.maybe_throttle_index_generation();
8689                            let SlotIndexGenerationInfo {
8690                                insert_time_us: insert_us,
8691                                num_accounts: total_this_slot,
8692                                num_accounts_rent_paying: rent_paying_this_slot,
8693                                accounts_data_len: accounts_data_len_this_slot,
8694                                amount_to_top_off_rent: amount_to_top_off_rent_this_slot,
8695                                rent_paying_accounts_by_partition:
8696                                    rent_paying_accounts_by_partition_this_slot,
8697                                zero_lamport_pubkeys: zero_pubkeys_this_slot,
8698                                all_accounts_are_zero_lamports,
8699                            } = self.generate_index_for_slot(
8700                                &storage,
8701                                *slot,
8702                                store_id,
8703                                &rent_collector,
8704                                &storage_info,
8705                            );
8706
8707                            if rent_paying_this_slot > 0 {
8708                                // We don't have any rent paying accounts on mainnet, so this code should never be hit.
8709                                rent_paying.fetch_add(rent_paying_this_slot, Ordering::Relaxed);
8710                                amount_to_top_off_rent
8711                                    .fetch_add(amount_to_top_off_rent_this_slot, Ordering::Relaxed);
8712                                let mut rent_paying_accounts_by_partition =
8713                                    rent_paying_accounts_by_partition.lock().unwrap();
8714                                rent_paying_accounts_by_partition_this_slot
8715                                    .iter()
8716                                    .for_each(|k| {
8717                                        rent_paying_accounts_by_partition.add_account(k);
8718                                    });
8719                            }
8720                            total_including_duplicates_sum += total_this_slot;
8721                            accounts_data_len_sum += accounts_data_len_this_slot;
8722                            if all_accounts_are_zero_lamports {
8723                                all_accounts_are_zero_lamports_slots_inner += 1;
8724                                all_zeros_slots_inner.push((*slot, Arc::clone(&storage)));
8725                            }
8726                            let mut zero_pubkeys = zero_lamport_pubkeys.lock().unwrap();
8727                            zero_pubkeys_this_slot.into_iter().for_each(|k| {
8728                                zero_pubkeys.insert(k);
8729                            });
8730
8731                            insert_us
8732                        } else {
8733                            // verify index matches expected and measure the time to get all items
8734                            assert!(verify);
8735                            let mut lookup_time = Measure::start("lookup_time");
8736                            storage.accounts.scan_accounts(|account_info| {
8737                                let key = account_info.pubkey();
8738                                let index_entry = self.accounts_index.get_cloned(key).unwrap();
8739                                let slot_list = index_entry.slot_list.read().unwrap();
8740                                let mut count = 0;
8741                                for (slot2, account_info2) in slot_list.iter() {
8742                                    if slot2 == slot {
8743                                        count += 1;
8744                                        let ai = AccountInfo::new(
8745                                            StorageLocation::AppendVec(
8746                                                store_id,
8747                                                account_info.offset(),
8748                                            ), // will never be cached
8749                                            account_info.lamports(),
8750                                        );
8751                                        assert_eq!(&ai, account_info2);
8752                                    }
8753                                }
8754                                assert_eq!(1, count);
8755                            });
8756                            lookup_time.stop();
8757                            lookup_time.as_us()
8758                        };
8759                        insert_time_sum += insert_us;
8760                    }
8761                    all_accounts_are_zero_lamports_slots.fetch_add(
8762                        all_accounts_are_zero_lamports_slots_inner,
8763                        Ordering::Relaxed,
8764                    );
8765                    all_zeros_slots
8766                        .lock()
8767                        .unwrap()
8768                        .append(&mut all_zeros_slots_inner);
8769                    insertion_time_us.fetch_add(insert_time_sum, Ordering::Relaxed);
8770                    total_including_duplicates
8771                        .fetch_add(total_including_duplicates_sum, Ordering::Relaxed);
8772                    accounts_data_len.fetch_add(accounts_data_len_sum, Ordering::Relaxed);
8773                    scan_time_sum
8774                })
8775                .sum();
8776            index_time.stop();
8777
8778            info!("rent_collector: {:?}", rent_collector);
8779
8780            let mut index_flush_us = 0;
8781            let total_duplicate_slot_keys = AtomicU64::default();
8782            let mut populate_duplicate_keys_us = 0;
8783            let mut total_items_in_mem = 0;
8784            let mut min_bin_size_in_mem = 0;
8785            let mut max_bin_size_in_mem = 0;
8786            let total_num_unique_duplicate_keys = AtomicU64::default();
8787
8788            // outer vec is accounts index bin (determined by pubkey value)
8789            // inner vec is the pubkeys within that bin that are present in > 1 slot
8790            let unique_pubkeys_by_bin = Mutex::new(Vec::<Vec<Pubkey>>::default());
8791            if pass == 0 {
8792                // tell accounts index we are done adding the initial accounts at startup
8793                let mut m = Measure::start("accounts_index_idle_us");
8794                self.accounts_index.set_startup(Startup::Normal);
8795                m.stop();
8796                index_flush_us = m.as_us();
8797
8798                populate_duplicate_keys_us = measure_us!({
8799                    // this has to happen before visit_duplicate_pubkeys_during_startup below
8800                    // get duplicate keys from acct idx. We have to wait until we've finished flushing.
8801                    self.accounts_index
8802                        .populate_and_retrieve_duplicate_keys_from_startup(|slot_keys| {
8803                            total_duplicate_slot_keys
8804                                .fetch_add(slot_keys.len() as u64, Ordering::Relaxed);
8805                            let unique_keys =
8806                                HashSet::<Pubkey>::from_iter(slot_keys.iter().map(|(_, key)| *key));
8807                            for (slot, key) in slot_keys {
8808                                self.uncleaned_pubkeys.entry(slot).or_default().push(key);
8809                            }
8810                            let unique_pubkeys_by_bin_inner =
8811                                unique_keys.into_iter().collect::<Vec<_>>();
8812                            total_num_unique_duplicate_keys.fetch_add(
8813                                unique_pubkeys_by_bin_inner.len() as u64,
8814                                Ordering::Relaxed,
8815                            );
8816                            // does not matter that this is not ordered by slot
8817                            unique_pubkeys_by_bin
8818                                .lock()
8819                                .unwrap()
8820                                .push(unique_pubkeys_by_bin_inner);
8821                        });
8822                })
8823                .1;
8824
8825                (total_items_in_mem, min_bin_size_in_mem, max_bin_size_in_mem) = self
8826                    .accounts_index
8827                    .account_maps
8828                    .iter()
8829                    .map(|map_bin| map_bin.len_for_stats())
8830                    .fold((0, usize::MAX, usize::MIN), |acc, len| {
8831                        (
8832                            acc.0 + len,
8833                            std::cmp::min(acc.1, len),
8834                            std::cmp::max(acc.2, len),
8835                        )
8836                    });
8837            }
8838            let unique_pubkeys_by_bin = unique_pubkeys_by_bin.into_inner().unwrap();
8839
8840            let mut timings = GenerateIndexTimings {
8841                index_flush_us,
8842                scan_time,
8843                index_time: index_time.as_us(),
8844                insertion_time_us: insertion_time_us.load(Ordering::Relaxed),
8845                min_bin_size_in_mem,
8846                max_bin_size_in_mem,
8847                total_items_in_mem,
8848                rent_paying,
8849                amount_to_top_off_rent,
8850                total_duplicate_slot_keys: total_duplicate_slot_keys.load(Ordering::Relaxed),
8851                total_num_unique_duplicate_keys: total_num_unique_duplicate_keys
8852                    .load(Ordering::Relaxed),
8853                populate_duplicate_keys_us,
8854                total_including_duplicates: total_including_duplicates.load(Ordering::Relaxed),
8855                total_slots: slots.len() as u64,
8856                all_accounts_are_zero_lamports_slots: all_accounts_are_zero_lamports_slots
8857                    .load(Ordering::Relaxed),
8858                ..GenerateIndexTimings::default()
8859            };
8860
8861            if pass == 0 {
8862                #[derive(Debug, Default)]
8863                struct DuplicatePubkeysVisitedInfo {
8864                    accounts_data_len_from_duplicates: u64,
8865                    num_duplicate_accounts: u64,
8866                    duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
8867                }
8868                impl DuplicatePubkeysVisitedInfo {
8869                    fn reduce(mut self, other: Self) -> Self {
8870                        self.accounts_data_len_from_duplicates +=
8871                            other.accounts_data_len_from_duplicates;
8872                        self.num_duplicate_accounts += other.num_duplicate_accounts;
8873
8874                        match (
8875                            self.duplicates_lt_hash.is_some(),
8876                            other.duplicates_lt_hash.is_some(),
8877                        ) {
8878                            (true, true) => {
8879                                // SAFETY: We just checked that both values are Some
8880                                self.duplicates_lt_hash
8881                                    .as_mut()
8882                                    .unwrap()
8883                                    .0
8884                                    .mix_in(&other.duplicates_lt_hash.as_ref().unwrap().0);
8885                            }
8886                            (true, false) => {
8887                                // nothing to do; `other` doesn't have a duplicates lt hash
8888                            }
8889                            (false, true) => {
8890                                // `self` doesn't have a duplicates lt hash, so pilfer from `other`
8891                                self.duplicates_lt_hash = other.duplicates_lt_hash;
8892                            }
8893                            (false, false) => {
8894                                // nothing to do; no duplicates lt hash at all
8895                            }
8896                        }
8897                        self
8898                    }
8899                }
8900
8901                let zero_lamport_pubkeys_to_visit =
8902                    std::mem::take(&mut *zero_lamport_pubkeys.lock().unwrap());
8903                let (num_zero_lamport_single_refs, visit_zero_lamports_us) =
8904                    measure_us!(self
8905                        .visit_zero_lamport_pubkeys_during_startup(&zero_lamport_pubkeys_to_visit));
8906                timings.visit_zero_lamports_us = visit_zero_lamports_us;
8907                timings.num_zero_lamport_single_refs = num_zero_lamport_single_refs;
8908
8909                // subtract data.len() from accounts_data_len for all old accounts that are in the index twice
8910                let mut accounts_data_len_dedup_timer =
8911                    Measure::start("handle accounts data len duplicates");
8912                let DuplicatePubkeysVisitedInfo {
8913                    accounts_data_len_from_duplicates,
8914                    num_duplicate_accounts,
8915                    duplicates_lt_hash,
8916                } = unique_pubkeys_by_bin
8917                    .par_iter()
8918                    .fold(
8919                        DuplicatePubkeysVisitedInfo::default,
8920                        |accum, pubkeys_by_bin| {
8921                            let intermediate = pubkeys_by_bin
8922                                .par_chunks(4096)
8923                                .fold(DuplicatePubkeysVisitedInfo::default, |accum, pubkeys| {
8924                                    let (
8925                                        accounts_data_len_from_duplicates,
8926                                        accounts_duplicates_num,
8927                                        duplicates_lt_hash,
8928                                    ) = self.visit_duplicate_pubkeys_during_startup(
8929                                        pubkeys,
8930                                        &rent_collector,
8931                                        &timings,
8932                                        should_calculate_duplicates_lt_hash,
8933                                    );
8934                                    let intermediate = DuplicatePubkeysVisitedInfo {
8935                                        accounts_data_len_from_duplicates,
8936                                        num_duplicate_accounts: accounts_duplicates_num,
8937                                        duplicates_lt_hash,
8938                                    };
8939                                    DuplicatePubkeysVisitedInfo::reduce(accum, intermediate)
8940                                })
8941                                .reduce(
8942                                    DuplicatePubkeysVisitedInfo::default,
8943                                    DuplicatePubkeysVisitedInfo::reduce,
8944                                );
8945                            DuplicatePubkeysVisitedInfo::reduce(accum, intermediate)
8946                        },
8947                    )
8948                    .reduce(
8949                        DuplicatePubkeysVisitedInfo::default,
8950                        DuplicatePubkeysVisitedInfo::reduce,
8951                    );
8952                accounts_data_len_dedup_timer.stop();
8953                timings.accounts_data_len_dedup_time_us = accounts_data_len_dedup_timer.as_us();
8954                timings.num_duplicate_accounts = num_duplicate_accounts;
8955
8956                accounts_data_len.fetch_sub(accounts_data_len_from_duplicates, Ordering::Relaxed);
8957                if let Some(duplicates_lt_hash) = duplicates_lt_hash {
8958                    let old_val = outer_duplicates_lt_hash.replace(duplicates_lt_hash);
8959                    assert!(old_val.is_none());
8960                }
8961                info!(
8962                    "accounts data len: {}",
8963                    accounts_data_len.load(Ordering::Relaxed)
8964                );
8965
8966                // insert all zero lamport account storage into the dirty stores and add them into the uncleaned roots for clean to pick up
8967                let all_zero_slots_to_clean = std::mem::take(all_zeros_slots.get_mut().unwrap());
8968                info!(
8969                    "insert all zero slots to clean at startup {}",
8970                    all_zero_slots_to_clean.len()
8971                );
8972                for (slot, storage) in all_zero_slots_to_clean {
8973                    self.dirty_stores.insert(slot, storage);
8974                }
8975            }
8976
8977            if pass == 0 {
8978                // Need to add these last, otherwise older updates will be cleaned
8979                for root in &slots {
8980                    self.accounts_index.add_root(*root);
8981                }
8982
8983                self.set_storage_count_and_alive_bytes(storage_info, &mut timings);
8984            }
8985            total_time.stop();
8986            timings.total_time_us = total_time.as_us();
8987            timings.report(self.accounts_index.get_startup_stats());
8988        }
8989
8990        self.accounts_index.log_secondary_indexes();
8991
8992        // The duplicates lt hash must be Some if should_calculate_duplicates_lt_hash is true.
8993        // But, if there were no duplicates, then we'd never set outer_duplicates_lt_hash to Some!
8994        // So do one last check here to ensure outer_duplicates_lt_hash is Some if we're supposed
8995        // to calculate the duplicates lt hash.
8996        if should_calculate_duplicates_lt_hash && outer_duplicates_lt_hash.is_none() {
8997            outer_duplicates_lt_hash = Some(Box::new(DuplicatesLtHash::default()));
8998        }
8999
9000        IndexGenerationInfo {
9001            accounts_data_len: accounts_data_len.load(Ordering::Relaxed),
9002            rent_paying_accounts_by_partition: rent_paying_accounts_by_partition
9003                .into_inner()
9004                .unwrap(),
9005            duplicates_lt_hash: outer_duplicates_lt_hash,
9006        }
9007    }
9008
9009    /// Startup processes can consume large amounts of memory while inserting accounts into the index as fast as possible.
9010    /// Calling this can slow down the insertion process to allow flushing to disk to keep pace.
9011    fn maybe_throttle_index_generation(&self) {
9012        // This number is chosen to keep the initial ram usage sufficiently small
9013        // The process of generating the index is goverened entirely by how fast the disk index can be populated.
9014        // 10M accounts is sufficiently small that it will never have memory usage. It seems sufficiently large that it will provide sufficient performance.
9015        // Performance is measured by total time to generate the index.
9016        // 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.
9017        // 550M was straining a 384G machine at startup.
9018        // This is a tunable parameter that just needs to be small enough to keep the generation threads from overwhelming RAM and oom at startup.
9019        const LIMIT: usize = 10_000_000;
9020        while self
9021            .accounts_index
9022            .get_startup_remaining_items_to_flush_estimate()
9023            > LIMIT
9024        {
9025            // 10 ms is long enough to allow some flushing to occur before insertion is resumed.
9026            // callers of this are typically run in parallel, so many threads will be sleeping at different starting intervals, waiting to resume insertion.
9027            sleep(Duration::from_millis(10));
9028        }
9029    }
9030
9031    /// Visit zero lamport pubkeys and populate zero_lamport_single_ref info on
9032    /// storage.
9033    /// Returns the number of zero lamport single ref accounts found.
9034    fn visit_zero_lamport_pubkeys_during_startup(&self, pubkeys: &HashSet<Pubkey>) -> u64 {
9035        let mut count = 0;
9036        self.accounts_index.scan(
9037            pubkeys.iter(),
9038            |_pubkey, slots_refs, _entry| {
9039                let (slot_list, ref_count) = slots_refs.unwrap();
9040                if ref_count == 1 {
9041                    assert_eq!(slot_list.len(), 1);
9042                    let (slot_alive, account_info) = slot_list.first().unwrap();
9043                    assert!(!account_info.is_cached());
9044                    if account_info.is_zero_lamport() {
9045                        count += 1;
9046                        self.zero_lamport_single_ref_found(*slot_alive, account_info.offset());
9047                    }
9048                }
9049                AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
9050            },
9051            None,
9052            false,
9053            ScanFilter::All,
9054        );
9055        count
9056    }
9057
9058    /// Used during generate_index() to:
9059    /// 1. get the _duplicate_ accounts data len from the given pubkeys
9060    /// 2. get the slots that contained duplicate pubkeys
9061    /// 3. update rent stats
9062    /// 4. build up the duplicates lt hash
9063    ///
9064    /// Note this should only be used when ALL entries in the accounts index are roots.
9065    ///
9066    /// returns tuple of:
9067    /// - data len sum of all older duplicates
9068    /// - number of duplicate accounts
9069    /// - slots that contained duplicate pubkeys
9070    /// - lt hash of duplicates
9071    fn visit_duplicate_pubkeys_during_startup(
9072        &self,
9073        pubkeys: &[Pubkey],
9074        rent_collector: &RentCollector,
9075        timings: &GenerateIndexTimings,
9076        should_calculate_duplicates_lt_hash: bool,
9077    ) -> (u64, u64, Option<Box<DuplicatesLtHash>>) {
9078        let mut accounts_data_len_from_duplicates = 0;
9079        let mut num_duplicate_accounts = 0_u64;
9080        let mut duplicates_lt_hash =
9081            should_calculate_duplicates_lt_hash.then(|| Box::new(DuplicatesLtHash::default()));
9082        let mut removed_rent_paying = 0;
9083        let mut removed_top_off = 0;
9084        let mut lt_hash_time = Duration::default();
9085        self.accounts_index.scan(
9086            pubkeys.iter(),
9087            |pubkey, slots_refs, _entry| {
9088                if let Some((slot_list, _ref_count)) = slots_refs {
9089                    if slot_list.len() > 1 {
9090                        // Only the account data len in the highest slot should be used, and the rest are
9091                        // duplicates.  So find the max slot to keep.
9092                        // Then sum up the remaining data len, which are the duplicates.
9093                        // All of the slots need to go in the 'uncleaned_slots' list. For clean to work properly,
9094                        // the slot where duplicate accounts are found in the index need to be in 'uncleaned_slots' list, too.
9095                        let max = slot_list.iter().map(|(slot, _)| slot).max().unwrap();
9096                        slot_list.iter().for_each(|(slot, account_info)| {
9097                            if slot == max {
9098                                // the info in 'max' is the most recent, current info for this pubkey
9099                                return;
9100                            }
9101                            let maybe_storage_entry = self
9102                                .storage
9103                                .get_account_storage_entry(*slot, account_info.store_id());
9104                            let mut accessor = LoadedAccountAccessor::Stored(
9105                                maybe_storage_entry.map(|entry| (entry, account_info.offset())),
9106                            );
9107                            accessor.check_and_get_loaded_account(|loaded_account| {
9108                                let data_len = loaded_account.data_len();
9109                                if loaded_account.lamports() > 0 {
9110                                    accounts_data_len_from_duplicates += data_len;
9111                                }
9112                                num_duplicate_accounts += 1;
9113                                if let Some(lamports_to_top_off) = Self::stats_for_rent_payers(
9114                                    pubkey,
9115                                    loaded_account.lamports(),
9116                                    data_len,
9117                                    loaded_account.rent_epoch(),
9118                                    loaded_account.executable(),
9119                                    rent_collector,
9120                                ) {
9121                                    removed_rent_paying += 1;
9122                                    removed_top_off += lamports_to_top_off;
9123                                }
9124                                if let Some(duplicates_lt_hash) = duplicates_lt_hash.as_mut() {
9125                                    let (_, duration) = meas_dur!({
9126                                        let account_lt_hash =
9127                                            Self::lt_hash_account(&loaded_account, pubkey);
9128                                        duplicates_lt_hash.0.mix_in(&account_lt_hash.0);
9129                                    });
9130                                    lt_hash_time += duration;
9131                                }
9132                            });
9133                        });
9134                    }
9135                }
9136                AccountsIndexScanResult::OnlyKeepInMemoryIfDirty
9137            },
9138            None,
9139            false,
9140            ScanFilter::All,
9141        );
9142        timings
9143            .rent_paying
9144            .fetch_sub(removed_rent_paying, Ordering::Relaxed);
9145        timings
9146            .amount_to_top_off_rent
9147            .fetch_sub(removed_top_off, Ordering::Relaxed);
9148        timings
9149            .par_duplicates_lt_hash_us
9150            .fetch_add(lt_hash_time.as_micros() as u64, Ordering::Relaxed);
9151        (
9152            accounts_data_len_from_duplicates as u64,
9153            num_duplicate_accounts,
9154            duplicates_lt_hash,
9155        )
9156    }
9157
9158    fn set_storage_count_and_alive_bytes(
9159        &self,
9160        stored_sizes_and_counts: StorageSizeAndCountMap,
9161        timings: &mut GenerateIndexTimings,
9162    ) {
9163        // store count and size for each storage
9164        let mut storage_size_storages_time = Measure::start("storage_size_storages");
9165        for (_slot, store) in self.storage.iter() {
9166            let id = store.id();
9167            // Should be default at this point
9168            assert_eq!(store.alive_bytes(), 0);
9169            if let Some(entry) = stored_sizes_and_counts.get(&id) {
9170                trace!(
9171                    "id: {} setting count: {} cur: {}",
9172                    id,
9173                    entry.count,
9174                    store.count(),
9175                );
9176                {
9177                    let mut count_and_status = store.count_and_status.lock_write();
9178                    assert_eq!(count_and_status.0, 0);
9179                    count_and_status.0 = entry.count;
9180                }
9181                store
9182                    .alive_bytes
9183                    .store(entry.stored_size, Ordering::Release);
9184            } else {
9185                trace!("id: {} clearing count", id);
9186                store.count_and_status.lock_write().0 = 0;
9187            }
9188        }
9189        storage_size_storages_time.stop();
9190        timings.storage_size_storages_us = storage_size_storages_time.as_us();
9191    }
9192
9193    pub fn print_accounts_stats(&self, label: &str) {
9194        self.print_index(label);
9195        self.print_count_and_status(label);
9196    }
9197
9198    fn print_index(&self, label: &str) {
9199        let mut alive_roots: Vec<_> = self.accounts_index.all_alive_roots();
9200        #[allow(clippy::stable_sort_primitive)]
9201        alive_roots.sort();
9202        info!("{}: accounts_index alive_roots: {:?}", label, alive_roots,);
9203        let full_pubkey_range = Pubkey::from([0; 32])..=Pubkey::from([0xff; 32]);
9204
9205        self.accounts_index.account_maps.iter().for_each(|map| {
9206            for (pubkey, account_entry) in map.items(&full_pubkey_range) {
9207                info!("  key: {} ref_count: {}", pubkey, account_entry.ref_count(),);
9208                info!(
9209                    "      slots: {:?}",
9210                    *account_entry.slot_list.read().unwrap()
9211                );
9212            }
9213        });
9214    }
9215
9216    pub fn print_count_and_status(&self, label: &str) {
9217        let mut slots: Vec<_> = self.storage.all_slots();
9218        #[allow(clippy::stable_sort_primitive)]
9219        slots.sort();
9220        info!("{}: count_and status for {} slots:", label, slots.len());
9221        for slot in &slots {
9222            let entry = self.storage.get_slot_storage_entry(*slot).unwrap();
9223            info!(
9224                "  slot: {} id: {} count_and_status: {:?} len: {} capacity: {}",
9225                slot,
9226                entry.id(),
9227                entry.count_and_status.read(),
9228                entry.accounts.len(),
9229                entry.accounts.capacity(),
9230            );
9231        }
9232    }
9233}
9234
9235/// Specify the source of the accounts data when calculating the accounts hash
9236///
9237/// Using the Index is meant for testing the hash calculation itself and debugging;
9238/// not intended during normal validator operation.
9239#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9240pub enum CalcAccountsHashDataSource {
9241    IndexForTests,
9242    Storages,
9243}
9244
9245#[derive(Debug, Copy, Clone)]
9246enum HandleReclaims<'a> {
9247    ProcessDeadSlots(&'a PurgeStats),
9248    DoNotProcessDeadSlots,
9249}
9250
9251/// Which accounts hash calculation is being performed?
9252#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9253pub enum CalcAccountsHashKind {
9254    Full,
9255    Incremental,
9256}
9257
9258impl CalcAccountsHashKind {
9259    /// How should zero-lamport accounts be handled by this accounts hash calculation?
9260    fn zero_lamport_accounts(&self) -> ZeroLamportAccounts {
9261        match self {
9262            CalcAccountsHashKind::Full => ZeroLamportAccounts::Excluded,
9263            CalcAccountsHashKind::Incremental => ZeroLamportAccounts::Included,
9264        }
9265    }
9266}
9267
9268pub(crate) enum UpdateIndexThreadSelection {
9269    /// Use current thread only
9270    Inline,
9271    /// Use a thread-pool if the number of updates exceeds a threshold
9272    PoolWithThreshold,
9273}
9274
9275/// How should old storages be handled in clean_accounts()?
9276#[derive(Debug, Copy, Clone, Eq, PartialEq)]
9277pub enum OldStoragesPolicy {
9278    /// Clean all old storages, even if they were not explictly marked as dirty.
9279    ///
9280    /// This is the default behavior when not skipping rewrites.
9281    Clean,
9282    /// Leave all old storages.
9283    ///
9284    /// When skipping rewrites, we intentionally will have ancient storages.
9285    /// Do not clean them up automatically in clean_accounts().
9286    Leave,
9287}
9288
9289// These functions/fields are only usable from a dev context (i.e. tests and benches)
9290#[cfg(feature = "dev-context-only-utils")]
9291impl AccountStorageEntry {
9292    fn accounts_count(&self) -> usize {
9293        let mut count = 0;
9294        self.accounts.scan_pubkeys(|_| {
9295            count += 1;
9296        });
9297        count
9298    }
9299}
9300
9301// These functions/fields are only usable from a dev context (i.e. tests and benches)
9302#[cfg(feature = "dev-context-only-utils")]
9303impl AccountsDb {
9304    /// Return the number of slots marked with uncleaned pubkeys.
9305    /// This is useful for testing clean aglorithms.
9306    pub fn get_len_of_slots_with_uncleaned_pubkeys(&self) -> usize {
9307        self.uncleaned_pubkeys.len()
9308    }
9309
9310    /// useful to adapt tests written prior to introduction of the write cache
9311    /// to use the write cache
9312    pub fn add_root_and_flush_write_cache(&self, slot: Slot) {
9313        self.add_root(slot);
9314        self.flush_root_write_cache(slot);
9315    }
9316
9317    /// Wrapper function to calculate accounts delta hash for `slot` (only used for testing and benchmarking.)
9318    pub fn calculate_accounts_delta_hash(&self, slot: Slot) -> AccountsDeltaHash {
9319        self.calculate_accounts_delta_hash_internal(slot, None, HashMap::default())
9320    }
9321
9322    pub fn load_without_fixed_root(
9323        &self,
9324        ancestors: &Ancestors,
9325        pubkey: &Pubkey,
9326    ) -> Option<(AccountSharedData, Slot)> {
9327        self.do_load(
9328            ancestors,
9329            pubkey,
9330            None,
9331            LoadHint::Unspecified,
9332            // callers of this expect zero lamport accounts that exist in the index to be returned as Some(empty)
9333            LoadZeroLamports::SomeWithZeroLamportAccountForTests,
9334        )
9335    }
9336
9337    pub fn accounts_delta_hashes(&self) -> &Mutex<HashMap<Slot, AccountsDeltaHash>> {
9338        &self.accounts_delta_hashes
9339    }
9340
9341    pub fn accounts_hashes(&self) -> &Mutex<HashMap<Slot, (AccountsHash, /*capitalization*/ u64)>> {
9342        &self.accounts_hashes
9343    }
9344
9345    pub fn assert_load_account(&self, slot: Slot, pubkey: Pubkey, expected_lamports: u64) {
9346        let ancestors = vec![(slot, 0)].into_iter().collect();
9347        let (account, slot) = self.load_without_fixed_root(&ancestors, &pubkey).unwrap();
9348        assert_eq!((account.lamports(), slot), (expected_lamports, slot));
9349    }
9350
9351    pub fn assert_not_load_account(&self, slot: Slot, pubkey: Pubkey) {
9352        let ancestors = vec![(slot, 0)].into_iter().collect();
9353        let load = self.load_without_fixed_root(&ancestors, &pubkey);
9354        assert!(load.is_none(), "{load:?}");
9355    }
9356
9357    pub fn check_accounts(&self, pubkeys: &[Pubkey], slot: Slot, num: usize, count: usize) {
9358        let ancestors = vec![(slot, 0)].into_iter().collect();
9359        for _ in 0..num {
9360            let idx = thread_rng().gen_range(0..num);
9361            let account = self.load_without_fixed_root(&ancestors, &pubkeys[idx]);
9362            let account1 = Some((
9363                AccountSharedData::new(
9364                    (idx + count) as u64,
9365                    0,
9366                    AccountSharedData::default().owner(),
9367                ),
9368                slot,
9369            ));
9370            assert_eq!(account, account1);
9371        }
9372    }
9373
9374    /// callers used to call store_uncached. But, this is not allowed anymore.
9375    pub fn store_for_tests(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
9376        self.store(
9377            (slot, accounts),
9378            &StoreTo::Cache,
9379            None,
9380            StoreReclaims::Default,
9381            UpdateIndexThreadSelection::PoolWithThreshold,
9382        );
9383    }
9384
9385    #[allow(clippy::needless_range_loop)]
9386    pub fn modify_accounts(&self, pubkeys: &[Pubkey], slot: Slot, num: usize, count: usize) {
9387        for idx in 0..num {
9388            let account = AccountSharedData::new(
9389                (idx + count) as u64,
9390                0,
9391                AccountSharedData::default().owner(),
9392            );
9393            self.store_for_tests(slot, &[(&pubkeys[idx], &account)]);
9394        }
9395    }
9396
9397    pub fn check_storage(&self, slot: Slot, alive_count: usize, total_count: usize) {
9398        let store = self.storage.get_slot_storage_entry(slot).unwrap();
9399        assert_eq!(store.status(), AccountStorageStatus::Available);
9400        assert_eq!(store.count(), alive_count);
9401        assert_eq!(store.accounts_count(), total_count);
9402    }
9403
9404    pub fn create_account(
9405        &self,
9406        pubkeys: &mut Vec<Pubkey>,
9407        slot: Slot,
9408        num: usize,
9409        space: usize,
9410        num_vote: usize,
9411    ) {
9412        let ancestors = vec![(slot, 0)].into_iter().collect();
9413        for t in 0..num {
9414            let pubkey = solana_pubkey::new_rand();
9415            let account =
9416                AccountSharedData::new((t + 1) as u64, space, AccountSharedData::default().owner());
9417            pubkeys.push(pubkey);
9418            assert!(self.load_without_fixed_root(&ancestors, &pubkey).is_none());
9419            self.store_for_tests(slot, &[(&pubkey, &account)]);
9420        }
9421        for t in 0..num_vote {
9422            let pubkey = solana_pubkey::new_rand();
9423            let account =
9424                AccountSharedData::new((num + t + 1) as u64, space, &solana_vote_program::id());
9425            pubkeys.push(pubkey);
9426            let ancestors = vec![(slot, 0)].into_iter().collect();
9427            assert!(self.load_without_fixed_root(&ancestors, &pubkey).is_none());
9428            self.store_for_tests(slot, &[(&pubkey, &account)]);
9429        }
9430    }
9431
9432    pub fn sizes_of_accounts_in_storage_for_tests(&self, slot: Slot) -> Vec<usize> {
9433        let mut sizes = Vec::default();
9434        if let Some(storage) = self.storage.get_slot_storage_entry(slot) {
9435            storage.accounts.scan_accounts(|account| {
9436                sizes.push(account.stored_size());
9437            });
9438        }
9439        sizes
9440    }
9441
9442    pub fn ref_count_for_pubkey(&self, pubkey: &Pubkey) -> RefCount {
9443        self.accounts_index.ref_count_from_storage(pubkey)
9444    }
9445
9446    pub fn alive_account_count_in_slot(&self, slot: Slot) -> usize {
9447        self.storage
9448            .get_slot_storage_entry(slot)
9449            .map(|storage| storage.count())
9450            .unwrap_or(0)
9451            .saturating_add(
9452                self.accounts_cache
9453                    .slot_cache(slot)
9454                    .map(|slot_cache| slot_cache.len())
9455                    .unwrap_or_default(),
9456            )
9457    }
9458
9459    /// useful to adapt tests written prior to introduction of the write cache
9460    /// to use the write cache
9461    pub fn flush_root_write_cache(&self, root: Slot) {
9462        assert!(
9463            self.accounts_index
9464                .roots_tracker
9465                .read()
9466                .unwrap()
9467                .alive_roots
9468                .contains(&root),
9469            "slot: {root}"
9470        );
9471        self.flush_accounts_cache(true, Some(root));
9472    }
9473
9474    pub fn all_account_count_in_accounts_file(&self, slot: Slot) -> usize {
9475        let store = self.storage.get_slot_storage_entry(slot);
9476        if let Some(store) = store {
9477            store.accounts_count()
9478        } else {
9479            0
9480        }
9481    }
9482
9483    pub fn verify_accounts_hash_and_lamports_for_tests(
9484        &self,
9485        slot: Slot,
9486        total_lamports: u64,
9487        config: VerifyAccountsHashAndLamportsConfig,
9488    ) -> Result<(), AccountsHashVerificationError> {
9489        let snapshot_storages = self.get_storages(..);
9490        let snapshot_storages_and_slots = (
9491            snapshot_storages.0.as_slice(),
9492            snapshot_storages.1.as_slice(),
9493        );
9494        self.verify_accounts_hash_and_lamports(
9495            snapshot_storages_and_slots,
9496            slot,
9497            total_lamports,
9498            None,
9499            config,
9500        )
9501    }
9502
9503    pub fn uncleaned_pubkeys(&self) -> &DashMap<Slot, Vec<Pubkey>, BuildNoHashHasher<Slot>> {
9504        &self.uncleaned_pubkeys
9505    }
9506}
9507
9508// These functions/fields are only usable from a dev context (i.e. tests and benches)
9509#[cfg(feature = "dev-context-only-utils")]
9510impl<'a> VerifyAccountsHashAndLamportsConfig<'a> {
9511    pub fn new_for_test(
9512        ancestors: &'a Ancestors,
9513        epoch_schedule: &'a EpochSchedule,
9514        rent_collector: &'a RentCollector,
9515    ) -> Self {
9516        Self {
9517            ancestors,
9518            test_hash_calculation: true,
9519            epoch_schedule,
9520            rent_collector,
9521            ignore_mismatch: false,
9522            store_detailed_debug_info: false,
9523            use_bg_thread_pool: false,
9524        }
9525    }
9526}
9527
9528/// A set of utility functions used for testing and benchmarking
9529#[cfg(feature = "dev-context-only-utils")]
9530pub mod test_utils {
9531    use {
9532        super::*,
9533        crate::{accounts::Accounts, append_vec::aligned_stored_size},
9534    };
9535
9536    pub fn create_test_accounts(
9537        accounts: &Accounts,
9538        pubkeys: &mut Vec<Pubkey>,
9539        num: usize,
9540        slot: Slot,
9541    ) {
9542        let data_size = 0;
9543        if accounts
9544            .accounts_db
9545            .storage
9546            .get_slot_storage_entry(slot)
9547            .is_none()
9548        {
9549            // Some callers relied on old behavior where the file size was rounded up to the
9550            // next page size because they append to the storage file after it was written.
9551            // This behavior is not supported by a normal running validator.  Since this function
9552            // is only called by tests/benches, add some extra capacity to the file to not break
9553            // the tests/benches.  Those tests/benches should be updated though!  Bypassing the
9554            // write cache in general is not supported.
9555            let bytes_required = num * aligned_stored_size(data_size) + 4096;
9556            // 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.
9557            _ = accounts.accounts_db.create_and_insert_store(
9558                slot,
9559                bytes_required as u64,
9560                "create_test_accounts",
9561            );
9562        }
9563
9564        for t in 0..num {
9565            let pubkey = solana_pubkey::new_rand();
9566            let account = AccountSharedData::new(
9567                (t + 1) as u64,
9568                data_size,
9569                AccountSharedData::default().owner(),
9570            );
9571            accounts.store_slow_uncached(slot, &pubkey, &account);
9572            pubkeys.push(pubkey);
9573        }
9574    }
9575
9576    // Only used by bench, not safe to call otherwise accounts can conflict with the
9577    // accounts cache!
9578    pub fn update_accounts_bench(accounts: &Accounts, pubkeys: &[Pubkey], slot: u64) {
9579        for pubkey in pubkeys {
9580            let amount = thread_rng().gen_range(0..10);
9581            let account = AccountSharedData::new(amount, 0, AccountSharedData::default().owner());
9582            accounts.store_slow_uncached(slot, pubkey, &account);
9583        }
9584    }
9585}