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