solana_runtime/
accounts_db.rs

1//! Persistent accounts are stored in below path location:
2//!  <path>/<pid>/data/
3//!
4//! The persistent store would allow for this mode of operation:
5//!  - Concurrent single thread append with many concurrent readers.
6//!
7//! The underlying memory is memory mapped to a file. The accounts would be
8//! stored across multiple files and the mappings of file and offset of a
9//! particular account would be stored in a shared index. This will allow for
10//! concurrent commits without blocking reads, which will sequentially write
11//! to memory, ssd or disk, and should be as fast as the hardware allow for.
12//! The only required in memory data structure with a write lock is the index,
13//! which should be fast to update.
14//!
15//! AppendVec's only store accounts for single slots.  To bootstrap the
16//! index from a persistent store of AppendVec's, the entries include
17//! a "write_version".  A single global atomic `AccountsDb::write_version`
18//! tracks the number of commits to the entire data store. So the latest
19//! commit for each slot entry would be indexed.
20
21use {
22    crate::{
23        account_info::{AccountInfo, Offset, StorageLocation, StoredSize},
24        accounts_background_service::{DroppedSlotsSender, SendDroppedBankCallback},
25        accounts_cache::{AccountsCache, CachedAccount, SlotCache},
26        accounts_hash::{
27            AccountsHash, CalcAccountsHashConfig, CalculateHashIntermediate, HashStats,
28            PreviousPass,
29        },
30        accounts_index::{
31            AccountIndexGetResult, AccountSecondaryIndexes, AccountsIndex, AccountsIndexConfig,
32            AccountsIndexRootsStats, AccountsIndexScanResult, IndexKey, IndexValue, IsCached,
33            RefCount, ScanConfig, ScanResult, SlotList, UpsertReclaim, ZeroLamport,
34            ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS, ACCOUNTS_INDEX_CONFIG_FOR_TESTING,
35        },
36        accounts_index_storage::Startup,
37        accounts_update_notifier_interface::AccountsUpdateNotifier,
38        active_stats::{ActiveStatItem, ActiveStats},
39        ancestors::Ancestors,
40        ancient_append_vecs::{
41            get_ancient_append_vec_capacity, is_ancient, is_full_ancient, AccountsToStore,
42            StorageSelector,
43        },
44        append_vec::{AppendVec, StoredAccountMeta, StoredMeta, StoredMetaWriteVersion},
45        bank::Rewrites,
46        cache_hash_data::CacheHashData,
47        contains::Contains,
48        pubkey_bins::PubkeyBinCalculator24,
49        read_only_accounts_cache::ReadOnlyAccountsCache,
50        rent_collector::RentCollector,
51        rent_paying_accounts_by_partition::RentPayingAccountsByPartition,
52        sorted_storages::SortedStorages,
53        storable_accounts::StorableAccounts,
54        verify_accounts_hash_in_background::VerifyAccountsHashInBackground,
55    },
56    blake3::traits::digest::Digest,
57    crossbeam_channel::{unbounded, Receiver, Sender},
58    dashmap::{
59        mapref::entry::Entry::{Occupied, Vacant},
60        DashMap, DashSet,
61    },
62    log::*,
63    rand::{thread_rng, Rng},
64    rayon::{prelude::*, ThreadPool},
65    serde::{Deserialize, Serialize},
66    safecoin_measure::measure::Measure,
67    safecoin_rayon_threadlimit::get_thread_count,
68    solana_sdk::{
69        account::{Account, AccountSharedData, ReadableAccount, WritableAccount},
70        clock::{BankId, Epoch, Slot, SlotCount},
71        epoch_schedule::EpochSchedule,
72        genesis_config::{ClusterType, GenesisConfig},
73        hash::Hash,
74        pubkey::Pubkey,
75        rent::Rent,
76        signature::Signature,
77        timing::AtomicInterval,
78    },
79    std::{
80        borrow::{Borrow, Cow},
81        boxed::Box,
82        collections::{hash_map::Entry, BTreeSet, HashMap, HashSet},
83        convert::TryFrom,
84        hash::{Hash as StdHash, Hasher as StdHasher},
85        io::{Error as IoError, Result as IoResult},
86        ops::{Range, RangeBounds},
87        path::{Path, PathBuf},
88        str::FromStr,
89        sync::{
90            atomic::{AtomicBool, AtomicU32, AtomicU64, AtomicUsize, Ordering},
91            Arc, Condvar, Mutex, MutexGuard, RwLock,
92        },
93        thread::{sleep, Builder},
94        time::{Duration, Instant},
95    },
96    tempfile::TempDir,
97};
98
99const PAGE_SIZE: u64 = 4 * 1024;
100const MAX_RECYCLE_STORES: usize = 1000;
101const STORE_META_OVERHEAD: usize = 256;
102// when the accounts write cache exceeds this many bytes, we will flush it
103// this can be specified on the command line, too (--accounts-db-cache-limit-mb)
104const WRITE_CACHE_LIMIT_BYTES_DEFAULT: u64 = 15_000_000_000;
105const SCAN_SLOT_PAR_ITER_THRESHOLD: usize = 4000;
106
107pub const DEFAULT_FILE_SIZE: u64 = PAGE_SIZE * 1024;
108pub const DEFAULT_NUM_THREADS: u32 = 8;
109pub const DEFAULT_NUM_DIRS: u32 = 4;
110
111// When calculating hashes, it is helpful to break the pubkeys found into bins based on the pubkey value.
112// More bins means smaller vectors to sort, copy, etc.
113pub const PUBKEY_BINS_FOR_CALCULATING_HASHES: usize = 65536;
114pub const NUM_SCAN_PASSES_DEFAULT: usize = 2;
115
116// Without chunks, we end up with 1 output vec for each outer snapshot storage.
117// This results in too many vectors to be efficient.
118// Chunks when scanning storages to calculate hashes.
119// If this is too big, we don't get enough parallelism of scanning storages.
120// If this is too small, then we produce too many output vectors to iterate.
121// Metrics indicate a sweet spot in the 2.5k-5k range for mnb.
122const MAX_ITEMS_PER_CHUNK: Slot = 2_500;
123
124// A specially reserved write version (identifier for ordering writes in an AppendVec)
125// for entries in the cache, so that  operations that take a storage entry can maintain
126// a common interface when interacting with cached accounts. This version is "virtual" in
127// that it doesn't actually map to an entry in an AppendVec.
128const CACHE_VIRTUAL_WRITE_VERSION: StoredMetaWriteVersion = 0;
129
130// A specially reserved offset (represents an offset into an AppendVec)
131// for entries in the cache, so that  operations that take a storage entry can maintain
132// a common interface when interacting with cached accounts. This version is "virtual" in
133// that it doesn't actually map to an entry in an AppendVec.
134pub(crate) const CACHE_VIRTUAL_OFFSET: Offset = 0;
135const CACHE_VIRTUAL_STORED_SIZE: StoredSize = 0;
136
137pub enum StoreReclaims {
138    /// normal reclaim mode
139    Default,
140    /// do not return reclaims from accounts index upsert
141    Ignore,
142}
143
144/// specifies how to return zero lamport accounts
145/// This will only be useful until a feature activation occurs.
146#[derive(Clone, Copy)]
147pub enum LoadZeroLamports {
148    /// return None if loaded account has zero lamports
149    None,
150    /// return Some(account with zero lamports) if loaded account has zero lamports
151    /// Today this is the default. With feature activation, this will no longer be possible.
152    SomeWithZeroLamportAccount,
153}
154
155// the current best way to add filler accounts is gradually.
156// In other scenarios, such as monitoring catchup with large # of accounts, it may be useful to be able to
157// add filler accounts at the beginning, so that code path remains but won't execute at the moment.
158const ADD_FILLER_ACCOUNTS_GRADUALLY: bool = true;
159
160pub const ACCOUNTS_DB_CONFIG_FOR_TESTING: AccountsDbConfig = AccountsDbConfig {
161    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_TESTING),
162    accounts_hash_cache_path: None,
163    filler_accounts_config: FillerAccountsConfig::const_default(),
164    hash_calc_num_passes: None,
165    write_cache_limit_bytes: None,
166    skip_rewrites: false,
167    ancient_append_vecs: false,
168    skip_initial_hash_calc: false,
169};
170pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig {
171    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS),
172    accounts_hash_cache_path: None,
173    filler_accounts_config: FillerAccountsConfig::const_default(),
174    hash_calc_num_passes: None,
175    write_cache_limit_bytes: None,
176    skip_rewrites: false,
177    ancient_append_vecs: false,
178    skip_initial_hash_calc: false,
179};
180
181pub type BinnedHashData = Vec<Vec<CalculateHashIntermediate>>;
182
183pub struct GetUniqueAccountsResult<'a> {
184    pub stored_accounts: HashMap<Pubkey, FoundStoredAccount<'a>>,
185    pub original_bytes: u64,
186    store_ids: Vec<AppendVecId>,
187}
188
189pub struct AccountsAddRootTiming {
190    pub index_us: u64,
191    pub cache_us: u64,
192    pub store_us: u64,
193}
194
195#[derive(Debug, Clone, Copy)]
196pub struct FillerAccountsConfig {
197    /// Number of filler accounts
198    pub count: usize,
199    /// Data size per account, in bytes
200    pub size: usize,
201}
202
203impl FillerAccountsConfig {
204    pub const fn const_default() -> Self {
205        Self { count: 0, size: 0 }
206    }
207}
208
209impl Default for FillerAccountsConfig {
210    fn default() -> Self {
211        Self::const_default()
212    }
213}
214
215#[derive(Debug, Default, Clone)]
216pub struct AccountsDbConfig {
217    pub index: Option<AccountsIndexConfig>,
218    pub accounts_hash_cache_path: Option<PathBuf>,
219    pub filler_accounts_config: FillerAccountsConfig,
220    pub hash_calc_num_passes: Option<usize>,
221    pub write_cache_limit_bytes: Option<u64>,
222    pub skip_rewrites: bool,
223    pub ancient_append_vecs: bool,
224    pub skip_initial_hash_calc: bool,
225}
226
227pub struct FoundStoredAccount<'a> {
228    pub account: StoredAccountMeta<'a>,
229    pub store_id: AppendVecId,
230}
231
232#[cfg(not(test))]
233const ABSURD_CONSECUTIVE_FAILED_ITERATIONS: usize = 100;
234
235type DashMapVersionHash = DashMap<Pubkey, (u64, Hash)>;
236
237#[derive(Debug, Clone, Copy)]
238pub enum AccountShrinkThreshold {
239    /// Measure the total space sparseness across all candidates
240    /// And select the candidates by using the top sparse account storage entries to shrink.
241    /// The value is the overall shrink threshold measured as ratio of the total live bytes
242    /// over the total bytes.
243    TotalSpace { shrink_ratio: f64 },
244    /// Use the following option to shrink all stores whose alive ratio is below
245    /// the specified threshold.
246    IndividualStore { shrink_ratio: f64 },
247}
248pub const DEFAULT_ACCOUNTS_SHRINK_OPTIMIZE_TOTAL_SPACE: bool = true;
249pub const DEFAULT_ACCOUNTS_SHRINK_RATIO: f64 = 0.80;
250// The default extra account space in percentage from the ideal target
251const DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION: AccountShrinkThreshold =
252    AccountShrinkThreshold::TotalSpace {
253        shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_RATIO,
254    };
255
256impl Default for AccountShrinkThreshold {
257    fn default() -> AccountShrinkThreshold {
258        DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION
259    }
260}
261
262pub enum ScanStorageResult<R, B> {
263    Cached(Vec<R>),
264    Stored(B),
265}
266
267#[derive(Debug, Default)]
268pub struct IndexGenerationInfo {
269    pub accounts_data_len: u64,
270    pub rent_paying_accounts_by_partition: RentPayingAccountsByPartition,
271}
272
273#[derive(Debug, Default)]
274struct SlotIndexGenerationInfo {
275    insert_time_us: u64,
276    num_accounts: u64,
277    num_accounts_rent_paying: usize,
278    accounts_data_len: u64,
279    amount_to_top_off_rent: u64,
280    rent_paying_accounts_by_partition: Vec<Pubkey>,
281}
282
283#[derive(Default, Debug)]
284struct GenerateIndexTimings {
285    pub index_time: u64,
286    pub scan_time: u64,
287    pub insertion_time_us: u64,
288    pub min_bin_size: usize,
289    pub max_bin_size: usize,
290    pub total_items: usize,
291    pub storage_size_accounts_map_us: u64,
292    pub storage_size_storages_us: u64,
293    pub storage_size_accounts_map_flatten_us: u64,
294    pub index_flush_us: u64,
295    pub rent_paying: AtomicUsize,
296    pub amount_to_top_off_rent: AtomicU64,
297    pub total_duplicates: u64,
298    pub accounts_data_len_dedup_time_us: u64,
299}
300
301#[derive(Default, Debug, PartialEq, Eq)]
302struct StorageSizeAndCount {
303    pub stored_size: usize,
304    pub count: usize,
305}
306type StorageSizeAndCountMap = DashMap<AppendVecId, StorageSizeAndCount>;
307
308impl GenerateIndexTimings {
309    pub fn report(&self) {
310        datapoint_info!(
311            "generate_index",
312            // we cannot accurately measure index insertion time because of many threads and lock contention
313            ("total_us", self.index_time, i64),
314            ("scan_stores_us", self.scan_time, i64),
315            ("insertion_time_us", self.insertion_time_us, i64),
316            ("min_bin_size", self.min_bin_size as i64, i64),
317            ("max_bin_size", self.max_bin_size as i64, i64),
318            (
319                "storage_size_accounts_map_us",
320                self.storage_size_accounts_map_us as i64,
321                i64
322            ),
323            (
324                "storage_size_storages_us",
325                self.storage_size_storages_us as i64,
326                i64
327            ),
328            (
329                "storage_size_accounts_map_flatten_us",
330                self.storage_size_accounts_map_flatten_us as i64,
331                i64
332            ),
333            ("index_flush_us", self.index_flush_us as i64, i64),
334            (
335                "total_rent_paying",
336                self.rent_paying.load(Ordering::Relaxed) as i64,
337                i64
338            ),
339            (
340                "amount_to_top_off_rent",
341                self.amount_to_top_off_rent.load(Ordering::Relaxed) as i64,
342                i64
343            ),
344            (
345                "total_items_with_duplicates",
346                self.total_duplicates as i64,
347                i64
348            ),
349            ("total_items", self.total_items as i64, i64),
350            (
351                "accounts_data_len_dedup_time_us",
352                self.accounts_data_len_dedup_time_us as i64,
353                i64
354            ),
355        );
356    }
357}
358
359impl IndexValue for AccountInfo {}
360
361impl ZeroLamport for AccountSharedData {
362    fn is_zero_lamport(&self) -> bool {
363        self.lamports() == 0
364    }
365}
366
367impl ZeroLamport for Account {
368    fn is_zero_lamport(&self) -> bool {
369        self.lamports() == 0
370    }
371}
372
373struct MultiThreadProgress<'a> {
374    last_update: Instant,
375    my_last_report_count: u64,
376    total_count: &'a AtomicU64,
377    report_delay_secs: u64,
378    first_caller: bool,
379    ultimate_count: u64,
380}
381
382impl<'a> MultiThreadProgress<'a> {
383    fn new(total_count: &'a AtomicU64, report_delay_secs: u64, ultimate_count: u64) -> Self {
384        Self {
385            last_update: Instant::now(),
386            my_last_report_count: 0,
387            total_count,
388            report_delay_secs,
389            first_caller: false,
390            ultimate_count,
391        }
392    }
393    fn report(&mut self, my_current_count: u64) {
394        let now = Instant::now();
395        if now.duration_since(self.last_update).as_secs() >= self.report_delay_secs {
396            let my_total_newly_processed_slots_since_last_report =
397                my_current_count - self.my_last_report_count;
398
399            self.my_last_report_count = my_current_count;
400            let previous_total_processed_slots_across_all_threads = self.total_count.fetch_add(
401                my_total_newly_processed_slots_since_last_report,
402                Ordering::Relaxed,
403            );
404            self.first_caller =
405                self.first_caller || 0 == previous_total_processed_slots_across_all_threads;
406            if self.first_caller {
407                info!(
408                    "generating index: {}/{} slots...",
409                    previous_total_processed_slots_across_all_threads
410                        + my_total_newly_processed_slots_since_last_report,
411                    self.ultimate_count
412                );
413            }
414            self.last_update = now;
415        }
416    }
417}
418
419/// An offset into the AccountsDb::storage vector
420pub type AtomicAppendVecId = AtomicU32;
421pub type AppendVecId = u32;
422pub type SnapshotStorage = Vec<Arc<AccountStorageEntry>>;
423pub type SnapshotStorages = Vec<SnapshotStorage>;
424
425// Each slot has a set of storage entries.
426pub(crate) type SlotStores = Arc<RwLock<HashMap<AppendVecId, Arc<AccountStorageEntry>>>>;
427
428type AccountSlots = HashMap<Pubkey, HashSet<Slot>>;
429type AppendVecOffsets = HashMap<AppendVecId, HashSet<usize>>;
430type ReclaimResult = (AccountSlots, AppendVecOffsets);
431type PubkeysRemovedFromAccountsIndex = HashSet<Pubkey>;
432type ShrinkCandidates = HashMap<Slot, HashMap<AppendVecId, Arc<AccountStorageEntry>>>;
433
434trait Versioned {
435    fn version(&self) -> u64;
436}
437
438impl Versioned for (u64, Hash) {
439    fn version(&self) -> u64 {
440        self.0
441    }
442}
443
444impl Versioned for (u64, AccountInfo) {
445    fn version(&self) -> u64 {
446        self.0
447    }
448}
449
450// Some hints for applicability of additional sanity checks for the do_load fast-path;
451// Slower fallback code path will be taken if the fast path has failed over the retry
452// threshold, regardless of these hints. Also, load cannot fail not-deterministically
453// even under very rare circumstances, unlike previously did allow.
454#[derive(Clone, Copy, Debug, PartialEq, Eq)]
455pub enum LoadHint {
456    // Caller hints that it's loading transactions for a block which is
457    // descended from the current root, and at the tip of its fork.
458    // Thereby, further this assumes AccountIndex::max_root should not increase
459    // during this load, meaning there should be no squash.
460    // Overall, this enables us to assert!() strictly while running the fast-path for
461    // account loading, while maintaining the determinism of account loading and resultant
462    // transaction execution thereof.
463    FixedMaxRoot,
464    // Caller can't hint the above safety assumption. Generally RPC and miscellaneous
465    // other call-site falls into this category. The likelihood of slower path is slightly
466    // increased as well.
467    Unspecified,
468}
469
470#[derive(Debug)]
471pub enum LoadedAccountAccessor<'a> {
472    // StoredAccountMeta can't be held directly here due to its lifetime dependency to
473    // AccountStorageEntry
474    Stored(Option<(Arc<AccountStorageEntry>, usize)>),
475    // None value in Cached variant means the cache was flushed
476    Cached(Option<Cow<'a, CachedAccount>>),
477}
478
479mod geyser_plugin_utils;
480
481impl<'a> LoadedAccountAccessor<'a> {
482    fn check_and_get_loaded_account(&mut self) -> LoadedAccount {
483        // all of these following .expect() and .unwrap() are like serious logic errors,
484        // ideal for representing this as rust type system....
485
486        match self {
487            LoadedAccountAccessor::Cached(None) | LoadedAccountAccessor::Stored(None) => {
488                panic!("Should have already been taken care of when creating this LoadedAccountAccessor");
489            }
490            LoadedAccountAccessor::Cached(Some(_cached_account)) => {
491                // Cached(Some(x)) variant always produces `Some` for get_loaded_account() since
492                // it just returns the inner `x` without additional fetches
493                self.get_loaded_account().unwrap()
494            }
495            LoadedAccountAccessor::Stored(Some(_maybe_storage_entry)) => {
496                // If we do find the storage entry, we can guarantee that the storage entry is
497                // safe to read from because we grabbed a reference to the storage entry while it
498                // was still in the storage map. This means even if the storage entry is removed
499                // from the storage map after we grabbed the storage entry, the recycler should not
500                // reset the storage entry until we drop the reference to the storage entry.
501                self.get_loaded_account()
502                    .expect("If a storage entry was found in the storage map, it must not have been reset yet")
503            }
504        }
505    }
506
507    fn get_loaded_account(&mut self) -> Option<LoadedAccount> {
508        match self {
509            LoadedAccountAccessor::Cached(cached_account) => {
510                let cached_account: Cow<'a, CachedAccount> = cached_account.take().expect(
511                    "Cache flushed/purged should be handled before trying to fetch account",
512                );
513                Some(LoadedAccount::Cached(cached_account))
514            }
515            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
516                // storage entry may not be present if slot was cleaned up in
517                // between reading the accounts index and calling this function to
518                // get account meta from the storage entry here
519                maybe_storage_entry
520                    .as_ref()
521                    .and_then(|(storage_entry, offset)| {
522                        storage_entry
523                            .get_stored_account_meta(*offset)
524                            .map(LoadedAccount::Stored)
525                    })
526            }
527        }
528    }
529}
530
531pub enum LoadedAccount<'a> {
532    Stored(StoredAccountMeta<'a>),
533    Cached(Cow<'a, CachedAccount>),
534}
535
536impl<'a> LoadedAccount<'a> {
537    pub fn loaded_hash(&self) -> Hash {
538        match self {
539            LoadedAccount::Stored(stored_account_meta) => *stored_account_meta.hash,
540            LoadedAccount::Cached(cached_account) => cached_account.hash(),
541        }
542    }
543
544    pub fn pubkey(&self) -> &Pubkey {
545        match self {
546            LoadedAccount::Stored(stored_account_meta) => &stored_account_meta.meta.pubkey,
547            LoadedAccount::Cached(cached_account) => cached_account.pubkey(),
548        }
549    }
550
551    pub fn write_version(&self) -> StoredMetaWriteVersion {
552        match self {
553            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.meta.write_version,
554            LoadedAccount::Cached(_) => CACHE_VIRTUAL_WRITE_VERSION,
555        }
556    }
557
558    pub fn compute_hash(&self, slot: Slot, pubkey: &Pubkey) -> Hash {
559        match self {
560            LoadedAccount::Stored(stored_account_meta) => AccountsDb::hash_account(
561                slot,
562                stored_account_meta,
563                &stored_account_meta.meta.pubkey,
564            ),
565            LoadedAccount::Cached(cached_account) => {
566                AccountsDb::hash_account(slot, &cached_account.account, pubkey)
567            }
568        }
569    }
570
571    pub fn stored_size(&self) -> usize {
572        match self {
573            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.stored_size,
574            LoadedAccount::Cached(_) => CACHE_VIRTUAL_STORED_SIZE as usize,
575        }
576    }
577
578    pub fn take_account(self) -> AccountSharedData {
579        match self {
580            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.clone_account(),
581            LoadedAccount::Cached(cached_account) => match cached_account {
582                Cow::Owned(cached_account) => cached_account.account.clone(),
583                Cow::Borrowed(cached_account) => cached_account.account.clone(),
584            },
585        }
586    }
587
588    pub fn is_cached(&self) -> bool {
589        match self {
590            LoadedAccount::Stored(_) => false,
591            LoadedAccount::Cached(_) => true,
592        }
593    }
594}
595
596impl<'a> ReadableAccount for LoadedAccount<'a> {
597    fn lamports(&self) -> u64 {
598        match self {
599            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.account_meta.lamports,
600            LoadedAccount::Cached(cached_account) => cached_account.account.lamports(),
601        }
602    }
603
604    fn data(&self) -> &[u8] {
605        match self {
606            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.data,
607            LoadedAccount::Cached(cached_account) => cached_account.account.data(),
608        }
609    }
610    fn owner(&self) -> &Pubkey {
611        match self {
612            LoadedAccount::Stored(stored_account_meta) => &stored_account_meta.account_meta.owner,
613            LoadedAccount::Cached(cached_account) => cached_account.account.owner(),
614        }
615    }
616    fn executable(&self) -> bool {
617        match self {
618            LoadedAccount::Stored(stored_account_meta) => {
619                stored_account_meta.account_meta.executable
620            }
621            LoadedAccount::Cached(cached_account) => cached_account.account.executable(),
622        }
623    }
624    fn rent_epoch(&self) -> Epoch {
625        match self {
626            LoadedAccount::Stored(stored_account_meta) => {
627                stored_account_meta.account_meta.rent_epoch
628            }
629            LoadedAccount::Cached(cached_account) => cached_account.account.rent_epoch(),
630        }
631    }
632    fn to_account_shared_data(&self) -> AccountSharedData {
633        match self {
634            LoadedAccount::Stored(_stored_account_meta) => AccountSharedData::create(
635                self.lamports(),
636                self.data().to_vec(),
637                *self.owner(),
638                self.executable(),
639                self.rent_epoch(),
640            ),
641            // clone here to prevent data copy
642            LoadedAccount::Cached(cached_account) => cached_account.account.clone(),
643        }
644    }
645}
646
647#[derive(Clone, Default, Debug)]
648pub struct AccountStorage {
649    pub map: DashMap<Slot, SlotStores>,
650}
651
652impl AccountStorage {
653    fn get_account_storage_entry(
654        &self,
655        slot: Slot,
656        store_id: AppendVecId,
657    ) -> Option<Arc<AccountStorageEntry>> {
658        self.get_slot_stores(slot)
659            .and_then(|storage_map| storage_map.read().unwrap().get(&store_id).cloned())
660    }
661
662    pub fn get_slot_stores(&self, slot: Slot) -> Option<SlotStores> {
663        self.map.get(&slot).map(|result| result.value().clone())
664    }
665
666    fn get_slot_storage_entries(&self, slot: Slot) -> Option<Vec<Arc<AccountStorageEntry>>> {
667        self.get_slot_stores(slot)
668            .map(|res| res.read().unwrap().values().cloned().collect())
669    }
670
671    fn slot_store_count(&self, slot: Slot, store_id: AppendVecId) -> Option<usize> {
672        self.get_account_storage_entry(slot, store_id)
673            .map(|store| store.count())
674    }
675
676    fn all_slots(&self) -> Vec<Slot> {
677        self.map.iter().map(|iter_item| *iter_item.key()).collect()
678    }
679}
680
681#[derive(Debug, Eq, PartialEq, Copy, Clone, Deserialize, Serialize, AbiExample, AbiEnumVisitor)]
682pub enum AccountStorageStatus {
683    Available = 0,
684    Full = 1,
685    Candidate = 2,
686}
687
688impl Default for AccountStorageStatus {
689    fn default() -> Self {
690        Self::Available
691    }
692}
693
694#[derive(Debug)]
695pub enum BankHashVerificationError {
696    MismatchedAccountHash,
697    MismatchedBankHash,
698    MissingBankHash,
699    MismatchedTotalLamports(u64, u64),
700}
701
702#[derive(Default)]
703struct CleanKeyTimings {
704    collect_delta_keys_us: u64,
705    delta_insert_us: u64,
706    hashset_to_vec_us: u64,
707    dirty_store_processing_us: u64,
708    delta_key_count: u64,
709    dirty_pubkeys_count: u64,
710    oldest_dirty_slot: Slot,
711}
712
713/// Persistent storage structure holding the accounts
714#[derive(Debug)]
715pub struct AccountStorageEntry {
716    pub(crate) id: AtomicAppendVecId,
717
718    pub(crate) slot: AtomicU64,
719
720    /// storage holding the accounts
721    pub(crate) accounts: AppendVec,
722
723    /// Keeps track of the number of accounts stored in a specific AppendVec.
724    ///  This is periodically checked to reuse the stores that do not have
725    ///  any accounts in it
726    /// status corresponding to the storage, lets us know that
727    ///  the append_vec, once maxed out, then emptied, can be reclaimed
728    count_and_status: RwLock<(usize, AccountStorageStatus)>,
729
730    /// This is the total number of accounts stored ever since initialized to keep
731    /// track of lifetime count of all store operations. And this differs from
732    /// count_and_status in that this field won't be decremented.
733    ///
734    /// This is used as a rough estimate for slot shrinking. As such a relaxed
735    /// use case, this value ARE NOT strictly synchronized with count_and_status!
736    approx_store_count: AtomicUsize,
737
738    alive_bytes: AtomicUsize,
739}
740
741impl AccountStorageEntry {
742    pub fn new(path: &Path, slot: Slot, id: AppendVecId, file_size: u64) -> Self {
743        let tail = AppendVec::file_name(slot, id);
744        let path = Path::new(path).join(tail);
745        let accounts = AppendVec::new(&path, true, file_size as usize);
746
747        Self {
748            id: AtomicAppendVecId::new(id),
749            slot: AtomicU64::new(slot),
750            accounts,
751            count_and_status: RwLock::new((0, AccountStorageStatus::Available)),
752            approx_store_count: AtomicUsize::new(0),
753            alive_bytes: AtomicUsize::new(0),
754        }
755    }
756
757    pub(crate) fn new_existing(
758        slot: Slot,
759        id: AppendVecId,
760        accounts: AppendVec,
761        num_accounts: usize,
762    ) -> Self {
763        Self {
764            id: AtomicAppendVecId::new(id),
765            slot: AtomicU64::new(slot),
766            accounts,
767            count_and_status: RwLock::new((0, AccountStorageStatus::Available)),
768            approx_store_count: AtomicUsize::new(num_accounts),
769            alive_bytes: AtomicUsize::new(0),
770        }
771    }
772
773    pub fn set_status(&self, mut status: AccountStorageStatus) {
774        let mut count_and_status = self.count_and_status.write().unwrap();
775
776        let count = count_and_status.0;
777
778        if status == AccountStorageStatus::Full && count == 0 {
779            // this case arises when the append_vec is full (store_ptrs fails),
780            //  but all accounts have already been removed from the storage
781            //
782            // the only time it's safe to call reset() on an append_vec is when
783            //  every account has been removed
784            //          **and**
785            //  the append_vec has previously been completely full
786            //
787            self.accounts.reset();
788            status = AccountStorageStatus::Available;
789        }
790
791        *count_and_status = (count, status);
792    }
793
794    pub fn recycle(&self, slot: Slot, id: AppendVecId) {
795        let mut count_and_status = self.count_and_status.write().unwrap();
796        self.accounts.reset();
797        *count_and_status = (0, AccountStorageStatus::Available);
798        self.slot.store(slot, Ordering::Release);
799        self.id.store(id, Ordering::Release);
800        self.approx_store_count.store(0, Ordering::Relaxed);
801        self.alive_bytes.store(0, Ordering::Release);
802    }
803
804    pub fn status(&self) -> AccountStorageStatus {
805        self.count_and_status.read().unwrap().1
806    }
807
808    pub fn count(&self) -> usize {
809        self.count_and_status.read().unwrap().0
810    }
811
812    pub fn approx_stored_count(&self) -> usize {
813        self.approx_store_count.load(Ordering::Relaxed)
814    }
815
816    pub fn alive_bytes(&self) -> usize {
817        self.alive_bytes.load(Ordering::SeqCst)
818    }
819
820    pub fn written_bytes(&self) -> u64 {
821        self.accounts.len() as u64
822    }
823
824    pub fn total_bytes(&self) -> u64 {
825        self.accounts.capacity()
826    }
827
828    pub fn has_accounts(&self) -> bool {
829        self.count() > 0
830    }
831
832    pub fn slot(&self) -> Slot {
833        self.slot.load(Ordering::Acquire)
834    }
835
836    pub fn append_vec_id(&self) -> AppendVecId {
837        self.id.load(Ordering::Acquire)
838    }
839
840    pub fn flush(&self) -> Result<(), IoError> {
841        self.accounts.flush()
842    }
843
844    fn get_stored_account_meta(&self, offset: usize) -> Option<StoredAccountMeta> {
845        Some(self.accounts.get_account(offset)?.0)
846    }
847
848    fn add_account(&self, num_bytes: usize) {
849        let mut count_and_status = self.count_and_status.write().unwrap();
850        *count_and_status = (count_and_status.0 + 1, count_and_status.1);
851        self.approx_store_count.fetch_add(1, Ordering::Relaxed);
852        self.alive_bytes.fetch_add(num_bytes, Ordering::SeqCst);
853    }
854
855    fn try_available(&self) -> bool {
856        let mut count_and_status = self.count_and_status.write().unwrap();
857        let (count, status) = *count_and_status;
858
859        if status == AccountStorageStatus::Available {
860            *count_and_status = (count, AccountStorageStatus::Candidate);
861            true
862        } else {
863            false
864        }
865    }
866
867    pub fn all_accounts(&self) -> Vec<StoredAccountMeta> {
868        self.accounts.accounts(0)
869    }
870
871    fn remove_account(&self, num_bytes: usize, reset_accounts: bool) -> usize {
872        let mut count_and_status = self.count_and_status.write().unwrap();
873        let (mut count, mut status) = *count_and_status;
874
875        if count == 1 && status == AccountStorageStatus::Full && reset_accounts {
876            // this case arises when we remove the last account from the
877            //  storage, but we've learned from previous write attempts that
878            //  the storage is full
879            //
880            // the only time it's safe to call reset() on an append_vec is when
881            //  every account has been removed
882            //          **and**
883            //  the append_vec has previously been completely full
884            //
885            // otherwise, the storage may be in flight with a store()
886            //   call
887            self.accounts.reset();
888            status = AccountStorageStatus::Available;
889        }
890
891        // Some code path is removing accounts too many; this may result in an
892        // unintended reveal of old state for unrelated accounts.
893        assert!(
894            count > 0,
895            "double remove of account in slot: {}/store: {}!!",
896            self.slot(),
897            self.append_vec_id(),
898        );
899
900        self.alive_bytes.fetch_sub(num_bytes, Ordering::SeqCst);
901        count -= 1;
902        *count_and_status = (count, status);
903        count
904    }
905
906    pub fn get_path(&self) -> PathBuf {
907        self.accounts.get_path()
908    }
909}
910
911pub fn get_temp_accounts_paths(count: u32) -> IoResult<(Vec<TempDir>, Vec<PathBuf>)> {
912    let temp_dirs: IoResult<Vec<TempDir>> = (0..count).map(|_| TempDir::new()).collect();
913    let temp_dirs = temp_dirs?;
914    let paths: Vec<PathBuf> = temp_dirs.iter().map(|t| t.path().to_path_buf()).collect();
915    Ok((temp_dirs, paths))
916}
917
918#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, Eq, AbiExample)]
919pub struct BankHashStats {
920    pub num_updated_accounts: u64,
921    pub num_removed_accounts: u64,
922    pub num_lamports_stored: u64,
923    pub total_data_len: u64,
924    pub num_executable_accounts: u64,
925}
926
927impl BankHashStats {
928    pub fn update<T: ReadableAccount + ZeroLamport>(&mut self, account: &T) {
929        if account.is_zero_lamport() {
930            self.num_removed_accounts += 1;
931        } else {
932            self.num_updated_accounts += 1;
933        }
934        self.total_data_len = self
935            .total_data_len
936            .wrapping_add(account.data().len() as u64);
937        if account.executable() {
938            self.num_executable_accounts += 1;
939        }
940        self.num_lamports_stored = self.num_lamports_stored.wrapping_add(account.lamports());
941    }
942
943    pub fn merge(&mut self, other: &BankHashStats) {
944        self.num_updated_accounts += other.num_updated_accounts;
945        self.num_removed_accounts += other.num_removed_accounts;
946        self.total_data_len = self.total_data_len.wrapping_add(other.total_data_len);
947        self.num_lamports_stored = self
948            .num_lamports_stored
949            .wrapping_add(other.num_lamports_stored);
950        self.num_executable_accounts += other.num_executable_accounts;
951    }
952}
953
954#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, Eq, AbiExample)]
955pub struct BankHashInfo {
956    pub hash: Hash,
957    pub snapshot_hash: Hash,
958    pub stats: BankHashStats,
959}
960
961#[derive(Default)]
962pub struct StoreAccountsTiming {
963    store_accounts_elapsed: u64,
964    update_index_elapsed: u64,
965    handle_reclaims_elapsed: u64,
966}
967
968#[derive(Debug, Default)]
969struct RecycleStores {
970    entries: Vec<(Instant, Arc<AccountStorageEntry>)>,
971    total_bytes: u64,
972}
973
974// 30 min should be enough to be certain there won't be any prospective recycle uses for given
975// store entry
976// That's because it already processed ~2500 slots and ~25 passes of AccountsBackgroundService
977pub const EXPIRATION_TTL_SECONDS: u64 = 1800;
978
979impl RecycleStores {
980    fn add_entry(&mut self, new_entry: Arc<AccountStorageEntry>) {
981        self.total_bytes += new_entry.total_bytes();
982        self.entries.push((Instant::now(), new_entry))
983    }
984
985    fn iter(&self) -> std::slice::Iter<(Instant, Arc<AccountStorageEntry>)> {
986        self.entries.iter()
987    }
988
989    fn add_entries(&mut self, new_entries: Vec<Arc<AccountStorageEntry>>) {
990        self.total_bytes += new_entries.iter().map(|e| e.total_bytes()).sum::<u64>();
991        let now = Instant::now();
992        for new_entry in new_entries {
993            self.entries.push((now, new_entry));
994        }
995    }
996
997    fn expire_old_entries(&mut self) -> Vec<Arc<AccountStorageEntry>> {
998        let mut expired = vec![];
999        let now = Instant::now();
1000        let mut expired_bytes = 0;
1001        self.entries.retain(|(recycled_time, entry)| {
1002            if now.duration_since(*recycled_time).as_secs() > EXPIRATION_TTL_SECONDS {
1003                if Arc::strong_count(entry) >= 2 {
1004                    warn!(
1005                        "Expiring still in-use recycled StorageEntry anyway...: id: {} slot: {}",
1006                        entry.append_vec_id(),
1007                        entry.slot(),
1008                    );
1009                }
1010                expired_bytes += entry.total_bytes();
1011                expired.push(entry.clone());
1012                false
1013            } else {
1014                true
1015            }
1016        });
1017
1018        self.total_bytes -= expired_bytes;
1019
1020        expired
1021    }
1022
1023    fn remove_entry(&mut self, index: usize) -> Arc<AccountStorageEntry> {
1024        let (_added_time, removed_entry) = self.entries.swap_remove(index);
1025        self.total_bytes -= removed_entry.total_bytes();
1026        removed_entry
1027    }
1028
1029    fn entry_count(&self) -> usize {
1030        self.entries.len()
1031    }
1032
1033    fn total_bytes(&self) -> u64 {
1034        self.total_bytes
1035    }
1036}
1037
1038/// Removing unrooted slots in Accounts Background Service needs to be synchronized with flushing
1039/// slots from the Accounts Cache.  This keeps track of those slots and the Mutex + Condvar for
1040/// synchronization.
1041#[derive(Debug, Default)]
1042struct RemoveUnrootedSlotsSynchronization {
1043    // slots being flushed from the cache or being purged
1044    slots_under_contention: Mutex<HashSet<Slot>>,
1045    signal: Condvar,
1046}
1047
1048type AccountInfoAccountsIndex = AccountsIndex<AccountInfo>;
1049
1050// This structure handles the load/store of the accounts
1051#[derive(Debug)]
1052pub struct AccountsDb {
1053    /// Keeps tracks of index into AppendVec on a per slot basis
1054    pub accounts_index: AccountInfoAccountsIndex,
1055
1056    /// slot that is one epoch older than the highest slot where accounts hash calculation has completed
1057    pub accounts_hash_complete_one_epoch_old: RwLock<Slot>,
1058
1059    /// true iff rent exempt accounts are not rewritten in their normal rent collection slot
1060    pub skip_rewrites: bool,
1061
1062    /// true iff we want to squash old append vecs together into 'ancient append vecs'
1063    pub ancient_append_vecs: bool,
1064
1065    /// true iff we want to skip the initial hash calculation on startup
1066    pub skip_initial_hash_calc: bool,
1067
1068    pub storage: AccountStorage,
1069
1070    pub accounts_cache: AccountsCache,
1071
1072    write_cache_limit_bytes: Option<u64>,
1073
1074    sender_bg_hasher: Option<Sender<CachedAccount>>,
1075    read_only_accounts_cache: ReadOnlyAccountsCache,
1076
1077    recycle_stores: RwLock<RecycleStores>,
1078
1079    /// distribute the accounts across storage lists
1080    pub next_id: AtomicAppendVecId,
1081
1082    /// Set of shrinkable stores organized by map of slot to append_vec_id
1083    pub shrink_candidate_slots: Mutex<ShrinkCandidates>,
1084
1085    /// Legacy shrink slots to support non-cached code-path.
1086    pub shrink_candidate_slots_v1: Mutex<Vec<Slot>>,
1087
1088    pub(crate) write_version: AtomicU64,
1089
1090    /// Set of storage paths to pick from
1091    pub(crate) paths: Vec<PathBuf>,
1092
1093    accounts_hash_cache_path: PathBuf,
1094
1095    // used by tests
1096    // holds this until we are dropped
1097    #[allow(dead_code)]
1098    temp_accounts_hash_cache_path: Option<TempDir>,
1099
1100    pub shrink_paths: RwLock<Option<Vec<PathBuf>>>,
1101
1102    /// Directory of paths this accounts_db needs to hold/remove
1103    #[allow(dead_code)]
1104    pub(crate) temp_paths: Option<Vec<TempDir>>,
1105
1106    /// Starting file size of appendvecs
1107    file_size: u64,
1108
1109    /// Thread pool used for par_iter
1110    pub thread_pool: ThreadPool,
1111
1112    pub thread_pool_clean: ThreadPool,
1113
1114    /// Number of append vecs to create to maximize parallelism when scanning
1115    /// the accounts
1116    min_num_stores: usize,
1117
1118    pub bank_hashes: RwLock<HashMap<Slot, BankHashInfo>>,
1119
1120    pub stats: AccountsStats,
1121
1122    clean_accounts_stats: CleanAccountsStats,
1123
1124    // Stats for purges called outside of clean_accounts()
1125    external_purge_slots_stats: PurgeStats,
1126
1127    shrink_stats: ShrinkStats,
1128
1129    shrink_ancient_stats: ShrinkAncientStats,
1130
1131    pub cluster_type: Option<ClusterType>,
1132
1133    pub account_indexes: AccountSecondaryIndexes,
1134
1135    pub caching_enabled: bool,
1136
1137    /// Set of unique keys per slot which is used
1138    /// to drive clean_accounts
1139    /// Generated by get_accounts_delta_hash
1140    uncleaned_pubkeys: DashMap<Slot, Vec<Pubkey>>,
1141
1142    #[cfg(test)]
1143    load_delay: u64,
1144
1145    #[cfg(test)]
1146    load_limit: AtomicU64,
1147
1148    /// true if drop_callback is attached to the bank.
1149    is_bank_drop_callback_enabled: AtomicBool,
1150
1151    /// Set of slots currently being flushed by `flush_slot_cache()` or removed
1152    /// by `remove_unrooted_slot()`. Used to ensure `remove_unrooted_slots(slots)`
1153    /// can safely clear the set of unrooted slots `slots`.
1154    remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization,
1155
1156    shrink_ratio: AccountShrinkThreshold,
1157
1158    /// Set of stores which are recently rooted or had accounts removed
1159    /// such that potentially a 0-lamport account update could be present which
1160    /// means we can remove the account from the index entirely.
1161    dirty_stores: DashMap<(Slot, AppendVecId), Arc<AccountStorageEntry>>,
1162
1163    /// Zero-lamport accounts that are *not* purged during clean because they need to stay alive
1164    /// for incremental snapshot support.
1165    zero_lamport_accounts_to_purge_after_full_snapshot: DashSet<(Slot, Pubkey)>,
1166
1167    /// GeyserPlugin accounts update notifier
1168    accounts_update_notifier: Option<AccountsUpdateNotifier>,
1169
1170    filler_accounts_config: FillerAccountsConfig,
1171    pub filler_account_suffix: Option<Pubkey>,
1172
1173    active_stats: ActiveStats,
1174
1175    /// number of filler accounts to add for each slot
1176    pub filler_accounts_per_slot: AtomicU64,
1177
1178    /// number of slots remaining where filler accounts should be added
1179    pub filler_account_slots_remaining: AtomicU64,
1180
1181    pub(crate) verify_accounts_hash_in_bg: VerifyAccountsHashInBackground,
1182
1183    // # of passes should be a function of the total # of accounts that are active.
1184    // higher passes = slower total time, lower dynamic memory usage
1185    // lower passes = faster total time, higher dynamic memory usage
1186    // passes=2 cuts dynamic memory usage in approximately half.
1187    pub num_hash_scan_passes: Option<usize>,
1188
1189    /// Used to disable logging dead slots during removal.
1190    /// allow disabling noisy log
1191    pub(crate) log_dead_slots: AtomicBool,
1192}
1193
1194#[derive(Debug, Default)]
1195pub struct AccountsStats {
1196    delta_hash_scan_time_total_us: AtomicU64,
1197    delta_hash_accumulate_time_total_us: AtomicU64,
1198    delta_hash_num: AtomicU64,
1199
1200    last_store_report: AtomicInterval,
1201    store_hash_accounts: AtomicU64,
1202    calc_stored_meta: AtomicU64,
1203    store_accounts: AtomicU64,
1204    store_update_index: AtomicU64,
1205    store_handle_reclaims: AtomicU64,
1206    store_append_accounts: AtomicU64,
1207    pub stakes_cache_check_and_store_us: AtomicU64,
1208    store_find_store: AtomicU64,
1209    store_num_accounts: AtomicU64,
1210    store_total_data: AtomicU64,
1211    recycle_store_count: AtomicU64,
1212    create_store_count: AtomicU64,
1213    store_get_slot_store: AtomicU64,
1214    store_find_existing: AtomicU64,
1215    dropped_stores: AtomicU64,
1216    store_uncleaned_update: AtomicU64,
1217}
1218
1219#[derive(Debug, Default)]
1220pub(crate) struct PurgeStats {
1221    last_report: AtomicInterval,
1222    safety_checks_elapsed: AtomicU64,
1223    remove_cache_elapsed: AtomicU64,
1224    remove_storage_entries_elapsed: AtomicU64,
1225    drop_storage_entries_elapsed: AtomicU64,
1226    num_cached_slots_removed: AtomicUsize,
1227    num_stored_slots_removed: AtomicUsize,
1228    total_removed_storage_entries: AtomicUsize,
1229    total_removed_cached_bytes: AtomicU64,
1230    total_removed_stored_bytes: AtomicU64,
1231    recycle_stores_write_elapsed: AtomicU64,
1232    scan_storages_elapsed: AtomicU64,
1233    purge_accounts_index_elapsed: AtomicU64,
1234    handle_reclaims_elapsed: AtomicU64,
1235}
1236
1237impl PurgeStats {
1238    fn report(&self, metric_name: &'static str, report_interval_ms: Option<u64>) {
1239        let should_report = report_interval_ms
1240            .map(|report_interval_ms| self.last_report.should_update(report_interval_ms))
1241            .unwrap_or(true);
1242
1243        if should_report {
1244            datapoint_info!(
1245                metric_name,
1246                (
1247                    "safety_checks_elapsed",
1248                    self.safety_checks_elapsed.swap(0, Ordering::Relaxed) as i64,
1249                    i64
1250                ),
1251                (
1252                    "remove_cache_elapsed",
1253                    self.remove_cache_elapsed.swap(0, Ordering::Relaxed) as i64,
1254                    i64
1255                ),
1256                (
1257                    "remove_storage_entries_elapsed",
1258                    self.remove_storage_entries_elapsed
1259                        .swap(0, Ordering::Relaxed) as i64,
1260                    i64
1261                ),
1262                (
1263                    "drop_storage_entries_elapsed",
1264                    self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
1265                    i64
1266                ),
1267                (
1268                    "num_cached_slots_removed",
1269                    self.num_cached_slots_removed.swap(0, Ordering::Relaxed) as i64,
1270                    i64
1271                ),
1272                (
1273                    "num_stored_slots_removed",
1274                    self.num_stored_slots_removed.swap(0, Ordering::Relaxed) as i64,
1275                    i64
1276                ),
1277                (
1278                    "total_removed_storage_entries",
1279                    self.total_removed_storage_entries
1280                        .swap(0, Ordering::Relaxed) as i64,
1281                    i64
1282                ),
1283                (
1284                    "total_removed_cached_bytes",
1285                    self.total_removed_cached_bytes.swap(0, Ordering::Relaxed) as i64,
1286                    i64
1287                ),
1288                (
1289                    "total_removed_stored_bytes",
1290                    self.total_removed_stored_bytes.swap(0, Ordering::Relaxed) as i64,
1291                    i64
1292                ),
1293                (
1294                    "recycle_stores_write_elapsed",
1295                    self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
1296                    i64
1297                ),
1298                (
1299                    "scan_storages_elapsed",
1300                    self.scan_storages_elapsed.swap(0, Ordering::Relaxed) as i64,
1301                    i64
1302                ),
1303                (
1304                    "purge_accounts_index_elapsed",
1305                    self.purge_accounts_index_elapsed.swap(0, Ordering::Relaxed) as i64,
1306                    i64
1307                ),
1308                (
1309                    "handle_reclaims_elapsed",
1310                    self.handle_reclaims_elapsed.swap(0, Ordering::Relaxed) as i64,
1311                    i64
1312                ),
1313            );
1314        }
1315    }
1316}
1317
1318#[derive(Debug, Default)]
1319struct FlushStats {
1320    num_flushed: usize,
1321    num_purged: usize,
1322    total_size: u64,
1323}
1324
1325#[derive(Debug, Default)]
1326struct LatestAccountsIndexRootsStats {
1327    roots_len: AtomicUsize,
1328    historical_roots_len: AtomicUsize,
1329    uncleaned_roots_len: AtomicUsize,
1330    previous_uncleaned_roots_len: AtomicUsize,
1331    roots_range: AtomicU64,
1332    rooted_cleaned_count: AtomicUsize,
1333    unrooted_cleaned_count: AtomicUsize,
1334    clean_unref_from_storage_us: AtomicU64,
1335    clean_dead_slot_us: AtomicU64,
1336}
1337
1338impl LatestAccountsIndexRootsStats {
1339    fn update(&self, accounts_index_roots_stats: &AccountsIndexRootsStats) {
1340        if let Some(value) = accounts_index_roots_stats.roots_len {
1341            self.roots_len.store(value, Ordering::Relaxed);
1342        }
1343        if let Some(value) = accounts_index_roots_stats.uncleaned_roots_len {
1344            self.uncleaned_roots_len.store(value, Ordering::Relaxed);
1345        }
1346        if let Some(value) = accounts_index_roots_stats.previous_uncleaned_roots_len {
1347            self.previous_uncleaned_roots_len
1348                .store(value, Ordering::Relaxed);
1349        }
1350        if let Some(value) = accounts_index_roots_stats.historical_roots_len {
1351            self.historical_roots_len.store(value, Ordering::Relaxed);
1352        }
1353        if let Some(value) = accounts_index_roots_stats.roots_range {
1354            self.roots_range.store(value, Ordering::Relaxed);
1355        }
1356        self.rooted_cleaned_count.fetch_add(
1357            accounts_index_roots_stats.rooted_cleaned_count,
1358            Ordering::Relaxed,
1359        );
1360        self.unrooted_cleaned_count.fetch_add(
1361            accounts_index_roots_stats.unrooted_cleaned_count,
1362            Ordering::Relaxed,
1363        );
1364        self.clean_unref_from_storage_us.fetch_add(
1365            accounts_index_roots_stats.clean_unref_from_storage_us,
1366            Ordering::Relaxed,
1367        );
1368        self.clean_dead_slot_us.fetch_add(
1369            accounts_index_roots_stats.clean_dead_slot_us,
1370            Ordering::Relaxed,
1371        );
1372    }
1373
1374    fn report(&self) {
1375        datapoint_info!(
1376            "accounts_index_roots_len",
1377            (
1378                "roots_len",
1379                self.roots_len.load(Ordering::Relaxed) as i64,
1380                i64
1381            ),
1382            (
1383                "historical_roots_len",
1384                self.historical_roots_len.load(Ordering::Relaxed) as i64,
1385                i64
1386            ),
1387            (
1388                "uncleaned_roots_len",
1389                self.uncleaned_roots_len.load(Ordering::Relaxed) as i64,
1390                i64
1391            ),
1392            (
1393                "previous_uncleaned_roots_len",
1394                self.previous_uncleaned_roots_len.load(Ordering::Relaxed) as i64,
1395                i64
1396            ),
1397            (
1398                "roots_range_width",
1399                self.roots_range.load(Ordering::Relaxed) as i64,
1400                i64
1401            ),
1402            (
1403                "unrooted_cleaned_count",
1404                self.unrooted_cleaned_count.swap(0, Ordering::Relaxed) as i64,
1405                i64
1406            ),
1407            (
1408                "rooted_cleaned_count",
1409                self.rooted_cleaned_count.swap(0, Ordering::Relaxed) as i64,
1410                i64
1411            ),
1412            (
1413                "clean_unref_from_storage_us",
1414                self.clean_unref_from_storage_us.swap(0, Ordering::Relaxed) as i64,
1415                i64
1416            ),
1417            (
1418                "clean_dead_slot_us",
1419                self.clean_dead_slot_us.swap(0, Ordering::Relaxed) as i64,
1420                i64
1421            ),
1422        );
1423
1424        // Don't need to reset since this tracks the latest updates, not a cumulative total
1425    }
1426}
1427
1428#[derive(Debug, Default)]
1429struct CleanAccountsStats {
1430    purge_stats: PurgeStats,
1431    latest_accounts_index_roots_stats: LatestAccountsIndexRootsStats,
1432
1433    // stats held here and reported by clean_accounts
1434    clean_old_root_us: AtomicU64,
1435    clean_old_root_reclaim_us: AtomicU64,
1436    reset_uncleaned_roots_us: AtomicU64,
1437    remove_dead_accounts_remove_us: AtomicU64,
1438    remove_dead_accounts_shrink_us: AtomicU64,
1439    clean_stored_dead_slots_us: AtomicU64,
1440}
1441
1442impl CleanAccountsStats {
1443    fn report(&self) {
1444        self.purge_stats.report("clean_purge_slots_stats", None);
1445        self.latest_accounts_index_roots_stats.report();
1446    }
1447}
1448
1449#[derive(Debug, Default)]
1450struct ShrinkAncientStats {
1451    shrink_stats: ShrinkStats,
1452}
1453
1454#[derive(Debug, Default)]
1455struct ShrinkStats {
1456    last_report: AtomicInterval,
1457    num_slots_shrunk: AtomicUsize,
1458    storage_read_elapsed: AtomicU64,
1459    index_read_elapsed: AtomicU64,
1460    find_alive_elapsed: AtomicU64,
1461    create_and_insert_store_elapsed: AtomicU64,
1462    store_accounts_elapsed: AtomicU64,
1463    update_index_elapsed: AtomicU64,
1464    handle_reclaims_elapsed: AtomicU64,
1465    write_storage_elapsed: AtomicU64,
1466    rewrite_elapsed: AtomicU64,
1467    drop_storage_entries_elapsed: AtomicU64,
1468    recycle_stores_write_elapsed: AtomicU64,
1469    accounts_removed: AtomicUsize,
1470    bytes_removed: AtomicU64,
1471    bytes_written: AtomicU64,
1472    skipped_shrink: AtomicU64,
1473    dead_accounts: AtomicU64,
1474    alive_accounts: AtomicU64,
1475    accounts_loaded: AtomicU64,
1476}
1477
1478impl ShrinkStats {
1479    fn report(&self) {
1480        if self.last_report.should_update(1000) {
1481            datapoint_info!(
1482                "shrink_stats",
1483                (
1484                    "num_slots_shrunk",
1485                    self.num_slots_shrunk.swap(0, Ordering::Relaxed) as i64,
1486                    i64
1487                ),
1488                (
1489                    "storage_read_elapsed",
1490                    self.storage_read_elapsed.swap(0, Ordering::Relaxed) as i64,
1491                    i64
1492                ),
1493                (
1494                    "index_read_elapsed",
1495                    self.index_read_elapsed.swap(0, Ordering::Relaxed) as i64,
1496                    i64
1497                ),
1498                (
1499                    "find_alive_elapsed",
1500                    self.find_alive_elapsed.swap(0, Ordering::Relaxed) as i64,
1501                    i64
1502                ),
1503                (
1504                    "create_and_insert_store_elapsed",
1505                    self.create_and_insert_store_elapsed
1506                        .swap(0, Ordering::Relaxed) as i64,
1507                    i64
1508                ),
1509                (
1510                    "store_accounts_elapsed",
1511                    self.store_accounts_elapsed.swap(0, Ordering::Relaxed) as i64,
1512                    i64
1513                ),
1514                (
1515                    "update_index_elapsed",
1516                    self.update_index_elapsed.swap(0, Ordering::Relaxed) as i64,
1517                    i64
1518                ),
1519                (
1520                    "handle_reclaims_elapsed",
1521                    self.handle_reclaims_elapsed.swap(0, Ordering::Relaxed) as i64,
1522                    i64
1523                ),
1524                (
1525                    "write_storage_elapsed",
1526                    self.write_storage_elapsed.swap(0, Ordering::Relaxed) as i64,
1527                    i64
1528                ),
1529                (
1530                    "rewrite_elapsed",
1531                    self.rewrite_elapsed.swap(0, Ordering::Relaxed) as i64,
1532                    i64
1533                ),
1534                (
1535                    "drop_storage_entries_elapsed",
1536                    self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
1537                    i64
1538                ),
1539                (
1540                    "recycle_stores_write_time",
1541                    self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
1542                    i64
1543                ),
1544                (
1545                    "accounts_removed",
1546                    self.accounts_removed.swap(0, Ordering::Relaxed) as i64,
1547                    i64
1548                ),
1549                (
1550                    "bytes_removed",
1551                    self.bytes_removed.swap(0, Ordering::Relaxed) as i64,
1552                    i64
1553                ),
1554                (
1555                    "bytes_written",
1556                    self.bytes_written.swap(0, Ordering::Relaxed) as i64,
1557                    i64
1558                ),
1559                (
1560                    "skipped_shrink",
1561                    self.skipped_shrink.swap(0, Ordering::Relaxed) as i64,
1562                    i64
1563                ),
1564                (
1565                    "alive_accounts",
1566                    self.alive_accounts.swap(0, Ordering::Relaxed) as i64,
1567                    i64
1568                ),
1569                (
1570                    "dead_accounts",
1571                    self.dead_accounts.swap(0, Ordering::Relaxed) as i64,
1572                    i64
1573                ),
1574                (
1575                    "accounts_loaded",
1576                    self.accounts_loaded.swap(0, Ordering::Relaxed) as i64,
1577                    i64
1578                ),
1579            );
1580        }
1581    }
1582}
1583
1584impl ShrinkAncientStats {
1585    fn report(&self) {
1586        if self.shrink_stats.last_report.should_update(1000) {
1587            datapoint_info!(
1588                "shrink_ancient_stats",
1589                (
1590                    "num_slots_shrunk",
1591                    self.shrink_stats
1592                        .num_slots_shrunk
1593                        .swap(0, Ordering::Relaxed) as i64,
1594                    i64
1595                ),
1596                (
1597                    "storage_read_elapsed",
1598                    self.shrink_stats
1599                        .storage_read_elapsed
1600                        .swap(0, Ordering::Relaxed) as i64,
1601                    i64
1602                ),
1603                (
1604                    "index_read_elapsed",
1605                    self.shrink_stats
1606                        .index_read_elapsed
1607                        .swap(0, Ordering::Relaxed) as i64,
1608                    i64
1609                ),
1610                (
1611                    "find_alive_elapsed",
1612                    self.shrink_stats
1613                        .find_alive_elapsed
1614                        .swap(0, Ordering::Relaxed) as i64,
1615                    i64
1616                ),
1617                (
1618                    "create_and_insert_store_elapsed",
1619                    self.shrink_stats
1620                        .create_and_insert_store_elapsed
1621                        .swap(0, Ordering::Relaxed) as i64,
1622                    i64
1623                ),
1624                (
1625                    "store_accounts_elapsed",
1626                    self.shrink_stats
1627                        .store_accounts_elapsed
1628                        .swap(0, Ordering::Relaxed) as i64,
1629                    i64
1630                ),
1631                (
1632                    "update_index_elapsed",
1633                    self.shrink_stats
1634                        .update_index_elapsed
1635                        .swap(0, Ordering::Relaxed) as i64,
1636                    i64
1637                ),
1638                (
1639                    "handle_reclaims_elapsed",
1640                    self.shrink_stats
1641                        .handle_reclaims_elapsed
1642                        .swap(0, Ordering::Relaxed) as i64,
1643                    i64
1644                ),
1645                (
1646                    "write_storage_elapsed",
1647                    self.shrink_stats
1648                        .write_storage_elapsed
1649                        .swap(0, Ordering::Relaxed) as i64,
1650                    i64
1651                ),
1652                (
1653                    "rewrite_elapsed",
1654                    self.shrink_stats.rewrite_elapsed.swap(0, Ordering::Relaxed) as i64,
1655                    i64
1656                ),
1657                (
1658                    "drop_storage_entries_elapsed",
1659                    self.shrink_stats
1660                        .drop_storage_entries_elapsed
1661                        .swap(0, Ordering::Relaxed) as i64,
1662                    i64
1663                ),
1664                (
1665                    "recycle_stores_write_time",
1666                    self.shrink_stats
1667                        .recycle_stores_write_elapsed
1668                        .swap(0, Ordering::Relaxed) as i64,
1669                    i64
1670                ),
1671                (
1672                    "accounts_removed",
1673                    self.shrink_stats
1674                        .accounts_removed
1675                        .swap(0, Ordering::Relaxed) as i64,
1676                    i64
1677                ),
1678                (
1679                    "bytes_removed",
1680                    self.shrink_stats.bytes_removed.swap(0, Ordering::Relaxed) as i64,
1681                    i64
1682                ),
1683                (
1684                    "bytes_written",
1685                    self.shrink_stats.bytes_written.swap(0, Ordering::Relaxed) as i64,
1686                    i64
1687                ),
1688                (
1689                    "skipped_shrink",
1690                    self.shrink_stats.skipped_shrink.swap(0, Ordering::Relaxed) as i64,
1691                    i64
1692                ),
1693                (
1694                    "alive_accounts",
1695                    self.shrink_stats.alive_accounts.swap(0, Ordering::Relaxed) as i64,
1696                    i64
1697                ),
1698                (
1699                    "dead_accounts",
1700                    self.shrink_stats.dead_accounts.swap(0, Ordering::Relaxed) as i64,
1701                    i64
1702                ),
1703                (
1704                    "accounts_loaded",
1705                    self.shrink_stats.accounts_loaded.swap(0, Ordering::Relaxed) as i64,
1706                    i64
1707                ),
1708            );
1709        }
1710    }
1711}
1712
1713pub fn quarter_thread_count() -> usize {
1714    std::cmp::max(2, num_cpus::get() / 4)
1715}
1716
1717pub fn make_min_priority_thread_pool() -> ThreadPool {
1718    // Use lower thread count to reduce priority.
1719    let num_threads = quarter_thread_count();
1720    rayon::ThreadPoolBuilder::new()
1721        .thread_name(|i| format!("solAccountsLo{:02}", i))
1722        .num_threads(num_threads)
1723        .build()
1724        .unwrap()
1725}
1726
1727#[cfg(all(test, RUSTC_WITH_SPECIALIZATION))]
1728impl safecoin_frozen_abi::abi_example::AbiExample for AccountsDb {
1729    fn example() -> Self {
1730        let accounts_db = AccountsDb::new_single_for_tests();
1731        let key = Pubkey::default();
1732        let some_data_len = 5;
1733        let some_slot: Slot = 0;
1734        let account = AccountSharedData::new(1, some_data_len, &key);
1735        accounts_db.store_uncached(some_slot, &[(&key, &account)]);
1736        accounts_db.add_root(0);
1737
1738        accounts_db
1739    }
1740}
1741
1742impl<'a> ZeroLamport for StoredAccountMeta<'a> {
1743    fn is_zero_lamport(&self) -> bool {
1744        self.lamports() == 0
1745    }
1746}
1747
1748impl<'a> ReadableAccount for StoredAccountMeta<'a> {
1749    fn lamports(&self) -> u64 {
1750        self.account_meta.lamports
1751    }
1752    fn data(&self) -> &[u8] {
1753        self.data
1754    }
1755    fn owner(&self) -> &Pubkey {
1756        &self.account_meta.owner
1757    }
1758    fn executable(&self) -> bool {
1759        self.account_meta.executable
1760    }
1761    fn rent_epoch(&self) -> Epoch {
1762        self.account_meta.rent_epoch
1763    }
1764}
1765
1766struct IndexAccountMapEntry<'a> {
1767    pub write_version: StoredMetaWriteVersion,
1768    pub store_id: AppendVecId,
1769    pub stored_account: StoredAccountMeta<'a>,
1770}
1771
1772type GenerateIndexAccountsMap<'a> = HashMap<Pubkey, IndexAccountMapEntry<'a>>;
1773
1774/// called on a struct while scanning append vecs
1775trait AppendVecScan: Send + Sync + Clone {
1776    /// return true if this pubkey should be included
1777    fn filter(&mut self, pubkey: &Pubkey) -> bool;
1778    /// set current slot of the scan
1779    fn set_slot(&mut self, slot: Slot);
1780    /// found `account` in the append vec
1781    fn found_account(&mut self, account: &LoadedAccount);
1782    /// scanning is done
1783    fn scanning_complete(self) -> BinnedHashData;
1784    /// initialize accumulator
1785    fn init_accum(&mut self, count: usize);
1786    fn get_accum(&mut self) -> BinnedHashData;
1787    fn set_accum(&mut self, accum: BinnedHashData);
1788}
1789
1790#[derive(Clone)]
1791/// state to keep while scanning append vec accounts for hash calculation
1792/// These would have been captured in a fn from within the scan function.
1793/// Some of these are constant across all pubkeys, some are constant across a slot.
1794/// Some could be unique per pubkey.
1795struct ScanState<'a> {
1796    /// slot we're currently scanning
1797    current_slot: Slot,
1798    /// accumulated results
1799    accum: BinnedHashData,
1800    bin_calculator: &'a PubkeyBinCalculator24,
1801    bin_range: &'a Range<usize>,
1802    config: &'a CalcAccountsHashConfig<'a>,
1803    mismatch_found: Arc<AtomicU64>,
1804    filler_account_suffix: Option<&'a Pubkey>,
1805    range: usize,
1806    sort_time: Arc<AtomicU64>,
1807    pubkey_to_bin_index: usize,
1808}
1809
1810impl<'a> AppendVecScan for ScanState<'a> {
1811    fn set_slot(&mut self, slot: Slot) {
1812        self.current_slot = slot;
1813    }
1814    fn filter(&mut self, pubkey: &Pubkey) -> bool {
1815        self.pubkey_to_bin_index = self.bin_calculator.bin_from_pubkey(pubkey);
1816        self.bin_range.contains(&self.pubkey_to_bin_index)
1817    }
1818    fn init_accum(&mut self, count: usize) {
1819        if self.accum.is_empty() {
1820            self.accum.append(&mut vec![Vec::new(); count]);
1821        }
1822    }
1823    fn found_account(&mut self, loaded_account: &LoadedAccount) {
1824        let pubkey = loaded_account.pubkey();
1825        assert!(self.bin_range.contains(&self.pubkey_to_bin_index)); // get rid of this once we have confidence
1826
1827        // when we are scanning with bin ranges, we don't need to use exact bin numbers. Subtract to make first bin we care about at index 0.
1828        self.pubkey_to_bin_index -= self.bin_range.start;
1829
1830        let raw_lamports = loaded_account.lamports();
1831        let zero_raw_lamports = raw_lamports == 0;
1832        let balance = if zero_raw_lamports {
1833            crate::accounts_hash::ZERO_RAW_LAMPORTS_SENTINEL
1834        } else {
1835            raw_lamports
1836        };
1837
1838        let loaded_hash = loaded_account.loaded_hash();
1839        let source_item = CalculateHashIntermediate::new(loaded_hash, balance, *pubkey);
1840
1841        if self.config.check_hash
1842            && !AccountsDb::is_filler_account_helper(pubkey, self.filler_account_suffix)
1843        {
1844            // this will not be supported anymore
1845            let computed_hash = loaded_account.compute_hash(self.current_slot, pubkey);
1846            if computed_hash != source_item.hash {
1847                info!(
1848                    "hash mismatch found: computed: {}, loaded: {}, pubkey: {}",
1849                    computed_hash, source_item.hash, pubkey
1850                );
1851                self.mismatch_found.fetch_add(1, Ordering::Relaxed);
1852            }
1853        }
1854        self.init_accum(self.range);
1855        self.accum[self.pubkey_to_bin_index].push(source_item);
1856    }
1857    fn scanning_complete(self) -> BinnedHashData {
1858        let (result, timing) = AccountsDb::sort_slot_storage_scan(self.accum);
1859        self.sort_time.fetch_add(timing, Ordering::Relaxed);
1860        result
1861    }
1862    fn get_accum(&mut self) -> BinnedHashData {
1863        std::mem::take(&mut self.accum)
1864    }
1865    fn set_accum(&mut self, accum: BinnedHashData) {
1866        self.accum = accum;
1867    }
1868}
1869
1870impl AccountsDb {
1871    pub fn default_for_tests() -> Self {
1872        Self::default_with_accounts_index(AccountInfoAccountsIndex::default_for_tests(), None, None)
1873    }
1874
1875    /// return (num_hash_scan_passes, bins_per_pass)
1876    fn bins_per_pass(num_hash_scan_passes: Option<usize>) -> (usize, usize) {
1877        let num_hash_scan_passes = num_hash_scan_passes.unwrap_or(NUM_SCAN_PASSES_DEFAULT);
1878        let bins_per_pass = PUBKEY_BINS_FOR_CALCULATING_HASHES / num_hash_scan_passes;
1879        assert!(
1880            num_hash_scan_passes <= PUBKEY_BINS_FOR_CALCULATING_HASHES,
1881            "num_hash_scan_passes must be <= {}",
1882            PUBKEY_BINS_FOR_CALCULATING_HASHES
1883        );
1884        assert_eq!(
1885            bins_per_pass * num_hash_scan_passes,
1886            PUBKEY_BINS_FOR_CALCULATING_HASHES
1887        ); // evenly divisible
1888
1889        (num_hash_scan_passes, bins_per_pass)
1890    }
1891
1892    fn default_with_accounts_index(
1893        accounts_index: AccountInfoAccountsIndex,
1894        accounts_hash_cache_path: Option<PathBuf>,
1895        num_hash_scan_passes: Option<usize>,
1896    ) -> Self {
1897        let num_threads = get_thread_count();
1898        const MAX_READ_ONLY_CACHE_DATA_SIZE: usize = 400_000_000; // 400M bytes
1899
1900        let mut temp_accounts_hash_cache_path = None;
1901        let accounts_hash_cache_path = accounts_hash_cache_path.unwrap_or_else(|| {
1902            temp_accounts_hash_cache_path = Some(TempDir::new().unwrap());
1903            temp_accounts_hash_cache_path
1904                .as_ref()
1905                .unwrap()
1906                .path()
1907                .to_path_buf()
1908        });
1909
1910        let mut bank_hashes = HashMap::new();
1911        bank_hashes.insert(0, BankHashInfo::default());
1912
1913        // validate inside here
1914        Self::bins_per_pass(num_hash_scan_passes);
1915
1916        // Increase the stack for accounts threads
1917        // rayon needs a lot of stack
1918        const ACCOUNTS_STACK_SIZE: usize = 8 * 1024 * 1024;
1919
1920        AccountsDb {
1921            verify_accounts_hash_in_bg: VerifyAccountsHashInBackground::default(),
1922            filler_accounts_per_slot: AtomicU64::default(),
1923            filler_account_slots_remaining: AtomicU64::default(),
1924            active_stats: ActiveStats::default(),
1925            accounts_hash_complete_one_epoch_old: RwLock::default(),
1926            skip_rewrites: false,
1927            skip_initial_hash_calc: false,
1928            ancient_append_vecs: false,
1929            accounts_index,
1930            storage: AccountStorage::default(),
1931            accounts_cache: AccountsCache::default(),
1932            sender_bg_hasher: None,
1933            read_only_accounts_cache: ReadOnlyAccountsCache::new(MAX_READ_ONLY_CACHE_DATA_SIZE),
1934            recycle_stores: RwLock::new(RecycleStores::default()),
1935            uncleaned_pubkeys: DashMap::new(),
1936            next_id: AtomicAppendVecId::new(0),
1937            shrink_candidate_slots_v1: Mutex::new(Vec::new()),
1938            shrink_candidate_slots: Mutex::new(HashMap::new()),
1939            write_cache_limit_bytes: None,
1940            write_version: AtomicU64::new(0),
1941            paths: vec![],
1942            accounts_hash_cache_path,
1943            temp_accounts_hash_cache_path,
1944            shrink_paths: RwLock::new(None),
1945            temp_paths: None,
1946            file_size: DEFAULT_FILE_SIZE,
1947            thread_pool: rayon::ThreadPoolBuilder::new()
1948                .num_threads(num_threads)
1949                .thread_name(|i| format!("solAccounts{:02}", i))
1950                .stack_size(ACCOUNTS_STACK_SIZE)
1951                .build()
1952                .unwrap(),
1953            thread_pool_clean: make_min_priority_thread_pool(),
1954            min_num_stores: num_threads,
1955            bank_hashes: RwLock::new(bank_hashes),
1956            external_purge_slots_stats: PurgeStats::default(),
1957            clean_accounts_stats: CleanAccountsStats::default(),
1958            shrink_stats: ShrinkStats::default(),
1959            shrink_ancient_stats: ShrinkAncientStats::default(),
1960            stats: AccountsStats::default(),
1961            cluster_type: None,
1962            account_indexes: AccountSecondaryIndexes::default(),
1963            caching_enabled: false,
1964            #[cfg(test)]
1965            load_delay: u64::default(),
1966            #[cfg(test)]
1967            load_limit: AtomicU64::default(),
1968            is_bank_drop_callback_enabled: AtomicBool::default(),
1969            remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization::default(),
1970            shrink_ratio: AccountShrinkThreshold::default(),
1971            dirty_stores: DashMap::default(),
1972            zero_lamport_accounts_to_purge_after_full_snapshot: DashSet::default(),
1973            accounts_update_notifier: None,
1974            filler_accounts_config: FillerAccountsConfig::default(),
1975            filler_account_suffix: None,
1976            num_hash_scan_passes,
1977            log_dead_slots: AtomicBool::new(true),
1978        }
1979    }
1980
1981    pub fn new_for_tests(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
1982        AccountsDb::new_with_config(
1983            paths,
1984            cluster_type,
1985            AccountSecondaryIndexes::default(),
1986            false,
1987            AccountShrinkThreshold::default(),
1988            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
1989            None,
1990        )
1991    }
1992
1993    pub fn new_for_tests_with_caching(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
1994        AccountsDb::new_with_config(
1995            paths,
1996            cluster_type,
1997            AccountSecondaryIndexes::default(),
1998            true,
1999            AccountShrinkThreshold::default(),
2000            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
2001            None,
2002        )
2003    }
2004
2005    pub fn new_with_config(
2006        paths: Vec<PathBuf>,
2007        cluster_type: &ClusterType,
2008        account_indexes: AccountSecondaryIndexes,
2009        caching_enabled: bool,
2010        shrink_ratio: AccountShrinkThreshold,
2011        mut accounts_db_config: Option<AccountsDbConfig>,
2012        accounts_update_notifier: Option<AccountsUpdateNotifier>,
2013    ) -> Self {
2014        let accounts_index =
2015            AccountsIndex::new(accounts_db_config.as_mut().and_then(|x| x.index.take()));
2016        let accounts_hash_cache_path = accounts_db_config
2017            .as_ref()
2018            .and_then(|x| x.accounts_hash_cache_path.clone());
2019
2020        let filler_accounts_config = accounts_db_config
2021            .as_ref()
2022            .map(|config| config.filler_accounts_config)
2023            .unwrap_or_default();
2024        let skip_rewrites = accounts_db_config
2025            .as_ref()
2026            .map(|config| config.skip_rewrites)
2027            .unwrap_or_default();
2028        let skip_initial_hash_calc = accounts_db_config
2029            .as_ref()
2030            .map(|config| config.skip_initial_hash_calc)
2031            .unwrap_or_default();
2032
2033        let ancient_append_vecs = accounts_db_config
2034            .as_ref()
2035            .map(|config| config.ancient_append_vecs)
2036            .unwrap_or_default();
2037
2038        let filler_account_suffix = if filler_accounts_config.count > 0 {
2039            Some(solana_sdk::pubkey::new_rand())
2040        } else {
2041            None
2042        };
2043        let paths_is_empty = paths.is_empty();
2044        let mut new = Self {
2045            paths,
2046            skip_rewrites,
2047            skip_initial_hash_calc,
2048            ancient_append_vecs,
2049            cluster_type: Some(*cluster_type),
2050            account_indexes,
2051            caching_enabled,
2052            shrink_ratio,
2053            accounts_update_notifier,
2054            filler_accounts_config,
2055            filler_account_suffix,
2056            write_cache_limit_bytes: accounts_db_config
2057                .as_ref()
2058                .and_then(|x| x.write_cache_limit_bytes),
2059            ..Self::default_with_accounts_index(
2060                accounts_index,
2061                accounts_hash_cache_path,
2062                accounts_db_config
2063                    .as_ref()
2064                    .and_then(|cfg| cfg.hash_calc_num_passes),
2065            )
2066        };
2067        if paths_is_empty {
2068            // Create a temporary set of accounts directories, used primarily
2069            // for testing
2070            let (temp_dirs, paths) = get_temp_accounts_paths(DEFAULT_NUM_DIRS).unwrap();
2071            new.accounts_update_notifier = None;
2072            new.paths = paths;
2073            new.temp_paths = Some(temp_dirs);
2074        };
2075
2076        new.start_background_hasher();
2077        {
2078            for path in new.paths.iter() {
2079                std::fs::create_dir_all(path).expect("Create directory failed.");
2080            }
2081        }
2082        new
2083    }
2084
2085    /// Gradual means filler accounts will be added over the course of an epoch, during cache flush.
2086    /// This is in contrast to adding all the filler accounts immediately before the validator starts.
2087    fn init_gradual_filler_accounts(&self, slots_per_epoch: Slot) {
2088        let count = self.filler_accounts_config.count;
2089        if count > 0 {
2090            // filler accounts are a debug only feature. integer division is fine here
2091            let accounts_per_slot = (count as u64) / slots_per_epoch;
2092            self.filler_accounts_per_slot
2093                .store(accounts_per_slot, Ordering::Release);
2094            self.filler_account_slots_remaining
2095                .store(slots_per_epoch, Ordering::Release);
2096        }
2097    }
2098
2099    pub fn set_shrink_paths(&self, paths: Vec<PathBuf>) {
2100        assert!(!paths.is_empty());
2101        let mut shrink_paths = self.shrink_paths.write().unwrap();
2102        for path in &paths {
2103            std::fs::create_dir_all(path).expect("Create directory failed.");
2104        }
2105        *shrink_paths = Some(paths);
2106    }
2107
2108    pub fn file_size(&self) -> u64 {
2109        self.file_size
2110    }
2111
2112    pub fn new_single_for_tests() -> Self {
2113        AccountsDb {
2114            min_num_stores: 0,
2115            ..AccountsDb::new_for_tests(Vec::new(), &ClusterType::Development)
2116        }
2117    }
2118
2119    pub fn new_single_for_tests_with_caching() -> Self {
2120        AccountsDb {
2121            min_num_stores: 0,
2122            ..AccountsDb::new_for_tests_with_caching(Vec::new(), &ClusterType::Development)
2123        }
2124    }
2125
2126    fn next_id(&self) -> AppendVecId {
2127        let next_id = self.next_id.fetch_add(1, Ordering::AcqRel);
2128        assert!(next_id != AppendVecId::MAX, "We've run out of storage ids!");
2129        next_id
2130    }
2131
2132    fn new_storage_entry(&self, slot: Slot, path: &Path, size: u64) -> AccountStorageEntry {
2133        AccountStorageEntry::new(path, slot, self.next_id(), size)
2134    }
2135
2136    pub fn expected_cluster_type(&self) -> ClusterType {
2137        self.cluster_type
2138            .expect("Cluster type must be set at initialization")
2139    }
2140
2141    /// Reclaim older states of accounts older than max_clean_root for AccountsDb bloat mitigation
2142    fn clean_accounts_older_than_root(
2143        &self,
2144        purges: Vec<Pubkey>,
2145        max_clean_root: Option<Slot>,
2146        ancient_account_cleans: &AtomicU64,
2147    ) -> (ReclaimResult, PubkeysRemovedFromAccountsIndex) {
2148        let pubkeys_removed_from_accounts_index = HashSet::default();
2149        if purges.is_empty() {
2150            return (
2151                ReclaimResult::default(),
2152                pubkeys_removed_from_accounts_index,
2153            );
2154        }
2155        // This number isn't carefully chosen; just guessed randomly such that
2156        // the hot loop will be the order of ~Xms.
2157        const INDEX_CLEAN_BULK_COUNT: usize = 4096;
2158
2159        let one_epoch_old = self.get_accounts_hash_complete_one_epoch_old();
2160        let pubkeys_removed_from_accounts_index = Mutex::new(pubkeys_removed_from_accounts_index);
2161
2162        let mut clean_rooted = Measure::start("clean_old_root-ms");
2163        let reclaim_vecs = purges
2164            .par_chunks(INDEX_CLEAN_BULK_COUNT)
2165            .filter_map(|pubkeys: &[Pubkey]| {
2166                let mut reclaims = Vec::new();
2167                for pubkey in pubkeys {
2168                    let removed_from_index = self.accounts_index.clean_rooted_entries(
2169                        pubkey,
2170                        &mut reclaims,
2171                        max_clean_root,
2172                    );
2173
2174                    if removed_from_index {
2175                        pubkeys_removed_from_accounts_index
2176                            .lock()
2177                            .unwrap()
2178                            .insert(*pubkey);
2179                    }
2180                }
2181
2182                (!reclaims.is_empty()).then(|| {
2183                    // figure out how many ancient accounts have been reclaimed
2184                    let old_reclaims = reclaims
2185                        .iter()
2186                        .filter_map(|(slot, _)| (slot < &one_epoch_old).then(|| 1))
2187                        .sum();
2188                    ancient_account_cleans.fetch_add(old_reclaims, Ordering::Relaxed);
2189                    reclaims
2190                })
2191            })
2192            .collect::<Vec<_>>();
2193        clean_rooted.stop();
2194        inc_new_counter_info!("clean-old-root-par-clean-ms", clean_rooted.as_ms() as usize);
2195        let pubkeys_removed_from_accounts_index =
2196            pubkeys_removed_from_accounts_index.into_inner().unwrap();
2197        self.clean_accounts_stats
2198            .clean_old_root_us
2199            .fetch_add(clean_rooted.as_us(), Ordering::Relaxed);
2200
2201        let mut measure = Measure::start("clean_old_root_reclaims");
2202
2203        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
2204        // and those stores may be used for background hashing.
2205        let reset_accounts = false;
2206
2207        let mut reclaim_result = ReclaimResult::default();
2208        self.handle_reclaims(
2209            (!reclaim_vecs.is_empty()).then(|| reclaim_vecs.iter().flatten()),
2210            None,
2211            Some((&self.clean_accounts_stats.purge_stats, &mut reclaim_result)),
2212            reset_accounts,
2213            &pubkeys_removed_from_accounts_index,
2214        );
2215        measure.stop();
2216        debug!("{} {}", clean_rooted, measure);
2217        inc_new_counter_info!("clean-old-root-reclaim-ms", measure.as_ms() as usize);
2218        self.clean_accounts_stats
2219            .clean_old_root_reclaim_us
2220            .fetch_add(measure.as_us(), Ordering::Relaxed);
2221        (reclaim_result, pubkeys_removed_from_accounts_index)
2222    }
2223
2224    fn do_reset_uncleaned_roots(&self, max_clean_root: Option<Slot>) {
2225        let mut measure = Measure::start("reset");
2226        self.accounts_index.reset_uncleaned_roots(max_clean_root);
2227        measure.stop();
2228        self.clean_accounts_stats
2229            .reset_uncleaned_roots_us
2230            .fetch_add(measure.as_us(), Ordering::Relaxed);
2231    }
2232
2233    fn calc_delete_dependencies(
2234        purges: &HashMap<Pubkey, (SlotList<AccountInfo>, RefCount)>,
2235        store_counts: &mut HashMap<AppendVecId, (usize, HashSet<Pubkey>)>,
2236    ) {
2237        // Another pass to check if there are some filtered accounts which
2238        // do not match the criteria of deleting all appendvecs which contain them
2239        // then increment their storage count.
2240        let mut already_counted = HashSet::new();
2241        for (pubkey, (account_infos, ref_count_from_storage)) in purges.iter() {
2242            let no_delete = if account_infos.len() as RefCount != *ref_count_from_storage {
2243                debug!(
2244                    "calc_delete_dependencies(),
2245                    pubkey: {},
2246                    account_infos: {:?},
2247                    account_infos_len: {},
2248                    ref_count_from_storage: {}",
2249                    pubkey,
2250                    account_infos,
2251                    account_infos.len(),
2252                    ref_count_from_storage,
2253                );
2254                true
2255            } else {
2256                let mut no_delete = false;
2257                for (_slot, account_info) in account_infos {
2258                    debug!(
2259                        "calc_delete_dependencies()
2260                        storage id: {},
2261                        count len: {}",
2262                        account_info.store_id(),
2263                        store_counts.get(&account_info.store_id()).unwrap().0,
2264                    );
2265                    if store_counts.get(&account_info.store_id()).unwrap().0 != 0 {
2266                        no_delete = true;
2267                        break;
2268                    }
2269                }
2270                no_delete
2271            };
2272            if no_delete {
2273                let mut pending_store_ids = HashSet::new();
2274                for (_bank_id, account_info) in account_infos {
2275                    if !already_counted.contains(&account_info.store_id()) {
2276                        pending_store_ids.insert(account_info.store_id());
2277                    }
2278                }
2279                while !pending_store_ids.is_empty() {
2280                    let id = pending_store_ids.iter().next().cloned().unwrap();
2281                    pending_store_ids.remove(&id);
2282                    if !already_counted.insert(id) {
2283                        continue;
2284                    }
2285                    store_counts.get_mut(&id).unwrap().0 += 1;
2286
2287                    let affected_pubkeys = &store_counts.get(&id).unwrap().1;
2288                    for key in affected_pubkeys {
2289                        for (_slot, account_info) in &purges.get(key).unwrap().0 {
2290                            if !already_counted.contains(&account_info.store_id()) {
2291                                pending_store_ids.insert(account_info.store_id());
2292                            }
2293                        }
2294                    }
2295                }
2296            }
2297        }
2298    }
2299
2300    fn background_hasher(receiver: Receiver<CachedAccount>) {
2301        loop {
2302            let result = receiver.recv();
2303            match result {
2304                Ok(account) => {
2305                    // if we hold the only ref, then this account doesn't need to be hashed, we ignore this account and it will disappear
2306                    if Arc::strong_count(&account) > 1 {
2307                        // this will cause the hash to be calculated and store inside account if it needs to be calculated
2308                        let _ = (*account).hash();
2309                    };
2310                }
2311                Err(_) => {
2312                    break;
2313                }
2314            }
2315        }
2316    }
2317
2318    fn start_background_hasher(&mut self) {
2319        let (sender, receiver) = unbounded();
2320        Builder::new()
2321            .name("solDbStoreHashr".to_string())
2322            .spawn(move || {
2323                Self::background_hasher(receiver);
2324            })
2325            .unwrap();
2326        self.sender_bg_hasher = Some(sender);
2327    }
2328
2329    #[must_use]
2330    pub(crate) fn purge_keys_exact<'a, C: 'a>(
2331        &'a self,
2332        pubkey_to_slot_set: impl Iterator<Item = &'a (Pubkey, C)>,
2333    ) -> (Vec<(Slot, AccountInfo)>, PubkeysRemovedFromAccountsIndex)
2334    where
2335        C: Contains<'a, Slot>,
2336    {
2337        let mut reclaims = Vec::new();
2338        let mut dead_keys = Vec::new();
2339
2340        for (pubkey, slots_set) in pubkey_to_slot_set {
2341            let is_empty = self
2342                .accounts_index
2343                .purge_exact(pubkey, slots_set, &mut reclaims);
2344            if is_empty {
2345                dead_keys.push(pubkey);
2346            }
2347        }
2348
2349        let pubkeys_removed_from_accounts_index = self
2350            .accounts_index
2351            .handle_dead_keys(&dead_keys, &self.account_indexes);
2352        (reclaims, pubkeys_removed_from_accounts_index)
2353    }
2354
2355    fn max_clean_root(&self, proposed_clean_root: Option<Slot>) -> Option<Slot> {
2356        match (
2357            self.accounts_index.min_ongoing_scan_root(),
2358            proposed_clean_root,
2359        ) {
2360            (None, None) => None,
2361            (Some(min_scan_root), None) => Some(min_scan_root),
2362            (None, Some(proposed_clean_root)) => Some(proposed_clean_root),
2363            (Some(min_scan_root), Some(proposed_clean_root)) => {
2364                Some(std::cmp::min(min_scan_root, proposed_clean_root))
2365            }
2366        }
2367    }
2368
2369    /// return 'slot' - slots_in_epoch
2370    fn get_slot_one_epoch_prior(slot: Slot, epoch_schedule: &EpochSchedule) -> Slot {
2371        // would like to use:
2372        // slot.saturating_sub(epoch_schedule.get_slots_in_epoch(epoch_schedule.get_epoch(slot)))
2373        // but there are problems with warmup and such on tests and probably test clusters.
2374        // So, just use the maximum below (epoch_schedule.slots_per_epoch)
2375        slot.saturating_sub(epoch_schedule.slots_per_epoch)
2376    }
2377
2378    /// hash calc is completed as of 'slot'
2379    /// so, any process that wants to take action on really old slots can now proceed up to 'completed_slot'-slots per epoch
2380    pub fn notify_accounts_hash_calculated_complete(
2381        &self,
2382        completed_slot: Slot,
2383        epoch_schedule: &EpochSchedule,
2384    ) {
2385        let one_epoch_old_slot = Self::get_slot_one_epoch_prior(completed_slot, epoch_schedule);
2386        let mut accounts_hash_complete_one_epoch_old =
2387            self.accounts_hash_complete_one_epoch_old.write().unwrap();
2388        *accounts_hash_complete_one_epoch_old =
2389            std::cmp::max(*accounts_hash_complete_one_epoch_old, one_epoch_old_slot);
2390        let accounts_hash_complete_one_epoch_old = *accounts_hash_complete_one_epoch_old;
2391
2392        // now that calculate_accounts_hash_without_index is complete, we can remove old historical roots
2393        self.remove_old_historical_roots(accounts_hash_complete_one_epoch_old);
2394    }
2395
2396    /// get the slot that is one epoch older than the highest slot that has been used for hash calculation
2397    fn get_accounts_hash_complete_one_epoch_old(&self) -> Slot {
2398        *self.accounts_hash_complete_one_epoch_old.read().unwrap()
2399    }
2400
2401    /// Collect all the uncleaned slots, up to a max slot
2402    ///
2403    /// Search through the uncleaned Pubkeys and return all the slots, up to a maximum slot.
2404    fn collect_uncleaned_slots_up_to_slot(&self, max_slot: Slot) -> Vec<Slot> {
2405        self.uncleaned_pubkeys
2406            .iter()
2407            .filter_map(|entry| {
2408                let slot = *entry.key();
2409                (slot <= max_slot).then(|| slot)
2410            })
2411            .collect()
2412    }
2413
2414    /// Remove `slots` from `uncleaned_pubkeys` and collect all pubkeys
2415    ///
2416    /// For each slot in the list of uncleaned slots, remove it from the `uncleaned_pubkeys` Map
2417    /// and collect all the pubkeys to return.
2418    fn remove_uncleaned_slots_and_collect_pubkeys(
2419        &self,
2420        uncleaned_slots: Vec<Slot>,
2421    ) -> Vec<Vec<Pubkey>> {
2422        uncleaned_slots
2423            .into_iter()
2424            .filter_map(|uncleaned_slot| {
2425                self.uncleaned_pubkeys
2426                    .remove(&uncleaned_slot)
2427                    .map(|(_removed_slot, removed_pubkeys)| removed_pubkeys)
2428            })
2429            .collect()
2430    }
2431
2432    /// Remove uncleaned slots, up to a maximum slot, and return the collected pubkeys
2433    ///
2434    fn remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(
2435        &self,
2436        max_slot: Slot,
2437    ) -> Vec<Vec<Pubkey>> {
2438        let uncleaned_slots = self.collect_uncleaned_slots_up_to_slot(max_slot);
2439        self.remove_uncleaned_slots_and_collect_pubkeys(uncleaned_slots)
2440    }
2441
2442    // Construct a vec of pubkeys for cleaning from:
2443    //   uncleaned_pubkeys - the delta set of updated pubkeys in rooted slots from the last clean
2444    //   dirty_stores - set of stores which had accounts removed or recently rooted
2445    fn construct_candidate_clean_keys(
2446        &self,
2447        max_clean_root: Option<Slot>,
2448        last_full_snapshot_slot: Option<Slot>,
2449        timings: &mut CleanKeyTimings,
2450    ) -> Vec<Pubkey> {
2451        let mut dirty_store_processing_time = Measure::start("dirty_store_processing");
2452        let max_slot = max_clean_root.unwrap_or_else(|| self.accounts_index.max_root_inclusive());
2453        let mut dirty_stores = Vec::with_capacity(self.dirty_stores.len());
2454        self.dirty_stores.retain(|(slot, _store_id), store| {
2455            if *slot > max_slot {
2456                true
2457            } else {
2458                dirty_stores.push((*slot, store.clone()));
2459                false
2460            }
2461        });
2462        let dirty_stores_len = dirty_stores.len();
2463        let pubkeys = DashSet::new();
2464        timings.oldest_dirty_slot = max_slot.saturating_add(1);
2465        for (slot, store) in dirty_stores {
2466            timings.oldest_dirty_slot = std::cmp::min(timings.oldest_dirty_slot, slot);
2467            store.accounts.account_iter().for_each(|account| {
2468                pubkeys.insert(account.meta.pubkey);
2469            });
2470        }
2471        trace!(
2472            "dirty_stores.len: {} pubkeys.len: {}",
2473            dirty_stores_len,
2474            pubkeys.len()
2475        );
2476        timings.dirty_pubkeys_count = pubkeys.len() as u64;
2477        dirty_store_processing_time.stop();
2478        timings.dirty_store_processing_us += dirty_store_processing_time.as_us();
2479
2480        let mut collect_delta_keys = Measure::start("key_create");
2481        let delta_keys = self.remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(max_slot);
2482        collect_delta_keys.stop();
2483        timings.collect_delta_keys_us += collect_delta_keys.as_us();
2484
2485        let mut delta_insert = Measure::start("delta_insert");
2486        self.thread_pool_clean.install(|| {
2487            delta_keys.par_iter().for_each(|keys| {
2488                for key in keys {
2489                    pubkeys.insert(*key);
2490                }
2491            });
2492        });
2493        delta_insert.stop();
2494        timings.delta_insert_us += delta_insert.as_us();
2495
2496        timings.delta_key_count = pubkeys.len() as u64;
2497
2498        let mut hashset_to_vec = Measure::start("flat_map");
2499        let mut pubkeys: Vec<Pubkey> = pubkeys.into_iter().collect();
2500        hashset_to_vec.stop();
2501        timings.hashset_to_vec_us += hashset_to_vec.as_us();
2502
2503        // Check if we should purge any of the zero_lamport_accounts_to_purge_later, based on the
2504        // last_full_snapshot_slot.
2505        assert!(
2506            last_full_snapshot_slot.is_some() || self.zero_lamport_accounts_to_purge_after_full_snapshot.is_empty(),
2507            "if snapshots are disabled, then zero_lamport_accounts_to_purge_later should always be empty"
2508        );
2509        if let Some(last_full_snapshot_slot) = last_full_snapshot_slot {
2510            self.zero_lamport_accounts_to_purge_after_full_snapshot
2511                .retain(|(slot, pubkey)| {
2512                    let is_candidate_for_clean =
2513                        max_slot >= *slot && last_full_snapshot_slot >= *slot;
2514                    if is_candidate_for_clean {
2515                        pubkeys.push(*pubkey);
2516                    }
2517                    !is_candidate_for_clean
2518                });
2519        }
2520
2521        pubkeys
2522    }
2523
2524    // Purge zero lamport accounts and older rooted account states as garbage
2525    // collection
2526    // Only remove those accounts where the entire rooted history of the account
2527    // can be purged because there are no live append vecs in the ancestors
2528    pub fn clean_accounts(
2529        &self,
2530        max_clean_root: Option<Slot>,
2531        is_startup: bool,
2532        last_full_snapshot_slot: Option<Slot>,
2533    ) {
2534        let _guard = self.active_stats.activate(ActiveStatItem::Clean);
2535
2536        let ancient_account_cleans = AtomicU64::default();
2537
2538        let mut measure_all = Measure::start("clean_accounts");
2539        let max_clean_root = self.max_clean_root(max_clean_root);
2540
2541        // hold a lock to prevent slot shrinking from running because it might modify some rooted
2542        // slot storages which can not happen as long as we're cleaning accounts because we're also
2543        // modifying the rooted slot storages!
2544        let mut candidates_v1 = self.shrink_candidate_slots_v1.lock().unwrap();
2545        self.report_store_stats();
2546
2547        let mut key_timings = CleanKeyTimings::default();
2548        let mut pubkeys = self.construct_candidate_clean_keys(
2549            max_clean_root,
2550            last_full_snapshot_slot,
2551            &mut key_timings,
2552        );
2553
2554        let mut sort = Measure::start("sort");
2555        if is_startup {
2556            pubkeys.par_sort_unstable();
2557        } else {
2558            self.thread_pool_clean
2559                .install(|| pubkeys.par_sort_unstable());
2560        }
2561        sort.stop();
2562
2563        let total_keys_count = pubkeys.len();
2564        let mut accounts_scan = Measure::start("accounts_scan");
2565        let uncleaned_roots = self.accounts_index.clone_uncleaned_roots();
2566        let found_not_zero_accum = AtomicU64::new(0);
2567        let not_found_on_fork_accum = AtomicU64::new(0);
2568        let missing_accum = AtomicU64::new(0);
2569        let useful_accum = AtomicU64::new(0);
2570
2571        // parallel scan the index.
2572        let (mut purges_zero_lamports, purges_old_accounts) = {
2573            let do_clean_scan = || {
2574                pubkeys
2575                    .par_chunks(4096)
2576                    .map(|pubkeys: &[Pubkey]| {
2577                        let mut purges_zero_lamports = HashMap::new();
2578                        let mut purges_old_accounts = Vec::new();
2579                        let mut found_not_zero = 0;
2580                        let mut not_found_on_fork = 0;
2581                        let mut missing = 0;
2582                        let mut useful = 0;
2583                        self.accounts_index
2584                            .scan(pubkeys.iter(), |pubkey, slots_refs| {
2585                                let mut useless = true;
2586                                if let Some((slot_list, ref_count)) = slots_refs {
2587                                    let index_in_slot_list = self.accounts_index.latest_slot(
2588                                        None,
2589                                        slot_list,
2590                                        max_clean_root,
2591                                    );
2592
2593                                    match index_in_slot_list {
2594                                        Some(index_in_slot_list) => {
2595                                            // found info relative to max_clean_root
2596                                            let (slot, account_info) =
2597                                                &slot_list[index_in_slot_list];
2598                                            if account_info.is_zero_lamport() {
2599                                                useless = false;
2600                                                purges_zero_lamports.insert(
2601                                                    *pubkey,
2602                                                    (
2603                                                        self.accounts_index.get_rooted_entries(
2604                                                            slot_list,
2605                                                            max_clean_root,
2606                                                        ),
2607                                                        ref_count,
2608                                                    ),
2609                                                );
2610                                            } else {
2611                                                found_not_zero += 1;
2612                                            }
2613                                            if uncleaned_roots.contains(slot) {
2614                                                // Assertion enforced by `accounts_index.get()`, the latest slot
2615                                                // will not be greater than the given `max_clean_root`
2616                                                if let Some(max_clean_root) = max_clean_root {
2617                                                    assert!(slot <= &max_clean_root);
2618                                                }
2619                                                purges_old_accounts.push(*pubkey);
2620                                                useless = false;
2621                                            }
2622                                        }
2623                                        None => {
2624                                            // This pubkey is in the index but not in a root slot, so clean
2625                                            // it up by adding it to the to-be-purged list.
2626                                            //
2627                                            // Also, this pubkey must have been touched by some slot since
2628                                            // it was in the dirty list, so we assume that the slot it was
2629                                            // touched in must be unrooted.
2630                                            not_found_on_fork += 1;
2631                                            useless = false;
2632                                            purges_old_accounts.push(*pubkey);
2633                                        }
2634                                    }
2635                                } else {
2636                                    missing += 1;
2637                                }
2638                                if !useless {
2639                                    useful += 1;
2640                                }
2641                                if useless {
2642                                    AccountsIndexScanResult::None
2643                                } else {
2644                                    AccountsIndexScanResult::KeepInMemory
2645                                }
2646                            });
2647                        found_not_zero_accum.fetch_add(found_not_zero, Ordering::Relaxed);
2648                        not_found_on_fork_accum.fetch_add(not_found_on_fork, Ordering::Relaxed);
2649                        missing_accum.fetch_add(missing, Ordering::Relaxed);
2650                        useful_accum.fetch_add(useful, Ordering::Relaxed);
2651                        (purges_zero_lamports, purges_old_accounts)
2652                    })
2653                    .reduce(
2654                        || (HashMap::new(), Vec::new()),
2655                        |mut m1, m2| {
2656                            // Collapse down the hashmaps/vecs into one.
2657                            m1.0.extend(m2.0);
2658                            m1.1.extend(m2.1);
2659                            m1
2660                        },
2661                    )
2662            };
2663            if is_startup {
2664                do_clean_scan()
2665            } else {
2666                self.thread_pool_clean.install(do_clean_scan)
2667            }
2668        };
2669        accounts_scan.stop();
2670
2671        let mut clean_old_rooted = Measure::start("clean_old_roots");
2672        let ((purged_account_slots, removed_accounts), mut pubkeys_removed_from_accounts_index) =
2673            self.clean_accounts_older_than_root(
2674                purges_old_accounts,
2675                max_clean_root,
2676                &ancient_account_cleans,
2677            );
2678
2679        if self.caching_enabled {
2680            self.do_reset_uncleaned_roots(max_clean_root);
2681        } else {
2682            self.do_reset_uncleaned_roots_v1(&mut candidates_v1, max_clean_root);
2683        }
2684        clean_old_rooted.stop();
2685
2686        let mut store_counts_time = Measure::start("store_counts");
2687
2688        // Calculate store counts as if everything was purged
2689        // Then purge if we can
2690        let mut store_counts: HashMap<AppendVecId, (usize, HashSet<Pubkey>)> = HashMap::new();
2691        for (key, (account_infos, ref_count)) in purges_zero_lamports.iter_mut() {
2692            if purged_account_slots.contains_key(key) {
2693                *ref_count = self.accounts_index.ref_count_from_storage(key);
2694            }
2695            account_infos.retain(|(slot, account_info)| {
2696                let was_slot_purged = purged_account_slots
2697                    .get(key)
2698                    .map(|slots_removed| slots_removed.contains(slot))
2699                    .unwrap_or(false);
2700                if was_slot_purged {
2701                    // No need to look up the slot storage below if the entire
2702                    // slot was purged
2703                    return false;
2704                }
2705                // Check if this update in `slot` to the account with `key` was reclaimed earlier by
2706                // `clean_accounts_older_than_root()`
2707                let was_reclaimed = removed_accounts
2708                    .get(&account_info.store_id())
2709                    .map(|store_removed| store_removed.contains(&account_info.offset()))
2710                    .unwrap_or(false);
2711                if was_reclaimed {
2712                    return false;
2713                }
2714                if let Some(store_count) = store_counts.get_mut(&account_info.store_id()) {
2715                    store_count.0 -= 1;
2716                    store_count.1.insert(*key);
2717                } else {
2718                    let mut key_set = HashSet::new();
2719                    key_set.insert(*key);
2720                    assert!(
2721                        !account_info.is_cached(),
2722                        "The Accounts Cache must be flushed first for this account info. pubkey: {}, slot: {}",
2723                        *key,
2724                        *slot
2725                    );
2726                    let count = self
2727                        .storage
2728                        .slot_store_count(*slot, account_info.store_id())
2729                        .unwrap()
2730                        - 1;
2731                    debug!(
2732                        "store_counts, inserting slot: {}, store id: {}, count: {}",
2733                        slot, account_info.store_id(), count
2734                    );
2735                    store_counts.insert(account_info.store_id(), (count, key_set));
2736                }
2737                true
2738            });
2739        }
2740        store_counts_time.stop();
2741
2742        let mut calc_deps_time = Measure::start("calc_deps");
2743        Self::calc_delete_dependencies(&purges_zero_lamports, &mut store_counts);
2744        calc_deps_time.stop();
2745
2746        let mut purge_filter = Measure::start("purge_filter");
2747        self.filter_zero_lamport_clean_for_incremental_snapshots(
2748            max_clean_root,
2749            last_full_snapshot_slot,
2750            &store_counts,
2751            &mut purges_zero_lamports,
2752        );
2753        purge_filter.stop();
2754
2755        let mut reclaims_time = Measure::start("reclaims");
2756        // Recalculate reclaims with new purge set
2757        let pubkey_to_slot_set: Vec<_> = purges_zero_lamports
2758            .into_iter()
2759            .map(|(key, (slots_list, _ref_count))| {
2760                (
2761                    key,
2762                    slots_list
2763                        .into_iter()
2764                        .map(|(slot, _)| slot)
2765                        .collect::<HashSet<Slot>>(),
2766                )
2767            })
2768            .collect();
2769
2770        let (reclaims, pubkeys_removed_from_accounts_index2) =
2771            self.purge_keys_exact(pubkey_to_slot_set.iter());
2772        pubkeys_removed_from_accounts_index
2773            .extend(pubkeys_removed_from_accounts_index2.into_iter());
2774
2775        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
2776        // and those stores may be used for background hashing.
2777        let reset_accounts = false;
2778        let mut reclaim_result = ReclaimResult::default();
2779        self.handle_reclaims(
2780            (!reclaims.is_empty()).then(|| reclaims.iter()),
2781            None,
2782            Some((&self.clean_accounts_stats.purge_stats, &mut reclaim_result)),
2783            reset_accounts,
2784            &pubkeys_removed_from_accounts_index,
2785        );
2786
2787        reclaims_time.stop();
2788        measure_all.stop();
2789
2790        self.clean_accounts_stats.report();
2791        datapoint_info!(
2792            "clean_accounts",
2793            ("total_us", measure_all.as_us(), i64),
2794            (
2795                "collect_delta_keys_us",
2796                key_timings.collect_delta_keys_us,
2797                i64
2798            ),
2799            ("oldest_dirty_slot", key_timings.oldest_dirty_slot, i64),
2800            (
2801                "pubkeys_removed_from_accounts_index",
2802                pubkeys_removed_from_accounts_index.len(),
2803                i64
2804            ),
2805            (
2806                "dirty_store_processing_us",
2807                key_timings.dirty_store_processing_us,
2808                i64
2809            ),
2810            ("accounts_scan", accounts_scan.as_us() as i64, i64),
2811            ("clean_old_rooted", clean_old_rooted.as_us() as i64, i64),
2812            ("store_counts", store_counts_time.as_us() as i64, i64),
2813            ("purge_filter", purge_filter.as_us() as i64, i64),
2814            ("calc_deps", calc_deps_time.as_us() as i64, i64),
2815            ("reclaims", reclaims_time.as_us() as i64, i64),
2816            ("delta_insert_us", key_timings.delta_insert_us, i64),
2817            ("delta_key_count", key_timings.delta_key_count, i64),
2818            ("dirty_pubkeys_count", key_timings.dirty_pubkeys_count, i64),
2819            ("sort_us", sort.as_us(), i64),
2820            ("useful_keys", useful_accum.load(Ordering::Relaxed), i64),
2821            ("total_keys_count", total_keys_count, i64),
2822            (
2823                "scan_found_not_zero",
2824                found_not_zero_accum.load(Ordering::Relaxed),
2825                i64
2826            ),
2827            (
2828                "scan_not_found_on_fork",
2829                not_found_on_fork_accum.load(Ordering::Relaxed),
2830                i64
2831            ),
2832            ("scan_missing", missing_accum.load(Ordering::Relaxed), i64),
2833            ("uncleaned_roots_len", uncleaned_roots.len(), i64),
2834            (
2835                "clean_old_root_us",
2836                self.clean_accounts_stats
2837                    .clean_old_root_us
2838                    .swap(0, Ordering::Relaxed),
2839                i64
2840            ),
2841            (
2842                "clean_old_root_reclaim_us",
2843                self.clean_accounts_stats
2844                    .clean_old_root_reclaim_us
2845                    .swap(0, Ordering::Relaxed),
2846                i64
2847            ),
2848            (
2849                "reset_uncleaned_roots_us",
2850                self.clean_accounts_stats
2851                    .reset_uncleaned_roots_us
2852                    .swap(0, Ordering::Relaxed),
2853                i64
2854            ),
2855            (
2856                "remove_dead_accounts_remove_us",
2857                self.clean_accounts_stats
2858                    .remove_dead_accounts_remove_us
2859                    .swap(0, Ordering::Relaxed),
2860                i64
2861            ),
2862            (
2863                "remove_dead_accounts_shrink_us",
2864                self.clean_accounts_stats
2865                    .remove_dead_accounts_shrink_us
2866                    .swap(0, Ordering::Relaxed),
2867                i64
2868            ),
2869            (
2870                "clean_stored_dead_slots_us",
2871                self.clean_accounts_stats
2872                    .clean_stored_dead_slots_us
2873                    .swap(0, Ordering::Relaxed),
2874                i64
2875            ),
2876            (
2877                "roots_added",
2878                self.accounts_index.roots_added.swap(0, Ordering::Relaxed) as i64,
2879                i64
2880            ),
2881            (
2882                "roots_removed",
2883                self.accounts_index.roots_removed.swap(0, Ordering::Relaxed) as i64,
2884                i64
2885            ),
2886            (
2887                "active_scans",
2888                self.accounts_index.active_scans.load(Ordering::Relaxed) as i64,
2889                i64
2890            ),
2891            (
2892                "max_distance_to_min_scan_slot",
2893                self.accounts_index
2894                    .max_distance_to_min_scan_slot
2895                    .swap(0, Ordering::Relaxed),
2896                i64
2897            ),
2898            (
2899                "ancient_account_cleans",
2900                ancient_account_cleans.load(Ordering::Relaxed),
2901                i64
2902            ),
2903            ("next_store_id", self.next_id.load(Ordering::Relaxed), i64),
2904        );
2905    }
2906
2907    /// Removes the accounts in the input `reclaims` from the tracked "count" of
2908    /// their corresponding  storage entries. Note this does not actually free
2909    /// the memory from the storage entries until all the storage entries for
2910    /// a given slot `S` are empty, at which point `process_dead_slots` will
2911    /// remove all the storage entries for `S`.
2912    ///
2913    /// # Arguments
2914    /// * `reclaims` - The accounts to remove from storage entries' "count". Note here
2915    ///    that we should not remove cache entries, only entries for accounts actually
2916    ///    stored in a storage entry.
2917    ///
2918    /// * `expected_single_dead_slot` - A correctness assertion. If this is equal to `Some(S)`,
2919    ///    then the function will check that the only slot being cleaned up in `reclaims`
2920    ///    is the slot == `S`. This is true for instance when `handle_reclaims` is called
2921    ///    from store or slot shrinking, as those should only touch the slot they are
2922    ///    currently storing to or shrinking.
2923    ///
2924    /// * `purge_stats_and_reclaim_result` - Option containing `purge_stats` and `reclaim_result`.
2925    ///    `purge_stats`. `purge_stats` are stats used to track performance of purging dead slots.
2926    ///    `reclaim_result` contains information about accounts that were removed from storage,
2927    ///    does not include accounts that were removed from the cache.
2928    ///    If `purge_stats_and_reclaim_result.is_none()`, this implies there can be no dead slots
2929    ///    that happen as a result of this call, and the function will check that no slots are
2930    ///    cleaned up/removed via `process_dead_slots`. For instance, on store, no slots should
2931    ///    be cleaned up, but during the background clean accounts purges accounts from old rooted
2932    ///    slots, so outdated slots may be removed.
2933    ///
2934    /// * `reset_accounts` - Reset the append_vec store when the store is dead (count==0)
2935    ///    From the clean and shrink paths it should be false since there may be an in-progress
2936    ///    hash operation and the stores may hold accounts that need to be unref'ed.
2937    /// * `pubkeys_removed_from_accounts_index` - These keys have already been removed from the accounts index
2938    ///    and should not be unref'd. If they exist in the accounts index, they are NEW.
2939    fn handle_reclaims<'a, I>(
2940        &'a self,
2941        reclaims: Option<I>,
2942        expected_single_dead_slot: Option<Slot>,
2943        purge_stats_and_reclaim_result: Option<(&PurgeStats, &mut ReclaimResult)>,
2944        reset_accounts: bool,
2945        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
2946    ) where
2947        I: Iterator<Item = &'a (Slot, AccountInfo)>,
2948    {
2949        if let Some(reclaims) = reclaims {
2950            let (purge_stats, purged_account_slots, reclaimed_offsets) = if let Some((
2951                purge_stats,
2952                (ref mut purged_account_slots, ref mut reclaimed_offsets),
2953            )) =
2954                purge_stats_and_reclaim_result
2955            {
2956                (
2957                    Some(purge_stats),
2958                    Some(purged_account_slots),
2959                    Some(reclaimed_offsets),
2960                )
2961            } else {
2962                (None, None, None)
2963            };
2964
2965            let dead_slots = self.remove_dead_accounts(
2966                reclaims,
2967                expected_single_dead_slot,
2968                reclaimed_offsets,
2969                reset_accounts,
2970            );
2971
2972            if let Some(purge_stats) = purge_stats {
2973                if let Some(expected_single_dead_slot) = expected_single_dead_slot {
2974                    assert!(dead_slots.len() <= 1);
2975                    if dead_slots.len() == 1 {
2976                        assert!(dead_slots.contains(&expected_single_dead_slot));
2977                    }
2978                }
2979
2980                self.process_dead_slots(
2981                    &dead_slots,
2982                    purged_account_slots,
2983                    purge_stats,
2984                    pubkeys_removed_from_accounts_index,
2985                );
2986            } else {
2987                // not sure why this fails yet with ancient append vecs
2988                if !self.ancient_append_vecs {
2989                    assert!(dead_slots.is_empty());
2990                }
2991            }
2992        }
2993    }
2994
2995    /// During clean, some zero-lamport accounts that are marked for purge should *not* actually
2996    /// get purged.  Filter out those accounts here.
2997    ///
2998    /// When using incremental snapshots, do not purge zero-lamport accounts if the slot is higher
2999    /// than the last full snapshot slot.  This is to protect against the following scenario:
3000    ///
3001    ///   ```text
3002    ///   A full snapshot is taken, and it contains an account with a non-zero balance.  Later,
3003    ///   that account's  goes to zero.  Evntually cleaning runs, and before, this account would be
3004    ///   cleaned up.  Finally, an incremental snapshot is taken.
3005    ///
3006    ///   Later, the incremental (and full) snapshot is used to rebuild the bank and accounts
3007    ///   database (e.x. if the node restarts).  The full snapshot _does_ contain the account (from
3008    ///   above) and its balance is non-zero, however, since the account was cleaned up in a later
3009    ///   slot, the incremental snapshot does not contain any info about this account, thus, the
3010    ///   accounts database will contain the old info from this account, which has its old non-zero
3011    ///   balance.  Very bad!
3012    ///   ```
3013    ///
3014    /// This filtering step can be skipped if there is no `last_full_snapshot_slot`, or if the
3015    /// `max_clean_root` is less-than-or-equal-to the `last_full_snapshot_slot`.
3016    fn filter_zero_lamport_clean_for_incremental_snapshots(
3017        &self,
3018        max_clean_root: Option<Slot>,
3019        last_full_snapshot_slot: Option<Slot>,
3020        store_counts: &HashMap<AppendVecId, (usize, HashSet<Pubkey>)>,
3021        purges_zero_lamports: &mut HashMap<Pubkey, (SlotList<AccountInfo>, RefCount)>,
3022    ) {
3023        let should_filter_for_incremental_snapshots =
3024            max_clean_root.unwrap_or(Slot::MAX) > last_full_snapshot_slot.unwrap_or(Slot::MAX);
3025        assert!(
3026            last_full_snapshot_slot.is_some() || !should_filter_for_incremental_snapshots,
3027            "if filtering for incremental snapshots, then snapshots should be enabled",
3028        );
3029
3030        purges_zero_lamports.retain(|pubkey, (slot_account_infos, _ref_count)| {
3031            // Only keep purges_zero_lamports where the entire history of the account in the root set
3032            // can be purged. All AppendVecs for those updates are dead.
3033            for (_slot, account_info) in slot_account_infos.iter() {
3034                if store_counts.get(&account_info.store_id()).unwrap().0 != 0 {
3035                    return false;
3036                }
3037            }
3038
3039            // Exit early if not filtering more for incremental snapshots
3040            if !should_filter_for_incremental_snapshots {
3041                return true;
3042            }
3043
3044            let slot_account_info_at_highest_slot = slot_account_infos
3045                .iter()
3046                .max_by_key(|(slot, _account_info)| slot);
3047
3048            slot_account_info_at_highest_slot.map_or(true, |(slot, account_info)| {
3049                // Do *not* purge zero-lamport accounts if the slot is greater than the last full
3050                // snapshot slot.  Since we're `retain`ing the accounts-to-purge, I felt creating
3051                // the `cannot_purge` variable made this easier to understand.  Accounts that do
3052                // not get purged here are added to a list so they be considered for purging later
3053                // (i.e. after the next full snapshot).
3054                assert!(account_info.is_zero_lamport());
3055                let cannot_purge = *slot > last_full_snapshot_slot.unwrap();
3056                if cannot_purge {
3057                    self.zero_lamport_accounts_to_purge_after_full_snapshot
3058                        .insert((*slot, *pubkey));
3059                }
3060                !cannot_purge
3061            })
3062        });
3063    }
3064
3065    // Must be kept private!, does sensitive cleanup that should only be called from
3066    // supported pipelines in AccountsDb
3067    // pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
3068    //    and should not be unref'd. If they exist in the accounts index, they are NEW.
3069    fn process_dead_slots(
3070        &self,
3071        dead_slots: &HashSet<Slot>,
3072        purged_account_slots: Option<&mut AccountSlots>,
3073        purge_stats: &PurgeStats,
3074        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
3075    ) {
3076        if dead_slots.is_empty() {
3077            return;
3078        }
3079        let mut clean_dead_slots = Measure::start("reclaims::clean_dead_slots");
3080        self.clean_stored_dead_slots(
3081            dead_slots,
3082            purged_account_slots,
3083            pubkeys_removed_from_accounts_index,
3084        );
3085        clean_dead_slots.stop();
3086
3087        let mut purge_removed_slots = Measure::start("reclaims::purge_removed_slots");
3088        self.purge_dead_slots_from_storage(dead_slots.iter(), purge_stats);
3089        purge_removed_slots.stop();
3090
3091        // If the slot is dead, remove the need to shrink the storages as
3092        // the storage entries will be purged.
3093        {
3094            let mut list = self.shrink_candidate_slots.lock().unwrap();
3095            for slot in dead_slots {
3096                list.remove(slot);
3097            }
3098        }
3099
3100        debug!(
3101            "process_dead_slots({}): {} {} {:?}",
3102            dead_slots.len(),
3103            clean_dead_slots,
3104            purge_removed_slots,
3105            dead_slots,
3106        );
3107    }
3108
3109    /// load the account index entry for the first `count` items in `accounts`
3110    /// store a reference to all alive accounts in `alive_accounts`
3111    /// unref and optionally store a reference to all pubkeys that are in the index, but dead in `unrefed_pubkeys`
3112    /// return sum of account size for all alive accounts
3113    fn load_accounts_index_for_shrink<'a>(
3114        &'a self,
3115        accounts: &'a [(Pubkey, FoundStoredAccount<'a>)],
3116        count: usize,
3117        alive_accounts: &mut Vec<&'a (Pubkey, FoundStoredAccount<'a>)>,
3118        mut unrefed_pubkeys: Option<&mut Vec<&'a Pubkey>>,
3119    ) -> usize {
3120        let mut alive_total = 0;
3121
3122        let mut alive = 0;
3123        let mut dead = 0;
3124        let mut index = 0;
3125        self.accounts_index.scan(
3126            accounts[..std::cmp::min(accounts.len(), count)]
3127                .iter()
3128                .map(|(key, _)| key),
3129            |pubkey, slots_refs| {
3130                let mut result = AccountsIndexScanResult::None;
3131                if let Some((slot_list, _ref_count)) = slots_refs {
3132                    let pair = &accounts[index];
3133                    let stored_account = &pair.1;
3134                    let is_alive = slot_list.iter().any(|(_slot, acct_info)| {
3135                        acct_info.matches_storage_location(
3136                            stored_account.store_id,
3137                            stored_account.account.offset,
3138                        )
3139                    });
3140                    if !is_alive {
3141                        // This pubkey was found in the storage, but no longer exists in the index.
3142                        // It would have had a ref to the storage from the initial store, but it will
3143                        // not exist in the re-written slot. Unref it to keep the index consistent with
3144                        // rewriting the storage entries.
3145                        if let Some(unrefed_pubkeys) = &mut unrefed_pubkeys {
3146                            unrefed_pubkeys.push(pubkey);
3147                        }
3148                        result = AccountsIndexScanResult::Unref;
3149                        dead += 1;
3150                    } else {
3151                        alive_accounts.push(pair);
3152                        alive_total += stored_account.account.stored_size;
3153                        alive += 1;
3154                    }
3155                }
3156                index += 1;
3157                result
3158            },
3159        );
3160        assert_eq!(index, std::cmp::min(accounts.len(), count));
3161        self.shrink_stats
3162            .alive_accounts
3163            .fetch_add(alive, Ordering::Relaxed);
3164        self.shrink_stats
3165            .dead_accounts
3166            .fetch_add(dead, Ordering::Relaxed);
3167
3168        alive_total
3169    }
3170
3171    /// get all accounts in all the storages passed in
3172    /// for duplicate pubkeys, the account with the highest write_value is returned
3173    pub(crate) fn get_unique_accounts_from_storages<'a, I>(
3174        &'a self,
3175        stores: I,
3176    ) -> GetUniqueAccountsResult<'a>
3177    where
3178        I: Iterator<Item = &'a Arc<AccountStorageEntry>>,
3179    {
3180        let mut stored_accounts: HashMap<Pubkey, FoundStoredAccount> = HashMap::new();
3181        let mut original_bytes = 0;
3182        let store_ids = stores
3183            .into_iter()
3184            .map(|store| {
3185                original_bytes += store.total_bytes();
3186                let store_id = store.append_vec_id();
3187                store.accounts.account_iter().for_each(|account| {
3188                    let new_entry = FoundStoredAccount { account, store_id };
3189                    match stored_accounts.entry(new_entry.account.meta.pubkey) {
3190                        Entry::Occupied(mut occupied_entry) => {
3191                            if new_entry.account.meta.write_version
3192                                > occupied_entry.get().account.meta.write_version
3193                            {
3194                                occupied_entry.insert(new_entry);
3195                            }
3196                        }
3197                        Entry::Vacant(vacant_entry) => {
3198                            vacant_entry.insert(new_entry);
3199                        }
3200                    }
3201                });
3202                store.append_vec_id()
3203            })
3204            .collect();
3205        GetUniqueAccountsResult {
3206            stored_accounts,
3207            original_bytes,
3208            store_ids,
3209        }
3210    }
3211
3212    fn do_shrink_slot_stores<'a, I>(&'a self, slot: Slot, stores: I) -> usize
3213    where
3214        I: Iterator<Item = &'a Arc<AccountStorageEntry>>,
3215    {
3216        debug!("do_shrink_slot_stores: slot: {}", slot);
3217        let GetUniqueAccountsResult {
3218            stored_accounts,
3219            original_bytes,
3220            store_ids,
3221        } = self.get_unique_accounts_from_storages(stores);
3222
3223        // sort by pubkey to keep account index lookups close
3224        let mut stored_accounts = stored_accounts.into_iter().collect::<Vec<_>>();
3225        stored_accounts.sort_unstable_by(|a, b| a.0.cmp(&b.0));
3226
3227        let mut index_read_elapsed = Measure::start("index_read_elapsed");
3228        let alive_total_collect = AtomicUsize::new(0);
3229
3230        let len = stored_accounts.len();
3231        let alive_accounts_collect = Mutex::new(Vec::with_capacity(len));
3232        let unrefed_pubkeys_collect = Mutex::new(Vec::with_capacity(len));
3233        self.shrink_stats
3234            .accounts_loaded
3235            .fetch_add(len as u64, Ordering::Relaxed);
3236
3237        self.thread_pool_clean.install(|| {
3238            let chunk_size = 50; // # accounts/thread
3239            let chunks = len / chunk_size + 1;
3240            (0..chunks).into_par_iter().for_each(|chunk| {
3241                let skip = chunk * chunk_size;
3242
3243                let mut alive_accounts = Vec::with_capacity(chunk_size);
3244                let mut unrefed_pubkeys = Vec::with_capacity(chunk_size);
3245                let alive_total = self.load_accounts_index_for_shrink(
3246                    &stored_accounts[skip..],
3247                    chunk_size,
3248                    &mut alive_accounts,
3249                    Some(&mut unrefed_pubkeys),
3250                );
3251
3252                // collect
3253                alive_accounts_collect
3254                    .lock()
3255                    .unwrap()
3256                    .append(&mut alive_accounts);
3257                unrefed_pubkeys_collect
3258                    .lock()
3259                    .unwrap()
3260                    .append(&mut unrefed_pubkeys);
3261                alive_total_collect.fetch_add(alive_total, Ordering::Relaxed);
3262            });
3263        });
3264
3265        let alive_accounts = alive_accounts_collect.into_inner().unwrap();
3266        let unrefed_pubkeys = unrefed_pubkeys_collect.into_inner().unwrap();
3267        let alive_total = alive_total_collect.load(Ordering::Relaxed);
3268
3269        index_read_elapsed.stop();
3270        let aligned_total: u64 = Self::page_align(alive_total as u64);
3271
3272        // This shouldn't happen if alive_bytes/approx_stored_count are accurate
3273        if Self::should_not_shrink(aligned_total, original_bytes, store_ids.len()) {
3274            self.shrink_stats
3275                .skipped_shrink
3276                .fetch_add(1, Ordering::Relaxed);
3277            for pubkey in unrefed_pubkeys {
3278                if let Some(locked_entry) = self.accounts_index.get_account_read_entry(pubkey) {
3279                    locked_entry.addref();
3280                }
3281            }
3282            return 0;
3283        }
3284
3285        let total_starting_accounts = stored_accounts.len();
3286        let total_accounts_after_shrink = alive_accounts.len();
3287        debug!(
3288            "shrinking: slot: {}, accounts: ({} => {}) bytes: ({} ; aligned to: {}) original: {}",
3289            slot,
3290            total_starting_accounts,
3291            total_accounts_after_shrink,
3292            alive_total,
3293            aligned_total,
3294            original_bytes,
3295        );
3296
3297        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
3298        let mut dead_storages = vec![];
3299        let mut find_alive_elapsed = 0;
3300        let mut create_and_insert_store_elapsed = 0;
3301        let mut write_storage_elapsed = 0;
3302        let mut store_accounts_timing = StoreAccountsTiming::default();
3303        if aligned_total > 0 {
3304            let mut start = Measure::start("find_alive_elapsed");
3305            let mut accounts = Vec::with_capacity(total_accounts_after_shrink);
3306            let mut hashes = Vec::with_capacity(total_accounts_after_shrink);
3307            let mut write_versions = Vec::with_capacity(total_accounts_after_shrink);
3308
3309            for (pubkey, alive_account) in alive_accounts {
3310                accounts.push((pubkey, &alive_account.account));
3311                hashes.push(alive_account.account.hash);
3312                write_versions.push(alive_account.account.meta.write_version);
3313            }
3314            start.stop();
3315            find_alive_elapsed = start.as_us();
3316
3317            let (shrunken_store, time) = self.get_store_for_shrink(slot, aligned_total);
3318            create_and_insert_store_elapsed = time.as_micros() as u64;
3319
3320            // here, we're writing back alive_accounts. That should be an atomic operation
3321            // without use of rather wide locks in this whole function, because we're
3322            // mutating rooted slots; There should be no writers to them.
3323            store_accounts_timing = self.store_accounts_frozen(
3324                (slot, &accounts[..]),
3325                Some(&hashes),
3326                Some(&shrunken_store),
3327                Some(Box::new(write_versions.into_iter())),
3328                StoreReclaims::Ignore,
3329            );
3330
3331            // `store_accounts_frozen()` above may have purged accounts from some
3332            // other storage entries (the ones that were just overwritten by this
3333            // new storage entry). This means some of those stores might have caused
3334            // this slot to be read to `self.shrink_candidate_slots`, so delete
3335            // those here
3336            self.shrink_candidate_slots.lock().unwrap().remove(&slot);
3337
3338            // Purge old, overwritten storage entries
3339            let mut start = Measure::start("write_storage_elapsed");
3340            let remaining_stores = self.mark_dirty_dead_stores(slot, &mut dead_storages, |store| {
3341                !store_ids.contains(&store.append_vec_id())
3342            });
3343            if remaining_stores > 1 {
3344                inc_new_counter_info!("accounts_db_shrink_extra_stores", 1);
3345                info!(
3346                    "after shrink, slot has extra stores: {}, {}",
3347                    slot, remaining_stores
3348                );
3349            }
3350            start.stop();
3351            write_storage_elapsed = start.as_us();
3352        }
3353        rewrite_elapsed.stop();
3354
3355        self.drop_or_recycle_stores(dead_storages);
3356
3357        self.shrink_stats
3358            .num_slots_shrunk
3359            .fetch_add(1, Ordering::Relaxed);
3360        self.shrink_stats
3361            .index_read_elapsed
3362            .fetch_add(index_read_elapsed.as_us(), Ordering::Relaxed);
3363        self.shrink_stats
3364            .find_alive_elapsed
3365            .fetch_add(find_alive_elapsed, Ordering::Relaxed);
3366        self.shrink_stats
3367            .create_and_insert_store_elapsed
3368            .fetch_add(create_and_insert_store_elapsed, Ordering::Relaxed);
3369        self.shrink_stats.store_accounts_elapsed.fetch_add(
3370            store_accounts_timing.store_accounts_elapsed,
3371            Ordering::Relaxed,
3372        );
3373        self.shrink_stats.update_index_elapsed.fetch_add(
3374            store_accounts_timing.update_index_elapsed,
3375            Ordering::Relaxed,
3376        );
3377        self.shrink_stats.handle_reclaims_elapsed.fetch_add(
3378            store_accounts_timing.handle_reclaims_elapsed,
3379            Ordering::Relaxed,
3380        );
3381        self.shrink_stats
3382            .write_storage_elapsed
3383            .fetch_add(write_storage_elapsed, Ordering::Relaxed);
3384        self.shrink_stats
3385            .rewrite_elapsed
3386            .fetch_add(rewrite_elapsed.as_us(), Ordering::Relaxed);
3387        self.shrink_stats.accounts_removed.fetch_add(
3388            total_starting_accounts - total_accounts_after_shrink,
3389            Ordering::Relaxed,
3390        );
3391        self.shrink_stats.bytes_removed.fetch_add(
3392            original_bytes.saturating_sub(aligned_total),
3393            Ordering::Relaxed,
3394        );
3395        self.shrink_stats
3396            .bytes_written
3397            .fetch_add(aligned_total, Ordering::Relaxed);
3398
3399        self.shrink_stats.report();
3400
3401        total_accounts_after_shrink
3402    }
3403
3404    /// get stores for 'slot'
3405    /// retain only the stores where 'should_retain(store)' == true
3406    /// for stores not retained, insert in 'dirty_stores' and 'dead_storages'
3407    /// returns # of remaining stores for this slot
3408    pub(crate) fn mark_dirty_dead_stores(
3409        &self,
3410        slot: Slot,
3411        dead_storages: &mut Vec<Arc<AccountStorageEntry>>,
3412        should_retain: impl Fn(&AccountStorageEntry) -> bool,
3413    ) -> usize {
3414        if let Some(slot_stores) = self.storage.get_slot_stores(slot) {
3415            let mut list = slot_stores.write().unwrap();
3416            list.retain(|_key, store| {
3417                if !should_retain(store) {
3418                    self.dirty_stores
3419                        .insert((slot, store.append_vec_id()), store.clone());
3420                    dead_storages.push(store.clone());
3421                    false
3422                } else {
3423                    true
3424                }
3425            });
3426            list.len()
3427        } else {
3428            0
3429        }
3430    }
3431
3432    pub(crate) fn drop_or_recycle_stores(&self, dead_storages: Vec<Arc<AccountStorageEntry>>) {
3433        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
3434        let mut recycle_stores = self.recycle_stores.write().unwrap();
3435        recycle_stores_write_elapsed.stop();
3436
3437        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
3438        if recycle_stores.entry_count() < MAX_RECYCLE_STORES {
3439            recycle_stores.add_entries(dead_storages);
3440            drop(recycle_stores);
3441        } else {
3442            self.stats
3443                .dropped_stores
3444                .fetch_add(dead_storages.len() as u64, Ordering::Relaxed);
3445            drop(recycle_stores);
3446            drop(dead_storages);
3447        }
3448        drop_storage_entries_elapsed.stop();
3449        self.shrink_stats
3450            .drop_storage_entries_elapsed
3451            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
3452        self.shrink_stats
3453            .recycle_stores_write_elapsed
3454            .fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
3455    }
3456
3457    /// return a store that can contain 'aligned_total' bytes and the time it took to execute
3458    pub(crate) fn get_store_for_shrink(
3459        &self,
3460        slot: Slot,
3461        aligned_total: u64,
3462    ) -> (Arc<AccountStorageEntry>, Duration) {
3463        let mut start = Measure::start("create_and_insert_store_elapsed");
3464        let shrunken_store = if let Some(new_store) =
3465            self.try_recycle_and_insert_store(slot, aligned_total, aligned_total + 1024)
3466        {
3467            new_store
3468        } else {
3469            let maybe_shrink_paths = self.shrink_paths.read().unwrap();
3470            if let Some(ref shrink_paths) = *maybe_shrink_paths {
3471                self.create_and_insert_store_with_paths(
3472                    slot,
3473                    aligned_total,
3474                    "shrink-w-path",
3475                    shrink_paths,
3476                )
3477            } else {
3478                self.create_and_insert_store(slot, aligned_total, "shrink")
3479            }
3480        };
3481        start.stop();
3482        (shrunken_store, Duration::from_micros(start.as_us()))
3483    }
3484
3485    // Reads all accounts in given slot's AppendVecs and filter only to alive,
3486    // then create a minimum AppendVec filled with the alive.
3487    fn shrink_slot_forced(&self, slot: Slot) -> usize {
3488        debug!("shrink_slot_forced: slot: {}", slot);
3489
3490        if let Some(stores_lock) = self.storage.get_slot_stores(slot) {
3491            let stores: Vec<Arc<AccountStorageEntry>> =
3492                stores_lock.read().unwrap().values().cloned().collect();
3493            if !Self::is_shrinking_productive(slot, stores.iter()) {
3494                return 0;
3495            }
3496            self.do_shrink_slot_stores(slot, stores.iter())
3497        } else {
3498            0
3499        }
3500    }
3501
3502    fn all_slots_in_storage(&self) -> Vec<Slot> {
3503        self.storage.all_slots()
3504    }
3505
3506    fn all_alive_roots_in_index(&self) -> Vec<Slot> {
3507        self.accounts_index.all_alive_roots()
3508    }
3509
3510    /// Given the input `ShrinkCandidates`, this function sorts the stores by their alive ratio
3511    /// in increasing order with the most sparse entries in the front. It will then simulate the
3512    /// shrinking by working on the most sparse entries first and if the overall alive ratio is
3513    /// achieved, it will stop and return the filtered-down candidates and the candidates which
3514    /// are skipped in this round and might be eligible for the future shrink.
3515    fn select_candidates_by_total_usage(
3516        shrink_slots: &ShrinkCandidates,
3517        shrink_ratio: f64,
3518    ) -> (ShrinkCandidates, ShrinkCandidates) {
3519        struct StoreUsageInfo {
3520            slot: Slot,
3521            alive_ratio: f64,
3522            store: Arc<AccountStorageEntry>,
3523        }
3524        let mut measure = Measure::start("select_top_sparse_storage_entries-ms");
3525        let mut store_usage: Vec<StoreUsageInfo> = Vec::with_capacity(shrink_slots.len());
3526        let mut total_alive_bytes: u64 = 0;
3527        let mut candidates_count: usize = 0;
3528        let mut total_bytes: u64 = 0;
3529        let mut total_candidate_stores: usize = 0;
3530        for (slot, slot_shrink_candidates) in shrink_slots {
3531            candidates_count += slot_shrink_candidates.len();
3532            for store in slot_shrink_candidates.values() {
3533                total_alive_bytes += Self::page_align(store.alive_bytes() as u64);
3534                total_bytes += store.total_bytes();
3535                let alive_ratio = Self::page_align(store.alive_bytes() as u64) as f64
3536                    / store.total_bytes() as f64;
3537                store_usage.push(StoreUsageInfo {
3538                    slot: *slot,
3539                    alive_ratio,
3540                    store: store.clone(),
3541                });
3542                total_candidate_stores += 1;
3543            }
3544        }
3545        store_usage.sort_by(|a, b| {
3546            a.alive_ratio
3547                .partial_cmp(&b.alive_ratio)
3548                .unwrap_or(std::cmp::Ordering::Equal)
3549        });
3550
3551        // Working from the beginning of store_usage which are the most sparse and see when we can stop
3552        // shrinking while still achieving the overall goals.
3553        let mut shrink_slots: ShrinkCandidates = HashMap::new();
3554        let mut shrink_slots_next_batch: ShrinkCandidates = HashMap::new();
3555        for usage in &store_usage {
3556            let store = &usage.store;
3557            let alive_ratio = (total_alive_bytes as f64) / (total_bytes as f64);
3558            debug!("alive_ratio: {:?} store_id: {:?}, store_ratio: {:?} requirment: {:?}, total_bytes: {:?} total_alive_bytes: {:?}",
3559                alive_ratio, usage.store.append_vec_id(), usage.alive_ratio, shrink_ratio, total_bytes, total_alive_bytes);
3560            if alive_ratio > shrink_ratio {
3561                // we have reached our goal, stop
3562                debug!(
3563                    "Shrinking goal can be achieved at slot {:?}, total_alive_bytes: {:?} \
3564                    total_bytes: {:?}, alive_ratio: {:}, shrink_ratio: {:?}",
3565                    usage.slot, total_alive_bytes, total_bytes, alive_ratio, shrink_ratio
3566                );
3567                if usage.alive_ratio < shrink_ratio {
3568                    shrink_slots_next_batch
3569                        .entry(usage.slot)
3570                        .or_default()
3571                        .insert(store.append_vec_id(), store.clone());
3572                } else {
3573                    break;
3574                }
3575            } else {
3576                let current_store_size = store.total_bytes();
3577                let after_shrink_size = Self::page_align(store.alive_bytes() as u64);
3578                let bytes_saved = current_store_size.saturating_sub(after_shrink_size);
3579                total_bytes -= bytes_saved;
3580                shrink_slots
3581                    .entry(usage.slot)
3582                    .or_default()
3583                    .insert(store.append_vec_id(), store.clone());
3584            }
3585        }
3586        measure.stop();
3587        inc_new_counter_debug!(
3588            "shrink_select_top_sparse_storage_entries-ms",
3589            measure.as_ms() as usize
3590        );
3591        inc_new_counter_debug!(
3592            "shrink_select_top_sparse_storage_entries-seeds",
3593            candidates_count
3594        );
3595        inc_new_counter_debug!(
3596            "shrink_total_preliminary_candidate_stores",
3597            total_candidate_stores
3598        );
3599
3600        (shrink_slots, shrink_slots_next_batch)
3601    }
3602
3603    fn get_roots_less_than(&self, slot: Slot) -> Vec<Slot> {
3604        self.accounts_index
3605            .roots_tracker
3606            .read()
3607            .unwrap()
3608            .alive_roots
3609            .get_all_less_than(slot)
3610    }
3611
3612    fn get_prior_root(&self, slot: Slot) -> Option<Slot> {
3613        self.accounts_index
3614            .roots_tracker
3615            .read()
3616            .unwrap()
3617            .alive_roots
3618            .get_prior(slot)
3619    }
3620
3621    /// get a sorted list of slots older than an epoch
3622    /// squash those slots into ancient append vecs
3623    fn shrink_ancient_slots(&self) {
3624        if !self.ancient_append_vecs {
3625            return;
3626        }
3627
3628        // If we squash accounts in a slot that is still within an epoch of a hash calculation's max slot, then
3629        //  we could calculate the wrong rent_epoch and slot for an individual account and thus the wrong overall accounts hash.
3630        // So, only squash accounts in slots that are more than 1 epoch older than the last hash calculation.
3631        // Subsequent hash calculations should be a higher slot.
3632        let mut old_slots =
3633            self.get_roots_less_than(self.get_accounts_hash_complete_one_epoch_old());
3634        old_slots.sort_unstable();
3635        self.combine_ancient_slots(old_slots);
3636    }
3637
3638    /// create new ancient append vec
3639    /// return it and the elapsed time for metrics
3640    fn create_ancient_append_vec(
3641        &self,
3642        slot: Slot,
3643    ) -> (Option<(Slot, Arc<AccountStorageEntry>)>, Duration) {
3644        let (new_ancient_storage, time) =
3645            self.get_store_for_shrink(slot, get_ancient_append_vec_capacity());
3646        info!(
3647            "ancient_append_vec: creating initial ancient append vec: {}, size: {}, id: {}",
3648            slot,
3649            get_ancient_append_vec_capacity(),
3650            new_ancient_storage.append_vec_id(),
3651        );
3652        (Some((slot, new_ancient_storage)), time)
3653    }
3654
3655    /// return true if created
3656    /// also return elapsed time for metrics
3657    fn maybe_create_ancient_append_vec(
3658        &self,
3659        current_ancient: &mut Option<(Slot, Arc<AccountStorageEntry>)>,
3660        slot: Slot,
3661    ) -> (bool, Duration) {
3662        if current_ancient.is_none() {
3663            // our oldest slot is not an append vec of max size, or we filled the previous one.
3664            // So, create a new ancient append vec at 'slot'
3665            let result = self.create_ancient_append_vec(slot);
3666            *current_ancient = result.0;
3667            (true, result.1)
3668        } else {
3669            (false, Duration::default())
3670        }
3671    }
3672
3673    fn get_storages_for_slot(&self, slot: Slot) -> Option<SnapshotStorage> {
3674        self.storage.map.get(&slot).map(|storages| {
3675            // per slot, get the storages. There should usually only be 1.
3676            storages
3677                .value()
3678                .read()
3679                .unwrap()
3680                .values()
3681                .cloned()
3682                .collect::<Vec<_>>()
3683        })
3684    }
3685
3686    /// helper function to cleanup call to 'store_accounts_frozen'
3687    fn store_ancient_accounts(
3688        &self,
3689        ancient_slot: Slot,
3690        ancient_store: &Arc<AccountStorageEntry>,
3691        accounts: &AccountsToStore,
3692        storage_selector: StorageSelector,
3693    ) -> StoreAccountsTiming {
3694        let (accounts, hashes) = accounts.get(storage_selector);
3695        self.store_accounts_frozen(
3696            (ancient_slot, accounts),
3697            Some(hashes),
3698            Some(ancient_store),
3699            None,
3700            StoreReclaims::Ignore,
3701        )
3702    }
3703
3704    /// get the storages from 'slot' to squash
3705    /// or None if this slot should be skipped
3706    fn get_storages_to_move_to_ancient_append_vec(
3707        &self,
3708        slot: Slot,
3709        current_ancient: &mut Option<(Slot, Arc<AccountStorageEntry>)>,
3710    ) -> Option<SnapshotStorage> {
3711        self.get_storages_for_slot(slot).and_then(|all_storages| {
3712            self.should_move_to_ancient_append_vec(&all_storages, current_ancient, slot)
3713                .then(|| all_storages)
3714        })
3715    }
3716
3717    /// return true if the accounts in this slot should be moved to an ancient append vec
3718    /// otherwise, return false and the caller can skip this slot
3719    /// side effect could be updating 'current_ancient'
3720    pub fn should_move_to_ancient_append_vec(
3721        &self,
3722        all_storages: &SnapshotStorage,
3723        current_ancient: &mut Option<(Slot, Arc<AccountStorageEntry>)>,
3724        slot: Slot,
3725    ) -> bool {
3726        if all_storages.len() != 1 {
3727            // we are dealing with roots that are more than 1 epoch old. I chose not to support or test the case where we have > 1 append vec per slot.
3728            // So, such slots will NOT participate in ancient shrinking.
3729            // since we skipped an ancient append vec, we don't want to append to whatever append vec USED to be the current one
3730            *current_ancient = None;
3731            return false;
3732        }
3733        let storage = all_storages.first().unwrap();
3734        let accounts = &storage.accounts;
3735        if is_full_ancient(accounts) {
3736            if self.is_candidate_for_shrink(storage, true) {
3737                // we are full, but we are a candidate for shrink, so either append us to the previous append vec
3738                // or recreate us as a new append vec and eliminate some contents
3739                info!("ancient_append_vec: shrinking full ancient: {}", slot);
3740                return true;
3741            }
3742            // since we skipped an ancient append vec, we don't want to append to whatever append vec USED to be the current one
3743            *current_ancient = None;
3744            return false; // skip this full ancient append vec completely
3745        }
3746
3747        if is_ancient(accounts) {
3748            if current_ancient.is_some() {
3749                info!("ancient_append_vec: shrinking full ancient: {}", slot);
3750            }
3751
3752            // this slot is ancient and can become the 'current' ancient for other slots to be squashed into
3753            *current_ancient = Some((slot, Arc::clone(storage)));
3754            return false; // we're done with this slot - this slot IS the ancient append vec
3755        }
3756
3757        // otherwise, yes, squash this slot into the current ancient append vec or create one at this slot
3758        true
3759    }
3760
3761    /// Combine all account data from storages in 'sorted_slots' into ancient append vecs.
3762    /// This keeps us from accumulating append vecs in slots older than an epoch.
3763    fn combine_ancient_slots(&self, sorted_slots: Vec<Slot>) {
3764        if sorted_slots.is_empty() {
3765            return;
3766        }
3767        let mut guard = None;
3768
3769        // the ancient append vec currently being written to
3770        let mut current_ancient = None;
3771        let mut dropped_roots = vec![];
3772
3773        if let Some(first_slot) = sorted_slots.first() {
3774            info!(
3775                "ancient_append_vec: combine_ancient_slots first slot: {}, num_roots: {}",
3776                first_slot,
3777                sorted_slots.len()
3778            );
3779        }
3780
3781        for slot in sorted_slots {
3782            let old_storages =
3783                match self.get_storages_to_move_to_ancient_append_vec(slot, &mut current_ancient) {
3784                    Some(old_storages) => old_storages,
3785                    None => {
3786                        // nothing to squash for this slot
3787                        continue;
3788                    }
3789                };
3790
3791            if guard.is_none() {
3792                // we are now doing interesting work in squashing ancient
3793                guard = Some(self.active_stats.activate(ActiveStatItem::SquashAncient))
3794            }
3795
3796            // this code is copied from shrink. I would like to combine it into a helper function, but the borrow checker has defeated my efforts so far.
3797            let GetUniqueAccountsResult {
3798                stored_accounts,
3799                original_bytes,
3800                store_ids: _,
3801            } = self.get_unique_accounts_from_storages(old_storages.iter());
3802
3803            // sort by pubkey to keep account index lookups close
3804            let mut stored_accounts = stored_accounts.into_iter().collect::<Vec<_>>();
3805            stored_accounts.sort_unstable_by(|a, b| a.0.cmp(&b.0));
3806
3807            let mut index_read_elapsed = Measure::start("index_read_elapsed");
3808            let alive_total_collect = AtomicUsize::new(0);
3809
3810            let len = stored_accounts.len();
3811            let alive_accounts_collect = Mutex::new(Vec::with_capacity(len));
3812            self.shrink_stats
3813                .accounts_loaded
3814                .fetch_add(len as u64, Ordering::Relaxed);
3815
3816            self.thread_pool_clean.install(|| {
3817                let chunk_size = 50; // # accounts/thread
3818                let chunks = len / chunk_size + 1;
3819                (0..chunks).into_par_iter().for_each(|chunk| {
3820                    let skip = chunk * chunk_size;
3821
3822                    let mut alive_accounts = Vec::with_capacity(chunk_size);
3823                    let alive_total = self.load_accounts_index_for_shrink(
3824                        &stored_accounts[skip..],
3825                        chunk_size,
3826                        &mut alive_accounts,
3827                        None,
3828                    );
3829
3830                    // collect
3831                    alive_accounts_collect
3832                        .lock()
3833                        .unwrap()
3834                        .append(&mut alive_accounts);
3835                    alive_total_collect.fetch_add(alive_total, Ordering::Relaxed);
3836                });
3837            });
3838
3839            let mut create_and_insert_store_elapsed = 0;
3840
3841            let alive_accounts = alive_accounts_collect.into_inner().unwrap();
3842            let alive_total = alive_total_collect.load(Ordering::Relaxed);
3843            index_read_elapsed.stop();
3844            let aligned_total: u64 = Self::page_align(alive_total as u64);
3845            // we could sort these
3846            // could follow what shrink does more closely
3847            if stored_accounts.is_empty() {
3848                continue; // skipping slot with no useful accounts to write
3849            }
3850
3851            let total_starting_accounts = stored_accounts.len();
3852            let total_accounts_after_shrink = alive_accounts.len();
3853
3854            let (_, time) = self.maybe_create_ancient_append_vec(&mut current_ancient, slot);
3855            create_and_insert_store_elapsed += time.as_micros() as u64;
3856            let (ancient_slot, ancient_store) =
3857                current_ancient.as_ref().map(|(a, b)| (*a, b)).unwrap();
3858            let available_bytes = ancient_store.accounts.remaining_bytes();
3859            let mut start = Measure::start("find_alive_elapsed");
3860            let to_store = AccountsToStore::new(available_bytes, &alive_accounts, slot);
3861            start.stop();
3862            let find_alive_elapsed = start.as_us();
3863
3864            let mut ids = vec![ancient_store.append_vec_id()];
3865            // if this slot is not the ancient slot we're writing to, then this root will be dropped
3866            let mut drop_root = slot != ancient_slot;
3867
3868            let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
3869            // write what we can to the current ancient storage
3870            let mut store_accounts_timing = self.store_ancient_accounts(
3871                ancient_slot,
3872                ancient_store,
3873                &to_store,
3874                StorageSelector::Primary,
3875            );
3876
3877            // handle accounts from 'slot' which did not fit into the current ancient append vec
3878            if to_store.has_overflow() {
3879                // we need a new ancient append vec
3880                let result = self.create_ancient_append_vec(slot);
3881                create_and_insert_store_elapsed += result.1.as_micros() as u64;
3882                current_ancient = result.0;
3883                let (ancient_slot, ancient_store) =
3884                    current_ancient.as_ref().map(|(a, b)| (*a, b)).unwrap();
3885                info!(
3886                    "ancient_append_vec: combine_ancient_slots {}, overflow: {} accounts",
3887                    slot,
3888                    to_store.get(StorageSelector::Overflow).0.len()
3889                );
3890
3891                ids.push(ancient_store.append_vec_id());
3892                // if this slot is not the ancient slot we're writing to, then this root will be dropped
3893                drop_root = slot != ancient_slot;
3894
3895                // write the rest to the next ancient storage
3896                let timing = self.store_ancient_accounts(
3897                    ancient_slot,
3898                    ancient_store,
3899                    &to_store,
3900                    StorageSelector::Overflow,
3901                );
3902                store_accounts_timing.store_accounts_elapsed = timing.store_accounts_elapsed;
3903                store_accounts_timing.update_index_elapsed = timing.update_index_elapsed;
3904                store_accounts_timing.handle_reclaims_elapsed = timing.handle_reclaims_elapsed;
3905            }
3906            rewrite_elapsed.stop();
3907
3908            let mut start = Measure::start("write_storage_elapsed");
3909            // Purge old, overwritten storage entries
3910            let mut dead_storages = vec![];
3911            self.mark_dirty_dead_stores(slot, &mut dead_storages, |store| {
3912                ids.contains(&store.append_vec_id())
3913            });
3914            start.stop();
3915            let write_storage_elapsed = start.as_us();
3916
3917            self.drop_or_recycle_stores(dead_storages);
3918
3919            if drop_root {
3920                dropped_roots.push(slot);
3921            }
3922
3923            self.shrink_ancient_stats
3924                .shrink_stats
3925                .index_read_elapsed
3926                .fetch_add(index_read_elapsed.as_us(), Ordering::Relaxed);
3927            self.shrink_ancient_stats
3928                .shrink_stats
3929                .create_and_insert_store_elapsed
3930                .fetch_add(create_and_insert_store_elapsed, Ordering::Relaxed);
3931            self.shrink_ancient_stats
3932                .shrink_stats
3933                .store_accounts_elapsed
3934                .fetch_add(
3935                    store_accounts_timing.store_accounts_elapsed,
3936                    Ordering::Relaxed,
3937                );
3938            self.shrink_ancient_stats
3939                .shrink_stats
3940                .update_index_elapsed
3941                .fetch_add(
3942                    store_accounts_timing.update_index_elapsed,
3943                    Ordering::Relaxed,
3944                );
3945            self.shrink_ancient_stats
3946                .shrink_stats
3947                .handle_reclaims_elapsed
3948                .fetch_add(
3949                    store_accounts_timing.handle_reclaims_elapsed,
3950                    Ordering::Relaxed,
3951                );
3952            self.shrink_ancient_stats
3953                .shrink_stats
3954                .write_storage_elapsed
3955                .fetch_add(write_storage_elapsed, Ordering::Relaxed);
3956            self.shrink_ancient_stats
3957                .shrink_stats
3958                .rewrite_elapsed
3959                .fetch_add(rewrite_elapsed.as_us(), Ordering::Relaxed);
3960            self.shrink_ancient_stats
3961                .shrink_stats
3962                .accounts_removed
3963                .fetch_add(
3964                    total_starting_accounts - total_accounts_after_shrink,
3965                    Ordering::Relaxed,
3966                );
3967            self.shrink_ancient_stats
3968                .shrink_stats
3969                .bytes_removed
3970                .fetch_add(
3971                    original_bytes.saturating_sub(aligned_total),
3972                    Ordering::Relaxed,
3973                );
3974            self.shrink_ancient_stats
3975                .shrink_stats
3976                .bytes_written
3977                .fetch_add(aligned_total, Ordering::Relaxed);
3978            self.shrink_ancient_stats
3979                .shrink_stats
3980                .find_alive_elapsed
3981                .fetch_add(find_alive_elapsed, Ordering::Relaxed);
3982            self.shrink_ancient_stats
3983                .shrink_stats
3984                .num_slots_shrunk
3985                .fetch_add(1, Ordering::Relaxed);
3986        }
3987
3988        if !dropped_roots.is_empty() {
3989            dropped_roots.iter().for_each(|slot| {
3990                self.accounts_index
3991                    .clean_dead_slot(*slot, &mut AccountsIndexRootsStats::default());
3992            });
3993        }
3994
3995        self.shrink_ancient_stats.report();
3996    }
3997
3998    pub fn shrink_candidate_slots(&self) -> usize {
3999        let shrink_candidates_slots =
4000            std::mem::take(&mut *self.shrink_candidate_slots.lock().unwrap());
4001        if !shrink_candidates_slots.is_empty() {
4002            self.shrink_ancient_slots();
4003        }
4004
4005        let (shrink_slots, shrink_slots_next_batch) = {
4006            if let AccountShrinkThreshold::TotalSpace { shrink_ratio } = self.shrink_ratio {
4007                let (shrink_slots, shrink_slots_next_batch) =
4008                    Self::select_candidates_by_total_usage(&shrink_candidates_slots, shrink_ratio);
4009                (shrink_slots, Some(shrink_slots_next_batch))
4010            } else {
4011                (shrink_candidates_slots, None)
4012            }
4013        };
4014
4015        if shrink_slots.is_empty()
4016            && shrink_slots_next_batch
4017                .as_ref()
4018                .map(|s| s.is_empty())
4019                .unwrap_or(true)
4020        {
4021            return 0;
4022        }
4023
4024        let _guard = self.active_stats.activate(ActiveStatItem::Shrink);
4025
4026        let mut measure_shrink_all_candidates = Measure::start("shrink_all_candidate_slots-ms");
4027        let num_candidates = shrink_slots.len();
4028        let shrink_candidates_count: usize = self.thread_pool_clean.install(|| {
4029            shrink_slots
4030                .into_par_iter()
4031                .map(|(slot, slot_shrink_candidates)| {
4032                    let mut measure = Measure::start("shrink_candidate_slots-ms");
4033                    self.do_shrink_slot_stores(slot, slot_shrink_candidates.values());
4034                    measure.stop();
4035                    inc_new_counter_info!("shrink_candidate_slots-ms", measure.as_ms() as usize);
4036                    slot_shrink_candidates.len()
4037                })
4038                .sum()
4039        });
4040        measure_shrink_all_candidates.stop();
4041        inc_new_counter_info!(
4042            "shrink_all_candidate_slots-ms",
4043            measure_shrink_all_candidates.as_ms() as usize
4044        );
4045        inc_new_counter_info!("shrink_all_candidate_slots-count", shrink_candidates_count);
4046        let mut pended_counts: usize = 0;
4047        if let Some(shrink_slots_next_batch) = shrink_slots_next_batch {
4048            let mut shrink_slots = self.shrink_candidate_slots.lock().unwrap();
4049            for (slot, stores) in shrink_slots_next_batch {
4050                pended_counts += stores.len();
4051                shrink_slots.entry(slot).or_default().extend(stores);
4052            }
4053        }
4054        inc_new_counter_info!("shrink_pended_stores-count", pended_counts);
4055
4056        num_candidates
4057    }
4058
4059    pub fn shrink_all_slots(&self, is_startup: bool, last_full_snapshot_slot: Option<Slot>) {
4060        let _guard = self.active_stats.activate(ActiveStatItem::Shrink);
4061        const DIRTY_STORES_CLEANING_THRESHOLD: usize = 10_000;
4062        const OUTER_CHUNK_SIZE: usize = 2000;
4063        if is_startup && self.caching_enabled {
4064            let slots = self.all_slots_in_storage();
4065            let threads = num_cpus::get();
4066            let inner_chunk_size = std::cmp::max(OUTER_CHUNK_SIZE / threads, 1);
4067            slots.chunks(OUTER_CHUNK_SIZE).for_each(|chunk| {
4068                chunk.par_chunks(inner_chunk_size).for_each(|slots| {
4069                    for slot in slots {
4070                        self.shrink_slot_forced(*slot);
4071                    }
4072                });
4073                if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
4074                    self.clean_accounts(None, is_startup, last_full_snapshot_slot);
4075                }
4076            });
4077        } else {
4078            for slot in self.all_slots_in_storage() {
4079                if self.caching_enabled {
4080                    self.shrink_slot_forced(slot);
4081                } else {
4082                    self.do_shrink_slot_forced_v1(slot);
4083                }
4084                if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
4085                    self.clean_accounts(None, is_startup, last_full_snapshot_slot);
4086                }
4087            }
4088        }
4089    }
4090
4091    pub fn scan_accounts<F, A>(
4092        &self,
4093        ancestors: &Ancestors,
4094        bank_id: BankId,
4095        scan_func: F,
4096        config: &ScanConfig,
4097    ) -> ScanResult<A>
4098    where
4099        F: Fn(&mut A, Option<(&Pubkey, AccountSharedData, Slot)>),
4100        A: Default,
4101    {
4102        let mut collector = A::default();
4103
4104        // This can error out if the slots being scanned over are aborted
4105        self.accounts_index.scan_accounts(
4106            ancestors,
4107            bank_id,
4108            |pubkey, (account_info, slot)| {
4109                let account_slot = self
4110                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4111                    .get_loaded_account()
4112                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot));
4113                scan_func(&mut collector, account_slot)
4114            },
4115            config,
4116        )?;
4117
4118        Ok(collector)
4119    }
4120
4121    pub fn unchecked_scan_accounts<F, A>(
4122        &self,
4123        metric_name: &'static str,
4124        ancestors: &Ancestors,
4125        scan_func: F,
4126        config: &ScanConfig,
4127    ) -> A
4128    where
4129        F: Fn(&mut A, (&Pubkey, LoadedAccount, Slot)),
4130        A: Default,
4131    {
4132        let mut collector = A::default();
4133        self.accounts_index.unchecked_scan_accounts(
4134            metric_name,
4135            ancestors,
4136            |pubkey, (account_info, slot)| {
4137                if let Some(loaded_account) = self
4138                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4139                    .get_loaded_account()
4140                {
4141                    scan_func(&mut collector, (pubkey, loaded_account, slot));
4142                }
4143            },
4144            config,
4145        );
4146        collector
4147    }
4148
4149    /// Only guaranteed to be safe when called from rent collection
4150    pub fn range_scan_accounts<F, A, R>(
4151        &self,
4152        metric_name: &'static str,
4153        ancestors: &Ancestors,
4154        range: R,
4155        config: &ScanConfig,
4156        scan_func: F,
4157    ) -> A
4158    where
4159        F: Fn(&mut A, Option<(&Pubkey, AccountSharedData, Slot)>),
4160        A: Default,
4161        R: RangeBounds<Pubkey> + std::fmt::Debug,
4162    {
4163        let mut collector = A::default();
4164        self.accounts_index.range_scan_accounts(
4165            metric_name,
4166            ancestors,
4167            range,
4168            config,
4169            |pubkey, (account_info, slot)| {
4170                // unlike other scan fns, this is called from Bank::collect_rent_eagerly(),
4171                // which is on-consensus processing in the banking/replaying stage.
4172                // This requires infallible and consistent account loading.
4173                // So, we unwrap Option<LoadedAccount> from get_loaded_account() here.
4174                // This is safe because this closure is invoked with the account_info,
4175                // while we lock the index entry at AccountsIndex::do_scan_accounts() ultimately,
4176                // meaning no other subsystems can invalidate the account_info before making their
4177                // changes to the index entry.
4178                // For details, see the comment in retry_to_get_account_accessor()
4179                if let Some(account_slot) = self
4180                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4181                    .get_loaded_account()
4182                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot))
4183                {
4184                    scan_func(&mut collector, Some(account_slot))
4185                }
4186            },
4187        );
4188        collector
4189    }
4190
4191    pub fn index_scan_accounts<F, A>(
4192        &self,
4193        ancestors: &Ancestors,
4194        bank_id: BankId,
4195        index_key: IndexKey,
4196        scan_func: F,
4197        config: &ScanConfig,
4198    ) -> ScanResult<(A, bool)>
4199    where
4200        F: Fn(&mut A, Option<(&Pubkey, AccountSharedData, Slot)>),
4201        A: Default,
4202    {
4203        let key = match &index_key {
4204            IndexKey::ProgramId(key) => key,
4205            IndexKey::SafeTokenMint(key) => key,
4206            IndexKey::SafeTokenOwner(key) => key,
4207        };
4208        if !self.account_indexes.include_key(key) {
4209            // the requested key was not indexed in the secondary index, so do a normal scan
4210            let used_index = false;
4211            let scan_result = self.scan_accounts(ancestors, bank_id, scan_func, config)?;
4212            return Ok((scan_result, used_index));
4213        }
4214
4215        let mut collector = A::default();
4216        self.accounts_index.index_scan_accounts(
4217            ancestors,
4218            bank_id,
4219            index_key,
4220            |pubkey, (account_info, slot)| {
4221                let account_slot = self
4222                    .get_account_accessor(slot, pubkey, &account_info.storage_location())
4223                    .get_loaded_account()
4224                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot));
4225                scan_func(&mut collector, account_slot)
4226            },
4227            config,
4228        )?;
4229        let used_index = true;
4230        Ok((collector, used_index))
4231    }
4232
4233    /// Scan a specific slot through all the account storage in parallel
4234    pub fn scan_account_storage<R, B>(
4235        &self,
4236        slot: Slot,
4237        cache_map_func: impl Fn(LoadedAccount) -> Option<R> + Sync,
4238        storage_scan_func: impl Fn(&B, LoadedAccount) + Sync,
4239    ) -> ScanStorageResult<R, B>
4240    where
4241        R: Send,
4242        B: Send + Default + Sync,
4243    {
4244        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
4245            // If we see the slot in the cache, then all the account information
4246            // is in this cached slot
4247            if slot_cache.len() > SCAN_SLOT_PAR_ITER_THRESHOLD {
4248                ScanStorageResult::Cached(self.thread_pool.install(|| {
4249                    slot_cache
4250                        .par_iter()
4251                        .filter_map(|cached_account| {
4252                            cache_map_func(LoadedAccount::Cached(Cow::Borrowed(
4253                                cached_account.value(),
4254                            )))
4255                        })
4256                        .collect()
4257                }))
4258            } else {
4259                ScanStorageResult::Cached(
4260                    slot_cache
4261                        .iter()
4262                        .filter_map(|cached_account| {
4263                            cache_map_func(LoadedAccount::Cached(Cow::Borrowed(
4264                                cached_account.value(),
4265                            )))
4266                        })
4267                        .collect(),
4268                )
4269            }
4270        } else {
4271            let retval = B::default();
4272            // If the slot is not in the cache, then all the account information must have
4273            // been flushed. This is guaranteed because we only remove the rooted slot from
4274            // the cache *after* we've finished flushing in `flush_slot_cache`.
4275            let storage_maps: Vec<Arc<AccountStorageEntry>> = self
4276                .storage
4277                .get_slot_storage_entries(slot)
4278                .unwrap_or_default();
4279            self.thread_pool.install(|| {
4280                storage_maps.par_iter().for_each(|storage| {
4281                    storage.accounts.account_iter().for_each(|account| {
4282                        storage_scan_func(&retval, LoadedAccount::Stored(account))
4283                    })
4284                });
4285            });
4286
4287            ScanStorageResult::Stored(retval)
4288        }
4289    }
4290
4291    pub fn set_hash(&self, slot: Slot, parent_slot: Slot) {
4292        let mut bank_hashes = self.bank_hashes.write().unwrap();
4293        if bank_hashes.get(&slot).is_some() {
4294            error!(
4295                "set_hash: already exists; multiple forks with shared slot {} as child (parent: {})!?",
4296                slot, parent_slot,
4297            );
4298            return;
4299        }
4300
4301        let new_hash_info = BankHashInfo {
4302            hash: Hash::default(),
4303            snapshot_hash: Hash::default(),
4304            stats: BankHashStats::default(),
4305        };
4306        bank_hashes.insert(slot, new_hash_info);
4307    }
4308
4309    pub fn load(
4310        &self,
4311        ancestors: &Ancestors,
4312        pubkey: &Pubkey,
4313        load_hint: LoadHint,
4314    ) -> Option<(AccountSharedData, Slot)> {
4315        self.do_load(ancestors, pubkey, None, load_hint)
4316    }
4317
4318    pub fn load_account_into_read_cache(&self, ancestors: &Ancestors, pubkey: &Pubkey) {
4319        self.do_load_with_populate_read_cache(ancestors, pubkey, None, LoadHint::Unspecified, true);
4320    }
4321
4322    pub fn load_with_fixed_root(
4323        &self,
4324        ancestors: &Ancestors,
4325        pubkey: &Pubkey,
4326        load_zero_lamports: LoadZeroLamports,
4327    ) -> Option<(AccountSharedData, Slot)> {
4328        self.load(ancestors, pubkey, LoadHint::FixedMaxRoot)
4329            .filter(|(account, _)| {
4330                matches!(
4331                    load_zero_lamports,
4332                    LoadZeroLamports::SomeWithZeroLamportAccount
4333                ) || !account.is_zero_lamport()
4334            })
4335    }
4336
4337    pub fn load_without_fixed_root(
4338        &self,
4339        ancestors: &Ancestors,
4340        pubkey: &Pubkey,
4341    ) -> Option<(AccountSharedData, Slot)> {
4342        self.load(ancestors, pubkey, LoadHint::Unspecified)
4343    }
4344
4345    fn read_index_for_accessor_or_load_slow<'a>(
4346        &'a self,
4347        ancestors: &Ancestors,
4348        pubkey: &'a Pubkey,
4349        max_root: Option<Slot>,
4350        clone_in_lock: bool,
4351    ) -> Option<(Slot, StorageLocation, Option<LoadedAccountAccessor<'a>>)> {
4352        let (lock, index) = match self.accounts_index.get(pubkey, Some(ancestors), max_root) {
4353            AccountIndexGetResult::Found(lock, index) => (lock, index),
4354            // we bail out pretty early for missing.
4355            AccountIndexGetResult::NotFound => {
4356                return None;
4357            }
4358        };
4359
4360        let slot_list = lock.slot_list();
4361        let (slot, info) = slot_list[index];
4362        let storage_location = info.storage_location();
4363        let some_from_slow_path = if clone_in_lock {
4364            // the fast path must have failed.... so take the slower approach
4365            // of copying potentially large Account::data inside the lock.
4366
4367            // calling check_and_get_loaded_account is safe as long as we're guaranteed to hold
4368            // the lock during the time and there should be no purge thanks to alive ancestors
4369            // held by our caller.
4370            Some(self.get_account_accessor(slot, pubkey, &storage_location))
4371        } else {
4372            None
4373        };
4374
4375        Some((slot, storage_location, some_from_slow_path))
4376        // `lock` is dropped here rather pretty quickly with clone_in_lock = false,
4377        // so the entry could be raced for mutation by other subsystems,
4378        // before we actually provision an account data for caller's use from now on.
4379        // This is traded for less contention and resultant performance, introducing fair amount of
4380        // delicate handling in retry_to_get_account_accessor() below ;)
4381        // you're warned!
4382    }
4383
4384    fn retry_to_get_account_accessor<'a>(
4385        &'a self,
4386        mut slot: Slot,
4387        mut storage_location: StorageLocation,
4388        ancestors: &'a Ancestors,
4389        pubkey: &'a Pubkey,
4390        max_root: Option<Slot>,
4391        load_hint: LoadHint,
4392    ) -> Option<(LoadedAccountAccessor<'a>, Slot)> {
4393        // Happy drawing time! :)
4394        //
4395        // Reader                               | Accessed data source for cached/stored
4396        // -------------------------------------+----------------------------------
4397        // R1 read_index_for_accessor_or_load_slow()| cached/stored: index
4398        //          |                           |
4399        //        <(store_id, offset, ..)>      |
4400        //          V                           |
4401        // R2 retry_to_get_account_accessor()/  | cached: map of caches & entry for (slot, pubkey)
4402        //        get_account_accessor()        | stored: map of stores
4403        //          |                           |
4404        //        <Accessor>                    |
4405        //          V                           |
4406        // R3 check_and_get_loaded_account()/   | cached: N/A (note: basically noop unwrap)
4407        //        get_loaded_account()          | stored: store's entry for slot
4408        //          |                           |
4409        //        <LoadedAccount>               |
4410        //          V                           |
4411        // R4 take_account()                    | cached/stored: entry of cache/storage for (slot, pubkey)
4412        //          |                           |
4413        //        <AccountSharedData>           |
4414        //          V                           |
4415        //    Account!!                         V
4416        //
4417        // Flusher                              | Accessed data source for cached/stored
4418        // -------------------------------------+----------------------------------
4419        // F1 flush_slot_cache()                | N/A
4420        //          |                           |
4421        //          V                           |
4422        // F2 store_accounts_frozen()/          | map of stores (creates new entry)
4423        //        write_accounts_to_storage()   |
4424        //          |                           |
4425        //          V                           |
4426        // F3 store_accounts_frozen()/          | index
4427        //        update_index()                | (replaces existing store_id, offset in caches)
4428        //          |                           |
4429        //          V                           |
4430        // F4 accounts_cache.remove_slot()      | map of caches (removes old entry)
4431        //                                      V
4432        //
4433        // Remarks for flusher: So, for any reading operations, it's a race condition where F4 happens
4434        // between R1 and R2. In that case, retrying from R1 is safu because F3 should have
4435        // been occurred.
4436        //
4437        // Shrinker                             | Accessed data source for stored
4438        // -------------------------------------+----------------------------------
4439        // S1 do_shrink_slot_stores()           | N/A
4440        //          |                           |
4441        //          V                           |
4442        // S2 store_accounts_frozen()/          | map of stores (creates new entry)
4443        //        write_accounts_to_storage()   |
4444        //          |                           |
4445        //          V                           |
4446        // S3 store_accounts_frozen()/          | index
4447        //        update_index()                | (replaces existing store_id, offset in stores)
4448        //          |                           |
4449        //          V                           |
4450        // S4 do_shrink_slot_stores()/          | map of stores (removes old entry)
4451        //        dead_storages
4452        //
4453        // Remarks for shrinker: So, for any reading operations, it's a race condition
4454        // where S4 happens between R1 and R2. In that case, retrying from R1 is safu because S3 should have
4455        // been occurred, and S3 atomically replaced the index accordingly.
4456        //
4457        // Cleaner                              | Accessed data source for stored
4458        // -------------------------------------+----------------------------------
4459        // C1 clean_accounts()                  | N/A
4460        //          |                           |
4461        //          V                           |
4462        // C2 clean_accounts()/                 | index
4463        //        purge_keys_exact()            | (removes existing store_id, offset for stores)
4464        //          |                           |
4465        //          V                           |
4466        // C3 clean_accounts()/                 | map of stores (removes old entry)
4467        //        handle_reclaims()             |
4468        //
4469        // Remarks for cleaner: So, for any reading operations, it's a race condition
4470        // where C3 happens between R1 and R2. In that case, retrying from R1 is safu.
4471        // In that case, None would be returned while bailing out at R1.
4472        //
4473        // Purger                                 | Accessed data source for cached/stored
4474        // ---------------------------------------+----------------------------------
4475        // P1 purge_slot()                        | N/A
4476        //          |                             |
4477        //          V                             |
4478        // P2 purge_slots_from_cache_and_store()  | map of caches/stores (removes old entry)
4479        //          |                             |
4480        //          V                             |
4481        // P3 purge_slots_from_cache_and_store()/ | index
4482        //       purge_slot_cache()/              |
4483        //          purge_slot_cache_pubkeys()    | (removes existing store_id, offset for cache)
4484        //       purge_slot_storage()/            |
4485        //          purge_keys_exact()            | (removes accounts index entries)
4486        //          handle_reclaims()             | (removes storage entries)
4487        //      OR                                |
4488        //    clean_accounts()/                   |
4489        //        clean_accounts_older_than_root()| (removes existing store_id, offset for stores)
4490        //                                        V
4491        //
4492        // Remarks for purger: So, for any reading operations, it's a race condition
4493        // where P2 happens between R1 and R2. In that case, retrying from R1 is safu.
4494        // In that case, we may bail at index read retry when P3 hasn't been run
4495
4496        #[cfg(test)]
4497        {
4498            // Give some time for cache flushing to occur here for unit tests
4499            sleep(Duration::from_millis(self.load_delay));
4500        }
4501
4502        // Failsafe for potential race conditions with other subsystems
4503        let mut num_acceptable_failed_iterations = 0;
4504        loop {
4505            let account_accessor = self.get_account_accessor(slot, pubkey, &storage_location);
4506            match account_accessor {
4507                LoadedAccountAccessor::Cached(Some(_)) | LoadedAccountAccessor::Stored(Some(_)) => {
4508                    // Great! There was no race, just return :) This is the most usual situation
4509                    return Some((account_accessor, slot));
4510                }
4511                LoadedAccountAccessor::Cached(None) => {
4512                    num_acceptable_failed_iterations += 1;
4513                    // Cache was flushed in between checking the index and retrieving from the cache,
4514                    // so retry. This works because in accounts cache flush, an account is written to
4515                    // storage *before* it is removed from the cache
4516                    match load_hint {
4517                        LoadHint::FixedMaxRoot => {
4518                            // it's impossible for this to fail for transaction loads from
4519                            // replaying/banking more than once.
4520                            // This is because:
4521                            // 1) For a slot `X` that's being replayed, there is only one
4522                            // latest ancestor containing the latest update for the account, and this
4523                            // ancestor can only be flushed once.
4524                            // 2) The root cannot move while replaying, so the index cannot continually
4525                            // find more up to date entries than the current `slot`
4526                            assert!(num_acceptable_failed_iterations <= 1);
4527                        }
4528                        LoadHint::Unspecified => {
4529                            // Because newer root can be added to the index (= not fixed),
4530                            // multiple flush race conditions can be observed under very rare
4531                            // condition, at least theoretically
4532                        }
4533                    }
4534                }
4535                LoadedAccountAccessor::Stored(None) => {
4536                    match load_hint {
4537                        LoadHint::FixedMaxRoot => {
4538                            // When running replay on the validator, or banking stage on the leader,
4539                            // it should be very rare that the storage entry doesn't exist if the
4540                            // entry in the accounts index is the latest version of this account.
4541                            //
4542                            // There are only a few places where the storage entry may not exist
4543                            // after reading the index:
4544                            // 1) Shrink has removed the old storage entry and rewritten to
4545                            // a newer storage entry
4546                            // 2) The `pubkey` asked for in this function is a zero-lamport account,
4547                            // and the storage entry holding this account qualified for zero-lamport clean.
4548                            //
4549                            // In both these cases, it should be safe to retry and recheck the accounts
4550                            // index indefinitely, without incrementing num_acceptable_failed_iterations.
4551                            // That's because if the root is fixed, there should be a bounded number
4552                            // of pending cleans/shrinks (depends how far behind the AccountsBackgroundService
4553                            // is), termination to the desired condition is guaranteed.
4554                            //
4555                            // Also note that in both cases, if we do find the storage entry,
4556                            // we can guarantee that the storage entry is safe to read from because
4557                            // we grabbed a reference to the storage entry while it was still in the
4558                            // storage map. This means even if the storage entry is removed from the storage
4559                            // map after we grabbed the storage entry, the recycler should not reset the
4560                            // storage entry until we drop the reference to the storage entry.
4561                            //
4562                            // eh, no code in this arm? yes!
4563                        }
4564                        LoadHint::Unspecified => {
4565                            // RPC get_account() may have fetched an old root from the index that was
4566                            // either:
4567                            // 1) Cleaned up by clean_accounts(), so the accounts index has been updated
4568                            // and the storage entries have been removed.
4569                            // 2) Dropped by purge_slots() because the slot was on a minor fork, which
4570                            // removes the slots' storage entries but doesn't purge from the accounts index
4571                            // (account index cleanup is left to clean for stored slots). Note that
4572                            // this generally is impossible to occur in the wild because the RPC
4573                            // should hold the slot's bank, preventing it from being purged() to
4574                            // begin with.
4575                            num_acceptable_failed_iterations += 1;
4576                        }
4577                    }
4578                }
4579            }
4580            #[cfg(not(test))]
4581            let load_limit = ABSURD_CONSECUTIVE_FAILED_ITERATIONS;
4582
4583            #[cfg(test)]
4584            let load_limit = self.load_limit.load(Ordering::Relaxed);
4585
4586            let fallback_to_slow_path = if num_acceptable_failed_iterations >= load_limit {
4587                // The latest version of the account existed in the index, but could not be
4588                // fetched from storage. This means a race occurred between this function and clean
4589                // accounts/purge_slots
4590                let message = format!(
4591                    "do_load() failed to get key: {} from storage, latest attempt was for \
4592                     slot: {}, storage_location: {:?}, load_hint: {:?}",
4593                    pubkey, slot, storage_location, load_hint,
4594                );
4595                datapoint_warn!("accounts_db-do_load_warn", ("warn", message, String));
4596                true
4597            } else {
4598                false
4599            };
4600
4601            // Because reading from the cache/storage failed, retry from the index read
4602            let (new_slot, new_storage_location, maybe_account_accessor) = self
4603                .read_index_for_accessor_or_load_slow(
4604                    ancestors,
4605                    pubkey,
4606                    max_root,
4607                    fallback_to_slow_path,
4608                )?;
4609            // Notice the subtle `?` at previous line, we bail out pretty early if missing.
4610
4611            if new_slot == slot && new_storage_location.is_store_id_equal(&storage_location) {
4612                inc_new_counter_info!("retry_to_get_account_accessor-panic", 1);
4613                let message = format!(
4614                    "Bad index entry detected ({}, {}, {:?}, {:?}, {:?}, {:?})",
4615                    pubkey,
4616                    slot,
4617                    storage_location,
4618                    load_hint,
4619                    new_storage_location,
4620                    self.accounts_index.get_account_read_entry(pubkey)
4621                );
4622                // Considering that we've failed to get accessor above and further that
4623                // the index still returned the same (slot, store_id) tuple, offset must be same
4624                // too.
4625                assert!(
4626                    new_storage_location.is_offset_equal(&storage_location),
4627                    "{message}"
4628                );
4629
4630                // If the entry was missing from the cache, that means it must have been flushed,
4631                // and the accounts index is always updated before cache flush, so store_id must
4632                // not indicate being cached at this point.
4633                assert!(!new_storage_location.is_cached(), "{message}");
4634
4635                // If this is not a cache entry, then this was a minor fork slot
4636                // that had its storage entries cleaned up by purge_slots() but hasn't been
4637                // cleaned yet. That means this must be rpc access and not replay/banking at the
4638                // very least. Note that purge shouldn't occur even for RPC as caller must hold all
4639                // of ancestor slots..
4640                assert_eq!(load_hint, LoadHint::Unspecified, "{message}");
4641
4642                // Everything being assert!()-ed, let's panic!() here as it's an error condition
4643                // after all....
4644                // That reasoning is based on the fact all of code-path reaching this fn
4645                // retry_to_get_account_accessor() must outlive the Arc<Bank> (and its all
4646                // ancestors) over this fn invocation, guaranteeing the prevention of being purged,
4647                // first of all.
4648                // For details, see the comment in AccountIndex::do_checked_scan_accounts(),
4649                // which is referring back here.
4650                panic!("{message}");
4651            } else if fallback_to_slow_path {
4652                // the above bad-index-entry check must had been checked first to retain the same
4653                // behavior
4654                return Some((
4655                    maybe_account_accessor.expect("must be some if clone_in_lock=true"),
4656                    new_slot,
4657                ));
4658            }
4659
4660            slot = new_slot;
4661            storage_location = new_storage_location;
4662        }
4663    }
4664
4665    fn do_load(
4666        &self,
4667        ancestors: &Ancestors,
4668        pubkey: &Pubkey,
4669        max_root: Option<Slot>,
4670        load_hint: LoadHint,
4671    ) -> Option<(AccountSharedData, Slot)> {
4672        self.do_load_with_populate_read_cache(ancestors, pubkey, max_root, load_hint, false)
4673    }
4674
4675    /// if 'load_into_read_cache_only', then return value is meaningless.
4676    ///   The goal is to get the account into the read-only cache.
4677    fn do_load_with_populate_read_cache(
4678        &self,
4679        ancestors: &Ancestors,
4680        pubkey: &Pubkey,
4681        max_root: Option<Slot>,
4682        load_hint: LoadHint,
4683        load_into_read_cache_only: bool,
4684    ) -> Option<(AccountSharedData, Slot)> {
4685        #[cfg(not(test))]
4686        assert!(max_root.is_none());
4687
4688        let (slot, storage_location, _maybe_account_accesor) =
4689            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
4690        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
4691
4692        if self.caching_enabled {
4693            let in_write_cache = storage_location.is_cached();
4694            if !load_into_read_cache_only {
4695                if !in_write_cache {
4696                    let result = self.read_only_accounts_cache.load(*pubkey, slot);
4697                    if let Some(account) = result {
4698                        return Some((account, slot));
4699                    }
4700                }
4701            } else {
4702                // goal is to load into read cache
4703                if in_write_cache {
4704                    // no reason to load in read cache. already in write cache
4705                    return None;
4706                }
4707                if self.read_only_accounts_cache.in_cache(pubkey, slot) {
4708                    // already in read cache
4709                    return None;
4710                }
4711            }
4712        }
4713
4714        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
4715            slot,
4716            storage_location,
4717            ancestors,
4718            pubkey,
4719            max_root,
4720            load_hint,
4721        )?;
4722        let loaded_account = account_accessor.check_and_get_loaded_account();
4723        let is_cached = loaded_account.is_cached();
4724        let account = loaded_account.take_account();
4725
4726        if self.caching_enabled && !is_cached {
4727            /*
4728            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
4729            safe/reflect 'A''s latest state on this fork.
4730            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
4731            not the read-only cache, after it's been updated in replay of slot 'S'.
4732            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
4733            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
4734            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
4735            which means '(S', A)' does not exist in the write cache yet.
4736            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
4737            must exist in the write cache, which is a contradiction.
4738            */
4739            self.read_only_accounts_cache
4740                .store(*pubkey, slot, account.clone());
4741        }
4742        Some((account, slot))
4743    }
4744
4745    pub fn load_account_hash(
4746        &self,
4747        ancestors: &Ancestors,
4748        pubkey: &Pubkey,
4749        max_root: Option<Slot>,
4750        load_hint: LoadHint,
4751    ) -> Option<Hash> {
4752        let (slot, storage_location, _maybe_account_accesor) =
4753            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
4754        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
4755
4756        let (mut account_accessor, _) = self.retry_to_get_account_accessor(
4757            slot,
4758            storage_location,
4759            ancestors,
4760            pubkey,
4761            max_root,
4762            load_hint,
4763        )?;
4764        let loaded_account = account_accessor.check_and_get_loaded_account();
4765        Some(loaded_account.loaded_hash())
4766    }
4767
4768    fn get_account_accessor<'a>(
4769        &'a self,
4770        slot: Slot,
4771        pubkey: &'a Pubkey,
4772        storage_location: &StorageLocation,
4773    ) -> LoadedAccountAccessor<'a> {
4774        match storage_location {
4775            StorageLocation::Cached => {
4776                let maybe_cached_account = self.accounts_cache.load(slot, pubkey).map(Cow::Owned);
4777                LoadedAccountAccessor::Cached(maybe_cached_account)
4778            }
4779            StorageLocation::AppendVec(store_id, offset) => {
4780                let maybe_storage_entry = self
4781                    .storage
4782                    .get_account_storage_entry(slot, *store_id)
4783                    .map(|account_storage_entry| (account_storage_entry, *offset));
4784                LoadedAccountAccessor::Stored(maybe_storage_entry)
4785            }
4786        }
4787    }
4788
4789    fn try_recycle_and_insert_store(
4790        &self,
4791        slot: Slot,
4792        min_size: u64,
4793        max_size: u64,
4794    ) -> Option<Arc<AccountStorageEntry>> {
4795        let store = self.try_recycle_store(slot, min_size, max_size)?;
4796        self.insert_store(slot, store.clone());
4797        Some(store)
4798    }
4799
4800    fn try_recycle_store(
4801        &self,
4802        slot: Slot,
4803        min_size: u64,
4804        max_size: u64,
4805    ) -> Option<Arc<AccountStorageEntry>> {
4806        let mut max = 0;
4807        let mut min = std::u64::MAX;
4808        let mut avail = 0;
4809        let mut recycle_stores = self.recycle_stores.write().unwrap();
4810        for (i, (_recycled_time, store)) in recycle_stores.iter().enumerate() {
4811            if Arc::strong_count(store) == 1 {
4812                max = std::cmp::max(store.accounts.capacity(), max);
4813                min = std::cmp::min(store.accounts.capacity(), min);
4814                avail += 1;
4815
4816                if store.accounts.capacity() >= min_size && store.accounts.capacity() < max_size {
4817                    let ret = recycle_stores.remove_entry(i);
4818                    drop(recycle_stores);
4819                    let old_id = ret.append_vec_id();
4820                    ret.recycle(slot, self.next_id());
4821                    debug!(
4822                        "recycling store: {} {:?} old_id: {}",
4823                        ret.append_vec_id(),
4824                        ret.get_path(),
4825                        old_id
4826                    );
4827                    return Some(ret);
4828                }
4829            }
4830        }
4831        debug!(
4832            "no recycle stores max: {} min: {} len: {} looking: {}, {} avail: {}",
4833            max,
4834            min,
4835            recycle_stores.entry_count(),
4836            min_size,
4837            max_size,
4838            avail,
4839        );
4840        None
4841    }
4842
4843    fn find_storage_candidate(&self, slot: Slot, size: usize) -> Arc<AccountStorageEntry> {
4844        let mut create_extra = false;
4845        let mut get_slot_stores = Measure::start("get_slot_stores");
4846        let slot_stores_lock = self.storage.get_slot_stores(slot);
4847        get_slot_stores.stop();
4848        self.stats
4849            .store_get_slot_store
4850            .fetch_add(get_slot_stores.as_us(), Ordering::Relaxed);
4851        let mut find_existing = Measure::start("find_existing");
4852        if let Some(slot_stores_lock) = slot_stores_lock {
4853            let slot_stores = slot_stores_lock.read().unwrap();
4854            if !slot_stores.is_empty() {
4855                if slot_stores.len() <= self.min_num_stores {
4856                    let mut total_accounts = 0;
4857                    for store in slot_stores.values() {
4858                        total_accounts += store.count();
4859                    }
4860
4861                    // Create more stores so that when scanning the storage all CPUs have work
4862                    if (total_accounts / 16) >= slot_stores.len() {
4863                        create_extra = true;
4864                    }
4865                }
4866
4867                // pick an available store at random by iterating from a random point
4868                let to_skip = thread_rng().gen_range(0, slot_stores.len());
4869
4870                for (i, store) in slot_stores.values().cycle().skip(to_skip).enumerate() {
4871                    if store.try_available() {
4872                        let ret = store.clone();
4873                        drop(slot_stores);
4874                        if create_extra {
4875                            if self
4876                                .try_recycle_and_insert_store(slot, size as u64, std::u64::MAX)
4877                                .is_none()
4878                            {
4879                                self.stats
4880                                    .create_store_count
4881                                    .fetch_add(1, Ordering::Relaxed);
4882                                self.create_and_insert_store(slot, self.file_size, "store extra");
4883                            } else {
4884                                self.stats
4885                                    .recycle_store_count
4886                                    .fetch_add(1, Ordering::Relaxed);
4887                            }
4888                        }
4889                        find_existing.stop();
4890                        self.stats
4891                            .store_find_existing
4892                            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
4893                        return ret;
4894                    }
4895                    // looked at every store, bail...
4896                    if i == slot_stores.len() {
4897                        break;
4898                    }
4899                }
4900            }
4901        }
4902        find_existing.stop();
4903        self.stats
4904            .store_find_existing
4905            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
4906
4907        let store = if let Some(store) = self.try_recycle_store(slot, size as u64, std::u64::MAX) {
4908            self.stats
4909                .recycle_store_count
4910                .fetch_add(1, Ordering::Relaxed);
4911            store
4912        } else {
4913            self.stats
4914                .create_store_count
4915                .fetch_add(1, Ordering::Relaxed);
4916            self.create_store(slot, self.file_size, "store", &self.paths)
4917        };
4918
4919        // try_available is like taking a lock on the store,
4920        // preventing other threads from using it.
4921        // It must succeed here and happen before insert,
4922        // otherwise another thread could also grab it from the index.
4923        assert!(store.try_available());
4924        self.insert_store(slot, store.clone());
4925        store
4926    }
4927
4928    pub(crate) fn page_align(size: u64) -> u64 {
4929        (size + (PAGE_SIZE - 1)) & !(PAGE_SIZE - 1)
4930    }
4931
4932    fn has_space_available(&self, slot: Slot, size: u64) -> bool {
4933        let slot_storage = self.storage.get_slot_stores(slot).unwrap();
4934        let slot_storage_r = slot_storage.read().unwrap();
4935        for (_id, store) in slot_storage_r.iter() {
4936            if store.status() == AccountStorageStatus::Available
4937                && (store.accounts.capacity() - store.accounts.len() as u64) > size
4938            {
4939                return true;
4940            }
4941        }
4942        false
4943    }
4944
4945    fn create_store(
4946        &self,
4947        slot: Slot,
4948        size: u64,
4949        from: &str,
4950        paths: &[PathBuf],
4951    ) -> Arc<AccountStorageEntry> {
4952        let path_index = thread_rng().gen_range(0, paths.len());
4953        let store = Arc::new(self.new_storage_entry(
4954            slot,
4955            Path::new(&paths[path_index]),
4956            Self::page_align(size),
4957        ));
4958
4959        debug!(
4960            "creating store: {} slot: {} len: {} size: {} from: {} path: {:?}",
4961            store.append_vec_id(),
4962            slot,
4963            store.accounts.len(),
4964            store.accounts.capacity(),
4965            from,
4966            store.accounts.get_path()
4967        );
4968
4969        store
4970    }
4971
4972    fn create_and_insert_store(
4973        &self,
4974        slot: Slot,
4975        size: u64,
4976        from: &str,
4977    ) -> Arc<AccountStorageEntry> {
4978        self.create_and_insert_store_with_paths(slot, size, from, &self.paths)
4979    }
4980
4981    fn create_and_insert_store_with_paths(
4982        &self,
4983        slot: Slot,
4984        size: u64,
4985        from: &str,
4986        paths: &[PathBuf],
4987    ) -> Arc<AccountStorageEntry> {
4988        let store = self.create_store(slot, size, from, paths);
4989        let store_for_index = store.clone();
4990
4991        self.insert_store(slot, store_for_index);
4992        store
4993    }
4994
4995    fn insert_store(&self, slot: Slot, store: Arc<AccountStorageEntry>) {
4996        let slot_storages: SlotStores = self.storage.get_slot_stores(slot).unwrap_or_else(||
4997            // DashMap entry.or_insert() returns a RefMut, essentially a write lock,
4998            // which is dropped after this block ends, minimizing time held by the lock.
4999            // However, we still want to persist the reference to the `SlotStores` behind
5000            // the lock, hence we clone it out, (`SlotStores` is an Arc so is cheap to clone).
5001            self.storage
5002                .map
5003                .entry(slot)
5004                .or_insert(Arc::new(RwLock::new(HashMap::new())))
5005                .clone());
5006
5007        assert!(slot_storages
5008            .write()
5009            .unwrap()
5010            .insert(store.append_vec_id(), store)
5011            .is_none());
5012    }
5013
5014    pub fn create_drop_bank_callback(
5015        &self,
5016        pruned_banks_sender: DroppedSlotsSender,
5017    ) -> SendDroppedBankCallback {
5018        self.is_bank_drop_callback_enabled
5019            .store(true, Ordering::Release);
5020        SendDroppedBankCallback::new(pruned_banks_sender)
5021    }
5022
5023    /// This should only be called after the `Bank::drop()` runs in bank.rs, See BANK_DROP_SAFETY
5024    /// comment below for more explanation.
5025    ///   * `is_serialized_with_abs` - indicates whehter this call runs sequentially with all other
5026    ///        accounts_db relevant calls, such as shrinking, purging etc., in account background
5027    ///        service.
5028    pub fn purge_slot(&self, slot: Slot, bank_id: BankId, is_serialized_with_abs: bool) {
5029        if self.is_bank_drop_callback_enabled.load(Ordering::Acquire) && !is_serialized_with_abs {
5030            panic!(
5031                "bad drop callpath detected; Bank::drop() must run serially with other logic in
5032                ABS like clean_accounts()"
5033            )
5034        }
5035
5036        // BANK_DROP_SAFETY: Because this function only runs once the bank is dropped,
5037        // we know that there are no longer any ongoing scans on this bank, because scans require
5038        // and hold a reference to the bank at the tip of the fork they're scanning. Hence it's
5039        // safe to remove this bank_id from the `removed_bank_ids` list at this point.
5040        if self
5041            .accounts_index
5042            .removed_bank_ids
5043            .lock()
5044            .unwrap()
5045            .remove(&bank_id)
5046        {
5047            // If this slot was already cleaned up, no need to do any further cleans
5048            return;
5049        }
5050
5051        self.purge_slots(std::iter::once(&slot));
5052    }
5053
5054    fn recycle_slot_stores(
5055        &self,
5056        total_removed_storage_entries: usize,
5057        slot_stores: &[SlotStores],
5058    ) -> u64 {
5059        let mut recycled_count = 0;
5060
5061        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_elapsed");
5062        let mut recycle_stores = self.recycle_stores.write().unwrap();
5063        recycle_stores_write_elapsed.stop();
5064
5065        for slot_entries in slot_stores {
5066            let entry = slot_entries.read().unwrap();
5067            for (_store_id, stores) in entry.iter() {
5068                if recycle_stores.entry_count() > MAX_RECYCLE_STORES {
5069                    let dropped_count = total_removed_storage_entries - recycled_count;
5070                    self.stats
5071                        .dropped_stores
5072                        .fetch_add(dropped_count as u64, Ordering::Relaxed);
5073                    return recycle_stores_write_elapsed.as_us();
5074                }
5075                recycle_stores.add_entry(stores.clone());
5076                recycled_count += 1;
5077            }
5078        }
5079        recycle_stores_write_elapsed.as_us()
5080    }
5081
5082    /// Purges every slot in `removed_slots` from both the cache and storage. This includes
5083    /// entries in the accounts index, cache entries, and any backing storage entries.
5084    pub(crate) fn purge_slots_from_cache_and_store<'a>(
5085        &self,
5086        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5087        purge_stats: &PurgeStats,
5088        log_accounts: bool,
5089    ) {
5090        let mut remove_cache_elapsed_across_slots = 0;
5091        let mut num_cached_slots_removed = 0;
5092        let mut total_removed_cached_bytes = 0;
5093        if log_accounts {
5094            if let Some(min) = removed_slots.clone().min() {
5095                info!(
5096                    "purge_slots_from_cache_and_store: {:?}",
5097                    self.get_pubkey_hash_for_slot(*min).0
5098                );
5099            }
5100        }
5101        for remove_slot in removed_slots {
5102            // This function is only currently safe with respect to `flush_slot_cache()` because
5103            // both functions run serially in AccountsBackgroundService.
5104            let mut remove_cache_elapsed = Measure::start("remove_cache_elapsed");
5105            // Note: we cannot remove this slot from the slot cache until we've removed its
5106            // entries from the accounts index first. This is because `scan_accounts()` relies on
5107            // holding the index lock, finding the index entry, and then looking up the entry
5108            // in the cache. If it fails to find that entry, it will panic in `get_loaded_account()`
5109            if let Some(slot_cache) = self.accounts_cache.slot_cache(*remove_slot) {
5110                // If the slot is still in the cache, remove the backing storages for
5111                // the slot and from the Accounts Index
5112                num_cached_slots_removed += 1;
5113                total_removed_cached_bytes += slot_cache.total_bytes();
5114                self.purge_slot_cache(*remove_slot, slot_cache);
5115                remove_cache_elapsed.stop();
5116                remove_cache_elapsed_across_slots += remove_cache_elapsed.as_us();
5117                // Nobody else shoud have removed the slot cache entry yet
5118                assert!(self.accounts_cache.remove_slot(*remove_slot).is_some());
5119            } else {
5120                self.purge_slot_storage(*remove_slot, purge_stats);
5121            }
5122            // It should not be possible that a slot is neither in the cache or storage. Even in
5123            // a slot with all ticks, `Bank::new_from_parent()` immediately stores some sysvars
5124            // on bank creation.
5125        }
5126
5127        purge_stats
5128            .remove_cache_elapsed
5129            .fetch_add(remove_cache_elapsed_across_slots, Ordering::Relaxed);
5130        purge_stats
5131            .num_cached_slots_removed
5132            .fetch_add(num_cached_slots_removed, Ordering::Relaxed);
5133        purge_stats
5134            .total_removed_cached_bytes
5135            .fetch_add(total_removed_cached_bytes, Ordering::Relaxed);
5136    }
5137
5138    /// Purge the backing storage entries for the given slot, does not purge from
5139    /// the cache!
5140    fn purge_dead_slots_from_storage<'a>(
5141        &'a self,
5142        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
5143        purge_stats: &PurgeStats,
5144    ) {
5145        // Check all slots `removed_slots` are no longer "relevant" roots.
5146        // Note that the slots here could have been rooted slots, but if they're passed here
5147        // for removal it means:
5148        // 1) All updates in that old root have been outdated by updates in newer roots
5149        // 2) Those slots/roots should have already been purged from the accounts index root
5150        // tracking metadata via `accounts_index.clean_dead_slot()`.
5151        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5152        assert!(self
5153            .accounts_index
5154            .get_rooted_from_list(removed_slots.clone())
5155            .is_empty());
5156        safety_checks_elapsed.stop();
5157        purge_stats
5158            .safety_checks_elapsed
5159            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5160
5161        let mut total_removed_storage_entries = 0;
5162        let mut total_removed_stored_bytes = 0;
5163        let mut all_removed_slot_storages = vec![];
5164
5165        let mut remove_storage_entries_elapsed = Measure::start("remove_storage_entries_elapsed");
5166        for remove_slot in removed_slots {
5167            // Remove the storage entries and collect some metrics
5168            if let Some((_, slot_storages_to_be_removed)) = self.storage.map.remove(remove_slot) {
5169                {
5170                    let r_slot_removed_storages = slot_storages_to_be_removed.read().unwrap();
5171                    total_removed_storage_entries += r_slot_removed_storages.len();
5172                    total_removed_stored_bytes += r_slot_removed_storages
5173                        .values()
5174                        .map(|i| i.accounts.capacity())
5175                        .sum::<u64>();
5176                }
5177                all_removed_slot_storages.push(slot_storages_to_be_removed.clone());
5178            }
5179        }
5180        remove_storage_entries_elapsed.stop();
5181        let num_stored_slots_removed = all_removed_slot_storages.len();
5182
5183        let recycle_stores_write_elapsed =
5184            self.recycle_slot_stores(total_removed_storage_entries, &all_removed_slot_storages);
5185
5186        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
5187        // Backing mmaps for removed storages entries explicitly dropped here outside
5188        // of any locks
5189        drop(all_removed_slot_storages);
5190        drop_storage_entries_elapsed.stop();
5191        purge_stats
5192            .remove_storage_entries_elapsed
5193            .fetch_add(remove_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5194        purge_stats
5195            .drop_storage_entries_elapsed
5196            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5197        purge_stats
5198            .num_stored_slots_removed
5199            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
5200        purge_stats
5201            .total_removed_storage_entries
5202            .fetch_add(total_removed_storage_entries, Ordering::Relaxed);
5203        purge_stats
5204            .total_removed_stored_bytes
5205            .fetch_add(total_removed_stored_bytes, Ordering::Relaxed);
5206        purge_stats
5207            .recycle_stores_write_elapsed
5208            .fetch_add(recycle_stores_write_elapsed, Ordering::Relaxed);
5209    }
5210
5211    fn purge_slot_cache(&self, purged_slot: Slot, slot_cache: SlotCache) {
5212        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
5213        let pubkey_to_slot_set: Vec<(Pubkey, Slot)> = slot_cache
5214            .iter()
5215            .map(|account| {
5216                purged_slot_pubkeys.insert((purged_slot, *account.key()));
5217                (*account.key(), purged_slot)
5218            })
5219            .collect();
5220        self.purge_slot_cache_pubkeys(
5221            purged_slot,
5222            purged_slot_pubkeys,
5223            pubkey_to_slot_set,
5224            true,
5225            &HashSet::default(),
5226        );
5227    }
5228
5229    fn purge_slot_cache_pubkeys(
5230        &self,
5231        purged_slot: Slot,
5232        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
5233        pubkey_to_slot_set: Vec<(Pubkey, Slot)>,
5234        is_dead: bool,
5235        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
5236    ) {
5237        // Slot purged from cache should not exist in the backing store
5238        assert!(self.storage.get_slot_stores(purged_slot).is_none());
5239        let num_purged_keys = pubkey_to_slot_set.len();
5240        let (reclaims, _) = self.purge_keys_exact(pubkey_to_slot_set.iter());
5241        assert_eq!(reclaims.len(), num_purged_keys);
5242        if is_dead {
5243            self.remove_dead_slots_metadata(
5244                std::iter::once(&purged_slot),
5245                purged_slot_pubkeys,
5246                None,
5247                pubkeys_removed_from_accounts_index,
5248            );
5249        }
5250    }
5251
5252    fn purge_slot_storage(&self, remove_slot: Slot, purge_stats: &PurgeStats) {
5253        // Because AccountsBackgroundService synchronously flushes from the accounts cache
5254        // and handles all Bank::drop() (the cleanup function that leads to this
5255        // function call), then we don't need to worry above an overlapping cache flush
5256        // with this function call. This means, if we get into this case, we can be
5257        // confident that the entire state for this slot has been flushed to the storage
5258        // already.
5259        let mut scan_storages_elasped = Measure::start("scan_storages_elasped");
5260        type ScanResult = ScanStorageResult<Pubkey, Arc<Mutex<HashSet<(Pubkey, Slot)>>>>;
5261        let scan_result: ScanResult = self.scan_account_storage(
5262            remove_slot,
5263            |loaded_account: LoadedAccount| Some(*loaded_account.pubkey()),
5264            |accum: &Arc<Mutex<HashSet<(Pubkey, Slot)>>>, loaded_account: LoadedAccount| {
5265                accum
5266                    .lock()
5267                    .unwrap()
5268                    .insert((*loaded_account.pubkey(), remove_slot));
5269            },
5270        );
5271        scan_storages_elasped.stop();
5272        purge_stats
5273            .scan_storages_elapsed
5274            .fetch_add(scan_storages_elasped.as_us(), Ordering::Relaxed);
5275
5276        let mut purge_accounts_index_elapsed = Measure::start("purge_accounts_index_elapsed");
5277        let (reclaims, pubkeys_removed_from_accounts_index) = match scan_result {
5278            ScanStorageResult::Cached(_) => {
5279                panic!("Should not see cached keys in this `else` branch, since we checked this slot did not exist in the cache above");
5280            }
5281            ScanStorageResult::Stored(stored_keys) => {
5282                // Purge this slot from the accounts index
5283                self.purge_keys_exact(stored_keys.lock().unwrap().iter())
5284            }
5285        };
5286        purge_accounts_index_elapsed.stop();
5287        purge_stats
5288            .purge_accounts_index_elapsed
5289            .fetch_add(purge_accounts_index_elapsed.as_us(), Ordering::Relaxed);
5290
5291        // `handle_reclaims()` should remove all the account index entries and
5292        // storage entries
5293        let mut handle_reclaims_elapsed = Measure::start("handle_reclaims_elapsed");
5294        // Slot should be dead after removing all its account entries
5295        let expected_dead_slot = Some(remove_slot);
5296        self.handle_reclaims(
5297            (!reclaims.is_empty()).then(|| reclaims.iter()),
5298            expected_dead_slot,
5299            Some((purge_stats, &mut ReclaimResult::default())),
5300            false,
5301            &pubkeys_removed_from_accounts_index,
5302        );
5303        handle_reclaims_elapsed.stop();
5304        purge_stats
5305            .handle_reclaims_elapsed
5306            .fetch_add(handle_reclaims_elapsed.as_us(), Ordering::Relaxed);
5307        // After handling the reclaimed entries, this slot's
5308        // storage entries should be purged from self.storage
5309        assert!(
5310            self.storage.get_slot_stores(remove_slot).is_none(),
5311            "slot {} is not none",
5312            remove_slot
5313        );
5314    }
5315
5316    #[allow(clippy::needless_collect)]
5317    fn purge_slots<'a>(&self, slots: impl Iterator<Item = &'a Slot> + Clone) {
5318        // `add_root()` should be called first
5319        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
5320        let non_roots = slots
5321            // Only safe to check when there are duplicate versions of a slot
5322            // because ReplayStage will not make new roots before dumping the
5323            // duplicate slots first. Thus we will not be in a case where we
5324            // root slot `S`, then try to dump some other version of slot `S`, the
5325            // dumping has to finish first
5326            //
5327            // Also note roots are never removed via `remove_unrooted_slot()`, so
5328            // it's safe to filter them out here as they won't need deletion from
5329            // self.accounts_index.removed_bank_ids in `purge_slots_from_cache_and_store()`.
5330            .filter(|slot| !self.accounts_index.is_alive_root(**slot));
5331        safety_checks_elapsed.stop();
5332        self.external_purge_slots_stats
5333            .safety_checks_elapsed
5334            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
5335        self.purge_slots_from_cache_and_store(non_roots, &self.external_purge_slots_stats, false);
5336        self.external_purge_slots_stats
5337            .report("external_purge_slots_stats", Some(1000));
5338    }
5339
5340    pub fn remove_unrooted_slots(&self, remove_slots: &[(Slot, BankId)]) {
5341        let rooted_slots = self
5342            .accounts_index
5343            .get_rooted_from_list(remove_slots.iter().map(|(slot, _)| slot));
5344        assert!(
5345            rooted_slots.is_empty(),
5346            "Trying to remove accounts for rooted slots {:?}",
5347            rooted_slots
5348        );
5349
5350        let RemoveUnrootedSlotsSynchronization {
5351            slots_under_contention,
5352            signal,
5353        } = &self.remove_unrooted_slots_synchronization;
5354
5355        {
5356            // Slots that are currently being flushed by flush_slot_cache()
5357
5358            let mut currently_contended_slots = slots_under_contention.lock().unwrap();
5359
5360            // Slots that are currently being flushed by flush_slot_cache() AND
5361            // we want to remove in this function
5362            let mut remaining_contended_flush_slots: Vec<Slot> = remove_slots
5363                .iter()
5364                .filter_map(|(remove_slot, _)| {
5365                    // Reserve the slots that we want to purge that aren't currently
5366                    // being flushed to prevent cache from flushing those slots in
5367                    // the future.
5368                    //
5369                    // Note that the single replay thread has to remove a specific slot `N`
5370                    // before another version of the same slot can be replayed. This means
5371                    // multiple threads should not call `remove_unrooted_slots()` simultaneously
5372                    // with the same slot.
5373                    let is_being_flushed = !currently_contended_slots.insert(*remove_slot);
5374                    // If the cache is currently flushing this slot, add it to the list
5375                    is_being_flushed.then(|| remove_slot)
5376                })
5377                .cloned()
5378                .collect();
5379
5380            // Wait for cache flushes to finish
5381            loop {
5382                if !remaining_contended_flush_slots.is_empty() {
5383                    // Wait for the signal that the cache has finished flushing a slot
5384                    //
5385                    // Don't wait if the remaining_contended_flush_slots is empty, otherwise
5386                    // we may never get a signal since there's no cache flush thread to
5387                    // do the signaling
5388                    currently_contended_slots = signal.wait(currently_contended_slots).unwrap();
5389                } else {
5390                    // There are no slots being flushed to wait on, so it's safe to continue
5391                    // to purging the slots we want to purge!
5392                    break;
5393                }
5394
5395                // For each slot the cache flush has finished, mark that we're about to start
5396                // purging these slots by reserving it in `currently_contended_slots`.
5397                remaining_contended_flush_slots.retain(|flush_slot| {
5398                    // returns true if slot was already in set. This means slot is being flushed
5399                    !currently_contended_slots.insert(*flush_slot)
5400                });
5401            }
5402        }
5403
5404        // Mark down these slots are about to be purged so that new attempts to scan these
5405        // banks fail, and any ongoing scans over these slots will detect that they should abort
5406        // their results
5407        {
5408            let mut locked_removed_bank_ids = self.accounts_index.removed_bank_ids.lock().unwrap();
5409            for (_slot, remove_bank_id) in remove_slots.iter() {
5410                locked_removed_bank_ids.insert(*remove_bank_id);
5411            }
5412        }
5413
5414        let remove_unrooted_purge_stats = PurgeStats::default();
5415        self.purge_slots_from_cache_and_store(
5416            remove_slots.iter().map(|(slot, _)| slot),
5417            &remove_unrooted_purge_stats,
5418            true,
5419        );
5420        remove_unrooted_purge_stats.report("remove_unrooted_slots_purge_slots_stats", Some(0));
5421
5422        let mut currently_contended_slots = slots_under_contention.lock().unwrap();
5423        for (remove_slot, _) in remove_slots {
5424            assert!(currently_contended_slots.remove(remove_slot));
5425        }
5426    }
5427
5428    pub fn hash_account_with_rent_epoch<T: ReadableAccount>(
5429        slot: Slot,
5430        account: &T,
5431        pubkey: &Pubkey,
5432        rent_epoch: Epoch,
5433    ) -> Hash {
5434        Self::hash_account_data(
5435            slot,
5436            account.lamports(),
5437            account.owner(),
5438            account.executable(),
5439            rent_epoch,
5440            account.data(),
5441            pubkey,
5442        )
5443    }
5444
5445    pub fn hash_account<T: ReadableAccount>(slot: Slot, account: &T, pubkey: &Pubkey) -> Hash {
5446        Self::hash_account_data(
5447            slot,
5448            account.lamports(),
5449            account.owner(),
5450            account.executable(),
5451            account.rent_epoch(),
5452            account.data(),
5453            pubkey,
5454        )
5455    }
5456
5457    fn hash_account_data(
5458        slot: Slot,
5459        lamports: u64,
5460        owner: &Pubkey,
5461        executable: bool,
5462        rent_epoch: Epoch,
5463        data: &[u8],
5464        pubkey: &Pubkey,
5465    ) -> Hash {
5466        if lamports == 0 {
5467            return Hash::default();
5468        }
5469
5470        let mut hasher = blake3::Hasher::new();
5471
5472        hasher.update(&lamports.to_le_bytes());
5473
5474        hasher.update(&slot.to_le_bytes());
5475
5476        hasher.update(&rent_epoch.to_le_bytes());
5477
5478        hasher.update(data);
5479
5480        if executable {
5481            hasher.update(&[1u8; 1]);
5482        } else {
5483            hasher.update(&[0u8; 1]);
5484        }
5485
5486        hasher.update(owner.as_ref());
5487        hasher.update(pubkey.as_ref());
5488
5489        Hash::new_from_array(
5490            <[u8; solana_sdk::hash::HASH_BYTES]>::try_from(hasher.finalize().as_slice()).unwrap(),
5491        )
5492    }
5493
5494    fn bulk_assign_write_version(&self, count: usize) -> StoredMetaWriteVersion {
5495        self.write_version
5496            .fetch_add(count as StoredMetaWriteVersion, Ordering::AcqRel)
5497    }
5498
5499    fn write_accounts_to_storage<F: FnMut(Slot, usize) -> Arc<AccountStorageEntry>>(
5500        &self,
5501        slot: Slot,
5502        hashes: &[impl Borrow<Hash>],
5503        mut storage_finder: F,
5504        accounts_and_meta_to_store: &[(StoredMeta, Option<&impl ReadableAccount>)],
5505    ) -> Vec<AccountInfo> {
5506        assert_eq!(hashes.len(), accounts_and_meta_to_store.len());
5507        let mut infos: Vec<AccountInfo> = Vec::with_capacity(accounts_and_meta_to_store.len());
5508        let mut total_append_accounts_us = 0;
5509        let mut total_storage_find_us = 0;
5510        while infos.len() < accounts_and_meta_to_store.len() {
5511            let mut storage_find = Measure::start("storage_finder");
5512            let data_len = accounts_and_meta_to_store[infos.len()]
5513                .1
5514                .map(|account| account.data().len())
5515                .unwrap_or_default();
5516            let storage = storage_finder(slot, data_len + STORE_META_OVERHEAD);
5517            storage_find.stop();
5518            total_storage_find_us += storage_find.as_us();
5519            let mut append_accounts = Measure::start("append_accounts");
5520            let rvs = storage.accounts.append_accounts(
5521                &accounts_and_meta_to_store[infos.len()..],
5522                &hashes[infos.len()..],
5523            );
5524            assert!(!rvs.is_empty());
5525            append_accounts.stop();
5526            total_append_accounts_us += append_accounts.as_us();
5527            if rvs.len() == 1 {
5528                storage.set_status(AccountStorageStatus::Full);
5529
5530                // See if an account overflows the append vecs in the slot.
5531                let data_len = (data_len + STORE_META_OVERHEAD) as u64;
5532                if !self.has_space_available(slot, data_len) {
5533                    let special_store_size = std::cmp::max(data_len * 2, self.file_size);
5534                    if self
5535                        .try_recycle_and_insert_store(slot, special_store_size, std::u64::MAX)
5536                        .is_none()
5537                    {
5538                        self.stats
5539                            .create_store_count
5540                            .fetch_add(1, Ordering::Relaxed);
5541                        self.create_and_insert_store(slot, special_store_size, "large create");
5542                    } else {
5543                        self.stats
5544                            .recycle_store_count
5545                            .fetch_add(1, Ordering::Relaxed);
5546                    }
5547                }
5548                continue;
5549            }
5550
5551            for (offsets, (_, account)) in rvs
5552                .windows(2)
5553                .zip(&accounts_and_meta_to_store[infos.len()..])
5554            {
5555                let stored_size = offsets[1] - offsets[0];
5556                storage.add_account(stored_size);
5557
5558                infos.push(AccountInfo::new(
5559                    StorageLocation::AppendVec(storage.append_vec_id(), offsets[0]),
5560                    stored_size as StoredSize, // stored_size should never exceed StoredSize::MAX because of max data len const
5561                    account
5562                        .map(|account| account.lamports())
5563                        .unwrap_or_default(),
5564                ));
5565            }
5566            // restore the state to available
5567            storage.set_status(AccountStorageStatus::Available);
5568        }
5569
5570        self.stats
5571            .store_append_accounts
5572            .fetch_add(total_append_accounts_us, Ordering::Relaxed);
5573        self.stats
5574            .store_find_store
5575            .fetch_add(total_storage_find_us, Ordering::Relaxed);
5576
5577        infos
5578    }
5579
5580    pub fn mark_slot_frozen(&self, slot: Slot) {
5581        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
5582            slot_cache.mark_slot_frozen();
5583            slot_cache.report_slot_store_metrics();
5584        }
5585        self.accounts_cache.report_size();
5586    }
5587
5588    pub fn expire_old_recycle_stores(&self) {
5589        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
5590        let recycle_stores = self.recycle_stores.write().unwrap().expire_old_entries();
5591        recycle_stores_write_elapsed.stop();
5592
5593        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
5594        drop(recycle_stores);
5595        drop_storage_entries_elapsed.stop();
5596
5597        self.clean_accounts_stats
5598            .purge_stats
5599            .drop_storage_entries_elapsed
5600            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
5601        self.clean_accounts_stats
5602            .purge_stats
5603            .recycle_stores_write_elapsed
5604            .fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
5605    }
5606
5607    pub fn flush_accounts_cache_slot(&self, slot: Slot) {
5608        self.flush_slot_cache(slot);
5609    }
5610
5611    /// true if write cache is too big
5612    fn should_aggressively_flush_cache(&self) -> bool {
5613        self.write_cache_limit_bytes
5614            .unwrap_or(WRITE_CACHE_LIMIT_BYTES_DEFAULT)
5615            < self.accounts_cache.size()
5616    }
5617
5618    // `force_flush` flushes all the cached roots `<= requested_flush_root`. It also then
5619    // flushes:
5620    // 1) excess remaining roots or unrooted slots while 'should_aggressively_flush_cache' is true
5621    pub fn flush_accounts_cache(&self, force_flush: bool, requested_flush_root: Option<Slot>) {
5622        #[cfg(not(test))]
5623        assert!(requested_flush_root.is_some());
5624
5625        if !force_flush && !self.should_aggressively_flush_cache() {
5626            return;
5627        }
5628
5629        // Flush only the roots <= requested_flush_root, so that snapshotting has all
5630        // the relevant roots in storage.
5631        let mut flush_roots_elapsed = Measure::start("flush_roots_elapsed");
5632        let mut account_bytes_saved = 0;
5633        let mut num_accounts_saved = 0;
5634
5635        let _guard = self.active_stats.activate(ActiveStatItem::Flush);
5636
5637        // Note even if force_flush is false, we will still flush all roots <= the
5638        // given `requested_flush_root`, even if some of the later roots cannot be used for
5639        // cleaning due to an ongoing scan
5640        let (total_new_cleaned_roots, num_cleaned_roots_flushed) = self
5641            .flush_rooted_accounts_cache(
5642                requested_flush_root,
5643                Some((&mut account_bytes_saved, &mut num_accounts_saved)),
5644            );
5645        flush_roots_elapsed.stop();
5646
5647        // Note we don't purge unrooted slots here because there may be ongoing scans/references
5648        // for those slot, let the Bank::drop() implementation do cleanup instead on dead
5649        // banks
5650
5651        // If 'should_aggressively_flush_cache', then flush the excess ones to storage
5652        let (total_new_excess_roots, num_excess_roots_flushed) =
5653            if self.should_aggressively_flush_cache() {
5654                // Start by flushing the roots
5655                //
5656                // Cannot do any cleaning on roots past `requested_flush_root` because future
5657                // snapshots may need updates from those later slots, hence we pass `None`
5658                // for `should_clean`.
5659                self.flush_rooted_accounts_cache(None, None)
5660            } else {
5661                (0, 0)
5662            };
5663
5664        let mut excess_slot_count = 0;
5665        let mut unflushable_unrooted_slot_count = 0;
5666        let max_flushed_root = self.accounts_cache.fetch_max_flush_root();
5667        if self.should_aggressively_flush_cache() {
5668            let old_slots = self.accounts_cache.cached_frozen_slots();
5669            excess_slot_count = old_slots.len();
5670            let mut flush_stats = FlushStats::default();
5671            old_slots.into_iter().for_each(|old_slot| {
5672                // Don't flush slots that are known to be unrooted
5673                if old_slot > max_flushed_root {
5674                    if self.should_aggressively_flush_cache() {
5675                        if let Some(stats) = self.flush_slot_cache(old_slot) {
5676                            flush_stats.num_flushed += stats.num_flushed;
5677                            flush_stats.num_purged += stats.num_purged;
5678                            flush_stats.total_size += stats.total_size;
5679                        }
5680                    }
5681                } else {
5682                    unflushable_unrooted_slot_count += 1;
5683                }
5684            });
5685            datapoint_info!(
5686                "accounts_db-flush_accounts_cache_aggressively",
5687                ("num_flushed", flush_stats.num_flushed, i64),
5688                ("num_purged", flush_stats.num_purged, i64),
5689                ("total_flush_size", flush_stats.total_size, i64),
5690                ("total_cache_size", self.accounts_cache.size(), i64),
5691                ("total_frozen_slots", excess_slot_count, i64),
5692                ("total_slots", self.accounts_cache.num_slots(), i64),
5693            );
5694        }
5695
5696        datapoint_info!(
5697            "accounts_db-flush_accounts_cache",
5698            ("total_new_cleaned_roots", total_new_cleaned_roots, i64),
5699            ("num_cleaned_roots_flushed", num_cleaned_roots_flushed, i64),
5700            ("total_new_excess_roots", total_new_excess_roots, i64),
5701            ("num_excess_roots_flushed", num_excess_roots_flushed, i64),
5702            ("excess_slot_count", excess_slot_count, i64),
5703            (
5704                "unflushable_unrooted_slot_count",
5705                unflushable_unrooted_slot_count,
5706                i64
5707            ),
5708            (
5709                "flush_roots_elapsed",
5710                flush_roots_elapsed.as_us() as i64,
5711                i64
5712            ),
5713            ("account_bytes_saved", account_bytes_saved, i64),
5714            ("num_accounts_saved", num_accounts_saved, i64),
5715        );
5716    }
5717
5718    fn flush_rooted_accounts_cache(
5719        &self,
5720        requested_flush_root: Option<Slot>,
5721        should_clean: Option<(&mut usize, &mut usize)>,
5722    ) -> (usize, usize) {
5723        let max_clean_root = should_clean.as_ref().and_then(|_| {
5724            // If there is a long running scan going on, this could prevent any cleaning
5725            // based on updates from slots > `max_clean_root`.
5726            self.max_clean_root(requested_flush_root)
5727        });
5728
5729        let mut written_accounts = HashSet::new();
5730
5731        // If `should_clean` is None, then`should_flush_f` is also None, which will cause
5732        // `flush_slot_cache` to flush all accounts to storage without cleaning any accounts.
5733        let mut should_flush_f = should_clean.map(|(account_bytes_saved, num_accounts_saved)| {
5734            move |&pubkey: &Pubkey, account: &AccountSharedData| {
5735                // if not in hashset, then not flushed previously, so flush it
5736                let should_flush = written_accounts.insert(pubkey);
5737                if !should_flush {
5738                    *account_bytes_saved += account.data().len();
5739                    *num_accounts_saved += 1;
5740                    // If a later root already wrote this account, no point
5741                    // in flushing it
5742                }
5743                should_flush
5744            }
5745        });
5746
5747        // Always flush up to `requested_flush_root`, which is necessary for things like snapshotting.
5748        let cached_roots: BTreeSet<Slot> = self.accounts_cache.clear_roots(requested_flush_root);
5749
5750        // Iterate from highest to lowest so that we don't need to flush earlier
5751        // outdated updates in earlier roots
5752        let mut num_roots_flushed = 0;
5753        for &root in cached_roots.iter().rev() {
5754            if self
5755                .flush_slot_cache_with_clean(&[root], should_flush_f.as_mut(), max_clean_root)
5756                .is_some()
5757            {
5758                num_roots_flushed += 1;
5759            }
5760
5761            // Regardless of whether this slot was *just* flushed from the cache by the above
5762            // `flush_slot_cache()`, we should update the `max_flush_root`.
5763            // This is because some rooted slots may be flushed to storage *before* they are marked as root.
5764            // This can occur for instance when
5765            //  the cache is overwhelmed, we flushed some yet to be rooted frozen slots
5766            // These slots may then *later* be marked as root, so we still need to handle updating the
5767            // `max_flush_root` in the accounts cache.
5768            self.accounts_cache.set_max_flush_root(root);
5769        }
5770
5771        // Only add to the uncleaned roots set *after* we've flushed the previous roots,
5772        // so that clean will actually be able to clean the slots.
5773        let num_new_roots = cached_roots.len();
5774        self.accounts_index.add_uncleaned_roots(cached_roots);
5775        (num_new_roots, num_roots_flushed)
5776    }
5777
5778    fn do_flush_slot_cache(
5779        &self,
5780        slot: Slot,
5781        slot_cache: &SlotCache,
5782        mut should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
5783        max_clean_root: Option<Slot>,
5784    ) -> FlushStats {
5785        let mut num_purged = 0;
5786        let mut total_size = 0;
5787        let mut num_flushed = 0;
5788        let iter_items: Vec<_> = slot_cache.iter().collect();
5789        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
5790        let mut pubkey_to_slot_set: Vec<(Pubkey, Slot)> = vec![];
5791        if should_flush_f.is_some() {
5792            if let Some(max_clean_root) = max_clean_root {
5793                if slot > max_clean_root {
5794                    // Only if the root is greater than the `max_clean_root` do we
5795                    // have to prevent cleaning, otherwise, just default to `should_flush_f`
5796                    // for any slots <= `max_clean_root`
5797                    should_flush_f = None;
5798                }
5799            }
5800        }
5801
5802        let mut filler_accounts = 0;
5803        if self.filler_accounts_enabled() {
5804            let slots_remaining = self.filler_account_slots_remaining.load(Ordering::Acquire);
5805            if slots_remaining > 0 {
5806                // figure out
5807                let pr = self.get_prior_root(slot);
5808
5809                if let Some(prior_root) = pr {
5810                    let filler_account_slots =
5811                        std::cmp::min(slot.saturating_sub(prior_root), slots_remaining);
5812                    self.filler_account_slots_remaining
5813                        .fetch_sub(filler_account_slots, Ordering::Release);
5814                    let filler_accounts_per_slot =
5815                        self.filler_accounts_per_slot.load(Ordering::Acquire);
5816                    filler_accounts = filler_account_slots * filler_accounts_per_slot;
5817
5818                    // keep space for filler accounts
5819                    let addl_size = (filler_accounts as u64)
5820                        * ((self.filler_accounts_config.size + STORE_META_OVERHEAD) as u64);
5821                    total_size += addl_size;
5822                }
5823            }
5824        }
5825
5826        let (accounts, hashes): (Vec<(&Pubkey, &AccountSharedData)>, Vec<Hash>) = iter_items
5827            .iter()
5828            .filter_map(|iter_item| {
5829                let key = iter_item.key();
5830                let account = &iter_item.value().account;
5831                let should_flush = should_flush_f
5832                    .as_mut()
5833                    .map(|should_flush_f| should_flush_f(key, account))
5834                    .unwrap_or(true);
5835                if should_flush {
5836                    let hash = iter_item.value().hash();
5837                    total_size += (account.data().len() + STORE_META_OVERHEAD) as u64;
5838                    num_flushed += 1;
5839                    Some(((key, account), hash))
5840                } else {
5841                    // If we don't flush, we have to remove the entry from the
5842                    // index, since it's equivalent to purging
5843                    purged_slot_pubkeys.insert((slot, *key));
5844                    pubkey_to_slot_set.push((*key, slot));
5845                    num_purged += 1;
5846                    None
5847                }
5848            })
5849            .unzip();
5850
5851        let is_dead_slot = accounts.is_empty();
5852        // Remove the account index entries from earlier roots that are outdated by later roots.
5853        // Safe because queries to the index will be reading updates from later roots.
5854        self.purge_slot_cache_pubkeys(
5855            slot,
5856            purged_slot_pubkeys,
5857            pubkey_to_slot_set,
5858            is_dead_slot,
5859            &HashSet::default(),
5860        );
5861
5862        if !is_dead_slot {
5863            let aligned_total_size = Self::page_align(total_size);
5864            // This ensures that all updates are written to an AppendVec, before any
5865            // updates to the index happen, so anybody that sees a real entry in the index,
5866            // will be able to find the account in storage
5867            let flushed_store =
5868                self.create_and_insert_store(slot, aligned_total_size, "flush_slot_cache");
5869            self.store_accounts_frozen(
5870                (slot, &accounts[..]),
5871                Some(&hashes),
5872                Some(&flushed_store),
5873                None,
5874                StoreReclaims::Default,
5875            );
5876
5877            if filler_accounts > 0 {
5878                // add extra filler accounts at the end of the append vec
5879                let (account, hash) = self.get_filler_account(&Rent::default());
5880                let mut accounts = Vec::with_capacity(filler_accounts as usize);
5881                let mut hashes = Vec::with_capacity(filler_accounts as usize);
5882                let pubkeys = self.get_filler_account_pubkeys(filler_accounts as usize);
5883                pubkeys.iter().for_each(|key| {
5884                    accounts.push((key, &account));
5885                    hashes.push(hash);
5886                });
5887                self.store_accounts_frozen(
5888                    (slot, &accounts[..]),
5889                    Some(&hashes),
5890                    Some(&flushed_store),
5891                    None,
5892                    StoreReclaims::Ignore,
5893                );
5894            }
5895
5896            // If the above sizing function is correct, just one AppendVec is enough to hold
5897            // all the data for the slot
5898            assert_eq!(
5899                self.storage
5900                    .get_slot_stores(slot)
5901                    .unwrap()
5902                    .read()
5903                    .unwrap()
5904                    .len(),
5905                1
5906            );
5907        }
5908
5909        // Remove this slot from the cache, which will to AccountsDb's new readers should look like an
5910        // atomic switch from the cache to storage.
5911        // There is some racy condition for existing readers who just has read exactly while
5912        // flushing. That case is handled by retry_to_get_account_accessor()
5913        assert!(self.accounts_cache.remove_slot(slot).is_some());
5914        FlushStats {
5915            num_flushed,
5916            num_purged,
5917            total_size,
5918        }
5919    }
5920
5921    /// flush all accounts in this slot
5922    fn flush_slot_cache(&self, slot: Slot) -> Option<FlushStats> {
5923        self.flush_slot_cache_with_clean(&[slot], None::<&mut fn(&_, &_) -> bool>, None)
5924    }
5925
5926    /// `should_flush_f` is an optional closure that determines whether a given
5927    /// account should be flushed. Passing `None` will by default flush all
5928    /// accounts
5929    fn flush_slot_cache_with_clean(
5930        &self,
5931        slots: &[Slot],
5932        should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
5933        max_clean_root: Option<Slot>,
5934    ) -> Option<FlushStats> {
5935        assert_eq!(1, slots.len());
5936        let slot = slots[0];
5937        if self
5938            .remove_unrooted_slots_synchronization
5939            .slots_under_contention
5940            .lock()
5941            .unwrap()
5942            .insert(slot)
5943        {
5944            // We have not seen this slot, flush it.
5945            let flush_stats = self.accounts_cache.slot_cache(slot).map(|slot_cache| {
5946                #[cfg(test)]
5947                {
5948                    // Give some time for cache flushing to occur here for unit tests
5949                    sleep(Duration::from_millis(self.load_delay));
5950                }
5951                // Since we added the slot to `slots_under_contention` AND this slot
5952                // still exists in the cache, we know the slot cannot be removed
5953                // by any other threads past this point. We are now responsible for
5954                // flushing this slot.
5955                self.do_flush_slot_cache(slot, &slot_cache, should_flush_f, max_clean_root)
5956            });
5957
5958            // Nobody else should have been purging this slot, so should not have been removed
5959            // from `self.remove_unrooted_slots_synchronization`.
5960
5961            slots.iter().for_each(|slot| {
5962                assert!(self
5963                    .remove_unrooted_slots_synchronization
5964                    .slots_under_contention
5965                    .lock()
5966                    .unwrap()
5967                    .remove(slot));
5968            });
5969
5970            // Signal to any threads blocked on `remove_unrooted_slots(slot)` that we have finished
5971            // flushing
5972            self.remove_unrooted_slots_synchronization
5973                .signal
5974                .notify_all();
5975            flush_stats
5976        } else {
5977            // We have already seen this slot. It is already under flushing. Skip.
5978            None
5979        }
5980    }
5981
5982    fn write_accounts_to_cache<'a>(
5983        &self,
5984        slot: Slot,
5985        hashes: Option<&[impl Borrow<Hash>]>,
5986        accounts_and_meta_to_store: &[(StoredMeta, Option<&impl ReadableAccount>)],
5987        txn_signatures_iter: Box<dyn std::iter::Iterator<Item = &Option<&Signature>> + 'a>,
5988    ) -> Vec<AccountInfo> {
5989        let len = accounts_and_meta_to_store.len();
5990        let hashes = hashes.map(|hashes| {
5991            assert_eq!(hashes.len(), len);
5992            hashes
5993        });
5994
5995        accounts_and_meta_to_store
5996            .iter()
5997            .zip(txn_signatures_iter)
5998            .enumerate()
5999            .map(|(i, ((meta, account), signature))| {
6000                let hash = hashes.map(|hashes| hashes[i].borrow());
6001
6002                let account = account
6003                    .map(|account| account.to_account_shared_data())
6004                    .unwrap_or_default();
6005                let account_info = AccountInfo::new(
6006                    StorageLocation::Cached,
6007                    CACHE_VIRTUAL_STORED_SIZE,
6008                    account.lamports(),
6009                );
6010
6011                self.notify_account_at_accounts_update(slot, meta, &account, signature);
6012
6013                let cached_account = self.accounts_cache.store(slot, &meta.pubkey, account, hash);
6014                // hash this account in the bg
6015                match &self.sender_bg_hasher {
6016                    Some(ref sender) => {
6017                        let _ = sender.send(cached_account);
6018                    }
6019                    None => (),
6020                };
6021                account_info
6022            })
6023            .collect()
6024    }
6025
6026    fn store_accounts_to<
6027        'a,
6028        F: FnMut(Slot, usize) -> Arc<AccountStorageEntry>,
6029        P: Iterator<Item = u64>,
6030        T: ReadableAccount + Sync + ZeroLamport,
6031    >(
6032        &self,
6033        accounts: &impl StorableAccounts<'a, T>,
6034        hashes: Option<&[impl Borrow<Hash>]>,
6035        storage_finder: F,
6036        mut write_version_producer: P,
6037        is_cached_store: bool,
6038        txn_signatures: Option<&'a [Option<&'a Signature>]>,
6039    ) -> Vec<AccountInfo> {
6040        let mut calc_stored_meta_time = Measure::start("calc_stored_meta");
6041        let slot = accounts.target_slot();
6042        let accounts_and_meta_to_store: Vec<_> = (0..accounts.len())
6043            .into_iter()
6044            .map(|index| {
6045                let (pubkey, account) = (accounts.pubkey(index), accounts.account(index));
6046                self.read_only_accounts_cache.remove(*pubkey, slot);
6047                // this is the source of Some(Account) or None.
6048                // Some(Account) = store 'Account'
6049                // None = store a default/empty account with 0 lamports
6050                let (account, data_len) = if account.is_zero_lamport() {
6051                    (None, 0)
6052                } else {
6053                    (Some(account), account.data().len() as u64)
6054                };
6055                let meta = StoredMeta {
6056                    write_version: write_version_producer.next().unwrap(),
6057                    pubkey: *pubkey,
6058                    data_len,
6059                };
6060                (meta, account)
6061            })
6062            .collect();
6063        calc_stored_meta_time.stop();
6064        self.stats
6065            .calc_stored_meta
6066            .fetch_add(calc_stored_meta_time.as_us(), Ordering::Relaxed);
6067
6068        if self.caching_enabled && is_cached_store {
6069            let signature_iter: Box<dyn std::iter::Iterator<Item = &Option<&Signature>>> =
6070                match txn_signatures {
6071                    Some(txn_signatures) => {
6072                        assert_eq!(txn_signatures.len(), accounts_and_meta_to_store.len());
6073                        Box::new(txn_signatures.iter())
6074                    }
6075                    None => {
6076                        Box::new(std::iter::repeat(&None).take(accounts_and_meta_to_store.len()))
6077                    }
6078                };
6079
6080            self.write_accounts_to_cache(slot, hashes, &accounts_and_meta_to_store, signature_iter)
6081        } else {
6082            match hashes {
6083                Some(hashes) => self.write_accounts_to_storage(
6084                    slot,
6085                    hashes,
6086                    storage_finder,
6087                    &accounts_and_meta_to_store,
6088                ),
6089                None => {
6090                    // hash any accounts where we were lazy in calculating the hash
6091                    let mut hash_time = Measure::start("hash_accounts");
6092                    let mut stats = BankHashStats::default();
6093                    let len = accounts_and_meta_to_store.len();
6094                    let mut hashes = Vec::with_capacity(len);
6095                    for index in 0..accounts.len() {
6096                        let (pubkey, account) = (accounts.pubkey(index), accounts.account(index));
6097                        stats.update(account);
6098                        let hash = Self::hash_account(slot, account, pubkey);
6099                        hashes.push(hash);
6100                    }
6101                    hash_time.stop();
6102                    self.stats
6103                        .store_hash_accounts
6104                        .fetch_add(hash_time.as_us(), Ordering::Relaxed);
6105
6106                    self.write_accounts_to_storage(
6107                        slot,
6108                        &hashes,
6109                        storage_finder,
6110                        &accounts_and_meta_to_store,
6111                    )
6112                }
6113            }
6114        }
6115    }
6116
6117    fn report_store_stats(&self) {
6118        let mut total_count = 0;
6119        let mut min = std::usize::MAX;
6120        let mut min_slot = 0;
6121        let mut max = 0;
6122        let mut max_slot = 0;
6123        let mut newest_slot = 0;
6124        let mut oldest_slot = std::u64::MAX;
6125        let mut total_bytes = 0;
6126        let mut total_alive_bytes = 0;
6127        for iter_item in self.storage.map.iter() {
6128            let slot = iter_item.key();
6129            let slot_stores = iter_item.value().read().unwrap();
6130            total_count += slot_stores.len();
6131            if slot_stores.len() < min {
6132                min = slot_stores.len();
6133                min_slot = *slot;
6134            }
6135
6136            if slot_stores.len() > max {
6137                max = slot_stores.len();
6138                max_slot = *slot;
6139            }
6140            if *slot > newest_slot {
6141                newest_slot = *slot;
6142            }
6143
6144            if *slot < oldest_slot {
6145                oldest_slot = *slot;
6146            }
6147
6148            for store in slot_stores.values() {
6149                total_alive_bytes += Self::page_align(store.alive_bytes() as u64);
6150                total_bytes += store.total_bytes();
6151            }
6152        }
6153        info!("total_stores: {}, newest_slot: {}, oldest_slot: {}, max_slot: {} (num={}), min_slot: {} (num={})",
6154              total_count, newest_slot, oldest_slot, max_slot, max, min_slot, min);
6155
6156        let total_alive_ratio = if total_bytes > 0 {
6157            total_alive_bytes as f64 / total_bytes as f64
6158        } else {
6159            0.
6160        };
6161
6162        datapoint_info!(
6163            "accounts_db-stores",
6164            ("total_count", total_count, i64),
6165            (
6166                "recycle_count",
6167                self.recycle_stores.read().unwrap().entry_count() as u64,
6168                i64
6169            ),
6170            ("total_bytes", total_bytes, i64),
6171            ("total_alive_bytes", total_alive_bytes, i64),
6172            ("total_alive_ratio", total_alive_ratio, f64),
6173        );
6174        datapoint_info!(
6175            "accounts_db-perf-stats",
6176            (
6177                "delta_hash_num",
6178                self.stats.delta_hash_num.swap(0, Ordering::Relaxed),
6179                i64
6180            ),
6181            (
6182                "delta_hash_scan_us",
6183                self.stats
6184                    .delta_hash_scan_time_total_us
6185                    .swap(0, Ordering::Relaxed),
6186                i64
6187            ),
6188            (
6189                "delta_hash_accumulate_us",
6190                self.stats
6191                    .delta_hash_accumulate_time_total_us
6192                    .swap(0, Ordering::Relaxed),
6193                i64
6194            ),
6195        );
6196    }
6197
6198    /// find slot >= 'slot' which is a root or in 'ancestors'
6199    pub fn find_unskipped_slot(&self, slot: Slot, ancestors: Option<&Ancestors>) -> Option<Slot> {
6200        self.accounts_index.get_next_original_root(slot, ancestors)
6201    }
6202
6203    pub fn checked_iterative_sum_for_capitalization(total_cap: u64, new_cap: u64) -> u64 {
6204        let new_total = total_cap as u128 + new_cap as u128;
6205        AccountsHash::checked_cast_for_capitalization(new_total)
6206    }
6207
6208    pub fn checked_sum_for_capitalization<T: Iterator<Item = u64>>(balances: T) -> u64 {
6209        AccountsHash::checked_cast_for_capitalization(balances.map(|b| b as u128).sum::<u128>())
6210    }
6211
6212    fn calculate_accounts_hash(
6213        &self,
6214        max_slot: Slot,
6215        config: &CalcAccountsHashConfig<'_>,
6216    ) -> Result<(Hash, u64), BankHashVerificationError> {
6217        use BankHashVerificationError::*;
6218        let mut collect = Measure::start("collect");
6219        let keys: Vec<_> = self
6220            .accounts_index
6221            .account_maps
6222            .iter()
6223            .flat_map(|map| {
6224                let mut keys = map.keys();
6225                keys.sort_unstable(); // hashmap is not ordered, but bins are relative to each other
6226                keys
6227            })
6228            .collect();
6229        collect.stop();
6230
6231        let mut scan = Measure::start("scan");
6232        let mismatch_found = AtomicU64::new(0);
6233        // Pick a chunk size big enough to allow us to produce output vectors that are smaller than the overall size.
6234        // We'll also accumulate the lamports within each chunk and fewer chunks results in less contention to accumulate the sum.
6235        let chunks = crate::accounts_hash::MERKLE_FANOUT.pow(4);
6236        let total_lamports = Mutex::<u64>::new(0);
6237        let stats = HashStats::default();
6238
6239        let get_hashes = || {
6240            keys.par_chunks(chunks)
6241                .map(|pubkeys| {
6242                    let mut sum = 0u128;
6243                    let result: Vec<Hash> = pubkeys
6244                        .iter()
6245                        .filter_map(|pubkey| {
6246                            if self.is_filler_account(pubkey) {
6247                                return None;
6248                            }
6249                            if let AccountIndexGetResult::Found(lock, index) =
6250                                self.accounts_index.get(pubkey, config.ancestors, Some(max_slot))
6251                            {
6252                                let (slot, account_info) = &lock.slot_list()[index];
6253                                if !account_info.is_zero_lamport() {
6254                                    // Because we're keeping the `lock' here, there is no need
6255                                    // to use retry_to_get_account_accessor()
6256                                    // In other words, flusher/shrinker/cleaner is blocked to
6257                                    // cause any Accessor(None) situtation.
6258                                    // Anyway this race condition concern is currently a moot
6259                                    // point because calculate_accounts_hash() should not
6260                                    // currently race with clean/shrink because the full hash
6261                                    // is synchronous with clean/shrink in
6262                                    // AccountsBackgroundService
6263                                    self.get_account_accessor(
6264                                        *slot,
6265                                        pubkey,
6266                                        &account_info.storage_location(),
6267                                    )
6268                                    .get_loaded_account()
6269                                    .and_then(
6270                                        |loaded_account| {
6271                                            let loaded_hash = loaded_account.loaded_hash();
6272                                            let balance = loaded_account.lamports();
6273                                            if config.check_hash && !self.is_filler_account(pubkey) {  // this will not be supported anymore
6274                                                let computed_hash =
6275                                                    loaded_account.compute_hash(*slot, pubkey);
6276                                                if computed_hash != loaded_hash {
6277                                                    info!("hash mismatch found: computed: {}, loaded: {}, pubkey: {}", computed_hash, loaded_hash, pubkey);
6278                                                    mismatch_found
6279                                                        .fetch_add(1, Ordering::Relaxed);
6280                                                    return None;
6281                                                }
6282                                            }
6283
6284                                            sum += balance as u128;
6285                                            Some(loaded_hash)
6286                                        },
6287                                    )
6288                                } else {
6289                                    None
6290                                }
6291                            } else {
6292                                None
6293                            }
6294                        })
6295                        .collect();
6296                    let mut total = total_lamports.lock().unwrap();
6297                    *total =
6298                        AccountsHash::checked_cast_for_capitalization(*total as u128 + sum);
6299                    result
6300                }).collect()
6301        };
6302
6303        let hashes: Vec<Vec<Hash>> = if config.check_hash {
6304            get_hashes()
6305        } else {
6306            self.thread_pool_clean.install(get_hashes)
6307        };
6308        if mismatch_found.load(Ordering::Relaxed) > 0 {
6309            warn!(
6310                "{} mismatched account hash(es) found",
6311                mismatch_found.load(Ordering::Relaxed)
6312            );
6313            return Err(MismatchedAccountHash);
6314        }
6315
6316        scan.stop();
6317        let total_lamports = *total_lamports.lock().unwrap();
6318
6319        let mut hash_time = Measure::start("hash");
6320        let (accumulated_hash, hash_total) = AccountsHash::calculate_hash(hashes);
6321        hash_time.stop();
6322        datapoint_info!(
6323            "update_accounts_hash",
6324            ("accounts_scan", scan.as_us(), i64),
6325            ("hash", hash_time.as_us(), i64),
6326            ("hash_total", hash_total, i64),
6327            ("collect", collect.as_us(), i64),
6328            (
6329                "rehashed_rewrites",
6330                stats.rehash_required.load(Ordering::Relaxed),
6331                i64
6332            ),
6333            (
6334                "rehashed_rewrites_unnecessary",
6335                stats.rehash_unnecessary.load(Ordering::Relaxed),
6336                i64
6337            ),
6338        );
6339        self.assert_safe_squashing_accounts_hash(max_slot, config.epoch_schedule);
6340
6341        Ok((accumulated_hash, total_lamports))
6342    }
6343
6344    pub fn get_accounts_hash(&self, slot: Slot) -> Hash {
6345        let bank_hashes = self.bank_hashes.read().unwrap();
6346        let bank_hash_info = bank_hashes.get(&slot).unwrap();
6347        bank_hash_info.snapshot_hash
6348    }
6349
6350    pub fn update_accounts_hash(
6351        &self,
6352        slot: Slot,
6353        ancestors: &Ancestors,
6354        epoch_schedule: &EpochSchedule,
6355        rent_collector: &RentCollector,
6356    ) -> (Hash, u64) {
6357        self.update_accounts_hash_with_index_option(
6358            true,
6359            false,
6360            slot,
6361            ancestors,
6362            None,
6363            false,
6364            epoch_schedule,
6365            rent_collector,
6366            false,
6367        )
6368    }
6369
6370    #[cfg(test)]
6371    fn update_accounts_hash_test(&self, slot: Slot, ancestors: &Ancestors) -> (Hash, u64) {
6372        self.update_accounts_hash_with_index_option(
6373            true,
6374            true,
6375            slot,
6376            ancestors,
6377            None,
6378            false,
6379            &EpochSchedule::default(),
6380            &RentCollector::default(),
6381            false,
6382        )
6383    }
6384
6385    fn scan_multiple_account_storages_one_slot<S>(
6386        storages: &[Arc<AccountStorageEntry>],
6387        scanner: &mut S,
6388    ) where
6389        S: AppendVecScan,
6390    {
6391        let mut len = storages.len();
6392        if len == 1 {
6393            // only 1 storage, so no need to interleave between multiple storages based on write_version
6394            storages[0].accounts.account_iter().for_each(|account| {
6395                if scanner.filter(&account.meta.pubkey) {
6396                    scanner.found_account(&LoadedAccount::Stored(account))
6397                }
6398            });
6399        } else {
6400            // we have to call the scan_func in order of write_version within a slot if there are multiple storages per slot
6401            let mut progress = Vec::with_capacity(len);
6402            let mut current =
6403                Vec::<(StoredMetaWriteVersion, Option<StoredAccountMeta<'_>>)>::with_capacity(len);
6404            for storage in storages {
6405                let mut iterator = storage.accounts.account_iter();
6406                if let Some(item) = iterator
6407                    .next()
6408                    .map(|stored_account| (stored_account.meta.write_version, Some(stored_account)))
6409                {
6410                    current.push(item);
6411                    progress.push(iterator);
6412                }
6413            }
6414            while !progress.is_empty() {
6415                let mut min = current[0].0;
6416                let mut min_index = 0;
6417                for (i, (item, _)) in current.iter().enumerate().take(len).skip(1) {
6418                    if item < &min {
6419                        min_index = i;
6420                        min = *item;
6421                    }
6422                }
6423                let found_account = &mut current[min_index];
6424                if scanner.filter(
6425                    &found_account
6426                        .1
6427                        .as_ref()
6428                        .map(|stored_account| stored_account.meta.pubkey)
6429                        .unwrap(), // will always be 'Some'
6430                ) {
6431                    let account = std::mem::take(found_account);
6432                    scanner.found_account(&LoadedAccount::Stored(account.1.unwrap()));
6433                }
6434                let next = progress[min_index].next().map(|stored_account| {
6435                    (stored_account.meta.write_version, Some(stored_account))
6436                });
6437                match next {
6438                    Some(item) => {
6439                        current[min_index] = item;
6440                    }
6441                    None => {
6442                        current.remove(min_index);
6443                        progress.remove(min_index);
6444                        len -= 1;
6445                    }
6446                }
6447            }
6448        }
6449    }
6450
6451    fn update_old_slot_stats(
6452        &self,
6453        stats: &HashStats,
6454        sub_storages: Option<&Vec<Arc<AccountStorageEntry>>>,
6455    ) {
6456        if let Some(sub_storages) = sub_storages {
6457            stats.roots_older_than_epoch.fetch_add(1, Ordering::Relaxed);
6458            let mut ancients = 0;
6459            let num_accounts = sub_storages
6460                .iter()
6461                .map(|storage| {
6462                    if is_ancient(&storage.accounts) {
6463                        ancients += 1;
6464                    }
6465                    storage.count()
6466                })
6467                .sum();
6468            let sizes = sub_storages
6469                .iter()
6470                .map(|storage| storage.total_bytes())
6471                .sum::<u64>();
6472            stats
6473                .append_vec_sizes_older_than_epoch
6474                .fetch_add(sizes as usize, Ordering::Relaxed);
6475            stats
6476                .accounts_in_roots_older_than_epoch
6477                .fetch_add(num_accounts, Ordering::Relaxed);
6478            stats
6479                .ancient_append_vecs
6480                .fetch_add(ancients, Ordering::Relaxed);
6481        }
6482    }
6483
6484    /// if ancient append vecs are enabled, return a slot one epoch old from 'max_slot_inclusive'
6485    /// otherwise, return 0
6486    fn get_one_epoch_old_slot_for_hash_calc_scan(
6487        &self,
6488        max_slot_inclusive: Slot,
6489        config: &CalcAccountsHashConfig<'_>,
6490    ) -> Slot {
6491        if self.ancient_append_vecs {
6492            // we are going to use a fixed slots per epoch here.
6493            // We are mainly interested in the network at steady state.
6494            let slots_in_epoch = config.epoch_schedule.slots_per_epoch;
6495            // For performance, this is required when ancient appendvecs are enabled
6496            max_slot_inclusive.saturating_sub(slots_in_epoch)
6497        } else {
6498            // This causes the entire range to be chunked together, treating older append vecs just like new ones.
6499            // This performs well if there are many old append vecs that haven't been cleaned yet.
6500            // 0 will have the effect of causing ALL older append vecs to be chunked together, just like every other append vec.
6501            0
6502        }
6503    }
6504
6505    /// Scan through all the account storage in parallel
6506    fn scan_account_storage_no_bank<S>(
6507        &self,
6508        cache_hash_data: &CacheHashData,
6509        config: &CalcAccountsHashConfig<'_>,
6510        snapshot_storages: &SortedStorages,
6511        scanner: S,
6512        bin_range: &Range<usize>,
6513        bin_calculator: &PubkeyBinCalculator24,
6514        stats: &HashStats,
6515    ) -> Vec<BinnedHashData>
6516    where
6517        S: AppendVecScan,
6518    {
6519        let start_bin_index = bin_range.start;
6520
6521        // any ancient append vecs should definitely be cached
6522        // We need to break the ranges into:
6523        // 1. individual ancient append vecs (may be empty)
6524        // 2. first unevenly divided chunk starting at 1 epoch old slot (may be empty)
6525        // 3. evenly divided full chunks in the middle
6526        // 4. unevenly divided chunk of most recent slots (may be empty)
6527        let max_slot_inclusive = snapshot_storages.max_slot_inclusive();
6528        let one_epoch_old_slot =
6529            self.get_one_epoch_old_slot_for_hash_calc_scan(max_slot_inclusive, config);
6530
6531        let range = snapshot_storages.range();
6532        let ancient_slots = snapshot_storages
6533            .iter_range(range.start..one_epoch_old_slot)
6534            .filter_map(|(slot, storages)| storages.map(|_| slot))
6535            .collect::<Vec<_>>();
6536        let ancient_slot_count = ancient_slots.len() as Slot;
6537        let slot0 = std::cmp::max(range.start, one_epoch_old_slot);
6538        let first_boundary =
6539            ((slot0 + MAX_ITEMS_PER_CHUNK) / MAX_ITEMS_PER_CHUNK) * MAX_ITEMS_PER_CHUNK;
6540
6541        let width = max_slot_inclusive - slot0;
6542        // 2 is for 2 special chunks - unaligned slots at the beginning and end
6543        let chunks = ancient_slot_count + 2 + (width as Slot / MAX_ITEMS_PER_CHUNK);
6544        (0..chunks)
6545            .into_par_iter()
6546            .map(|mut chunk| {
6547                let mut scanner = scanner.clone();
6548                // calculate start, end_exclusive
6549                let (single_cached_slot, (start, mut end_exclusive)) = if chunk < ancient_slot_count
6550                {
6551                    let ancient_slot = ancient_slots[chunk as usize];
6552                    (true, (ancient_slot, ancient_slot + 1))
6553                } else {
6554                    (false, {
6555                        chunk -= ancient_slot_count;
6556                        if chunk == 0 {
6557                            if slot0 == first_boundary {
6558                                return scanner.scanning_complete(); // if we evenly divide, nothing for special chunk 0 to do
6559                            }
6560                            // otherwise first chunk is not 'full'
6561                            (slot0, first_boundary)
6562                        } else {
6563                            // normal chunk in the middle or at the end
6564                            let start = first_boundary + MAX_ITEMS_PER_CHUNK * (chunk - 1);
6565                            let end_exclusive = start + MAX_ITEMS_PER_CHUNK;
6566                            (start, end_exclusive)
6567                        }
6568                    })
6569                };
6570                end_exclusive = std::cmp::min(end_exclusive, range.end);
6571                if start == end_exclusive {
6572                    return scanner.scanning_complete();
6573                }
6574
6575                let should_cache_hash_data = CalcAccountsHashConfig::get_should_cache_hash_data()
6576                    || config.store_detailed_debug_info_on_failure;
6577
6578                // if we're using the write cache, then we can't rely on cached append vecs since the append vecs may not include every account
6579                // Single cached slots get cached and full chunks get cached.
6580                // chunks that don't divide evenly would include some cached append vecs that are no longer part of this range and some that are, so we have to ignore caching on non-evenly dividing chunks.
6581                let eligible_for_caching = !config.use_write_cache
6582                    && (single_cached_slot
6583                        || end_exclusive.saturating_sub(start) == MAX_ITEMS_PER_CHUNK);
6584
6585                if eligible_for_caching || config.store_detailed_debug_info_on_failure {
6586                    let range = bin_range.end - bin_range.start;
6587                    scanner.init_accum(range);
6588                }
6589
6590                let slots_per_epoch = config
6591                    .rent_collector
6592                    .epoch_schedule
6593                    .get_slots_in_epoch(config.rent_collector.epoch);
6594                let one_epoch_old = snapshot_storages
6595                    .range()
6596                    .end
6597                    .saturating_sub(slots_per_epoch);
6598
6599                let mut file_name = String::default();
6600                // if we're using the write cache, we can't cache the hash calc results because not all accounts are in append vecs.
6601                if (should_cache_hash_data && eligible_for_caching)
6602                    || config.store_detailed_debug_info_on_failure
6603                {
6604                    let mut load_from_cache = true;
6605                    let mut hasher = std::collections::hash_map::DefaultHasher::new(); // wrong one?
6606
6607                    for (slot, sub_storages) in snapshot_storages.iter_range(start..end_exclusive) {
6608                        if bin_range.start == 0 && slot < one_epoch_old {
6609                            self.update_old_slot_stats(stats, sub_storages);
6610                        }
6611                        bin_range.start.hash(&mut hasher);
6612                        bin_range.end.hash(&mut hasher);
6613                        if let Some(sub_storages) = sub_storages {
6614                            if sub_storages.len() > 1 {
6615                                load_from_cache = false;
6616                                break;
6617                            }
6618                            let storage_file = sub_storages.first().unwrap().accounts.get_path();
6619                            slot.hash(&mut hasher);
6620                            storage_file.hash(&mut hasher);
6621                            // check alive_bytes, etc. here?
6622                            let amod = std::fs::metadata(storage_file);
6623                            if amod.is_err() {
6624                                load_from_cache = false;
6625                                break;
6626                            }
6627                            let amod = amod.unwrap().modified();
6628                            if amod.is_err() {
6629                                load_from_cache = false;
6630                                break;
6631                            }
6632                            let amod = amod
6633                                .unwrap()
6634                                .duration_since(std::time::UNIX_EPOCH)
6635                                .unwrap()
6636                                .as_secs();
6637                            amod.hash(&mut hasher);
6638                        }
6639                    }
6640                    if load_from_cache {
6641                        // we have a hash value for all the storages in this slot
6642                        // so, build a file name:
6643                        let hash = hasher.finish();
6644                        file_name = format!(
6645                            "{}.{}.{}.{}.{}",
6646                            start, end_exclusive, bin_range.start, bin_range.end, hash
6647                        );
6648                        let mut retval = scanner.get_accum();
6649                        if eligible_for_caching
6650                            && cache_hash_data
6651                                .load(
6652                                    &Path::new(&file_name),
6653                                    &mut retval,
6654                                    start_bin_index,
6655                                    bin_calculator,
6656                                )
6657                                .is_ok()
6658                        {
6659                            return retval;
6660                        }
6661                        scanner.set_accum(retval);
6662
6663                        // fall through and load normally - we failed to load
6664                    }
6665                } else {
6666                    for (slot, sub_storages) in snapshot_storages.iter_range(start..end_exclusive) {
6667                        if bin_range.start == 0 && slot < one_epoch_old {
6668                            self.update_old_slot_stats(stats, sub_storages);
6669                        }
6670                    }
6671                }
6672
6673                for (slot, sub_storages) in snapshot_storages.iter_range(start..end_exclusive) {
6674                    scanner.set_slot(slot);
6675                    let valid_slot = sub_storages.is_some();
6676                    if config.use_write_cache {
6677                        let ancestors = config.ancestors.as_ref().unwrap();
6678                        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
6679                            if valid_slot
6680                                || ancestors.contains_key(&slot)
6681                                || self.accounts_index.is_alive_root(slot)
6682                            {
6683                                let keys = slot_cache.get_all_pubkeys();
6684                                for key in keys {
6685                                    if scanner.filter(&key) {
6686                                        if let Some(cached_account) = slot_cache.get_cloned(&key) {
6687                                            let mut accessor = LoadedAccountAccessor::Cached(Some(
6688                                                Cow::Owned(cached_account),
6689                                            ));
6690                                            let account = accessor.get_loaded_account().unwrap();
6691                                            scanner.found_account(&account);
6692                                        }
6693                                    }
6694                                }
6695                            }
6696                        }
6697                    }
6698
6699                    if let Some(sub_storages) = sub_storages {
6700                        Self::scan_multiple_account_storages_one_slot(sub_storages, &mut scanner);
6701                    }
6702                }
6703                let r = scanner.scanning_complete();
6704                if !file_name.is_empty() {
6705                    let result = cache_hash_data.save(Path::new(&file_name), &r);
6706
6707                    if result.is_err() {
6708                        info!(
6709                            "FAILED_TO_SAVE: {}-{}, {}, first_boundary: {}, {:?}, error: {:?}",
6710                            range.start, range.end, width, first_boundary, file_name, result,
6711                        );
6712                    }
6713                }
6714                r
6715            })
6716            .filter(|x| !x.is_empty())
6717            .collect()
6718    }
6719
6720    /// storages are sorted by slot and have range info.
6721    /// add all stores older than slots_per_epoch to dirty_stores so clean visits these slots
6722    fn mark_old_slots_as_dirty(&self, storages: &SortedStorages, slots_per_epoch: Slot) {
6723        let max = storages.max_slot_inclusive();
6724        let acceptable_straggler_slot_count = 100; // do nothing special for these old stores which will likely get cleaned up shortly
6725        let sub = slots_per_epoch + acceptable_straggler_slot_count;
6726        let in_epoch_range_start = max.saturating_sub(sub);
6727        for (slot, storages) in storages.iter_range(..in_epoch_range_start) {
6728            if let Some(storages) = storages {
6729                storages.iter().for_each(|store| {
6730                    self.dirty_stores
6731                        .insert((slot, store.append_vec_id()), store.clone());
6732                });
6733            }
6734        }
6735    }
6736
6737    pub(crate) fn calculate_accounts_hash_helper(
6738        &self,
6739        use_index: bool,
6740        slot: Slot,
6741        config: &CalcAccountsHashConfig<'_>,
6742    ) -> Result<(Hash, u64), BankHashVerificationError> {
6743        if !use_index {
6744            let mut collect_time = Measure::start("collect");
6745            let (combined_maps, slots) = self.get_snapshot_storages(slot, None, config.ancestors);
6746            collect_time.stop();
6747
6748            let mut sort_time = Measure::start("sort_storages");
6749            let min_root = self.accounts_index.min_alive_root();
6750            let storages = SortedStorages::new_with_slots(
6751                combined_maps.iter().zip(slots.into_iter()),
6752                min_root,
6753                Some(slot),
6754            );
6755            sort_time.stop();
6756
6757            let mut timings = HashStats {
6758                collect_snapshots_us: collect_time.as_us(),
6759                storage_sort_us: sort_time.as_us(),
6760                ..HashStats::default()
6761            };
6762            timings.calc_storage_size_quartiles(&combined_maps);
6763
6764            self.calculate_accounts_hash_without_index(config, &storages, timings)
6765        } else {
6766            self.calculate_accounts_hash(slot, config)
6767        }
6768    }
6769
6770    #[allow(clippy::too_many_arguments)]
6771    fn calculate_accounts_hash_helper_with_verify(
6772        &self,
6773        use_index: bool,
6774        debug_verify: bool,
6775        slot: Slot,
6776        config: CalcAccountsHashConfig<'_>,
6777        expected_capitalization: Option<u64>,
6778    ) -> Result<(Hash, u64), BankHashVerificationError> {
6779        let (hash, total_lamports) =
6780            self.calculate_accounts_hash_helper(use_index, slot, &config)?;
6781        if debug_verify {
6782            // calculate the other way (store or non-store) and verify results match.
6783            let (hash_other, total_lamports_other) =
6784                self.calculate_accounts_hash_helper(!use_index, slot, &config)?;
6785
6786            let success = hash == hash_other
6787                && total_lamports == total_lamports_other
6788                && total_lamports == expected_capitalization.unwrap_or(total_lamports);
6789            assert!(success, "update_accounts_hash_with_index_option mismatch. hashes: {}, {}; lamports: {}, {}; expected lamports: {:?}, using index: {}, slot: {}", hash, hash_other, total_lamports, total_lamports_other, expected_capitalization, use_index, slot);
6790        }
6791        Ok((hash, total_lamports))
6792    }
6793
6794    #[allow(clippy::too_many_arguments)]
6795    pub fn update_accounts_hash_with_index_option(
6796        &self,
6797        use_index: bool,
6798        debug_verify: bool,
6799        slot: Slot,
6800        ancestors: &Ancestors,
6801        expected_capitalization: Option<u64>,
6802        can_cached_slot_be_unflushed: bool,
6803        epoch_schedule: &EpochSchedule,
6804        rent_collector: &RentCollector,
6805        is_startup: bool,
6806    ) -> (Hash, u64) {
6807        let check_hash = false;
6808        let (hash, total_lamports) = self
6809            .calculate_accounts_hash_helper_with_verify(
6810                use_index,
6811                debug_verify,
6812                slot,
6813                CalcAccountsHashConfig {
6814                    use_bg_thread_pool: !is_startup,
6815                    check_hash,
6816                    ancestors: Some(ancestors),
6817                    use_write_cache: can_cached_slot_be_unflushed,
6818                    epoch_schedule,
6819                    rent_collector,
6820                    store_detailed_debug_info_on_failure: false,
6821                    full_snapshot: None,
6822                },
6823                expected_capitalization,
6824            )
6825            .unwrap(); // unwrap here will never fail since check_hash = false
6826        self.set_accounts_hash(slot, hash);
6827        (hash, total_lamports)
6828    }
6829
6830    /// update hash for this slot in the 'bank_hashes' map
6831    pub(crate) fn set_accounts_hash(&self, slot: Slot, hash: Hash) {
6832        let mut bank_hashes = self.bank_hashes.write().unwrap();
6833        let mut bank_hash_info = bank_hashes.get_mut(&slot).unwrap();
6834        bank_hash_info.snapshot_hash = hash;
6835    }
6836
6837    fn scan_snapshot_stores_with_cache(
6838        &self,
6839        cache_hash_data: &CacheHashData,
6840        storage: &SortedStorages,
6841        mut stats: &mut crate::accounts_hash::HashStats,
6842        bins: usize,
6843        bin_range: &Range<usize>,
6844        config: &CalcAccountsHashConfig<'_>,
6845        filler_account_suffix: Option<&Pubkey>,
6846    ) -> Result<Vec<BinnedHashData>, BankHashVerificationError> {
6847        let bin_calculator = PubkeyBinCalculator24::new(bins);
6848        assert!(bin_range.start < bins && bin_range.end <= bins && bin_range.start < bin_range.end);
6849        let mut time = Measure::start("scan all accounts");
6850        stats.num_snapshot_storage = storage.storage_count();
6851        stats.num_slots = storage.slot_count();
6852        let mismatch_found = Arc::new(AtomicU64::new(0));
6853        let range = bin_range.end - bin_range.start;
6854        let sort_time = Arc::new(AtomicU64::new(0));
6855
6856        let scanner = ScanState {
6857            current_slot: Slot::default(),
6858            accum: BinnedHashData::default(),
6859            bin_calculator: &bin_calculator,
6860            config,
6861            mismatch_found: mismatch_found.clone(),
6862            filler_account_suffix,
6863            range,
6864            bin_range,
6865            sort_time: sort_time.clone(),
6866            pubkey_to_bin_index: 0,
6867        };
6868
6869        let result: Vec<BinnedHashData> = self.scan_account_storage_no_bank(
6870            cache_hash_data,
6871            config,
6872            storage,
6873            scanner,
6874            bin_range,
6875            &bin_calculator,
6876            stats,
6877        );
6878
6879        stats.sort_time_total_us += sort_time.load(Ordering::Relaxed);
6880
6881        if config.check_hash && mismatch_found.load(Ordering::Relaxed) > 0 {
6882            warn!(
6883                "{} mismatched account hash(es) found",
6884                mismatch_found.load(Ordering::Relaxed)
6885            );
6886            return Err(BankHashVerificationError::MismatchedAccountHash);
6887        }
6888
6889        time.stop();
6890        stats.scan_time_total_us += time.as_us();
6891
6892        Ok(result)
6893    }
6894
6895    fn sort_slot_storage_scan(accum: BinnedHashData) -> (BinnedHashData, u64) {
6896        let time = AtomicU64::new(0);
6897        (
6898            accum
6899                .into_iter()
6900                .map(|mut items| {
6901                    let mut sort_time = Measure::start("sort");
6902                    {
6903                        // sort_by vs unstable because slot and write_version are already in order
6904                        items.sort_by(AccountsHash::compare_two_hash_entries);
6905                    }
6906                    sort_time.stop();
6907                    time.fetch_add(sort_time.as_us(), Ordering::Relaxed);
6908                    items
6909                })
6910                .collect(),
6911            time.load(Ordering::Relaxed),
6912        )
6913    }
6914
6915    /// if we ever try to calc hash where there are squashed append vecs within the last epoch, we will fail
6916    fn assert_safe_squashing_accounts_hash(&self, slot: Slot, epoch_schedule: &EpochSchedule) {
6917        let previous = self.get_accounts_hash_complete_one_epoch_old();
6918        let current = Self::get_slot_one_epoch_prior(slot, epoch_schedule);
6919        assert!(
6920            previous <= current,
6921            "get_accounts_hash_complete_one_epoch_old: {}, get_slot_one_epoch_prior: {}, slot: {}",
6922            previous,
6923            current,
6924            slot
6925        );
6926    }
6927
6928    /// normal code path returns the common cache path
6929    /// when called after a failure has been detected, redirect the cache storage to a separate folder for debugging later
6930    fn get_cache_hash_data(&self, config: &CalcAccountsHashConfig<'_>) -> CacheHashData {
6931        if !config.store_detailed_debug_info_on_failure {
6932            CacheHashData::new(&self.accounts_hash_cache_path)
6933        } else {
6934            // this path executes when we are failing with a hash mismatch
6935            let mut new = self.accounts_hash_cache_path.clone();
6936            new.push("failed_calculate_accounts_hash_cache");
6937            let _ = std::fs::remove_dir_all(&new);
6938            CacheHashData::new(&new)
6939        }
6940    }
6941
6942    // modeled after get_accounts_delta_hash
6943    // intended to be faster than calculate_accounts_hash
6944    pub fn calculate_accounts_hash_without_index(
6945        &self,
6946        config: &CalcAccountsHashConfig<'_>,
6947        storages: &SortedStorages<'_>,
6948        mut stats: HashStats,
6949    ) -> Result<(Hash, u64), BankHashVerificationError> {
6950        let _guard = self.active_stats.activate(ActiveStatItem::Hash);
6951
6952        stats.oldest_root = storages.range().start;
6953
6954        self.mark_old_slots_as_dirty(storages, config.epoch_schedule.slots_per_epoch);
6955
6956        let (num_hash_scan_passes, bins_per_pass) = Self::bins_per_pass(self.num_hash_scan_passes);
6957        let use_bg_thread_pool = config.use_bg_thread_pool;
6958        let mut scan_and_hash = move || {
6959            let mut previous_pass = PreviousPass::default();
6960            let mut final_result = (Hash::default(), 0);
6961
6962            let cache_hash_data = self.get_cache_hash_data(config);
6963
6964            for pass in 0..num_hash_scan_passes {
6965                let bounds = Range {
6966                    start: pass * bins_per_pass,
6967                    end: (pass + 1) * bins_per_pass,
6968                };
6969
6970                let hash = AccountsHash {
6971                    filler_account_suffix: if self.filler_accounts_config.count > 0 {
6972                        self.filler_account_suffix
6973                    } else {
6974                        None
6975                    },
6976                };
6977
6978                let result = self.scan_snapshot_stores_with_cache(
6979                    &cache_hash_data,
6980                    storages,
6981                    &mut stats,
6982                    PUBKEY_BINS_FOR_CALCULATING_HASHES,
6983                    &bounds,
6984                    config,
6985                    hash.filler_account_suffix.as_ref(),
6986                )?;
6987
6988                let (hash, lamports, for_next_pass) = hash.rest_of_hash_calculation(
6989                    result,
6990                    &mut stats,
6991                    pass == num_hash_scan_passes - 1,
6992                    previous_pass,
6993                    bins_per_pass,
6994                );
6995                previous_pass = for_next_pass;
6996                final_result = (hash, lamports);
6997            }
6998
6999            info!(
7000                "calculate_accounts_hash_without_index: slot: {} {:?}",
7001                storages.max_slot_inclusive(),
7002                final_result
7003            );
7004            Ok(final_result)
7005        };
7006        let result = if use_bg_thread_pool {
7007            self.thread_pool_clean.install(scan_and_hash)
7008        } else {
7009            scan_and_hash()
7010        };
7011        self.assert_safe_squashing_accounts_hash(
7012            storages.max_slot_inclusive(),
7013            config.epoch_schedule,
7014        );
7015        result
7016    }
7017
7018    /// return alive roots to retain, even though they are ancient
7019    fn calc_alive_ancient_historical_roots(&self, min_root: Slot) -> HashSet<Slot> {
7020        let mut ancient_alive_roots = HashSet::default();
7021        {
7022            let all_roots = self.accounts_index.roots_tracker.read().unwrap();
7023
7024            if let Some(min) = all_roots.historical_roots.min() {
7025                for slot in min..min_root {
7026                    if all_roots.alive_roots.contains(&slot) {
7027                        // there was a storage for this root, so it counts as a root
7028                        ancient_alive_roots.insert(slot);
7029                    }
7030                }
7031            }
7032        }
7033        ancient_alive_roots
7034    }
7035
7036    /// get rid of historical roots that are older than 'min_root'.
7037    /// These will be older than an epoch from a current root.
7038    fn remove_old_historical_roots(&self, min_root: Slot) {
7039        let alive_roots = self.calc_alive_ancient_historical_roots(min_root);
7040        self.accounts_index
7041            .remove_old_historical_roots(min_root, &alive_roots);
7042    }
7043
7044    /// Only called from startup or test code.
7045    pub fn verify_bank_hash_and_lamports(
7046        &self,
7047        slot: Slot,
7048        ancestors: &Ancestors,
7049        total_lamports: u64,
7050        test_hash_calculation: bool,
7051        epoch_schedule: &EpochSchedule,
7052        rent_collector: &RentCollector,
7053        can_cached_slot_be_unflushed: bool,
7054        use_bg_thread_pool: bool,
7055    ) -> Result<(), BankHashVerificationError> {
7056        self.verify_bank_hash_and_lamports_new(
7057            slot,
7058            ancestors,
7059            total_lamports,
7060            test_hash_calculation,
7061            epoch_schedule,
7062            rent_collector,
7063            can_cached_slot_be_unflushed,
7064            false,
7065            false,
7066            use_bg_thread_pool,
7067        )
7068    }
7069
7070    /// Only called from startup or test code.
7071    #[allow(clippy::too_many_arguments)]
7072    pub fn verify_bank_hash_and_lamports_new(
7073        &self,
7074        slot: Slot,
7075        ancestors: &Ancestors,
7076        total_lamports: u64,
7077        test_hash_calculation: bool,
7078        epoch_schedule: &EpochSchedule,
7079        rent_collector: &RentCollector,
7080        can_cached_slot_be_unflushed: bool,
7081        ignore_mismatch: bool,
7082        store_hash_raw_data_for_debug: bool,
7083        use_bg_thread_pool: bool,
7084    ) -> Result<(), BankHashVerificationError> {
7085        use BankHashVerificationError::*;
7086
7087        let use_index = false;
7088        let check_hash = false; // this will not be supported anymore
7089        let (calculated_hash, calculated_lamports) = self
7090            .calculate_accounts_hash_helper_with_verify(
7091                use_index,
7092                test_hash_calculation,
7093                slot,
7094                CalcAccountsHashConfig {
7095                    use_bg_thread_pool,
7096                    check_hash,
7097                    ancestors: Some(ancestors),
7098                    use_write_cache: can_cached_slot_be_unflushed,
7099                    epoch_schedule,
7100                    rent_collector,
7101                    store_detailed_debug_info_on_failure: store_hash_raw_data_for_debug,
7102                    full_snapshot: None,
7103                },
7104                None,
7105            )?;
7106
7107        if calculated_lamports != total_lamports {
7108            warn!(
7109                "Mismatched total lamports: {} calculated: {}",
7110                total_lamports, calculated_lamports
7111            );
7112            return Err(MismatchedTotalLamports(calculated_lamports, total_lamports));
7113        }
7114
7115        if ignore_mismatch {
7116            Ok(())
7117        } else {
7118            let bank_hashes = self.bank_hashes.read().unwrap();
7119            if let Some(found_hash_info) = bank_hashes.get(&slot) {
7120                if calculated_hash == found_hash_info.snapshot_hash {
7121                    Ok(())
7122                } else {
7123                    warn!(
7124                        "mismatched bank hash for slot {}: {} (calculated) != {} (expected)",
7125                        slot, calculated_hash, found_hash_info.snapshot_hash
7126                    );
7127                    Err(MismatchedBankHash)
7128                }
7129            } else {
7130                Err(MissingBankHash)
7131            }
7132        }
7133    }
7134
7135    /// Perform the scan for pubkeys that were written to in a slot
7136    fn do_scan_slot_for_dirty_pubkeys(
7137        &self,
7138        slot: Slot,
7139    ) -> ScanStorageResult<Pubkey, DashSet<Pubkey>> {
7140        self.scan_account_storage(
7141            slot,
7142            |loaded_account: LoadedAccount| Some(*loaded_account.pubkey()),
7143            |accum: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
7144                accum.insert(*loaded_account.pubkey());
7145            },
7146        )
7147    }
7148
7149    /// Reduce the scan result of dirty pubkeys after calling `scan_account_storage()` into a
7150    /// single vec of Pubkeys.
7151    fn do_reduce_scan_slot_for_dirty_pubkeys(
7152        scan_result: ScanStorageResult<Pubkey, DashSet<Pubkey>>,
7153    ) -> Vec<Pubkey> {
7154        match scan_result {
7155            ScanStorageResult::Cached(cached_result) => cached_result,
7156            ScanStorageResult::Stored(stored_result) => {
7157                stored_result.into_iter().collect::<Vec<_>>()
7158            }
7159        }
7160    }
7161
7162    /// Scan a slot for dirty pubkeys
7163    fn scan_slot_for_dirty_pubkeys(&self, slot: Slot) -> Vec<Pubkey> {
7164        let dirty_pubkeys = self.do_scan_slot_for_dirty_pubkeys(slot);
7165        Self::do_reduce_scan_slot_for_dirty_pubkeys(dirty_pubkeys)
7166    }
7167
7168    /// Scan a slot in the account storage for dirty pubkeys and insert them into the list of
7169    /// uncleaned pubkeys
7170    ///
7171    /// This function is called in Bank::drop() when the bank is _not_ frozen, so that its pubkeys
7172    /// are considered for cleanup.
7173    pub fn scan_slot_and_insert_dirty_pubkeys_into_uncleaned_pubkeys(&self, slot: Slot) {
7174        let dirty_pubkeys = self.scan_slot_for_dirty_pubkeys(slot);
7175        self.uncleaned_pubkeys.insert(slot, dirty_pubkeys);
7176    }
7177
7178    pub fn get_accounts_delta_hash(&self, slot: Slot) -> Hash {
7179        self.get_accounts_delta_hash_with_rewrites(slot, &Rewrites::default())
7180    }
7181
7182    /// helper to return
7183    /// 1. pubkey, hash pairs for the slot
7184    /// 2. us spent scanning
7185    /// 3. Measure started when we began accumulating
7186    fn get_pubkey_hash_for_slot(&self, slot: Slot) -> (Vec<(Pubkey, Hash)>, u64, Measure) {
7187        let mut scan = Measure::start("scan");
7188
7189        let scan_result: ScanStorageResult<(Pubkey, Hash), DashMapVersionHash> = self
7190            .scan_account_storage(
7191                slot,
7192                |loaded_account: LoadedAccount| {
7193                    // Cache only has one version per key, don't need to worry about versioning
7194                    Some((*loaded_account.pubkey(), loaded_account.loaded_hash()))
7195                },
7196                |accum: &DashMap<Pubkey, (u64, Hash)>, loaded_account: LoadedAccount| {
7197                    let loaded_write_version = loaded_account.write_version();
7198                    let loaded_hash = loaded_account.loaded_hash();
7199                    // keep the latest write version for each pubkey
7200                    match accum.entry(*loaded_account.pubkey()) {
7201                        Occupied(mut occupied_entry) => {
7202                            if loaded_write_version > occupied_entry.get().version() {
7203                                occupied_entry.insert((loaded_write_version, loaded_hash));
7204                            }
7205                        }
7206
7207                        Vacant(vacant_entry) => {
7208                            vacant_entry.insert((loaded_write_version, loaded_hash));
7209                        }
7210                    }
7211                },
7212            );
7213        scan.stop();
7214
7215        let accumulate = Measure::start("accumulate");
7216        let hashes: Vec<_> = match scan_result {
7217            ScanStorageResult::Cached(cached_result) => cached_result,
7218            ScanStorageResult::Stored(stored_result) => stored_result
7219                .into_iter()
7220                .map(|(pubkey, (_latest_write_version, hash))| (pubkey, hash))
7221                .collect(),
7222        };
7223        (hashes, scan.as_us(), accumulate)
7224    }
7225
7226    pub fn get_accounts_delta_hash_with_rewrites(
7227        &self,
7228        slot: Slot,
7229        skipped_rewrites: &Rewrites,
7230    ) -> Hash {
7231        let (mut hashes, scan_us, mut accumulate) = self.get_pubkey_hash_for_slot(slot);
7232        let dirty_keys = hashes.iter().map(|(pubkey, _hash)| *pubkey).collect();
7233
7234        if self.filler_accounts_enabled() {
7235            // filler accounts must be added to 'dirty_keys' above but cannot be used to calculate hash
7236            hashes.retain(|(pubkey, _hash)| !self.is_filler_account(pubkey));
7237        }
7238
7239        self.extend_hashes_with_skipped_rewrites(&mut hashes, skipped_rewrites);
7240
7241        let ret = AccountsHash::accumulate_account_hashes(hashes);
7242        accumulate.stop();
7243        let mut uncleaned_time = Measure::start("uncleaned_index");
7244        self.uncleaned_pubkeys.insert(slot, dirty_keys);
7245        uncleaned_time.stop();
7246        self.stats
7247            .store_uncleaned_update
7248            .fetch_add(uncleaned_time.as_us(), Ordering::Relaxed);
7249
7250        self.stats
7251            .delta_hash_scan_time_total_us
7252            .fetch_add(scan_us, Ordering::Relaxed);
7253        self.stats
7254            .delta_hash_accumulate_time_total_us
7255            .fetch_add(accumulate.as_us(), Ordering::Relaxed);
7256        self.stats.delta_hash_num.fetch_add(1, Ordering::Relaxed);
7257        ret
7258    }
7259
7260    /// add all items from 'skipped_rewrites' to 'hashes' where the pubkey doesn't already exist in 'hashes'
7261    fn extend_hashes_with_skipped_rewrites(
7262        &self,
7263        hashes: &mut Vec<(Pubkey, Hash)>,
7264        skipped_rewrites: &Rewrites,
7265    ) {
7266        let mut skipped_rewrites = skipped_rewrites.read().unwrap().clone();
7267        hashes.iter().for_each(|(key, _)| {
7268            skipped_rewrites.remove(key);
7269        });
7270
7271        if self.filler_accounts_enabled() {
7272            // simulate the time we would normally spend hashing the filler accounts
7273            // this is an over approximation but at least takes a stab at simulating what the validator would spend time doing
7274            let _ = AccountsHash::accumulate_account_hashes(
7275                skipped_rewrites
7276                    .iter()
7277                    .map(|(k, v)| (*k, *v))
7278                    .collect::<Vec<_>>(),
7279            );
7280
7281            // filler accounts do not get their updated hash values hashed into the delta hash
7282            skipped_rewrites.retain(|key, _| !self.is_filler_account(key));
7283        }
7284
7285        hashes.extend(skipped_rewrites.into_iter());
7286    }
7287
7288    // previous_slot_entry_was_cached = true means we just need to assert that after this update is complete
7289    //  that there are no items we would have put in reclaims that are not cached
7290    fn update_index<'a, T: ReadableAccount + Sync>(
7291        &self,
7292        infos: Vec<AccountInfo>,
7293        accounts: impl StorableAccounts<'a, T>,
7294        reclaim: UpsertReclaim,
7295    ) -> SlotList<AccountInfo> {
7296        let target_slot = accounts.target_slot();
7297        // using a thread pool here results in deadlock panics from bank_hashes.write()
7298        // so, instead we limit how many threads will be created to the same size as the bg thread pool
7299        let len = std::cmp::min(accounts.len(), infos.len());
7300        let threshold = 1;
7301        let update = |start, end| {
7302            let mut reclaims = Vec::with_capacity((end - start) / 2);
7303
7304            (start..end).into_iter().for_each(|i| {
7305                let info = infos[i];
7306                let pubkey_account = (accounts.pubkey(i), accounts.account(i));
7307                let pubkey = pubkey_account.0;
7308                let old_slot = accounts.slot(i);
7309                self.accounts_index.upsert(
7310                    target_slot,
7311                    old_slot,
7312                    pubkey,
7313                    pubkey_account.1,
7314                    &self.account_indexes,
7315                    info,
7316                    &mut reclaims,
7317                    reclaim,
7318                );
7319            });
7320            reclaims
7321        };
7322        if len > threshold {
7323            let chunk_size = std::cmp::max(1, len / quarter_thread_count()); // # pubkeys/thread
7324            let batches = 1 + len / chunk_size;
7325            (0..batches)
7326                .into_par_iter()
7327                .map(|batch| {
7328                    let start = batch * chunk_size;
7329                    let end = std::cmp::min(start + chunk_size, len);
7330                    update(start, end)
7331                })
7332                .flatten()
7333                .collect::<Vec<_>>()
7334        } else {
7335            update(0, len)
7336        }
7337    }
7338
7339    fn should_not_shrink(aligned_bytes: u64, total_bytes: u64, num_stores: usize) -> bool {
7340        aligned_bytes + PAGE_SIZE > total_bytes && num_stores == 1
7341    }
7342
7343    fn is_shrinking_productive<'a, I>(slot: Slot, stores: I) -> bool
7344    where
7345        I: IntoIterator<Item = &'a Arc<AccountStorageEntry>>,
7346    {
7347        let mut alive_count = 0;
7348        let mut stored_count = 0;
7349        let mut alive_bytes = 0;
7350        let mut total_bytes = 0;
7351        let mut count = 0;
7352
7353        for store in stores {
7354            count += 1;
7355            alive_count += store.count();
7356            stored_count += store.approx_stored_count();
7357            alive_bytes += store.alive_bytes();
7358            total_bytes += store.total_bytes();
7359        }
7360
7361        let aligned_bytes = Self::page_align(alive_bytes as u64);
7362        if Self::should_not_shrink(aligned_bytes, total_bytes, count) {
7363            trace!(
7364                "shrink_slot_forced ({}, {}): not able to shrink at all: alive/stored: ({} / {}) ({}b / {}b) save: {}",
7365                slot,
7366                count,
7367                alive_count,
7368                stored_count,
7369                aligned_bytes,
7370                total_bytes,
7371                total_bytes.saturating_sub(aligned_bytes),
7372            );
7373            return false;
7374        }
7375
7376        true
7377    }
7378
7379    fn is_candidate_for_shrink(
7380        &self,
7381        store: &Arc<AccountStorageEntry>,
7382        allow_shrink_ancient: bool,
7383    ) -> bool {
7384        let total_bytes = if is_ancient(&store.accounts) {
7385            if !allow_shrink_ancient {
7386                return false;
7387            }
7388
7389            store.written_bytes()
7390        } else {
7391            store.total_bytes()
7392        };
7393        match self.shrink_ratio {
7394            AccountShrinkThreshold::TotalSpace { shrink_ratio: _ } => {
7395                Self::page_align(store.alive_bytes() as u64) < total_bytes
7396            }
7397            AccountShrinkThreshold::IndividualStore { shrink_ratio } => {
7398                (Self::page_align(store.alive_bytes() as u64) as f64 / total_bytes as f64)
7399                    < shrink_ratio
7400            }
7401        }
7402    }
7403
7404    fn remove_dead_accounts<'a, I>(
7405        &'a self,
7406        reclaims: I,
7407        expected_slot: Option<Slot>,
7408        mut reclaimed_offsets: Option<&mut AppendVecOffsets>,
7409        reset_accounts: bool,
7410    ) -> HashSet<Slot>
7411    where
7412        I: Iterator<Item = &'a (Slot, AccountInfo)>,
7413    {
7414        let mut dead_slots = HashSet::new();
7415        let mut new_shrink_candidates: ShrinkCandidates = HashMap::new();
7416        let mut measure = Measure::start("remove");
7417        for (slot, account_info) in reclaims {
7418            // No cached accounts should make it here
7419            assert!(!account_info.is_cached());
7420            if let Some(ref mut reclaimed_offsets) = reclaimed_offsets {
7421                reclaimed_offsets
7422                    .entry(account_info.store_id())
7423                    .or_default()
7424                    .insert(account_info.offset());
7425            }
7426            if let Some(expected_slot) = expected_slot {
7427                // not sure why this fails yet with ancient append vecs
7428                if !self.ancient_append_vecs {
7429                    assert_eq!(*slot, expected_slot);
7430                }
7431            }
7432            if let Some(store) = self
7433                .storage
7434                .get_account_storage_entry(*slot, account_info.store_id())
7435            {
7436                assert_eq!(
7437                    *slot, store.slot(),
7438                    "AccountsDB::accounts_index corrupted. Storage pointed to: {}, expected: {}, should only point to one slot",
7439                    store.slot(), *slot
7440                );
7441                let count =
7442                    store.remove_account(account_info.stored_size() as usize, reset_accounts);
7443                if count == 0 {
7444                    self.dirty_stores
7445                        .insert((*slot, store.append_vec_id()), store.clone());
7446                    dead_slots.insert(*slot);
7447                } else if self.caching_enabled
7448                    && Self::is_shrinking_productive(*slot, [&store].into_iter())
7449                    && self.is_candidate_for_shrink(&store, false)
7450                {
7451                    // Checking that this single storage entry is ready for shrinking,
7452                    // should be a sufficient indication that the slot is ready to be shrunk
7453                    // because slots should only have one storage entry, namely the one that was
7454                    // created by `flush_slot_cache()`.
7455                    {
7456                        new_shrink_candidates
7457                            .entry(*slot)
7458                            .or_default()
7459                            .insert(store.append_vec_id(), store);
7460                    }
7461                }
7462            }
7463        }
7464        measure.stop();
7465        self.clean_accounts_stats
7466            .remove_dead_accounts_remove_us
7467            .fetch_add(measure.as_us(), Ordering::Relaxed);
7468
7469        if self.caching_enabled {
7470            let mut measure = Measure::start("shrink");
7471            let mut shrink_candidate_slots = self.shrink_candidate_slots.lock().unwrap();
7472            for (slot, slot_shrink_candidates) in new_shrink_candidates {
7473                for (store_id, store) in slot_shrink_candidates {
7474                    // count could be == 0 if multiple accounts are removed
7475                    // at once
7476                    if store.count() != 0 {
7477                        debug!(
7478                            "adding: {} {} to shrink candidates: count: {}/{} bytes: {}/{}",
7479                            store_id,
7480                            slot,
7481                            store.approx_stored_count(),
7482                            store.count(),
7483                            store.alive_bytes(),
7484                            store.total_bytes()
7485                        );
7486
7487                        shrink_candidate_slots
7488                            .entry(slot)
7489                            .or_default()
7490                            .insert(store_id, store);
7491                    }
7492                }
7493            }
7494            measure.stop();
7495            self.clean_accounts_stats
7496                .remove_dead_accounts_shrink_us
7497                .fetch_add(measure.as_us(), Ordering::Relaxed);
7498        }
7499
7500        dead_slots.retain(|slot| {
7501            if let Some(slot_stores) = self.storage.get_slot_stores(*slot) {
7502                for x in slot_stores.read().unwrap().values() {
7503                    if x.count() != 0 {
7504                        return false;
7505                    }
7506                }
7507            }
7508            true
7509        });
7510
7511        dead_slots
7512    }
7513
7514    // pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7515    //    and should not be unref'd. If they exist in the accounts index, they are NEW.
7516    fn remove_dead_slots_metadata<'a>(
7517        &'a self,
7518        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
7519        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7520        // Should only be `Some` for non-cached slots
7521        purged_stored_account_slots: Option<&mut AccountSlots>,
7522        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7523    ) {
7524        let mut measure = Measure::start("remove_dead_slots_metadata-ms");
7525        self.clean_dead_slots_from_accounts_index(
7526            dead_slots_iter.clone(),
7527            purged_slot_pubkeys,
7528            purged_stored_account_slots,
7529            pubkeys_removed_from_accounts_index,
7530        );
7531        {
7532            let mut bank_hashes = self.bank_hashes.write().unwrap();
7533            for slot in dead_slots_iter {
7534                bank_hashes.remove(slot);
7535            }
7536        }
7537        measure.stop();
7538        inc_new_counter_info!("remove_dead_slots_metadata-ms", measure.as_ms() as usize);
7539    }
7540
7541    fn clean_dead_slots_from_accounts_index<'a>(
7542        &'a self,
7543        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
7544        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
7545        // Should only be `Some` for non-cached slots
7546        purged_stored_account_slots: Option<&mut AccountSlots>,
7547        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7548    ) {
7549        let mut accounts_index_root_stats = AccountsIndexRootsStats::default();
7550        let mut measure = Measure::start("unref_from_storage");
7551        if let Some(purged_stored_account_slots) = purged_stored_account_slots {
7552            let len = purged_slot_pubkeys.len();
7553            const BATCH_SIZE: usize = 10_000;
7554            let batches = 1 + (len / BATCH_SIZE);
7555            self.thread_pool_clean.install(|| {
7556                (0..batches).into_par_iter().for_each(|batch| {
7557                    let skip = batch * BATCH_SIZE;
7558                    self.accounts_index.scan(
7559                        purged_slot_pubkeys
7560                            .iter()
7561                            .skip(skip)
7562                            .take(BATCH_SIZE)
7563                            .filter_map(|(_slot, pubkey)| {
7564                                // filter out pubkeys that have already been removed from the accounts index in a previous step
7565                                let already_removed =
7566                                    pubkeys_removed_from_accounts_index.contains(pubkey);
7567                                (!already_removed).then(|| pubkey)
7568                            }),
7569                        |_pubkey, _slots_refs| AccountsIndexScanResult::Unref,
7570                    )
7571                })
7572            });
7573            for (slot, pubkey) in purged_slot_pubkeys {
7574                purged_stored_account_slots
7575                    .entry(pubkey)
7576                    .or_default()
7577                    .insert(slot);
7578            }
7579        }
7580        measure.stop();
7581        accounts_index_root_stats.clean_unref_from_storage_us += measure.as_us();
7582
7583        let mut measure = Measure::start("clean_dead_slot");
7584        let mut rooted_cleaned_count = 0;
7585        let mut unrooted_cleaned_count = 0;
7586        let dead_slots: Vec<_> = dead_slots_iter
7587            .map(|slot| {
7588                if self
7589                    .accounts_index
7590                    .clean_dead_slot(*slot, &mut accounts_index_root_stats)
7591                {
7592                    rooted_cleaned_count += 1;
7593                } else {
7594                    unrooted_cleaned_count += 1;
7595                }
7596                *slot
7597            })
7598            .collect();
7599        measure.stop();
7600        accounts_index_root_stats.clean_dead_slot_us += measure.as_us();
7601        if self.log_dead_slots.load(Ordering::Relaxed) {
7602            info!(
7603                "remove_dead_slots_metadata: {} dead slots",
7604                dead_slots.len()
7605            );
7606            trace!("remove_dead_slots_metadata: dead_slots: {:?}", dead_slots);
7607        }
7608
7609        accounts_index_root_stats.rooted_cleaned_count += rooted_cleaned_count;
7610        accounts_index_root_stats.unrooted_cleaned_count += unrooted_cleaned_count;
7611
7612        self.clean_accounts_stats
7613            .latest_accounts_index_roots_stats
7614            .update(&accounts_index_root_stats);
7615    }
7616
7617    // pubkeys_removed_from_accounts_index - These keys have already been removed from the accounts index
7618    //    and should not be unref'd. If they exist in the accounts index, they are NEW.
7619    fn clean_stored_dead_slots(
7620        &self,
7621        dead_slots: &HashSet<Slot>,
7622        purged_account_slots: Option<&mut AccountSlots>,
7623        pubkeys_removed_from_accounts_index: &PubkeysRemovedFromAccountsIndex,
7624    ) {
7625        let mut measure = Measure::start("clean_stored_dead_slots-ms");
7626        let mut stores: Vec<Arc<AccountStorageEntry>> = vec![];
7627        for slot in dead_slots.iter() {
7628            if let Some(slot_storage) = self.storage.get_slot_stores(*slot) {
7629                for store in slot_storage.read().unwrap().values() {
7630                    stores.push(store.clone());
7631                }
7632            }
7633        }
7634        let purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = {
7635            self.thread_pool_clean.install(|| {
7636                stores
7637                    .into_par_iter()
7638                    .map(|store| {
7639                        let accounts = store.all_accounts();
7640                        let slot = store.slot();
7641                        accounts
7642                            .into_iter()
7643                            .map(|account| (slot, account.meta.pubkey))
7644                            .collect::<HashSet<(Slot, Pubkey)>>()
7645                    })
7646                    .reduce(HashSet::new, |mut reduced, store_pubkeys| {
7647                        reduced.extend(store_pubkeys);
7648                        reduced
7649                    })
7650            })
7651        };
7652        self.remove_dead_slots_metadata(
7653            dead_slots.iter(),
7654            purged_slot_pubkeys,
7655            purged_account_slots,
7656            pubkeys_removed_from_accounts_index,
7657        );
7658        measure.stop();
7659        inc_new_counter_info!("clean_stored_dead_slots-ms", measure.as_ms() as usize);
7660        self.clean_accounts_stats
7661            .clean_stored_dead_slots_us
7662            .fetch_add(measure.as_us(), Ordering::Relaxed);
7663    }
7664
7665    pub fn store_cached<'a, T: ReadableAccount + Sync + ZeroLamport>(
7666        &self,
7667        accounts: impl StorableAccounts<'a, T>,
7668        txn_signatures: Option<&'a [Option<&'a Signature>]>,
7669    ) {
7670        self.store(
7671            accounts,
7672            self.caching_enabled,
7673            txn_signatures,
7674            StoreReclaims::Default,
7675        );
7676    }
7677
7678    /// Store the account update.
7679    /// only called by tests
7680    pub fn store_uncached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
7681        self.store((slot, accounts), false, None, StoreReclaims::Default);
7682    }
7683
7684    fn store<'a, T: ReadableAccount + Sync + ZeroLamport>(
7685        &self,
7686        accounts: impl StorableAccounts<'a, T>,
7687        is_cached_store: bool,
7688        txn_signatures: Option<&'a [Option<&'a Signature>]>,
7689        reclaim: StoreReclaims,
7690    ) {
7691        // If all transactions in a batch are errored,
7692        // it's possible to get a store with no accounts.
7693        if accounts.is_empty() {
7694            return;
7695        }
7696
7697        let mut stats = BankHashStats::default();
7698        let mut total_data = 0;
7699        (0..accounts.len()).for_each(|index| {
7700            let account = accounts.account(index);
7701            total_data += account.data().len();
7702            stats.update(account);
7703        });
7704
7705        self.stats
7706            .store_total_data
7707            .fetch_add(total_data as u64, Ordering::Relaxed);
7708
7709        {
7710            // we need to drop bank_hashes to prevent deadlocks
7711            let mut bank_hashes = self.bank_hashes.write().unwrap();
7712            let slot_info = bank_hashes
7713                .entry(accounts.target_slot())
7714                .or_insert_with(BankHashInfo::default);
7715            slot_info.stats.merge(&stats);
7716        }
7717
7718        // we use default hashes for now since the same account may be stored to the cache multiple times
7719        self.store_accounts_unfrozen(accounts, None, is_cached_store, txn_signatures, reclaim);
7720        self.report_store_timings();
7721    }
7722
7723    fn report_store_timings(&self) {
7724        if self.stats.last_store_report.should_update(1000) {
7725            let (read_only_cache_hits, read_only_cache_misses, read_only_cache_evicts) =
7726                self.read_only_accounts_cache.get_and_reset_stats();
7727            datapoint_info!(
7728                "accounts_db_store_timings",
7729                (
7730                    "hash_accounts",
7731                    self.stats.store_hash_accounts.swap(0, Ordering::Relaxed),
7732                    i64
7733                ),
7734                (
7735                    "store_accounts",
7736                    self.stats.store_accounts.swap(0, Ordering::Relaxed),
7737                    i64
7738                ),
7739                (
7740                    "update_index",
7741                    self.stats.store_update_index.swap(0, Ordering::Relaxed),
7742                    i64
7743                ),
7744                (
7745                    "handle_reclaims",
7746                    self.stats.store_handle_reclaims.swap(0, Ordering::Relaxed),
7747                    i64
7748                ),
7749                (
7750                    "append_accounts",
7751                    self.stats.store_append_accounts.swap(0, Ordering::Relaxed),
7752                    i64
7753                ),
7754                (
7755                    "stakes_cache_check_and_store_us",
7756                    self.stats
7757                        .stakes_cache_check_and_store_us
7758                        .swap(0, Ordering::Relaxed),
7759                    i64
7760                ),
7761                (
7762                    "find_storage",
7763                    self.stats.store_find_store.swap(0, Ordering::Relaxed),
7764                    i64
7765                ),
7766                (
7767                    "num_accounts",
7768                    self.stats.store_num_accounts.swap(0, Ordering::Relaxed),
7769                    i64
7770                ),
7771                (
7772                    "total_data",
7773                    self.stats.store_total_data.swap(0, Ordering::Relaxed),
7774                    i64
7775                ),
7776                (
7777                    "read_only_accounts_cache_entries",
7778                    self.read_only_accounts_cache.cache_len(),
7779                    i64
7780                ),
7781                (
7782                    "read_only_accounts_cache_data_size",
7783                    self.read_only_accounts_cache.data_size(),
7784                    i64
7785                ),
7786                ("read_only_accounts_cache_hits", read_only_cache_hits, i64),
7787                (
7788                    "read_only_accounts_cache_misses",
7789                    read_only_cache_misses,
7790                    i64
7791                ),
7792                (
7793                    "read_only_accounts_cache_evicts",
7794                    read_only_cache_evicts,
7795                    i64
7796                ),
7797                (
7798                    "calc_stored_meta_us",
7799                    self.stats.calc_stored_meta.swap(0, Ordering::Relaxed),
7800                    i64
7801                ),
7802            );
7803
7804            let recycle_stores = self.recycle_stores.read().unwrap();
7805            datapoint_info!(
7806                "accounts_db_store_timings2",
7807                (
7808                    "recycle_store_count",
7809                    self.stats.recycle_store_count.swap(0, Ordering::Relaxed),
7810                    i64
7811                ),
7812                (
7813                    "current_recycle_store_count",
7814                    recycle_stores.entry_count(),
7815                    i64
7816                ),
7817                (
7818                    "current_recycle_store_bytes",
7819                    recycle_stores.total_bytes(),
7820                    i64
7821                ),
7822                (
7823                    "create_store_count",
7824                    self.stats.create_store_count.swap(0, Ordering::Relaxed),
7825                    i64
7826                ),
7827                (
7828                    "store_get_slot_store",
7829                    self.stats.store_get_slot_store.swap(0, Ordering::Relaxed),
7830                    i64
7831                ),
7832                (
7833                    "store_find_existing",
7834                    self.stats.store_find_existing.swap(0, Ordering::Relaxed),
7835                    i64
7836                ),
7837                (
7838                    "dropped_stores",
7839                    self.stats.dropped_stores.swap(0, Ordering::Relaxed),
7840                    i64
7841                ),
7842            );
7843        }
7844    }
7845
7846    fn store_accounts_unfrozen<'a, T: ReadableAccount + Sync + ZeroLamport>(
7847        &self,
7848        accounts: impl StorableAccounts<'a, T>,
7849        hashes: Option<&[&Hash]>,
7850        is_cached_store: bool,
7851        txn_signatures: Option<&'a [Option<&'a Signature>]>,
7852        reclaim: StoreReclaims,
7853    ) {
7854        // This path comes from a store to a non-frozen slot.
7855        // If a store is dead here, then a newer update for
7856        // each pubkey in the store must exist in another
7857        // store in the slot. Thus it is safe to reset the store and
7858        // re-use it for a future store op. The pubkey ref counts should still
7859        // hold just 1 ref from this slot.
7860        let reset_accounts = true;
7861
7862        self.store_accounts_custom(
7863            accounts,
7864            hashes,
7865            None,
7866            None::<Box<dyn Iterator<Item = u64>>>,
7867            is_cached_store,
7868            reset_accounts,
7869            txn_signatures,
7870            reclaim,
7871        );
7872    }
7873
7874    pub(crate) fn store_accounts_frozen<'a, T: ReadableAccount + Sync + ZeroLamport>(
7875        &'a self,
7876        accounts: impl StorableAccounts<'a, T>,
7877        hashes: Option<&[impl Borrow<Hash>]>,
7878        storage: Option<&'a Arc<AccountStorageEntry>>,
7879        write_version_producer: Option<Box<dyn Iterator<Item = StoredMetaWriteVersion>>>,
7880        reclaim: StoreReclaims,
7881    ) -> StoreAccountsTiming {
7882        // stores on a frozen slot should not reset
7883        // the append vec so that hashing could happen on the store
7884        // and accounts in the append_vec can be unrefed correctly
7885        let reset_accounts = false;
7886        let is_cached_store = false;
7887        self.store_accounts_custom(
7888            accounts,
7889            hashes,
7890            storage,
7891            write_version_producer,
7892            is_cached_store,
7893            reset_accounts,
7894            None,
7895            reclaim,
7896        )
7897    }
7898
7899    fn store_accounts_custom<'a, 'b, T: ReadableAccount + Sync + ZeroLamport>(
7900        &'a self,
7901        accounts: impl StorableAccounts<'b, T>,
7902        hashes: Option<&[impl Borrow<Hash>]>,
7903        storage: Option<&'a Arc<AccountStorageEntry>>,
7904        write_version_producer: Option<Box<dyn Iterator<Item = u64>>>,
7905        is_cached_store: bool,
7906        reset_accounts: bool,
7907        txn_signatures: Option<&'b [Option<&'b Signature>]>,
7908        reclaim: StoreReclaims,
7909    ) -> StoreAccountsTiming {
7910        let storage_finder = Box::new(move |slot, size| {
7911            storage
7912                .cloned()
7913                .unwrap_or_else(|| self.find_storage_candidate(slot, size))
7914        });
7915
7916        let write_version_producer: Box<dyn Iterator<Item = u64>> = write_version_producer
7917            .unwrap_or_else(|| {
7918                let mut current_version = self.bulk_assign_write_version(accounts.len());
7919                Box::new(std::iter::from_fn(move || {
7920                    let ret = current_version;
7921                    current_version += 1;
7922                    Some(ret)
7923                }))
7924            });
7925
7926        self.stats
7927            .store_num_accounts
7928            .fetch_add(accounts.len() as u64, Ordering::Relaxed);
7929        let mut store_accounts_time = Measure::start("store_accounts");
7930        let infos = self.store_accounts_to(
7931            &accounts,
7932            hashes,
7933            storage_finder,
7934            write_version_producer,
7935            is_cached_store,
7936            txn_signatures,
7937        );
7938        store_accounts_time.stop();
7939        self.stats
7940            .store_accounts
7941            .fetch_add(store_accounts_time.as_us(), Ordering::Relaxed);
7942        let mut update_index_time = Measure::start("update_index");
7943
7944        let reclaim = if matches!(reclaim, StoreReclaims::Ignore) {
7945            UpsertReclaim::IgnoreReclaims
7946        } else if self.caching_enabled && is_cached_store {
7947            UpsertReclaim::PreviousSlotEntryWasCached
7948        } else {
7949            UpsertReclaim::PopulateReclaims
7950        };
7951
7952        // if we are squashing a single slot, then we can expect a single dead slot
7953        let expected_single_dead_slot =
7954            (!accounts.contains_multiple_slots()).then(|| accounts.target_slot());
7955
7956        // If the cache was flushed, then because `update_index` occurs
7957        // after the account are stored by the above `store_accounts_to`
7958        // call and all the accounts are stored, all reads after this point
7959        // will know to not check the cache anymore
7960        let mut reclaims = self.update_index(infos, accounts, reclaim);
7961
7962        // For each updated account, `reclaims` should only have at most one
7963        // item (if the account was previously updated in this slot).
7964        // filter out the cached reclaims as those don't actually map
7965        // to anything that needs to be cleaned in the backing storage
7966        // entries
7967        if self.caching_enabled {
7968            reclaims.retain(|(_, r)| !r.is_cached());
7969
7970            if is_cached_store {
7971                assert!(reclaims.is_empty());
7972            }
7973        }
7974
7975        update_index_time.stop();
7976        self.stats
7977            .store_update_index
7978            .fetch_add(update_index_time.as_us(), Ordering::Relaxed);
7979
7980        // A store for a single slot should:
7981        // 1) Only make "reclaims" for the same slot
7982        // 2) Should not cause any slots to be removed from the storage
7983        // database because
7984        //    a) this slot  has at least one account (the one being stored),
7985        //    b)From 1) we know no other slots are included in the "reclaims"
7986        //
7987        // From 1) and 2) we guarantee passing `no_purge_stats` == None, which is
7988        // equivalent to asserting there will be no dead slots, is safe.
7989        let mut handle_reclaims_time = Measure::start("handle_reclaims");
7990        self.handle_reclaims(
7991            (!reclaims.is_empty()).then(|| reclaims.iter()),
7992            expected_single_dead_slot,
7993            None,
7994            reset_accounts,
7995            &HashSet::default(),
7996        );
7997        handle_reclaims_time.stop();
7998        self.stats
7999            .store_handle_reclaims
8000            .fetch_add(handle_reclaims_time.as_us(), Ordering::Relaxed);
8001
8002        StoreAccountsTiming {
8003            store_accounts_elapsed: store_accounts_time.as_us(),
8004            update_index_elapsed: update_index_time.as_us(),
8005            handle_reclaims_elapsed: handle_reclaims_time.as_us(),
8006        }
8007    }
8008
8009    pub fn add_root(&self, slot: Slot) -> AccountsAddRootTiming {
8010        let mut index_time = Measure::start("index_add_root");
8011        self.accounts_index.add_root(slot, self.caching_enabled);
8012        index_time.stop();
8013        let mut cache_time = Measure::start("cache_add_root");
8014        if self.caching_enabled {
8015            self.accounts_cache.add_root(slot);
8016        }
8017        cache_time.stop();
8018        let mut store_time = Measure::start("store_add_root");
8019        if let Some(slot_stores) = self.storage.get_slot_stores(slot) {
8020            for (store_id, store) in slot_stores.read().unwrap().iter() {
8021                self.dirty_stores.insert((slot, *store_id), store.clone());
8022            }
8023        }
8024        store_time.stop();
8025
8026        AccountsAddRootTiming {
8027            index_us: index_time.as_us(),
8028            cache_us: cache_time.as_us(),
8029            store_us: store_time.as_us(),
8030        }
8031    }
8032
8033    pub fn get_snapshot_storages(
8034        &self,
8035        snapshot_slot: Slot,
8036        snapshot_base_slot: Option<Slot>,
8037        ancestors: Option<&Ancestors>,
8038    ) -> (SnapshotStorages, Vec<Slot>) {
8039        let mut m = Measure::start("get slots");
8040        let slots = self
8041            .storage
8042            .map
8043            .iter()
8044            .map(|k| *k.key() as Slot)
8045            .collect::<Vec<_>>();
8046        m.stop();
8047        let mut m2 = Measure::start("filter");
8048
8049        let chunk_size = 5_000;
8050        let wide = self.thread_pool_clean.install(|| {
8051            slots
8052                .par_chunks(chunk_size)
8053                .map(|slots| {
8054                    slots
8055                        .iter()
8056                        .filter_map(|slot| {
8057                            if *slot <= snapshot_slot
8058                                && snapshot_base_slot
8059                                    .map_or(true, |snapshot_base_slot| *slot > snapshot_base_slot)
8060                                && (self.accounts_index.is_alive_root(*slot)
8061                                    || ancestors
8062                                        .map(|ancestors| ancestors.contains_key(slot))
8063                                        .unwrap_or_default())
8064                            {
8065                                self.storage.map.get(slot).map_or_else(
8066                                    || None,
8067                                    |item| {
8068                                        let storages = item
8069                                            .value()
8070                                            .read()
8071                                            .unwrap()
8072                                            .values()
8073                                            .filter(|x| x.has_accounts())
8074                                            .cloned()
8075                                            .collect::<Vec<_>>();
8076                                        if !storages.is_empty() {
8077                                            Some((storages, *slot))
8078                                        } else {
8079                                            None
8080                                        }
8081                                    },
8082                                )
8083                            } else {
8084                                None
8085                            }
8086                        })
8087                        .collect::<Vec<(SnapshotStorage, Slot)>>()
8088                })
8089                .collect::<Vec<_>>()
8090        });
8091        m2.stop();
8092        let mut m3 = Measure::start("flatten");
8093        // some slots we found above may not have been a root or met the slot # constraint.
8094        // So the resulting 'slots' vector we return will be a subset of the raw keys we got initially.
8095        let mut slots = Vec::with_capacity(slots.len());
8096        let result = wide
8097            .into_iter()
8098            .flatten()
8099            .map(|(storage, slot)| {
8100                slots.push(slot);
8101                storage
8102            })
8103            .collect::<Vec<_>>();
8104        m3.stop();
8105
8106        debug!(
8107            "hash_total: get slots: {}, filter: {}, flatten: {}",
8108            m.as_us(),
8109            m2.as_us(),
8110            m3.as_us()
8111        );
8112        (result, slots)
8113    }
8114
8115    fn process_storage_slot<'a>(
8116        &self,
8117        storage_maps: &'a [Arc<AccountStorageEntry>],
8118    ) -> GenerateIndexAccountsMap<'a> {
8119        let num_accounts = storage_maps
8120            .iter()
8121            .map(|storage| storage.approx_stored_count())
8122            .sum();
8123        let mut accounts_map = GenerateIndexAccountsMap::with_capacity(num_accounts);
8124        storage_maps.iter().for_each(|storage| {
8125            storage.accounts.account_iter().for_each(|stored_account| {
8126                let this_version = stored_account.meta.write_version;
8127                let pubkey = stored_account.meta.pubkey;
8128                assert!(!self.is_filler_account(&pubkey));
8129                match accounts_map.entry(pubkey) {
8130                    Entry::Vacant(entry) => {
8131                        entry.insert(IndexAccountMapEntry {
8132                            write_version: this_version,
8133                            store_id: storage.append_vec_id(),
8134                            stored_account,
8135                        });
8136                    }
8137                    Entry::Occupied(mut entry) => {
8138                        let occupied_version = entry.get().write_version;
8139                        if occupied_version < this_version {
8140                            entry.insert(IndexAccountMapEntry {
8141                                write_version: this_version,
8142                                store_id: storage.append_vec_id(),
8143                                stored_account,
8144                            });
8145                        } else {
8146                            assert_ne!(occupied_version, this_version);
8147                        }
8148                    }
8149                }
8150            })
8151        });
8152        accounts_map
8153    }
8154
8155    /// return Some(lamports_to_top_off) if 'account' would collect rent
8156    fn stats_for_rent_payers<T: ReadableAccount>(
8157        pubkey: &Pubkey,
8158        account: &T,
8159        rent_collector: &RentCollector,
8160    ) -> Option<u64> {
8161        if account.lamports() == 0 {
8162            return None;
8163        }
8164        (rent_collector.should_collect_rent(pubkey, account)
8165            && !rent_collector.get_rent_due(account).is_exempt())
8166        .then(|| {
8167            let min_balance = rent_collector.rent.minimum_balance(account.data().len());
8168            // return lamports required to top off this account to make it rent exempt
8169            min_balance.saturating_sub(account.lamports())
8170        })
8171    }
8172
8173    fn generate_index_for_slot<'a>(
8174        &self,
8175        accounts_map: GenerateIndexAccountsMap<'a>,
8176        slot: &Slot,
8177        rent_collector: &RentCollector,
8178    ) -> SlotIndexGenerationInfo {
8179        if accounts_map.is_empty() {
8180            return SlotIndexGenerationInfo::default();
8181        }
8182
8183        let secondary = !self.account_indexes.is_empty();
8184
8185        let mut rent_paying_accounts_by_partition = Vec::default();
8186        let mut accounts_data_len = 0;
8187        let mut num_accounts_rent_paying = 0;
8188        let num_accounts = accounts_map.len();
8189        let mut amount_to_top_off_rent = 0;
8190        let items = accounts_map.into_iter().map(
8191            |(
8192                pubkey,
8193                IndexAccountMapEntry {
8194                    write_version: _write_version,
8195                    store_id,
8196                    stored_account,
8197                },
8198            )| {
8199                if secondary {
8200                    self.accounts_index.update_secondary_indexes(
8201                        &pubkey,
8202                        &stored_account,
8203                        &self.account_indexes,
8204                    );
8205                }
8206                if !stored_account.is_zero_lamport() {
8207                    accounts_data_len += stored_account.data().len() as u64;
8208                }
8209
8210                if let Some(amount_to_top_off_rent_this_account) =
8211                    Self::stats_for_rent_payers(&pubkey, &stored_account, rent_collector)
8212                {
8213                    amount_to_top_off_rent += amount_to_top_off_rent_this_account;
8214                    num_accounts_rent_paying += 1;
8215                    // remember this rent-paying account pubkey
8216                    rent_paying_accounts_by_partition.push(pubkey);
8217                }
8218
8219                (
8220                    pubkey,
8221                    AccountInfo::new(
8222                        StorageLocation::AppendVec(store_id, stored_account.offset), // will never be cached
8223                        stored_account.stored_size as StoredSize, // stored_size should never exceed StoredSize::MAX because of max data len const
8224                        stored_account.account_meta.lamports,
8225                    ),
8226                )
8227            },
8228        );
8229
8230        let (dirty_pubkeys, insert_time_us) = self
8231            .accounts_index
8232            .insert_new_if_missing_into_primary_index(*slot, num_accounts, items);
8233
8234        // dirty_pubkeys will contain a pubkey if an item has multiple rooted entries for
8235        // a given pubkey. If there is just a single item, there is no cleaning to
8236        // be done on that pubkey. Use only those pubkeys with multiple updates.
8237        if !dirty_pubkeys.is_empty() {
8238            self.uncleaned_pubkeys.insert(*slot, dirty_pubkeys);
8239        }
8240        SlotIndexGenerationInfo {
8241            insert_time_us,
8242            num_accounts: num_accounts as u64,
8243            num_accounts_rent_paying,
8244            accounts_data_len,
8245            amount_to_top_off_rent,
8246            rent_paying_accounts_by_partition,
8247        }
8248    }
8249
8250    fn filler_unique_id_bytes() -> usize {
8251        std::mem::size_of::<u32>()
8252    }
8253
8254    fn filler_rent_partition_prefix_bytes() -> usize {
8255        std::mem::size_of::<u64>()
8256    }
8257
8258    fn filler_prefix_bytes() -> usize {
8259        Self::filler_unique_id_bytes() + Self::filler_rent_partition_prefix_bytes()
8260    }
8261
8262    pub fn is_filler_account_helper(
8263        pubkey: &Pubkey,
8264        filler_account_suffix: Option<&Pubkey>,
8265    ) -> bool {
8266        let offset = Self::filler_prefix_bytes();
8267        filler_account_suffix
8268            .as_ref()
8269            .map(|filler_account_suffix| {
8270                pubkey.as_ref()[offset..] == filler_account_suffix.as_ref()[offset..]
8271            })
8272            .unwrap_or_default()
8273    }
8274
8275    /// true if 'pubkey' is a filler account
8276    pub fn is_filler_account(&self, pubkey: &Pubkey) -> bool {
8277        Self::is_filler_account_helper(pubkey, self.filler_account_suffix.as_ref())
8278    }
8279
8280    /// true if it is possible that there are filler accounts present
8281    pub fn filler_accounts_enabled(&self) -> bool {
8282        self.filler_account_suffix.is_some()
8283    }
8284
8285    /// retain slots in 'roots' that are > (max(roots) - slots_per_epoch)
8286    fn retain_roots_within_one_epoch_range(roots: &mut Vec<Slot>, slots_per_epoch: SlotCount) {
8287        if let Some(max) = roots.iter().max() {
8288            let min = max - slots_per_epoch;
8289            roots.retain(|slot| slot > &min);
8290        }
8291    }
8292
8293    /// return 'AccountSharedData' and a hash for a filler account
8294    fn get_filler_account(&self, rent: &Rent) -> (AccountSharedData, Hash) {
8295        let string = "FiLLERACCoUNTooooooooooooooooooooooooooooooo";
8296        let hash = Hash::from_str(string).unwrap();
8297        let owner = Pubkey::from_str(string).unwrap();
8298        let space = self.filler_accounts_config.size;
8299        let rent_exempt_reserve = rent.minimum_balance(space);
8300        let lamports = rent_exempt_reserve;
8301        let mut account = AccountSharedData::new(lamports, space, &owner);
8302        // just non-zero rent epoch. filler accounts are rent-exempt
8303        let dummy_rent_epoch = 2;
8304        account.set_rent_epoch(dummy_rent_epoch);
8305        (account, hash)
8306    }
8307
8308    fn get_filler_account_pubkeys(&self, count: usize) -> Vec<Pubkey> {
8309        (0..count)
8310            .map(|_| {
8311                let subrange = solana_sdk::pubkey::new_rand();
8312                self.get_filler_account_pubkey(&subrange)
8313            })
8314            .collect()
8315    }
8316
8317    fn get_filler_account_pubkey(&self, subrange: &Pubkey) -> Pubkey {
8318        // pubkey begins life as entire filler 'suffix' pubkey
8319        let mut key = self.filler_account_suffix.unwrap();
8320        let rent_prefix_bytes = Self::filler_rent_partition_prefix_bytes();
8321        // first bytes are replaced with rent partition range: filler_rent_partition_prefix_bytes
8322        key.as_mut()[0..rent_prefix_bytes]
8323            .copy_from_slice(&subrange.as_ref()[0..rent_prefix_bytes]);
8324        key
8325    }
8326
8327    /// filler accounts are space-holding accounts which are ignored by hash calculations and rent.
8328    /// They are designed to allow a validator to run against a network successfully while simulating having many more accounts present.
8329    /// All filler accounts share a common pubkey suffix. The suffix is randomly generated per validator on startup.
8330    /// The filler accounts are added to each slot in the snapshot after index generation.
8331    /// The accounts added in a slot are setup to have pubkeys such that rent will be collected from them before (or when?) their slot becomes an epoch old.
8332    /// Thus, the filler accounts are rewritten by rent and the old slot can be thrown away successfully.
8333    pub fn maybe_add_filler_accounts(
8334        &self,
8335        epoch_schedule: &EpochSchedule,
8336        rent: &Rent,
8337        slot: Slot,
8338    ) {
8339        if self.filler_accounts_config.count == 0 {
8340            return;
8341        }
8342
8343        if ADD_FILLER_ACCOUNTS_GRADUALLY {
8344            self.init_gradual_filler_accounts(
8345                epoch_schedule.get_slots_in_epoch(epoch_schedule.get_epoch(slot)),
8346            );
8347            return;
8348        }
8349
8350        let max_root_inclusive = self.accounts_index.max_root_inclusive();
8351        let epoch = epoch_schedule.get_epoch(max_root_inclusive);
8352
8353        info!(
8354            "adding {} filler accounts with size {}",
8355            self.filler_accounts_config.count, self.filler_accounts_config.size,
8356        );
8357        // break this up to force the accounts out of memory after each pass
8358        let passes = 100;
8359        let mut roots = self.storage.all_slots();
8360        Self::retain_roots_within_one_epoch_range(
8361            &mut roots,
8362            epoch_schedule.get_slots_in_epoch(epoch),
8363        );
8364        let root_count = roots.len();
8365        let per_pass = std::cmp::max(1, root_count / passes);
8366        let overall_index = AtomicUsize::new(0);
8367        let (account, hash) = self.get_filler_account(rent);
8368        let added = AtomicU32::default();
8369        let rent_prefix_bytes = Self::filler_rent_partition_prefix_bytes();
8370        for pass in 0..=passes {
8371            self.accounts_index
8372                .set_startup(Startup::StartupWithExtraThreads);
8373            let roots_in_this_pass = roots
8374                .iter()
8375                .skip(pass * per_pass)
8376                .take(per_pass)
8377                .collect::<Vec<_>>();
8378            roots_in_this_pass.into_par_iter().for_each(|slot| {
8379                let storage_maps: Vec<Arc<AccountStorageEntry>> = self
8380                    .storage
8381                    .get_slot_storage_entries(*slot)
8382                    .unwrap_or_default();
8383                if storage_maps.is_empty() {
8384                    return;
8385                }
8386
8387                let partition = crate::bank::Bank::variable_cycle_partition_from_previous_slot(
8388                    epoch_schedule,
8389                    *slot,
8390                );
8391                let subrange = crate::bank::Bank::pubkey_range_from_partition(partition);
8392
8393                let idx = overall_index.fetch_add(1, Ordering::Relaxed);
8394                let filler_entries = (idx + 1) * self.filler_accounts_config.count / root_count
8395                    - idx * self.filler_accounts_config.count / root_count;
8396                let accounts = (0..filler_entries)
8397                    .map(|_| {
8398                        let my_id = added.fetch_add(1, Ordering::Relaxed);
8399                        let mut key = self.get_filler_account_pubkey(subrange.start());
8400                        // next bytes are replaced with my_id: filler_unique_id_bytes
8401                        let my_id_bytes = u32::to_be_bytes(my_id);
8402                        key.as_mut()[rent_prefix_bytes
8403                            ..(rent_prefix_bytes + Self::filler_unique_id_bytes())]
8404                            .copy_from_slice(&my_id_bytes);
8405                        key
8406                    })
8407                    .collect::<Vec<_>>();
8408                let add = accounts
8409                    .iter()
8410                    .map(|key| (key, &account))
8411                    .collect::<Vec<_>>();
8412                let hashes = (0..filler_entries).map(|_| hash).collect::<Vec<_>>();
8413                self.maybe_throttle_index_generation();
8414                self.store_accounts_frozen(
8415                    (*slot, &add[..]),
8416                    Some(&hashes[..]),
8417                    None,
8418                    None,
8419                    StoreReclaims::Ignore,
8420                );
8421            });
8422            self.accounts_index.set_startup(Startup::Normal);
8423        }
8424        info!("added {} filler accounts", added.load(Ordering::Relaxed));
8425    }
8426
8427    #[allow(clippy::needless_collect)]
8428    pub fn generate_index(
8429        &self,
8430        limit_load_slot_count_from_snapshot: Option<usize>,
8431        verify: bool,
8432        genesis_config: &GenesisConfig,
8433    ) -> IndexGenerationInfo {
8434        let mut slots = self.storage.all_slots();
8435        #[allow(clippy::stable_sort_primitive)]
8436        slots.sort();
8437        if let Some(limit) = limit_load_slot_count_from_snapshot {
8438            slots.truncate(limit); // get rid of the newer slots and keep just the older
8439        }
8440        let max_slot = slots.last().cloned().unwrap_or_default();
8441        let schedule = genesis_config.epoch_schedule;
8442        let rent_collector = RentCollector::new(
8443            schedule.get_epoch(max_slot),
8444            schedule,
8445            genesis_config.slots_per_year(),
8446            genesis_config.rent,
8447        );
8448        let accounts_data_len = AtomicU64::new(0);
8449
8450        let rent_paying_accounts_by_partition =
8451            Mutex::new(RentPayingAccountsByPartition::new(&schedule));
8452
8453        // pass == 0 always runs and generates the index
8454        // pass == 1 only runs if verify == true.
8455        // verify checks that all the expected items are in the accounts index and measures how long it takes to look them all up
8456        let passes = if verify { 2 } else { 1 };
8457        for pass in 0..passes {
8458            if pass == 0 {
8459                self.accounts_index
8460                    .set_startup(Startup::StartupWithExtraThreads);
8461            }
8462            let storage_info = StorageSizeAndCountMap::default();
8463            let total_processed_slots_across_all_threads = AtomicU64::new(0);
8464            let outer_slots_len = slots.len();
8465            let threads = if self.accounts_index.is_disk_index_enabled() {
8466                // these write directly to disk, so the more threads, the better
8467                num_cpus::get()
8468            } else {
8469                // seems to be a good hueristic given varying # cpus for in-mem disk index
8470                8
8471            };
8472            let chunk_size = (outer_slots_len / (std::cmp::max(1, threads.saturating_sub(1)))) + 1; // approximately 400k slots in a snapshot
8473            let mut index_time = Measure::start("index");
8474            let insertion_time_us = AtomicU64::new(0);
8475            let rent_paying = AtomicUsize::new(0);
8476            let amount_to_top_off_rent = AtomicU64::new(0);
8477            let total_duplicates = AtomicU64::new(0);
8478            let storage_info_timings = Mutex::new(GenerateIndexTimings::default());
8479            let scan_time: u64 = slots
8480                .par_chunks(chunk_size)
8481                .map(|slots| {
8482                    let mut log_status = MultiThreadProgress::new(
8483                        &total_processed_slots_across_all_threads,
8484                        2,
8485                        outer_slots_len as u64,
8486                    );
8487                    let mut scan_time_sum = 0;
8488                    for (index, slot) in slots.iter().enumerate() {
8489                        let mut scan_time = Measure::start("scan");
8490                        log_status.report(index as u64);
8491                        let storage_maps: Vec<Arc<AccountStorageEntry>> = self
8492                            .storage
8493                            .get_slot_storage_entries(*slot)
8494                            .unwrap_or_default();
8495                        let accounts_map = self.process_storage_slot(&storage_maps);
8496                        scan_time.stop();
8497                        scan_time_sum += scan_time.as_us();
8498                        Self::update_storage_info(
8499                            &storage_info,
8500                            &accounts_map,
8501                            &storage_info_timings,
8502                        );
8503
8504                        let insert_us = if pass == 0 {
8505                            // generate index
8506                            self.maybe_throttle_index_generation();
8507                            let SlotIndexGenerationInfo {
8508                                insert_time_us: insert_us,
8509                                num_accounts: total_this_slot,
8510                                num_accounts_rent_paying: rent_paying_this_slot,
8511                                accounts_data_len: accounts_data_len_this_slot,
8512                                amount_to_top_off_rent: amount_to_top_off_rent_this_slot,
8513                                rent_paying_accounts_by_partition:
8514                                    rent_paying_accounts_by_partition_this_slot,
8515                            } = self.generate_index_for_slot(accounts_map, slot, &rent_collector);
8516                            rent_paying.fetch_add(rent_paying_this_slot, Ordering::Relaxed);
8517                            amount_to_top_off_rent
8518                                .fetch_add(amount_to_top_off_rent_this_slot, Ordering::Relaxed);
8519                            total_duplicates.fetch_add(total_this_slot, Ordering::Relaxed);
8520                            accounts_data_len
8521                                .fetch_add(accounts_data_len_this_slot, Ordering::Relaxed);
8522                            let mut rent_paying_accounts_by_partition =
8523                                rent_paying_accounts_by_partition.lock().unwrap();
8524                            rent_paying_accounts_by_partition_this_slot
8525                                .iter()
8526                                .for_each(|k| {
8527                                    rent_paying_accounts_by_partition.add_account(k);
8528                                });
8529
8530                            insert_us
8531                        } else {
8532                            // verify index matches expected and measure the time to get all items
8533                            assert!(verify);
8534                            let mut lookup_time = Measure::start("lookup_time");
8535                            for account in accounts_map.into_iter() {
8536                                let (key, account_info) = account;
8537                                let lock = self.accounts_index.get_account_maps_read_lock(&key);
8538                                let x = lock.get(&key).unwrap();
8539                                let sl = x.slot_list.read().unwrap();
8540                                let mut count = 0;
8541                                for (slot2, account_info2) in sl.iter() {
8542                                    if slot2 == slot {
8543                                        count += 1;
8544                                        let ai = AccountInfo::new(
8545                                            StorageLocation::AppendVec(
8546                                                account_info.store_id,
8547                                                account_info.stored_account.offset,
8548                                            ), // will never be cached
8549                                            account_info.stored_account.stored_size as StoredSize, // stored_size should never exceed StoredSize::MAX because of max data len const
8550                                            account_info.stored_account.account_meta.lamports,
8551                                        );
8552                                        assert_eq!(&ai, account_info2);
8553                                    }
8554                                }
8555                                assert_eq!(1, count);
8556                            }
8557                            lookup_time.stop();
8558                            lookup_time.as_us()
8559                        };
8560                        insertion_time_us.fetch_add(insert_us, Ordering::Relaxed);
8561                    }
8562                    scan_time_sum
8563                })
8564                .sum();
8565            index_time.stop();
8566
8567            info!("rent_collector: {:?}", rent_collector);
8568            let mut min_bin_size = usize::MAX;
8569            let mut max_bin_size = usize::MIN;
8570            let total_items = self
8571                .accounts_index
8572                .account_maps
8573                .iter()
8574                .map(|map_bin| {
8575                    let len = map_bin.len_for_stats();
8576                    min_bin_size = std::cmp::min(min_bin_size, len);
8577                    max_bin_size = std::cmp::max(max_bin_size, len);
8578                    len as usize
8579                })
8580                .sum();
8581
8582            let mut index_flush_us = 0;
8583            if pass == 0 {
8584                // tell accounts index we are done adding the initial accounts at startup
8585                let mut m = Measure::start("accounts_index_idle_us");
8586                self.accounts_index.set_startup(Startup::Normal);
8587                m.stop();
8588                index_flush_us = m.as_us();
8589
8590                // this has to happen before visit_duplicate_pubkeys_during_startup below
8591                // get duplicate keys from acct idx. We have to wait until we've finished flushing.
8592                for (slot, key) in self
8593                    .accounts_index
8594                    .retrieve_duplicate_keys_from_startup()
8595                    .into_iter()
8596                    .flatten()
8597                {
8598                    match self.uncleaned_pubkeys.entry(slot) {
8599                        Occupied(mut occupied) => occupied.get_mut().push(key),
8600                        Vacant(vacant) => {
8601                            vacant.insert(vec![key]);
8602                        }
8603                    }
8604                }
8605            }
8606
8607            let storage_info_timings = storage_info_timings.into_inner().unwrap();
8608            let mut timings = GenerateIndexTimings {
8609                index_flush_us,
8610                scan_time,
8611                index_time: index_time.as_us(),
8612                insertion_time_us: insertion_time_us.load(Ordering::Relaxed),
8613                min_bin_size,
8614                max_bin_size,
8615                total_items,
8616                rent_paying,
8617                amount_to_top_off_rent,
8618                total_duplicates: total_duplicates.load(Ordering::Relaxed),
8619                storage_size_accounts_map_us: storage_info_timings.storage_size_accounts_map_us,
8620                storage_size_accounts_map_flatten_us: storage_info_timings
8621                    .storage_size_accounts_map_flatten_us,
8622                ..GenerateIndexTimings::default()
8623            };
8624
8625            // subtract data.len() from accounts_data_len for all old accounts that are in the index twice
8626            let mut accounts_data_len_dedup_timer =
8627                Measure::start("handle accounts data len duplicates");
8628            let uncleaned_roots = Mutex::new(HashSet::<Slot>::default());
8629            if pass == 0 {
8630                let mut unique_pubkeys = HashSet::<Pubkey>::default();
8631                self.uncleaned_pubkeys.iter().for_each(|entry| {
8632                    entry.value().iter().for_each(|pubkey| {
8633                        unique_pubkeys.insert(*pubkey);
8634                    })
8635                });
8636                let accounts_data_len_from_duplicates = unique_pubkeys
8637                    .into_iter()
8638                    .collect::<Vec<_>>()
8639                    .par_chunks(4096)
8640                    .map(|pubkeys| {
8641                        let (count, uncleaned_roots_this_group) = self
8642                            .visit_duplicate_pubkeys_during_startup(
8643                                pubkeys,
8644                                &rent_collector,
8645                                &timings,
8646                            );
8647                        let mut uncleaned_roots = uncleaned_roots.lock().unwrap();
8648                        uncleaned_roots_this_group.into_iter().for_each(|slot| {
8649                            uncleaned_roots.insert(slot);
8650                        });
8651                        count
8652                    })
8653                    .sum();
8654                accounts_data_len.fetch_sub(accounts_data_len_from_duplicates, Ordering::Relaxed);
8655                info!(
8656                    "accounts data len: {}",
8657                    accounts_data_len.load(Ordering::Relaxed)
8658                );
8659            }
8660            accounts_data_len_dedup_timer.stop();
8661            timings.accounts_data_len_dedup_time_us = accounts_data_len_dedup_timer.as_us();
8662
8663            if pass == 0 {
8664                let uncleaned_roots = uncleaned_roots.into_inner().unwrap();
8665                // Need to add these last, otherwise older updates will be cleaned
8666                for root in &slots {
8667                    // passing 'false' to 'add_root' causes 'root' to be added to 'accounts_index.roots_tracker.uncleaned_roots'
8668                    // passing 'true' to 'add_root' does NOT add 'root' to 'accounts_index.roots_tracker.uncleaned_roots'
8669                    // So, don't add all slots to 'uncleaned_roots' here since we know which slots contain duplicate pubkeys.
8670                    let uncleaned_root = uncleaned_roots.contains(root);
8671                    self.accounts_index.add_root(*root, !uncleaned_root);
8672                }
8673
8674                self.set_storage_count_and_alive_bytes(storage_info, &mut timings);
8675            }
8676            timings.report();
8677        }
8678
8679        self.accounts_index.log_secondary_indexes();
8680
8681        IndexGenerationInfo {
8682            accounts_data_len: accounts_data_len.load(Ordering::Relaxed),
8683            rent_paying_accounts_by_partition: rent_paying_accounts_by_partition
8684                .into_inner()
8685                .unwrap(),
8686        }
8687    }
8688
8689    /// Startup processes can consume large amounts of memory while inserting accounts into the index as fast as possible.
8690    /// Calling this can slow down the insertion process to allow flushing to disk to keep pace.
8691    fn maybe_throttle_index_generation(&self) {
8692        // This number is chosen to keep the initial ram usage sufficiently small
8693        // The process of generating the index is goverened entirely by how fast the disk index can be populated.
8694        // 10M accounts is sufficiently small that it will never have memory usage. It seems sufficiently large that it will provide sufficient performance.
8695        // Performance is measured by total time to generate the index.
8696        // 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.
8697        // 550M was straining a 384G machine at startup.
8698        // This is a tunable parameter that just needs to be small enough to keep the generation threads from overwhelming RAM and oom at startup.
8699        const LIMIT: usize = 10_000_000;
8700        while self
8701            .accounts_index
8702            .get_startup_remaining_items_to_flush_estimate()
8703            > LIMIT
8704        {
8705            // 10 ms is long enough to allow some flushing to occur before insertion is resumed.
8706            // callers of this are typically run in parallel, so many threads will be sleeping at different starting intervals, waiting to resume insertion.
8707            sleep(Duration::from_millis(10));
8708        }
8709    }
8710
8711    /// Used during generate_index() to:
8712    /// 1. get the _duplicate_ accounts data len from the given pubkeys
8713    /// 2. get the slots that contained duplicate pubkeys
8714    /// 3. update rent stats
8715    /// Note this should only be used when ALL entries in the accounts index are roots.
8716    /// returns (data len sum of all older duplicates, slots that contained duplicate pubkeys)
8717    fn visit_duplicate_pubkeys_during_startup(
8718        &self,
8719        pubkeys: &[Pubkey],
8720        rent_collector: &RentCollector,
8721        timings: &GenerateIndexTimings,
8722    ) -> (u64, HashSet<Slot>) {
8723        let mut accounts_data_len_from_duplicates = 0;
8724        let mut uncleaned_slots = HashSet::<Slot>::default();
8725        let mut removed_rent_paying = 0;
8726        let mut removed_top_off = 0;
8727        pubkeys.iter().for_each(|pubkey| {
8728            if let Some(entry) = self.accounts_index.get_account_read_entry(pubkey) {
8729                let slot_list = entry.slot_list();
8730                if slot_list.len() < 2 {
8731                    return;
8732                }
8733                // Only the account data len in the highest slot should be used, and the rest are
8734                // duplicates.  So find the max slot to keep.
8735                // Then sum up the remaining data len, which are the duplicates.
8736                // All of the slots need to go in the 'uncleaned_slots' list. For clean to work properly,
8737                // the slot where duplicate accounts are found in the index need to be in 'uncleaned_slots' list, too.
8738                let max = slot_list.iter().map(|(slot, _)| slot).max().unwrap();
8739                slot_list.iter().for_each(|(slot, account_info)| {
8740                    uncleaned_slots.insert(*slot);
8741                    if slot == max {
8742                        // the info in 'max' is the most recent, current info for this pubkey
8743                        return;
8744                    }
8745                    let maybe_storage_entry = self
8746                        .storage
8747                        .get_account_storage_entry(*slot, account_info.store_id());
8748                    let mut accessor = LoadedAccountAccessor::Stored(
8749                        maybe_storage_entry.map(|entry| (entry, account_info.offset())),
8750                    );
8751                    let loaded_account = accessor.check_and_get_loaded_account();
8752                    accounts_data_len_from_duplicates += loaded_account.data().len();
8753                    if let Some(lamports_to_top_off) =
8754                        Self::stats_for_rent_payers(pubkey, &loaded_account, rent_collector)
8755                    {
8756                        removed_rent_paying += 1;
8757                        removed_top_off += lamports_to_top_off;
8758                    }
8759                });
8760            }
8761        });
8762        timings
8763            .rent_paying
8764            .fetch_sub(removed_rent_paying, Ordering::Relaxed);
8765        timings
8766            .amount_to_top_off_rent
8767            .fetch_sub(removed_top_off, Ordering::Relaxed);
8768        (accounts_data_len_from_duplicates as u64, uncleaned_slots)
8769    }
8770
8771    fn update_storage_info(
8772        storage_info: &StorageSizeAndCountMap,
8773        accounts_map: &GenerateIndexAccountsMap<'_>,
8774        timings: &Mutex<GenerateIndexTimings>,
8775    ) {
8776        let mut storage_size_accounts_map_time = Measure::start("storage_size_accounts_map");
8777
8778        let mut storage_info_local = HashMap::<AppendVecId, StorageSizeAndCount>::default();
8779        // first collect into a local HashMap with no lock contention
8780        for (_, v) in accounts_map.iter() {
8781            let mut info = storage_info_local
8782                .entry(v.store_id)
8783                .or_insert_with(StorageSizeAndCount::default);
8784            info.stored_size += v.stored_account.stored_size;
8785            info.count += 1;
8786        }
8787        storage_size_accounts_map_time.stop();
8788        // second, collect into the shared DashMap once we've figured out all the info per store_id
8789        let mut storage_size_accounts_map_flatten_time =
8790            Measure::start("storage_size_accounts_map_flatten_time");
8791        for (store_id, v) in storage_info_local.into_iter() {
8792            let mut info = storage_info
8793                .entry(store_id)
8794                .or_insert_with(StorageSizeAndCount::default);
8795            info.stored_size += v.stored_size;
8796            info.count += v.count;
8797        }
8798        storage_size_accounts_map_flatten_time.stop();
8799
8800        let mut timings = timings.lock().unwrap();
8801        timings.storage_size_accounts_map_us += storage_size_accounts_map_time.as_us();
8802        timings.storage_size_accounts_map_flatten_us +=
8803            storage_size_accounts_map_flatten_time.as_us();
8804    }
8805    fn set_storage_count_and_alive_bytes(
8806        &self,
8807        stored_sizes_and_counts: StorageSizeAndCountMap,
8808        timings: &mut GenerateIndexTimings,
8809    ) {
8810        // store count and size for each storage
8811        let mut storage_size_storages_time = Measure::start("storage_size_storages");
8812        for slot_stores in self.storage.map.iter() {
8813            for (id, store) in slot_stores.value().read().unwrap().iter() {
8814                // Should be default at this point
8815                assert_eq!(store.alive_bytes(), 0);
8816                if let Some(entry) = stored_sizes_and_counts.get(id) {
8817                    trace!(
8818                        "id: {} setting count: {} cur: {}",
8819                        id,
8820                        entry.count,
8821                        store.count(),
8822                    );
8823                    store.count_and_status.write().unwrap().0 = entry.count;
8824                    store.alive_bytes.store(entry.stored_size, Ordering::SeqCst);
8825                } else {
8826                    trace!("id: {} clearing count", id);
8827                    store.count_and_status.write().unwrap().0 = 0;
8828                }
8829            }
8830        }
8831        storage_size_storages_time.stop();
8832        timings.storage_size_storages_us = storage_size_storages_time.as_us();
8833    }
8834
8835    pub(crate) fn print_accounts_stats(&self, label: &str) {
8836        self.print_index(label);
8837        self.print_count_and_status(label);
8838        info!("recycle_stores:");
8839        let recycle_stores = self.recycle_stores.read().unwrap();
8840        for (recycled_time, entry) in recycle_stores.iter() {
8841            info!(
8842                "  slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {} (recycled: {:?})",
8843                entry.slot(),
8844                entry.append_vec_id(),
8845                *entry.count_and_status.read().unwrap(),
8846                entry.approx_store_count.load(Ordering::Relaxed),
8847                entry.accounts.len(),
8848                entry.accounts.capacity(),
8849                recycled_time,
8850            );
8851        }
8852    }
8853
8854    fn print_index(&self, label: &str) {
8855        let mut alive_roots: Vec<_> = self.accounts_index.all_alive_roots();
8856        #[allow(clippy::stable_sort_primitive)]
8857        alive_roots.sort();
8858        info!("{}: accounts_index alive_roots: {:?}", label, alive_roots,);
8859        let full_pubkey_range = Pubkey::from([0; 32])..=Pubkey::from([0xff; 32]);
8860
8861        self.accounts_index.account_maps.iter().for_each(|map| {
8862            for (pubkey, account_entry) in map.items(&full_pubkey_range) {
8863                info!("  key: {} ref_count: {}", pubkey, account_entry.ref_count(),);
8864                info!(
8865                    "      slots: {:?}",
8866                    *account_entry.slot_list.read().unwrap()
8867                );
8868            }
8869        });
8870    }
8871
8872    fn print_count_and_status(&self, label: &str) {
8873        let mut slots: Vec<_> = self.storage.all_slots();
8874        #[allow(clippy::stable_sort_primitive)]
8875        slots.sort();
8876        info!("{}: count_and status for {} slots:", label, slots.len());
8877        for slot in &slots {
8878            let slot_stores = self.storage.get_slot_stores(*slot).unwrap();
8879            let r_slot_stores = slot_stores.read().unwrap();
8880            let mut ids: Vec<_> = r_slot_stores.keys().cloned().collect();
8881            #[allow(clippy::stable_sort_primitive)]
8882            ids.sort();
8883            for id in &ids {
8884                let entry = r_slot_stores.get(id).unwrap();
8885                info!(
8886                    "  slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {}",
8887                    slot,
8888                    id,
8889                    *entry.count_and_status.read().unwrap(),
8890                    entry.approx_store_count.load(Ordering::Relaxed),
8891                    entry.accounts.len(),
8892                    entry.accounts.capacity(),
8893                );
8894            }
8895        }
8896    }
8897}
8898
8899#[cfg(test)]
8900impl AccountsDb {
8901    pub fn new(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
8902        Self::new_for_tests(paths, cluster_type)
8903    }
8904
8905    pub fn new_with_config_for_tests(
8906        paths: Vec<PathBuf>,
8907        cluster_type: &ClusterType,
8908        account_indexes: AccountSecondaryIndexes,
8909        caching_enabled: bool,
8910        shrink_ratio: AccountShrinkThreshold,
8911    ) -> Self {
8912        Self::new_with_config(
8913            paths,
8914            cluster_type,
8915            account_indexes,
8916            caching_enabled,
8917            shrink_ratio,
8918            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
8919            None,
8920        )
8921    }
8922
8923    pub fn new_sized(paths: Vec<PathBuf>, file_size: u64) -> Self {
8924        AccountsDb {
8925            file_size,
8926            ..AccountsDb::new(paths, &ClusterType::Development)
8927        }
8928    }
8929
8930    pub fn new_sized_no_extra_stores(paths: Vec<PathBuf>, file_size: u64) -> Self {
8931        AccountsDb {
8932            file_size,
8933            min_num_stores: 0,
8934            ..AccountsDb::new(paths, &ClusterType::Development)
8935        }
8936    }
8937
8938    pub fn get_append_vec_id(&self, pubkey: &Pubkey, slot: Slot) -> Option<AppendVecId> {
8939        let ancestors = vec![(slot, 1)].into_iter().collect();
8940        let result = self.accounts_index.get(pubkey, Some(&ancestors), None);
8941        result.map(|(list, index)| list.slot_list()[index].1.store_id())
8942    }
8943
8944    pub fn alive_account_count_in_slot(&self, slot: Slot) -> usize {
8945        self.storage
8946            .get_slot_stores(slot)
8947            .map(|storages| storages.read().unwrap().values().map(|s| s.count()).sum())
8948            .unwrap_or(0)
8949    }
8950}
8951
8952/// Legacy shrink functions to support non-cached path.
8953/// Should be able to be deleted after cache path is the only path.
8954impl AccountsDb {
8955    // Reads all accounts in given slot's AppendVecs and filter only to alive,
8956    // then create a minimum AppendVec filled with the alive.
8957    // v1 path shrinks all stores in the slot
8958    //
8959    // Requires all stores in the slot to be re-written otherwise the accounts_index
8960    // store ref count could become incorrect.
8961    fn do_shrink_slot_v1(&self, slot: Slot, forced: bool) -> usize {
8962        trace!("shrink_stale_slot: slot: {}", slot);
8963
8964        if let Some(stores_lock) = self.storage.get_slot_stores(slot) {
8965            let stores: Vec<_> = stores_lock.read().unwrap().values().cloned().collect();
8966            let mut alive_count = 0;
8967            let mut stored_count = 0;
8968            let mut written_bytes = 0;
8969            let mut total_bytes = 0;
8970            for store in &stores {
8971                alive_count += store.count();
8972                stored_count += store.approx_stored_count();
8973                written_bytes += store.written_bytes();
8974                total_bytes += store.total_bytes();
8975            }
8976            if alive_count == stored_count && stores.len() == 1 {
8977                trace!(
8978                    "shrink_stale_slot ({}): not able to shrink at all: alive/stored: {} / {} {}",
8979                    slot,
8980                    alive_count,
8981                    stored_count,
8982                    if forced { " (forced)" } else { "" },
8983                );
8984                return 0;
8985            } else if !forced {
8986                let sparse_by_count = (alive_count as f32 / stored_count as f32) <= 0.8;
8987                let sparse_by_bytes = (written_bytes as f32 / total_bytes as f32) <= 0.8;
8988                let not_sparse = !sparse_by_count && !sparse_by_bytes;
8989                let too_small_to_shrink = total_bytes <= PAGE_SIZE;
8990                if not_sparse || too_small_to_shrink {
8991                    return 0;
8992                }
8993                info!(
8994                    "shrink_stale_slot ({}): not_sparse: {} count: {}/{} byte: {}/{}",
8995                    slot, not_sparse, alive_count, stored_count, written_bytes, total_bytes,
8996                );
8997            }
8998
8999            self.do_shrink_slot_stores(slot, stores.iter())
9000        } else {
9001            0
9002        }
9003    }
9004
9005    fn do_reset_uncleaned_roots_v1(
9006        &self,
9007        candidates: &mut MutexGuard<Vec<Slot>>,
9008        max_clean_root: Option<Slot>,
9009    ) {
9010        let previous_roots = self.accounts_index.reset_uncleaned_roots(max_clean_root);
9011        candidates.extend(previous_roots);
9012    }
9013
9014    #[cfg(test)]
9015    fn reset_uncleaned_roots_v1(&self) {
9016        self.do_reset_uncleaned_roots_v1(&mut self.shrink_candidate_slots_v1.lock().unwrap(), None);
9017    }
9018
9019    fn do_shrink_stale_slot_v1(&self, slot: Slot) -> usize {
9020        self.do_shrink_slot_v1(slot, false)
9021    }
9022    fn do_shrink_slot_forced_v1(&self, slot: Slot) {
9023        self.do_shrink_slot_v1(slot, true);
9024    }
9025
9026    fn shrink_stale_slot_v1(&self, candidates: &mut MutexGuard<Vec<Slot>>) -> usize {
9027        let mut shrunken_account_total = 0;
9028        let mut shrunk_slot_count = 0;
9029        let start = Instant::now();
9030        let num_roots = self.accounts_index.num_alive_roots();
9031        loop {
9032            if let Some(slot) = self.do_next_shrink_slot_v1(candidates) {
9033                shrunken_account_total += self.do_shrink_stale_slot_v1(slot);
9034            } else {
9035                return 0;
9036            }
9037            if start.elapsed().as_millis() > 100 || shrunk_slot_count > num_roots / 10 {
9038                debug!(
9039                    "do_shrink_stale_slot_v1: {} {} {}us",
9040                    shrunk_slot_count,
9041                    candidates.len(),
9042                    start.elapsed().as_micros()
9043                );
9044                break;
9045            }
9046            shrunk_slot_count += 1;
9047        }
9048        shrunken_account_total
9049    }
9050
9051    // Infinitely returns rooted roots in cyclic order
9052    fn do_next_shrink_slot_v1(&self, candidates: &mut MutexGuard<Vec<Slot>>) -> Option<Slot> {
9053        // At this point, a lock (= candidates) is ensured to be held to keep
9054        // do_reset_uncleaned_roots() (in clean_accounts()) from updating candidates.
9055        // Also, candidates in the lock may be swapped here if it's empty.
9056        let next = candidates.pop();
9057
9058        if next.is_some() {
9059            next
9060        } else {
9061            let mut new_all_slots = self.all_alive_roots_in_index();
9062            let next = new_all_slots.pop();
9063            // refresh candidates for later calls!
9064            **candidates = new_all_slots;
9065
9066            next
9067        }
9068    }
9069
9070    #[cfg(test)]
9071    fn next_shrink_slot_v1(&self) -> Option<Slot> {
9072        let mut candidates = self.shrink_candidate_slots_v1.lock().unwrap();
9073        self.do_next_shrink_slot_v1(&mut candidates)
9074    }
9075
9076    pub fn process_stale_slot_v1(&self) -> usize {
9077        let mut measure = Measure::start("stale_slot_shrink-ms");
9078        let candidates = self.shrink_candidate_slots_v1.try_lock();
9079        if candidates.is_err() {
9080            // skip and return immediately if locked by clean_accounts()
9081            // the calling background thread will just retry later.
9082            return 0;
9083        }
9084        // hold this lock as long as this shrinking process is running to avoid conflicts
9085        // with clean_accounts().
9086        let mut candidates = candidates.unwrap();
9087
9088        let count = self.shrink_stale_slot_v1(&mut candidates);
9089        measure.stop();
9090        inc_new_counter_info!("stale_slot_shrink-ms", measure.as_ms() as usize);
9091
9092        count
9093    }
9094
9095    #[cfg(test)]
9096    fn shrink_all_stale_slots_v1(&self) {
9097        for slot in self.all_slots_in_storage() {
9098            self.do_shrink_stale_slot_v1(slot);
9099        }
9100    }
9101}
9102
9103#[cfg(test)]
9104pub mod tests {
9105    use {
9106        super::*,
9107        crate::{
9108            accounts_hash::MERKLE_FANOUT,
9109            accounts_index::{tests::*, AccountSecondaryIndexesIncludeExclude, RefCount},
9110            append_vec::{test_utils::TempFile, AccountMeta},
9111            inline_safe_token,
9112        },
9113        assert_matches::assert_matches,
9114        rand::{prelude::SliceRandom, thread_rng, Rng},
9115        solana_sdk::{
9116            account::{
9117                accounts_equal, Account, AccountSharedData, ReadableAccount, WritableAccount,
9118            },
9119            hash::HASH_BYTES,
9120            pubkey::PUBKEY_BYTES,
9121        },
9122        std::{
9123            iter::FromIterator,
9124            str::FromStr,
9125            sync::atomic::AtomicBool,
9126            thread::{self, Builder, JoinHandle},
9127        },
9128    };
9129
9130    fn linear_ancestors(end_slot: u64) -> Ancestors {
9131        let mut ancestors: Ancestors = vec![(0, 0)].into_iter().collect();
9132        for i in 1..end_slot {
9133            ancestors.insert(i, (i - 1) as usize);
9134        }
9135        ancestors
9136    }
9137
9138    fn empty_storages<'a>() -> SortedStorages<'a> {
9139        SortedStorages::new(&[])
9140    }
9141
9142    impl AccountsDb {
9143        fn scan_snapshot_stores(
9144            &self,
9145            storage: &SortedStorages,
9146            stats: &mut crate::accounts_hash::HashStats,
9147            bins: usize,
9148            bin_range: &Range<usize>,
9149            check_hash: bool,
9150        ) -> Result<Vec<BinnedHashData>, BankHashVerificationError> {
9151            let temp_dir = TempDir::new().unwrap();
9152            let accounts_hash_cache_path = temp_dir.path();
9153            self.scan_snapshot_stores_with_cache(
9154                &CacheHashData::new(&accounts_hash_cache_path),
9155                storage,
9156                stats,
9157                bins,
9158                bin_range,
9159                &CalcAccountsHashConfig {
9160                    check_hash,
9161                    ..CalcAccountsHashConfig::default()
9162                },
9163                None,
9164            )
9165        }
9166    }
9167
9168    #[test]
9169    fn test_retain_roots_within_one_epoch_range() {
9170        let mut roots = vec![0, 1, 2];
9171        let slots_per_epoch = 2;
9172        AccountsDb::retain_roots_within_one_epoch_range(&mut roots, slots_per_epoch);
9173        assert_eq!(&vec![1, 2], &roots);
9174    }
9175
9176    #[test]
9177    #[should_panic(
9178        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
9179    )]
9180    fn test_accountsdb_scan_snapshot_stores_illegal_range_start() {
9181        let mut stats = HashStats::default();
9182        let bounds = Range { start: 2, end: 2 };
9183        let accounts_db = AccountsDb::new_single_for_tests();
9184
9185        accounts_db
9186            .scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false)
9187            .unwrap();
9188    }
9189    #[test]
9190    #[should_panic(
9191        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
9192    )]
9193    fn test_accountsdb_scan_snapshot_stores_illegal_range_end() {
9194        let mut stats = HashStats::default();
9195        let bounds = Range { start: 1, end: 3 };
9196
9197        let accounts_db = AccountsDb::new_single_for_tests();
9198        accounts_db
9199            .scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false)
9200            .unwrap();
9201    }
9202
9203    #[test]
9204    #[should_panic(
9205        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
9206    )]
9207    fn test_accountsdb_scan_snapshot_stores_illegal_range_inverse() {
9208        let mut stats = HashStats::default();
9209        let bounds = Range { start: 1, end: 0 };
9210
9211        let accounts_db = AccountsDb::new_single_for_tests();
9212        accounts_db
9213            .scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false)
9214            .unwrap();
9215    }
9216
9217    fn sample_storages_and_account_in_slot(
9218        slot: Slot,
9219    ) -> (SnapshotStorages, Vec<CalculateHashIntermediate>) {
9220        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
9221        let pubkey0 = Pubkey::from([0u8; 32]);
9222        let pubkey127 = Pubkey::from([0x7fu8; 32]);
9223        let pubkey128 = Pubkey::from([0x80u8; 32]);
9224        let pubkey255 = Pubkey::from([0xffu8; 32]);
9225
9226        let mut raw_expected = vec![
9227            CalculateHashIntermediate::new(Hash::default(), 1, pubkey0),
9228            CalculateHashIntermediate::new(Hash::default(), 128, pubkey127),
9229            CalculateHashIntermediate::new(Hash::default(), 129, pubkey128),
9230            CalculateHashIntermediate::new(Hash::default(), 256, pubkey255),
9231        ];
9232
9233        let expected_hashes = vec![
9234            Hash::from_str("5K3NW73xFHwgTWVe4LyCg4QfQda8f88uZj2ypDx2kmmH").unwrap(),
9235            Hash::from_str("84ozw83MZ8oeSF4hRAg7SeW1Tqs9LMXagX1BrDRjtZEx").unwrap(),
9236            Hash::from_str("5XqtnEJ41CG2JWNp7MAg9nxkRUAnyjLxfsKsdrLxQUbC").unwrap(),
9237            Hash::from_str("DpvwJcznzwULYh19Zu5CuAA4AT6WTBe4H6n15prATmqj").unwrap(),
9238        ];
9239
9240        let mut raw_accounts = Vec::default();
9241
9242        for i in 0..raw_expected.len() {
9243            raw_accounts.push(AccountSharedData::new(
9244                raw_expected[i].lamports,
9245                1,
9246                AccountSharedData::default().owner(),
9247            ));
9248            let hash = AccountsDb::hash_account(slot, &raw_accounts[i], &raw_expected[i].pubkey);
9249            if slot == 1 {
9250                assert_eq!(hash, expected_hashes[i]);
9251            }
9252            raw_expected[i].hash = hash;
9253        }
9254
9255        let to_store = raw_accounts
9256            .iter()
9257            .zip(raw_expected.iter())
9258            .map(|(account, intermediate)| (&intermediate.pubkey, account))
9259            .collect::<Vec<_>>();
9260
9261        accounts.store_uncached(slot, &to_store[..]);
9262        accounts.add_root(slot);
9263
9264        let (storages, slots) = accounts.get_snapshot_storages(slot, None, None);
9265        assert_eq!(storages.len(), slots.len());
9266        storages
9267            .iter()
9268            .zip(slots.iter())
9269            .for_each(|(storages, slot)| {
9270                for storage in storages {
9271                    assert_eq!(&storage.slot(), slot);
9272                }
9273            });
9274        (storages, raw_expected)
9275    }
9276
9277    fn sample_storages_and_accounts() -> (SnapshotStorages, Vec<CalculateHashIntermediate>) {
9278        sample_storages_and_account_in_slot(1)
9279    }
9280
9281    fn get_storage_refs(input: &[SnapshotStorage]) -> SortedStorages {
9282        SortedStorages::new(input)
9283    }
9284
9285    #[test]
9286    fn test_accountsdb_scan_snapshot_stores() {
9287        solana_logger::setup();
9288        let (storages, raw_expected) = sample_storages_and_accounts();
9289
9290        let bins = 1;
9291        let mut stats = HashStats::default();
9292
9293        let accounts_db = AccountsDb::new_single_for_tests();
9294        let result = accounts_db
9295            .scan_snapshot_stores(
9296                &get_storage_refs(&storages),
9297                &mut stats,
9298                bins,
9299                &Range {
9300                    start: 0,
9301                    end: bins,
9302                },
9303                false,
9304            )
9305            .unwrap();
9306        assert_eq!(result, vec![vec![raw_expected.clone()]]);
9307
9308        let bins = 2;
9309        let accounts_db = AccountsDb::new_single_for_tests();
9310        let result = accounts_db
9311            .scan_snapshot_stores(
9312                &get_storage_refs(&storages),
9313                &mut stats,
9314                bins,
9315                &Range {
9316                    start: 0,
9317                    end: bins,
9318                },
9319                false,
9320            )
9321            .unwrap();
9322        let mut expected = vec![Vec::new(); bins];
9323        expected[0].push(raw_expected[0].clone());
9324        expected[0].push(raw_expected[1].clone());
9325        expected[bins - 1].push(raw_expected[2].clone());
9326        expected[bins - 1].push(raw_expected[3].clone());
9327        assert_eq!(result, vec![expected]);
9328
9329        let bins = 4;
9330        let accounts_db = AccountsDb::new_single_for_tests();
9331        let result = accounts_db
9332            .scan_snapshot_stores(
9333                &get_storage_refs(&storages),
9334                &mut stats,
9335                bins,
9336                &Range {
9337                    start: 0,
9338                    end: bins,
9339                },
9340                false,
9341            )
9342            .unwrap();
9343        let mut expected = vec![Vec::new(); bins];
9344        expected[0].push(raw_expected[0].clone());
9345        expected[1].push(raw_expected[1].clone());
9346        expected[2].push(raw_expected[2].clone());
9347        expected[bins - 1].push(raw_expected[3].clone());
9348        assert_eq!(result, vec![expected]);
9349
9350        let bins = 256;
9351        let accounts_db = AccountsDb::new_single_for_tests();
9352        let result = accounts_db
9353            .scan_snapshot_stores(
9354                &get_storage_refs(&storages),
9355                &mut stats,
9356                bins,
9357                &Range {
9358                    start: 0,
9359                    end: bins,
9360                },
9361                false,
9362            )
9363            .unwrap();
9364        let mut expected = vec![Vec::new(); bins];
9365        expected[0].push(raw_expected[0].clone());
9366        expected[127].push(raw_expected[1].clone());
9367        expected[128].push(raw_expected[2].clone());
9368        expected[bins - 1].push(raw_expected.last().unwrap().clone());
9369        assert_eq!(result, vec![expected]);
9370    }
9371
9372    #[test]
9373    fn test_accountsdb_scan_snapshot_stores_2nd_chunk() {
9374        // enough stores to get to 2nd chunk
9375        let bins = 1;
9376        let slot = MAX_ITEMS_PER_CHUNK as Slot;
9377        let (storages, raw_expected) = sample_storages_and_account_in_slot(slot);
9378        let storage_data = vec![(&storages[0], slot)];
9379
9380        let sorted_storages =
9381            SortedStorages::new_debug(&storage_data[..], 0, MAX_ITEMS_PER_CHUNK as usize + 1);
9382
9383        let mut stats = HashStats::default();
9384        let accounts_db = AccountsDb::new_single_for_tests();
9385        let result = accounts_db
9386            .scan_snapshot_stores(
9387                &sorted_storages,
9388                &mut stats,
9389                bins,
9390                &Range {
9391                    start: 0,
9392                    end: bins,
9393                },
9394                false,
9395            )
9396            .unwrap();
9397        assert_eq!(result.len(), 2); // 2 chunks
9398        assert_eq!(result[0].len(), bins);
9399        assert_eq!(0, result[0].iter().map(|x| x.len()).sum::<usize>()); // nothing found in bin 0
9400        assert_eq!(result[1].len(), bins);
9401        assert_eq!(result[1], vec![raw_expected]);
9402    }
9403
9404    #[test]
9405    fn test_accountsdb_scan_snapshot_stores_binning() {
9406        let mut stats = HashStats::default();
9407        let (storages, raw_expected) = sample_storages_and_accounts();
9408
9409        // just the first bin of 2
9410        let bins = 2;
9411        let half_bins = bins / 2;
9412        let accounts_db = AccountsDb::new_single_for_tests();
9413        let result = accounts_db
9414            .scan_snapshot_stores(
9415                &get_storage_refs(&storages),
9416                &mut stats,
9417                bins,
9418                &Range {
9419                    start: 0,
9420                    end: half_bins,
9421                },
9422                false,
9423            )
9424            .unwrap();
9425        let mut expected = vec![Vec::new(); half_bins];
9426        expected[0].push(raw_expected[0].clone());
9427        expected[0].push(raw_expected[1].clone());
9428        assert_eq!(result, vec![expected]);
9429
9430        // just the second bin of 2
9431        let accounts_db = AccountsDb::new_single_for_tests();
9432        let result = accounts_db
9433            .scan_snapshot_stores(
9434                &get_storage_refs(&storages),
9435                &mut stats,
9436                bins,
9437                &Range {
9438                    start: 1,
9439                    end: bins,
9440                },
9441                false,
9442            )
9443            .unwrap();
9444
9445        let mut expected = vec![Vec::new(); half_bins];
9446        let starting_bin_index = 0;
9447        expected[starting_bin_index].push(raw_expected[2].clone());
9448        expected[starting_bin_index].push(raw_expected[3].clone());
9449        assert_eq!(result, vec![expected]);
9450
9451        // 1 bin at a time of 4
9452        let bins = 4;
9453        let accounts_db = AccountsDb::new_single_for_tests();
9454
9455        for (bin, expected_item) in raw_expected.iter().enumerate().take(bins) {
9456            let result = accounts_db
9457                .scan_snapshot_stores(
9458                    &get_storage_refs(&storages),
9459                    &mut stats,
9460                    bins,
9461                    &Range {
9462                        start: bin,
9463                        end: bin + 1,
9464                    },
9465                    false,
9466                )
9467                .unwrap();
9468            let mut expected = vec![Vec::new(); 1];
9469            expected[0].push(expected_item.clone());
9470            assert_eq!(result, vec![expected]);
9471        }
9472
9473        let bins = 256;
9474        let bin_locations = vec![0, 127, 128, 255];
9475        let range = 1;
9476        for bin in 0..bins {
9477            let accounts_db = AccountsDb::new_single_for_tests();
9478            let result = accounts_db
9479                .scan_snapshot_stores(
9480                    &get_storage_refs(&storages),
9481                    &mut stats,
9482                    bins,
9483                    &Range {
9484                        start: bin,
9485                        end: bin + range,
9486                    },
9487                    false,
9488                )
9489                .unwrap();
9490            let mut expected = vec![];
9491            if let Some(index) = bin_locations.iter().position(|&r| r == bin) {
9492                expected = vec![vec![Vec::new(); range]];
9493                expected[0][0].push(raw_expected[index].clone());
9494            }
9495            assert_eq!(result, expected);
9496        }
9497    }
9498
9499    #[test]
9500    fn test_accountsdb_scan_snapshot_stores_binning_2nd_chunk() {
9501        // enough stores to get to 2nd chunk
9502        // range is for only 1 bin out of 256.
9503        let bins = 256;
9504        let slot = MAX_ITEMS_PER_CHUNK as Slot;
9505        let (storages, raw_expected) = sample_storages_and_account_in_slot(slot);
9506        let storage_data = vec![(&storages[0], slot)];
9507
9508        let sorted_storages =
9509            SortedStorages::new_debug(&storage_data[..], 0, MAX_ITEMS_PER_CHUNK as usize + 1);
9510
9511        let mut stats = HashStats::default();
9512        let range = 1;
9513        let start = 127;
9514        let accounts_db = AccountsDb::new_single_for_tests();
9515        let result = accounts_db
9516            .scan_snapshot_stores(
9517                &sorted_storages,
9518                &mut stats,
9519                bins,
9520                &Range {
9521                    start,
9522                    end: start + range,
9523                },
9524                false,
9525            )
9526            .unwrap();
9527        assert_eq!(result.len(), 2); // 2 chunks
9528        assert_eq!(result[0].len(), range);
9529        assert_eq!(0, result[0].iter().map(|x| x.len()).sum::<usize>()); // nothing found in bin 0
9530        let mut expected = vec![Vec::new(); range];
9531        expected[0].push(raw_expected[1].clone());
9532        assert_eq!(result[1].len(), 1);
9533        assert_eq!(result[1], expected);
9534    }
9535
9536    #[test]
9537    fn test_accountsdb_calculate_accounts_hash_without_index_simple() {
9538        solana_logger::setup();
9539
9540        let (storages, _size, _slot_expected) = sample_storage();
9541        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9542        let result = db
9543            .calculate_accounts_hash_without_index(
9544                &CalcAccountsHashConfig::default(),
9545                &get_storage_refs(&storages),
9546                HashStats::default(),
9547            )
9548            .unwrap();
9549        let expected_hash = Hash::from_str("GKot5hBsd81kMupNCXHaqbhv3huEbxAFMLnpcX2hniwn").unwrap();
9550        assert_eq!(result, (expected_hash, 0));
9551    }
9552
9553    #[test]
9554    fn test_accountsdb_calculate_accounts_hash_without_index() {
9555        solana_logger::setup();
9556
9557        let (storages, raw_expected) = sample_storages_and_accounts();
9558        let expected_hash =
9559            AccountsHash::compute_merkle_root_loop(raw_expected.clone(), MERKLE_FANOUT, |item| {
9560                item.hash
9561            });
9562        let sum = raw_expected.iter().map(|item| item.lamports).sum();
9563        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9564        let result = db
9565            .calculate_accounts_hash_without_index(
9566                &CalcAccountsHashConfig::default(),
9567                &get_storage_refs(&storages),
9568                HashStats::default(),
9569            )
9570            .unwrap();
9571
9572        assert_eq!(result, (expected_hash, sum));
9573    }
9574
9575    fn sample_storage() -> (SnapshotStorages, usize, Slot) {
9576        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
9577        let slot_expected: Slot = 0;
9578        let size: usize = 123;
9579        let data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
9580
9581        let arc = Arc::new(data);
9582        let storages = vec![vec![arc]];
9583        (storages, size, slot_expected)
9584    }
9585
9586    #[derive(Clone)]
9587    struct TestScan {
9588        calls: Arc<AtomicU64>,
9589        pubkey: Pubkey,
9590        slot_expected: Slot,
9591        accum: BinnedHashData,
9592        current_slot: Slot,
9593        value_to_use_for_lamports: u64,
9594    }
9595
9596    impl AppendVecScan for TestScan {
9597        fn filter(&mut self, _pubkey: &Pubkey) -> bool {
9598            true
9599        }
9600        fn set_slot(&mut self, slot: Slot) {
9601            self.current_slot = slot;
9602        }
9603        fn init_accum(&mut self, _count: usize) {}
9604        fn get_accum(&mut self) -> BinnedHashData {
9605            std::mem::take(&mut self.accum)
9606        }
9607        fn set_accum(&mut self, accum: BinnedHashData) {
9608            self.accum = accum;
9609        }
9610        fn found_account(&mut self, loaded_account: &LoadedAccount) {
9611            self.calls.fetch_add(1, Ordering::Relaxed);
9612            assert_eq!(loaded_account.pubkey(), &self.pubkey);
9613            assert_eq!(self.slot_expected, self.current_slot);
9614            self.accum.push(vec![CalculateHashIntermediate::new(
9615                Hash::default(),
9616                self.value_to_use_for_lamports,
9617                self.pubkey,
9618            )]);
9619        }
9620        fn scanning_complete(self) -> BinnedHashData {
9621            self.accum
9622        }
9623    }
9624
9625    #[test]
9626    fn test_accountsdb_scan_account_storage_no_bank() {
9627        solana_logger::setup();
9628
9629        let expected = 1;
9630        let tf = crate::append_vec::test_utils::get_append_vec_path(
9631            "test_accountsdb_scan_account_storage_no_bank",
9632        );
9633        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
9634        let slot_expected: Slot = 0;
9635        let size: usize = 123;
9636        let mut data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
9637        let av = AppendVec::new(&tf.path, true, 1024 * 1024);
9638        data.accounts = av;
9639
9640        let arc = Arc::new(data);
9641        let storages = vec![vec![arc]];
9642        let pubkey = solana_sdk::pubkey::new_rand();
9643        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
9644        let sm = StoredMeta {
9645            data_len: 1,
9646            pubkey,
9647            write_version: 1,
9648        };
9649        storages[0][0]
9650            .accounts
9651            .append_accounts(&[(sm, Some(&acc))], &[&Hash::default()]);
9652
9653        let calls = Arc::new(AtomicU64::new(0));
9654        let temp_dir = TempDir::new().unwrap();
9655        let accounts_hash_cache_path = temp_dir.path();
9656        let accounts_db = AccountsDb::new_single_for_tests();
9657
9658        let test_scan = TestScan {
9659            calls: calls.clone(),
9660            pubkey,
9661            slot_expected,
9662            accum: Vec::default(),
9663            current_slot: 0,
9664            value_to_use_for_lamports: expected,
9665        };
9666
9667        let result = accounts_db.scan_account_storage_no_bank(
9668            &CacheHashData::new(&accounts_hash_cache_path),
9669            &CalcAccountsHashConfig::default(),
9670            &get_storage_refs(&storages),
9671            test_scan,
9672            &Range { start: 0, end: 1 },
9673            &PubkeyBinCalculator24::new(1),
9674            &HashStats::default(),
9675        );
9676        assert_eq!(calls.load(Ordering::Relaxed), 1);
9677        assert_eq!(
9678            result,
9679            vec![vec![vec![CalculateHashIntermediate::new(
9680                Hash::default(),
9681                expected,
9682                pubkey
9683            )]]]
9684        );
9685    }
9686
9687    #[test]
9688    fn test_accountsdb_scan_account_storage_no_bank_one_slot() {
9689        solana_logger::setup();
9690
9691        let expected = 1;
9692        let tf = crate::append_vec::test_utils::get_append_vec_path(
9693            "test_accountsdb_scan_account_storage_no_bank",
9694        );
9695        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
9696        let slot_expected: Slot = 0;
9697        let size: usize = 123;
9698        let mut data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
9699        let av = AppendVec::new(&tf.path, true, 1024 * 1024);
9700        data.accounts = av;
9701
9702        let arc = Arc::new(data);
9703        let storages = vec![vec![arc]];
9704        let pubkey = solana_sdk::pubkey::new_rand();
9705        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
9706        let sm = StoredMeta {
9707            data_len: 1,
9708            pubkey,
9709            write_version: 1,
9710        };
9711        storages[0][0]
9712            .accounts
9713            .append_accounts(&[(sm, Some(&acc))], &[&Hash::default()]);
9714
9715        let calls = Arc::new(AtomicU64::new(0));
9716
9717        let mut test_scan = TestScan {
9718            calls: calls.clone(),
9719            pubkey,
9720            slot_expected,
9721            accum: Vec::default(),
9722            current_slot: 0,
9723            value_to_use_for_lamports: expected,
9724        };
9725
9726        AccountsDb::scan_multiple_account_storages_one_slot(&storages[0], &mut test_scan);
9727        let accum = test_scan.scanning_complete();
9728        assert_eq!(calls.load(Ordering::Relaxed), 1);
9729        assert_eq!(
9730            accum
9731                .iter()
9732                .flatten()
9733                .map(|a| a.lamports)
9734                .collect::<Vec<_>>(),
9735            vec![expected]
9736        );
9737    }
9738
9739    fn sample_storage_with_entries(
9740        tf: &TempFile,
9741        write_version: StoredMetaWriteVersion,
9742        slot: Slot,
9743        pubkey: &Pubkey,
9744    ) -> SnapshotStorages {
9745        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
9746        let size: usize = 123;
9747        let mut data = AccountStorageEntry::new(&paths[0], slot, 0, size as u64);
9748        let av = AppendVec::new(&tf.path, true, 1024 * 1024);
9749        data.accounts = av;
9750
9751        let arc = Arc::new(data);
9752        let storages = vec![vec![arc]];
9753        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
9754        let sm = StoredMeta {
9755            data_len: 1,
9756            pubkey: *pubkey,
9757            write_version,
9758        };
9759        storages[0][0]
9760            .accounts
9761            .append_accounts(&[(sm, Some(&acc))], &[&Hash::default()]);
9762        storages
9763    }
9764
9765    #[test]
9766    fn test_accountsdb_scan_multiple_account_storage_no_bank_one_slot() {
9767        solana_logger::setup();
9768
9769        let slot_expected: Slot = 0;
9770        let tf = crate::append_vec::test_utils::get_append_vec_path(
9771            "test_accountsdb_scan_account_storage_no_bank",
9772        );
9773        let write_version1 = 0;
9774        let write_version2 = 1;
9775        let pubkey1 = solana_sdk::pubkey::new_rand();
9776        let pubkey2 = solana_sdk::pubkey::new_rand();
9777        for swap in [false, true].iter() {
9778            let mut storages = [
9779                sample_storage_with_entries(&tf, write_version1, slot_expected, &pubkey1)
9780                    .remove(0)
9781                    .remove(0),
9782                sample_storage_with_entries(&tf, write_version2, slot_expected, &pubkey2)
9783                    .remove(0)
9784                    .remove(0),
9785            ];
9786            if *swap {
9787                storages[..].swap(0, 1);
9788            }
9789            let calls = Arc::new(AtomicU64::new(0));
9790            let mut scanner = TestScanSimple {
9791                current_slot: 0,
9792                slot_expected,
9793                pubkey1,
9794                pubkey2,
9795                accum: Vec::default(),
9796                calls: calls.clone(),
9797                write_version1,
9798                write_version2,
9799            };
9800            AccountsDb::scan_multiple_account_storages_one_slot(&storages, &mut scanner);
9801            let accum = scanner.scanning_complete();
9802            assert_eq!(calls.load(Ordering::Relaxed), storages.len() as u64);
9803            assert_eq!(
9804                accum
9805                    .iter()
9806                    .flatten()
9807                    .map(|a| a.lamports)
9808                    .collect::<Vec<_>>(),
9809                vec![write_version1, write_version2]
9810            );
9811        }
9812    }
9813
9814    #[derive(Clone)]
9815    struct TestScanSimple {
9816        current_slot: Slot,
9817        slot_expected: Slot,
9818        calls: Arc<AtomicU64>,
9819        accum: BinnedHashData,
9820        pubkey1: Pubkey,
9821        pubkey2: Pubkey,
9822        write_version1: u64,
9823        write_version2: u64,
9824    }
9825
9826    impl AppendVecScan for TestScanSimple {
9827        fn set_slot(&mut self, slot: Slot) {
9828            self.current_slot = slot;
9829        }
9830        fn filter(&mut self, _pubkey: &Pubkey) -> bool {
9831            true
9832        }
9833        fn init_accum(&mut self, _count: usize) {}
9834        fn found_account(&mut self, loaded_account: &LoadedAccount) {
9835            self.calls.fetch_add(1, Ordering::Relaxed);
9836            let write_version = loaded_account.write_version();
9837            let first =
9838                loaded_account.pubkey() == &self.pubkey1 && write_version == self.write_version1;
9839            assert!(
9840                first
9841                    || loaded_account.pubkey() == &self.pubkey2
9842                        && write_version == self.write_version2
9843            );
9844            assert_eq!(self.slot_expected, self.current_slot);
9845            if first {
9846                assert!(self.accum.is_empty());
9847            } else {
9848                assert!(self.accum.len() == 1);
9849            }
9850            self.accum.push(vec![CalculateHashIntermediate {
9851                hash: Hash::default(),
9852                lamports: write_version,
9853                pubkey: Pubkey::default(),
9854            }]);
9855        }
9856        fn scanning_complete(self) -> BinnedHashData {
9857            self.accum
9858        }
9859        fn get_accum(&mut self) -> BinnedHashData {
9860            std::mem::take(&mut self.accum)
9861        }
9862        fn set_accum(&mut self, accum: BinnedHashData) {
9863            self.accum = accum;
9864        }
9865    }
9866
9867    #[test]
9868    fn test_accountsdb_add_root() {
9869        solana_logger::setup();
9870        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9871        let key = Pubkey::default();
9872        let account0 = AccountSharedData::new(1, 0, &key);
9873
9874        db.store_uncached(0, &[(&key, &account0)]);
9875        db.add_root(0);
9876        let ancestors = vec![(1, 1)].into_iter().collect();
9877        assert_eq!(
9878            db.load_without_fixed_root(&ancestors, &key),
9879            Some((account0, 0))
9880        );
9881    }
9882
9883    #[test]
9884    fn test_accountsdb_latest_ancestor() {
9885        solana_logger::setup();
9886        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9887        let key = Pubkey::default();
9888        let account0 = AccountSharedData::new(1, 0, &key);
9889
9890        db.store_uncached(0, &[(&key, &account0)]);
9891
9892        let account1 = AccountSharedData::new(0, 0, &key);
9893        db.store_uncached(1, &[(&key, &account1)]);
9894
9895        let ancestors = vec![(1, 1)].into_iter().collect();
9896        assert_eq!(
9897            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
9898            &account1
9899        );
9900
9901        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
9902        assert_eq!(
9903            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
9904            &account1
9905        );
9906
9907        let accounts: Vec<AccountSharedData> = db.unchecked_scan_accounts(
9908            "",
9909            &ancestors,
9910            |accounts: &mut Vec<AccountSharedData>, option| {
9911                accounts.push(option.1.take_account());
9912            },
9913            &ScanConfig::default(),
9914        );
9915        assert_eq!(accounts, vec![account1]);
9916    }
9917
9918    #[test]
9919    fn test_accountsdb_latest_ancestor_with_root() {
9920        solana_logger::setup();
9921        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9922        let key = Pubkey::default();
9923        let account0 = AccountSharedData::new(1, 0, &key);
9924
9925        db.store_uncached(0, &[(&key, &account0)]);
9926
9927        let account1 = AccountSharedData::new(0, 0, &key);
9928        db.store_uncached(1, &[(&key, &account1)]);
9929        db.add_root(0);
9930
9931        let ancestors = vec![(1, 1)].into_iter().collect();
9932        assert_eq!(
9933            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
9934            &account1
9935        );
9936
9937        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
9938        assert_eq!(
9939            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
9940            &account1
9941        );
9942    }
9943
9944    #[test]
9945    fn test_accountsdb_root_one_slot() {
9946        solana_logger::setup();
9947        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9948
9949        let key = Pubkey::default();
9950        let account0 = AccountSharedData::new(1, 0, &key);
9951
9952        // store value 1 in the "root", i.e. db zero
9953        db.store_uncached(0, &[(&key, &account0)]);
9954
9955        // now we have:
9956        //
9957        //                       root0 -> key.lamports==1
9958        //                        / \
9959        //                       /   \
9960        //  key.lamports==0 <- slot1    \
9961        //                             slot2 -> key.lamports==1
9962        //                                       (via root0)
9963
9964        // store value 0 in one child
9965        let account1 = AccountSharedData::new(0, 0, &key);
9966        db.store_uncached(1, &[(&key, &account1)]);
9967
9968        // masking accounts is done at the Accounts level, at accountsDB we see
9969        // original account (but could also accept "None", which is implemented
9970        // at the Accounts level)
9971        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
9972        assert_eq!(
9973            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
9974            &account1
9975        );
9976
9977        // we should see 1 token in slot 2
9978        let ancestors = vec![(0, 0), (2, 2)].into_iter().collect();
9979        assert_eq!(
9980            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
9981            &account0
9982        );
9983
9984        db.add_root(0);
9985
9986        let ancestors = vec![(1, 1)].into_iter().collect();
9987        assert_eq!(
9988            db.load_without_fixed_root(&ancestors, &key),
9989            Some((account1, 1))
9990        );
9991        let ancestors = vec![(2, 2)].into_iter().collect();
9992        assert_eq!(
9993            db.load_without_fixed_root(&ancestors, &key),
9994            Some((account0, 0))
9995        ); // original value
9996    }
9997
9998    #[test]
9999    fn test_accountsdb_add_root_many() {
10000        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10001
10002        let mut pubkeys: Vec<Pubkey> = vec![];
10003        create_account(&db, &mut pubkeys, 0, 100, 0, 0);
10004        for _ in 1..100 {
10005            let idx = thread_rng().gen_range(0, 99);
10006            let ancestors = vec![(0, 0)].into_iter().collect();
10007            let account = db
10008                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10009                .unwrap();
10010            let default_account = AccountSharedData::from(Account {
10011                lamports: (idx + 1) as u64,
10012                ..Account::default()
10013            });
10014            assert_eq!((default_account, 0), account);
10015        }
10016
10017        db.add_root(0);
10018
10019        // check that all the accounts appear with a new root
10020        for _ in 1..100 {
10021            let idx = thread_rng().gen_range(0, 99);
10022            let ancestors = vec![(0, 0)].into_iter().collect();
10023            let account0 = db
10024                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10025                .unwrap();
10026            let ancestors = vec![(1, 1)].into_iter().collect();
10027            let account1 = db
10028                .load_without_fixed_root(&ancestors, &pubkeys[idx])
10029                .unwrap();
10030            let default_account = AccountSharedData::from(Account {
10031                lamports: (idx + 1) as u64,
10032                ..Account::default()
10033            });
10034            assert_eq!(&default_account, &account0.0);
10035            assert_eq!(&default_account, &account1.0);
10036        }
10037    }
10038
10039    #[test]
10040    fn test_accountsdb_count_stores() {
10041        solana_logger::setup();
10042        let db = AccountsDb::new_single_for_tests();
10043
10044        let mut pubkeys: Vec<Pubkey> = vec![];
10045        create_account(&db, &mut pubkeys, 0, 2, DEFAULT_FILE_SIZE as usize / 3, 0);
10046        assert!(check_storage(&db, 0, 2));
10047
10048        let pubkey = solana_sdk::pubkey::new_rand();
10049        let account = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 3, &pubkey);
10050        db.store_uncached(1, &[(&pubkey, &account)]);
10051        db.store_uncached(1, &[(&pubkeys[0], &account)]);
10052        {
10053            let slot_0_stores = &db.storage.get_slot_stores(0).unwrap();
10054            let slot_1_stores = &db.storage.get_slot_stores(1).unwrap();
10055            let r_slot_0_stores = slot_0_stores.read().unwrap();
10056            let r_slot_1_stores = slot_1_stores.read().unwrap();
10057            assert_eq!(r_slot_0_stores.len(), 1);
10058            assert_eq!(r_slot_1_stores.len(), 1);
10059            assert_eq!(r_slot_0_stores.get(&0).unwrap().count(), 2);
10060            assert_eq!(r_slot_1_stores[&1].count(), 2);
10061            assert_eq!(r_slot_0_stores.get(&0).unwrap().approx_stored_count(), 2);
10062            assert_eq!(r_slot_1_stores[&1].approx_stored_count(), 2);
10063        }
10064
10065        // adding root doesn't change anything
10066        db.get_accounts_delta_hash(1);
10067        db.add_root(1);
10068        {
10069            let slot_0_stores = &db.storage.get_slot_stores(0).unwrap();
10070            let slot_1_stores = &db.storage.get_slot_stores(1).unwrap();
10071            let r_slot_0_stores = slot_0_stores.read().unwrap();
10072            let r_slot_1_stores = slot_1_stores.read().unwrap();
10073            assert_eq!(r_slot_0_stores.len(), 1);
10074            assert_eq!(r_slot_1_stores.len(), 1);
10075            assert_eq!(r_slot_0_stores.get(&0).unwrap().count(), 2);
10076            assert_eq!(r_slot_1_stores[&1].count(), 2);
10077            assert_eq!(r_slot_0_stores.get(&0).unwrap().approx_stored_count(), 2);
10078            assert_eq!(r_slot_1_stores[&1].approx_stored_count(), 2);
10079        }
10080
10081        // overwrite old rooted account version; only the r_slot_0_stores.count() should be
10082        // decremented
10083        db.store_uncached(2, &[(&pubkeys[0], &account)]);
10084        db.clean_accounts(None, false, None);
10085        {
10086            let slot_0_stores = &db.storage.get_slot_stores(0).unwrap();
10087            let slot_1_stores = &db.storage.get_slot_stores(1).unwrap();
10088            let r_slot_0_stores = slot_0_stores.read().unwrap();
10089            let r_slot_1_stores = slot_1_stores.read().unwrap();
10090            assert_eq!(r_slot_0_stores.len(), 1);
10091            assert_eq!(r_slot_1_stores.len(), 1);
10092            assert_eq!(r_slot_0_stores.get(&0).unwrap().count(), 1);
10093            assert_eq!(r_slot_1_stores[&1].count(), 2);
10094            assert_eq!(r_slot_0_stores.get(&0).unwrap().approx_stored_count(), 2);
10095            assert_eq!(r_slot_1_stores[&1].approx_stored_count(), 2);
10096        }
10097    }
10098
10099    #[test]
10100    fn test_accounts_unsquashed() {
10101        let key = Pubkey::default();
10102
10103        // 1 token in the "root", i.e. db zero
10104        let db0 = AccountsDb::new(Vec::new(), &ClusterType::Development);
10105        let account0 = AccountSharedData::new(1, 0, &key);
10106        db0.store_uncached(0, &[(&key, &account0)]);
10107
10108        // 0 lamports in the child
10109        let account1 = AccountSharedData::new(0, 0, &key);
10110        db0.store_uncached(1, &[(&key, &account1)]);
10111
10112        // masking accounts is done at the Accounts level, at accountsDB we see
10113        // original account
10114        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
10115        assert_eq!(
10116            db0.load_without_fixed_root(&ancestors, &key),
10117            Some((account1, 1))
10118        );
10119        let ancestors = vec![(0, 0)].into_iter().collect();
10120        assert_eq!(
10121            db0.load_without_fixed_root(&ancestors, &key),
10122            Some((account0, 0))
10123        );
10124    }
10125
10126    fn run_test_remove_unrooted_slot(is_cached: bool) {
10127        let unrooted_slot = 9;
10128        let unrooted_bank_id = 9;
10129        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10130        db.caching_enabled = true;
10131        let key = Pubkey::default();
10132        let account0 = AccountSharedData::new(1, 0, &key);
10133        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
10134        if is_cached {
10135            db.store_cached((unrooted_slot, &[(&key, &account0)][..]), None);
10136        } else {
10137            db.store_uncached(unrooted_slot, &[(&key, &account0)]);
10138        }
10139        db.bank_hashes
10140            .write()
10141            .unwrap()
10142            .insert(unrooted_slot, BankHashInfo::default());
10143        assert!(db
10144            .accounts_index
10145            .get(&key, Some(&ancestors), None)
10146            .is_some());
10147        assert_load_account(&db, unrooted_slot, key, 1);
10148
10149        // Purge the slot
10150        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
10151        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
10152        assert!(db.bank_hashes.read().unwrap().get(&unrooted_slot).is_none());
10153        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_none());
10154        assert!(db.storage.map.get(&unrooted_slot).is_none());
10155        assert!(db.accounts_index.get_account_read_entry(&key).is_none());
10156        assert!(db
10157            .accounts_index
10158            .get(&key, Some(&ancestors), None)
10159            .is_none());
10160
10161        // Test we can store for the same slot again and get the right information
10162        let account0 = AccountSharedData::new(2, 0, &key);
10163        db.store_uncached(unrooted_slot, &[(&key, &account0)]);
10164        assert_load_account(&db, unrooted_slot, key, 2);
10165    }
10166
10167    #[test]
10168    fn test_remove_unrooted_slot_cached() {
10169        run_test_remove_unrooted_slot(true);
10170    }
10171
10172    #[test]
10173    fn test_remove_unrooted_slot_storage() {
10174        run_test_remove_unrooted_slot(false);
10175    }
10176
10177    #[test]
10178    fn test_remove_unrooted_slot_snapshot() {
10179        solana_logger::setup();
10180        let unrooted_slot = 9;
10181        let unrooted_bank_id = 9;
10182        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10183        let key = solana_sdk::pubkey::new_rand();
10184        let account0 = AccountSharedData::new(1, 0, &key);
10185        db.store_uncached(unrooted_slot, &[(&key, &account0)]);
10186
10187        // Purge the slot
10188        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
10189
10190        // Add a new root
10191        let key2 = solana_sdk::pubkey::new_rand();
10192        let new_root = unrooted_slot + 1;
10193        db.store_uncached(new_root, &[(&key2, &account0)]);
10194        db.add_root(new_root);
10195
10196        // Simulate reconstruction from snapshot
10197        let db = reconstruct_accounts_db_via_serialization(&db, new_root);
10198
10199        // Check root account exists
10200        assert_load_account(&db, new_root, key2, 1);
10201
10202        // Check purged account stays gone
10203        let unrooted_slot_ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
10204        assert!(db
10205            .load_without_fixed_root(&unrooted_slot_ancestors, &key)
10206            .is_none());
10207    }
10208
10209    fn create_account(
10210        accounts: &AccountsDb,
10211        pubkeys: &mut Vec<Pubkey>,
10212        slot: Slot,
10213        num: usize,
10214        space: usize,
10215        num_vote: usize,
10216    ) {
10217        let ancestors = vec![(slot, 0)].into_iter().collect();
10218        for t in 0..num {
10219            let pubkey = solana_sdk::pubkey::new_rand();
10220            let account =
10221                AccountSharedData::new((t + 1) as u64, space, AccountSharedData::default().owner());
10222            pubkeys.push(pubkey);
10223            assert!(accounts
10224                .load_without_fixed_root(&ancestors, &pubkey)
10225                .is_none());
10226            accounts.store_uncached(slot, &[(&pubkey, &account)]);
10227        }
10228        for t in 0..num_vote {
10229            let pubkey = solana_sdk::pubkey::new_rand();
10230            let account =
10231                AccountSharedData::new((num + t + 1) as u64, space, &solana_vote_program::id());
10232            pubkeys.push(pubkey);
10233            let ancestors = vec![(slot, 0)].into_iter().collect();
10234            assert!(accounts
10235                .load_without_fixed_root(&ancestors, &pubkey)
10236                .is_none());
10237            accounts.store_uncached(slot, &[(&pubkey, &account)]);
10238        }
10239    }
10240
10241    fn update_accounts(accounts: &AccountsDb, pubkeys: &[Pubkey], slot: Slot, range: usize) {
10242        for _ in 1..1000 {
10243            let idx = thread_rng().gen_range(0, range);
10244            let ancestors = vec![(slot, 0)].into_iter().collect();
10245            if let Some((mut account, _)) =
10246                accounts.load_without_fixed_root(&ancestors, &pubkeys[idx])
10247            {
10248                account.checked_add_lamports(1).unwrap();
10249                accounts.store_uncached(slot, &[(&pubkeys[idx], &account)]);
10250                if account.is_zero_lamport() {
10251                    let ancestors = vec![(slot, 0)].into_iter().collect();
10252                    assert!(accounts
10253                        .load_without_fixed_root(&ancestors, &pubkeys[idx])
10254                        .is_none());
10255                } else {
10256                    let default_account = AccountSharedData::from(Account {
10257                        lamports: account.lamports(),
10258                        ..Account::default()
10259                    });
10260                    assert_eq!(default_account, account);
10261                }
10262            }
10263        }
10264    }
10265
10266    fn check_storage(accounts: &AccountsDb, slot: Slot, count: usize) -> bool {
10267        assert_eq!(
10268            accounts
10269                .storage
10270                .get_slot_stores(slot)
10271                .unwrap()
10272                .read()
10273                .unwrap()
10274                .len(),
10275            1
10276        );
10277        let slot_storages = accounts.storage.get_slot_stores(slot).unwrap();
10278        let mut total_count: usize = 0;
10279        let r_slot_storages = slot_storages.read().unwrap();
10280        for store in r_slot_storages.values() {
10281            assert_eq!(store.status(), AccountStorageStatus::Available);
10282            total_count += store.count();
10283        }
10284        assert_eq!(total_count, count);
10285        let (expected_store_count, actual_store_count): (usize, usize) = (
10286            r_slot_storages
10287                .values()
10288                .map(|s| s.approx_stored_count())
10289                .sum(),
10290            r_slot_storages
10291                .values()
10292                .map(|s| s.all_accounts().len())
10293                .sum(),
10294        );
10295        assert_eq!(expected_store_count, actual_store_count);
10296        total_count == count
10297    }
10298
10299    fn check_accounts(
10300        accounts: &AccountsDb,
10301        pubkeys: &[Pubkey],
10302        slot: Slot,
10303        num: usize,
10304        count: usize,
10305    ) {
10306        let ancestors = vec![(slot, 0)].into_iter().collect();
10307        for _ in 0..num {
10308            let idx = thread_rng().gen_range(0, num);
10309            let account = accounts.load_without_fixed_root(&ancestors, &pubkeys[idx]);
10310            let account1 = Some((
10311                AccountSharedData::new(
10312                    (idx + count) as u64,
10313                    0,
10314                    AccountSharedData::default().owner(),
10315                ),
10316                slot,
10317            ));
10318            assert_eq!(account, account1);
10319        }
10320    }
10321
10322    #[allow(clippy::needless_range_loop)]
10323    fn modify_accounts(
10324        accounts: &AccountsDb,
10325        pubkeys: &[Pubkey],
10326        slot: Slot,
10327        num: usize,
10328        count: usize,
10329    ) {
10330        for idx in 0..num {
10331            let account = AccountSharedData::new(
10332                (idx + count) as u64,
10333                0,
10334                AccountSharedData::default().owner(),
10335            );
10336            accounts.store_uncached(slot, &[(&pubkeys[idx], &account)]);
10337        }
10338    }
10339
10340    #[test]
10341    fn test_account_one() {
10342        let (_accounts_dirs, paths) = get_temp_accounts_paths(1).unwrap();
10343        let db = AccountsDb::new(paths, &ClusterType::Development);
10344        let mut pubkeys: Vec<Pubkey> = vec![];
10345        create_account(&db, &mut pubkeys, 0, 1, 0, 0);
10346        let ancestors = vec![(0, 0)].into_iter().collect();
10347        let account = db.load_without_fixed_root(&ancestors, &pubkeys[0]).unwrap();
10348        let default_account = AccountSharedData::from(Account {
10349            lamports: 1,
10350            ..Account::default()
10351        });
10352        assert_eq!((default_account, 0), account);
10353    }
10354
10355    #[test]
10356    fn test_account_many() {
10357        let (_accounts_dirs, paths) = get_temp_accounts_paths(2).unwrap();
10358        let db = AccountsDb::new(paths, &ClusterType::Development);
10359        let mut pubkeys: Vec<Pubkey> = vec![];
10360        create_account(&db, &mut pubkeys, 0, 100, 0, 0);
10361        check_accounts(&db, &pubkeys, 0, 100, 1);
10362    }
10363
10364    #[test]
10365    fn test_account_update() {
10366        let accounts = AccountsDb::new_single_for_tests();
10367        let mut pubkeys: Vec<Pubkey> = vec![];
10368        create_account(&accounts, &mut pubkeys, 0, 100, 0, 0);
10369        update_accounts(&accounts, &pubkeys, 0, 99);
10370        assert!(check_storage(&accounts, 0, 100));
10371    }
10372
10373    #[test]
10374    fn test_account_grow_many() {
10375        let (_accounts_dir, paths) = get_temp_accounts_paths(2).unwrap();
10376        let size = 4096;
10377        let accounts = AccountsDb::new_sized(paths, size);
10378        let mut keys = vec![];
10379        for i in 0..9 {
10380            let key = solana_sdk::pubkey::new_rand();
10381            let account = AccountSharedData::new(i + 1, size as usize / 4, &key);
10382            accounts.store_uncached(0, &[(&key, &account)]);
10383            keys.push(key);
10384        }
10385        let ancestors = vec![(0, 0)].into_iter().collect();
10386        for (i, key) in keys.iter().enumerate() {
10387            assert_eq!(
10388                accounts
10389                    .load_without_fixed_root(&ancestors, key)
10390                    .unwrap()
10391                    .0
10392                    .lamports(),
10393                (i as u64) + 1
10394            );
10395        }
10396
10397        let mut append_vec_histogram = HashMap::new();
10398        let mut all_storages = vec![];
10399        for slot_storage in accounts.storage.map.iter() {
10400            all_storages.extend(slot_storage.read().unwrap().values().cloned())
10401        }
10402        for storage in all_storages {
10403            *append_vec_histogram.entry(storage.slot()).or_insert(0) += 1;
10404        }
10405        for count in append_vec_histogram.values() {
10406            assert!(*count >= 2);
10407        }
10408    }
10409
10410    #[test]
10411    fn test_account_grow() {
10412        let accounts = AccountsDb::new_single_for_tests();
10413
10414        let status = [AccountStorageStatus::Available, AccountStorageStatus::Full];
10415        let pubkey1 = solana_sdk::pubkey::new_rand();
10416        let account1 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey1);
10417        accounts.store_uncached(0, &[(&pubkey1, &account1)]);
10418        {
10419            let stores = &accounts.storage.get_slot_stores(0).unwrap();
10420            let r_stores = stores.read().unwrap();
10421            assert_eq!(r_stores.len(), 1);
10422            assert_eq!(r_stores[&0].count(), 1);
10423            assert_eq!(r_stores[&0].status(), AccountStorageStatus::Available);
10424        }
10425
10426        let pubkey2 = solana_sdk::pubkey::new_rand();
10427        let account2 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey2);
10428        accounts.store_uncached(0, &[(&pubkey2, &account2)]);
10429        {
10430            assert_eq!(accounts.storage.map.len(), 1);
10431            let stores = &accounts.storage.get_slot_stores(0).unwrap();
10432            let r_stores = stores.read().unwrap();
10433            assert_eq!(r_stores.len(), 2);
10434            assert_eq!(r_stores[&0].count(), 1);
10435            assert_eq!(r_stores[&0].status(), AccountStorageStatus::Full);
10436            assert_eq!(r_stores[&1].count(), 1);
10437            assert_eq!(r_stores[&1].status(), AccountStorageStatus::Available);
10438        }
10439        let ancestors = vec![(0, 0)].into_iter().collect();
10440        assert_eq!(
10441            accounts
10442                .load_without_fixed_root(&ancestors, &pubkey1)
10443                .unwrap()
10444                .0,
10445            account1
10446        );
10447        assert_eq!(
10448            accounts
10449                .load_without_fixed_root(&ancestors, &pubkey2)
10450                .unwrap()
10451                .0,
10452            account2
10453        );
10454
10455        // lots of stores, but 7 storages should be enough for everything
10456        for _ in 0..25 {
10457            accounts.store_uncached(0, &[(&pubkey1, &account1)]);
10458            {
10459                assert_eq!(accounts.storage.map.len(), 1);
10460                let stores = &accounts.storage.get_slot_stores(0).unwrap();
10461                let r_stores = stores.read().unwrap();
10462                assert!(r_stores.len() <= 7);
10463                assert_eq!(r_stores[&0].status(), status[0]);
10464            }
10465            let ancestors = vec![(0, 0)].into_iter().collect();
10466            assert_eq!(
10467                accounts
10468                    .load_without_fixed_root(&ancestors, &pubkey1)
10469                    .unwrap()
10470                    .0,
10471                account1
10472            );
10473            assert_eq!(
10474                accounts
10475                    .load_without_fixed_root(&ancestors, &pubkey2)
10476                    .unwrap()
10477                    .0,
10478                account2
10479            );
10480        }
10481    }
10482
10483    #[test]
10484    fn test_lazy_gc_slot() {
10485        solana_logger::setup();
10486        //This test is pedantic
10487        //A slot is purged when a non root bank is cleaned up.  If a slot is behind root but it is
10488        //not root, it means we are retaining dead banks.
10489        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10490        let pubkey = solana_sdk::pubkey::new_rand();
10491        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10492        //store an account
10493        accounts.store_uncached(0, &[(&pubkey, &account)]);
10494        let ancestors = vec![(0, 0)].into_iter().collect();
10495        let id = {
10496            let (lock, idx) = accounts
10497                .accounts_index
10498                .get_for_tests(&pubkey, Some(&ancestors), None)
10499                .unwrap();
10500            lock.slot_list()[idx].1.store_id()
10501        };
10502        accounts.get_accounts_delta_hash(0);
10503        accounts.add_root(1);
10504
10505        //slot is still there, since gc is lazy
10506        assert!(accounts
10507            .storage
10508            .get_slot_stores(0)
10509            .unwrap()
10510            .read()
10511            .unwrap()
10512            .get(&id)
10513            .is_some());
10514
10515        //store causes clean
10516        accounts.store_uncached(1, &[(&pubkey, &account)]);
10517
10518        // generate delta state for slot 1, so clean operates on it.
10519        accounts.get_accounts_delta_hash(1);
10520
10521        //slot is gone
10522        accounts.print_accounts_stats("pre-clean");
10523        accounts.clean_accounts(None, false, None);
10524        assert!(accounts.storage.map.get(&0).is_none());
10525
10526        //new value is there
10527        let ancestors = vec![(1, 1)].into_iter().collect();
10528        assert_eq!(
10529            accounts.load_without_fixed_root(&ancestors, &pubkey),
10530            Some((account, 1))
10531        );
10532    }
10533
10534    impl AccountsDb {
10535        fn all_account_count_in_append_vec(&self, slot: Slot) -> usize {
10536            let slot_storage = self.storage.get_slot_stores(slot);
10537            if let Some(slot_storage) = slot_storage {
10538                let r_slot_storage = slot_storage.read().unwrap();
10539                let count = r_slot_storage
10540                    .values()
10541                    .map(|store| store.all_accounts().len())
10542                    .sum();
10543                let stored_count: usize = r_slot_storage
10544                    .values()
10545                    .map(|store| store.approx_stored_count())
10546                    .sum();
10547                assert_eq!(stored_count, count);
10548                count
10549            } else {
10550                0
10551            }
10552        }
10553
10554        pub fn ref_count_for_pubkey(&self, pubkey: &Pubkey) -> RefCount {
10555            self.accounts_index.ref_count_from_storage(pubkey)
10556        }
10557    }
10558
10559    #[test]
10560    fn test_clean_zero_lamport_and_dead_slot() {
10561        solana_logger::setup();
10562
10563        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10564        let pubkey1 = solana_sdk::pubkey::new_rand();
10565        let pubkey2 = solana_sdk::pubkey::new_rand();
10566        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
10567        let zero_lamport_account =
10568            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10569
10570        // Store two accounts
10571        accounts.store_uncached(0, &[(&pubkey1, &account)]);
10572        accounts.store_uncached(0, &[(&pubkey2, &account)]);
10573
10574        // Make sure both accounts are in the same AppendVec in slot 0, which
10575        // will prevent pubkey1 from being cleaned up later even when it's a
10576        // zero-lamport account
10577        let ancestors = vec![(0, 1)].into_iter().collect();
10578        let (slot1, account_info1) = accounts
10579            .accounts_index
10580            .get_for_tests(&pubkey1, Some(&ancestors), None)
10581            .map(|(account_list1, index1)| account_list1.slot_list()[index1])
10582            .unwrap();
10583        let (slot2, account_info2) = accounts
10584            .accounts_index
10585            .get_for_tests(&pubkey2, Some(&ancestors), None)
10586            .map(|(account_list2, index2)| account_list2.slot_list()[index2])
10587            .unwrap();
10588        assert_eq!(slot1, 0);
10589        assert_eq!(slot1, slot2);
10590        assert_eq!(account_info1.store_id(), account_info2.store_id());
10591
10592        // Update account 1 in slot 1
10593        accounts.store_uncached(1, &[(&pubkey1, &account)]);
10594
10595        // Update account 1 as  zero lamports account
10596        accounts.store_uncached(2, &[(&pubkey1, &zero_lamport_account)]);
10597
10598        // Pubkey 1 was the only account in slot 1, and it was updated in slot 2, so
10599        // slot 1 should be purged
10600        accounts.add_root(0);
10601        accounts.add_root(1);
10602        accounts.add_root(2);
10603
10604        // Slot 1 should be removed, slot 0 cannot be removed because it still has
10605        // the latest update for pubkey 2
10606        accounts.clean_accounts(None, false, None);
10607        assert!(accounts.storage.get_slot_stores(0).is_some());
10608        assert!(accounts.storage.get_slot_stores(1).is_none());
10609
10610        // Slot 1 should be cleaned because all it's accounts are
10611        // zero lamports, and are not present in any other slot's
10612        // storage entries
10613        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
10614    }
10615
10616    #[test]
10617    fn test_clean_multiple_zero_lamport_decrements_index_ref_count() {
10618        solana_logger::setup();
10619
10620        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10621        let pubkey1 = solana_sdk::pubkey::new_rand();
10622        let pubkey2 = solana_sdk::pubkey::new_rand();
10623        let zero_lamport_account =
10624            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10625
10626        // Store 2 accounts in slot 0, then update account 1 in two more slots
10627        accounts.store_uncached(0, &[(&pubkey1, &zero_lamport_account)]);
10628        accounts.store_uncached(0, &[(&pubkey2, &zero_lamport_account)]);
10629        accounts.store_uncached(1, &[(&pubkey1, &zero_lamport_account)]);
10630        accounts.store_uncached(2, &[(&pubkey1, &zero_lamport_account)]);
10631        // Root all slots
10632        accounts.add_root(0);
10633        accounts.add_root(1);
10634        accounts.add_root(2);
10635
10636        // Account ref counts should match how many slots they were stored in
10637        // Account 1 = 3 slots; account 2 = 1 slot
10638        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 3);
10639        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 1);
10640
10641        accounts.clean_accounts(None, false, None);
10642        // Slots 0 and 1 should each have been cleaned because all of their
10643        // accounts are zero lamports
10644        assert!(accounts.storage.get_slot_stores(0).is_none());
10645        assert!(accounts.storage.get_slot_stores(1).is_none());
10646        // Slot 2 only has a zero lamport account as well. But, calc_delete_dependencies()
10647        // should exclude slot 2 from the clean due to changes in other slots
10648        assert!(accounts.storage.get_slot_stores(2).is_some());
10649        // Index ref counts should be consistent with the slot stores. Account 1 ref count
10650        // should be 1 since slot 2 is the only alive slot; account 2 should have a ref
10651        // count of 0 due to slot 0 being dead
10652        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 1);
10653        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 0);
10654
10655        accounts.clean_accounts(None, false, None);
10656        // Slot 2 will now be cleaned, which will leave account 1 with a ref count of 0
10657        assert!(accounts.storage.get_slot_stores(2).is_none());
10658        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
10659    }
10660
10661    #[test]
10662    fn test_clean_zero_lamport_and_old_roots() {
10663        solana_logger::setup();
10664
10665        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10666        let pubkey = solana_sdk::pubkey::new_rand();
10667        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10668        let zero_lamport_account =
10669            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10670
10671        // Store a zero-lamport account
10672        accounts.store_uncached(0, &[(&pubkey, &account)]);
10673        accounts.store_uncached(1, &[(&pubkey, &zero_lamport_account)]);
10674
10675        // Simulate rooting the zero-lamport account, should be a
10676        // candidate for cleaning
10677        accounts.add_root(0);
10678        accounts.add_root(1);
10679
10680        // Slot 0 should be removed, and
10681        // zero-lamport account should be cleaned
10682        accounts.clean_accounts(None, false, None);
10683
10684        assert!(accounts.storage.get_slot_stores(0).is_none());
10685        assert!(accounts.storage.get_slot_stores(1).is_none());
10686
10687        // Slot 0 should be cleaned because all it's accounts have been
10688        // updated in the rooted slot 1
10689        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
10690
10691        // Slot 1 should be cleaned because all it's accounts are
10692        // zero lamports, and are not present in any other slot's
10693        // storage entries
10694        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
10695
10696        // zero lamport account, should no longer exist in accounts index
10697        // because it has been removed
10698        assert!(accounts
10699            .accounts_index
10700            .get_for_tests(&pubkey, None, None)
10701            .is_none());
10702    }
10703
10704    #[test]
10705    fn test_clean_old_with_normal_account() {
10706        solana_logger::setup();
10707
10708        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10709        let pubkey = solana_sdk::pubkey::new_rand();
10710        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10711        //store an account
10712        accounts.store_uncached(0, &[(&pubkey, &account)]);
10713        accounts.store_uncached(1, &[(&pubkey, &account)]);
10714
10715        // simulate slots are rooted after while
10716        accounts.get_accounts_delta_hash(0);
10717        accounts.add_root(0);
10718        accounts.get_accounts_delta_hash(1);
10719        accounts.add_root(1);
10720
10721        //even if rooted, old state isn't cleaned up
10722        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
10723        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
10724
10725        accounts.clean_accounts(None, false, None);
10726
10727        //now old state is cleaned up
10728        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
10729        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
10730    }
10731
10732    #[test]
10733    fn test_clean_old_with_zero_lamport_account() {
10734        solana_logger::setup();
10735
10736        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10737        let pubkey1 = solana_sdk::pubkey::new_rand();
10738        let pubkey2 = solana_sdk::pubkey::new_rand();
10739        let normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10740        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10741        //store an account
10742        accounts.store_uncached(0, &[(&pubkey1, &normal_account)]);
10743        accounts.store_uncached(1, &[(&pubkey1, &zero_account)]);
10744        accounts.store_uncached(0, &[(&pubkey2, &normal_account)]);
10745        accounts.store_uncached(1, &[(&pubkey2, &normal_account)]);
10746
10747        //simulate slots are rooted after while
10748        accounts.get_accounts_delta_hash(0);
10749        accounts.add_root(0);
10750        accounts.get_accounts_delta_hash(1);
10751        accounts.add_root(1);
10752
10753        //even if rooted, old state isn't cleaned up
10754        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
10755        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
10756
10757        accounts.print_accounts_stats("");
10758
10759        accounts.clean_accounts(None, false, None);
10760
10761        //Old state behind zero-lamport account is cleaned up
10762        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
10763        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
10764    }
10765
10766    #[test]
10767    fn test_clean_old_with_both_normal_and_zero_lamport_accounts() {
10768        solana_logger::setup();
10769
10770        let mut accounts = AccountsDb::new_with_config_for_tests(
10771            Vec::new(),
10772            &ClusterType::Development,
10773            safe_token_mint_index_enabled(),
10774            false,
10775            AccountShrinkThreshold::default(),
10776        );
10777        let pubkey1 = solana_sdk::pubkey::new_rand();
10778        let pubkey2 = solana_sdk::pubkey::new_rand();
10779
10780        // Set up account to be added to secondary index
10781        let mint_key = Pubkey::new_unique();
10782        let mut account_data_with_mint = vec![0; inline_safe_token::Account::get_packed_len()];
10783        account_data_with_mint[..PUBKEY_BYTES].clone_from_slice(&(mint_key.to_bytes()));
10784
10785        let mut normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10786        normal_account.set_owner(inline_safe_token::id());
10787        normal_account.set_data(account_data_with_mint.clone());
10788        let mut zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10789        zero_account.set_owner(inline_safe_token::id());
10790        zero_account.set_data(account_data_with_mint);
10791
10792        //store an account
10793        accounts.store_uncached(0, &[(&pubkey1, &normal_account)]);
10794        accounts.store_uncached(0, &[(&pubkey1, &normal_account)]);
10795        accounts.store_uncached(1, &[(&pubkey1, &zero_account)]);
10796        accounts.store_uncached(0, &[(&pubkey2, &normal_account)]);
10797        accounts.store_uncached(2, &[(&pubkey2, &normal_account)]);
10798
10799        //simulate slots are rooted after while
10800        accounts.get_accounts_delta_hash(0);
10801        accounts.add_root(0);
10802        accounts.get_accounts_delta_hash(1);
10803        accounts.add_root(1);
10804        accounts.get_accounts_delta_hash(2);
10805        accounts.add_root(2);
10806
10807        //even if rooted, old state isn't cleaned up
10808        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
10809        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
10810        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
10811
10812        // Secondary index should still find both pubkeys
10813        let mut found_accounts = HashSet::new();
10814        let index_key = IndexKey::SafeTokenMint(mint_key);
10815        let bank_id = 0;
10816        accounts
10817            .accounts_index
10818            .index_scan_accounts(
10819                &Ancestors::default(),
10820                bank_id,
10821                index_key,
10822                |key, _| {
10823                    found_accounts.insert(*key);
10824                },
10825                &ScanConfig::default(),
10826            )
10827            .unwrap();
10828        assert_eq!(found_accounts.len(), 2);
10829        assert!(found_accounts.contains(&pubkey1));
10830        assert!(found_accounts.contains(&pubkey2));
10831
10832        {
10833            accounts.account_indexes.keys = Some(AccountSecondaryIndexesIncludeExclude {
10834                exclude: true,
10835                keys: [mint_key].iter().cloned().collect::<HashSet<Pubkey>>(),
10836            });
10837            // Secondary index can't be used - do normal scan: should still find both pubkeys
10838            let found_accounts = accounts
10839                .index_scan_accounts(
10840                    &Ancestors::default(),
10841                    bank_id,
10842                    index_key,
10843                    |collection: &mut HashSet<Pubkey>, account| {
10844                        collection.insert(*account.unwrap().0);
10845                    },
10846                    &ScanConfig::default(),
10847                )
10848                .unwrap();
10849            assert!(!found_accounts.1);
10850            assert_eq!(found_accounts.0.len(), 2);
10851            assert!(found_accounts.0.contains(&pubkey1));
10852            assert!(found_accounts.0.contains(&pubkey2));
10853
10854            accounts.account_indexes.keys = None;
10855
10856            // Secondary index can now be used since it isn't marked as excluded
10857            let found_accounts = accounts
10858                .index_scan_accounts(
10859                    &Ancestors::default(),
10860                    bank_id,
10861                    index_key,
10862                    |collection: &mut HashSet<Pubkey>, account| {
10863                        collection.insert(*account.unwrap().0);
10864                    },
10865                    &ScanConfig::default(),
10866                )
10867                .unwrap();
10868            assert!(found_accounts.1);
10869            assert_eq!(found_accounts.0.len(), 2);
10870            assert!(found_accounts.0.contains(&pubkey1));
10871            assert!(found_accounts.0.contains(&pubkey2));
10872
10873            accounts.account_indexes.keys = None;
10874        }
10875
10876        accounts.clean_accounts(None, false, None);
10877
10878        //both zero lamport and normal accounts are cleaned up
10879        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
10880        // The only store to slot 1 was a zero lamport account, should
10881        // be purged by zero-lamport cleaning logic because slot 1 is
10882        // rooted
10883        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
10884        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
10885
10886        // `pubkey1`, a zero lamport account, should no longer exist in accounts index
10887        // because it has been removed by the clean
10888        assert!(accounts
10889            .accounts_index
10890            .get_for_tests(&pubkey1, None, None)
10891            .is_none());
10892
10893        // Secondary index should have purged `pubkey1` as well
10894        let mut found_accounts = vec![];
10895        accounts
10896            .accounts_index
10897            .index_scan_accounts(
10898                &Ancestors::default(),
10899                bank_id,
10900                IndexKey::SafeTokenMint(mint_key),
10901                |key, _| found_accounts.push(*key),
10902                &ScanConfig::default(),
10903            )
10904            .unwrap();
10905        assert_eq!(found_accounts, vec![pubkey2]);
10906    }
10907
10908    #[test]
10909    fn test_clean_max_slot_zero_lamport_account() {
10910        solana_logger::setup();
10911
10912        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10913        let pubkey = solana_sdk::pubkey::new_rand();
10914        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10915        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10916
10917        // store an account, make it a zero lamport account
10918        // in slot 1
10919        accounts.store_uncached(0, &[(&pubkey, &account)]);
10920        accounts.store_uncached(1, &[(&pubkey, &zero_account)]);
10921
10922        // simulate slots are rooted after while
10923        accounts.add_root(0);
10924        accounts.add_root(1);
10925
10926        // Only clean up to account 0, should not purge slot 0 based on
10927        // updates in later slots in slot 1
10928        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
10929        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
10930        accounts.clean_accounts(Some(0), false, None);
10931        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
10932        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
10933        assert!(accounts
10934            .accounts_index
10935            .get_for_tests(&pubkey, None, None)
10936            .is_some());
10937
10938        // Now the account can be cleaned up
10939        accounts.clean_accounts(Some(1), false, None);
10940        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
10941        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
10942
10943        // The zero lamport account, should no longer exist in accounts index
10944        // because it has been removed
10945        assert!(accounts
10946            .accounts_index
10947            .get_for_tests(&pubkey, None, None)
10948            .is_none());
10949    }
10950
10951    #[test]
10952    fn test_uncleaned_roots_with_account() {
10953        solana_logger::setup();
10954
10955        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10956        let pubkey = solana_sdk::pubkey::new_rand();
10957        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
10958        //store an account
10959        accounts.store_uncached(0, &[(&pubkey, &account)]);
10960        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
10961
10962        // simulate slots are rooted after while
10963        accounts.add_root(0);
10964        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
10965
10966        //now uncleaned roots are cleaned up
10967        accounts.clean_accounts(None, false, None);
10968        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
10969    }
10970
10971    #[test]
10972    fn test_uncleaned_roots_with_no_account() {
10973        solana_logger::setup();
10974
10975        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
10976
10977        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
10978
10979        // simulate slots are rooted after while
10980        accounts.add_root(0);
10981        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
10982
10983        //now uncleaned roots are cleaned up
10984        accounts.clean_accounts(None, false, None);
10985        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
10986    }
10987
10988    #[test]
10989    fn test_accounts_db_serialize1() {
10990        solana_logger::setup();
10991        let accounts = AccountsDb::new_single_for_tests();
10992        let mut pubkeys: Vec<Pubkey> = vec![];
10993
10994        // Create 100 accounts in slot 0
10995        create_account(&accounts, &mut pubkeys, 0, 100, 0, 0);
10996        accounts.clean_accounts(None, false, None);
10997        check_accounts(&accounts, &pubkeys, 0, 100, 1);
10998
10999        // do some updates to those accounts and re-check
11000        modify_accounts(&accounts, &pubkeys, 0, 100, 2);
11001        assert!(check_storage(&accounts, 0, 100));
11002        check_accounts(&accounts, &pubkeys, 0, 100, 2);
11003        accounts.get_accounts_delta_hash(0);
11004        accounts.add_root(0);
11005
11006        let mut pubkeys1: Vec<Pubkey> = vec![];
11007
11008        // CREATE SLOT 1
11009        let latest_slot = 1;
11010
11011        // Modify the first 10 of the accounts from slot 0 in slot 1
11012        modify_accounts(&accounts, &pubkeys, latest_slot, 10, 3);
11013        // Overwrite account 30 from slot 0 with lamports=0 into slot 1.
11014        // Slot 1 should now have 10 + 1 = 11 accounts
11015        let account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11016        accounts.store_uncached(latest_slot, &[(&pubkeys[30], &account)]);
11017
11018        // Create 10 new accounts in slot 1, should now have 11 + 10 = 21
11019        // accounts
11020        create_account(&accounts, &mut pubkeys1, latest_slot, 10, 0, 0);
11021
11022        accounts.get_accounts_delta_hash(latest_slot);
11023        accounts.add_root(latest_slot);
11024        assert!(check_storage(&accounts, 1, 21));
11025
11026        // CREATE SLOT 2
11027        let latest_slot = 2;
11028        let mut pubkeys2: Vec<Pubkey> = vec![];
11029
11030        // Modify first 20 of the accounts from slot 0 in slot 2
11031        modify_accounts(&accounts, &pubkeys, latest_slot, 20, 4);
11032        accounts.clean_accounts(None, false, None);
11033        // Overwrite account 31 from slot 0 with lamports=0 into slot 2.
11034        // Slot 2 should now have 20 + 1 = 21 accounts
11035        let account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11036        accounts.store_uncached(latest_slot, &[(&pubkeys[31], &account)]);
11037
11038        // Create 10 new accounts in slot 2. Slot 2 should now have
11039        // 21 + 10 = 31 accounts
11040        create_account(&accounts, &mut pubkeys2, latest_slot, 10, 0, 0);
11041
11042        accounts.get_accounts_delta_hash(latest_slot);
11043        accounts.add_root(latest_slot);
11044        assert!(check_storage(&accounts, 2, 31));
11045
11046        accounts.clean_accounts(None, false, None);
11047        // The first 20 accounts of slot 0 have been updated in slot 2, as well as
11048        // accounts 30 and  31 (overwritten with zero-lamport accounts in slot 1 and
11049        // slot 2 respectively), so only 78 accounts are left in slot 0's storage entries.
11050        assert!(check_storage(&accounts, 0, 78));
11051        // 10 of the 21 accounts have been modified in slot 2, so only 11
11052        // accounts left in slot 1.
11053        assert!(check_storage(&accounts, 1, 11));
11054        assert!(check_storage(&accounts, 2, 31));
11055
11056        let daccounts = reconstruct_accounts_db_via_serialization(&accounts, latest_slot);
11057
11058        assert_eq!(
11059            daccounts.write_version.load(Ordering::Acquire),
11060            accounts.write_version.load(Ordering::Acquire)
11061        );
11062
11063        // Get the hash for the latest slot, which should be the only hash in the
11064        // bank_hashes map on the deserialized AccountsDb
11065        assert_eq!(daccounts.bank_hashes.read().unwrap().len(), 2);
11066        assert_eq!(
11067            daccounts.bank_hashes.read().unwrap().get(&latest_slot),
11068            accounts.bank_hashes.read().unwrap().get(&latest_slot)
11069        );
11070
11071        daccounts.print_count_and_status("daccounts");
11072
11073        // Don't check the first 35 accounts which have not been modified on slot 0
11074        check_accounts(&daccounts, &pubkeys[35..], 0, 65, 37);
11075        check_accounts(&daccounts, &pubkeys1, 1, 10, 1);
11076        assert!(check_storage(&daccounts, 0, 100));
11077        assert!(check_storage(&daccounts, 1, 21));
11078        assert!(check_storage(&daccounts, 2, 31));
11079
11080        let ancestors = linear_ancestors(latest_slot);
11081        assert_eq!(
11082            daccounts.update_accounts_hash(
11083                latest_slot,
11084                &ancestors,
11085                &EpochSchedule::default(),
11086                &RentCollector::default()
11087            ),
11088            accounts.update_accounts_hash(
11089                latest_slot,
11090                &ancestors,
11091                &EpochSchedule::default(),
11092                &RentCollector::default()
11093            )
11094        );
11095    }
11096
11097    fn assert_load_account(
11098        accounts: &AccountsDb,
11099        slot: Slot,
11100        pubkey: Pubkey,
11101        expected_lamports: u64,
11102    ) {
11103        let ancestors = vec![(slot, 0)].into_iter().collect();
11104        let (account, slot) = accounts
11105            .load_without_fixed_root(&ancestors, &pubkey)
11106            .unwrap();
11107        assert_eq!((account.lamports(), slot), (expected_lamports, slot));
11108    }
11109
11110    fn assert_not_load_account(accounts: &AccountsDb, slot: Slot, pubkey: Pubkey) {
11111        let ancestors = vec![(slot, 0)].into_iter().collect();
11112        let load = accounts.load_without_fixed_root(&ancestors, &pubkey);
11113        assert!(load.is_none(), "{:?}", load);
11114    }
11115
11116    fn reconstruct_accounts_db_via_serialization(accounts: &AccountsDb, slot: Slot) -> AccountsDb {
11117        let daccounts =
11118            crate::serde_snapshot::reconstruct_accounts_db_via_serialization(accounts, slot);
11119        daccounts.print_count_and_status("daccounts");
11120        daccounts
11121    }
11122
11123    fn assert_no_stores(accounts: &AccountsDb, slot: Slot) {
11124        let slot_stores = accounts.storage.get_slot_stores(slot);
11125        let r_slot_stores = slot_stores.as_ref().map(|slot_stores| {
11126            let r_slot_stores = slot_stores.read().unwrap();
11127            info!("{:?}", *r_slot_stores);
11128            r_slot_stores
11129        });
11130        assert!(r_slot_stores.is_none() || r_slot_stores.unwrap().is_empty());
11131    }
11132
11133    #[test]
11134    fn test_accounts_db_purge_keep_live() {
11135        solana_logger::setup();
11136        let some_lamport = 223;
11137        let zero_lamport = 0;
11138        let no_data = 0;
11139        let owner = *AccountSharedData::default().owner();
11140
11141        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11142        let pubkey = solana_sdk::pubkey::new_rand();
11143
11144        let account2 = AccountSharedData::new(some_lamport, no_data, &owner);
11145        let pubkey2 = solana_sdk::pubkey::new_rand();
11146
11147        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11148
11149        let accounts = AccountsDb::new_single_for_tests();
11150        accounts.add_root(0);
11151
11152        // Step A
11153        let mut current_slot = 1;
11154        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
11155        // Store another live account to slot 1 which will prevent any purge
11156        // since the store count will not be zero
11157        accounts.store_uncached(current_slot, &[(&pubkey2, &account2)]);
11158        accounts.add_root(current_slot);
11159        let (slot1, account_info1) = accounts
11160            .accounts_index
11161            .get_for_tests(&pubkey, None, None)
11162            .map(|(account_list1, index1)| account_list1.slot_list()[index1])
11163            .unwrap();
11164        let (slot2, account_info2) = accounts
11165            .accounts_index
11166            .get_for_tests(&pubkey2, None, None)
11167            .map(|(account_list2, index2)| account_list2.slot_list()[index2])
11168            .unwrap();
11169        assert_eq!(slot1, current_slot);
11170        assert_eq!(slot1, slot2);
11171        assert_eq!(account_info1.store_id(), account_info2.store_id());
11172
11173        // Step B
11174        current_slot += 1;
11175        let zero_lamport_slot = current_slot;
11176        accounts.store_uncached(current_slot, &[(&pubkey, &zero_lamport_account)]);
11177        accounts.add_root(current_slot);
11178
11179        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
11180
11181        current_slot += 1;
11182        accounts.add_root(current_slot);
11183
11184        accounts.print_accounts_stats("pre_purge");
11185
11186        accounts.clean_accounts(None, false, None);
11187
11188        accounts.print_accounts_stats("post_purge");
11189
11190        // The earlier entry for pubkey in the account index is purged,
11191        let (slot_list_len, index_slot) = {
11192            let account_entry = accounts
11193                .accounts_index
11194                .get_account_read_entry(&pubkey)
11195                .unwrap();
11196            let slot_list = account_entry.slot_list();
11197            (slot_list.len(), slot_list[0].0)
11198        };
11199        assert_eq!(slot_list_len, 1);
11200        // Zero lamport entry was not the one purged
11201        assert_eq!(index_slot, zero_lamport_slot);
11202        // The ref count should still be 2 because no slots were purged
11203        assert_eq!(accounts.ref_count_for_pubkey(&pubkey), 2);
11204
11205        // storage for slot 1 had 2 accounts, now has 1 after pubkey 1
11206        // was reclaimed
11207        check_storage(&accounts, 1, 1);
11208        // storage for slot 2 had 1 accounts, now has 1
11209        check_storage(&accounts, 2, 1);
11210    }
11211
11212    #[test]
11213    fn test_accounts_db_purge1() {
11214        solana_logger::setup();
11215        let some_lamport = 223;
11216        let zero_lamport = 0;
11217        let no_data = 0;
11218        let owner = *AccountSharedData::default().owner();
11219
11220        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11221        let pubkey = solana_sdk::pubkey::new_rand();
11222
11223        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11224
11225        let accounts = AccountsDb::new_single_for_tests();
11226        accounts.add_root(0);
11227
11228        let mut current_slot = 1;
11229        accounts.set_hash(current_slot, current_slot - 1);
11230        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
11231        accounts.add_root(current_slot);
11232
11233        current_slot += 1;
11234        accounts.set_hash(current_slot, current_slot - 1);
11235        accounts.store_uncached(current_slot, &[(&pubkey, &zero_lamport_account)]);
11236        accounts.add_root(current_slot);
11237
11238        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
11239
11240        // Otherwise slot 2 will not be removed
11241        current_slot += 1;
11242        accounts.set_hash(current_slot, current_slot - 1);
11243        accounts.add_root(current_slot);
11244
11245        accounts.print_accounts_stats("pre_purge");
11246
11247        let ancestors = linear_ancestors(current_slot);
11248        info!("ancestors: {:?}", ancestors);
11249        let hash = accounts.update_accounts_hash_test(current_slot, &ancestors);
11250
11251        accounts.clean_accounts(None, false, None);
11252
11253        assert_eq!(
11254            accounts.update_accounts_hash_test(current_slot, &ancestors),
11255            hash
11256        );
11257
11258        accounts.print_accounts_stats("post_purge");
11259
11260        // Make sure the index is for pubkey cleared
11261        assert!(accounts
11262            .accounts_index
11263            .get_account_read_entry(&pubkey)
11264            .is_none());
11265
11266        // slot 1 & 2 should not have any stores
11267        assert_no_stores(&accounts, 1);
11268        assert_no_stores(&accounts, 2);
11269    }
11270
11271    #[test]
11272    fn test_accounts_db_serialize_zero_and_free() {
11273        solana_logger::setup();
11274
11275        let some_lamport = 223;
11276        let zero_lamport = 0;
11277        let no_data = 0;
11278        let owner = *AccountSharedData::default().owner();
11279
11280        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11281        let pubkey = solana_sdk::pubkey::new_rand();
11282        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11283
11284        let account2 = AccountSharedData::new(some_lamport + 1, no_data, &owner);
11285        let pubkey2 = solana_sdk::pubkey::new_rand();
11286
11287        let filler_account = AccountSharedData::new(some_lamport, no_data, &owner);
11288        let filler_account_pubkey = solana_sdk::pubkey::new_rand();
11289
11290        let accounts = AccountsDb::new_single_for_tests();
11291
11292        let mut current_slot = 1;
11293        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
11294        accounts.add_root(current_slot);
11295
11296        current_slot += 1;
11297        accounts.store_uncached(current_slot, &[(&pubkey, &zero_lamport_account)]);
11298        accounts.store_uncached(current_slot, &[(&pubkey2, &account2)]);
11299
11300        // Store enough accounts such that an additional store for slot 2 is created.
11301        while accounts
11302            .storage
11303            .get_slot_stores(current_slot)
11304            .unwrap()
11305            .read()
11306            .unwrap()
11307            .len()
11308            < 2
11309        {
11310            accounts.store_uncached(current_slot, &[(&filler_account_pubkey, &filler_account)]);
11311        }
11312        accounts.add_root(current_slot);
11313
11314        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
11315
11316        accounts.print_accounts_stats("accounts");
11317
11318        accounts.clean_accounts(None, false, None);
11319
11320        accounts.print_accounts_stats("accounts_post_purge");
11321        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
11322
11323        accounts.print_accounts_stats("reconstructed");
11324
11325        assert_load_account(&accounts, current_slot, pubkey, zero_lamport);
11326    }
11327
11328    fn with_chained_zero_lamport_accounts<F>(f: F)
11329    where
11330        F: Fn(AccountsDb, Slot) -> AccountsDb,
11331    {
11332        let some_lamport = 223;
11333        let zero_lamport = 0;
11334        let dummy_lamport = 999;
11335        let no_data = 0;
11336        let owner = *AccountSharedData::default().owner();
11337
11338        let account = AccountSharedData::new(some_lamport, no_data, &owner);
11339        let account2 = AccountSharedData::new(some_lamport + 100_001, no_data, &owner);
11340        let account3 = AccountSharedData::new(some_lamport + 100_002, no_data, &owner);
11341        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
11342
11343        let pubkey = solana_sdk::pubkey::new_rand();
11344        let purged_pubkey1 = solana_sdk::pubkey::new_rand();
11345        let purged_pubkey2 = solana_sdk::pubkey::new_rand();
11346
11347        let dummy_account = AccountSharedData::new(dummy_lamport, no_data, &owner);
11348        let dummy_pubkey = Pubkey::default();
11349
11350        let accounts = AccountsDb::new_single_for_tests();
11351
11352        let mut current_slot = 1;
11353        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
11354        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
11355        accounts.add_root(current_slot);
11356
11357        current_slot += 1;
11358        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &zero_lamport_account)]);
11359        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &account3)]);
11360        accounts.add_root(current_slot);
11361
11362        current_slot += 1;
11363        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &zero_lamport_account)]);
11364        accounts.add_root(current_slot);
11365
11366        current_slot += 1;
11367        accounts.store_uncached(current_slot, &[(&dummy_pubkey, &dummy_account)]);
11368        accounts.add_root(current_slot);
11369
11370        accounts.print_accounts_stats("pre_f");
11371        accounts.update_accounts_hash(
11372            4,
11373            &Ancestors::default(),
11374            &EpochSchedule::default(),
11375            &RentCollector::default(),
11376        );
11377
11378        let accounts = f(accounts, current_slot);
11379
11380        accounts.print_accounts_stats("post_f");
11381
11382        assert_load_account(&accounts, current_slot, pubkey, some_lamport);
11383        assert_load_account(&accounts, current_slot, purged_pubkey1, 0);
11384        assert_load_account(&accounts, current_slot, purged_pubkey2, 0);
11385        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
11386
11387        accounts
11388            .verify_bank_hash_and_lamports(
11389                4,
11390                &Ancestors::default(),
11391                1222,
11392                true,
11393                &EpochSchedule::default(),
11394                &RentCollector::default(),
11395                false,
11396                false,
11397            )
11398            .unwrap();
11399    }
11400
11401    #[test]
11402    fn test_accounts_purge_chained_purge_before_snapshot_restore() {
11403        solana_logger::setup();
11404        with_chained_zero_lamport_accounts(|accounts, current_slot| {
11405            accounts.clean_accounts(None, false, None);
11406            reconstruct_accounts_db_via_serialization(&accounts, current_slot)
11407        });
11408    }
11409
11410    #[test]
11411    fn test_accounts_purge_chained_purge_after_snapshot_restore() {
11412        solana_logger::setup();
11413        with_chained_zero_lamport_accounts(|accounts, current_slot| {
11414            let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
11415            accounts.print_accounts_stats("after_reconstruct");
11416            accounts.clean_accounts(None, false, None);
11417            reconstruct_accounts_db_via_serialization(&accounts, current_slot)
11418        });
11419    }
11420
11421    #[test]
11422    #[ignore]
11423    fn test_store_account_stress() {
11424        let slot = 42;
11425        let num_threads = 2;
11426
11427        let min_file_bytes = std::mem::size_of::<StoredMeta>()
11428            + std::mem::size_of::<crate::append_vec::AccountMeta>();
11429
11430        let db = Arc::new(AccountsDb::new_sized(Vec::new(), min_file_bytes as u64));
11431
11432        db.add_root(slot);
11433        let thread_hdls: Vec<_> = (0..num_threads)
11434            .map(|_| {
11435                let db = db.clone();
11436                std::thread::Builder::new()
11437                    .name("account-writers".to_string())
11438                    .spawn(move || {
11439                        let pubkey = solana_sdk::pubkey::new_rand();
11440                        let mut account = AccountSharedData::new(1, 0, &pubkey);
11441                        let mut i = 0;
11442                        loop {
11443                            let account_bal = thread_rng().gen_range(1, 99);
11444                            account.set_lamports(account_bal);
11445                            db.store_uncached(slot, &[(&pubkey, &account)]);
11446
11447                            let (account, slot) = db
11448                                .load_without_fixed_root(&Ancestors::default(), &pubkey)
11449                                .unwrap_or_else(|| {
11450                                    panic!("Could not fetch stored account {}, iter {}", pubkey, i)
11451                                });
11452                            assert_eq!(slot, slot);
11453                            assert_eq!(account.lamports(), account_bal);
11454                            i += 1;
11455                        }
11456                    })
11457                    .unwrap()
11458            })
11459            .collect();
11460
11461        for t in thread_hdls {
11462            t.join().unwrap();
11463        }
11464    }
11465
11466    #[test]
11467    fn test_accountsdb_scan_accounts() {
11468        solana_logger::setup();
11469        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11470        let key = Pubkey::default();
11471        let key0 = solana_sdk::pubkey::new_rand();
11472        let account0 = AccountSharedData::new(1, 0, &key);
11473
11474        db.store_uncached(0, &[(&key0, &account0)]);
11475
11476        let key1 = solana_sdk::pubkey::new_rand();
11477        let account1 = AccountSharedData::new(2, 0, &key);
11478        db.store_uncached(1, &[(&key1, &account1)]);
11479
11480        let ancestors = vec![(0, 0)].into_iter().collect();
11481        let accounts: Vec<AccountSharedData> = db.unchecked_scan_accounts(
11482            "",
11483            &ancestors,
11484            |accounts: &mut Vec<AccountSharedData>, option| {
11485                accounts.push(option.1.take_account());
11486            },
11487            &ScanConfig::default(),
11488        );
11489        assert_eq!(accounts, vec![account0]);
11490
11491        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
11492        let accounts: Vec<AccountSharedData> = db.unchecked_scan_accounts(
11493            "",
11494            &ancestors,
11495            |accounts: &mut Vec<AccountSharedData>, option| {
11496                accounts.push(option.1.take_account());
11497            },
11498            &ScanConfig::default(),
11499        );
11500        assert_eq!(accounts.len(), 2);
11501    }
11502
11503    #[test]
11504    fn test_cleanup_key_not_removed() {
11505        solana_logger::setup();
11506        let db = AccountsDb::new_single_for_tests();
11507
11508        let key = Pubkey::default();
11509        let key0 = solana_sdk::pubkey::new_rand();
11510        let account0 = AccountSharedData::new(1, 0, &key);
11511
11512        db.store_uncached(0, &[(&key0, &account0)]);
11513
11514        let key1 = solana_sdk::pubkey::new_rand();
11515        let account1 = AccountSharedData::new(2, 0, &key);
11516        db.store_uncached(1, &[(&key1, &account1)]);
11517
11518        db.print_accounts_stats("pre");
11519
11520        let slots: HashSet<Slot> = vec![1].into_iter().collect();
11521        let purge_keys = vec![(key1, slots)];
11522        let _ = db.purge_keys_exact(purge_keys.iter());
11523
11524        let account2 = AccountSharedData::new(3, 0, &key);
11525        db.store_uncached(2, &[(&key1, &account2)]);
11526
11527        db.print_accounts_stats("post");
11528        let ancestors = vec![(2, 0)].into_iter().collect();
11529        assert_eq!(
11530            db.load_without_fixed_root(&ancestors, &key1)
11531                .unwrap()
11532                .0
11533                .lamports(),
11534            3
11535        );
11536    }
11537
11538    #[test]
11539    fn test_store_large_account() {
11540        solana_logger::setup();
11541        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11542
11543        let key = Pubkey::default();
11544        let data_len = DEFAULT_FILE_SIZE as usize + 7;
11545        let account = AccountSharedData::new(1, data_len, &key);
11546
11547        db.store_uncached(0, &[(&key, &account)]);
11548
11549        let ancestors = vec![(0, 0)].into_iter().collect();
11550        let ret = db.load_without_fixed_root(&ancestors, &key).unwrap();
11551        assert_eq!(ret.0.data().len(), data_len);
11552    }
11553
11554    #[test]
11555    fn test_stored_readable_account() {
11556        let lamports = 1;
11557        let owner = Pubkey::new_unique();
11558        let executable = true;
11559        let rent_epoch = 2;
11560        let meta = StoredMeta {
11561            write_version: 5,
11562            pubkey: Pubkey::new_unique(),
11563            data_len: 7,
11564        };
11565        let account_meta = AccountMeta {
11566            lamports,
11567            owner,
11568            executable,
11569            rent_epoch,
11570        };
11571        let data = Vec::new();
11572        let account = Account {
11573            lamports,
11574            owner,
11575            executable,
11576            rent_epoch,
11577            data: data.clone(),
11578        };
11579        let offset = 99;
11580        let stored_size = 101;
11581        let hash = Hash::new_unique();
11582        let stored_account = StoredAccountMeta {
11583            meta: &meta,
11584            account_meta: &account_meta,
11585            data: &data,
11586            offset,
11587            stored_size,
11588            hash: &hash,
11589        };
11590        assert!(accounts_equal(&account, &stored_account));
11591    }
11592
11593    #[test]
11594    fn test_hash_stored_account() {
11595        // This test uses some UNSAFE tricks to detect most of account's field
11596        // addition and deletion without changing the hash code
11597
11598        const ACCOUNT_DATA_LEN: usize = 3;
11599        // the type of InputTuple elements must not contain references;
11600        // they should be simple scalars or data blobs
11601        type InputTuple = (
11602            Slot,
11603            StoredMeta,
11604            AccountMeta,
11605            [u8; ACCOUNT_DATA_LEN],
11606            usize, // for StoredAccountMeta::offset
11607            Hash,
11608        );
11609        const INPUT_LEN: usize = std::mem::size_of::<InputTuple>();
11610        type InputBlob = [u8; INPUT_LEN];
11611        let mut blob: InputBlob = [0u8; INPUT_LEN];
11612
11613        // spray memory with decreasing counts so that, data layout can be detected.
11614        for (i, byte) in blob.iter_mut().enumerate() {
11615            *byte = (INPUT_LEN - i) as u8;
11616        }
11617
11618        //UNSAFE: forcibly cast the special byte pattern to actual account fields.
11619        let (slot, meta, account_meta, data, offset, hash): InputTuple =
11620            unsafe { std::mem::transmute::<InputBlob, InputTuple>(blob) };
11621
11622        let stored_account = StoredAccountMeta {
11623            meta: &meta,
11624            account_meta: &account_meta,
11625            data: &data,
11626            offset,
11627            stored_size: CACHE_VIRTUAL_STORED_SIZE as usize,
11628            hash: &hash,
11629        };
11630        let account = stored_account.clone_account();
11631
11632        let expected_account_hash = if cfg!(debug_assertions) {
11633            Hash::from_str("4StuvYHFd7xuShVXB94uHHvpqGMCaacdZnYB74QQkPA1").unwrap()
11634        } else {
11635            Hash::from_str("33ruy7m3Xto7irYfsBSN74aAzQwCQxsfoZxXuZy2Rra3").unwrap()
11636        };
11637
11638        assert_eq!(
11639            AccountsDb::hash_account(slot, &stored_account, &stored_account.meta.pubkey),
11640            expected_account_hash,
11641            "StoredAccountMeta's data layout might be changed; update hashing if needed."
11642        );
11643        assert_eq!(
11644            AccountsDb::hash_account(slot, &account, &stored_account.meta.pubkey),
11645            expected_account_hash,
11646            "Account-based hashing must be consistent with StoredAccountMeta-based one."
11647        );
11648    }
11649
11650    #[test]
11651    fn test_bank_hash_stats() {
11652        solana_logger::setup();
11653        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11654
11655        let key = Pubkey::default();
11656        let some_data_len = 5;
11657        let some_slot: Slot = 0;
11658        let account = AccountSharedData::new(1, some_data_len, &key);
11659        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11660
11661        db.store_uncached(some_slot, &[(&key, &account)]);
11662        let mut account = db.load_without_fixed_root(&ancestors, &key).unwrap().0;
11663        account.checked_sub_lamports(1).unwrap();
11664        account.set_executable(true);
11665        db.store_uncached(some_slot, &[(&key, &account)]);
11666        db.add_root(some_slot);
11667
11668        let bank_hashes = db.bank_hashes.read().unwrap();
11669        let bank_hash = bank_hashes.get(&some_slot).unwrap();
11670        assert_eq!(bank_hash.stats.num_updated_accounts, 1);
11671        assert_eq!(bank_hash.stats.num_removed_accounts, 1);
11672        assert_eq!(bank_hash.stats.num_lamports_stored, 1);
11673        assert_eq!(bank_hash.stats.total_data_len, 2 * some_data_len as u64);
11674        assert_eq!(bank_hash.stats.num_executable_accounts, 1);
11675    }
11676
11677    #[test]
11678    fn test_calculate_accounts_hash_check_hash_mismatch() {
11679        solana_logger::setup();
11680        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11681
11682        let key = solana_sdk::pubkey::new_rand();
11683        let some_data_len = 0;
11684        let some_slot: Slot = 0;
11685        let account = AccountSharedData::new(1, some_data_len, &key);
11686
11687        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11688
11689        // put wrong hash value in store so we get a mismatch
11690        db.store_accounts_unfrozen(
11691            (some_slot, &[(&key, &account)][..]),
11692            Some(&[&Hash::default()]),
11693            false,
11694            None,
11695            StoreReclaims::Default,
11696        );
11697        db.add_root(some_slot);
11698        let check_hash = true;
11699        for use_index in [true, false] {
11700            assert!(db
11701                .calculate_accounts_hash_helper(
11702                    use_index,
11703                    some_slot,
11704                    &CalcAccountsHashConfig {
11705                        use_bg_thread_pool: true, // is_startup used to be false
11706                        check_hash,
11707                        ancestors: Some(&ancestors),
11708                        ..CalcAccountsHashConfig::default()
11709                    },
11710                )
11711                .is_err());
11712        }
11713    }
11714
11715    // something we can get a ref to
11716    lazy_static! {
11717        pub static ref EPOCH_SCHEDULE: EpochSchedule = EpochSchedule::default();
11718        pub static ref RENT_COLLECTOR: RentCollector = RentCollector::default();
11719    }
11720
11721    impl<'a> CalcAccountsHashConfig<'a> {
11722        fn default() -> Self {
11723            Self {
11724                use_bg_thread_pool: false,
11725                check_hash: false,
11726                ancestors: None,
11727                use_write_cache: false,
11728                epoch_schedule: &EPOCH_SCHEDULE,
11729                rent_collector: &RENT_COLLECTOR,
11730                store_detailed_debug_info_on_failure: false,
11731                full_snapshot: None,
11732            }
11733        }
11734    }
11735
11736    #[test]
11737    fn test_calculate_accounts_hash_check_hash() {
11738        solana_logger::setup();
11739        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11740
11741        let key = solana_sdk::pubkey::new_rand();
11742        let some_data_len = 0;
11743        let some_slot: Slot = 0;
11744        let account = AccountSharedData::new(1, some_data_len, &key);
11745
11746        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11747
11748        db.store_uncached(some_slot, &[(&key, &account)]);
11749        db.add_root(some_slot);
11750        let check_hash = true;
11751        assert_eq!(
11752            db.calculate_accounts_hash_helper(
11753                false,
11754                some_slot,
11755                &CalcAccountsHashConfig {
11756                    use_bg_thread_pool: true, // is_startup used to be false
11757                    check_hash,
11758                    ancestors: Some(&ancestors),
11759                    ..CalcAccountsHashConfig::default()
11760                },
11761            )
11762            .unwrap(),
11763            db.calculate_accounts_hash_helper(
11764                true,
11765                some_slot,
11766                &CalcAccountsHashConfig {
11767                    use_bg_thread_pool: true, // is_startup used to be false
11768                    check_hash,
11769                    ancestors: Some(&ancestors),
11770                    ..CalcAccountsHashConfig::default()
11771                },
11772            )
11773            .unwrap(),
11774        );
11775    }
11776
11777    #[test]
11778    fn test_verify_bank_hash() {
11779        use BankHashVerificationError::*;
11780        solana_logger::setup();
11781        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11782
11783        let key = solana_sdk::pubkey::new_rand();
11784        let some_data_len = 0;
11785        let some_slot: Slot = 0;
11786        let account = AccountSharedData::new(1, some_data_len, &key);
11787        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11788
11789        db.store_uncached(some_slot, &[(&key, &account)]);
11790        db.add_root(some_slot);
11791        db.update_accounts_hash_test(some_slot, &ancestors);
11792        assert_matches!(
11793            db.verify_bank_hash_and_lamports(
11794                some_slot,
11795                &ancestors,
11796                1,
11797                true,
11798                &EpochSchedule::default(),
11799                &RentCollector::default(),
11800                false,
11801                false,
11802            ),
11803            Ok(_)
11804        );
11805
11806        db.bank_hashes.write().unwrap().remove(&some_slot).unwrap();
11807        assert_matches!(
11808            db.verify_bank_hash_and_lamports(
11809                some_slot,
11810                &ancestors,
11811                1,
11812                true,
11813                &EpochSchedule::default(),
11814                &RentCollector::default(),
11815                false,
11816                false,
11817            ),
11818            Err(MissingBankHash)
11819        );
11820
11821        let some_bank_hash = Hash::new(&[0xca; HASH_BYTES]);
11822        let bank_hash_info = BankHashInfo {
11823            hash: some_bank_hash,
11824            snapshot_hash: Hash::new(&[0xca; HASH_BYTES]),
11825            stats: BankHashStats::default(),
11826        };
11827        db.bank_hashes
11828            .write()
11829            .unwrap()
11830            .insert(some_slot, bank_hash_info);
11831        assert_matches!(
11832            db.verify_bank_hash_and_lamports(
11833                some_slot,
11834                &ancestors,
11835                1,
11836                true,
11837                &EpochSchedule::default(),
11838                &RentCollector::default(),
11839                false,
11840                false,
11841            ),
11842            Err(MismatchedBankHash)
11843        );
11844    }
11845
11846    #[test]
11847    fn test_verify_bank_capitalization() {
11848        use BankHashVerificationError::*;
11849        solana_logger::setup();
11850        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11851
11852        let key = solana_sdk::pubkey::new_rand();
11853        let some_data_len = 0;
11854        let some_slot: Slot = 0;
11855        let account = AccountSharedData::new(1, some_data_len, &key);
11856        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11857
11858        db.store_uncached(some_slot, &[(&key, &account)]);
11859        db.add_root(some_slot);
11860        db.update_accounts_hash_test(some_slot, &ancestors);
11861        assert_matches!(
11862            db.verify_bank_hash_and_lamports(
11863                some_slot,
11864                &ancestors,
11865                1,
11866                true,
11867                &EpochSchedule::default(),
11868                &RentCollector::default(),
11869                false,
11870                false,
11871            ),
11872            Ok(_)
11873        );
11874
11875        let native_account_pubkey = solana_sdk::pubkey::new_rand();
11876        db.store_uncached(
11877            some_slot,
11878            &[(
11879                &native_account_pubkey,
11880                &solana_sdk::native_loader::create_loadable_account_for_test("foo"),
11881            )],
11882        );
11883        db.update_accounts_hash_test(some_slot, &ancestors);
11884        assert_matches!(
11885            db.verify_bank_hash_and_lamports(
11886                some_slot,
11887                &ancestors,
11888                2,
11889                true,
11890                &EpochSchedule::default(),
11891                &RentCollector::default(),
11892                false,
11893                false,
11894            ),
11895            Ok(_)
11896        );
11897
11898        assert_matches!(
11899            db.verify_bank_hash_and_lamports(some_slot, &ancestors, 10, true, &EpochSchedule::default(), &RentCollector::default(), false, false),
11900            Err(MismatchedTotalLamports(expected, actual)) if expected == 2 && actual == 10
11901        );
11902    }
11903
11904    #[test]
11905    fn test_verify_bank_hash_no_account() {
11906        solana_logger::setup();
11907        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11908
11909        let some_slot: Slot = 0;
11910        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11911
11912        db.bank_hashes
11913            .write()
11914            .unwrap()
11915            .insert(some_slot, BankHashInfo::default());
11916        db.add_root(some_slot);
11917        db.update_accounts_hash_test(some_slot, &ancestors);
11918        assert_matches!(
11919            db.verify_bank_hash_and_lamports(
11920                some_slot,
11921                &ancestors,
11922                0,
11923                true,
11924                &EpochSchedule::default(),
11925                &RentCollector::default(),
11926                false,
11927                false,
11928            ),
11929            Ok(_)
11930        );
11931    }
11932
11933    #[test]
11934    fn test_verify_bank_hash_bad_account_hash() {
11935        use BankHashVerificationError::*;
11936        solana_logger::setup();
11937        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11938
11939        let key = Pubkey::default();
11940        let some_data_len = 0;
11941        let some_slot: Slot = 0;
11942        let account = AccountSharedData::new(1, some_data_len, &key);
11943        let ancestors = vec![(some_slot, 0)].into_iter().collect();
11944
11945        let accounts = &[(&key, &account)][..];
11946        // update AccountsDb's bank hash
11947        {
11948            let mut bank_hashes = db.bank_hashes.write().unwrap();
11949            bank_hashes
11950                .entry(some_slot)
11951                .or_insert_with(BankHashInfo::default);
11952        }
11953        // provide bogus account hashes
11954        let some_hash = Hash::new(&[0xca; HASH_BYTES]);
11955        db.store_accounts_unfrozen(
11956            (some_slot, accounts),
11957            Some(&[&some_hash]),
11958            false,
11959            None,
11960            StoreReclaims::Default,
11961        );
11962        db.add_root(some_slot);
11963        assert_matches!(
11964            db.verify_bank_hash_and_lamports(
11965                some_slot,
11966                &ancestors,
11967                1,
11968                true,
11969                &EpochSchedule::default(),
11970                &RentCollector::default(),
11971                false,
11972                false,
11973            ),
11974            Err(MismatchedBankHash)
11975        );
11976    }
11977
11978    #[test]
11979    fn test_storage_finder() {
11980        solana_logger::setup();
11981        let db = AccountsDb::new_sized(Vec::new(), 16 * 1024);
11982        let key = solana_sdk::pubkey::new_rand();
11983        let lamports = 100;
11984        let data_len = 8190;
11985        let account = AccountSharedData::new(lamports, data_len, &solana_sdk::pubkey::new_rand());
11986        // pre-populate with a smaller empty store
11987        db.create_and_insert_store(1, 8192, "test_storage_finder");
11988        db.store_uncached(1, &[(&key, &account)]);
11989    }
11990
11991    #[test]
11992    fn test_get_snapshot_storages_empty() {
11993        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11994        assert!(db.get_snapshot_storages(0, None, None).0.is_empty());
11995    }
11996
11997    #[test]
11998    fn test_get_snapshot_storages_only_older_than_or_equal_to_snapshot_slot() {
11999        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12000
12001        let key = Pubkey::default();
12002        let account = AccountSharedData::new(1, 0, &key);
12003        let before_slot = 0;
12004        let base_slot = before_slot + 1;
12005        let after_slot = base_slot + 1;
12006
12007        db.add_root(base_slot);
12008        db.store_uncached(base_slot, &[(&key, &account)]);
12009        assert!(db
12010            .get_snapshot_storages(before_slot, None, None)
12011            .0
12012            .is_empty());
12013
12014        assert_eq!(1, db.get_snapshot_storages(base_slot, None, None).0.len());
12015        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
12016    }
12017
12018    #[test]
12019    fn test_get_snapshot_storages_only_non_empty() {
12020        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12021
12022        let key = Pubkey::default();
12023        let account = AccountSharedData::new(1, 0, &key);
12024        let base_slot = 0;
12025        let after_slot = base_slot + 1;
12026
12027        db.store_uncached(base_slot, &[(&key, &account)]);
12028        db.storage
12029            .get_slot_stores(base_slot)
12030            .unwrap()
12031            .write()
12032            .unwrap()
12033            .clear();
12034        db.add_root(base_slot);
12035        assert!(db
12036            .get_snapshot_storages(after_slot, None, None)
12037            .0
12038            .is_empty());
12039
12040        db.store_uncached(base_slot, &[(&key, &account)]);
12041        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
12042    }
12043
12044    #[test]
12045    fn test_get_snapshot_storages_only_roots() {
12046        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12047
12048        let key = Pubkey::default();
12049        let account = AccountSharedData::new(1, 0, &key);
12050        let base_slot = 0;
12051        let after_slot = base_slot + 1;
12052
12053        db.store_uncached(base_slot, &[(&key, &account)]);
12054        assert!(db
12055            .get_snapshot_storages(after_slot, None, None)
12056            .0
12057            .is_empty());
12058
12059        db.add_root(base_slot);
12060        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
12061    }
12062
12063    #[test]
12064    fn test_get_snapshot_storages_exclude_empty() {
12065        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12066
12067        let key = Pubkey::default();
12068        let account = AccountSharedData::new(1, 0, &key);
12069        let base_slot = 0;
12070        let after_slot = base_slot + 1;
12071
12072        db.store_uncached(base_slot, &[(&key, &account)]);
12073        db.add_root(base_slot);
12074        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
12075
12076        db.storage
12077            .get_slot_stores(0)
12078            .unwrap()
12079            .read()
12080            .unwrap()
12081            .values()
12082            .next()
12083            .unwrap()
12084            .remove_account(0, true);
12085        assert!(db
12086            .get_snapshot_storages(after_slot, None, None)
12087            .0
12088            .is_empty());
12089    }
12090
12091    #[test]
12092    fn test_get_snapshot_storages_with_base_slot() {
12093        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12094
12095        let key = Pubkey::default();
12096        let account = AccountSharedData::new(1, 0, &key);
12097
12098        let slot = 10;
12099        db.store_uncached(slot, &[(&key, &account)]);
12100        db.add_root(slot);
12101        assert_eq!(
12102            0,
12103            db.get_snapshot_storages(slot + 1, Some(slot), None).0.len()
12104        );
12105        assert_eq!(
12106            1,
12107            db.get_snapshot_storages(slot + 1, Some(slot - 1), None)
12108                .0
12109                .len()
12110        );
12111    }
12112
12113    #[test]
12114    #[should_panic(expected = "double remove of account in slot: 0/store: 0!!")]
12115    fn test_storage_remove_account_double_remove() {
12116        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
12117        let pubkey = solana_sdk::pubkey::new_rand();
12118        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12119        accounts.store_uncached(0, &[(&pubkey, &account)]);
12120        let storage_entry = accounts
12121            .storage
12122            .get_slot_stores(0)
12123            .unwrap()
12124            .read()
12125            .unwrap()
12126            .values()
12127            .next()
12128            .unwrap()
12129            .clone();
12130        storage_entry.remove_account(0, true);
12131        storage_entry.remove_account(0, true);
12132    }
12133
12134    #[test]
12135    fn test_accounts_purge_long_chained_after_snapshot_restore() {
12136        solana_logger::setup();
12137        let old_lamport = 223;
12138        let zero_lamport = 0;
12139        let no_data = 0;
12140        let owner = *AccountSharedData::default().owner();
12141
12142        let account = AccountSharedData::new(old_lamport, no_data, &owner);
12143        let account2 = AccountSharedData::new(old_lamport + 100_001, no_data, &owner);
12144        let account3 = AccountSharedData::new(old_lamport + 100_002, no_data, &owner);
12145        let dummy_account = AccountSharedData::new(99_999_999, no_data, &owner);
12146        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
12147
12148        let pubkey = solana_sdk::pubkey::new_rand();
12149        let dummy_pubkey = solana_sdk::pubkey::new_rand();
12150        let purged_pubkey1 = solana_sdk::pubkey::new_rand();
12151        let purged_pubkey2 = solana_sdk::pubkey::new_rand();
12152
12153        let mut current_slot = 0;
12154        let accounts = AccountsDb::new_single_for_tests();
12155
12156        // create intermediate updates to purged_pubkey1 so that
12157        // generate_index must add slots as root last at once
12158        current_slot += 1;
12159        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
12160        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
12161        accounts.add_root(current_slot);
12162
12163        current_slot += 1;
12164        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
12165        accounts.add_root(current_slot);
12166
12167        current_slot += 1;
12168        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
12169        accounts.add_root(current_slot);
12170
12171        current_slot += 1;
12172        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &zero_lamport_account)]);
12173        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &account3)]);
12174        accounts.add_root(current_slot);
12175
12176        current_slot += 1;
12177        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &zero_lamport_account)]);
12178        accounts.add_root(current_slot);
12179
12180        current_slot += 1;
12181        accounts.store_uncached(current_slot, &[(&dummy_pubkey, &dummy_account)]);
12182        accounts.add_root(current_slot);
12183
12184        accounts.print_count_and_status("before reconstruct");
12185        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
12186        accounts.print_count_and_status("before purge zero");
12187        accounts.clean_accounts(None, false, None);
12188        accounts.print_count_and_status("after purge zero");
12189
12190        assert_load_account(&accounts, current_slot, pubkey, old_lamport);
12191        assert_load_account(&accounts, current_slot, purged_pubkey1, 0);
12192        assert_load_account(&accounts, current_slot, purged_pubkey2, 0);
12193    }
12194
12195    fn do_full_clean_refcount(store1_first: bool, store_size: u64) {
12196        let pubkey1 = Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
12197        let pubkey2 = Pubkey::from_str("My22211111111111111111111111111111111111111").unwrap();
12198        let pubkey3 = Pubkey::from_str("My33311111111111111111111111111111111111111").unwrap();
12199
12200        let old_lamport = 223;
12201        let zero_lamport = 0;
12202        let dummy_lamport = 999_999;
12203
12204        // size data so only 1 fits in a 4k store
12205        let data_size = 2200;
12206
12207        let owner = *AccountSharedData::default().owner();
12208
12209        let account = AccountSharedData::new(old_lamport, data_size, &owner);
12210        let account2 = AccountSharedData::new(old_lamport + 100_001, data_size, &owner);
12211        let account3 = AccountSharedData::new(old_lamport + 100_002, data_size, &owner);
12212        let account4 = AccountSharedData::new(dummy_lamport, data_size, &owner);
12213        let zero_lamport_account = AccountSharedData::new(zero_lamport, data_size, &owner);
12214
12215        let mut current_slot = 0;
12216        let accounts = AccountsDb::new_sized_no_extra_stores(Vec::new(), store_size);
12217
12218        // A: Initialize AccountsDb with pubkey1 and pubkey2
12219        current_slot += 1;
12220        if store1_first {
12221            accounts.store_uncached(current_slot, &[(&pubkey1, &account)]);
12222            accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
12223        } else {
12224            accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
12225            accounts.store_uncached(current_slot, &[(&pubkey1, &account)]);
12226        }
12227        accounts.get_accounts_delta_hash(current_slot);
12228        accounts.add_root(current_slot);
12229
12230        info!("post A");
12231        accounts.print_accounts_stats("Post-A");
12232
12233        // B: Test multiple updates to pubkey1 in a single slot/storage
12234        current_slot += 1;
12235        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
12236        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
12237        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
12238        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
12239        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
12240        // Stores to same pubkey, same slot only count once towards the
12241        // ref count
12242        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12243        accounts.get_accounts_delta_hash(current_slot);
12244        accounts.add_root(current_slot);
12245
12246        accounts.print_accounts_stats("Post-B pre-clean");
12247
12248        accounts.clean_accounts(None, false, None);
12249
12250        info!("post B");
12251        accounts.print_accounts_stats("Post-B");
12252
12253        // C: more updates to trigger clean of previous updates
12254        current_slot += 1;
12255        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12256        accounts.store_uncached(current_slot, &[(&pubkey1, &account3)]);
12257        accounts.store_uncached(current_slot, &[(&pubkey2, &account3)]);
12258        accounts.store_uncached(current_slot, &[(&pubkey3, &account4)]);
12259        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12260        accounts.get_accounts_delta_hash(current_slot);
12261        accounts.add_root(current_slot);
12262
12263        info!("post C");
12264
12265        accounts.print_accounts_stats("Post-C");
12266
12267        // D: Make all keys 0-lamport, cleans all keys
12268        current_slot += 1;
12269        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12270        accounts.store_uncached(current_slot, &[(&pubkey1, &zero_lamport_account)]);
12271        accounts.store_uncached(current_slot, &[(&pubkey2, &zero_lamport_account)]);
12272        accounts.store_uncached(current_slot, &[(&pubkey3, &zero_lamport_account)]);
12273
12274        let snapshot_stores = accounts.get_snapshot_storages(current_slot, None, None).0;
12275        let total_accounts: usize = snapshot_stores
12276            .iter()
12277            .flatten()
12278            .map(|s| s.all_accounts().len())
12279            .sum();
12280        assert!(!snapshot_stores.is_empty());
12281        assert!(total_accounts > 0);
12282
12283        info!("post D");
12284        accounts.print_accounts_stats("Post-D");
12285
12286        accounts.get_accounts_delta_hash(current_slot);
12287        accounts.add_root(current_slot);
12288        accounts.clean_accounts(None, false, None);
12289
12290        accounts.print_accounts_stats("Post-D clean");
12291
12292        let total_accounts_post_clean: usize = snapshot_stores
12293            .iter()
12294            .flatten()
12295            .map(|s| s.all_accounts().len())
12296            .sum();
12297        assert_eq!(total_accounts, total_accounts_post_clean);
12298
12299        // should clean all 3 pubkeys
12300        assert_eq!(accounts.ref_count_for_pubkey(&pubkey1), 0);
12301        assert_eq!(accounts.ref_count_for_pubkey(&pubkey2), 0);
12302        assert_eq!(accounts.ref_count_for_pubkey(&pubkey3), 0);
12303    }
12304
12305    #[test]
12306    fn test_full_clean_refcount() {
12307        solana_logger::setup();
12308
12309        // Setup 3 scenarios which try to differentiate between pubkey1 being in an
12310        // Available slot or a Full slot which would cause a different reset behavior
12311        // when pubkey1 is cleaned and therefore cause the ref count to be incorrect
12312        // preventing a removal of that key.
12313        //
12314        // do stores with a 4mb size so only 1 store is created per slot
12315        do_full_clean_refcount(false, 4 * 1024 * 1024);
12316
12317        // do stores with a 4k size and store pubkey1 first
12318        do_full_clean_refcount(false, 4096);
12319
12320        // do stores with a 4k size and store pubkey1 2nd
12321        do_full_clean_refcount(true, 4096);
12322    }
12323
12324    #[test]
12325    fn test_accounts_clean_after_snapshot_restore_then_old_revives() {
12326        solana_logger::setup();
12327        let old_lamport = 223;
12328        let zero_lamport = 0;
12329        let no_data = 0;
12330        let dummy_lamport = 999_999;
12331        let owner = *AccountSharedData::default().owner();
12332
12333        let account = AccountSharedData::new(old_lamport, no_data, &owner);
12334        let account2 = AccountSharedData::new(old_lamport + 100_001, no_data, &owner);
12335        let account3 = AccountSharedData::new(old_lamport + 100_002, no_data, &owner);
12336        let dummy_account = AccountSharedData::new(dummy_lamport, no_data, &owner);
12337        let zero_lamport_account = AccountSharedData::new(zero_lamport, no_data, &owner);
12338
12339        let pubkey1 = solana_sdk::pubkey::new_rand();
12340        let pubkey2 = solana_sdk::pubkey::new_rand();
12341        let dummy_pubkey = solana_sdk::pubkey::new_rand();
12342
12343        let mut current_slot = 0;
12344        let accounts = AccountsDb::new_single_for_tests();
12345
12346        // A: Initialize AccountsDb with pubkey1 and pubkey2
12347        current_slot += 1;
12348        accounts.store_uncached(current_slot, &[(&pubkey1, &account)]);
12349        accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
12350        accounts.get_accounts_delta_hash(current_slot);
12351        accounts.add_root(current_slot);
12352
12353        // B: Test multiple updates to pubkey1 in a single slot/storage
12354        current_slot += 1;
12355        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
12356        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
12357        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
12358        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
12359        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
12360        // Stores to same pubkey, same slot only count once towards the
12361        // ref count
12362        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12363        accounts.get_accounts_delta_hash(current_slot);
12364        accounts.add_root(current_slot);
12365
12366        // C: Yet more update to trigger lazy clean of step A
12367        current_slot += 1;
12368        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
12369        accounts.store_uncached(current_slot, &[(&pubkey1, &account3)]);
12370        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12371        accounts.get_accounts_delta_hash(current_slot);
12372        accounts.add_root(current_slot);
12373
12374        // D: Make pubkey1 0-lamport; also triggers clean of step B
12375        current_slot += 1;
12376        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
12377        accounts.store_uncached(current_slot, &[(&pubkey1, &zero_lamport_account)]);
12378        accounts.clean_accounts(None, false, None);
12379
12380        assert_eq!(
12381            // Removed one reference from the dead slot (reference only counted once
12382            // even though there were two stores to the pubkey in that slot)
12383            3, /* == 3 - 1 + 1 */
12384            accounts.ref_count_for_pubkey(&pubkey1)
12385        );
12386        accounts.get_accounts_delta_hash(current_slot);
12387        accounts.add_root(current_slot);
12388
12389        // E: Avoid missing bank hash error
12390        current_slot += 1;
12391        accounts.store_uncached(current_slot, &[(&dummy_pubkey, &dummy_account)]);
12392        accounts.get_accounts_delta_hash(current_slot);
12393        accounts.add_root(current_slot);
12394
12395        assert_load_account(&accounts, current_slot, pubkey1, zero_lamport);
12396        assert_load_account(&accounts, current_slot, pubkey2, old_lamport);
12397        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
12398
12399        // At this point, there is no index entries for A and B
12400        // If step C and step D should be purged, snapshot restore would cause
12401        // pubkey1 to be revived as the state of step A.
12402        // So, prevent that from happening by introducing refcount
12403        accounts.clean_accounts(None, false, None);
12404        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
12405        accounts.clean_accounts(None, false, None);
12406
12407        info!("pubkey: {}", pubkey1);
12408        accounts.print_accounts_stats("pre_clean");
12409        assert_load_account(&accounts, current_slot, pubkey1, zero_lamport);
12410        assert_load_account(&accounts, current_slot, pubkey2, old_lamport);
12411        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
12412
12413        // F: Finally, make Step A cleanable
12414        current_slot += 1;
12415        accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
12416        accounts.get_accounts_delta_hash(current_slot);
12417        accounts.add_root(current_slot);
12418
12419        // Do clean
12420        accounts.clean_accounts(None, false, None);
12421
12422        // 2nd clean needed to clean-up pubkey1
12423        accounts.clean_accounts(None, false, None);
12424
12425        // Ensure pubkey2 is cleaned from the index finally
12426        assert_not_load_account(&accounts, current_slot, pubkey1);
12427        assert_load_account(&accounts, current_slot, pubkey2, old_lamport);
12428        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_lamport);
12429    }
12430
12431    #[test]
12432    fn test_clean_stored_dead_slots_empty() {
12433        let accounts = AccountsDb::new_single_for_tests();
12434        let mut dead_slots = HashSet::new();
12435        dead_slots.insert(10);
12436        accounts.clean_stored_dead_slots(&dead_slots, None, &HashSet::default());
12437    }
12438
12439    #[test]
12440    fn test_shrink_all_slots_none() {
12441        for startup in &[false, true] {
12442            let accounts = AccountsDb::new_single_for_tests();
12443
12444            for _ in 0..10 {
12445                accounts.shrink_candidate_slots();
12446            }
12447
12448            accounts.shrink_all_slots(*startup, None);
12449        }
12450    }
12451
12452    #[test]
12453    fn test_shrink_next_slots() {
12454        let mut accounts = AccountsDb::new_single_for_tests();
12455        accounts.caching_enabled = false;
12456
12457        let mut current_slot = 7;
12458
12459        assert_eq!(
12460            vec![None, None, None],
12461            (0..3)
12462                .map(|_| accounts.next_shrink_slot_v1())
12463                .collect::<Vec<_>>()
12464        );
12465
12466        accounts.get_accounts_delta_hash(current_slot);
12467        accounts.add_root(current_slot);
12468
12469        assert_eq!(
12470            vec![Some(7), Some(7), Some(7)],
12471            (0..3)
12472                .map(|_| accounts.next_shrink_slot_v1())
12473                .collect::<Vec<_>>()
12474        );
12475
12476        current_slot += 1;
12477        accounts.get_accounts_delta_hash(current_slot);
12478        accounts.add_root(current_slot);
12479
12480        let slots = (0..6)
12481            .map(|_| accounts.next_shrink_slot_v1())
12482            .collect::<Vec<_>>();
12483
12484        // Because the origin of this data is HashMap (not BTreeMap), key order is arbitrary per cycle.
12485        assert!(
12486            vec![Some(7), Some(8), Some(7), Some(8), Some(7), Some(8)] == slots
12487                || vec![Some(8), Some(7), Some(8), Some(7), Some(8), Some(7)] == slots
12488        );
12489    }
12490
12491    #[test]
12492    fn test_shrink_reset_uncleaned_roots() {
12493        let mut accounts = AccountsDb::new_single_for_tests();
12494        accounts.caching_enabled = false;
12495
12496        accounts.reset_uncleaned_roots_v1();
12497        assert_eq!(
12498            *accounts.shrink_candidate_slots_v1.lock().unwrap(),
12499            vec![] as Vec<Slot>
12500        );
12501
12502        accounts.get_accounts_delta_hash(0);
12503        accounts.add_root(0);
12504        accounts.get_accounts_delta_hash(1);
12505        accounts.add_root(1);
12506        accounts.get_accounts_delta_hash(2);
12507        accounts.add_root(2);
12508
12509        accounts.reset_uncleaned_roots_v1();
12510        let actual_slots = accounts.shrink_candidate_slots_v1.lock().unwrap().clone();
12511        assert_eq!(actual_slots, vec![] as Vec<Slot>);
12512
12513        accounts.reset_uncleaned_roots_v1();
12514        let mut actual_slots = accounts.shrink_candidate_slots_v1.lock().unwrap().clone();
12515        actual_slots.sort_unstable();
12516        assert_eq!(actual_slots, vec![0, 1, 2]);
12517
12518        accounts.accounts_index.clear_roots();
12519        let mut actual_slots = (0..5)
12520            .map(|_| accounts.next_shrink_slot_v1())
12521            .collect::<Vec<_>>();
12522        actual_slots.sort();
12523        assert_eq!(actual_slots, vec![None, None, Some(0), Some(1), Some(2)],);
12524    }
12525
12526    #[test]
12527    fn test_shrink_stale_slots_processed() {
12528        solana_logger::setup();
12529
12530        for startup in &[false, true] {
12531            let accounts = AccountsDb::new_single_for_tests();
12532
12533            let pubkey_count = 100;
12534            let pubkeys: Vec<_> = (0..pubkey_count)
12535                .map(|_| solana_sdk::pubkey::new_rand())
12536                .collect();
12537
12538            let some_lamport = 223;
12539            let no_data = 0;
12540            let owner = *AccountSharedData::default().owner();
12541
12542            let account = AccountSharedData::new(some_lamport, no_data, &owner);
12543
12544            let mut current_slot = 0;
12545
12546            current_slot += 1;
12547            for pubkey in &pubkeys {
12548                accounts.store_uncached(current_slot, &[(pubkey, &account)]);
12549            }
12550            let shrink_slot = current_slot;
12551            accounts.get_accounts_delta_hash(current_slot);
12552            accounts.add_root(current_slot);
12553
12554            current_slot += 1;
12555            let pubkey_count_after_shrink = 10;
12556            let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
12557
12558            for pubkey in updated_pubkeys {
12559                accounts.store_uncached(current_slot, &[(pubkey, &account)]);
12560            }
12561            accounts.get_accounts_delta_hash(current_slot);
12562            accounts.add_root(current_slot);
12563
12564            accounts.clean_accounts(None, false, None);
12565
12566            assert_eq!(
12567                pubkey_count,
12568                accounts.all_account_count_in_append_vec(shrink_slot)
12569            );
12570            accounts.shrink_all_slots(*startup, None);
12571            assert_eq!(
12572                pubkey_count_after_shrink,
12573                accounts.all_account_count_in_append_vec(shrink_slot)
12574            );
12575
12576            let no_ancestors = Ancestors::default();
12577            accounts.update_accounts_hash(
12578                current_slot,
12579                &no_ancestors,
12580                &EpochSchedule::default(),
12581                &RentCollector::default(),
12582            );
12583            accounts
12584                .verify_bank_hash_and_lamports(
12585                    current_slot,
12586                    &no_ancestors,
12587                    22300,
12588                    true,
12589                    &EpochSchedule::default(),
12590                    &RentCollector::default(),
12591                    false,
12592                    false,
12593                )
12594                .unwrap();
12595
12596            let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
12597            accounts
12598                .verify_bank_hash_and_lamports(
12599                    current_slot,
12600                    &no_ancestors,
12601                    22300,
12602                    true,
12603                    &EpochSchedule::default(),
12604                    &RentCollector::default(),
12605                    false,
12606                    false,
12607                )
12608                .unwrap();
12609
12610            // repeating should be no-op
12611            accounts.shrink_all_slots(*startup, None);
12612            assert_eq!(
12613                pubkey_count_after_shrink,
12614                accounts.all_account_count_in_append_vec(shrink_slot)
12615            );
12616        }
12617    }
12618
12619    #[test]
12620    fn test_shrink_candidate_slots() {
12621        solana_logger::setup();
12622
12623        let accounts = AccountsDb::new_single_for_tests();
12624
12625        let pubkey_count = 30000;
12626        let pubkeys: Vec<_> = (0..pubkey_count)
12627            .map(|_| solana_sdk::pubkey::new_rand())
12628            .collect();
12629
12630        let some_lamport = 223;
12631        let no_data = 0;
12632        let owner = *AccountSharedData::default().owner();
12633
12634        let account = AccountSharedData::new(some_lamport, no_data, &owner);
12635
12636        let mut current_slot = 0;
12637
12638        current_slot += 1;
12639        for pubkey in &pubkeys {
12640            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
12641        }
12642        let shrink_slot = current_slot;
12643        accounts.get_accounts_delta_hash(current_slot);
12644        accounts.add_root(current_slot);
12645
12646        current_slot += 1;
12647        let pubkey_count_after_shrink = 25000;
12648        let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
12649
12650        for pubkey in updated_pubkeys {
12651            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
12652        }
12653        accounts.get_accounts_delta_hash(current_slot);
12654        accounts.add_root(current_slot);
12655        accounts.clean_accounts(None, false, None);
12656
12657        assert_eq!(
12658            pubkey_count,
12659            accounts.all_account_count_in_append_vec(shrink_slot)
12660        );
12661
12662        // Only, try to shrink stale slots, nothing happens because 90/100
12663        // is not small enough to do a shrink
12664        accounts.shrink_candidate_slots();
12665        assert_eq!(
12666            pubkey_count,
12667            accounts.all_account_count_in_append_vec(shrink_slot)
12668        );
12669
12670        // Now, do full-shrink.
12671        accounts.shrink_all_slots(false, None);
12672        assert_eq!(
12673            pubkey_count_after_shrink,
12674            accounts.all_account_count_in_append_vec(shrink_slot)
12675        );
12676    }
12677
12678    #[test]
12679    fn test_select_candidates_by_total_usage_no_candidates() {
12680        // no input candidates -- none should be selected
12681        solana_logger::setup();
12682        let candidates: ShrinkCandidates = HashMap::new();
12683
12684        let (selected_candidates, next_candidates) = AccountsDb::select_candidates_by_total_usage(
12685            &candidates,
12686            DEFAULT_ACCOUNTS_SHRINK_RATIO,
12687        );
12688
12689        assert_eq!(0, selected_candidates.len());
12690        assert_eq!(0, next_candidates.len());
12691    }
12692
12693    #[test]
12694    fn test_select_candidates_by_total_usage_3_way_split_condition() {
12695        // three candidates, one selected for shrink, one is put back to the candidate list and one is ignored
12696        solana_logger::setup();
12697        let mut candidates: ShrinkCandidates = HashMap::new();
12698
12699        let common_store_path = Path::new("");
12700        let common_slot_id = 12;
12701        let store_file_size = 2 * PAGE_SIZE;
12702
12703        let store1_id = 22;
12704        let store1 = Arc::new(AccountStorageEntry::new(
12705            common_store_path,
12706            common_slot_id,
12707            store1_id,
12708            store_file_size,
12709        ));
12710        store1.alive_bytes.store(0, Ordering::Release);
12711
12712        candidates
12713            .entry(common_slot_id)
12714            .or_default()
12715            .insert(store1.append_vec_id(), store1.clone());
12716
12717        let store2_id = 44;
12718        let store2 = Arc::new(AccountStorageEntry::new(
12719            common_store_path,
12720            common_slot_id,
12721            store2_id,
12722            store_file_size,
12723        ));
12724
12725        // The store2's alive_ratio is 0.5: as its page aligned alive size is 1 page.
12726        let store2_alive_bytes = (PAGE_SIZE - 1) as usize;
12727        store2
12728            .alive_bytes
12729            .store(store2_alive_bytes, Ordering::Release);
12730        candidates
12731            .entry(common_slot_id)
12732            .or_default()
12733            .insert(store2.append_vec_id(), store2.clone());
12734
12735        let store3_id = 55;
12736        let entry3 = Arc::new(AccountStorageEntry::new(
12737            common_store_path,
12738            common_slot_id,
12739            store3_id,
12740            store_file_size,
12741        ));
12742
12743        // The store3's alive ratio is 1.0 as its page-aligned alive size is 2 pages
12744        let store3_alive_bytes = (PAGE_SIZE + 1) as usize;
12745        entry3
12746            .alive_bytes
12747            .store(store3_alive_bytes, Ordering::Release);
12748
12749        candidates
12750            .entry(common_slot_id)
12751            .or_default()
12752            .insert(entry3.append_vec_id(), entry3.clone());
12753
12754        // Set the target alive ratio to 0.6 so that we can just get rid of store1, the remaining two stores
12755        // alive ratio can be > the target ratio: the actual ratio is 0.75 because of 3 alive pages / 4 total pages.
12756        // The target ratio is also set to larger than store2's alive ratio: 0.5 so that it would be added
12757        // to the candidates list for next round.
12758        let target_alive_ratio = 0.6;
12759        let (selected_candidates, next_candidates) =
12760            AccountsDb::select_candidates_by_total_usage(&candidates, target_alive_ratio);
12761        assert_eq!(1, selected_candidates.len());
12762        assert_eq!(1, selected_candidates[&common_slot_id].len());
12763        assert!(selected_candidates[&common_slot_id].contains(&store1.append_vec_id()));
12764        assert_eq!(1, next_candidates.len());
12765        assert!(next_candidates[&common_slot_id].contains(&store2.append_vec_id()));
12766    }
12767
12768    #[test]
12769    fn test_select_candidates_by_total_usage_2_way_split_condition() {
12770        // three candidates, 2 are selected for shrink, one is ignored
12771        solana_logger::setup();
12772        let mut candidates: ShrinkCandidates = HashMap::new();
12773
12774        let common_store_path = Path::new("");
12775        let common_slot_id = 12;
12776        let store_file_size = 2 * PAGE_SIZE;
12777
12778        let store1_id = 22;
12779        let store1 = Arc::new(AccountStorageEntry::new(
12780            common_store_path,
12781            common_slot_id,
12782            store1_id,
12783            store_file_size,
12784        ));
12785        store1.alive_bytes.store(0, Ordering::Release);
12786
12787        candidates
12788            .entry(common_slot_id)
12789            .or_default()
12790            .insert(store1.append_vec_id(), store1.clone());
12791
12792        let store2_id = 44;
12793        let store2 = Arc::new(AccountStorageEntry::new(
12794            common_store_path,
12795            common_slot_id,
12796            store2_id,
12797            store_file_size,
12798        ));
12799
12800        // The store2's alive_ratio is 0.5: as its page aligned alive size is 1 page.
12801        let store2_alive_bytes = (PAGE_SIZE - 1) as usize;
12802        store2
12803            .alive_bytes
12804            .store(store2_alive_bytes, Ordering::Release);
12805        candidates
12806            .entry(common_slot_id)
12807            .or_default()
12808            .insert(store2.append_vec_id(), store2.clone());
12809
12810        let store3_id = 55;
12811        let entry3 = Arc::new(AccountStorageEntry::new(
12812            common_store_path,
12813            common_slot_id,
12814            store3_id,
12815            store_file_size,
12816        ));
12817
12818        // The store3's alive ratio is 1.0 as its page-aligned alive size is 2 pages
12819        let store3_alive_bytes = (PAGE_SIZE + 1) as usize;
12820        entry3
12821            .alive_bytes
12822            .store(store3_alive_bytes, Ordering::Release);
12823
12824        candidates
12825            .entry(common_slot_id)
12826            .or_default()
12827            .insert(entry3.append_vec_id(), entry3.clone());
12828
12829        // Set the target ratio to default (0.8), both store1 and store2 must be selected and store3 is ignored.
12830        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
12831        let (selected_candidates, next_candidates) =
12832            AccountsDb::select_candidates_by_total_usage(&candidates, target_alive_ratio);
12833        assert_eq!(1, selected_candidates.len());
12834        assert_eq!(2, selected_candidates[&common_slot_id].len());
12835        assert!(selected_candidates[&common_slot_id].contains(&store1.append_vec_id()));
12836        assert!(selected_candidates[&common_slot_id].contains(&store2.append_vec_id()));
12837        assert_eq!(0, next_candidates.len());
12838    }
12839
12840    #[test]
12841    fn test_select_candidates_by_total_usage_all_clean() {
12842        // 2 candidates, they must be selected to achieve the target alive ratio
12843        solana_logger::setup();
12844        let mut candidates: ShrinkCandidates = HashMap::new();
12845
12846        let slot1 = 12;
12847        let common_store_path = Path::new("");
12848
12849        let store_file_size = 4 * PAGE_SIZE;
12850        let store1_id = 22;
12851        let store1 = Arc::new(AccountStorageEntry::new(
12852            common_store_path,
12853            slot1,
12854            store1_id,
12855            store_file_size,
12856        ));
12857
12858        // store1 has 1 page-aligned alive bytes, its alive ratio is 1/4: 0.25
12859        let store1_alive_bytes = (PAGE_SIZE - 1) as usize;
12860        store1
12861            .alive_bytes
12862            .store(store1_alive_bytes, Ordering::Release);
12863
12864        candidates
12865            .entry(slot1)
12866            .or_default()
12867            .insert(store1.append_vec_id(), store1.clone());
12868
12869        let store2_id = 44;
12870        let slot2 = 44;
12871        let store2 = Arc::new(AccountStorageEntry::new(
12872            common_store_path,
12873            slot2,
12874            store2_id,
12875            store_file_size,
12876        ));
12877
12878        // store2 has 2 page-aligned bytes, its alive ratio is 2/4: 0.5
12879        let store2_alive_bytes = (PAGE_SIZE + 1) as usize;
12880        store2
12881            .alive_bytes
12882            .store(store2_alive_bytes, Ordering::Release);
12883
12884        candidates
12885            .entry(slot2)
12886            .or_default()
12887            .insert(store2.append_vec_id(), store2.clone());
12888
12889        // Set the target ratio to default (0.8), both stores from the two different slots must be selected.
12890        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
12891        let (selected_candidates, next_candidates) =
12892            AccountsDb::select_candidates_by_total_usage(&candidates, target_alive_ratio);
12893        assert_eq!(2, selected_candidates.len());
12894        assert_eq!(1, selected_candidates[&slot1].len());
12895        assert_eq!(1, selected_candidates[&slot2].len());
12896
12897        assert!(selected_candidates[&slot1].contains(&store1.append_vec_id()));
12898        assert!(selected_candidates[&slot2].contains(&store2.append_vec_id()));
12899        assert_eq!(0, next_candidates.len());
12900    }
12901
12902    #[test]
12903    fn test_shrink_stale_slots_skipped() {
12904        solana_logger::setup();
12905
12906        let mut accounts = AccountsDb::new_single_for_tests();
12907        accounts.caching_enabled = false;
12908
12909        let pubkey_count = 30000;
12910        let pubkeys: Vec<_> = (0..pubkey_count)
12911            .map(|_| solana_sdk::pubkey::new_rand())
12912            .collect();
12913
12914        let some_lamport = 223;
12915        let no_data = 0;
12916        let owner = *AccountSharedData::default().owner();
12917
12918        let account = AccountSharedData::new(some_lamport, no_data, &owner);
12919
12920        let mut current_slot = 0;
12921
12922        current_slot += 1;
12923        for pubkey in &pubkeys {
12924            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
12925        }
12926        let shrink_slot = current_slot;
12927        accounts.get_accounts_delta_hash(current_slot);
12928        accounts.add_root(current_slot);
12929
12930        current_slot += 1;
12931        let pubkey_count_after_shrink = 25000;
12932        let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
12933
12934        for pubkey in updated_pubkeys {
12935            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
12936        }
12937        accounts.get_accounts_delta_hash(current_slot);
12938        accounts.add_root(current_slot);
12939
12940        accounts.clean_accounts(None, false, None);
12941
12942        assert_eq!(
12943            pubkey_count,
12944            accounts.all_account_count_in_append_vec(shrink_slot)
12945        );
12946
12947        // Only, try to shrink stale slots.
12948        accounts.shrink_all_stale_slots_v1();
12949        assert_eq!(
12950            pubkey_count,
12951            accounts.all_account_count_in_append_vec(shrink_slot)
12952        );
12953
12954        // Now, do full-shrink.
12955        accounts.shrink_all_slots(false, None);
12956        assert_eq!(
12957            pubkey_count_after_shrink,
12958            accounts.all_account_count_in_append_vec(shrink_slot)
12959        );
12960    }
12961
12962    const UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE: UpsertReclaim =
12963        UpsertReclaim::PopulateReclaims;
12964
12965    #[test]
12966    fn test_delete_dependencies() {
12967        solana_logger::setup();
12968        let accounts_index = AccountsIndex::default_for_tests();
12969        let key0 = Pubkey::new_from_array([0u8; 32]);
12970        let key1 = Pubkey::new_from_array([1u8; 32]);
12971        let key2 = Pubkey::new_from_array([2u8; 32]);
12972        let info0 = AccountInfo::new(StorageLocation::AppendVec(0, 0), 0, 0);
12973        let info1 = AccountInfo::new(StorageLocation::AppendVec(1, 0), 0, 0);
12974        let info2 = AccountInfo::new(StorageLocation::AppendVec(2, 0), 0, 0);
12975        let info3 = AccountInfo::new(StorageLocation::AppendVec(3, 0), 0, 0);
12976        let mut reclaims = vec![];
12977        accounts_index.upsert(
12978            0,
12979            0,
12980            &key0,
12981            &AccountSharedData::default(),
12982            &AccountSecondaryIndexes::default(),
12983            info0,
12984            &mut reclaims,
12985            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
12986        );
12987        accounts_index.upsert(
12988            1,
12989            1,
12990            &key0,
12991            &AccountSharedData::default(),
12992            &AccountSecondaryIndexes::default(),
12993            info1,
12994            &mut reclaims,
12995            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
12996        );
12997        accounts_index.upsert(
12998            1,
12999            1,
13000            &key1,
13001            &AccountSharedData::default(),
13002            &AccountSecondaryIndexes::default(),
13003            info1,
13004            &mut reclaims,
13005            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
13006        );
13007        accounts_index.upsert(
13008            2,
13009            2,
13010            &key1,
13011            &AccountSharedData::default(),
13012            &AccountSecondaryIndexes::default(),
13013            info2,
13014            &mut reclaims,
13015            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
13016        );
13017        accounts_index.upsert(
13018            2,
13019            2,
13020            &key2,
13021            &AccountSharedData::default(),
13022            &AccountSecondaryIndexes::default(),
13023            info2,
13024            &mut reclaims,
13025            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
13026        );
13027        accounts_index.upsert(
13028            3,
13029            3,
13030            &key2,
13031            &AccountSharedData::default(),
13032            &AccountSecondaryIndexes::default(),
13033            info3,
13034            &mut reclaims,
13035            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
13036        );
13037        accounts_index.add_root(0, false);
13038        accounts_index.add_root(1, false);
13039        accounts_index.add_root(2, false);
13040        accounts_index.add_root(3, false);
13041        let mut purges = HashMap::new();
13042        let (key0_entry, _) = accounts_index.get_for_tests(&key0, None, None).unwrap();
13043        purges.insert(key0, accounts_index.roots_and_ref_count(&key0_entry, None));
13044        let (key1_entry, _) = accounts_index.get_for_tests(&key1, None, None).unwrap();
13045        purges.insert(key1, accounts_index.roots_and_ref_count(&key1_entry, None));
13046        let (key2_entry, _) = accounts_index.get_for_tests(&key2, None, None).unwrap();
13047        purges.insert(key2, accounts_index.roots_and_ref_count(&key2_entry, None));
13048        for (key, (list, ref_count)) in &purges {
13049            info!(" purge {} ref_count {} =>", key, ref_count);
13050            for x in list {
13051                info!("  {:?}", x);
13052            }
13053        }
13054
13055        let mut store_counts = HashMap::new();
13056        store_counts.insert(0, (0, HashSet::from_iter(vec![key0])));
13057        store_counts.insert(1, (0, HashSet::from_iter(vec![key0, key1])));
13058        store_counts.insert(2, (0, HashSet::from_iter(vec![key1, key2])));
13059        store_counts.insert(3, (1, HashSet::from_iter(vec![key2])));
13060        AccountsDb::calc_delete_dependencies(&purges, &mut store_counts);
13061        let mut stores: Vec<_> = store_counts.keys().cloned().collect();
13062        stores.sort_unstable();
13063        for store in &stores {
13064            info!(
13065                "store: {:?} : {:?}",
13066                store,
13067                store_counts.get(store).unwrap()
13068            );
13069        }
13070        for x in 0..3 {
13071            assert!(store_counts[&x].0 >= 1);
13072        }
13073    }
13074
13075    #[test]
13076    fn test_account_balance_for_capitalization_sysvar() {
13077        let normal_sysvar = solana_sdk::account::create_account_for_test(
13078            &solana_sdk::slot_history::SlotHistory::default(),
13079        );
13080        assert_eq!(normal_sysvar.lamports(), 1);
13081    }
13082
13083    #[test]
13084    fn test_account_balance_for_capitalization_native_program() {
13085        let normal_native_program =
13086            solana_sdk::native_loader::create_loadable_account_for_test("foo");
13087        assert_eq!(normal_native_program.lamports(), 1);
13088    }
13089
13090    #[test]
13091    fn test_checked_sum_for_capitalization_normal() {
13092        assert_eq!(
13093            AccountsDb::checked_sum_for_capitalization(vec![1, 2].into_iter()),
13094            3
13095        );
13096    }
13097
13098    #[test]
13099    #[should_panic(expected = "overflow is detected while summing capitalization")]
13100    fn test_checked_sum_for_capitalization_overflow() {
13101        assert_eq!(
13102            AccountsDb::checked_sum_for_capitalization(vec![1, u64::max_value()].into_iter()),
13103            3
13104        );
13105    }
13106
13107    #[test]
13108    fn test_store_overhead() {
13109        solana_logger::setup();
13110        let accounts = AccountsDb::new_single_for_tests();
13111        let account = AccountSharedData::default();
13112        let pubkey = solana_sdk::pubkey::new_rand();
13113        accounts.store_uncached(0, &[(&pubkey, &account)]);
13114        let slot_stores = accounts.storage.get_slot_stores(0).unwrap();
13115        let mut total_len = 0;
13116        for (_id, store) in slot_stores.read().unwrap().iter() {
13117            total_len += store.accounts.len();
13118        }
13119        info!("total: {}", total_len);
13120        assert!(total_len < STORE_META_OVERHEAD);
13121    }
13122
13123    #[test]
13124    fn test_store_clean_after_shrink() {
13125        solana_logger::setup();
13126        let accounts = AccountsDb::new_with_config_for_tests(
13127            vec![],
13128            &ClusterType::Development,
13129            AccountSecondaryIndexes::default(),
13130            true,
13131            AccountShrinkThreshold::default(),
13132        );
13133
13134        let account = AccountSharedData::new(1, 16 * 4096, &Pubkey::default());
13135        let pubkey1 = solana_sdk::pubkey::new_rand();
13136        accounts.store_cached((0, &[(&pubkey1, &account)][..]), None);
13137
13138        let pubkey2 = solana_sdk::pubkey::new_rand();
13139        accounts.store_cached((0, &[(&pubkey2, &account)][..]), None);
13140
13141        let zero_account = AccountSharedData::new(0, 1, &Pubkey::default());
13142        accounts.store_cached((1, &[(&pubkey1, &zero_account)][..]), None);
13143
13144        // Add root 0 and flush separately
13145        accounts.get_accounts_delta_hash(0);
13146        accounts.add_root(0);
13147        accounts.flush_accounts_cache(true, None);
13148
13149        // clear out the dirty keys
13150        accounts.clean_accounts(None, false, None);
13151
13152        // flush 1
13153        accounts.get_accounts_delta_hash(1);
13154        accounts.add_root(1);
13155        accounts.flush_accounts_cache(true, None);
13156
13157        accounts.print_accounts_stats("pre-clean");
13158
13159        // clean to remove pubkey1 from 0,
13160        // shrink to shrink pubkey1 from 0
13161        // then another clean to remove pubkey1 from slot 1
13162        accounts.clean_accounts(None, false, None);
13163
13164        accounts.shrink_candidate_slots();
13165
13166        accounts.clean_accounts(None, false, None);
13167
13168        accounts.print_accounts_stats("post-clean");
13169        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
13170    }
13171
13172    #[test]
13173    fn test_store_reuse() {
13174        solana_logger::setup();
13175        let accounts = AccountsDb::new_sized(vec![], 4096);
13176
13177        let size = 100;
13178        let num_accounts: usize = 100;
13179        let mut keys = Vec::new();
13180        for i in 0..num_accounts {
13181            let account = AccountSharedData::new((i + 1) as u64, size, &Pubkey::default());
13182            let pubkey = solana_sdk::pubkey::new_rand();
13183            accounts.store_uncached(0, &[(&pubkey, &account)]);
13184            keys.push(pubkey);
13185        }
13186        accounts.add_root(0);
13187
13188        for (i, key) in keys[1..].iter().enumerate() {
13189            let account =
13190                AccountSharedData::new((1 + i + num_accounts) as u64, size, &Pubkey::default());
13191            accounts.store_uncached(1, &[(key, &account)]);
13192        }
13193        accounts.add_root(1);
13194        accounts.clean_accounts(None, false, None);
13195        accounts.shrink_all_slots(false, None);
13196
13197        // Clean again to flush the dirty stores
13198        // and allow them to be recycled in the next step
13199        accounts.clean_accounts(None, false, None);
13200        accounts.print_accounts_stats("post-shrink");
13201        let num_stores = accounts.recycle_stores.read().unwrap().entry_count();
13202        assert!(num_stores > 0);
13203
13204        let mut account_refs = Vec::new();
13205        let num_to_store = 20;
13206        for (i, key) in keys[..num_to_store].iter().enumerate() {
13207            let account = AccountSharedData::new(
13208                (1 + i + 2 * num_accounts) as u64,
13209                i + 20,
13210                &Pubkey::default(),
13211            );
13212            accounts.store_uncached(2, &[(key, &account)]);
13213            account_refs.push(account);
13214        }
13215        assert!(accounts.recycle_stores.read().unwrap().entry_count() < num_stores);
13216
13217        accounts.print_accounts_stats("post-store");
13218
13219        let mut ancestors = Ancestors::default();
13220        ancestors.insert(1, 0);
13221        ancestors.insert(2, 1);
13222        for (key, account_ref) in keys[..num_to_store].iter().zip(account_refs) {
13223            assert_eq!(
13224                accounts.load_without_fixed_root(&ancestors, key).unwrap().0,
13225                account_ref
13226            );
13227        }
13228    }
13229
13230    #[test]
13231    #[should_panic(expected = "We've run out of storage ids!")]
13232    fn test_wrapping_append_vec_id() {
13233        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13234        let zero_lamport_account =
13235            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13236
13237        // set 'next' id to the max possible value
13238        db.next_id.store(AppendVecId::MAX, Ordering::Release);
13239        let slots = 3;
13240        let keys = (0..slots).map(|_| Pubkey::new_unique()).collect::<Vec<_>>();
13241        // write unique keys to successive slots
13242        keys.iter().enumerate().for_each(|(slot, key)| {
13243            let slot = slot as Slot;
13244            db.store_uncached(slot, &[(key, &zero_lamport_account)]);
13245            db.get_accounts_delta_hash(slot);
13246            db.add_root(slot);
13247        });
13248        assert_eq!(slots - 1, db.next_id.load(Ordering::Acquire));
13249        let ancestors = Ancestors::default();
13250        keys.iter().for_each(|key| {
13251            assert!(db.load_without_fixed_root(&ancestors, key).is_some());
13252        });
13253    }
13254
13255    #[test]
13256    #[should_panic(expected = "We've run out of storage ids!")]
13257    fn test_reuse_append_vec_id() {
13258        solana_logger::setup();
13259        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13260        let zero_lamport_account =
13261            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13262
13263        // set 'next' id to the max possible value
13264        db.next_id.store(AppendVecId::MAX, Ordering::Release);
13265        let slots = 3;
13266        let keys = (0..slots).map(|_| Pubkey::new_unique()).collect::<Vec<_>>();
13267        // write unique keys to successive slots
13268        keys.iter().enumerate().for_each(|(slot, key)| {
13269            let slot = slot as Slot;
13270            db.store_uncached(slot, &[(key, &zero_lamport_account)]);
13271            db.get_accounts_delta_hash(slot);
13272            db.add_root(slot);
13273            // reset next_id to what it was previously to cause us to re-use the same id
13274            db.next_id.store(AppendVecId::MAX, Ordering::Release);
13275        });
13276        let ancestors = Ancestors::default();
13277        keys.iter().for_each(|key| {
13278            assert!(db.load_without_fixed_root(&ancestors, key).is_some());
13279        });
13280    }
13281
13282    #[test]
13283    fn test_zero_lamport_new_root_not_cleaned() {
13284        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13285        let account_key = Pubkey::new_unique();
13286        let zero_lamport_account =
13287            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13288
13289        // Store zero lamport account into slots 0 and 1, root both slots
13290        db.store_uncached(0, &[(&account_key, &zero_lamport_account)]);
13291        db.store_uncached(1, &[(&account_key, &zero_lamport_account)]);
13292        db.get_accounts_delta_hash(0);
13293        db.add_root(0);
13294        db.get_accounts_delta_hash(1);
13295        db.add_root(1);
13296
13297        // Only clean zero lamport accounts up to slot 0
13298        db.clean_accounts(Some(0), false, None);
13299
13300        // Should still be able to find zero lamport account in slot 1
13301        assert_eq!(
13302            db.load_without_fixed_root(&Ancestors::default(), &account_key),
13303            Some((zero_lamport_account, 1))
13304        );
13305    }
13306
13307    #[test]
13308    fn test_store_load_cached() {
13309        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13310        db.caching_enabled = true;
13311        let key = Pubkey::default();
13312        let account0 = AccountSharedData::new(1, 0, &key);
13313        let slot = 0;
13314        db.store_cached((slot, &[(&key, &account0)][..]), None);
13315
13316        // Load with no ancestors and no root will return nothing
13317        assert!(db
13318            .load_without_fixed_root(&Ancestors::default(), &key)
13319            .is_none());
13320
13321        // Load with ancestors not equal to `slot` will return nothing
13322        let ancestors = vec![(slot + 1, 1)].into_iter().collect();
13323        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
13324
13325        // Load with ancestors equal to `slot` will return the account
13326        let ancestors = vec![(slot, 1)].into_iter().collect();
13327        assert_eq!(
13328            db.load_without_fixed_root(&ancestors, &key),
13329            Some((account0.clone(), slot))
13330        );
13331
13332        // Adding root will return the account even without ancestors
13333        db.add_root(slot);
13334        assert_eq!(
13335            db.load_without_fixed_root(&Ancestors::default(), &key),
13336            Some((account0, slot))
13337        );
13338    }
13339
13340    #[test]
13341    fn test_store_flush_load_cached() {
13342        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13343        db.caching_enabled = true;
13344        let key = Pubkey::default();
13345        let account0 = AccountSharedData::new(1, 0, &key);
13346        let slot = 0;
13347        db.store_cached((slot, &[(&key, &account0)][..]), None);
13348        db.mark_slot_frozen(slot);
13349
13350        // No root was added yet, requires an ancestor to find
13351        // the account
13352        db.flush_accounts_cache(true, None);
13353        let ancestors = vec![(slot, 1)].into_iter().collect();
13354        assert_eq!(
13355            db.load_without_fixed_root(&ancestors, &key),
13356            Some((account0.clone(), slot))
13357        );
13358
13359        // Add root then flush
13360        db.add_root(slot);
13361        db.flush_accounts_cache(true, None);
13362        assert_eq!(
13363            db.load_without_fixed_root(&Ancestors::default(), &key),
13364            Some((account0, slot))
13365        );
13366    }
13367
13368    #[test]
13369    fn test_flush_accounts_cache() {
13370        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13371        db.caching_enabled = true;
13372        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
13373
13374        let unrooted_slot = 4;
13375        let root5 = 5;
13376        let root6 = 6;
13377        let unrooted_key = solana_sdk::pubkey::new_rand();
13378        let key5 = solana_sdk::pubkey::new_rand();
13379        let key6 = solana_sdk::pubkey::new_rand();
13380        db.store_cached((unrooted_slot, &[(&unrooted_key, &account0)][..]), None);
13381        db.store_cached((root5, &[(&key5, &account0)][..]), None);
13382        db.store_cached((root6, &[(&key6, &account0)][..]), None);
13383        for slot in &[unrooted_slot, root5, root6] {
13384            db.mark_slot_frozen(*slot);
13385        }
13386        db.add_root(root5);
13387        db.add_root(root6);
13388
13389        // Unrooted slot should be able to be fetched before the flush
13390        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
13391        assert_eq!(
13392            db.load_without_fixed_root(&ancestors, &unrooted_key),
13393            Some((account0.clone(), unrooted_slot))
13394        );
13395        db.flush_accounts_cache(true, None);
13396
13397        // After the flush, the unrooted slot is still in the cache
13398        assert!(db
13399            .load_without_fixed_root(&ancestors, &unrooted_key)
13400            .is_some());
13401        assert!(db
13402            .accounts_index
13403            .get_account_read_entry(&unrooted_key)
13404            .is_some());
13405        assert_eq!(db.accounts_cache.num_slots(), 1);
13406        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_some());
13407        assert_eq!(
13408            db.load_without_fixed_root(&Ancestors::default(), &key5),
13409            Some((account0.clone(), root5))
13410        );
13411        assert_eq!(
13412            db.load_without_fixed_root(&Ancestors::default(), &key6),
13413            Some((account0, root6))
13414        );
13415    }
13416
13417    fn max_cache_slots() -> usize {
13418        // this used to be the limiting factor - used here to facilitate tests.
13419        200
13420    }
13421
13422    #[test]
13423    fn test_flush_accounts_cache_if_needed() {
13424        run_test_flush_accounts_cache_if_needed(0, 2 * max_cache_slots());
13425        run_test_flush_accounts_cache_if_needed(2 * max_cache_slots(), 0);
13426        run_test_flush_accounts_cache_if_needed(max_cache_slots() - 1, 0);
13427        run_test_flush_accounts_cache_if_needed(0, max_cache_slots() - 1);
13428        run_test_flush_accounts_cache_if_needed(max_cache_slots(), 0);
13429        run_test_flush_accounts_cache_if_needed(0, max_cache_slots());
13430        run_test_flush_accounts_cache_if_needed(2 * max_cache_slots(), 2 * max_cache_slots());
13431        run_test_flush_accounts_cache_if_needed(max_cache_slots() - 1, max_cache_slots() - 1);
13432        run_test_flush_accounts_cache_if_needed(max_cache_slots(), max_cache_slots());
13433    }
13434
13435    fn run_test_flush_accounts_cache_if_needed(num_roots: usize, num_unrooted: usize) {
13436        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
13437        db.write_cache_limit_bytes = Some(max_cache_slots() as u64);
13438        db.caching_enabled = true;
13439        let space = 1; // # data bytes per account. write cache counts data len
13440        let account0 = AccountSharedData::new(1, space, &Pubkey::default());
13441        let mut keys = vec![];
13442        let num_slots = 2 * max_cache_slots();
13443        for i in 0..num_roots + num_unrooted {
13444            let key = Pubkey::new_unique();
13445            db.store_cached((i as Slot, &[(&key, &account0)][..]), None);
13446            keys.push(key);
13447            db.mark_slot_frozen(i as Slot);
13448            if i < num_roots {
13449                db.add_root(i as Slot);
13450            }
13451        }
13452
13453        db.flush_accounts_cache(false, None);
13454
13455        let total_slots = num_roots + num_unrooted;
13456        // If there's <= the max size, then nothing will be flushed from the slot
13457        if total_slots <= max_cache_slots() {
13458            assert_eq!(db.accounts_cache.num_slots(), total_slots);
13459        } else {
13460            // Otherwise, all the roots are flushed, and only at most max_cache_slots()
13461            // of the unrooted slots are kept in the cache
13462            let expected_size = std::cmp::min(num_unrooted, max_cache_slots());
13463            if expected_size > 0 {
13464                // +1: slot is 1-based. slot 1 has 1 byte of data
13465                for unrooted_slot in (total_slots - expected_size + 1)..total_slots {
13466                    assert!(
13467                        db.accounts_cache
13468                            .slot_cache(unrooted_slot as Slot)
13469                            .is_some(),
13470                        "unrooted_slot: {}, total_slots: {}, expected_size: {}",
13471                        unrooted_slot,
13472                        total_slots,
13473                        expected_size
13474                    );
13475                }
13476            }
13477        }
13478
13479        // Should still be able to fetch all the accounts after flush
13480        for (slot, key) in (0..num_slots as Slot).zip(keys) {
13481            let ancestors = if slot < num_roots as Slot {
13482                Ancestors::default()
13483            } else {
13484                vec![(slot, 1)].into_iter().collect()
13485            };
13486            assert_eq!(
13487                db.load_without_fixed_root(&ancestors, &key),
13488                Some((account0.clone(), slot))
13489            );
13490        }
13491    }
13492
13493    fn slot_stores(db: &AccountsDb, slot: Slot) -> Vec<Arc<AccountStorageEntry>> {
13494        db.storage
13495            .get_slot_storage_entries(slot)
13496            .unwrap_or_default()
13497    }
13498
13499    #[test]
13500    fn test_read_only_accounts_cache() {
13501        let caching_enabled = true;
13502        let db = Arc::new(AccountsDb::new_with_config_for_tests(
13503            Vec::new(),
13504            &ClusterType::Development,
13505            AccountSecondaryIndexes::default(),
13506            caching_enabled,
13507            AccountShrinkThreshold::default(),
13508        ));
13509
13510        let account_key = Pubkey::new_unique();
13511        let zero_lamport_account =
13512            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13513        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13514        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13515        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13516
13517        db.add_root(0);
13518        db.add_root(1);
13519        db.clean_accounts(None, false, None);
13520        db.flush_accounts_cache(true, None);
13521        db.clean_accounts(None, false, None);
13522        db.add_root(2);
13523
13524        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13525        let account = db
13526            .load_with_fixed_root(&Ancestors::default(), &account_key, LoadZeroLamports::None)
13527            .map(|(account, _)| account)
13528            .unwrap();
13529        assert_eq!(account.lamports(), 1);
13530        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13531        let account = db
13532            .load_with_fixed_root(&Ancestors::default(), &account_key, LoadZeroLamports::None)
13533            .map(|(account, _)| account)
13534            .unwrap();
13535        assert_eq!(account.lamports(), 1);
13536        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13537        db.store_cached((2, &[(&account_key, &zero_lamport_account)][..]), None);
13538        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13539        let account = db
13540            .load_with_fixed_root(&Ancestors::default(), &account_key, LoadZeroLamports::None)
13541            .map(|(account, _)| account);
13542        assert!(account.is_none());
13543        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
13544    }
13545
13546    #[test]
13547    fn test_flush_cache_clean() {
13548        let caching_enabled = true;
13549        let db = Arc::new(AccountsDb::new_with_config_for_tests(
13550            Vec::new(),
13551            &ClusterType::Development,
13552            AccountSecondaryIndexes::default(),
13553            caching_enabled,
13554            AccountShrinkThreshold::default(),
13555        ));
13556
13557        let account_key = Pubkey::new_unique();
13558        let zero_lamport_account =
13559            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13560        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13561        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13562        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13563
13564        db.add_root(0);
13565        db.add_root(1);
13566
13567        // Clean should not remove anything yet as nothing has been flushed
13568        db.clean_accounts(None, false, None);
13569        let account = db
13570            .do_load(
13571                &Ancestors::default(),
13572                &account_key,
13573                Some(0),
13574                LoadHint::Unspecified,
13575            )
13576            .unwrap();
13577        assert_eq!(account.0.lamports(), 0);
13578        // since this item is in the cache, it should not be in the read only cache
13579        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
13580
13581        // Flush, then clean again. Should not need another root to initiate the cleaning
13582        // because `accounts_index.uncleaned_roots` should be correct
13583        db.flush_accounts_cache(true, None);
13584        db.clean_accounts(None, false, None);
13585        assert!(db
13586            .do_load(
13587                &Ancestors::default(),
13588                &account_key,
13589                Some(0),
13590                LoadHint::Unspecified
13591            )
13592            .is_none());
13593    }
13594
13595    #[test]
13596    fn test_flush_cache_dont_clean_zero_lamport_account() {
13597        let caching_enabled = true;
13598        let db = Arc::new(AccountsDb::new_with_config_for_tests(
13599            Vec::new(),
13600            &ClusterType::Development,
13601            AccountSecondaryIndexes::default(),
13602            caching_enabled,
13603            AccountShrinkThreshold::default(),
13604        ));
13605
13606        let zero_lamport_account_key = Pubkey::new_unique();
13607        let other_account_key = Pubkey::new_unique();
13608
13609        let original_lamports = 1;
13610        let slot0_account =
13611            AccountSharedData::new(original_lamports, 1, AccountSharedData::default().owner());
13612        let zero_lamport_account =
13613            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13614
13615        // Store into slot 0, and then flush the slot to storage
13616        db.store_cached(
13617            (0, &[(&zero_lamport_account_key, &slot0_account)][..]),
13618            None,
13619        );
13620        // Second key keeps other lamport account entry for slot 0 alive,
13621        // preventing clean of the zero_lamport_account in slot 1.
13622        db.store_cached((0, &[(&other_account_key, &slot0_account)][..]), None);
13623        db.add_root(0);
13624        db.flush_accounts_cache(true, None);
13625        assert!(!db.storage.get_slot_storage_entries(0).unwrap().is_empty());
13626
13627        // Store into slot 1, a dummy slot that will be dead and purged before flush
13628        db.store_cached(
13629            (1, &[(&zero_lamport_account_key, &zero_lamport_account)][..]),
13630            None,
13631        );
13632
13633        // Store into slot 2, which makes all updates from slot 1 outdated.
13634        // This means slot 1 is a dead slot. Later, slot 1 will be cleaned/purged
13635        // before it even reaches storage, but this purge of slot 1should not affect
13636        // the refcount of `zero_lamport_account_key` because cached keys do not bump
13637        // the refcount in the index. This means clean should *not* remove
13638        // `zero_lamport_account_key` from slot 2
13639        db.store_cached(
13640            (2, &[(&zero_lamport_account_key, &zero_lamport_account)][..]),
13641            None,
13642        );
13643        db.add_root(1);
13644        db.add_root(2);
13645
13646        // Flush, then clean. Should not need another root to initiate the cleaning
13647        // because `accounts_index.uncleaned_roots` should be correct
13648        db.flush_accounts_cache(true, None);
13649        db.clean_accounts(None, false, None);
13650
13651        // The `zero_lamport_account_key` is still alive in slot 1, so refcount for the
13652        // pubkey should be 2
13653        assert_eq!(
13654            db.accounts_index
13655                .ref_count_from_storage(&zero_lamport_account_key),
13656            2
13657        );
13658        assert_eq!(
13659            db.accounts_index.ref_count_from_storage(&other_account_key),
13660            1
13661        );
13662
13663        // The zero-lamport account in slot 2 should not be purged yet, because the
13664        // entry in slot 1 is blocking cleanup of the zero-lamport account.
13665        let max_root = None;
13666        // Fine to simulate a transaction load since we are not doing any out of band
13667        // removals, only using clean_accounts
13668        let load_hint = LoadHint::FixedMaxRoot;
13669        assert_eq!(
13670            db.do_load(
13671                &Ancestors::default(),
13672                &zero_lamport_account_key,
13673                max_root,
13674                load_hint
13675            )
13676            .unwrap()
13677            .0
13678            .lamports(),
13679            0
13680        );
13681    }
13682
13683    struct ScanTracker {
13684        t_scan: JoinHandle<()>,
13685        exit: Arc<AtomicBool>,
13686    }
13687
13688    impl ScanTracker {
13689        fn exit(self) -> thread::Result<()> {
13690            self.exit.store(true, Ordering::Relaxed);
13691            self.t_scan.join()
13692        }
13693    }
13694
13695    fn setup_scan(
13696        db: Arc<AccountsDb>,
13697        scan_ancestors: Arc<Ancestors>,
13698        bank_id: BankId,
13699        stall_key: Pubkey,
13700    ) -> ScanTracker {
13701        let exit = Arc::new(AtomicBool::new(false));
13702        let exit_ = exit.clone();
13703        let ready = Arc::new(AtomicBool::new(false));
13704        let ready_ = ready.clone();
13705
13706        let t_scan = Builder::new()
13707            .name("scan".to_string())
13708            .spawn(move || {
13709                db.scan_accounts(
13710                    &scan_ancestors,
13711                    bank_id,
13712                    |_collector: &mut Vec<(Pubkey, AccountSharedData)>, maybe_account| {
13713                        ready_.store(true, Ordering::Relaxed);
13714                        if let Some((pubkey, _, _)) = maybe_account {
13715                            if *pubkey == stall_key {
13716                                loop {
13717                                    if exit_.load(Ordering::Relaxed) {
13718                                        break;
13719                                    } else {
13720                                        sleep(Duration::from_millis(10));
13721                                    }
13722                                }
13723                            }
13724                        }
13725                    },
13726                    &ScanConfig::default(),
13727                )
13728                .unwrap();
13729            })
13730            .unwrap();
13731
13732        // Wait for scan to start
13733        while !ready.load(Ordering::Relaxed) {
13734            sleep(Duration::from_millis(10));
13735        }
13736
13737        ScanTracker { t_scan, exit }
13738    }
13739
13740    #[test]
13741    fn test_scan_flush_accounts_cache_then_clean_drop() {
13742        let caching_enabled = true;
13743        let db = Arc::new(AccountsDb::new_with_config_for_tests(
13744            Vec::new(),
13745            &ClusterType::Development,
13746            AccountSecondaryIndexes::default(),
13747            caching_enabled,
13748            AccountShrinkThreshold::default(),
13749        ));
13750        let account_key = Pubkey::new_unique();
13751        let account_key2 = Pubkey::new_unique();
13752        let zero_lamport_account =
13753            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
13754        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
13755        let slot2_account = AccountSharedData::new(2, 1, AccountSharedData::default().owner());
13756
13757        /*
13758            Store zero lamport account into slots 0, 1, 2 where
13759            root slots are 0, 2, and slot 1 is unrooted.
13760                                    0 (root)
13761                                /        \
13762                              1            2 (root)
13763        */
13764        db.store_cached((0, &[(&account_key, &zero_lamport_account)][..]), None);
13765        db.store_cached((1, &[(&account_key, &slot1_account)][..]), None);
13766        // Fodder for the scan so that the lock on `account_key` is not held
13767        db.store_cached((1, &[(&account_key2, &slot1_account)][..]), None);
13768        db.store_cached((2, &[(&account_key, &slot2_account)][..]), None);
13769        db.get_accounts_delta_hash(0);
13770
13771        let max_scan_root = 0;
13772        db.add_root(max_scan_root);
13773        let scan_ancestors: Arc<Ancestors> = Arc::new(vec![(0, 1), (1, 1)].into_iter().collect());
13774        let bank_id = 0;
13775        let scan_tracker = setup_scan(db.clone(), scan_ancestors.clone(), bank_id, account_key2);
13776
13777        // Add a new root 2
13778        let new_root = 2;
13779        db.get_accounts_delta_hash(new_root);
13780        db.add_root(new_root);
13781
13782        // Check that the scan is properly set up
13783        assert_eq!(
13784            db.accounts_index.min_ongoing_scan_root().unwrap(),
13785            max_scan_root
13786        );
13787
13788        // If we specify a requested_flush_root == 2, then `slot 2 <= max_flush_slot` will
13789        // be flushed even though `slot 2 > max_scan_root`. The unrooted slot 1 should
13790        // remain in the cache
13791        db.flush_accounts_cache(true, Some(new_root));
13792        assert_eq!(db.accounts_cache.num_slots(), 1);
13793        assert!(db.accounts_cache.slot_cache(1).is_some());
13794
13795        // Intra cache cleaning should not clean the entry for `account_key` from slot 0,
13796        // even though it was updated in slot `2` because of the ongoing scan
13797        let account = db
13798            .do_load(
13799                &Ancestors::default(),
13800                &account_key,
13801                Some(0),
13802                LoadHint::Unspecified,
13803            )
13804            .unwrap();
13805        assert_eq!(account.0.lamports(), zero_lamport_account.lamports());
13806
13807        // Run clean, unrooted slot 1 should not be purged, and still readable from the cache,
13808        // because we're still doing a scan on it.
13809        db.clean_accounts(None, false, None);
13810        let account = db
13811            .do_load(
13812                &scan_ancestors,
13813                &account_key,
13814                Some(max_scan_root),
13815                LoadHint::Unspecified,
13816            )
13817            .unwrap();
13818        assert_eq!(account.0.lamports(), slot1_account.lamports());
13819
13820        // When the scan is over, clean should not panic and should not purge something
13821        // still in the cache.
13822        scan_tracker.exit().unwrap();
13823        db.clean_accounts(None, false, None);
13824        let account = db
13825            .do_load(
13826                &scan_ancestors,
13827                &account_key,
13828                Some(max_scan_root),
13829                LoadHint::Unspecified,
13830            )
13831            .unwrap();
13832        assert_eq!(account.0.lamports(), slot1_account.lamports());
13833
13834        // Simulate dropping the bank, which finally removes the slot from the cache
13835        let bank_id = 1;
13836        db.purge_slot(1, bank_id, false);
13837        assert!(db
13838            .do_load(
13839                &scan_ancestors,
13840                &account_key,
13841                Some(max_scan_root),
13842                LoadHint::Unspecified
13843            )
13844            .is_none());
13845    }
13846
13847    #[test]
13848    fn test_alive_bytes() {
13849        let caching_enabled = true;
13850        let accounts_db = AccountsDb::new_with_config_for_tests(
13851            Vec::new(),
13852            &ClusterType::Development,
13853            AccountSecondaryIndexes::default(),
13854            caching_enabled,
13855            AccountShrinkThreshold::default(),
13856        );
13857        let slot: Slot = 0;
13858        let num_keys = 10;
13859
13860        for data_size in 0..num_keys {
13861            let account = AccountSharedData::new(1, data_size, &Pubkey::default());
13862            accounts_db.store_cached((slot, &[(&Pubkey::new_unique(), &account)][..]), None);
13863        }
13864
13865        accounts_db.add_root(slot);
13866        accounts_db.flush_accounts_cache(true, None);
13867
13868        let mut storage_maps: Vec<Arc<AccountStorageEntry>> = accounts_db
13869            .storage
13870            .get_slot_storage_entries(slot)
13871            .unwrap_or_default();
13872
13873        // Flushing cache should only create one storage entry
13874        assert_eq!(storage_maps.len(), 1);
13875        let storage0 = storage_maps.pop().unwrap();
13876        let accounts = storage0.all_accounts();
13877
13878        for account in accounts {
13879            let before_size = storage0.alive_bytes.load(Ordering::Acquire);
13880            let account_info = accounts_db
13881                .accounts_index
13882                .get_account_read_entry(&account.meta.pubkey)
13883                .map(|locked_entry| {
13884                    // Should only be one entry per key, since every key was only stored to slot 0
13885                    locked_entry.slot_list()[0]
13886                })
13887                .unwrap();
13888            let removed_data_size = account_info.1.stored_size();
13889            // Fetching the account from storage should return the same
13890            // stored size as in the index.
13891            assert_eq!(removed_data_size, account.stored_size as StoredSize);
13892            assert_eq!(account_info.0, slot);
13893            let reclaims = vec![account_info];
13894            accounts_db.remove_dead_accounts(reclaims.iter(), None, None, true);
13895            let after_size = storage0.alive_bytes.load(Ordering::Acquire);
13896            assert_eq!(before_size, after_size + account.stored_size);
13897        }
13898    }
13899
13900    fn setup_accounts_db_cache_clean(
13901        num_slots: usize,
13902        scan_slot: Option<Slot>,
13903        write_cache_limit_bytes: Option<u64>,
13904    ) -> (Arc<AccountsDb>, Vec<Pubkey>, Vec<Slot>, Option<ScanTracker>) {
13905        let caching_enabled = true;
13906        let mut accounts_db = AccountsDb::new_with_config_for_tests(
13907            Vec::new(),
13908            &ClusterType::Development,
13909            AccountSecondaryIndexes::default(),
13910            caching_enabled,
13911            AccountShrinkThreshold::default(),
13912        );
13913        accounts_db.write_cache_limit_bytes = write_cache_limit_bytes;
13914        let accounts_db = Arc::new(accounts_db);
13915
13916        let slots: Vec<_> = (0..num_slots as Slot).into_iter().collect();
13917        let stall_slot = num_slots as Slot;
13918        let scan_stall_key = Pubkey::new_unique();
13919        let keys: Vec<Pubkey> = std::iter::repeat_with(Pubkey::new_unique)
13920            .take(num_slots)
13921            .collect();
13922        if scan_slot.is_some() {
13923            accounts_db.store_cached(
13924                // Store it in a slot that isn't returned in `slots`
13925                (
13926                    stall_slot,
13927                    &[(
13928                        &scan_stall_key,
13929                        &AccountSharedData::new(1, 0, &Pubkey::default()),
13930                    )][..],
13931                ),
13932                None,
13933            );
13934        }
13935
13936        // Store some subset of the keys in slots 0..num_slots
13937        let mut scan_tracker = None;
13938        for slot in &slots {
13939            for key in &keys[*slot as usize..] {
13940                let space = 1; // 1 byte allows us to track by size
13941                accounts_db.store_cached(
13942                    (
13943                        *slot,
13944                        &[(key, &AccountSharedData::new(1, space, &Pubkey::default()))][..],
13945                    ),
13946                    None,
13947                );
13948            }
13949            accounts_db.add_root(*slot as Slot);
13950            if Some(*slot) == scan_slot {
13951                let ancestors = Arc::new(vec![(stall_slot, 1), (*slot, 1)].into_iter().collect());
13952                let bank_id = 0;
13953                scan_tracker = Some(setup_scan(
13954                    accounts_db.clone(),
13955                    ancestors,
13956                    bank_id,
13957                    scan_stall_key,
13958                ));
13959                assert_eq!(
13960                    accounts_db.accounts_index.min_ongoing_scan_root().unwrap(),
13961                    *slot
13962                );
13963            }
13964        }
13965
13966        accounts_db.accounts_cache.remove_slot(stall_slot);
13967
13968        // If there's <= max_cache_slots(), no slots should be flushed
13969        if accounts_db.accounts_cache.num_slots() <= max_cache_slots() {
13970            accounts_db.flush_accounts_cache(false, None);
13971            assert_eq!(accounts_db.accounts_cache.num_slots(), num_slots);
13972        }
13973
13974        (accounts_db, keys, slots, scan_tracker)
13975    }
13976
13977    #[test]
13978    fn test_accounts_db_cache_clean_dead_slots() {
13979        let num_slots = 10;
13980        let (accounts_db, keys, mut slots, _) =
13981            setup_accounts_db_cache_clean(num_slots, None, None);
13982        let last_dead_slot = (num_slots - 1) as Slot;
13983        assert_eq!(*slots.last().unwrap(), last_dead_slot);
13984        let alive_slot = last_dead_slot as Slot + 1;
13985        slots.push(alive_slot);
13986        for key in &keys {
13987            // Store a slot that overwrites all previous keys, rendering all previous keys dead
13988            accounts_db.store_cached(
13989                (
13990                    alive_slot,
13991                    &[(key, &AccountSharedData::new(1, 0, &Pubkey::default()))][..],
13992                ),
13993                None,
13994            );
13995            accounts_db.add_root(alive_slot);
13996        }
13997
13998        // Before the flush, we can find entries in the database for slots < alive_slot if we specify
13999        // a smaller max root
14000        for key in &keys {
14001            assert!(accounts_db
14002                .do_load(
14003                    &Ancestors::default(),
14004                    key,
14005                    Some(last_dead_slot),
14006                    LoadHint::Unspecified
14007                )
14008                .is_some());
14009        }
14010
14011        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
14012        accounts_db.flush_accounts_cache(true, None);
14013        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
14014        let mut uncleaned_roots = accounts_db
14015            .accounts_index
14016            .clear_uncleaned_roots(None)
14017            .into_iter()
14018            .collect::<Vec<_>>();
14019        uncleaned_roots.sort_unstable();
14020        assert_eq!(uncleaned_roots, slots);
14021        assert_eq!(
14022            accounts_db.accounts_cache.fetch_max_flush_root(),
14023            alive_slot,
14024        );
14025
14026        // Specifying a max_root < alive_slot, should not return any more entries,
14027        // as those have been purged from the accounts index for the dead slots.
14028        for key in &keys {
14029            assert!(accounts_db
14030                .do_load(
14031                    &Ancestors::default(),
14032                    key,
14033                    Some(last_dead_slot),
14034                    LoadHint::Unspecified
14035                )
14036                .is_none());
14037        }
14038        // Each slot should only have one entry in the storage, since all other accounts were
14039        // cleaned due to later updates
14040        for slot in &slots {
14041            if let ScanStorageResult::Stored(slot_accounts) = accounts_db.scan_account_storage(
14042                *slot as Slot,
14043                |_| Some(0),
14044                |slot_accounts: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
14045                    slot_accounts.insert(*loaded_account.pubkey());
14046                },
14047            ) {
14048                if *slot == alive_slot {
14049                    assert_eq!(slot_accounts.len(), keys.len());
14050                } else {
14051                    assert!(slot_accounts.is_empty());
14052                }
14053            } else {
14054                panic!("Expected slot to be in storage, not cache");
14055            }
14056        }
14057    }
14058
14059    #[test]
14060    fn test_accounts_db_cache_clean() {
14061        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(10, None, None);
14062
14063        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
14064        accounts_db.flush_accounts_cache(true, None);
14065        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
14066        let mut uncleaned_roots = accounts_db
14067            .accounts_index
14068            .clear_uncleaned_roots(None)
14069            .into_iter()
14070            .collect::<Vec<_>>();
14071        uncleaned_roots.sort_unstable();
14072        assert_eq!(uncleaned_roots, slots);
14073        assert_eq!(
14074            accounts_db.accounts_cache.fetch_max_flush_root(),
14075            *slots.last().unwrap()
14076        );
14077
14078        // Each slot should only have one entry in the storage, since all other accounts were
14079        // cleaned due to later updates
14080        for slot in &slots {
14081            if let ScanStorageResult::Stored(slot_account) = accounts_db.scan_account_storage(
14082                *slot as Slot,
14083                |_| Some(0),
14084                |slot_account: &Arc<RwLock<Pubkey>>, loaded_account: LoadedAccount| {
14085                    *slot_account.write().unwrap() = *loaded_account.pubkey();
14086                },
14087            ) {
14088                assert_eq!(*slot_account.read().unwrap(), keys[*slot as usize]);
14089            } else {
14090                panic!("Everything should have been flushed")
14091            }
14092        }
14093    }
14094
14095    fn run_test_accounts_db_cache_clean_max_root(
14096        num_slots: usize,
14097        requested_flush_root: Slot,
14098        scan_root: Option<Slot>,
14099    ) {
14100        assert!(requested_flush_root < (num_slots as Slot));
14101        let (accounts_db, keys, slots, scan_tracker) =
14102            setup_accounts_db_cache_clean(num_slots, scan_root, Some(max_cache_slots() as u64));
14103        let is_cache_at_limit = num_slots - requested_flush_root as usize - 1 > max_cache_slots();
14104
14105        // If:
14106        // 1) `requested_flush_root` is specified,
14107        // 2) not at the cache limit, i.e. `is_cache_at_limit == false`, then
14108        // `flush_accounts_cache()` should clean and flush only slots <= requested_flush_root,
14109        accounts_db.flush_accounts_cache(true, Some(requested_flush_root));
14110
14111        if !is_cache_at_limit {
14112            // Should flush all slots between 0..=requested_flush_root
14113            assert_eq!(
14114                accounts_db.accounts_cache.num_slots(),
14115                slots.len() - requested_flush_root as usize - 1
14116            );
14117        } else {
14118            // Otherwise, if we are at the cache limit, all roots will be flushed
14119            assert_eq!(accounts_db.accounts_cache.num_slots(), 0,);
14120        }
14121
14122        let mut uncleaned_roots = accounts_db
14123            .accounts_index
14124            .clear_uncleaned_roots(None)
14125            .into_iter()
14126            .collect::<Vec<_>>();
14127        uncleaned_roots.sort_unstable();
14128
14129        let expected_max_flushed_root = if !is_cache_at_limit {
14130            // Should flush all slots between 0..=requested_flush_root
14131            requested_flush_root
14132        } else {
14133            // Otherwise, if we are at the cache limit, all roots will be flushed
14134            num_slots as Slot - 1
14135        };
14136
14137        assert_eq!(
14138            uncleaned_roots,
14139            slots[0..=expected_max_flushed_root as usize].to_vec()
14140        );
14141        assert_eq!(
14142            accounts_db.accounts_cache.fetch_max_flush_root(),
14143            expected_max_flushed_root,
14144        );
14145
14146        for slot in &slots {
14147            let slot_accounts = accounts_db.scan_account_storage(
14148                *slot as Slot,
14149                |loaded_account: LoadedAccount| {
14150                    assert!(
14151                        !is_cache_at_limit,
14152                        "When cache is at limit, all roots should have been flushed to storage"
14153                    );
14154                    // All slots <= requested_flush_root should have been flushed, regardless
14155                    // of ongoing scans
14156                    assert!(*slot > requested_flush_root);
14157                    Some(*loaded_account.pubkey())
14158                },
14159                |slot_accounts: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
14160                    slot_accounts.insert(*loaded_account.pubkey());
14161                    if !is_cache_at_limit {
14162                        // Only true when the limit hasn't been reached and there are still
14163                        // slots left in the cache
14164                        assert!(*slot <= requested_flush_root);
14165                    }
14166                },
14167            );
14168
14169            let slot_accounts = match slot_accounts {
14170                ScanStorageResult::Cached(slot_accounts) => {
14171                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
14172                }
14173                ScanStorageResult::Stored(slot_accounts) => {
14174                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
14175                }
14176            };
14177
14178            let expected_accounts =
14179                if *slot >= requested_flush_root || *slot >= scan_root.unwrap_or(Slot::MAX) {
14180                    // 1) If slot > `requested_flush_root`, then  either:
14181                    //   a) If `is_cache_at_limit == false`, still in the cache
14182                    //   b) if `is_cache_at_limit == true`, were not cleaned before being flushed to storage.
14183                    //
14184                    // In both cases all the *original* updates at index `slot` were uncleaned and thus
14185                    // should be discoverable by this scan.
14186                    //
14187                    // 2) If slot == `requested_flush_root`, the slot was not cleaned before being flushed to storage,
14188                    // so it also contains all the original updates.
14189                    //
14190                    // 3) If *slot >= scan_root, then we should not clean it either
14191                    keys[*slot as usize..]
14192                        .iter()
14193                        .cloned()
14194                        .collect::<HashSet<Pubkey>>()
14195                } else {
14196                    // Slots less than `requested_flush_root` and `scan_root` were cleaned in the cache before being flushed
14197                    // to storage, should only contain one account
14198                    std::iter::once(keys[*slot as usize])
14199                        .into_iter()
14200                        .collect::<HashSet<Pubkey>>()
14201                };
14202
14203            assert_eq!(slot_accounts, expected_accounts);
14204        }
14205
14206        if let Some(scan_tracker) = scan_tracker {
14207            scan_tracker.exit().unwrap();
14208        }
14209    }
14210
14211    #[test]
14212    fn test_accounts_db_cache_clean_max_root() {
14213        let requested_flush_root = 5;
14214        run_test_accounts_db_cache_clean_max_root(10, requested_flush_root, None);
14215    }
14216
14217    #[test]
14218    fn test_accounts_db_cache_clean_max_root_with_scan() {
14219        let requested_flush_root = 5;
14220        run_test_accounts_db_cache_clean_max_root(
14221            10,
14222            requested_flush_root,
14223            Some(requested_flush_root - 1),
14224        );
14225        run_test_accounts_db_cache_clean_max_root(
14226            10,
14227            requested_flush_root,
14228            Some(requested_flush_root + 1),
14229        );
14230    }
14231
14232    #[test]
14233    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit() {
14234        let requested_flush_root = 5;
14235        // Test that if there are > max_cache_slots() in the cache after flush, then more roots
14236        // will be flushed
14237        run_test_accounts_db_cache_clean_max_root(
14238            max_cache_slots() + requested_flush_root as usize + 2,
14239            requested_flush_root,
14240            None,
14241        );
14242    }
14243
14244    #[test]
14245    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit_and_scan() {
14246        let requested_flush_root = 5;
14247        // Test that if there are > max_cache_slots() in the cache after flush, then more roots
14248        // will be flushed
14249        run_test_accounts_db_cache_clean_max_root(
14250            max_cache_slots() + requested_flush_root as usize + 2,
14251            requested_flush_root,
14252            Some(requested_flush_root - 1),
14253        );
14254        run_test_accounts_db_cache_clean_max_root(
14255            max_cache_slots() + requested_flush_root as usize + 2,
14256            requested_flush_root,
14257            Some(requested_flush_root + 1),
14258        );
14259    }
14260
14261    fn run_flush_rooted_accounts_cache(should_clean: bool) {
14262        let num_slots = 10;
14263        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(num_slots, None, None);
14264        let mut cleaned_bytes = 0;
14265        let mut cleaned_accounts = 0;
14266        let should_clean_tracker = if should_clean {
14267            Some((&mut cleaned_bytes, &mut cleaned_accounts))
14268        } else {
14269            None
14270        };
14271
14272        // If no cleaning is specified, then flush everything
14273        accounts_db.flush_rooted_accounts_cache(None, should_clean_tracker);
14274        for slot in &slots {
14275            let slot_accounts = if let ScanStorageResult::Stored(slot_accounts) = accounts_db
14276                .scan_account_storage(
14277                    *slot as Slot,
14278                    |_| Some(0),
14279                    |slot_account: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
14280                        slot_account.insert(*loaded_account.pubkey());
14281                    },
14282                ) {
14283                slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
14284            } else {
14285                panic!("All roots should have been flushed to storage");
14286            };
14287            let expected_accounts = if !should_clean || slot == slots.last().unwrap() {
14288                // The slot was not cleaned before being flushed to storage,
14289                // so it also contains all the original updates.
14290                keys[*slot as usize..]
14291                    .iter()
14292                    .cloned()
14293                    .collect::<HashSet<Pubkey>>()
14294            } else {
14295                // If clean was specified, only the latest slot should have all the updates.
14296                // All these other slots have been cleaned before flush
14297                std::iter::once(keys[*slot as usize])
14298                    .into_iter()
14299                    .collect::<HashSet<Pubkey>>()
14300            };
14301            assert_eq!(slot_accounts, expected_accounts);
14302        }
14303    }
14304
14305    #[test]
14306    fn test_flush_rooted_accounts_cache_with_clean() {
14307        run_flush_rooted_accounts_cache(true);
14308    }
14309
14310    #[test]
14311    fn test_flush_rooted_accounts_cache_without_clean() {
14312        run_flush_rooted_accounts_cache(false);
14313    }
14314
14315    fn run_test_shrink_unref(do_intra_cache_clean: bool) {
14316        // Enable caching so that we use the straightforward implementation
14317        // of shrink that will shrink all candidate slots
14318        let caching_enabled = true;
14319        let db = AccountsDb::new_with_config_for_tests(
14320            Vec::new(),
14321            &ClusterType::Development,
14322            AccountSecondaryIndexes::default(),
14323            caching_enabled,
14324            AccountShrinkThreshold::default(),
14325        );
14326        let account_key1 = Pubkey::new_unique();
14327        let account_key2 = Pubkey::new_unique();
14328        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14329
14330        // Store into slot 0
14331        db.store_cached((0, &[(&account_key1, &account1)][..]), None);
14332        db.store_cached((0, &[(&account_key2, &account1)][..]), None);
14333        db.add_root(0);
14334        if !do_intra_cache_clean {
14335            // If we don't want the cache doing purges before flush,
14336            // then we cannot flush multiple roots at once, otherwise the later
14337            // roots will clean the earlier roots before they are stored.
14338            // Thus flush the roots individually
14339            db.flush_accounts_cache(true, None);
14340
14341            // Add an additional ref within the same slot to pubkey 1
14342            db.store_uncached(0, &[(&account_key1, &account1)]);
14343        }
14344
14345        // Make account_key1 in slot 0 outdated by updating in rooted slot 1
14346        db.store_cached((1, &[(&account_key1, &account1)][..]), None);
14347        db.add_root(1);
14348        // Flushes all roots
14349        db.flush_accounts_cache(true, None);
14350        db.get_accounts_delta_hash(0);
14351        db.get_accounts_delta_hash(1);
14352
14353        // Clean to remove outdated entry from slot 0
14354        db.clean_accounts(Some(1), false, None);
14355
14356        // Shrink Slot 0
14357        let mut slot0_stores = db.storage.get_slot_storage_entries(0).unwrap();
14358        assert_eq!(slot0_stores.len(), 1);
14359        let slot0_store = slot0_stores.pop().unwrap();
14360        {
14361            let mut shrink_candidate_slots = db.shrink_candidate_slots.lock().unwrap();
14362            shrink_candidate_slots
14363                .entry(0)
14364                .or_default()
14365                .insert(slot0_store.append_vec_id(), slot0_store);
14366        }
14367        db.shrink_candidate_slots();
14368
14369        // Make slot 0 dead by updating the remaining key
14370        db.store_cached((2, &[(&account_key2, &account1)][..]), None);
14371        db.add_root(2);
14372
14373        // Flushes all roots
14374        db.flush_accounts_cache(true, None);
14375
14376        // Should be one store before clean for slot 0
14377        assert_eq!(db.storage.get_slot_storage_entries(0).unwrap().len(), 1);
14378        db.get_accounts_delta_hash(2);
14379        db.clean_accounts(Some(2), false, None);
14380
14381        // No stores should exist for slot 0 after clean
14382        assert!(db.storage.get_slot_storage_entries(0).is_none());
14383
14384        // Ref count for `account_key1` (account removed earlier by shrink)
14385        // should be 1, since it was only stored in slot 0 and 1, and slot 0
14386        // is now dead
14387        assert_eq!(db.accounts_index.ref_count_from_storage(&account_key1), 1);
14388    }
14389
14390    #[test]
14391    fn test_shrink_unref() {
14392        run_test_shrink_unref(false)
14393    }
14394
14395    #[test]
14396    fn test_shrink_unref_with_intra_slot_cleaning() {
14397        run_test_shrink_unref(true)
14398    }
14399
14400    #[test]
14401    fn test_partial_clean() {
14402        solana_logger::setup();
14403        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
14404        let account_key1 = Pubkey::new_unique();
14405        let account_key2 = Pubkey::new_unique();
14406        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14407        let account2 = AccountSharedData::new(2, 0, AccountSharedData::default().owner());
14408        let account3 = AccountSharedData::new(3, 0, AccountSharedData::default().owner());
14409        let account4 = AccountSharedData::new(4, 0, AccountSharedData::default().owner());
14410
14411        // Store accounts into slots 0 and 1
14412        db.store_uncached(0, &[(&account_key1, &account1)]);
14413        db.store_uncached(0, &[(&account_key2, &account1)]);
14414        db.store_uncached(1, &[(&account_key1, &account2)]);
14415        db.get_accounts_delta_hash(0);
14416        db.get_accounts_delta_hash(1);
14417
14418        db.print_accounts_stats("pre-clean1");
14419
14420        // clean accounts - no accounts should be cleaned, since no rooted slots
14421        //
14422        // Checking that the uncleaned_pubkeys are not pre-maturely removed
14423        // such that when the slots are rooted, and can actually be cleaned, then the
14424        // delta keys are still there.
14425        db.clean_accounts(None, false, None);
14426
14427        db.print_accounts_stats("post-clean1");
14428        // Check stores > 0
14429        assert!(!slot_stores(&db, 0).is_empty());
14430        assert!(!slot_stores(&db, 1).is_empty());
14431
14432        // root slot 0
14433        db.add_root(0);
14434
14435        // store into slot 2
14436        db.store_uncached(2, &[(&account_key2, &account3)]);
14437        db.store_uncached(2, &[(&account_key1, &account3)]);
14438        db.get_accounts_delta_hash(2);
14439
14440        db.clean_accounts(None, false, None);
14441        db.print_accounts_stats("post-clean2");
14442
14443        // root slots 1
14444        db.add_root(1);
14445        db.clean_accounts(None, false, None);
14446
14447        db.print_accounts_stats("post-clean3");
14448
14449        db.store_uncached(3, &[(&account_key2, &account4)]);
14450        db.get_accounts_delta_hash(3);
14451        db.add_root(3);
14452
14453        // Check that we can clean where max_root=3 and slot=2 is not rooted
14454        db.clean_accounts(None, false, None);
14455
14456        assert!(db.uncleaned_pubkeys.is_empty());
14457
14458        db.print_accounts_stats("post-clean4");
14459
14460        assert!(slot_stores(&db, 0).is_empty());
14461        assert!(!slot_stores(&db, 1).is_empty());
14462    }
14463
14464    #[test]
14465    fn test_recycle_stores_expiration() {
14466        solana_logger::setup();
14467
14468        let common_store_path = Path::new("");
14469        let common_slot_id = 12;
14470        let store_file_size = 1000;
14471
14472        let store1_id = 22;
14473        let entry1 = Arc::new(AccountStorageEntry::new(
14474            common_store_path,
14475            common_slot_id,
14476            store1_id,
14477            store_file_size,
14478        ));
14479
14480        let store2_id = 44;
14481        let entry2 = Arc::new(AccountStorageEntry::new(
14482            common_store_path,
14483            common_slot_id,
14484            store2_id,
14485            store_file_size,
14486        ));
14487
14488        let mut recycle_stores = RecycleStores::default();
14489        recycle_stores.add_entry(entry1);
14490        recycle_stores.add_entry(entry2);
14491        assert_eq!(recycle_stores.entry_count(), 2);
14492
14493        // no expiration for newly added entries
14494        let expired = recycle_stores.expire_old_entries();
14495        assert_eq!(
14496            expired
14497                .iter()
14498                .map(|e| e.append_vec_id())
14499                .collect::<Vec<_>>(),
14500            Vec::<AppendVecId>::new()
14501        );
14502        assert_eq!(
14503            recycle_stores
14504                .iter()
14505                .map(|(_, e)| e.append_vec_id())
14506                .collect::<Vec<_>>(),
14507            vec![store1_id, store2_id]
14508        );
14509        assert_eq!(recycle_stores.entry_count(), 2);
14510        assert_eq!(recycle_stores.total_bytes(), store_file_size * 2);
14511
14512        // expiration for only too old entries
14513        recycle_stores.entries[0].0 =
14514            Instant::now() - Duration::from_secs(EXPIRATION_TTL_SECONDS + 1);
14515        let expired = recycle_stores.expire_old_entries();
14516        assert_eq!(
14517            expired
14518                .iter()
14519                .map(|e| e.append_vec_id())
14520                .collect::<Vec<_>>(),
14521            vec![store1_id]
14522        );
14523        assert_eq!(
14524            recycle_stores
14525                .iter()
14526                .map(|(_, e)| e.append_vec_id())
14527                .collect::<Vec<_>>(),
14528            vec![store2_id]
14529        );
14530        assert_eq!(recycle_stores.entry_count(), 1);
14531        assert_eq!(recycle_stores.total_bytes(), store_file_size);
14532    }
14533
14534    const RACY_SLEEP_MS: u64 = 10;
14535    const RACE_TIME: u64 = 5;
14536
14537    fn start_load_thread(
14538        with_retry: bool,
14539        ancestors: Ancestors,
14540        db: Arc<AccountsDb>,
14541        exit: Arc<AtomicBool>,
14542        pubkey: Arc<Pubkey>,
14543        expected_lamports: impl Fn(&(AccountSharedData, Slot)) -> u64 + Send + 'static,
14544    ) -> JoinHandle<()> {
14545        let load_hint = if with_retry {
14546            LoadHint::FixedMaxRoot
14547        } else {
14548            LoadHint::Unspecified
14549        };
14550
14551        std::thread::Builder::new()
14552            .name("account-do-load".to_string())
14553            .spawn(move || {
14554                loop {
14555                    if exit.load(Ordering::Relaxed) {
14556                        return;
14557                    }
14558                    // Meddle load_limit to cover all branches of implementation.
14559                    // There should absolutely no behaviorial difference; the load_limit triggered
14560                    // slow branch should only affect the performance.
14561                    // Ordering::Relaxed is ok because of no data dependencies; the modified field is
14562                    // completely free-standing cfg(test) control-flow knob.
14563                    db.load_limit
14564                        .store(thread_rng().gen_range(0, 10) as u64, Ordering::Relaxed);
14565
14566                    // Load should never be unable to find this key
14567                    let loaded_account = db.do_load(&ancestors, &pubkey, None, load_hint).unwrap();
14568                    // slot + 1 == account.lamports because of the account-cache-flush thread
14569                    assert_eq!(
14570                        loaded_account.0.lamports(),
14571                        expected_lamports(&loaded_account)
14572                    );
14573                }
14574            })
14575            .unwrap()
14576    }
14577
14578    fn do_test_load_account_and_cache_flush_race(with_retry: bool) {
14579        solana_logger::setup();
14580
14581        let caching_enabled = true;
14582        let mut db = AccountsDb::new_with_config_for_tests(
14583            Vec::new(),
14584            &ClusterType::Development,
14585            AccountSecondaryIndexes::default(),
14586            caching_enabled,
14587            AccountShrinkThreshold::default(),
14588        );
14589        db.load_delay = RACY_SLEEP_MS;
14590        let db = Arc::new(db);
14591        let pubkey = Arc::new(Pubkey::new_unique());
14592        let exit = Arc::new(AtomicBool::new(false));
14593        db.store_cached(
14594            (
14595                0,
14596                &[(
14597                    pubkey.as_ref(),
14598                    &AccountSharedData::new(1, 0, AccountSharedData::default().owner()),
14599                )][..],
14600            ),
14601            None,
14602        );
14603        db.add_root(0);
14604        db.flush_accounts_cache(true, None);
14605
14606        let t_flush_accounts_cache = {
14607            let db = db.clone();
14608            let exit = exit.clone();
14609            let pubkey = pubkey.clone();
14610            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14611            std::thread::Builder::new()
14612                .name("account-cache-flush".to_string())
14613                .spawn(move || {
14614                    let mut slot: Slot = 1;
14615                    loop {
14616                        if exit.load(Ordering::Relaxed) {
14617                            return;
14618                        }
14619                        account.set_lamports(slot + 1);
14620                        db.store_cached((slot, &[(pubkey.as_ref(), &account)][..]), None);
14621                        db.add_root(slot);
14622                        sleep(Duration::from_millis(RACY_SLEEP_MS));
14623                        db.flush_accounts_cache(true, None);
14624                        slot += 1;
14625                    }
14626                })
14627                .unwrap()
14628        };
14629
14630        let t_do_load = start_load_thread(
14631            with_retry,
14632            Ancestors::default(),
14633            db,
14634            exit.clone(),
14635            pubkey,
14636            |(_, slot)| slot + 1,
14637        );
14638
14639        sleep(Duration::from_secs(RACE_TIME));
14640        exit.store(true, Ordering::Relaxed);
14641        t_flush_accounts_cache.join().unwrap();
14642        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
14643    }
14644
14645    #[test]
14646    fn test_load_account_and_cache_flush_race_with_retry() {
14647        do_test_load_account_and_cache_flush_race(true);
14648    }
14649
14650    #[test]
14651    fn test_load_account_and_cache_flush_race_without_retry() {
14652        do_test_load_account_and_cache_flush_race(false);
14653    }
14654
14655    fn do_test_load_account_and_shrink_race(with_retry: bool) {
14656        let caching_enabled = true;
14657        let mut db = AccountsDb::new_with_config_for_tests(
14658            Vec::new(),
14659            &ClusterType::Development,
14660            AccountSecondaryIndexes::default(),
14661            caching_enabled,
14662            AccountShrinkThreshold::default(),
14663        );
14664        db.load_delay = RACY_SLEEP_MS;
14665        let db = Arc::new(db);
14666        let pubkey = Arc::new(Pubkey::new_unique());
14667        let exit = Arc::new(AtomicBool::new(false));
14668        let slot = 1;
14669
14670        // Store an account
14671        let lamports = 42;
14672        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14673        account.set_lamports(lamports);
14674        db.store_uncached(slot, &[(&pubkey, &account)]);
14675
14676        // Set the slot as a root so account loads will see the contents of this slot
14677        db.add_root(slot);
14678
14679        let t_shrink_accounts = {
14680            let db = db.clone();
14681            let exit = exit.clone();
14682
14683            std::thread::Builder::new()
14684                .name("account-shrink".to_string())
14685                .spawn(move || loop {
14686                    if exit.load(Ordering::Relaxed) {
14687                        return;
14688                    }
14689                    // Simulate adding shrink candidates from clean_accounts()
14690                    let stores = db.storage.get_slot_storage_entries(slot).unwrap();
14691                    assert_eq!(stores.len(), 1);
14692                    let store = &stores[0];
14693                    let store_id = store.append_vec_id();
14694                    db.shrink_candidate_slots
14695                        .lock()
14696                        .unwrap()
14697                        .entry(slot)
14698                        .or_default()
14699                        .insert(store_id, store.clone());
14700                    db.shrink_candidate_slots();
14701                })
14702                .unwrap()
14703        };
14704
14705        let t_do_load = start_load_thread(
14706            with_retry,
14707            Ancestors::default(),
14708            db,
14709            exit.clone(),
14710            pubkey,
14711            move |_| lamports,
14712        );
14713
14714        sleep(Duration::from_secs(RACE_TIME));
14715        exit.store(true, Ordering::Relaxed);
14716        t_shrink_accounts.join().unwrap();
14717        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
14718    }
14719
14720    #[test]
14721    fn test_load_account_and_shrink_race_with_retry() {
14722        do_test_load_account_and_shrink_race(true);
14723    }
14724
14725    #[test]
14726    fn test_load_account_and_shrink_race_without_retry() {
14727        do_test_load_account_and_shrink_race(false);
14728    }
14729
14730    #[test]
14731    fn test_cache_flush_delayed_remove_unrooted_race() {
14732        let caching_enabled = true;
14733        let mut db = AccountsDb::new_with_config_for_tests(
14734            Vec::new(),
14735            &ClusterType::Development,
14736            AccountSecondaryIndexes::default(),
14737            caching_enabled,
14738            AccountShrinkThreshold::default(),
14739        );
14740        db.load_delay = RACY_SLEEP_MS;
14741        let db = Arc::new(db);
14742        let slot = 10;
14743        let bank_id = 10;
14744
14745        let lamports = 42;
14746        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14747        account.set_lamports(lamports);
14748
14749        // Start up a thread to flush the accounts cache
14750        let (flush_trial_start_sender, flush_trial_start_receiver) = unbounded();
14751        let (flush_done_sender, flush_done_receiver) = unbounded();
14752        let t_flush_cache = {
14753            let db = db.clone();
14754            std::thread::Builder::new()
14755                .name("account-cache-flush".to_string())
14756                .spawn(move || loop {
14757                    // Wait for the signal to start a trial
14758                    if flush_trial_start_receiver.recv().is_err() {
14759                        return;
14760                    }
14761                    db.flush_slot_cache(10);
14762                    flush_done_sender.send(()).unwrap();
14763                })
14764                .unwrap()
14765        };
14766
14767        // Start up a thread remove the slot
14768        let (remove_trial_start_sender, remove_trial_start_receiver) = unbounded();
14769        let (remove_done_sender, remove_done_receiver) = unbounded();
14770        let t_remove = {
14771            let db = db.clone();
14772            std::thread::Builder::new()
14773                .name("account-remove".to_string())
14774                .spawn(move || loop {
14775                    // Wait for the signal to start a trial
14776                    if remove_trial_start_receiver.recv().is_err() {
14777                        return;
14778                    }
14779                    db.remove_unrooted_slots(&[(slot, bank_id)]);
14780                    remove_done_sender.send(()).unwrap();
14781                })
14782                .unwrap()
14783        };
14784
14785        let num_trials = 10;
14786        for _ in 0..num_trials {
14787            let pubkey = Pubkey::new_unique();
14788            db.store_cached((slot, &[(&pubkey, &account)][..]), None);
14789            // Wait for both threads to finish
14790            flush_trial_start_sender.send(()).unwrap();
14791            remove_trial_start_sender.send(()).unwrap();
14792            let _ = flush_done_receiver.recv();
14793            let _ = remove_done_receiver.recv();
14794        }
14795
14796        drop(flush_trial_start_sender);
14797        drop(remove_trial_start_sender);
14798        t_flush_cache.join().unwrap();
14799        t_remove.join().unwrap();
14800    }
14801
14802    #[test]
14803    fn test_cache_flush_remove_unrooted_race_multiple_slots() {
14804        let caching_enabled = true;
14805        let db = AccountsDb::new_with_config_for_tests(
14806            Vec::new(),
14807            &ClusterType::Development,
14808            AccountSecondaryIndexes::default(),
14809            caching_enabled,
14810            AccountShrinkThreshold::default(),
14811        );
14812        let db = Arc::new(db);
14813        let num_cached_slots = 100;
14814
14815        let num_trials = 100;
14816        let (new_trial_start_sender, new_trial_start_receiver) = unbounded();
14817        let (flush_done_sender, flush_done_receiver) = unbounded();
14818        // Start up a thread to flush the accounts cache
14819        let t_flush_cache = {
14820            let db = db.clone();
14821
14822            std::thread::Builder::new()
14823                .name("account-cache-flush".to_string())
14824                .spawn(move || loop {
14825                    // Wait for the signal to start a trial
14826                    if new_trial_start_receiver.recv().is_err() {
14827                        return;
14828                    }
14829                    for slot in 0..num_cached_slots {
14830                        db.flush_slot_cache(slot);
14831                    }
14832                    flush_done_sender.send(()).unwrap();
14833                })
14834                .unwrap()
14835        };
14836
14837        let exit = Arc::new(AtomicBool::new(false));
14838
14839        let t_spurious_signal = {
14840            let db = db.clone();
14841            let exit = exit.clone();
14842            std::thread::Builder::new()
14843                .name("account-cache-flush".to_string())
14844                .spawn(move || loop {
14845                    if exit.load(Ordering::Relaxed) {
14846                        return;
14847                    }
14848                    // Simulate spurious wake-up that can happen, but is too rare to
14849                    // otherwise depend on in tests.
14850                    db.remove_unrooted_slots_synchronization.signal.notify_all();
14851                })
14852                .unwrap()
14853        };
14854
14855        // Run multiple trials. Has the added benefit of rewriting the same slots after we've
14856        // dumped them in previous trials.
14857        for _ in 0..num_trials {
14858            // Store an account
14859            let lamports = 42;
14860            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
14861            account.set_lamports(lamports);
14862
14863            // Pick random 50% of the slots to pass to `remove_unrooted_slots()`
14864            let mut all_slots: Vec<(Slot, BankId)> = (0..num_cached_slots)
14865                .map(|slot| {
14866                    let bank_id = slot + 1;
14867                    (slot, bank_id)
14868                })
14869                .collect();
14870            all_slots.shuffle(&mut rand::thread_rng());
14871            let slots_to_dump = &all_slots[0..num_cached_slots as usize / 2];
14872            let slots_to_keep = &all_slots[num_cached_slots as usize / 2..];
14873
14874            // Set up a one account per slot across many different slots, track which
14875            // pubkey was stored in each slot.
14876            let slot_to_pubkey_map: HashMap<Slot, Pubkey> = (0..num_cached_slots)
14877                .map(|slot| {
14878                    let pubkey = Pubkey::new_unique();
14879                    db.store_cached((slot, &[(&pubkey, &account)][..]), None);
14880                    (slot, pubkey)
14881                })
14882                .collect();
14883
14884            // Signal the flushing shred to start flushing
14885            new_trial_start_sender.send(()).unwrap();
14886
14887            // Here we want to test both:
14888            // 1) Flush thread starts flushing a slot before we try dumping it.
14889            // 2) Flushing thread trying to flush while/after we're trying to dump the slot,
14890            // in which case flush should ignore/move past the slot to be dumped
14891            //
14892            // Hence, we split into chunks to get the dumping of each chunk to race with the
14893            // flushes. If we were to dump the entire chunk at once, then this reduces the possibility
14894            // of the flush occurring first since the dumping logic reserves all the slots it's about
14895            // to dump immediately.
14896
14897            for chunks in slots_to_dump.chunks(slots_to_dump.len() / 2) {
14898                db.remove_unrooted_slots(chunks);
14899            }
14900
14901            // Check that all the slots in `slots_to_dump` were completely removed from the
14902            // cache, storage, and index
14903
14904            for (slot, _) in slots_to_dump {
14905                assert!(db.storage.get_slot_storage_entries(*slot).is_none());
14906                assert!(db.accounts_cache.slot_cache(*slot).is_none());
14907                let account_in_slot = slot_to_pubkey_map[slot];
14908                let item = db.accounts_index.get_account_read_entry(&account_in_slot);
14909                assert!(item.is_none(), "item: {:?}", item);
14910            }
14911
14912            // Wait for flush to finish before starting next trial
14913
14914            flush_done_receiver.recv().unwrap();
14915
14916            for (slot, bank_id) in slots_to_keep {
14917                let account_in_slot = slot_to_pubkey_map[slot];
14918                assert!(db
14919                    .load(
14920                        &Ancestors::from(vec![(*slot, 0)]),
14921                        &account_in_slot,
14922                        LoadHint::FixedMaxRoot
14923                    )
14924                    .is_some());
14925                // Clear for next iteration so that `assert!(self.storage.get_slot_stores(purged_slot).is_none());`
14926                // in `purge_slot_pubkeys()` doesn't trigger
14927                db.remove_unrooted_slots(&[(*slot, *bank_id)]);
14928            }
14929        }
14930
14931        exit.store(true, Ordering::Relaxed);
14932        drop(new_trial_start_sender);
14933        t_flush_cache.join().unwrap();
14934
14935        t_spurious_signal.join().unwrap();
14936    }
14937
14938    #[test]
14939    fn test_collect_uncleaned_slots_up_to_slot() {
14940        solana_logger::setup();
14941        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
14942
14943        let slot1 = 11;
14944        let slot2 = 222;
14945        let slot3 = 3333;
14946
14947        let pubkey1 = Pubkey::new_unique();
14948        let pubkey2 = Pubkey::new_unique();
14949        let pubkey3 = Pubkey::new_unique();
14950
14951        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
14952        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
14953        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
14954
14955        let mut uncleaned_slots1 = db.collect_uncleaned_slots_up_to_slot(slot1);
14956        let mut uncleaned_slots2 = db.collect_uncleaned_slots_up_to_slot(slot2);
14957        let mut uncleaned_slots3 = db.collect_uncleaned_slots_up_to_slot(slot3);
14958
14959        uncleaned_slots1.sort_unstable();
14960        uncleaned_slots2.sort_unstable();
14961        uncleaned_slots3.sort_unstable();
14962
14963        assert_eq!(uncleaned_slots1, [slot1]);
14964        assert_eq!(uncleaned_slots2, [slot1, slot2]);
14965        assert_eq!(uncleaned_slots3, [slot1, slot2, slot3]);
14966    }
14967
14968    #[test]
14969    fn test_remove_uncleaned_slots_and_collect_pubkeys() {
14970        solana_logger::setup();
14971        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
14972
14973        let slot1 = 11;
14974        let slot2 = 222;
14975        let slot3 = 3333;
14976
14977        let pubkey1 = Pubkey::new_unique();
14978        let pubkey2 = Pubkey::new_unique();
14979        let pubkey3 = Pubkey::new_unique();
14980
14981        let account1 = AccountSharedData::new(0, 0, &pubkey1);
14982        let account2 = AccountSharedData::new(0, 0, &pubkey2);
14983        let account3 = AccountSharedData::new(0, 0, &pubkey3);
14984
14985        db.store_uncached(slot1, &[(&pubkey1, &account1)]);
14986        db.store_uncached(slot2, &[(&pubkey2, &account2)]);
14987        db.store_uncached(slot3, &[(&pubkey3, &account3)]);
14988
14989        db.add_root(slot1);
14990        // slot 2 is _not_ a root on purpose
14991        db.add_root(slot3);
14992
14993        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
14994        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
14995        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
14996
14997        let uncleaned_pubkeys1 = db
14998            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot1])
14999            .into_iter()
15000            .flatten()
15001            .collect::<Vec<_>>();
15002        let uncleaned_pubkeys2 = db
15003            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot2])
15004            .into_iter()
15005            .flatten()
15006            .collect::<Vec<_>>();
15007        let uncleaned_pubkeys3 = db
15008            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot3])
15009            .into_iter()
15010            .flatten()
15011            .collect::<Vec<_>>();
15012
15013        assert!(uncleaned_pubkeys1.contains(&pubkey1));
15014        assert!(!uncleaned_pubkeys1.contains(&pubkey2));
15015        assert!(!uncleaned_pubkeys1.contains(&pubkey3));
15016
15017        assert!(!uncleaned_pubkeys2.contains(&pubkey1));
15018        assert!(uncleaned_pubkeys2.contains(&pubkey2));
15019        assert!(!uncleaned_pubkeys2.contains(&pubkey3));
15020
15021        assert!(!uncleaned_pubkeys3.contains(&pubkey1));
15022        assert!(!uncleaned_pubkeys3.contains(&pubkey2));
15023        assert!(uncleaned_pubkeys3.contains(&pubkey3));
15024    }
15025
15026    #[test]
15027    fn test_remove_uncleaned_slots_and_collect_pubkeys_up_to_slot() {
15028        solana_logger::setup();
15029        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
15030
15031        let slot1 = 11;
15032        let slot2 = 222;
15033        let slot3 = 3333;
15034
15035        let pubkey1 = Pubkey::new_unique();
15036        let pubkey2 = Pubkey::new_unique();
15037        let pubkey3 = Pubkey::new_unique();
15038
15039        let account1 = AccountSharedData::new(0, 0, &pubkey1);
15040        let account2 = AccountSharedData::new(0, 0, &pubkey2);
15041        let account3 = AccountSharedData::new(0, 0, &pubkey3);
15042
15043        db.store_uncached(slot1, &[(&pubkey1, &account1)]);
15044        db.store_uncached(slot2, &[(&pubkey2, &account2)]);
15045        db.store_uncached(slot3, &[(&pubkey3, &account3)]);
15046
15047        // slot 1 is _not_ a root on purpose
15048        db.add_root(slot2);
15049        db.add_root(slot3);
15050
15051        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
15052        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
15053        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
15054
15055        let uncleaned_pubkeys = db
15056            .remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(slot3)
15057            .into_iter()
15058            .flatten()
15059            .collect::<Vec<_>>();
15060
15061        assert!(uncleaned_pubkeys.contains(&pubkey1));
15062        assert!(uncleaned_pubkeys.contains(&pubkey2));
15063        assert!(uncleaned_pubkeys.contains(&pubkey3));
15064    }
15065
15066    #[test]
15067    fn test_shrink_productive() {
15068        solana_logger::setup();
15069        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, 1024);
15070        let stores = vec![Arc::new(s1)];
15071        assert!(!AccountsDb::is_shrinking_productive(0, &stores));
15072
15073        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, PAGE_SIZE * 4);
15074        let stores = vec![Arc::new(s1)];
15075        stores[0].add_account((3 * PAGE_SIZE as usize) - 1);
15076        stores[0].add_account(10);
15077        stores[0].remove_account(10, false);
15078        assert!(AccountsDb::is_shrinking_productive(0, &stores));
15079
15080        stores[0].add_account(PAGE_SIZE as usize);
15081        assert!(!AccountsDb::is_shrinking_productive(0, &stores));
15082
15083        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, PAGE_SIZE + 1);
15084        s1.add_account(PAGE_SIZE as usize);
15085        let s2 = AccountStorageEntry::new(Path::new("."), 0, 1, PAGE_SIZE + 1);
15086        s2.add_account(PAGE_SIZE as usize);
15087        let stores = vec![Arc::new(s1), Arc::new(s2)];
15088        assert!(AccountsDb::is_shrinking_productive(0, &stores));
15089    }
15090
15091    #[test]
15092    fn test_is_candidate_for_shrink() {
15093        solana_logger::setup();
15094
15095        let mut accounts = AccountsDb::new_single_for_tests();
15096        let common_store_path = Path::new("");
15097        let store_file_size = 2 * PAGE_SIZE;
15098        let entry = Arc::new(AccountStorageEntry::new(
15099            common_store_path,
15100            0,
15101            1,
15102            store_file_size,
15103        ));
15104        match accounts.shrink_ratio {
15105            AccountShrinkThreshold::TotalSpace { shrink_ratio } => {
15106                assert_eq!(
15107                    (DEFAULT_ACCOUNTS_SHRINK_RATIO * 100.) as u64,
15108                    (shrink_ratio * 100.) as u64
15109                )
15110            }
15111            AccountShrinkThreshold::IndividualStore { shrink_ratio: _ } => {
15112                panic!("Expect the default to be TotalSpace")
15113            }
15114        }
15115        entry.alive_bytes.store(3000, Ordering::Release);
15116        assert!(accounts.is_candidate_for_shrink(&entry, false));
15117        entry.alive_bytes.store(5000, Ordering::Release);
15118        assert!(!accounts.is_candidate_for_shrink(&entry, false));
15119        accounts.shrink_ratio = AccountShrinkThreshold::TotalSpace { shrink_ratio: 0.3 };
15120        entry.alive_bytes.store(3000, Ordering::Release);
15121        assert!(accounts.is_candidate_for_shrink(&entry, false));
15122        accounts.shrink_ratio = AccountShrinkThreshold::IndividualStore { shrink_ratio: 0.3 };
15123        assert!(!accounts.is_candidate_for_shrink(&entry, false));
15124    }
15125
15126    #[test]
15127    fn test_calculate_storage_count_and_alive_bytes() {
15128        let accounts = AccountsDb::new_single_for_tests();
15129        let shared_key = solana_sdk::pubkey::new_rand();
15130        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15131        let slot0 = 0;
15132        accounts.store_uncached(slot0, &[(&shared_key, &account)]);
15133
15134        let storage_maps = accounts
15135            .storage
15136            .get_slot_storage_entries(slot0)
15137            .unwrap_or_default();
15138        let storage_info = StorageSizeAndCountMap::default();
15139        let accounts_map = accounts.process_storage_slot(&storage_maps[..]);
15140        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
15141        assert_eq!(storage_info.len(), 1);
15142        for entry in storage_info.iter() {
15143            assert_eq!(
15144                (entry.key(), entry.value().count, entry.value().stored_size),
15145                (&0, 1, 144)
15146            );
15147        }
15148    }
15149
15150    #[test]
15151    fn test_calculate_storage_count_and_alive_bytes_0_accounts() {
15152        let accounts = AccountsDb::new_single_for_tests();
15153        let storage_maps = vec![];
15154        let storage_info = StorageSizeAndCountMap::default();
15155        let accounts_map = accounts.process_storage_slot(&storage_maps[..]);
15156        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
15157        assert!(storage_info.is_empty());
15158    }
15159
15160    #[test]
15161    fn test_calculate_storage_count_and_alive_bytes_2_accounts() {
15162        let accounts = AccountsDb::new_single_for_tests();
15163        let keys = [
15164            solana_sdk::pubkey::Pubkey::from([0; 32]),
15165            solana_sdk::pubkey::Pubkey::from([255; 32]),
15166        ];
15167        // make sure accounts are in 2 different bins
15168        assert!(
15169            (accounts.accounts_index.bins() == 1)
15170                ^ (accounts
15171                    .accounts_index
15172                    .bin_calculator
15173                    .bin_from_pubkey(&keys[0])
15174                    != accounts
15175                        .accounts_index
15176                        .bin_calculator
15177                        .bin_from_pubkey(&keys[1]))
15178        );
15179        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15180        let account_big = AccountSharedData::new(1, 1000, AccountSharedData::default().owner());
15181        let slot0 = 0;
15182        accounts.store_uncached(slot0, &[(&keys[0], &account)]);
15183        accounts.store_uncached(slot0, &[(&keys[1], &account_big)]);
15184
15185        let storage_maps = accounts
15186            .storage
15187            .get_slot_storage_entries(slot0)
15188            .unwrap_or_default();
15189        let storage_info = StorageSizeAndCountMap::default();
15190        let accounts_map = accounts.process_storage_slot(&storage_maps[..]);
15191        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
15192        assert_eq!(storage_info.len(), 1);
15193        for entry in storage_info.iter() {
15194            assert_eq!(
15195                (entry.key(), entry.value().count, entry.value().stored_size),
15196                (&0, 2, 1280)
15197            );
15198        }
15199    }
15200
15201    #[test]
15202    fn test_set_storage_count_and_alive_bytes() {
15203        let accounts = AccountsDb::new_single_for_tests();
15204
15205        // make sure we have storage 0
15206        let shared_key = solana_sdk::pubkey::new_rand();
15207        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15208        let slot0 = 0;
15209        accounts.store_uncached(slot0, &[(&shared_key, &account)]);
15210
15211        // fake out the store count to avoid the assert
15212        for slot_stores in accounts.storage.map.iter() {
15213            for (_id, store) in slot_stores.value().read().unwrap().iter() {
15214                store.alive_bytes.store(0, Ordering::Release);
15215            }
15216        }
15217
15218        // populate based on made up hash data
15219        let dashmap = DashMap::default();
15220        dashmap.insert(
15221            0,
15222            StorageSizeAndCount {
15223                stored_size: 2,
15224                count: 3,
15225            },
15226        );
15227        accounts.set_storage_count_and_alive_bytes(dashmap, &mut GenerateIndexTimings::default());
15228        assert_eq!(accounts.storage.map.len(), 1);
15229        for slot_stores in accounts.storage.map.iter() {
15230            for (id, store) in slot_stores.value().read().unwrap().iter() {
15231                assert_eq!(id, &0);
15232                assert_eq!(store.count_and_status.read().unwrap().0, 3);
15233                assert_eq!(store.alive_bytes.load(Ordering::Acquire), 2);
15234            }
15235        }
15236    }
15237
15238    #[test]
15239    fn test_purge_alive_unrooted_slots_after_clean() {
15240        let accounts = AccountsDb::new_single_for_tests();
15241
15242        // Key shared between rooted and nonrooted slot
15243        let shared_key = solana_sdk::pubkey::new_rand();
15244        // Key to keep the storage entry for the unrooted slot alive
15245        let unrooted_key = solana_sdk::pubkey::new_rand();
15246        let slot0 = 0;
15247        let slot1 = 1;
15248
15249        // Store accounts with greater than 0 lamports
15250        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
15251        accounts.store_uncached(slot0, &[(&shared_key, &account)]);
15252        accounts.store_uncached(slot0, &[(&unrooted_key, &account)]);
15253
15254        // Simulate adding dirty pubkeys on bank freeze. Note this is
15255        // not a rooted slot
15256        accounts.get_accounts_delta_hash(slot0);
15257
15258        // On the next *rooted* slot, update the `shared_key` account to zero lamports
15259        let zero_lamport_account =
15260            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
15261        accounts.store_uncached(slot1, &[(&shared_key, &zero_lamport_account)]);
15262
15263        // Simulate adding dirty pubkeys on bank freeze, set root
15264        accounts.get_accounts_delta_hash(slot1);
15265        accounts.add_root(slot1);
15266
15267        // The later rooted zero-lamport update to `shared_key` cannot be cleaned
15268        // because it is kept alive by the unrooted slot.
15269        accounts.clean_accounts(None, false, None);
15270        assert!(accounts
15271            .accounts_index
15272            .get_account_read_entry(&shared_key)
15273            .is_some());
15274
15275        // Simulate purge_slot() all from AccountsBackgroundService
15276        accounts.purge_slot(slot0, 0, true);
15277
15278        // Now clean should clean up the remaining key
15279        accounts.clean_accounts(None, false, None);
15280        assert!(accounts
15281            .accounts_index
15282            .get_account_read_entry(&shared_key)
15283            .is_none());
15284        assert!(accounts.storage.get_slot_storage_entries(slot0).is_none());
15285    }
15286
15287    /// Test to make sure `clean_accounts()` works properly with the `last_full_snapshot_slot`
15288    /// parameter.  Basically:
15289    ///
15290    /// - slot 1: set Account1's balance to non-zero
15291    /// - slot 2: set Account1's balance to a different non-zero amount
15292    /// - slot 3: set Account1's balance to zero
15293    /// - call `clean_accounts()` with `max_clean_root` set to 2
15294    ///     - ensure Account1 has *not* been purged
15295    ///     - ensure the store from slot 1 is cleaned up
15296    /// - call `clean_accounts()` with `last_full_snapshot_slot` set to 2
15297    ///     - ensure Account1 has *not* been purged
15298    /// - call `clean_accounts()` with `last_full_snapshot_slot` set to 3
15299    ///     - ensure Account1 *has* been purged
15300    #[test]
15301    fn test_clean_accounts_with_last_full_snapshot_slot() {
15302        solana_logger::setup();
15303        let accounts_db = AccountsDb::new_single_for_tests();
15304        let pubkey = solana_sdk::pubkey::new_rand();
15305        let owner = solana_sdk::pubkey::new_rand();
15306        let space = 0;
15307
15308        let slot1: Slot = 1;
15309        let account = AccountSharedData::new(111, space, &owner);
15310        accounts_db.store_cached((slot1, &[(&pubkey, &account)][..]), None);
15311        accounts_db.get_accounts_delta_hash(slot1);
15312        accounts_db.add_root(slot1);
15313
15314        let slot2: Slot = 2;
15315        let account = AccountSharedData::new(222, space, &owner);
15316        accounts_db.store_cached((slot2, &[(&pubkey, &account)][..]), None);
15317        accounts_db.get_accounts_delta_hash(slot2);
15318        accounts_db.add_root(slot2);
15319
15320        let slot3: Slot = 3;
15321        let account = AccountSharedData::new(0, space, &owner);
15322        accounts_db.store_cached((slot3, &[(&pubkey, &account)][..]), None);
15323        accounts_db.get_accounts_delta_hash(slot3);
15324        accounts_db.add_root(slot3);
15325
15326        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 3);
15327
15328        accounts_db.clean_accounts(Some(slot2), false, Some(slot2));
15329        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 2);
15330
15331        accounts_db.clean_accounts(None, false, Some(slot2));
15332        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 1);
15333
15334        accounts_db.clean_accounts(None, false, Some(slot3));
15335        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 0);
15336    }
15337
15338    #[test]
15339    fn test_filter_zero_lamport_clean_for_incremental_snapshots() {
15340        solana_logger::setup();
15341        let slot = 10;
15342
15343        struct TestParameters {
15344            last_full_snapshot_slot: Option<Slot>,
15345            max_clean_root: Option<Slot>,
15346            should_contain: bool,
15347        }
15348
15349        let do_test = |test_params: TestParameters| {
15350            let account_info = AccountInfo::new(StorageLocation::AppendVec(42, 128), 234, 0);
15351            let pubkey = solana_sdk::pubkey::new_rand();
15352            let mut key_set = HashSet::default();
15353            key_set.insert(pubkey);
15354            let store_count = 0;
15355            let mut store_counts = HashMap::default();
15356            store_counts.insert(account_info.store_id(), (store_count, key_set));
15357            let mut purges_zero_lamports = HashMap::default();
15358            purges_zero_lamports.insert(pubkey, (vec![(slot, account_info)], 1));
15359
15360            let accounts_db = AccountsDb::new_single_for_tests();
15361            accounts_db.filter_zero_lamport_clean_for_incremental_snapshots(
15362                test_params.max_clean_root,
15363                test_params.last_full_snapshot_slot,
15364                &store_counts,
15365                &mut purges_zero_lamports,
15366            );
15367
15368            assert_eq!(
15369                purges_zero_lamports.contains_key(&pubkey),
15370                test_params.should_contain
15371            );
15372        };
15373
15374        // Scenario 1: last full snapshot is NONE
15375        // In this scenario incremental snapshots are OFF, so always purge
15376        {
15377            let last_full_snapshot_slot = None;
15378
15379            do_test(TestParameters {
15380                last_full_snapshot_slot,
15381                max_clean_root: Some(slot),
15382                should_contain: true,
15383            });
15384
15385            do_test(TestParameters {
15386                last_full_snapshot_slot,
15387                max_clean_root: None,
15388                should_contain: true,
15389            });
15390        }
15391
15392        // Scenario 2: last full snapshot is GREATER THAN zero lamport account slot
15393        // In this scenario always purge, and just test the various permutations of
15394        // `should_filter_for_incremental_snapshots` based on `max_clean_root`.
15395        {
15396            let last_full_snapshot_slot = Some(slot + 1);
15397
15398            do_test(TestParameters {
15399                last_full_snapshot_slot,
15400                max_clean_root: last_full_snapshot_slot,
15401                should_contain: true,
15402            });
15403
15404            do_test(TestParameters {
15405                last_full_snapshot_slot,
15406                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
15407                should_contain: true,
15408            });
15409
15410            do_test(TestParameters {
15411                last_full_snapshot_slot,
15412                max_clean_root: None,
15413                should_contain: true,
15414            });
15415        }
15416
15417        // Scenario 3: last full snapshot is EQUAL TO zero lamport account slot
15418        // In this scenario always purge, as it's the same as Scenario 2.
15419        {
15420            let last_full_snapshot_slot = Some(slot);
15421
15422            do_test(TestParameters {
15423                last_full_snapshot_slot,
15424                max_clean_root: last_full_snapshot_slot,
15425                should_contain: true,
15426            });
15427
15428            do_test(TestParameters {
15429                last_full_snapshot_slot,
15430                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
15431                should_contain: true,
15432            });
15433
15434            do_test(TestParameters {
15435                last_full_snapshot_slot,
15436                max_clean_root: None,
15437                should_contain: true,
15438            });
15439        }
15440
15441        // Scenario 4: last full snapshot is LESS THAN zero lamport account slot
15442        // In this scenario do *not* purge, except when `should_filter_for_incremental_snapshots`
15443        // is false
15444        {
15445            let last_full_snapshot_slot = Some(slot - 1);
15446
15447            do_test(TestParameters {
15448                last_full_snapshot_slot,
15449                max_clean_root: last_full_snapshot_slot,
15450                should_contain: true,
15451            });
15452
15453            do_test(TestParameters {
15454                last_full_snapshot_slot,
15455                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
15456                should_contain: false,
15457            });
15458
15459            do_test(TestParameters {
15460                last_full_snapshot_slot,
15461                max_clean_root: None,
15462                should_contain: false,
15463            });
15464        }
15465    }
15466
15467    #[test]
15468    fn test_hash_account_with_rent_epoch() {
15469        let owner = solana_sdk::pubkey::new_rand();
15470        let pubkey = solana_sdk::pubkey::new_rand();
15471        let slot = 9;
15472        let mut account = AccountSharedData::new(2, 1, &owner);
15473        for rent in 0..3 {
15474            account.set_rent_epoch(rent);
15475            assert_eq!(
15476                AccountsDb::hash_account(slot, &account, &pubkey),
15477                AccountsDb::hash_account_with_rent_epoch(slot, &account, &pubkey, rent)
15478            );
15479        }
15480    }
15481
15482    #[test]
15483    fn test_extend_hashes_with_skipped_rewrites() {
15484        let db = AccountsDb::new_single_for_tests();
15485        let mut hashes = Vec::default();
15486        let rewrites = Rewrites::default();
15487        db.extend_hashes_with_skipped_rewrites(&mut hashes, &rewrites);
15488        assert!(hashes.is_empty());
15489        let pubkey = Pubkey::from([1; 32]);
15490        let hash = Hash::new(&[2; 32]);
15491        rewrites.write().unwrap().insert(pubkey, hash);
15492        db.extend_hashes_with_skipped_rewrites(&mut hashes, &rewrites);
15493        assert_eq!(hashes, vec![(pubkey, hash)]);
15494        // pubkey is already in hashes, will not be added a second time
15495        db.extend_hashes_with_skipped_rewrites(&mut hashes, &rewrites);
15496        assert_eq!(hashes, vec![(pubkey, hash)]);
15497        let pubkey2 = Pubkey::from([2; 32]);
15498        let hash2 = Hash::new(&[3; 32]);
15499        rewrites.write().unwrap().insert(pubkey2, hash2);
15500        db.extend_hashes_with_skipped_rewrites(&mut hashes, &rewrites);
15501        assert_eq!(hashes, vec![(pubkey, hash), (pubkey2, hash2)]);
15502    }
15503
15504    #[test]
15505    fn test_calc_alive_ancient_historical_roots() {
15506        let db = AccountsDb::new_single_for_tests();
15507        let min_root = 0;
15508        let result = db.calc_alive_ancient_historical_roots(min_root);
15509        assert!(result.is_empty());
15510        for extra in 1..3 {
15511            let result = db.calc_alive_ancient_historical_roots(extra);
15512            assert_eq!(result, HashSet::default(), "extra: {}", extra);
15513        }
15514
15515        let extra = 3;
15516        let active_root = 2;
15517        db.accounts_index.add_root(active_root, false);
15518        let result = db.calc_alive_ancient_historical_roots(extra);
15519        let expected_alive_roots = [active_root].into_iter().collect();
15520        assert_eq!(result, expected_alive_roots, "extra: {}", extra);
15521    }
15522}