gemachain_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 crate::{
22    accounts_background_service::{DroppedSlotsSender, SendDroppedBankCallback},
23    accounts_cache::{AccountsCache, CachedAccount, SlotCache},
24    accounts_hash::{AccountsHash, CalculateHashIntermediate, HashStats, PreviousPass},
25    accounts_index::{
26        AccountIndexGetResult, AccountSecondaryIndexes, AccountsIndex, AccountsIndexConfig,
27        AccountsIndexRootsStats, IndexKey, IndexValue, IsCached, RefCount, ScanResult, SlotList,
28        SlotSlice, ZeroCarat, ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS,
29        ACCOUNTS_INDEX_CONFIG_FOR_TESTING,
30    },
31    ancestors::Ancestors,
32    append_vec::{AppendVec, StoredAccountMeta, StoredMeta, StoredMetaWriteVersion},
33    cache_hash_data::CacheHashData,
34    contains::Contains,
35    pubkey_bins::PubkeyBinCalculator16,
36    read_only_accounts_cache::ReadOnlyAccountsCache,
37    sorted_storages::SortedStorages,
38};
39use blake3::traits::digest::Digest;
40use crossbeam_channel::{unbounded, Receiver, Sender};
41use dashmap::{
42    mapref::entry::Entry::{Occupied, Vacant},
43    DashMap, DashSet,
44};
45use lazy_static::lazy_static;
46use log::*;
47use rand::{prelude::SliceRandom, thread_rng, Rng};
48use rayon::{prelude::*, ThreadPool};
49use serde::{Deserialize, Serialize};
50use gemachain_measure::measure::Measure;
51use gemachain_rayon_threadlimit::get_thread_count;
52use gemachain_sdk::{
53    account::{AccountSharedData, ReadableAccount},
54    clock::{BankId, Epoch, Slot},
55    genesis_config::ClusterType,
56    hash::{Hash, Hasher},
57    pubkey::Pubkey,
58    timing::AtomicInterval,
59};
60use gemachain_vote_program::vote_state::MAX_LOCKOUT_HISTORY;
61use std::{
62    borrow::{Borrow, Cow},
63    boxed::Box,
64    collections::{hash_map::Entry, BTreeSet, HashMap, HashSet},
65    convert::TryFrom,
66    hash::{Hash as StdHash, Hasher as StdHasher},
67    io::{Error as IoError, Result as IoResult},
68    ops::{Range, RangeBounds},
69    path::{Path, PathBuf},
70    sync::atomic::{AtomicBool, AtomicU64, AtomicUsize, Ordering},
71    sync::{Arc, Condvar, Mutex, MutexGuard, RwLock},
72    thread::Builder,
73    time::Instant,
74};
75use tempfile::TempDir;
76
77#[cfg(test)]
78use std::{thread::sleep, time::Duration};
79
80const PAGE_SIZE: u64 = 4 * 1024;
81const MAX_RECYCLE_STORES: usize = 1000;
82const STORE_META_OVERHEAD: usize = 256;
83const MAX_CACHE_SLOTS: usize = 200;
84const FLUSH_CACHE_RANDOM_THRESHOLD: usize = MAX_LOCKOUT_HISTORY;
85const SCAN_SLOT_PAR_ITER_THRESHOLD: usize = 4000;
86
87pub const DEFAULT_FILE_SIZE: u64 = PAGE_SIZE * 1024;
88pub const DEFAULT_NUM_THREADS: u32 = 8;
89pub const DEFAULT_NUM_DIRS: u32 = 4;
90
91// When calculating hashes, it is helpful to break the pubkeys found into bins based on the pubkey value.
92// More bins means smaller vectors to sort, copy, etc.
93pub const PUBKEY_BINS_FOR_CALCULATING_HASHES: usize = 65536;
94// # of passes should be a function of the total # of accounts that are active.
95// higher passes = slower total time, lower dynamic memory usage
96// lower passes = faster total time, higher dynamic memory usage
97// passes=2 cuts dynamic memory usage in approximately half.
98pub const NUM_SCAN_PASSES: usize = 2;
99pub const BINS_PER_PASS: usize = PUBKEY_BINS_FOR_CALCULATING_HASHES / NUM_SCAN_PASSES;
100
101// Without chunks, we end up with 1 output vec for each outer snapshot storage.
102// This results in too many vectors to be efficient.
103// Chunks when scanning storages to calculate hashes.
104// If this is too big, we don't get enough parallelism of scanning storages.
105// If this is too small, then we produce too many output vectors to iterate.
106// Metrics indicate a sweet spot in the 2.5k-5k range for mnb.
107const MAX_ITEMS_PER_CHUNK: Slot = 2_500;
108
109// A specially reserved storage id just for entries in the cache, so that
110// operations that take a storage entry can maintain a common interface
111// when interacting with cached accounts. This id is "virtual" in that it
112// doesn't actually refer to an actual storage entry.
113const CACHE_VIRTUAL_STORAGE_ID: usize = AppendVecId::MAX;
114
115// A specially reserved write version (identifier for ordering writes in an AppendVec)
116// for entries in the cache, so that  operations that take a storage entry can maintain
117// a common interface when interacting with cached accounts. This version is "virtual" in
118// that it doesn't actually map to an entry in an AppendVec.
119const CACHE_VIRTUAL_WRITE_VERSION: StoredMetaWriteVersion = 0;
120
121// A specially reserved offset (represents an offset into an AppendVec)
122// for entries in the cache, so that  operations that take a storage entry can maintain
123// a common interface when interacting with cached accounts. This version is "virtual" in
124// that it doesn't actually map to an entry in an AppendVec.
125const CACHE_VIRTUAL_OFFSET: usize = 0;
126const CACHE_VIRTUAL_STORED_SIZE: usize = 0;
127
128pub const ACCOUNTS_DB_CONFIG_FOR_TESTING: AccountsDbConfig = AccountsDbConfig {
129    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_TESTING),
130    accounts_hash_cache_path: None,
131};
132pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig {
133    index: Some(ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS),
134    accounts_hash_cache_path: None,
135};
136
137pub type BinnedHashData = Vec<Vec<CalculateHashIntermediate>>;
138
139#[derive(Debug, Default, Clone)]
140pub struct AccountsDbConfig {
141    pub index: Option<AccountsIndexConfig>,
142    pub accounts_hash_cache_path: Option<PathBuf>,
143}
144
145struct FoundStoredAccount<'a> {
146    pub account: StoredAccountMeta<'a>,
147    pub store_id: AppendVecId,
148    pub account_size: usize,
149}
150
151#[cfg(not(test))]
152const ABSURD_CONSECUTIVE_FAILED_ITERATIONS: usize = 100;
153
154type DashMapVersionHash = DashMap<Pubkey, (u64, Hash)>;
155
156lazy_static! {
157    // FROZEN_ACCOUNT_PANIC is used to signal local_cluster that an AccountsDb panic has occurred,
158    // as |cargo test| cannot observe panics in other threads
159    pub static ref FROZEN_ACCOUNT_PANIC: Arc<AtomicBool> = Arc::new(AtomicBool::new(false));
160}
161
162#[derive(Debug, Clone, Copy)]
163pub enum AccountShrinkThreshold {
164    /// Measure the total space sparseness across all candididates
165    /// And select the candidiates by using the top sparse account storage entries to shrink.
166    /// The value is the overall shrink threshold measured as ratio of the total live bytes
167    /// over the total bytes.
168    TotalSpace { shrink_ratio: f64 },
169    /// Use the following option to shrink all stores whose alive ratio is below
170    /// the specified threshold.
171    IndividalStore { shrink_ratio: f64 },
172}
173pub const DEFAULT_ACCOUNTS_SHRINK_OPTIMIZE_TOTAL_SPACE: bool = true;
174pub const DEFAULT_ACCOUNTS_SHRINK_RATIO: f64 = 0.80;
175// The default extra account space in percentage from the ideal target
176const DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION: AccountShrinkThreshold =
177    AccountShrinkThreshold::TotalSpace {
178        shrink_ratio: DEFAULT_ACCOUNTS_SHRINK_RATIO,
179    };
180
181impl Default for AccountShrinkThreshold {
182    fn default() -> AccountShrinkThreshold {
183        DEFAULT_ACCOUNTS_SHRINK_THRESHOLD_OPTION
184    }
185}
186
187pub enum ScanStorageResult<R, B> {
188    Cached(Vec<R>),
189    Stored(B),
190}
191
192#[derive(Debug, Default)]
193pub struct ErrorCounters {
194    pub total: usize,
195    pub account_in_use: usize,
196    pub account_loaded_twice: usize,
197    pub account_not_found: usize,
198    pub blockhash_not_found: usize,
199    pub blockhash_too_old: usize,
200    pub call_chain_too_deep: usize,
201    pub already_processed: usize,
202    pub instruction_error: usize,
203    pub insufficient_funds: usize,
204    pub invalid_account_for_fee: usize,
205    pub invalid_account_index: usize,
206    pub invalid_program_for_execution: usize,
207    pub not_allowed_during_cluster_maintenance: usize,
208    pub invalid_writable_account: usize,
209}
210
211#[derive(Default, Debug)]
212struct GenerateIndexTimings {
213    pub index_time: u64,
214    pub scan_time: u64,
215    pub insertion_time_us: u64,
216    pub min_bin_size: usize,
217    pub max_bin_size: usize,
218    pub total_items: usize,
219    pub storage_size_accounts_map_us: u64,
220    pub storage_size_storages_us: u64,
221    pub storage_size_accounts_map_flatten_us: u64,
222    pub index_flush_us: u64,
223}
224
225#[derive(Default, Debug, PartialEq)]
226struct StorageSizeAndCount {
227    pub stored_size: usize,
228    pub count: usize,
229}
230type StorageSizeAndCountMap = DashMap<AppendVecId, StorageSizeAndCount>;
231
232impl GenerateIndexTimings {
233    pub fn report(&self) {
234        datapoint_info!(
235            "generate_index",
236            // we cannot accurately measure index insertion time because of many threads and lock contention
237            ("total_us", self.index_time, i64),
238            ("scan_stores_us", self.scan_time, i64),
239            ("insertion_time_us", self.insertion_time_us, i64),
240            ("min_bin_size", self.min_bin_size as i64, i64),
241            ("max_bin_size", self.max_bin_size as i64, i64),
242            (
243                "storage_size_accounts_map_us",
244                self.storage_size_accounts_map_us as i64,
245                i64
246            ),
247            (
248                "storage_size_storages_us",
249                self.storage_size_storages_us as i64,
250                i64
251            ),
252            (
253                "storage_size_accounts_map_flatten_us",
254                self.storage_size_accounts_map_flatten_us as i64,
255                i64
256            ),
257            ("index_flush_us", self.index_flush_us as i64, i64),
258        );
259    }
260}
261
262#[derive(Default, Debug, PartialEq, Clone, Copy)]
263pub struct AccountInfo {
264    /// index identifying the append storage
265    store_id: AppendVecId,
266
267    /// offset into the storage
268    offset: usize,
269
270    /// needed to track shrink candidacy in bytes. Used to update the number
271    /// of alive bytes in an AppendVec as newer slots purge outdated entries
272    stored_size: usize,
273
274    /// carats in the account used when squashing kept for optimization
275    /// purposes to remove accounts with zero balance.
276    carats: u64,
277}
278impl IsCached for AccountInfo {
279    fn is_cached(&self) -> bool {
280        self.store_id == CACHE_VIRTUAL_STORAGE_ID
281    }
282}
283
284impl IndexValue for AccountInfo {}
285
286impl ZeroCarat for AccountInfo {
287    fn is_zero_carat(&self) -> bool {
288        self.carats == 0
289    }
290}
291
292struct MultiThreadProgress<'a> {
293    last_update: Instant,
294    my_last_report_count: u64,
295    total_count: &'a AtomicU64,
296    report_delay_secs: u64,
297    first_caller: bool,
298    ultimate_count: u64,
299}
300
301impl<'a> MultiThreadProgress<'a> {
302    fn new(total_count: &'a AtomicU64, report_delay_secs: u64, ultimate_count: u64) -> Self {
303        Self {
304            last_update: Instant::now(),
305            my_last_report_count: 0,
306            total_count,
307            report_delay_secs,
308            first_caller: false,
309            ultimate_count,
310        }
311    }
312    fn report(&mut self, my_current_count: u64) {
313        let now = Instant::now();
314        if now.duration_since(self.last_update).as_secs() >= self.report_delay_secs {
315            let my_total_newly_processed_slots_since_last_report =
316                my_current_count - self.my_last_report_count;
317
318            self.my_last_report_count = my_current_count;
319            let previous_total_processed_slots_across_all_threads = self.total_count.fetch_add(
320                my_total_newly_processed_slots_since_last_report,
321                Ordering::Relaxed,
322            );
323            self.first_caller =
324                self.first_caller || 0 == previous_total_processed_slots_across_all_threads;
325            if self.first_caller {
326                info!(
327                    "generating index: {}/{} slots...",
328                    previous_total_processed_slots_across_all_threads
329                        + my_total_newly_processed_slots_since_last_report,
330                    self.ultimate_count
331                );
332            }
333            self.last_update = now;
334        }
335    }
336}
337
338/// An offset into the AccountsDb::storage vector
339pub type AppendVecId = usize;
340pub type SnapshotStorage = Vec<Arc<AccountStorageEntry>>;
341pub type SnapshotStorages = Vec<SnapshotStorage>;
342
343// Each slot has a set of storage entries.
344pub(crate) type SlotStores = Arc<RwLock<HashMap<usize, Arc<AccountStorageEntry>>>>;
345
346type AccountSlots = HashMap<Pubkey, HashSet<Slot>>;
347type AppendVecOffsets = HashMap<AppendVecId, HashSet<usize>>;
348type ReclaimResult = (AccountSlots, AppendVecOffsets);
349type StorageFinder<'a> = Box<dyn Fn(Slot, usize) -> Arc<AccountStorageEntry> + 'a>;
350type ShrinkCandidates = HashMap<Slot, HashMap<AppendVecId, Arc<AccountStorageEntry>>>;
351
352trait Versioned {
353    fn version(&self) -> u64;
354}
355
356impl Versioned for (u64, Hash) {
357    fn version(&self) -> u64 {
358        self.0
359    }
360}
361
362impl Versioned for (u64, AccountInfo) {
363    fn version(&self) -> u64 {
364        self.0
365    }
366}
367
368// Some hints for applicability of additional sanity checks for the do_load fast-path;
369// Slower fallback code path will be taken if the fast path has failed over the retry
370// threshold, regardless of these hints. Also, load cannot fail not-deterministically
371// even under very rare circumstances, unlike previously did allow.
372#[derive(Clone, Copy, Debug, PartialEq)]
373pub enum LoadHint {
374    // Caller hints that it's loading transactions for a block which is
375    // descended from the current root, and at the tip of its fork.
376    // Thereby, further this assumes AccountIndex::max_root should not increase
377    // during this load, meaning there should be no squash.
378    // Overall, this enables us to assert!() strictly while running the fast-path for
379    // account loading, while maintaining the determinism of account loading and resultant
380    // transaction execution thereof.
381    FixedMaxRoot,
382    // Caller can't hint the above safety assumption. Generally RPC and miscellaneous
383    // other call-site falls into this category. The likelihood of slower path is slightly
384    // increased as well.
385    Unspecified,
386}
387
388#[derive(Debug)]
389pub enum LoadedAccountAccessor<'a> {
390    // StoredAccountMeta can't be held directly here due to its lifetime dependency to
391    // AccountStorageEntry
392    Stored(Option<(Arc<AccountStorageEntry>, usize)>),
393    // None value in Cached variant means the cache was flushed
394    Cached(Option<(Pubkey, Cow<'a, CachedAccount>)>),
395}
396
397impl<'a> LoadedAccountAccessor<'a> {
398    fn check_and_get_loaded_account(&mut self) -> LoadedAccount {
399        // all of these following .expect() and .unwrap() are like serious logic errors,
400        // ideal for representing this as rust type system....
401
402        match self {
403            LoadedAccountAccessor::Cached(None) | LoadedAccountAccessor::Stored(None) => {
404                panic!("Should have already been taken care of when creating this LoadedAccountAccessor");
405            }
406            LoadedAccountAccessor::Cached(Some(_cached_account)) => {
407                // Cached(Some(x)) variant always produces `Some` for get_loaded_account() since
408                // it just returns the inner `x` without additional fetches
409                self.get_loaded_account().unwrap()
410            }
411            LoadedAccountAccessor::Stored(Some(_maybe_storage_entry)) => {
412                // If we do find the storage entry, we can guarantee that the storage entry is
413                // safe to read from because we grabbed a reference to the storage entry while it
414                // was still in the storage map. This means even if the storage entry is removed
415                // from the storage map after we grabbed the storage entry, the recycler should not
416                // reset the storage entry until we drop the reference to the storage entry.
417                self.get_loaded_account()
418                    .expect("If a storage entry was found in the storage map, it must not have been reset yet")
419            }
420        }
421    }
422
423    fn get_loaded_account(&mut self) -> Option<LoadedAccount> {
424        match self {
425            LoadedAccountAccessor::Cached(cached_account) => {
426                let cached_account: (Pubkey, Cow<'a, CachedAccount>) =
427                    cached_account.take().expect(
428                        "Cache flushed/purged should be handled before trying to fetch account",
429                    );
430                Some(LoadedAccount::Cached(cached_account))
431            }
432            LoadedAccountAccessor::Stored(maybe_storage_entry) => {
433                // storage entry may not be present if slot was cleaned up in
434                // between reading the accounts index and calling this function to
435                // get account meta from the storage entry here
436                maybe_storage_entry
437                    .as_ref()
438                    .and_then(|(storage_entry, offset)| {
439                        storage_entry
440                            .get_stored_account_meta(*offset)
441                            .map(LoadedAccount::Stored)
442                    })
443            }
444        }
445    }
446}
447
448pub enum LoadedAccount<'a> {
449    Stored(StoredAccountMeta<'a>),
450    Cached((Pubkey, Cow<'a, CachedAccount>)),
451}
452
453impl<'a> LoadedAccount<'a> {
454    pub fn owner(&self) -> &Pubkey {
455        match self {
456            LoadedAccount::Stored(stored_account_meta) => &stored_account_meta.account_meta.owner,
457            LoadedAccount::Cached((_, cached_account)) => cached_account.account.owner(),
458        }
459    }
460
461    pub fn executable(&self) -> bool {
462        match self {
463            LoadedAccount::Stored(stored_account_meta) => {
464                stored_account_meta.account_meta.executable
465            }
466            LoadedAccount::Cached((_, cached_account)) => cached_account.account.executable(),
467        }
468    }
469
470    pub fn loaded_hash(&self) -> Hash {
471        match self {
472            LoadedAccount::Stored(stored_account_meta) => *stored_account_meta.hash,
473            LoadedAccount::Cached((_, cached_account)) => cached_account.hash(),
474        }
475    }
476
477    pub fn pubkey(&self) -> &Pubkey {
478        match self {
479            LoadedAccount::Stored(stored_account_meta) => &stored_account_meta.meta.pubkey,
480            LoadedAccount::Cached((pubkey, _)) => pubkey,
481        }
482    }
483
484    pub fn write_version(&self) -> StoredMetaWriteVersion {
485        match self {
486            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.meta.write_version,
487            LoadedAccount::Cached(_) => CACHE_VIRTUAL_WRITE_VERSION,
488        }
489    }
490
491    pub fn compute_hash(&self, slot: Slot, pubkey: &Pubkey) -> Hash {
492        match self {
493            LoadedAccount::Stored(stored_account_meta) => {
494                AccountsDb::hash_stored_account(slot, stored_account_meta)
495            }
496            LoadedAccount::Cached((_, cached_account)) => {
497                AccountsDb::hash_account(slot, &cached_account.account, pubkey)
498            }
499        }
500    }
501
502    pub fn stored_size(&self) -> usize {
503        match self {
504            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.stored_size,
505            LoadedAccount::Cached(_) => CACHE_VIRTUAL_STORED_SIZE,
506        }
507    }
508
509    pub fn carats(&self) -> u64 {
510        match self {
511            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.account_meta.carats,
512            LoadedAccount::Cached((_, cached_account)) => cached_account.account.carats(),
513        }
514    }
515
516    pub fn take_account(self) -> AccountSharedData {
517        match self {
518            LoadedAccount::Stored(stored_account_meta) => stored_account_meta.clone_account(),
519            LoadedAccount::Cached((_, cached_account)) => match cached_account {
520                Cow::Owned(cached_account) => cached_account.account.clone(),
521                Cow::Borrowed(cached_account) => cached_account.account.clone(),
522            },
523        }
524    }
525
526    pub fn is_cached(&self) -> bool {
527        match self {
528            LoadedAccount::Stored(_) => false,
529            LoadedAccount::Cached(_) => true,
530        }
531    }
532}
533
534#[derive(Clone, Default, Debug)]
535pub struct AccountStorage(pub DashMap<Slot, SlotStores>);
536
537impl AccountStorage {
538    fn get_account_storage_entry(
539        &self,
540        slot: Slot,
541        store_id: AppendVecId,
542    ) -> Option<Arc<AccountStorageEntry>> {
543        self.get_slot_stores(slot)
544            .and_then(|storage_map| storage_map.read().unwrap().get(&store_id).cloned())
545    }
546
547    pub fn get_slot_stores(&self, slot: Slot) -> Option<SlotStores> {
548        self.0.get(&slot).map(|result| result.value().clone())
549    }
550
551    fn get_slot_storage_entries(&self, slot: Slot) -> Option<Vec<Arc<AccountStorageEntry>>> {
552        self.get_slot_stores(slot)
553            .map(|res| res.read().unwrap().values().cloned().collect())
554    }
555
556    fn slot_store_count(&self, slot: Slot, store_id: AppendVecId) -> Option<usize> {
557        self.get_account_storage_entry(slot, store_id)
558            .map(|store| store.count())
559    }
560
561    fn all_slots(&self) -> Vec<Slot> {
562        self.0.iter().map(|iter_item| *iter_item.key()).collect()
563    }
564}
565
566#[derive(Debug, Eq, PartialEq, Copy, Clone, Deserialize, Serialize, AbiExample, AbiEnumVisitor)]
567pub enum AccountStorageStatus {
568    Available = 0,
569    Full = 1,
570    Candidate = 2,
571}
572
573impl Default for AccountStorageStatus {
574    fn default() -> Self {
575        Self::Available
576    }
577}
578
579#[derive(Debug)]
580pub enum BankHashVerificationError {
581    MismatchedAccountHash,
582    MismatchedBankHash,
583    MissingBankHash,
584    MismatchedTotalCarats(u64, u64),
585}
586
587#[derive(Default)]
588struct CleanKeyTimings {
589    collect_delta_keys_us: u64,
590    delta_insert_us: u64,
591    hashset_to_vec_us: u64,
592    dirty_store_processing_us: u64,
593    delta_key_count: u64,
594    dirty_pubkeys_count: u64,
595}
596
597/// Persistent storage structure holding the accounts
598#[derive(Debug)]
599pub struct AccountStorageEntry {
600    pub(crate) id: AtomicUsize,
601
602    pub(crate) slot: AtomicU64,
603
604    /// storage holding the accounts
605    pub(crate) accounts: AppendVec,
606
607    /// Keeps track of the number of accounts stored in a specific AppendVec.
608    ///  This is periodically checked to reuse the stores that do not have
609    ///  any accounts in it
610    /// status corresponding to the storage, lets us know that
611    ///  the append_vec, once maxed out, then emptied, can be reclaimed
612    count_and_status: RwLock<(usize, AccountStorageStatus)>,
613
614    /// This is the total number of accounts stored ever since initialized to keep
615    /// track of lifetime count of all store operations. And this differs from
616    /// count_and_status in that this field won't be decremented.
617    ///
618    /// This is used as a rough estimate for slot shrinking. As such a relaxed
619    /// use case, this value ARE NOT strictly synchronized with count_and_status!
620    approx_store_count: AtomicUsize,
621
622    alive_bytes: AtomicUsize,
623}
624
625impl AccountStorageEntry {
626    pub fn new(path: &Path, slot: Slot, id: usize, file_size: u64) -> Self {
627        let tail = AppendVec::file_name(slot, id);
628        let path = Path::new(path).join(tail);
629        let accounts = AppendVec::new(&path, true, file_size as usize);
630
631        Self {
632            id: AtomicUsize::new(id),
633            slot: AtomicU64::new(slot),
634            accounts,
635            count_and_status: RwLock::new((0, AccountStorageStatus::Available)),
636            approx_store_count: AtomicUsize::new(0),
637            alive_bytes: AtomicUsize::new(0),
638        }
639    }
640
641    pub(crate) fn new_existing(
642        slot: Slot,
643        id: AppendVecId,
644        accounts: AppendVec,
645        num_accounts: usize,
646    ) -> Self {
647        Self {
648            id: AtomicUsize::new(id),
649            slot: AtomicU64::new(slot),
650            accounts,
651            count_and_status: RwLock::new((0, AccountStorageStatus::Available)),
652            approx_store_count: AtomicUsize::new(num_accounts),
653            alive_bytes: AtomicUsize::new(0),
654        }
655    }
656
657    pub fn set_status(&self, mut status: AccountStorageStatus) {
658        let mut count_and_status = self.count_and_status.write().unwrap();
659
660        let count = count_and_status.0;
661
662        if status == AccountStorageStatus::Full && count == 0 {
663            // this case arises when the append_vec is full (store_ptrs fails),
664            //  but all accounts have already been removed from the storage
665            //
666            // the only time it's safe to call reset() on an append_vec is when
667            //  every account has been removed
668            //          **and**
669            //  the append_vec has previously been completely full
670            //
671            self.accounts.reset();
672            status = AccountStorageStatus::Available;
673        }
674
675        *count_and_status = (count, status);
676    }
677
678    pub fn recycle(&self, slot: Slot, id: usize) {
679        let mut count_and_status = self.count_and_status.write().unwrap();
680        self.accounts.reset();
681        *count_and_status = (0, AccountStorageStatus::Available);
682        self.slot.store(slot, Ordering::Release);
683        self.id.store(id, Ordering::Relaxed);
684        self.approx_store_count.store(0, Ordering::Relaxed);
685        self.alive_bytes.store(0, Ordering::Relaxed);
686    }
687
688    pub fn status(&self) -> AccountStorageStatus {
689        self.count_and_status.read().unwrap().1
690    }
691
692    pub fn count(&self) -> usize {
693        self.count_and_status.read().unwrap().0
694    }
695
696    pub fn approx_stored_count(&self) -> usize {
697        self.approx_store_count.load(Ordering::Relaxed)
698    }
699
700    pub fn alive_bytes(&self) -> usize {
701        self.alive_bytes.load(Ordering::SeqCst)
702    }
703
704    pub fn written_bytes(&self) -> u64 {
705        self.accounts.len() as u64
706    }
707
708    pub fn total_bytes(&self) -> u64 {
709        self.accounts.capacity()
710    }
711
712    pub fn has_accounts(&self) -> bool {
713        self.count() > 0
714    }
715
716    pub fn slot(&self) -> Slot {
717        self.slot.load(Ordering::Acquire)
718    }
719
720    pub fn append_vec_id(&self) -> AppendVecId {
721        self.id.load(Ordering::Relaxed)
722    }
723
724    pub fn flush(&self) -> Result<(), IoError> {
725        self.accounts.flush()
726    }
727
728    fn get_stored_account_meta(&self, offset: usize) -> Option<StoredAccountMeta> {
729        Some(self.accounts.get_account(offset)?.0)
730    }
731
732    fn add_account(&self, num_bytes: usize) {
733        let mut count_and_status = self.count_and_status.write().unwrap();
734        *count_and_status = (count_and_status.0 + 1, count_and_status.1);
735        self.approx_store_count.fetch_add(1, Ordering::Relaxed);
736        self.alive_bytes.fetch_add(num_bytes, Ordering::SeqCst);
737    }
738
739    fn try_available(&self) -> bool {
740        let mut count_and_status = self.count_and_status.write().unwrap();
741        let (count, status) = *count_and_status;
742
743        if status == AccountStorageStatus::Available {
744            *count_and_status = (count, AccountStorageStatus::Candidate);
745            true
746        } else {
747            false
748        }
749    }
750
751    pub fn all_accounts(&self) -> Vec<StoredAccountMeta> {
752        self.accounts.accounts(0)
753    }
754
755    fn remove_account(&self, num_bytes: usize, reset_accounts: bool) -> usize {
756        let mut count_and_status = self.count_and_status.write().unwrap();
757        let (mut count, mut status) = *count_and_status;
758
759        if count == 1 && status == AccountStorageStatus::Full && reset_accounts {
760            // this case arises when we remove the last account from the
761            //  storage, but we've learned from previous write attempts that
762            //  the storage is full
763            //
764            // the only time it's safe to call reset() on an append_vec is when
765            //  every account has been removed
766            //          **and**
767            //  the append_vec has previously been completely full
768            //
769            // otherwise, the storage may be in flight with a store()
770            //   call
771            self.accounts.reset();
772            status = AccountStorageStatus::Available;
773        }
774
775        // Some code path is removing accounts too many; this may result in an
776        // unintended reveal of old state for unrelated accounts.
777        assert!(
778            count > 0,
779            "double remove of account in slot: {}/store: {}!!",
780            self.slot(),
781            self.append_vec_id(),
782        );
783
784        self.alive_bytes.fetch_sub(num_bytes, Ordering::SeqCst);
785        count -= 1;
786        *count_and_status = (count, status);
787        count
788    }
789
790    pub fn get_path(&self) -> PathBuf {
791        self.accounts.get_path()
792    }
793}
794
795pub fn get_temp_accounts_paths(count: u32) -> IoResult<(Vec<TempDir>, Vec<PathBuf>)> {
796    let temp_dirs: IoResult<Vec<TempDir>> = (0..count).map(|_| TempDir::new()).collect();
797    let temp_dirs = temp_dirs?;
798    let paths: Vec<PathBuf> = temp_dirs.iter().map(|t| t.path().to_path_buf()).collect();
799    Ok((temp_dirs, paths))
800}
801
802#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, AbiExample)]
803pub struct BankHashStats {
804    pub num_updated_accounts: u64,
805    pub num_removed_accounts: u64,
806    pub num_carats_stored: u64,
807    pub total_data_len: u64,
808    pub num_executable_accounts: u64,
809}
810
811impl BankHashStats {
812    pub fn update<T: ReadableAccount>(&mut self, account: &T) {
813        if account.carats() == 0 {
814            self.num_removed_accounts += 1;
815        } else {
816            self.num_updated_accounts += 1;
817        }
818        self.total_data_len = self
819            .total_data_len
820            .wrapping_add(account.data().len() as u64);
821        if account.executable() {
822            self.num_executable_accounts += 1;
823        }
824        self.num_carats_stored = self.num_carats_stored.wrapping_add(account.carats());
825    }
826
827    pub fn merge(&mut self, other: &BankHashStats) {
828        self.num_updated_accounts += other.num_updated_accounts;
829        self.num_removed_accounts += other.num_removed_accounts;
830        self.total_data_len = self.total_data_len.wrapping_add(other.total_data_len);
831        self.num_carats_stored = self
832            .num_carats_stored
833            .wrapping_add(other.num_carats_stored);
834        self.num_executable_accounts += other.num_executable_accounts;
835    }
836}
837
838#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, AbiExample)]
839pub struct BankHashInfo {
840    pub hash: Hash,
841    pub snapshot_hash: Hash,
842    pub stats: BankHashStats,
843}
844
845#[derive(Debug)]
846struct FrozenAccountInfo {
847    pub hash: Hash,    // Hash generated by hash_frozen_account_data()
848    pub carats: u64, // Account balance cannot be lower than this amount
849}
850
851#[derive(Default)]
852pub struct StoreAccountsTiming {
853    store_accounts_elapsed: u64,
854    update_index_elapsed: u64,
855    handle_reclaims_elapsed: u64,
856}
857
858#[derive(Debug, Default)]
859struct RecycleStores {
860    entries: Vec<(Instant, Arc<AccountStorageEntry>)>,
861    total_bytes: u64,
862}
863
864// 30 min should be enough to be certain there won't be any prospective recycle uses for given
865// store entry
866// That's because it already processed ~2500 slots and ~25 passes of AccountsBackgroundService
867pub const EXPIRATION_TTL_SECONDS: u64 = 1800;
868
869impl RecycleStores {
870    fn add_entry(&mut self, new_entry: Arc<AccountStorageEntry>) {
871        self.total_bytes += new_entry.total_bytes();
872        self.entries.push((Instant::now(), new_entry))
873    }
874
875    fn iter(&self) -> std::slice::Iter<(Instant, Arc<AccountStorageEntry>)> {
876        self.entries.iter()
877    }
878
879    fn add_entries(&mut self, new_entries: Vec<Arc<AccountStorageEntry>>) {
880        self.total_bytes += new_entries.iter().map(|e| e.total_bytes()).sum::<u64>();
881        let now = Instant::now();
882        for new_entry in new_entries {
883            self.entries.push((now, new_entry));
884        }
885    }
886
887    fn expire_old_entries(&mut self) -> Vec<Arc<AccountStorageEntry>> {
888        let mut expired = vec![];
889        let now = Instant::now();
890        let mut expired_bytes = 0;
891        self.entries.retain(|(recycled_time, entry)| {
892            if now.duration_since(*recycled_time).as_secs() > EXPIRATION_TTL_SECONDS {
893                if Arc::strong_count(entry) >= 2 {
894                    warn!(
895                        "Expiring still in-use recycled StorageEntry anyway...: id: {} slot: {}",
896                        entry.append_vec_id(),
897                        entry.slot(),
898                    );
899                }
900                expired_bytes += entry.total_bytes();
901                expired.push(entry.clone());
902                false
903            } else {
904                true
905            }
906        });
907
908        self.total_bytes -= expired_bytes;
909
910        expired
911    }
912
913    fn remove_entry(&mut self, index: usize) -> Arc<AccountStorageEntry> {
914        let (_added_time, removed_entry) = self.entries.swap_remove(index);
915        self.total_bytes -= removed_entry.total_bytes();
916        removed_entry
917    }
918
919    fn entry_count(&self) -> usize {
920        self.entries.len()
921    }
922
923    fn total_bytes(&self) -> u64 {
924        self.total_bytes
925    }
926}
927
928/// Removing unrooted slots in Accounts Background Service needs to be synchronized with flushing
929/// slots from the Accounts Cache.  This keeps track of those slots and the Mutex + Condvar for
930/// synchronization.
931#[derive(Debug, Default)]
932struct RemoveUnrootedSlotsSynchronization {
933    // slots being flushed from the cache or being purged
934    slots_under_contention: Mutex<HashSet<Slot>>,
935    signal: Condvar,
936}
937
938type AccountInfoAccountsIndex = AccountsIndex<AccountInfo>;
939
940// This structure handles the load/store of the accounts
941#[derive(Debug)]
942pub struct AccountsDb {
943    /// Keeps tracks of index into AppendVec on a per slot basis
944    pub accounts_index: AccountInfoAccountsIndex,
945
946    pub storage: AccountStorage,
947
948    pub accounts_cache: AccountsCache,
949
950    sender_bg_hasher: Option<Sender<CachedAccount>>,
951    pub read_only_accounts_cache: ReadOnlyAccountsCache,
952
953    recycle_stores: RwLock<RecycleStores>,
954
955    /// distribute the accounts across storage lists
956    pub next_id: AtomicUsize,
957
958    /// Set of shrinkable stores organized by map of slot to append_vec_id
959    pub shrink_candidate_slots: Mutex<ShrinkCandidates>,
960
961    /// Legacy shrink slots to support non-cached code-path.
962    pub shrink_candidate_slots_v1: Mutex<Vec<Slot>>,
963
964    pub(crate) write_version: AtomicU64,
965
966    /// Set of storage paths to pick from
967    pub(crate) paths: Vec<PathBuf>,
968
969    accounts_hash_cache_path: PathBuf,
970
971    // used by tests
972    // holds this until we are dropped
973    temp_accounts_hash_cache_path: Option<TempDir>,
974
975    pub shrink_paths: RwLock<Option<Vec<PathBuf>>>,
976
977    /// Directory of paths this accounts_db needs to hold/remove
978    pub(crate) temp_paths: Option<Vec<TempDir>>,
979
980    /// Starting file size of appendvecs
981    file_size: u64,
982
983    /// Accounts that will cause a panic! if data modified or carats decrease
984    frozen_accounts: HashMap<Pubkey, FrozenAccountInfo>,
985
986    /// Thread pool used for par_iter
987    pub thread_pool: ThreadPool,
988
989    pub thread_pool_clean: ThreadPool,
990
991    /// Number of append vecs to create to maximize parallelism when scanning
992    /// the accounts
993    min_num_stores: usize,
994
995    pub bank_hashes: RwLock<HashMap<Slot, BankHashInfo>>,
996
997    stats: AccountsStats,
998
999    clean_accounts_stats: CleanAccountsStats,
1000
1001    // Stats for purges called outside of clean_accounts()
1002    external_purge_slots_stats: PurgeStats,
1003
1004    shrink_stats: ShrinkStats,
1005
1006    pub cluster_type: Option<ClusterType>,
1007
1008    pub account_indexes: AccountSecondaryIndexes,
1009
1010    pub caching_enabled: bool,
1011
1012    /// Set of unique keys per slot which is used
1013    /// to drive clean_accounts
1014    /// Generated by get_accounts_delta_hash
1015    uncleaned_pubkeys: DashMap<Slot, Vec<Pubkey>>,
1016
1017    #[cfg(test)]
1018    load_delay: u64,
1019
1020    #[cfg(test)]
1021    load_limit: AtomicU64,
1022
1023    is_bank_drop_callback_enabled: AtomicBool,
1024
1025    /// Set of slots currently being flushed by `flush_slot_cache()` or removed
1026    /// by `remove_unrooted_slot()`. Used to ensure `remove_unrooted_slots(slots)`
1027    /// can safely clear the set of unrooted slots `slots`.
1028    remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization,
1029
1030    shrink_ratio: AccountShrinkThreshold,
1031
1032    /// Set of stores which are recently rooted or had accounts removed
1033    /// such that potentially a 0-carat account update could be present which
1034    /// means we can remove the account from the index entirely.
1035    dirty_stores: DashMap<(Slot, AppendVecId), Arc<AccountStorageEntry>>,
1036
1037    /// Zero-carat accounts that are *not* purged during clean because they need to stay alive
1038    /// for incremental snapshot support.
1039    zero_carat_accounts_to_purge_after_full_snapshot: DashSet<(Slot, Pubkey)>,
1040}
1041
1042#[derive(Debug, Default)]
1043struct AccountsStats {
1044    delta_hash_scan_time_total_us: AtomicU64,
1045    delta_hash_accumulate_time_total_us: AtomicU64,
1046    delta_hash_num: AtomicU64,
1047
1048    last_store_report: AtomicInterval,
1049    store_hash_accounts: AtomicU64,
1050    calc_stored_meta: AtomicU64,
1051    store_accounts: AtomicU64,
1052    store_update_index: AtomicU64,
1053    store_handle_reclaims: AtomicU64,
1054    store_append_accounts: AtomicU64,
1055    store_find_store: AtomicU64,
1056    store_num_accounts: AtomicU64,
1057    store_total_data: AtomicU64,
1058    recycle_store_count: AtomicU64,
1059    create_store_count: AtomicU64,
1060    store_get_slot_store: AtomicU64,
1061    store_find_existing: AtomicU64,
1062    dropped_stores: AtomicU64,
1063    store_uncleaned_update: AtomicU64,
1064}
1065
1066#[derive(Debug, Default)]
1067struct PurgeStats {
1068    last_report: AtomicInterval,
1069    safety_checks_elapsed: AtomicU64,
1070    remove_cache_elapsed: AtomicU64,
1071    remove_storage_entries_elapsed: AtomicU64,
1072    drop_storage_entries_elapsed: AtomicU64,
1073    num_cached_slots_removed: AtomicUsize,
1074    num_stored_slots_removed: AtomicUsize,
1075    total_removed_storage_entries: AtomicUsize,
1076    total_removed_cached_bytes: AtomicU64,
1077    total_removed_stored_bytes: AtomicU64,
1078    recycle_stores_write_elapsed: AtomicU64,
1079    scan_storages_elasped: AtomicU64,
1080    purge_accounts_index_elapsed: AtomicU64,
1081    handle_reclaims_elapsed: AtomicU64,
1082}
1083
1084impl PurgeStats {
1085    fn report(&self, metric_name: &'static str, report_interval_ms: Option<u64>) {
1086        let should_report = report_interval_ms
1087            .map(|report_interval_ms| self.last_report.should_update(report_interval_ms))
1088            .unwrap_or(true);
1089
1090        if should_report {
1091            datapoint_info!(
1092                metric_name,
1093                (
1094                    "safety_checks_elapsed",
1095                    self.safety_checks_elapsed.swap(0, Ordering::Relaxed) as i64,
1096                    i64
1097                ),
1098                (
1099                    "remove_cache_elapsed",
1100                    self.remove_cache_elapsed.swap(0, Ordering::Relaxed) as i64,
1101                    i64
1102                ),
1103                (
1104                    "remove_storage_entries_elapsed",
1105                    self.remove_storage_entries_elapsed
1106                        .swap(0, Ordering::Relaxed) as i64,
1107                    i64
1108                ),
1109                (
1110                    "drop_storage_entries_elapsed",
1111                    self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
1112                    i64
1113                ),
1114                (
1115                    "num_cached_slots_removed",
1116                    self.num_cached_slots_removed.swap(0, Ordering::Relaxed) as i64,
1117                    i64
1118                ),
1119                (
1120                    "num_stored_slots_removed",
1121                    self.num_stored_slots_removed.swap(0, Ordering::Relaxed) as i64,
1122                    i64
1123                ),
1124                (
1125                    "total_removed_storage_entries",
1126                    self.total_removed_storage_entries
1127                        .swap(0, Ordering::Relaxed) as i64,
1128                    i64
1129                ),
1130                (
1131                    "total_removed_cached_bytes",
1132                    self.total_removed_cached_bytes.swap(0, Ordering::Relaxed) as i64,
1133                    i64
1134                ),
1135                (
1136                    "total_removed_stored_bytes",
1137                    self.total_removed_stored_bytes.swap(0, Ordering::Relaxed) as i64,
1138                    i64
1139                ),
1140                (
1141                    "recycle_stores_write_elapsed",
1142                    self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
1143                    i64
1144                ),
1145                (
1146                    "scan_storages_elasped",
1147                    self.scan_storages_elasped.swap(0, Ordering::Relaxed) as i64,
1148                    i64
1149                ),
1150                (
1151                    "purge_accounts_index_elapsed",
1152                    self.purge_accounts_index_elapsed.swap(0, Ordering::Relaxed) as i64,
1153                    i64
1154                ),
1155                (
1156                    "handle_reclaims_elapsed",
1157                    self.handle_reclaims_elapsed.swap(0, Ordering::Relaxed) as i64,
1158                    i64
1159                ),
1160            );
1161        }
1162    }
1163}
1164
1165#[derive(Debug)]
1166struct FlushStats {
1167    slot: Slot,
1168    num_flushed: usize,
1169    num_purged: usize,
1170    total_size: u64,
1171}
1172
1173#[derive(Debug, Default)]
1174struct LatestAccountsIndexRootsStats {
1175    roots_len: AtomicUsize,
1176    uncleaned_roots_len: AtomicUsize,
1177    previous_uncleaned_roots_len: AtomicUsize,
1178    roots_range: AtomicU64,
1179    rooted_cleaned_count: AtomicUsize,
1180    unrooted_cleaned_count: AtomicUsize,
1181}
1182
1183impl LatestAccountsIndexRootsStats {
1184    fn update(&self, accounts_index_roots_stats: &AccountsIndexRootsStats) {
1185        self.roots_len
1186            .store(accounts_index_roots_stats.roots_len, Ordering::Relaxed);
1187        self.uncleaned_roots_len.store(
1188            accounts_index_roots_stats.uncleaned_roots_len,
1189            Ordering::Relaxed,
1190        );
1191        self.previous_uncleaned_roots_len.store(
1192            accounts_index_roots_stats.previous_uncleaned_roots_len,
1193            Ordering::Relaxed,
1194        );
1195        self.roots_range
1196            .store(accounts_index_roots_stats.roots_range, Ordering::Relaxed);
1197        self.rooted_cleaned_count.fetch_add(
1198            accounts_index_roots_stats.rooted_cleaned_count,
1199            Ordering::Relaxed,
1200        );
1201        self.unrooted_cleaned_count.fetch_add(
1202            accounts_index_roots_stats.unrooted_cleaned_count,
1203            Ordering::Relaxed,
1204        );
1205    }
1206
1207    fn report(&self) {
1208        datapoint_info!(
1209            "accounts_index_roots_len",
1210            (
1211                "roots_len",
1212                self.roots_len.load(Ordering::Relaxed) as i64,
1213                i64
1214            ),
1215            (
1216                "uncleaned_roots_len",
1217                self.uncleaned_roots_len.load(Ordering::Relaxed) as i64,
1218                i64
1219            ),
1220            (
1221                "previous_uncleaned_roots_len",
1222                self.previous_uncleaned_roots_len.load(Ordering::Relaxed) as i64,
1223                i64
1224            ),
1225            (
1226                "roots_range_width",
1227                self.roots_range.load(Ordering::Relaxed) as i64,
1228                i64
1229            ),
1230            (
1231                "unrooted_cleaned_count",
1232                self.unrooted_cleaned_count.swap(0, Ordering::Relaxed) as i64,
1233                i64
1234            ),
1235            (
1236                "rooted_cleaned_count",
1237                self.rooted_cleaned_count.swap(0, Ordering::Relaxed) as i64,
1238                i64
1239            ),
1240        );
1241
1242        // Don't need to reset since this tracks the latest updates, not a cumulative total
1243    }
1244}
1245
1246#[derive(Debug, Default)]
1247struct CleanAccountsStats {
1248    purge_stats: PurgeStats,
1249    latest_accounts_index_roots_stats: LatestAccountsIndexRootsStats,
1250}
1251
1252impl CleanAccountsStats {
1253    fn report(&self) {
1254        self.purge_stats.report("clean_purge_slots_stats", None);
1255        self.latest_accounts_index_roots_stats.report();
1256    }
1257}
1258
1259#[derive(Debug, Default)]
1260struct ShrinkStats {
1261    last_report: AtomicInterval,
1262    num_slots_shrunk: AtomicUsize,
1263    storage_read_elapsed: AtomicU64,
1264    index_read_elapsed: AtomicU64,
1265    find_alive_elapsed: AtomicU64,
1266    create_and_insert_store_elapsed: AtomicU64,
1267    store_accounts_elapsed: AtomicU64,
1268    update_index_elapsed: AtomicU64,
1269    handle_reclaims_elapsed: AtomicU64,
1270    write_storage_elapsed: AtomicU64,
1271    rewrite_elapsed: AtomicU64,
1272    drop_storage_entries_elapsed: AtomicU64,
1273    recycle_stores_write_elapsed: AtomicU64,
1274    accounts_removed: AtomicUsize,
1275    bytes_removed: AtomicU64,
1276    bytes_written: AtomicU64,
1277    skipped_shrink: AtomicU64,
1278    dead_accounts: AtomicU64,
1279    alive_accounts: AtomicU64,
1280    accounts_loaded: AtomicU64,
1281}
1282
1283impl ShrinkStats {
1284    fn report(&self) {
1285        if self.last_report.should_update(1000) {
1286            datapoint_info!(
1287                "shrink_stats",
1288                (
1289                    "num_slots_shrunk",
1290                    self.num_slots_shrunk.swap(0, Ordering::Relaxed) as i64,
1291                    i64
1292                ),
1293                (
1294                    "storage_read_elapsed",
1295                    self.storage_read_elapsed.swap(0, Ordering::Relaxed) as i64,
1296                    i64
1297                ),
1298                (
1299                    "index_read_elapsed",
1300                    self.index_read_elapsed.swap(0, Ordering::Relaxed) as i64,
1301                    i64
1302                ),
1303                (
1304                    "find_alive_elapsed",
1305                    self.find_alive_elapsed.swap(0, Ordering::Relaxed) as i64,
1306                    i64
1307                ),
1308                (
1309                    "create_and_insert_store_elapsed",
1310                    self.create_and_insert_store_elapsed
1311                        .swap(0, Ordering::Relaxed) as i64,
1312                    i64
1313                ),
1314                (
1315                    "store_accounts_elapsed",
1316                    self.store_accounts_elapsed.swap(0, Ordering::Relaxed) as i64,
1317                    i64
1318                ),
1319                (
1320                    "update_index_elapsed",
1321                    self.update_index_elapsed.swap(0, Ordering::Relaxed) as i64,
1322                    i64
1323                ),
1324                (
1325                    "handle_reclaims_elapsed",
1326                    self.handle_reclaims_elapsed.swap(0, Ordering::Relaxed) as i64,
1327                    i64
1328                ),
1329                (
1330                    "write_storage_elapsed",
1331                    self.write_storage_elapsed.swap(0, Ordering::Relaxed) as i64,
1332                    i64
1333                ),
1334                (
1335                    "rewrite_elapsed",
1336                    self.rewrite_elapsed.swap(0, Ordering::Relaxed) as i64,
1337                    i64
1338                ),
1339                (
1340                    "drop_storage_entries_elapsed",
1341                    self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
1342                    i64
1343                ),
1344                (
1345                    "recycle_stores_write_time",
1346                    self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
1347                    i64
1348                ),
1349                (
1350                    "accounts_removed",
1351                    self.accounts_removed.swap(0, Ordering::Relaxed) as i64,
1352                    i64
1353                ),
1354                (
1355                    "bytes_removed",
1356                    self.bytes_removed.swap(0, Ordering::Relaxed) as i64,
1357                    i64
1358                ),
1359                (
1360                    "bytes_written",
1361                    self.bytes_written.swap(0, Ordering::Relaxed) as i64,
1362                    i64
1363                ),
1364                (
1365                    "skipped_shrink",
1366                    self.skipped_shrink.swap(0, Ordering::Relaxed) as i64,
1367                    i64
1368                ),
1369                (
1370                    "alive_accounts",
1371                    self.alive_accounts.swap(0, Ordering::Relaxed) as i64,
1372                    i64
1373                ),
1374                (
1375                    "dead_accounts",
1376                    self.dead_accounts.swap(0, Ordering::Relaxed) as i64,
1377                    i64
1378                ),
1379                (
1380                    "accounts_loaded",
1381                    self.accounts_loaded.swap(0, Ordering::Relaxed) as i64,
1382                    i64
1383                ),
1384            );
1385        }
1386    }
1387}
1388
1389pub fn make_min_priority_thread_pool() -> ThreadPool {
1390    // Use lower thread count to reduce priority.
1391    let num_threads = std::cmp::max(2, num_cpus::get() / 4);
1392    rayon::ThreadPoolBuilder::new()
1393        .thread_name(|i| format!("gemachain-cleanup-accounts-{}", i))
1394        .num_threads(num_threads)
1395        .build()
1396        .unwrap()
1397}
1398
1399#[cfg(all(test, RUSTC_WITH_SPECIALIZATION))]
1400impl gemachain_frozen_abi::abi_example::AbiExample for AccountsDb {
1401    fn example() -> Self {
1402        let accounts_db = AccountsDb::new_single_for_tests();
1403        let key = Pubkey::default();
1404        let some_data_len = 5;
1405        let some_slot: Slot = 0;
1406        let account = AccountSharedData::new(1, some_data_len, &key);
1407        accounts_db.store_uncached(some_slot, &[(&key, &account)]);
1408        accounts_db.add_root(0);
1409
1410        accounts_db
1411    }
1412}
1413
1414impl<'a> ReadableAccount for StoredAccountMeta<'a> {
1415    fn carats(&self) -> u64 {
1416        self.account_meta.carats
1417    }
1418    fn data(&self) -> &[u8] {
1419        self.data
1420    }
1421    fn owner(&self) -> &Pubkey {
1422        &self.account_meta.owner
1423    }
1424    fn executable(&self) -> bool {
1425        self.account_meta.executable
1426    }
1427    fn rent_epoch(&self) -> Epoch {
1428        self.account_meta.rent_epoch
1429    }
1430}
1431
1432struct IndexAccountMapEntry<'a> {
1433    pub write_version: StoredMetaWriteVersion,
1434    pub store_id: AppendVecId,
1435    pub stored_account: StoredAccountMeta<'a>,
1436}
1437
1438type GenerateIndexAccountsMap<'a> = HashMap<Pubkey, IndexAccountMapEntry<'a>>;
1439
1440impl AccountsDb {
1441    pub fn default_for_tests() -> Self {
1442        Self::default_with_accounts_index(AccountInfoAccountsIndex::default_for_tests(), None)
1443    }
1444
1445    fn default_with_accounts_index(
1446        accounts_index: AccountInfoAccountsIndex,
1447        accounts_hash_cache_path: Option<PathBuf>,
1448    ) -> Self {
1449        let num_threads = get_thread_count();
1450        const MAX_READ_ONLY_CACHE_DATA_SIZE: usize = 200_000_000;
1451
1452        let mut temp_accounts_hash_cache_path = None;
1453        let accounts_hash_cache_path = accounts_hash_cache_path.unwrap_or_else(|| {
1454            temp_accounts_hash_cache_path = Some(TempDir::new().unwrap());
1455            temp_accounts_hash_cache_path
1456                .as_ref()
1457                .unwrap()
1458                .path()
1459                .to_path_buf()
1460        });
1461
1462        let mut bank_hashes = HashMap::new();
1463        bank_hashes.insert(0, BankHashInfo::default());
1464        AccountsDb {
1465            accounts_index,
1466            storage: AccountStorage::default(),
1467            accounts_cache: AccountsCache::default(),
1468            sender_bg_hasher: None,
1469            read_only_accounts_cache: ReadOnlyAccountsCache::new(MAX_READ_ONLY_CACHE_DATA_SIZE),
1470            recycle_stores: RwLock::new(RecycleStores::default()),
1471            uncleaned_pubkeys: DashMap::new(),
1472            next_id: AtomicUsize::new(0),
1473            shrink_candidate_slots_v1: Mutex::new(Vec::new()),
1474            shrink_candidate_slots: Mutex::new(HashMap::new()),
1475            write_version: AtomicU64::new(0),
1476            paths: vec![],
1477            accounts_hash_cache_path,
1478            temp_accounts_hash_cache_path,
1479            shrink_paths: RwLock::new(None),
1480            temp_paths: None,
1481            file_size: DEFAULT_FILE_SIZE,
1482            thread_pool: rayon::ThreadPoolBuilder::new()
1483                .num_threads(num_threads)
1484                .thread_name(|i| format!("gemachain-db-accounts-{}", i))
1485                .build()
1486                .unwrap(),
1487            thread_pool_clean: make_min_priority_thread_pool(),
1488            min_num_stores: num_threads,
1489            bank_hashes: RwLock::new(bank_hashes),
1490            frozen_accounts: HashMap::new(),
1491            external_purge_slots_stats: PurgeStats::default(),
1492            clean_accounts_stats: CleanAccountsStats::default(),
1493            shrink_stats: ShrinkStats::default(),
1494            stats: AccountsStats::default(),
1495            cluster_type: None,
1496            account_indexes: AccountSecondaryIndexes::default(),
1497            caching_enabled: false,
1498            #[cfg(test)]
1499            load_delay: u64::default(),
1500            #[cfg(test)]
1501            load_limit: AtomicU64::default(),
1502            is_bank_drop_callback_enabled: AtomicBool::default(),
1503            remove_unrooted_slots_synchronization: RemoveUnrootedSlotsSynchronization::default(),
1504            shrink_ratio: AccountShrinkThreshold::default(),
1505            dirty_stores: DashMap::default(),
1506            zero_carat_accounts_to_purge_after_full_snapshot: DashSet::default(),
1507        }
1508    }
1509
1510    pub fn new_for_tests(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
1511        AccountsDb::new_with_config(
1512            paths,
1513            cluster_type,
1514            AccountSecondaryIndexes::default(),
1515            false,
1516            AccountShrinkThreshold::default(),
1517            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
1518        )
1519    }
1520
1521    pub fn new_with_config(
1522        paths: Vec<PathBuf>,
1523        cluster_type: &ClusterType,
1524        account_indexes: AccountSecondaryIndexes,
1525        caching_enabled: bool,
1526        shrink_ratio: AccountShrinkThreshold,
1527        accounts_db_config: Option<AccountsDbConfig>,
1528    ) -> Self {
1529        let accounts_index =
1530            AccountsIndex::new(accounts_db_config.as_ref().and_then(|x| x.index.clone()));
1531        let accounts_hash_cache_path = accounts_db_config
1532            .as_ref()
1533            .and_then(|x| x.accounts_hash_cache_path.clone());
1534        let mut new = if !paths.is_empty() {
1535            Self {
1536                paths,
1537                temp_paths: None,
1538                cluster_type: Some(*cluster_type),
1539                account_indexes,
1540                caching_enabled,
1541                shrink_ratio,
1542                ..Self::default_with_accounts_index(accounts_index, accounts_hash_cache_path)
1543            }
1544        } else {
1545            // Create a temporary set of accounts directories, used primarily
1546            // for testing
1547            let (temp_dirs, paths) = get_temp_accounts_paths(DEFAULT_NUM_DIRS).unwrap();
1548            Self {
1549                paths,
1550                temp_paths: Some(temp_dirs),
1551                cluster_type: Some(*cluster_type),
1552                account_indexes,
1553                caching_enabled,
1554                shrink_ratio,
1555                ..Self::default_with_accounts_index(accounts_index, accounts_hash_cache_path)
1556            }
1557        };
1558
1559        new.start_background_hasher();
1560        {
1561            for path in new.paths.iter() {
1562                std::fs::create_dir_all(path).expect("Create directory failed.");
1563            }
1564        }
1565        new
1566    }
1567
1568    pub fn set_shrink_paths(&self, paths: Vec<PathBuf>) {
1569        assert!(!paths.is_empty());
1570        let mut shrink_paths = self.shrink_paths.write().unwrap();
1571        for path in &paths {
1572            std::fs::create_dir_all(path).expect("Create directory failed.");
1573        }
1574        *shrink_paths = Some(paths);
1575    }
1576
1577    pub fn file_size(&self) -> u64 {
1578        self.file_size
1579    }
1580
1581    pub fn new_single_for_tests() -> Self {
1582        AccountsDb {
1583            min_num_stores: 0,
1584            ..AccountsDb::new_for_tests(Vec::new(), &ClusterType::Development)
1585        }
1586    }
1587
1588    fn new_storage_entry(&self, slot: Slot, path: &Path, size: u64) -> AccountStorageEntry {
1589        AccountStorageEntry::new(
1590            path,
1591            slot,
1592            self.next_id.fetch_add(1, Ordering::Relaxed),
1593            size,
1594        )
1595    }
1596
1597    pub fn expected_cluster_type(&self) -> ClusterType {
1598        self.cluster_type
1599            .expect("Cluster type must be set at initialization")
1600    }
1601
1602    /// Reclaim older states of accounts older than max_clean_root for AccountsDb bloat mitigation
1603    fn clean_accounts_older_than_root(
1604        &self,
1605        purges: Vec<Pubkey>,
1606        max_clean_root: Option<Slot>,
1607    ) -> ReclaimResult {
1608        if purges.is_empty() {
1609            return ReclaimResult::default();
1610        }
1611        // This number isn't carefully chosen; just guessed randomly such that
1612        // the hot loop will be the order of ~Xms.
1613        const INDEX_CLEAN_BULK_COUNT: usize = 4096;
1614
1615        let mut clean_rooted = Measure::start("clean_old_root-ms");
1616        let reclaim_vecs = purges
1617            .par_chunks(INDEX_CLEAN_BULK_COUNT)
1618            .map(|pubkeys: &[Pubkey]| {
1619                let mut reclaims = Vec::new();
1620                for pubkey in pubkeys {
1621                    self.accounts_index
1622                        .clean_rooted_entries(pubkey, &mut reclaims, max_clean_root);
1623                }
1624                reclaims
1625            });
1626        let reclaims: Vec<_> = reclaim_vecs.flatten().collect();
1627        clean_rooted.stop();
1628        inc_new_counter_info!("clean-old-root-par-clean-ms", clean_rooted.as_ms() as usize);
1629
1630        let mut measure = Measure::start("clean_old_root_reclaims");
1631
1632        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
1633        // and those stores may be used for background hashing.
1634        let reset_accounts = false;
1635
1636        let mut reclaim_result = ReclaimResult::default();
1637        self.handle_reclaims(
1638            &reclaims,
1639            None,
1640            Some(&self.clean_accounts_stats.purge_stats),
1641            Some(&mut reclaim_result),
1642            reset_accounts,
1643        );
1644        measure.stop();
1645        debug!("{} {}", clean_rooted, measure);
1646        inc_new_counter_info!("clean-old-root-reclaim-ms", measure.as_ms() as usize);
1647        reclaim_result
1648    }
1649
1650    fn do_reset_uncleaned_roots(&self, max_clean_root: Option<Slot>) {
1651        self.accounts_index.reset_uncleaned_roots(max_clean_root);
1652    }
1653
1654    fn calc_delete_dependencies(
1655        purges: &HashMap<Pubkey, (SlotList<AccountInfo>, u64)>,
1656        store_counts: &mut HashMap<AppendVecId, (usize, HashSet<Pubkey>)>,
1657    ) {
1658        // Another pass to check if there are some filtered accounts which
1659        // do not match the criteria of deleting all appendvecs which contain them
1660        // then increment their storage count.
1661        let mut already_counted = HashSet::new();
1662        for (pubkey, (account_infos, ref_count_from_storage)) in purges.iter() {
1663            let no_delete = if account_infos.len() as u64 != *ref_count_from_storage {
1664                debug!(
1665                    "calc_delete_dependencies(),
1666                    pubkey: {},
1667                    account_infos: {:?},
1668                    account_infos_len: {},
1669                    ref_count_from_storage: {}",
1670                    pubkey,
1671                    account_infos,
1672                    account_infos.len(),
1673                    ref_count_from_storage,
1674                );
1675                true
1676            } else {
1677                let mut no_delete = false;
1678                for (_slot, account_info) in account_infos {
1679                    debug!(
1680                        "calc_delete_dependencies()
1681                        storage id: {},
1682                        count len: {}",
1683                        account_info.store_id,
1684                        store_counts.get(&account_info.store_id).unwrap().0,
1685                    );
1686                    if store_counts.get(&account_info.store_id).unwrap().0 != 0 {
1687                        no_delete = true;
1688                        break;
1689                    }
1690                }
1691                no_delete
1692            };
1693            if no_delete {
1694                let mut pending_store_ids: HashSet<usize> = HashSet::new();
1695                for (_bank_id, account_info) in account_infos {
1696                    if !already_counted.contains(&account_info.store_id) {
1697                        pending_store_ids.insert(account_info.store_id);
1698                    }
1699                }
1700                while !pending_store_ids.is_empty() {
1701                    let id = pending_store_ids.iter().next().cloned().unwrap();
1702                    pending_store_ids.remove(&id);
1703                    if already_counted.contains(&id) {
1704                        continue;
1705                    }
1706                    store_counts.get_mut(&id).unwrap().0 += 1;
1707                    already_counted.insert(id);
1708
1709                    let affected_pubkeys = &store_counts.get(&id).unwrap().1;
1710                    for key in affected_pubkeys {
1711                        for (_slot, account_info) in &purges.get(key).unwrap().0 {
1712                            if !already_counted.contains(&account_info.store_id) {
1713                                pending_store_ids.insert(account_info.store_id);
1714                            }
1715                        }
1716                    }
1717                }
1718            }
1719        }
1720    }
1721
1722    fn background_hasher(receiver: Receiver<CachedAccount>) {
1723        loop {
1724            let result = receiver.recv();
1725            match result {
1726                Ok(account) => {
1727                    // if we hold the only ref, then this account doesn't need to be hashed, we ignore this account and it will disappear
1728                    if Arc::strong_count(&account) > 1 {
1729                        // this will cause the hash to be calculated and store inside account if it needs to be calculated
1730                        let _ = (*account).hash();
1731                    };
1732                }
1733                Err(_) => {
1734                    break;
1735                }
1736            }
1737        }
1738    }
1739
1740    fn start_background_hasher(&mut self) {
1741        let (sender, receiver) = unbounded();
1742        Builder::new()
1743            .name("gemachain-db-store-hasher-accounts".to_string())
1744            .spawn(move || {
1745                Self::background_hasher(receiver);
1746            })
1747            .unwrap();
1748        self.sender_bg_hasher = Some(sender);
1749    }
1750
1751    fn purge_keys_exact<'a, C: 'a>(
1752        &'a self,
1753        pubkey_to_slot_set: impl Iterator<Item = &'a (Pubkey, C)>,
1754    ) -> Vec<(u64, AccountInfo)>
1755    where
1756        C: Contains<'a, Slot>,
1757    {
1758        let mut reclaims = Vec::new();
1759        let mut dead_keys = Vec::new();
1760
1761        for (pubkey, slots_set) in pubkey_to_slot_set {
1762            let is_empty = self
1763                .accounts_index
1764                .purge_exact(pubkey, slots_set, &mut reclaims);
1765            if is_empty {
1766                dead_keys.push(pubkey);
1767            }
1768        }
1769
1770        self.accounts_index
1771            .handle_dead_keys(&dead_keys, &self.account_indexes);
1772        reclaims
1773    }
1774
1775    fn max_clean_root(&self, proposed_clean_root: Option<Slot>) -> Option<Slot> {
1776        match (
1777            self.accounts_index.min_ongoing_scan_root(),
1778            proposed_clean_root,
1779        ) {
1780            (None, None) => None,
1781            (Some(min_scan_root), None) => Some(min_scan_root),
1782            (None, Some(proposed_clean_root)) => Some(proposed_clean_root),
1783            (Some(min_scan_root), Some(proposed_clean_root)) => {
1784                Some(std::cmp::min(min_scan_root, proposed_clean_root))
1785            }
1786        }
1787    }
1788
1789    /// Collect all the uncleaned slots, up to a max slot
1790    ///
1791    /// Search through the uncleaned Pubkeys and return all the slots, up to a maximum slot.
1792    fn collect_uncleaned_slots_up_to_slot(&self, max_slot: Slot) -> Vec<Slot> {
1793        self.uncleaned_pubkeys
1794            .iter()
1795            .filter_map(|entry| {
1796                let slot = *entry.key();
1797                (slot <= max_slot).then(|| slot)
1798            })
1799            .collect()
1800    }
1801
1802    /// Remove `slots` from `uncleaned_pubkeys` and collect all pubkeys
1803    ///
1804    /// For each slot in the list of uncleaned slots, remove it from the `uncleaned_pubkeys` Map
1805    /// and collect all the pubkeys to return.
1806    fn remove_uncleaned_slots_and_collect_pubkeys(
1807        &self,
1808        uncleaned_slots: Vec<Slot>,
1809    ) -> Vec<Vec<Pubkey>> {
1810        uncleaned_slots
1811            .into_iter()
1812            .filter_map(|uncleaned_slot| {
1813                self.uncleaned_pubkeys
1814                    .remove(&uncleaned_slot)
1815                    .map(|(_removed_slot, removed_pubkeys)| removed_pubkeys)
1816            })
1817            .collect()
1818    }
1819
1820    /// Remove uncleaned slots, up to a maximum slot, and return the collected pubkeys
1821    ///
1822    fn remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(
1823        &self,
1824        max_slot: Slot,
1825    ) -> Vec<Vec<Pubkey>> {
1826        let uncleaned_slots = self.collect_uncleaned_slots_up_to_slot(max_slot);
1827        self.remove_uncleaned_slots_and_collect_pubkeys(uncleaned_slots)
1828    }
1829
1830    // Construct a vec of pubkeys for cleaning from:
1831    //   uncleaned_pubkeys - the delta set of updated pubkeys in rooted slots from the last clean
1832    //   dirty_stores - set of stores which had accounts removed or recently rooted
1833    fn construct_candidate_clean_keys(
1834        &self,
1835        max_clean_root: Option<Slot>,
1836        last_full_snapshot_slot: Option<Slot>,
1837        timings: &mut CleanKeyTimings,
1838    ) -> Vec<Pubkey> {
1839        let mut dirty_store_processing_time = Measure::start("dirty_store_processing");
1840        let max_slot = max_clean_root.unwrap_or_else(|| self.accounts_index.max_root());
1841        let mut dirty_stores = Vec::with_capacity(self.dirty_stores.len());
1842        self.dirty_stores.retain(|(slot, _store_id), store| {
1843            if *slot > max_slot {
1844                true
1845            } else {
1846                dirty_stores.push((*slot, store.clone()));
1847                false
1848            }
1849        });
1850        let dirty_stores_len = dirty_stores.len();
1851        let pubkeys = DashSet::new();
1852        for (_slot, store) in dirty_stores {
1853            for account in store.accounts.accounts(0) {
1854                pubkeys.insert(account.meta.pubkey);
1855            }
1856        }
1857        trace!(
1858            "dirty_stores.len: {} pubkeys.len: {}",
1859            dirty_stores_len,
1860            pubkeys.len()
1861        );
1862        timings.dirty_pubkeys_count = pubkeys.len() as u64;
1863        dirty_store_processing_time.stop();
1864        timings.dirty_store_processing_us += dirty_store_processing_time.as_us();
1865
1866        let mut collect_delta_keys = Measure::start("key_create");
1867        let delta_keys = self.remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(max_slot);
1868        collect_delta_keys.stop();
1869        timings.collect_delta_keys_us += collect_delta_keys.as_us();
1870
1871        let mut delta_insert = Measure::start("delta_insert");
1872        self.thread_pool_clean.install(|| {
1873            delta_keys.par_iter().for_each(|keys| {
1874                for key in keys {
1875                    pubkeys.insert(*key);
1876                }
1877            });
1878        });
1879        delta_insert.stop();
1880        timings.delta_insert_us += delta_insert.as_us();
1881
1882        timings.delta_key_count = pubkeys.len() as u64;
1883
1884        let mut hashset_to_vec = Measure::start("flat_map");
1885        let mut pubkeys: Vec<Pubkey> = pubkeys.into_iter().collect();
1886        hashset_to_vec.stop();
1887        timings.hashset_to_vec_us += hashset_to_vec.as_us();
1888
1889        // Check if we should purge any of the zero_carat_accounts_to_purge_later, based on the
1890        // last_full_snapshot_slot.
1891        assert!(
1892            last_full_snapshot_slot.is_some() || self.zero_carat_accounts_to_purge_after_full_snapshot.is_empty(),
1893            "if snapshots are disabled, then zero_carat_accounts_to_purge_later should always be empty"
1894        );
1895        if let Some(last_full_snapshot_slot) = last_full_snapshot_slot {
1896            self.zero_carat_accounts_to_purge_after_full_snapshot
1897                .retain(|(slot, pubkey)| {
1898                    let is_candidate_for_clean =
1899                        max_slot >= *slot && last_full_snapshot_slot >= *slot;
1900                    if is_candidate_for_clean {
1901                        pubkeys.push(*pubkey);
1902                    }
1903                    !is_candidate_for_clean
1904                });
1905        }
1906
1907        pubkeys
1908    }
1909
1910    // Purge zero carat accounts and older rooted account states as garbage
1911    // collection
1912    // Only remove those accounts where the entire rooted history of the account
1913    // can be purged because there are no live append vecs in the ancestors
1914    pub fn clean_accounts(
1915        &self,
1916        max_clean_root: Option<Slot>,
1917        is_startup: bool,
1918        last_full_snapshot_slot: Option<Slot>,
1919    ) {
1920        let max_clean_root = self.max_clean_root(max_clean_root);
1921
1922        // hold a lock to prevent slot shrinking from running because it might modify some rooted
1923        // slot storages which can not happen as long as we're cleaning accounts because we're also
1924        // modifying the rooted slot storages!
1925        let mut candidates_v1 = self.shrink_candidate_slots_v1.lock().unwrap();
1926        self.report_store_stats();
1927
1928        let mut key_timings = CleanKeyTimings::default();
1929        let pubkeys = self.construct_candidate_clean_keys(
1930            max_clean_root,
1931            last_full_snapshot_slot,
1932            &mut key_timings,
1933        );
1934
1935        let total_keys_count = pubkeys.len();
1936        let mut accounts_scan = Measure::start("accounts_scan");
1937        // parallel scan the index.
1938        let (mut purges_zero_carats, purges_old_accounts) = {
1939            let do_clean_scan = || {
1940                pubkeys
1941                    .par_chunks(4096)
1942                    .map(|pubkeys: &[Pubkey]| {
1943                        let mut purges_zero_carats = HashMap::new();
1944                        let mut purges_old_accounts = Vec::new();
1945                        for pubkey in pubkeys {
1946                            match self.accounts_index.get(pubkey, None, max_clean_root) {
1947                                AccountIndexGetResult::Found(locked_entry, index) => {
1948                                    let slot_list = locked_entry.slot_list();
1949                                    let (slot, account_info) = &slot_list[index];
1950                                    if account_info.carats == 0 {
1951                                        purges_zero_carats.insert(
1952                                            *pubkey,
1953                                            self.accounts_index
1954                                                .roots_and_ref_count(&locked_entry, max_clean_root),
1955                                        );
1956                                    }
1957                                    // Release the lock
1958                                    let slot = *slot;
1959                                    drop(locked_entry);
1960
1961                                    if self.accounts_index.is_uncleaned_root(slot) {
1962                                        // Assertion enforced by `accounts_index.get()`, the latest slot
1963                                        // will not be greater than the given `max_clean_root`
1964                                        if let Some(max_clean_root) = max_clean_root {
1965                                            assert!(slot <= max_clean_root);
1966                                        }
1967                                        purges_old_accounts.push(*pubkey);
1968                                    }
1969                                }
1970                                AccountIndexGetResult::NotFoundOnFork => {
1971                                    // This pubkey is in the index but not in a root slot, so clean
1972                                    // it up by adding it to the to-be-purged list.
1973                                    //
1974                                    // Also, this pubkey must have been touched by some slot since
1975                                    // it was in the dirty list, so we assume that the slot it was
1976                                    // touched in must be unrooted.
1977                                    purges_old_accounts.push(*pubkey);
1978                                }
1979                                AccountIndexGetResult::Missing(_lock) => {}
1980                            };
1981                        }
1982                        (purges_zero_carats, purges_old_accounts)
1983                    })
1984                    .reduce(
1985                        || (HashMap::new(), Vec::new()),
1986                        |mut m1, m2| {
1987                            // Collapse down the hashmaps/vecs into one.
1988                            m1.0.extend(m2.0);
1989                            m1.1.extend(m2.1);
1990                            m1
1991                        },
1992                    )
1993            };
1994            if is_startup {
1995                do_clean_scan()
1996            } else {
1997                self.thread_pool_clean.install(do_clean_scan)
1998            }
1999        };
2000        accounts_scan.stop();
2001
2002        let mut clean_old_rooted = Measure::start("clean_old_roots");
2003        let (purged_account_slots, removed_accounts) =
2004            self.clean_accounts_older_than_root(purges_old_accounts, max_clean_root);
2005
2006        if self.caching_enabled {
2007            self.do_reset_uncleaned_roots(max_clean_root);
2008        } else {
2009            self.do_reset_uncleaned_roots_v1(&mut candidates_v1, max_clean_root);
2010        }
2011        clean_old_rooted.stop();
2012
2013        let mut store_counts_time = Measure::start("store_counts");
2014
2015        // Calculate store counts as if everything was purged
2016        // Then purge if we can
2017        let mut store_counts: HashMap<AppendVecId, (usize, HashSet<Pubkey>)> = HashMap::new();
2018        for (key, (account_infos, ref_count)) in purges_zero_carats.iter_mut() {
2019            if purged_account_slots.contains_key(key) {
2020                *ref_count = self.accounts_index.ref_count_from_storage(key);
2021            }
2022            account_infos.retain(|(slot, account_info)| {
2023                let was_slot_purged = purged_account_slots
2024                    .get(key)
2025                    .map(|slots_removed| slots_removed.contains(slot))
2026                    .unwrap_or(false);
2027                if was_slot_purged {
2028                    // No need to look up the slot storage below if the entire
2029                    // slot was purged
2030                    return false;
2031                }
2032                // Check if this update in `slot` to the account with `key` was reclaimed earlier by
2033                // `clean_accounts_older_than_root()`
2034                let was_reclaimed = removed_accounts
2035                    .get(&account_info.store_id)
2036                    .map(|store_removed| store_removed.contains(&account_info.offset))
2037                    .unwrap_or(false);
2038                if was_reclaimed {
2039                    return false;
2040                }
2041                if let Some(store_count) = store_counts.get_mut(&account_info.store_id) {
2042                    store_count.0 -= 1;
2043                    store_count.1.insert(*key);
2044                } else {
2045                    let mut key_set = HashSet::new();
2046                    key_set.insert(*key);
2047                    assert!(
2048                        !account_info.is_cached(),
2049                        "The Accounts Cache must be flushed first for this account info. pubkey: {}, slot: {}",
2050                        *key,
2051                        *slot
2052                    );
2053                    let count = self
2054                        .storage
2055                        .slot_store_count(*slot, account_info.store_id)
2056                        .unwrap()
2057                        - 1;
2058                    debug!(
2059                        "store_counts, inserting slot: {}, store id: {}, count: {}",
2060                        slot, account_info.store_id, count
2061                    );
2062                    store_counts.insert(account_info.store_id, (count, key_set));
2063                }
2064                true
2065            });
2066        }
2067        store_counts_time.stop();
2068
2069        let mut calc_deps_time = Measure::start("calc_deps");
2070        Self::calc_delete_dependencies(&purges_zero_carats, &mut store_counts);
2071        calc_deps_time.stop();
2072
2073        let mut purge_filter = Measure::start("purge_filter");
2074        self.filter_zero_carat_clean_for_incremental_snapshots(
2075            max_clean_root,
2076            last_full_snapshot_slot,
2077            &store_counts,
2078            &mut purges_zero_carats,
2079        );
2080        purge_filter.stop();
2081
2082        let mut reclaims_time = Measure::start("reclaims");
2083        // Recalculate reclaims with new purge set
2084        let pubkey_to_slot_set: Vec<_> = purges_zero_carats
2085            .into_iter()
2086            .map(|(key, (slots_list, _ref_count))| {
2087                (
2088                    key,
2089                    slots_list
2090                        .into_iter()
2091                        .map(|(slot, _)| slot)
2092                        .collect::<HashSet<Slot>>(),
2093                )
2094            })
2095            .collect();
2096
2097        let reclaims = self.purge_keys_exact(pubkey_to_slot_set.iter());
2098
2099        // Don't reset from clean, since the pubkeys in those stores may need to be unref'ed
2100        // and those stores may be used for background hashing.
2101        let reset_accounts = false;
2102        let mut reclaim_result = ReclaimResult::default();
2103        let reclaim_result = Some(&mut reclaim_result);
2104        self.handle_reclaims(
2105            &reclaims,
2106            None,
2107            Some(&self.clean_accounts_stats.purge_stats),
2108            reclaim_result,
2109            reset_accounts,
2110        );
2111
2112        reclaims_time.stop();
2113
2114        self.clean_accounts_stats.report();
2115        datapoint_info!(
2116            "clean_accounts",
2117            (
2118                "collect_delta_keys_us",
2119                key_timings.collect_delta_keys_us,
2120                i64
2121            ),
2122            (
2123                "dirty_store_processing_us",
2124                key_timings.dirty_store_processing_us,
2125                i64
2126            ),
2127            ("accounts_scan", accounts_scan.as_us() as i64, i64),
2128            ("clean_old_rooted", clean_old_rooted.as_us() as i64, i64),
2129            ("store_counts", store_counts_time.as_us() as i64, i64),
2130            ("purge_filter", purge_filter.as_us() as i64, i64),
2131            ("calc_deps", calc_deps_time.as_us() as i64, i64),
2132            ("reclaims", reclaims_time.as_us() as i64, i64),
2133            ("delta_key_count", key_timings.delta_key_count, i64),
2134            ("dirty_pubkeys_count", key_timings.dirty_pubkeys_count, i64),
2135            ("total_keys_count", total_keys_count, i64),
2136        );
2137    }
2138
2139    /// Removes the accounts in the input `reclaims` from the tracked "count" of
2140    /// their corresponding  storage entries. Note this does not actually free
2141    /// the memory from the storage entries until all the storage entries for
2142    /// a given slot `S` are empty, at which point `process_dead_slots` will
2143    /// remove all the storage entries for `S`.
2144    ///
2145    /// # Arguments
2146    /// * `reclaims` - The accounts to remove from storage entries' "count". Note here
2147    ///    that we should not remove cache entries, only entries for accounts actually
2148    ///    stored in a storage entry.
2149    ///
2150    /// * `expected_single_dead_slot` - A correctness assertion. If this is equal to `Some(S)`,
2151    ///    then the function will check that the only slot being cleaned up in `reclaims`
2152    ///    is the slot == `S`. This is true for instance when `handle_reclaims` is called
2153    ///    from store or slot shrinking, as those should only touch the slot they are
2154    ///    currently storing to or shrinking.
2155    ///
2156    /// * `purge_stats` - The stats used to track performance of purging dead slots. This
2157    ///    also serves a correctness assertion. If `purge_stats.is_none()`, this implies
2158    ///    there can be no dead slots that happen as a result of this call, and the function
2159    ///    will check that no slots are cleaned up/removed via `process_dead_slots`. For instance,
2160    ///    on store, no slots should be cleaned up, but during the background clean accounts
2161    ///    purges accounts from old rooted slots, so outdated slots may be removed.
2162    ///
2163    /// * `reclaim_result` - Information about accounts that were removed from storage, does
2164    ///    not include accounts that were removed from the cache
2165    ///
2166    /// * `reset_accounts` - Reset the append_vec store when the store is dead (count==0)
2167    ///    From the clean and shrink paths it should be false since there may be an in-progress
2168    ///    hash operation and the stores may hold accounts that need to be unref'ed.
2169    fn handle_reclaims(
2170        &self,
2171        reclaims: SlotSlice<AccountInfo>,
2172        expected_single_dead_slot: Option<Slot>,
2173        // TODO: coalesce `purge_stats` and `reclaim_result` together into one option, as they
2174        // are both either Some or None
2175        purge_stats: Option<&PurgeStats>,
2176        reclaim_result: Option<&mut ReclaimResult>,
2177        reset_accounts: bool,
2178    ) {
2179        if reclaims.is_empty() {
2180            return;
2181        }
2182        let (purged_account_slots, reclaimed_offsets) =
2183            if let Some((ref mut x, ref mut y)) = reclaim_result {
2184                (Some(x), Some(y))
2185            } else {
2186                (None, None)
2187            };
2188        let dead_slots = self.remove_dead_accounts(
2189            reclaims,
2190            expected_single_dead_slot,
2191            reclaimed_offsets,
2192            reset_accounts,
2193        );
2194        if purge_stats.is_none() {
2195            assert!(dead_slots.is_empty());
2196        } else if let Some(expected_single_dead_slot) = expected_single_dead_slot {
2197            assert!(dead_slots.len() <= 1);
2198            if dead_slots.len() == 1 {
2199                assert!(dead_slots.contains(&expected_single_dead_slot));
2200            }
2201        }
2202
2203        if let Some(purge_stats) = purge_stats {
2204            self.process_dead_slots(&dead_slots, purged_account_slots, purge_stats);
2205        }
2206    }
2207
2208    /// During clean, some zero-carat accounts that are marked for purge should *not* actually
2209    /// get purged.  Filter out those accounts here.
2210    ///
2211    /// When using incremental snapshots, do not purge zero-carat accounts if the slot is higher
2212    /// than the last full snapshot slot.  This is to protect against the following scenario:
2213    ///
2214    ///   ```text
2215    ///   A full snapshot is taken, and it contains an account with a non-zero balance.  Later,
2216    ///   that account's  goes to zero.  Evntually cleaning runs, and before, this account would be
2217    ///   cleaned up.  Finally, an incremental snapshot is taken.
2218    ///
2219    ///   Later, the incremental (and full) snapshot is used to rebuild the bank and accounts
2220    ///   database (e.x. if the node restarts).  The full snapshot _does_ contain the account (from
2221    ///   above) and its balance is non-zero, however, since the account was cleaned up in a later
2222    ///   slot, the incremental snapshot does not contain any info about this account, thus, the
2223    ///   accounts database will contain the old info from this account, which has its old non-zero
2224    ///   balance.  Very bad!
2225    ///   ```
2226    ///
2227    /// This filtering step can be skipped if there is no `last_full_snapshot_slot`, or if the
2228    /// `max_clean_root` is less-than-or-equal-to the `last_full_snapshot_slot`.
2229    fn filter_zero_carat_clean_for_incremental_snapshots(
2230        &self,
2231        max_clean_root: Option<Slot>,
2232        last_full_snapshot_slot: Option<Slot>,
2233        store_counts: &HashMap<AppendVecId, (usize, HashSet<Pubkey>)>,
2234        purges_zero_carats: &mut HashMap<Pubkey, (SlotList<AccountInfo>, RefCount)>,
2235    ) {
2236        let should_filter_for_incremental_snapshots =
2237            max_clean_root.unwrap_or(Slot::MAX) > last_full_snapshot_slot.unwrap_or(Slot::MAX);
2238        assert!(
2239            last_full_snapshot_slot.is_some() || !should_filter_for_incremental_snapshots,
2240            "if filtering for incremental snapshots, then snapshots should be enabled",
2241        );
2242
2243        purges_zero_carats.retain(|pubkey, (slot_account_infos, _ref_count)| {
2244            // Only keep purges_zero_carats where the entire history of the account in the root set
2245            // can be purged. All AppendVecs for those updates are dead.
2246            for (_slot, account_info) in slot_account_infos.iter() {
2247                if store_counts.get(&account_info.store_id).unwrap().0 != 0 {
2248                    return false;
2249                }
2250            }
2251
2252            // Exit early if not filtering more for incremental snapshots
2253            if !should_filter_for_incremental_snapshots {
2254                return true;
2255            }
2256
2257            let slot_account_info_at_highest_slot = slot_account_infos
2258                .iter()
2259                .max_by_key(|(slot, _account_info)| slot);
2260
2261            slot_account_info_at_highest_slot.map_or(true, |(slot, account_info)| {
2262                // Do *not* purge zero-carat accounts if the slot is greater than the last full
2263                // snapshot slot.  Since we're `retain`ing the accounts-to-purge, I felt creating
2264                // the `cannot_purge` variable made this easier to understand.  Accounts that do
2265                // not get purged here are added to a list so they be considered for purging later
2266                // (i.e. after the next full snapshot).
2267                assert!(account_info.is_zero_carat());
2268                let cannot_purge = *slot > last_full_snapshot_slot.unwrap();
2269                if cannot_purge {
2270                    self.zero_carat_accounts_to_purge_after_full_snapshot
2271                        .insert((*slot, *pubkey));
2272                }
2273                !cannot_purge
2274            })
2275        });
2276    }
2277
2278    // Must be kept private!, does sensitive cleanup that should only be called from
2279    // supported pipelines in AccountsDb
2280    fn process_dead_slots(
2281        &self,
2282        dead_slots: &HashSet<Slot>,
2283        purged_account_slots: Option<&mut AccountSlots>,
2284        purge_stats: &PurgeStats,
2285    ) {
2286        if dead_slots.is_empty() {
2287            return;
2288        }
2289        let mut clean_dead_slots = Measure::start("reclaims::clean_dead_slots");
2290        self.clean_stored_dead_slots(dead_slots, purged_account_slots);
2291        clean_dead_slots.stop();
2292
2293        let mut purge_removed_slots = Measure::start("reclaims::purge_removed_slots");
2294        self.purge_dead_slots_from_storage(dead_slots.iter(), purge_stats);
2295        purge_removed_slots.stop();
2296
2297        // If the slot is dead, remove the need to shrink the storages as
2298        // the storage entries will be purged.
2299        {
2300            let mut list = self.shrink_candidate_slots.lock().unwrap();
2301            for slot in dead_slots {
2302                list.remove(slot);
2303            }
2304        }
2305
2306        debug!(
2307            "process_dead_slots({}): {} {} {:?}",
2308            dead_slots.len(),
2309            clean_dead_slots,
2310            purge_removed_slots,
2311            dead_slots,
2312        );
2313    }
2314
2315    fn load_accounts_index_for_shrink<'a, I>(
2316        &'a self,
2317        iter: I,
2318        alive_accounts: &mut Vec<(&'a Pubkey, &'a FoundStoredAccount<'a>)>,
2319        unrefed_pubkeys: &mut Vec<&'a Pubkey>,
2320    ) -> usize
2321    where
2322        I: Iterator<Item = (&'a Pubkey, &'a FoundStoredAccount<'a>)>,
2323    {
2324        let mut alive_total = 0;
2325
2326        let mut alive = 0;
2327        let mut dead = 0;
2328        iter.for_each(|(pubkey, stored_account)| {
2329            let lookup = self.accounts_index.get_account_read_entry(pubkey);
2330            if let Some(locked_entry) = lookup {
2331                let is_alive = locked_entry.slot_list().iter().any(|(_slot, i)| {
2332                    i.store_id == stored_account.store_id
2333                        && i.offset == stored_account.account.offset
2334                });
2335                if !is_alive {
2336                    // This pubkey was found in the storage, but no longer exists in the index.
2337                    // It would have had a ref to the storage from the initial store, but it will
2338                    // not exist in the re-written slot. Unref it to keep the index consistent with
2339                    // rewriting the storage entries.
2340                    unrefed_pubkeys.push(pubkey);
2341                    locked_entry.unref();
2342                    dead += 1;
2343                } else {
2344                    alive_accounts.push((pubkey, stored_account));
2345                    alive_total += stored_account.account_size;
2346                    alive += 1;
2347                }
2348            }
2349        });
2350        self.shrink_stats
2351            .alive_accounts
2352            .fetch_add(alive, Ordering::Relaxed);
2353        self.shrink_stats
2354            .dead_accounts
2355            .fetch_add(dead, Ordering::Relaxed);
2356
2357        alive_total
2358    }
2359
2360    fn do_shrink_slot_stores<'a, I>(&'a self, slot: Slot, stores: I) -> usize
2361    where
2362        I: Iterator<Item = &'a Arc<AccountStorageEntry>>,
2363    {
2364        debug!("do_shrink_slot_stores: slot: {}", slot);
2365        let mut stored_accounts: HashMap<Pubkey, FoundStoredAccount> = HashMap::new();
2366        let mut original_bytes = 0;
2367        let mut num_stores = 0;
2368        for store in stores {
2369            let mut start = 0;
2370            original_bytes += store.total_bytes();
2371            while let Some((account, next)) = store.accounts.get_account(start) {
2372                let new_entry = FoundStoredAccount {
2373                    account,
2374                    store_id: store.append_vec_id(),
2375                    account_size: next - start,
2376                };
2377                match stored_accounts.entry(new_entry.account.meta.pubkey) {
2378                    Entry::Occupied(mut occupied_entry) => {
2379                        if new_entry.account.meta.write_version
2380                            > occupied_entry.get().account.meta.write_version
2381                        {
2382                            occupied_entry.insert(new_entry);
2383                        }
2384                    }
2385                    Entry::Vacant(vacant_entry) => {
2386                        vacant_entry.insert(new_entry);
2387                    }
2388                }
2389                start = next;
2390            }
2391            num_stores += 1;
2392        }
2393
2394        let mut index_read_elapsed = Measure::start("index_read_elapsed");
2395        let alive_total_collect = AtomicUsize::new(0);
2396
2397        let len = stored_accounts.len();
2398        let alive_accounts_collect = Mutex::new(Vec::with_capacity(len));
2399        let unrefed_pubkeys_collect = Mutex::new(Vec::with_capacity(len));
2400        self.shrink_stats
2401            .accounts_loaded
2402            .fetch_add(len as u64, Ordering::Relaxed);
2403
2404        self.thread_pool.install(|| {
2405            let chunk_size = 50; // # accounts/thread
2406            let chunks = len / chunk_size + 1;
2407            (0..chunks).into_par_iter().for_each(|chunk| {
2408                let skip = chunk * chunk_size;
2409
2410                let mut alive_accounts = Vec::with_capacity(chunk_size);
2411                let mut unrefed_pubkeys = Vec::with_capacity(chunk_size);
2412                let alive_total = self.load_accounts_index_for_shrink(
2413                    stored_accounts.iter().skip(skip).take(chunk_size),
2414                    &mut alive_accounts,
2415                    &mut unrefed_pubkeys,
2416                );
2417
2418                // collect
2419                alive_accounts_collect
2420                    .lock()
2421                    .unwrap()
2422                    .append(&mut alive_accounts);
2423                unrefed_pubkeys_collect
2424                    .lock()
2425                    .unwrap()
2426                    .append(&mut unrefed_pubkeys);
2427                alive_total_collect.fetch_add(alive_total, Ordering::Relaxed);
2428            });
2429        });
2430
2431        let alive_accounts = alive_accounts_collect.into_inner().unwrap();
2432        let unrefed_pubkeys = unrefed_pubkeys_collect.into_inner().unwrap();
2433        let alive_total = alive_total_collect.load(Ordering::Relaxed);
2434
2435        index_read_elapsed.stop();
2436        let aligned_total: u64 = Self::page_align(alive_total as u64);
2437
2438        // This shouldn't happen if alive_bytes/approx_stored_count are accurate
2439        if Self::should_not_shrink(aligned_total, original_bytes, num_stores) {
2440            self.shrink_stats
2441                .skipped_shrink
2442                .fetch_add(1, Ordering::Relaxed);
2443            for pubkey in unrefed_pubkeys {
2444                if let Some(locked_entry) = self.accounts_index.get_account_read_entry(pubkey) {
2445                    locked_entry.addref();
2446                }
2447            }
2448            return 0;
2449        }
2450
2451        let total_starting_accounts = stored_accounts.len();
2452        let total_accounts_after_shrink = alive_accounts.len();
2453        debug!(
2454            "shrinking: slot: {}, accounts: ({} => {}) bytes: ({} ; aligned to: {}) original: {}",
2455            slot,
2456            total_starting_accounts,
2457            total_accounts_after_shrink,
2458            alive_total,
2459            aligned_total,
2460            original_bytes,
2461        );
2462
2463        let mut rewrite_elapsed = Measure::start("rewrite_elapsed");
2464        let mut dead_storages = vec![];
2465        let mut find_alive_elapsed = 0;
2466        let mut create_and_insert_store_elapsed = 0;
2467        let mut write_storage_elapsed = 0;
2468        let mut store_accounts_timing = StoreAccountsTiming::default();
2469        if aligned_total > 0 {
2470            let mut start = Measure::start("find_alive_elapsed");
2471            let mut accounts = Vec::with_capacity(alive_accounts.len());
2472            let mut hashes = Vec::with_capacity(alive_accounts.len());
2473            let mut write_versions = Vec::with_capacity(alive_accounts.len());
2474
2475            for (pubkey, alive_account) in alive_accounts {
2476                accounts.push((pubkey, &alive_account.account));
2477                hashes.push(alive_account.account.hash);
2478                write_versions.push(alive_account.account.meta.write_version);
2479            }
2480            start.stop();
2481            find_alive_elapsed = start.as_us();
2482
2483            let mut start = Measure::start("create_and_insert_store_elapsed");
2484            let shrunken_store = if let Some(new_store) =
2485                self.try_recycle_and_insert_store(slot, aligned_total, aligned_total + 1024)
2486            {
2487                new_store
2488            } else {
2489                let maybe_shrink_paths = self.shrink_paths.read().unwrap();
2490                if let Some(ref shrink_paths) = *maybe_shrink_paths {
2491                    self.create_and_insert_store_with_paths(
2492                        slot,
2493                        aligned_total,
2494                        "shrink-w-path",
2495                        shrink_paths,
2496                    )
2497                } else {
2498                    self.create_and_insert_store(slot, aligned_total, "shrink")
2499                }
2500            };
2501            start.stop();
2502            create_and_insert_store_elapsed = start.as_us();
2503
2504            // here, we're writing back alive_accounts. That should be an atomic operation
2505            // without use of rather wide locks in this whole function, because we're
2506            // mutating rooted slots; There should be no writers to them.
2507            store_accounts_timing = self.store_accounts_frozen(
2508                slot,
2509                &accounts,
2510                Some(&hashes),
2511                Some(Box::new(move |_, _| shrunken_store.clone())),
2512                Some(Box::new(write_versions.into_iter())),
2513            );
2514
2515            // `store_accounts_frozen()` above may have purged accounts from some
2516            // other storage entries (the ones that were just overwritten by this
2517            // new storage entry). This means some of those stores might have caused
2518            // this slot to be read to `self.shrink_candidate_slots`, so delete
2519            // those here
2520            self.shrink_candidate_slots.lock().unwrap().remove(&slot);
2521
2522            // Purge old, overwritten storage entries
2523            let mut start = Measure::start("write_storage_elapsed");
2524            if let Some(slot_stores) = self.storage.get_slot_stores(slot) {
2525                slot_stores.write().unwrap().retain(|_key, store| {
2526                    if store.count() == 0 {
2527                        self.dirty_stores
2528                            .insert((slot, store.append_vec_id()), store.clone());
2529                        dead_storages.push(store.clone());
2530                        false
2531                    } else {
2532                        true
2533                    }
2534                });
2535            }
2536            start.stop();
2537            write_storage_elapsed = start.as_us();
2538        }
2539        rewrite_elapsed.stop();
2540
2541        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
2542        let mut recycle_stores = self.recycle_stores.write().unwrap();
2543        recycle_stores_write_elapsed.stop();
2544
2545        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
2546        if recycle_stores.entry_count() < MAX_RECYCLE_STORES {
2547            recycle_stores.add_entries(dead_storages);
2548            drop(recycle_stores);
2549        } else {
2550            self.stats
2551                .dropped_stores
2552                .fetch_add(dead_storages.len() as u64, Ordering::Relaxed);
2553            drop(recycle_stores);
2554            drop(dead_storages);
2555        }
2556        drop_storage_entries_elapsed.stop();
2557
2558        self.shrink_stats
2559            .num_slots_shrunk
2560            .fetch_add(1, Ordering::Relaxed);
2561        self.shrink_stats
2562            .index_read_elapsed
2563            .fetch_add(index_read_elapsed.as_us(), Ordering::Relaxed);
2564        self.shrink_stats
2565            .find_alive_elapsed
2566            .fetch_add(find_alive_elapsed, Ordering::Relaxed);
2567        self.shrink_stats
2568            .create_and_insert_store_elapsed
2569            .fetch_add(create_and_insert_store_elapsed, Ordering::Relaxed);
2570        self.shrink_stats.store_accounts_elapsed.fetch_add(
2571            store_accounts_timing.store_accounts_elapsed,
2572            Ordering::Relaxed,
2573        );
2574        self.shrink_stats.update_index_elapsed.fetch_add(
2575            store_accounts_timing.update_index_elapsed,
2576            Ordering::Relaxed,
2577        );
2578        self.shrink_stats.handle_reclaims_elapsed.fetch_add(
2579            store_accounts_timing.handle_reclaims_elapsed,
2580            Ordering::Relaxed,
2581        );
2582        self.shrink_stats
2583            .write_storage_elapsed
2584            .fetch_add(write_storage_elapsed, Ordering::Relaxed);
2585        self.shrink_stats
2586            .rewrite_elapsed
2587            .fetch_add(rewrite_elapsed.as_us(), Ordering::Relaxed);
2588        self.shrink_stats
2589            .drop_storage_entries_elapsed
2590            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
2591        self.shrink_stats
2592            .recycle_stores_write_elapsed
2593            .fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
2594        self.shrink_stats.accounts_removed.fetch_add(
2595            total_starting_accounts - total_accounts_after_shrink,
2596            Ordering::Relaxed,
2597        );
2598        self.shrink_stats.bytes_removed.fetch_add(
2599            original_bytes.saturating_sub(aligned_total),
2600            Ordering::Relaxed,
2601        );
2602        self.shrink_stats
2603            .bytes_written
2604            .fetch_add(aligned_total, Ordering::Relaxed);
2605
2606        self.shrink_stats.report();
2607
2608        total_accounts_after_shrink
2609    }
2610
2611    // Reads all accounts in given slot's AppendVecs and filter only to alive,
2612    // then create a minimum AppendVec filled with the alive.
2613    fn shrink_slot_forced(&self, slot: Slot) -> usize {
2614        debug!("shrink_slot_forced: slot: {}", slot);
2615
2616        if let Some(stores_lock) = self.storage.get_slot_stores(slot) {
2617            let stores: Vec<Arc<AccountStorageEntry>> =
2618                stores_lock.read().unwrap().values().cloned().collect();
2619            if !Self::is_shrinking_productive(slot, &stores) {
2620                return 0;
2621            }
2622            self.do_shrink_slot_stores(slot, stores.iter())
2623        } else {
2624            0
2625        }
2626    }
2627
2628    fn all_slots_in_storage(&self) -> Vec<Slot> {
2629        self.storage.all_slots()
2630    }
2631
2632    fn all_root_slots_in_index(&self) -> Vec<Slot> {
2633        self.accounts_index.all_roots()
2634    }
2635
2636    /// Given the input `ShrinkCandidates`, this function sorts the stores by their alive ratio
2637    /// in increasing order with the most sparse entries in the front. It will then simulate the
2638    /// shrinking by working on the most sparse entries first and if the overall alive ratio is
2639    /// achieved, it will stop and return the filtered-down candidates and the candidates which
2640    /// are skipped in this round and might be eligible for the future shrink.
2641    fn select_candidates_by_total_usage(
2642        &self,
2643        shrink_slots: &ShrinkCandidates,
2644        shrink_ratio: f64,
2645    ) -> (ShrinkCandidates, ShrinkCandidates) {
2646        struct StoreUsageInfo {
2647            slot: Slot,
2648            alive_ratio: f64,
2649            store: Arc<AccountStorageEntry>,
2650        }
2651        let mut measure = Measure::start("select_top_sparse_storage_entries-ms");
2652        let mut store_usage: Vec<StoreUsageInfo> = Vec::with_capacity(shrink_slots.len());
2653        let mut total_alive_bytes: u64 = 0;
2654        let mut candidates_count: usize = 0;
2655        let mut total_bytes: u64 = 0;
2656        let mut total_candidate_stores: usize = 0;
2657        for (slot, slot_shrink_candidates) in shrink_slots {
2658            candidates_count += slot_shrink_candidates.len();
2659            for store in slot_shrink_candidates.values() {
2660                total_alive_bytes += Self::page_align(store.alive_bytes() as u64);
2661                total_bytes += store.total_bytes();
2662                let alive_ratio = Self::page_align(store.alive_bytes() as u64) as f64
2663                    / store.total_bytes() as f64;
2664                store_usage.push(StoreUsageInfo {
2665                    slot: *slot,
2666                    alive_ratio,
2667                    store: store.clone(),
2668                });
2669                total_candidate_stores += 1;
2670            }
2671        }
2672        store_usage.sort_by(|a, b| {
2673            a.alive_ratio
2674                .partial_cmp(&b.alive_ratio)
2675                .unwrap_or(std::cmp::Ordering::Equal)
2676        });
2677
2678        // Working from the beginning of store_usage which are the most sparse and see when we can stop
2679        // shrinking while still achieving the overall goals.
2680        let mut shrink_slots: ShrinkCandidates = HashMap::new();
2681        let mut shrink_slots_next_batch: ShrinkCandidates = HashMap::new();
2682        for usage in &store_usage {
2683            let store = &usage.store;
2684            let alive_ratio = (total_alive_bytes as f64) / (total_bytes as f64);
2685            debug!("alive_ratio: {:?} store_id: {:?}, store_ratio: {:?} requirment: {:?}, total_bytes: {:?} total_alive_bytes: {:?}",
2686                alive_ratio, usage.store.append_vec_id(), usage.alive_ratio, shrink_ratio, total_bytes, total_alive_bytes);
2687            if alive_ratio > shrink_ratio {
2688                // we have reached our goal, stop
2689                debug!(
2690                    "Shrinking goal can be achieved at slot {:?}, total_alive_bytes: {:?} \
2691                    total_bytes: {:?}, alive_ratio: {:}, shrink_ratio: {:?}",
2692                    usage.slot, total_alive_bytes, total_bytes, alive_ratio, shrink_ratio
2693                );
2694                if usage.alive_ratio < shrink_ratio {
2695                    shrink_slots_next_batch
2696                        .entry(usage.slot)
2697                        .or_default()
2698                        .insert(store.append_vec_id(), store.clone());
2699                } else {
2700                    break;
2701                }
2702            } else {
2703                let current_store_size = store.total_bytes();
2704                let after_shrink_size = Self::page_align(store.alive_bytes() as u64);
2705                let bytes_saved = current_store_size.saturating_sub(after_shrink_size);
2706                total_bytes -= bytes_saved;
2707                shrink_slots
2708                    .entry(usage.slot)
2709                    .or_default()
2710                    .insert(store.append_vec_id(), store.clone());
2711            }
2712        }
2713        measure.stop();
2714        inc_new_counter_info!(
2715            "shrink_select_top_sparse_storage_entries-ms",
2716            measure.as_ms() as usize
2717        );
2718        inc_new_counter_info!(
2719            "shrink_select_top_sparse_storage_entries-seeds",
2720            candidates_count
2721        );
2722        inc_new_counter_info!(
2723            "shrink_total_preliminary_candidate_stores",
2724            total_candidate_stores
2725        );
2726
2727        (shrink_slots, shrink_slots_next_batch)
2728    }
2729
2730    pub fn shrink_candidate_slots(&self) -> usize {
2731        let shrink_candidates_slots =
2732            std::mem::take(&mut *self.shrink_candidate_slots.lock().unwrap());
2733        let (shrink_slots, shrink_slots_next_batch) = {
2734            if let AccountShrinkThreshold::TotalSpace { shrink_ratio } = self.shrink_ratio {
2735                let (shrink_slots, shrink_slots_next_batch) =
2736                    self.select_candidates_by_total_usage(&shrink_candidates_slots, shrink_ratio);
2737                (shrink_slots, Some(shrink_slots_next_batch))
2738            } else {
2739                (shrink_candidates_slots, None)
2740            }
2741        };
2742
2743        let mut measure_shrink_all_candidates = Measure::start("shrink_all_candidate_slots-ms");
2744        let num_candidates = shrink_slots.len();
2745        let shrink_candidates_count: usize = self.thread_pool.install(|| {
2746            shrink_slots
2747                .into_par_iter()
2748                .map(|(slot, slot_shrink_candidates)| {
2749                    let mut measure = Measure::start("shrink_candidate_slots-ms");
2750                    self.do_shrink_slot_stores(slot, slot_shrink_candidates.values());
2751                    measure.stop();
2752                    inc_new_counter_info!("shrink_candidate_slots-ms", measure.as_ms() as usize);
2753                    slot_shrink_candidates.len()
2754                })
2755                .sum()
2756        });
2757        measure_shrink_all_candidates.stop();
2758        inc_new_counter_info!(
2759            "shrink_all_candidate_slots-ms",
2760            measure_shrink_all_candidates.as_ms() as usize
2761        );
2762        inc_new_counter_info!("shrink_all_candidate_slots-count", shrink_candidates_count);
2763        let mut pended_counts: usize = 0;
2764        if let Some(shrink_slots_next_batch) = shrink_slots_next_batch {
2765            let mut shrink_slots = self.shrink_candidate_slots.lock().unwrap();
2766            for (slot, stores) in shrink_slots_next_batch {
2767                pended_counts += stores.len();
2768                shrink_slots.entry(slot).or_default().extend(stores);
2769            }
2770        }
2771        inc_new_counter_info!("shrink_pended_stores-count", pended_counts);
2772
2773        num_candidates
2774    }
2775
2776    pub fn shrink_all_slots(&self, is_startup: bool, last_full_snapshot_slot: Option<Slot>) {
2777        const DIRTY_STORES_CLEANING_THRESHOLD: usize = 10_000;
2778        const OUTER_CHUNK_SIZE: usize = 2000;
2779        const INNER_CHUNK_SIZE: usize = OUTER_CHUNK_SIZE / 8;
2780        if is_startup && self.caching_enabled {
2781            let slots = self.all_slots_in_storage();
2782            let inner_chunk_size = std::cmp::max(INNER_CHUNK_SIZE, 1);
2783            slots.chunks(OUTER_CHUNK_SIZE).for_each(|chunk| {
2784                chunk.par_chunks(inner_chunk_size).for_each(|slots| {
2785                    for slot in slots {
2786                        self.shrink_slot_forced(*slot);
2787                    }
2788                });
2789                if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
2790                    self.clean_accounts(None, is_startup, last_full_snapshot_slot);
2791                }
2792            });
2793        } else {
2794            for slot in self.all_slots_in_storage() {
2795                if self.caching_enabled {
2796                    self.shrink_slot_forced(slot);
2797                } else {
2798                    self.do_shrink_slot_forced_v1(slot);
2799                }
2800                if self.dirty_stores.len() > DIRTY_STORES_CLEANING_THRESHOLD {
2801                    self.clean_accounts(None, is_startup, last_full_snapshot_slot);
2802                }
2803            }
2804        }
2805    }
2806
2807    pub fn scan_accounts<F, A>(
2808        &self,
2809        ancestors: &Ancestors,
2810        bank_id: BankId,
2811        scan_func: F,
2812    ) -> ScanResult<A>
2813    where
2814        F: Fn(&mut A, Option<(&Pubkey, AccountSharedData, Slot)>),
2815        A: Default,
2816    {
2817        let mut collector = A::default();
2818
2819        // This can error out if the slots being scanned over are aborted
2820        self.accounts_index
2821            .scan_accounts(ancestors, bank_id, |pubkey, (account_info, slot)| {
2822                let account_slot = self
2823                    .get_account_accessor(slot, pubkey, account_info.store_id, account_info.offset)
2824                    .get_loaded_account()
2825                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot));
2826                scan_func(&mut collector, account_slot)
2827            })?;
2828
2829        Ok(collector)
2830    }
2831
2832    pub fn unchecked_scan_accounts<F, A>(
2833        &self,
2834        metric_name: &'static str,
2835        ancestors: &Ancestors,
2836        scan_func: F,
2837        collect_all_unsorted: bool,
2838    ) -> A
2839    where
2840        F: Fn(&mut A, (&Pubkey, LoadedAccount, Slot)),
2841        A: Default,
2842    {
2843        let mut collector = A::default();
2844        self.accounts_index.unchecked_scan_accounts(
2845            metric_name,
2846            ancestors,
2847            |pubkey, (account_info, slot)| {
2848                if let Some(loaded_account) = self
2849                    .get_account_accessor(slot, pubkey, account_info.store_id, account_info.offset)
2850                    .get_loaded_account()
2851                {
2852                    scan_func(&mut collector, (pubkey, loaded_account, slot));
2853                }
2854            },
2855            collect_all_unsorted,
2856        );
2857        collector
2858    }
2859
2860    pub fn range_scan_accounts<F, A, R>(
2861        &self,
2862        metric_name: &'static str,
2863        ancestors: &Ancestors,
2864        range: R,
2865        collect_all_unsorted: bool,
2866        scan_func: F,
2867    ) -> A
2868    where
2869        F: Fn(&mut A, Option<(&Pubkey, AccountSharedData, Slot)>),
2870        A: Default,
2871        R: RangeBounds<Pubkey> + std::fmt::Debug,
2872    {
2873        let mut collector = A::default();
2874        self.accounts_index.range_scan_accounts(
2875            metric_name,
2876            ancestors,
2877            range,
2878            collect_all_unsorted,
2879            |pubkey, (account_info, slot)| {
2880                // unlike other scan fns, this is called from Bank::collect_rent_eagerly(),
2881                // which is on-consensus processing in the banking/replaying stage.
2882                // This requires infallible and consistent account loading.
2883                // So, we unwrap Option<LoadedAccount> from get_loaded_account() here.
2884                // This is safe because this closure is invoked with the account_info,
2885                // while we lock the index entry at AccountsIndex::do_scan_accounts() ultimately,
2886                // meaning no other subsystems can invalidate the account_info before making their
2887                // changes to the index entry.
2888                // For details, see the comment in retry_to_get_account_accessor()
2889                let account_slot = self
2890                    .get_account_accessor(slot, pubkey, account_info.store_id, account_info.offset)
2891                    .get_loaded_account()
2892                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot))
2893                    .unwrap();
2894                scan_func(&mut collector, Some(account_slot))
2895            },
2896        );
2897        collector
2898    }
2899
2900    pub fn index_scan_accounts<F, A>(
2901        &self,
2902        ancestors: &Ancestors,
2903        bank_id: BankId,
2904        index_key: IndexKey,
2905        scan_func: F,
2906    ) -> ScanResult<(A, bool)>
2907    where
2908        F: Fn(&mut A, Option<(&Pubkey, AccountSharedData, Slot)>),
2909        A: Default,
2910    {
2911        let key = match &index_key {
2912            IndexKey::ProgramId(key) => key,
2913            IndexKey::SplTokenMint(key) => key,
2914            IndexKey::SplTokenOwner(key) => key,
2915        };
2916        if !self.account_indexes.include_key(key) {
2917            // the requested key was not indexed in the secondary index, so do a normal scan
2918            let used_index = false;
2919            let scan_result = self.scan_accounts(ancestors, bank_id, scan_func)?;
2920            return Ok((scan_result, used_index));
2921        }
2922
2923        let mut collector = A::default();
2924        self.accounts_index.index_scan_accounts(
2925            ancestors,
2926            bank_id,
2927            index_key,
2928            |pubkey, (account_info, slot)| {
2929                let account_slot = self
2930                    .get_account_accessor(slot, pubkey, account_info.store_id, account_info.offset)
2931                    .get_loaded_account()
2932                    .map(|loaded_account| (pubkey, loaded_account.take_account(), slot));
2933                scan_func(&mut collector, account_slot)
2934            },
2935        )?;
2936        let used_index = true;
2937        Ok((collector, used_index))
2938    }
2939
2940    /// Scan a specific slot through all the account storage in parallel
2941    pub fn scan_account_storage<R, B>(
2942        &self,
2943        slot: Slot,
2944        cache_map_func: impl Fn(LoadedAccount) -> Option<R> + Sync,
2945        storage_scan_func: impl Fn(&B, LoadedAccount) + Sync,
2946    ) -> ScanStorageResult<R, B>
2947    where
2948        R: Send,
2949        B: Send + Default + Sync,
2950    {
2951        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
2952            // If we see the slot in the cache, then all the account information
2953            // is in this cached slot
2954            if slot_cache.len() > SCAN_SLOT_PAR_ITER_THRESHOLD {
2955                ScanStorageResult::Cached(self.thread_pool.install(|| {
2956                    slot_cache
2957                        .par_iter()
2958                        .filter_map(|cached_account| {
2959                            cache_map_func(LoadedAccount::Cached((
2960                                *cached_account.key(),
2961                                Cow::Borrowed(cached_account.value()),
2962                            )))
2963                        })
2964                        .collect()
2965                }))
2966            } else {
2967                ScanStorageResult::Cached(
2968                    slot_cache
2969                        .iter()
2970                        .filter_map(|cached_account| {
2971                            cache_map_func(LoadedAccount::Cached((
2972                                *cached_account.key(),
2973                                Cow::Borrowed(cached_account.value()),
2974                            )))
2975                        })
2976                        .collect(),
2977                )
2978            }
2979        } else {
2980            let retval = B::default();
2981            // If the slot is not in the cache, then all the account information must have
2982            // been flushed. This is guaranteed because we only remove the rooted slot from
2983            // the cache *after* we've finished flushing in `flush_slot_cache`.
2984            let storage_maps: Vec<Arc<AccountStorageEntry>> = self
2985                .storage
2986                .get_slot_storage_entries(slot)
2987                .unwrap_or_default();
2988            self.thread_pool.install(|| {
2989                storage_maps
2990                    .par_iter()
2991                    .flat_map(|storage| storage.all_accounts())
2992                    .for_each(|account| storage_scan_func(&retval, LoadedAccount::Stored(account)));
2993            });
2994
2995            ScanStorageResult::Stored(retval)
2996        }
2997    }
2998
2999    pub fn set_hash(&self, slot: Slot, parent_slot: Slot) {
3000        let mut bank_hashes = self.bank_hashes.write().unwrap();
3001        if bank_hashes.get(&slot).is_some() {
3002            error!(
3003                "set_hash: already exists; multiple forks with shared slot {} as child (parent: {})!?",
3004                slot, parent_slot,
3005            );
3006            return;
3007        }
3008
3009        let new_hash_info = BankHashInfo {
3010            hash: Hash::default(),
3011            snapshot_hash: Hash::default(),
3012            stats: BankHashStats::default(),
3013        };
3014        bank_hashes.insert(slot, new_hash_info);
3015    }
3016
3017    pub fn load(
3018        &self,
3019        ancestors: &Ancestors,
3020        pubkey: &Pubkey,
3021        load_hint: LoadHint,
3022    ) -> Option<(AccountSharedData, Slot)> {
3023        self.do_load(ancestors, pubkey, None, load_hint)
3024    }
3025
3026    pub fn load_with_fixed_root(
3027        &self,
3028        ancestors: &Ancestors,
3029        pubkey: &Pubkey,
3030    ) -> Option<(AccountSharedData, Slot)> {
3031        self.load(ancestors, pubkey, LoadHint::FixedMaxRoot)
3032    }
3033
3034    pub fn load_without_fixed_root(
3035        &self,
3036        ancestors: &Ancestors,
3037        pubkey: &Pubkey,
3038    ) -> Option<(AccountSharedData, Slot)> {
3039        self.load(ancestors, pubkey, LoadHint::Unspecified)
3040    }
3041
3042    fn read_index_for_accessor_or_load_slow<'a>(
3043        &'a self,
3044        ancestors: &Ancestors,
3045        pubkey: &'a Pubkey,
3046        max_root: Option<Slot>,
3047        clone_in_lock: bool,
3048    ) -> Option<(Slot, AppendVecId, usize, Option<LoadedAccountAccessor<'a>>)> {
3049        let (lock, index) = match self.accounts_index.get(pubkey, Some(ancestors), max_root) {
3050            AccountIndexGetResult::Found(lock, index) => (lock, index),
3051            // we bail out pretty early for missing.
3052            AccountIndexGetResult::NotFoundOnFork => {
3053                return None;
3054            }
3055            AccountIndexGetResult::Missing(_) => {
3056                return None;
3057            }
3058        };
3059
3060        let slot_list = lock.slot_list();
3061        let (
3062            slot,
3063            AccountInfo {
3064                store_id, offset, ..
3065            },
3066        ) = slot_list[index];
3067
3068        let some_from_slow_path = if clone_in_lock {
3069            // the fast path must have failed.... so take the slower approach
3070            // of copying potentially large Account::data inside the lock.
3071
3072            // calling check_and_get_loaded_account is safe as long as we're guaranteed to hold
3073            // the lock during the time and there should be no purge thanks to alive ancestors
3074            // held by our caller.
3075            Some(self.get_account_accessor(slot, pubkey, store_id, offset))
3076        } else {
3077            None
3078        };
3079
3080        Some((slot, store_id, offset, some_from_slow_path))
3081        // `lock` is dropped here rather pretty quickly with clone_in_lock = false,
3082        // so the entry could be raced for mutation by other subsystems,
3083        // before we actually provision an account data for caller's use from now on.
3084        // This is traded for less contention and resultant performance, introducing fair amount of
3085        // delicate handling in retry_to_get_account_accessor() below ;)
3086        // you're warned!
3087    }
3088
3089    fn retry_to_get_account_accessor<'a>(
3090        &'a self,
3091        mut slot: Slot,
3092        mut store_id: usize,
3093        mut offset: usize,
3094        ancestors: &'a Ancestors,
3095        pubkey: &'a Pubkey,
3096        max_root: Option<Slot>,
3097        load_hint: LoadHint,
3098    ) -> Option<(LoadedAccountAccessor<'a>, Slot)> {
3099        // Happy drawing time! :)
3100        //
3101        // Reader                               | Accessed data source for cached/stored
3102        // -------------------------------------+----------------------------------
3103        // R1 read_index_for_accessor_or_load_slow()| cached/stored: index
3104        //          |                           |
3105        //        <(store_id, offset, ..)>      |
3106        //          V                           |
3107        // R2 retry_to_get_account_accessor()/  | cached: map of caches & entry for (slot, pubkey)
3108        //        get_account_accessor()        | stored: map of stores
3109        //          |                           |
3110        //        <Accessor>                    |
3111        //          V                           |
3112        // R3 check_and_get_loaded_account()/   | cached: N/A (note: basically noop unwrap)
3113        //        get_loaded_account()          | stored: store's entry for slot
3114        //          |                           |
3115        //        <LoadedAccount>               |
3116        //          V                           |
3117        // R4 take_account()                    | cached/stored: entry of cache/storage for (slot, pubkey)
3118        //          |                           |
3119        //        <AccountSharedData>           |
3120        //          V                           |
3121        //    Account!!                         V
3122        //
3123        // Flusher                              | Accessed data source for cached/stored
3124        // -------------------------------------+----------------------------------
3125        // F1 flush_slot_cache()                | N/A
3126        //          |                           |
3127        //          V                           |
3128        // F2 store_accounts_frozen()/          | map of stores (creates new entry)
3129        //        write_accounts_to_storage()   |
3130        //          |                           |
3131        //          V                           |
3132        // F3 store_accounts_frozen()/          | index
3133        //        update_index()                | (replaces existing store_id, offset in caches)
3134        //          |                           |
3135        //          V                           |
3136        // F4 accounts_cache.remove_slot()      | map of caches (removes old entry)
3137        //                                      V
3138        //
3139        // Remarks for flusher: So, for any reading operations, it's a race condition where F4 happens
3140        // between R1 and R2. In that case, retrying from R1 is safu because F3 should have
3141        // been occurred.
3142        //
3143        // Shrinker                             | Accessed data source for stored
3144        // -------------------------------------+----------------------------------
3145        // S1 do_shrink_slot_stores()           | N/A
3146        //          |                           |
3147        //          V                           |
3148        // S2 store_accounts_frozen()/          | map of stores (creates new entry)
3149        //        write_accounts_to_storage()   |
3150        //          |                           |
3151        //          V                           |
3152        // S3 store_accounts_frozen()/          | index
3153        //        update_index()                | (replaces existing store_id, offset in stores)
3154        //          |                           |
3155        //          V                           |
3156        // S4 do_shrink_slot_stores()/          | map of stores (removes old entry)
3157        //        dead_storages
3158        //
3159        // Remarks for shrinker: So, for any reading operations, it's a race condition
3160        // where S4 happens between R1 and R2. In that case, retrying from R1 is safu because S3 should have
3161        // been occurred, and S3 atomically replaced the index accordingly.
3162        //
3163        // Cleaner                              | Accessed data source for stored
3164        // -------------------------------------+----------------------------------
3165        // C1 clean_accounts()                  | N/A
3166        //          |                           |
3167        //          V                           |
3168        // C2 clean_accounts()/                 | index
3169        //        purge_keys_exact()            | (removes existing store_id, offset for stores)
3170        //          |                           |
3171        //          V                           |
3172        // C3 clean_accounts()/                 | map of stores (removes old entry)
3173        //        handle_reclaims()             |
3174        //
3175        // Remarks for cleaner: So, for any reading operations, it's a race condition
3176        // where C3 happens between R1 and R2. In that case, retrying from R1 is safu.
3177        // In that case, None would be returned while bailing out at R1.
3178        //
3179        // Purger                                 | Accessed data source for cached/stored
3180        // ---------------------------------------+----------------------------------
3181        // P1 purge_slot()                        | N/A
3182        //          |                             |
3183        //          V                             |
3184        // P2 purge_slots_from_cache_and_store()  | map of caches/stores (removes old entry)
3185        //          |                             |
3186        //          V                             |
3187        // P3 purge_slots_from_cache_and_store()/ | index
3188        //       purge_slot_cache()/              |
3189        //          purge_slot_cache_pubkeys()    | (removes existing store_id, offset for cache)
3190        //       purge_slot_storage()/            |
3191        //          purge_keys_exact()            | (removes accounts index entries)
3192        //          handle_reclaims()             | (removes storage entries)
3193        //      OR                                |
3194        //    clean_accounts()/                   |
3195        //        clean_accounts_older_than_root()| (removes existing store_id, offset for stores)
3196        //                                        V
3197        //
3198        // Remarks for purger: So, for any reading operations, it's a race condition
3199        // where P2 happens between R1 and R2. In that case, retrying from R1 is safu.
3200        // In that case, we may bail at index read retry when P3 hasn't been run
3201
3202        #[cfg(test)]
3203        {
3204            // Give some time for cache flushing to occur here for unit tests
3205            sleep(Duration::from_millis(self.load_delay));
3206        }
3207
3208        // Failsafe for potential race conditions with other subsystems
3209        let mut num_acceptable_failed_iterations = 0;
3210        loop {
3211            let account_accessor = self.get_account_accessor(slot, pubkey, store_id, offset);
3212            match account_accessor {
3213                LoadedAccountAccessor::Cached(Some(_)) | LoadedAccountAccessor::Stored(Some(_)) => {
3214                    // Great! There was no race, just return :) This is the most usual situation
3215                    return Some((account_accessor, slot));
3216                }
3217                LoadedAccountAccessor::Cached(None) => {
3218                    num_acceptable_failed_iterations += 1;
3219                    // Cache was flushed in between checking the index and retrieving from the cache,
3220                    // so retry. This works because in accounts cache flush, an account is written to
3221                    // storage *before* it is removed from the cache
3222                    match load_hint {
3223                        LoadHint::FixedMaxRoot => {
3224                            // it's impossible for this to fail for transaction loads from
3225                            // replaying/banking more than once.
3226                            // This is because:
3227                            // 1) For a slot `X` that's being replayed, there is only one
3228                            // latest ancestor containing the latest update for the account, and this
3229                            // ancestor can only be flushed once.
3230                            // 2) The root cannot move while replaying, so the index cannot continually
3231                            // find more up to date entries than the current `slot`
3232                            assert!(num_acceptable_failed_iterations <= 1);
3233                        }
3234                        LoadHint::Unspecified => {
3235                            // Because newer root can be added to the index (= not fixed),
3236                            // multiple flush race conditions can be observed under very rare
3237                            // condition, at least theoretically
3238                        }
3239                    }
3240                }
3241                LoadedAccountAccessor::Stored(None) => {
3242                    match load_hint {
3243                        LoadHint::FixedMaxRoot => {
3244                            // When running replay on the validator, or banking stage on the leader,
3245                            // it should be very rare that the storage entry doesn't exist if the
3246                            // entry in the accounts index is the latest version of this account.
3247                            //
3248                            // There are only a few places where the storage entry may not exist
3249                            // after reading the index:
3250                            // 1) Shrink has removed the old storage entry and rewritten to
3251                            // a newer storage entry
3252                            // 2) The `pubkey` asked for in this function is a zero-carat account,
3253                            // and the storage entry holding this account qualified for zero-carat clean.
3254                            //
3255                            // In both these cases, it should be safe to retry and recheck the accounts
3256                            // index indefinitely, without incrementing num_acceptable_failed_iterations.
3257                            // That's because if the root is fixed, there should be a bounded number
3258                            // of pending cleans/shrinks (depends how far behind the AccountsBackgroundService
3259                            // is), termination to the desired condition is guaranteed.
3260                            //
3261                            // Also note that in both cases, if we do find the storage entry,
3262                            // we can guarantee that the storage entry is safe to read from because
3263                            // we grabbed a reference to the storage entry while it was still in the
3264                            // storage map. This means even if the storage entry is removed from the storage
3265                            // map after we grabbed the storage entry, the recycler should not reset the
3266                            // storage entry until we drop the reference to the storage entry.
3267                            //
3268                            // eh, no code in this arm? yes!
3269                        }
3270                        LoadHint::Unspecified => {
3271                            // RPC get_account() may have fetched an old root from the index that was
3272                            // either:
3273                            // 1) Cleaned up by clean_accounts(), so the accounts index has been updated
3274                            // and the storage entries have been removed.
3275                            // 2) Dropped by purge_slots() because the slot was on a minor fork, which
3276                            // removes the slots' storage entries but doesn't purge from the accounts index
3277                            // (account index cleanup is left to clean for stored slots). Note that
3278                            // this generally is impossible to occur in the wild because the RPC
3279                            // should hold the slot's bank, preventing it from being purged() to
3280                            // begin with.
3281                            num_acceptable_failed_iterations += 1;
3282                        }
3283                    }
3284                }
3285            }
3286            #[cfg(not(test))]
3287            let load_limit = ABSURD_CONSECUTIVE_FAILED_ITERATIONS;
3288
3289            #[cfg(test)]
3290            let load_limit = self.load_limit.load(Ordering::Relaxed);
3291
3292            let fallback_to_slow_path = if num_acceptable_failed_iterations >= load_limit {
3293                // The latest version of the account existed in the index, but could not be
3294                // fetched from storage. This means a race occurred between this function and clean
3295                // accounts/purge_slots
3296                let message = format!(
3297                    "do_load() failed to get key: {} from storage, latest attempt was for \
3298                     slot: {}, storage_entry: {} offset: {}, load_hint: {:?}",
3299                    pubkey, slot, store_id, offset, load_hint,
3300                );
3301                datapoint_warn!("accounts_db-do_load_warn", ("warn", message, String));
3302                true
3303            } else {
3304                false
3305            };
3306
3307            // Because reading from the cache/storage failed, retry from the index read
3308            let (new_slot, new_store_id, new_offset, maybe_account_accessor) = self
3309                .read_index_for_accessor_or_load_slow(
3310                    ancestors,
3311                    pubkey,
3312                    max_root,
3313                    fallback_to_slow_path,
3314                )?;
3315            // Notice the subtle `?` at previous line, we bail out pretty early if missing.
3316
3317            if new_slot == slot && new_store_id == store_id {
3318                // Considering that we're failed to get accessor above and further that
3319                // the index still returned the same (slot, store_id) tuple, offset must be same
3320                // too.
3321                assert!(new_offset == offset);
3322
3323                // If the entry was missing from the cache, that means it must have been flushed,
3324                // and the accounts index is always updated before cache flush, so store_id must
3325                // not indicate being cached at this point.
3326                assert!(new_store_id != CACHE_VIRTUAL_STORAGE_ID);
3327
3328                // If this is not a cache entry, then this was a minor fork slot
3329                // that had its storage entries cleaned up by purge_slots() but hasn't been
3330                // cleaned yet. That means this must be rpc access and not replay/banking at the
3331                // very least. Note that purge shouldn't occur even for RPC as caller must hold all
3332                // of ancestor slots..
3333                assert!(load_hint == LoadHint::Unspecified);
3334
3335                // Everything being assert!()-ed, let's panic!() here as it's an error condition
3336                // after all....
3337                // That reasoning is based on the fact all of code-path reaching this fn
3338                // retry_to_get_account_accessor() must outlive the Arc<Bank> (and its all
3339                // ancestors) over this fn invocation, guaranteeing the prevention of being purged,
3340                // first of all.
3341                // For details, see the comment in AccountIndex::do_checked_scan_accounts(),
3342                // which is referring back here.
3343                panic!(
3344                    "Bad index entry detected ({}, {}, {}, {}, {:?})",
3345                    pubkey, slot, store_id, offset, load_hint
3346                );
3347            } else if fallback_to_slow_path {
3348                // the above bad-index-entry check must had been checked first to retain the same
3349                // behavior
3350                return Some((
3351                    maybe_account_accessor.expect("must be some if clone_in_lock=true"),
3352                    new_slot,
3353                ));
3354            }
3355
3356            slot = new_slot;
3357            store_id = new_store_id;
3358            offset = new_offset;
3359        }
3360    }
3361
3362    fn do_load(
3363        &self,
3364        ancestors: &Ancestors,
3365        pubkey: &Pubkey,
3366        max_root: Option<Slot>,
3367        load_hint: LoadHint,
3368    ) -> Option<(AccountSharedData, Slot)> {
3369        #[cfg(not(test))]
3370        assert!(max_root.is_none());
3371
3372        let (slot, store_id, offset, _maybe_account_accesor) =
3373            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
3374        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
3375
3376        if self.caching_enabled && store_id != CACHE_VIRTUAL_STORAGE_ID {
3377            let result = self.read_only_accounts_cache.load(pubkey, slot);
3378            if let Some(account) = result {
3379                return Some((account, slot));
3380            }
3381        }
3382
3383        let (mut account_accessor, slot) = self.retry_to_get_account_accessor(
3384            slot, store_id, offset, ancestors, pubkey, max_root, load_hint,
3385        )?;
3386        let loaded_account = account_accessor.check_and_get_loaded_account();
3387        let is_cached = loaded_account.is_cached();
3388        let account = loaded_account.take_account();
3389
3390        if self.caching_enabled && !is_cached {
3391            /*
3392            We show this store into the read-only cache for account 'A' and future loads of 'A' from the read-only cache are
3393            safe/reflect 'A''s latest state on this fork.
3394            This safety holds if during replay of slot 'S', we show we only read 'A' from the write cache,
3395            not the read-only cache, after it's been updated in replay of slot 'S'.
3396            Assume for contradiction this is not true, and we read 'A' from the read-only cache *after* it had been updated in 'S'.
3397            This means an entry '(S, A)' was added to the read-only cache after 'A' had been updated in 'S'.
3398            Now when '(S, A)' was being added to the read-only cache, it must have been true that  'is_cache == false',
3399            which means '(S', A)' does not exist in the write cache yet.
3400            However, by the assumption for contradiction above ,  'A' has already been updated in 'S' which means '(S, A)'
3401            must exist in the write cache, which is a contradiction.
3402            */
3403            self.read_only_accounts_cache.store(pubkey, slot, &account);
3404        }
3405        Some((account, slot))
3406    }
3407
3408    pub fn load_account_hash(
3409        &self,
3410        ancestors: &Ancestors,
3411        pubkey: &Pubkey,
3412        max_root: Option<Slot>,
3413        load_hint: LoadHint,
3414    ) -> Option<Hash> {
3415        let (slot, store_id, offset, _maybe_account_accesor) =
3416            self.read_index_for_accessor_or_load_slow(ancestors, pubkey, max_root, false)?;
3417        // Notice the subtle `?` at previous line, we bail out pretty early if missing.
3418
3419        let (mut account_accessor, _) = self.retry_to_get_account_accessor(
3420            slot, store_id, offset, ancestors, pubkey, max_root, load_hint,
3421        )?;
3422        let loaded_account = account_accessor.check_and_get_loaded_account();
3423        Some(loaded_account.loaded_hash())
3424    }
3425
3426    fn get_account_accessor<'a>(
3427        &'a self,
3428        slot: Slot,
3429        pubkey: &'a Pubkey,
3430        store_id: usize,
3431        offset: usize,
3432    ) -> LoadedAccountAccessor<'a> {
3433        if store_id == CACHE_VIRTUAL_STORAGE_ID {
3434            let maybe_cached_account = self
3435                .accounts_cache
3436                .load(slot, pubkey)
3437                .map(|cached_account| (*pubkey, Cow::Owned(cached_account)));
3438            LoadedAccountAccessor::Cached(maybe_cached_account)
3439        } else {
3440            let maybe_storage_entry = self
3441                .storage
3442                .get_account_storage_entry(slot, store_id)
3443                .map(|account_storage_entry| (account_storage_entry, offset));
3444            LoadedAccountAccessor::Stored(maybe_storage_entry)
3445        }
3446    }
3447
3448    fn try_recycle_and_insert_store(
3449        &self,
3450        slot: Slot,
3451        min_size: u64,
3452        max_size: u64,
3453    ) -> Option<Arc<AccountStorageEntry>> {
3454        let store = self.try_recycle_store(slot, min_size, max_size)?;
3455        self.insert_store(slot, store.clone());
3456        Some(store)
3457    }
3458
3459    fn try_recycle_store(
3460        &self,
3461        slot: Slot,
3462        min_size: u64,
3463        max_size: u64,
3464    ) -> Option<Arc<AccountStorageEntry>> {
3465        let mut max = 0;
3466        let mut min = std::u64::MAX;
3467        let mut avail = 0;
3468        let mut recycle_stores = self.recycle_stores.write().unwrap();
3469        for (i, (_recycled_time, store)) in recycle_stores.iter().enumerate() {
3470            if Arc::strong_count(store) == 1 {
3471                max = std::cmp::max(store.accounts.capacity(), max);
3472                min = std::cmp::min(store.accounts.capacity(), min);
3473                avail += 1;
3474
3475                if store.accounts.capacity() >= min_size && store.accounts.capacity() < max_size {
3476                    let ret = recycle_stores.remove_entry(i);
3477                    drop(recycle_stores);
3478                    let old_id = ret.append_vec_id();
3479                    ret.recycle(slot, self.next_id.fetch_add(1, Ordering::Relaxed));
3480                    debug!(
3481                        "recycling store: {} {:?} old_id: {}",
3482                        ret.append_vec_id(),
3483                        ret.get_path(),
3484                        old_id
3485                    );
3486                    return Some(ret);
3487                }
3488            }
3489        }
3490        debug!(
3491            "no recycle stores max: {} min: {} len: {} looking: {}, {} avail: {}",
3492            max,
3493            min,
3494            recycle_stores.entry_count(),
3495            min_size,
3496            max_size,
3497            avail,
3498        );
3499        None
3500    }
3501
3502    fn find_storage_candidate(&self, slot: Slot, size: usize) -> Arc<AccountStorageEntry> {
3503        let mut create_extra = false;
3504        let mut get_slot_stores = Measure::start("get_slot_stores");
3505        let slot_stores_lock = self.storage.get_slot_stores(slot);
3506        get_slot_stores.stop();
3507        self.stats
3508            .store_get_slot_store
3509            .fetch_add(get_slot_stores.as_us(), Ordering::Relaxed);
3510        let mut find_existing = Measure::start("find_existing");
3511        if let Some(slot_stores_lock) = slot_stores_lock {
3512            let slot_stores = slot_stores_lock.read().unwrap();
3513            if !slot_stores.is_empty() {
3514                if slot_stores.len() <= self.min_num_stores {
3515                    let mut total_accounts = 0;
3516                    for store in slot_stores.values() {
3517                        total_accounts += store.count();
3518                    }
3519
3520                    // Create more stores so that when scanning the storage all CPUs have work
3521                    if (total_accounts / 16) >= slot_stores.len() {
3522                        create_extra = true;
3523                    }
3524                }
3525
3526                // pick an available store at random by iterating from a random point
3527                let to_skip = thread_rng().gen_range(0, slot_stores.len());
3528
3529                for (i, store) in slot_stores.values().cycle().skip(to_skip).enumerate() {
3530                    if store.try_available() {
3531                        let ret = store.clone();
3532                        drop(slot_stores);
3533                        if create_extra {
3534                            if self
3535                                .try_recycle_and_insert_store(slot, size as u64, std::u64::MAX)
3536                                .is_none()
3537                            {
3538                                self.stats
3539                                    .create_store_count
3540                                    .fetch_add(1, Ordering::Relaxed);
3541                                self.create_and_insert_store(slot, self.file_size, "store extra");
3542                            } else {
3543                                self.stats
3544                                    .recycle_store_count
3545                                    .fetch_add(1, Ordering::Relaxed);
3546                            }
3547                        }
3548                        find_existing.stop();
3549                        self.stats
3550                            .store_find_existing
3551                            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
3552                        return ret;
3553                    }
3554                    // looked at every store, bail...
3555                    if i == slot_stores.len() {
3556                        break;
3557                    }
3558                }
3559            }
3560        }
3561        find_existing.stop();
3562        self.stats
3563            .store_find_existing
3564            .fetch_add(find_existing.as_us(), Ordering::Relaxed);
3565
3566        let store = if let Some(store) = self.try_recycle_store(slot, size as u64, std::u64::MAX) {
3567            self.stats
3568                .recycle_store_count
3569                .fetch_add(1, Ordering::Relaxed);
3570            store
3571        } else {
3572            self.stats
3573                .create_store_count
3574                .fetch_add(1, Ordering::Relaxed);
3575            self.create_store(slot, self.file_size, "store", &self.paths)
3576        };
3577
3578        // try_available is like taking a lock on the store,
3579        // preventing other threads from using it.
3580        // It must succeed here and happen before insert,
3581        // otherwise another thread could also grab it from the index.
3582        assert!(store.try_available());
3583        self.insert_store(slot, store.clone());
3584        store
3585    }
3586
3587    fn page_align(size: u64) -> u64 {
3588        (size + (PAGE_SIZE - 1)) & !(PAGE_SIZE - 1)
3589    }
3590
3591    fn has_space_available(&self, slot: Slot, size: u64) -> bool {
3592        let slot_storage = self.storage.get_slot_stores(slot).unwrap();
3593        let slot_storage_r = slot_storage.read().unwrap();
3594        for (_id, store) in slot_storage_r.iter() {
3595            if store.status() == AccountStorageStatus::Available
3596                && (store.accounts.capacity() - store.accounts.len() as u64) > size
3597            {
3598                return true;
3599            }
3600        }
3601        false
3602    }
3603
3604    fn create_store(
3605        &self,
3606        slot: Slot,
3607        size: u64,
3608        from: &str,
3609        paths: &[PathBuf],
3610    ) -> Arc<AccountStorageEntry> {
3611        let path_index = thread_rng().gen_range(0, paths.len());
3612        let store = Arc::new(self.new_storage_entry(
3613            slot,
3614            Path::new(&paths[path_index]),
3615            Self::page_align(size),
3616        ));
3617
3618        if store.append_vec_id() == CACHE_VIRTUAL_STORAGE_ID {
3619            panic!("We've run out of storage ids!");
3620        }
3621
3622        debug!(
3623            "creating store: {} slot: {} len: {} size: {} from: {} path: {:?}",
3624            store.append_vec_id(),
3625            slot,
3626            store.accounts.len(),
3627            store.accounts.capacity(),
3628            from,
3629            store.accounts.get_path()
3630        );
3631
3632        store
3633    }
3634
3635    fn create_and_insert_store(
3636        &self,
3637        slot: Slot,
3638        size: u64,
3639        from: &str,
3640    ) -> Arc<AccountStorageEntry> {
3641        self.create_and_insert_store_with_paths(slot, size, from, &self.paths)
3642    }
3643
3644    fn create_and_insert_store_with_paths(
3645        &self,
3646        slot: Slot,
3647        size: u64,
3648        from: &str,
3649        paths: &[PathBuf],
3650    ) -> Arc<AccountStorageEntry> {
3651        let store = self.create_store(slot, size, from, paths);
3652        let store_for_index = store.clone();
3653
3654        self.insert_store(slot, store_for_index);
3655        store
3656    }
3657
3658    fn insert_store(&self, slot: Slot, store: Arc<AccountStorageEntry>) {
3659        let slot_storages: SlotStores = self.storage.get_slot_stores(slot).unwrap_or_else(||
3660            // DashMap entry.or_insert() returns a RefMut, essentially a write lock,
3661            // which is dropped after this block ends, minimizing time held by the lock.
3662            // However, we still want to persist the reference to the `SlotStores` behind
3663            // the lock, hence we clone it out, (`SlotStores` is an Arc so is cheap to clone).
3664            self.storage
3665                .0
3666                .entry(slot)
3667                .or_insert(Arc::new(RwLock::new(HashMap::new())))
3668                .clone());
3669
3670        assert!(slot_storages
3671            .write()
3672            .unwrap()
3673            .insert(store.append_vec_id(), store)
3674            .is_none());
3675    }
3676
3677    pub fn create_drop_bank_callback(
3678        &self,
3679        pruned_banks_sender: DroppedSlotsSender,
3680    ) -> SendDroppedBankCallback {
3681        self.is_bank_drop_callback_enabled
3682            .store(true, Ordering::SeqCst);
3683        SendDroppedBankCallback::new(pruned_banks_sender)
3684    }
3685
3686    /// This should only be called after the `Bank::drop()` runs in bank.rs, See BANK_DROP_SAFETY
3687    /// comment below for more explanation.
3688    /// `is_from_abs` is true if the caller is the AccountsBackgroundService
3689    pub fn purge_slot(&self, slot: Slot, bank_id: BankId, is_from_abs: bool) {
3690        if self.is_bank_drop_callback_enabled.load(Ordering::SeqCst) && !is_from_abs {
3691            panic!("bad drop callpath detected; Bank::drop() must run serially with other logic in ABS like clean_accounts()")
3692        }
3693        // BANK_DROP_SAFETY: Because this function only runs once the bank is dropped,
3694        // we know that there are no longer any ongoing scans on this bank, because scans require
3695        // and hold a reference to the bank at the tip of the fork they're scanning. Hence it's
3696        // safe to remove this bank_id from the `removed_bank_ids` list at this point.
3697        if self
3698            .accounts_index
3699            .removed_bank_ids
3700            .lock()
3701            .unwrap()
3702            .remove(&bank_id)
3703        {
3704            // If this slot was already cleaned up, no need to do any further cleans
3705            return;
3706        }
3707
3708        self.purge_slots(std::iter::once(&slot));
3709    }
3710
3711    fn recycle_slot_stores(
3712        &self,
3713        total_removed_storage_entries: usize,
3714        slot_stores: &[SlotStores],
3715    ) -> u64 {
3716        let mut recycled_count = 0;
3717
3718        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_elapsed");
3719        let mut recycle_stores = self.recycle_stores.write().unwrap();
3720        recycle_stores_write_elapsed.stop();
3721
3722        for slot_entries in slot_stores {
3723            let entry = slot_entries.read().unwrap();
3724            for (_store_id, stores) in entry.iter() {
3725                if recycle_stores.entry_count() > MAX_RECYCLE_STORES {
3726                    let dropped_count = total_removed_storage_entries - recycled_count;
3727                    self.stats
3728                        .dropped_stores
3729                        .fetch_add(dropped_count as u64, Ordering::Relaxed);
3730                    return recycle_stores_write_elapsed.as_us();
3731                }
3732                recycle_stores.add_entry(stores.clone());
3733                recycled_count += 1;
3734            }
3735        }
3736        recycle_stores_write_elapsed.as_us()
3737    }
3738
3739    /// Purges every slot in `removed_slots` from both the cache and storage. This includes
3740    /// entries in the accounts index, cache entries, and any backing storage entries.
3741    fn purge_slots_from_cache_and_store<'a>(
3742        &self,
3743        removed_slots: impl Iterator<Item = &'a Slot>,
3744        purge_stats: &PurgeStats,
3745    ) {
3746        let mut remove_cache_elapsed_across_slots = 0;
3747        let mut num_cached_slots_removed = 0;
3748        let mut total_removed_cached_bytes = 0;
3749        for remove_slot in removed_slots {
3750            // This function is only currently safe with respect to `flush_slot_cache()` because
3751            // both functions run serially in AccountsBackgroundService.
3752            let mut remove_cache_elapsed = Measure::start("remove_cache_elapsed");
3753            // Note: we cannot remove this slot from the slot cache until we've removed its
3754            // entries from the accounts index first. This is because `scan_accounts()` relies on
3755            // holding the index lock, finding the index entry, and then looking up the entry
3756            // in the cache. If it fails to find that entry, it will panic in `get_loaded_account()`
3757            if let Some(slot_cache) = self.accounts_cache.slot_cache(*remove_slot) {
3758                // If the slot is still in the cache, remove the backing storages for
3759                // the slot and from the Accounts Index
3760                num_cached_slots_removed += 1;
3761                total_removed_cached_bytes += slot_cache.total_bytes();
3762                self.purge_slot_cache(*remove_slot, slot_cache);
3763                remove_cache_elapsed.stop();
3764                remove_cache_elapsed_across_slots += remove_cache_elapsed.as_us();
3765                // Nobody else shoud have removed the slot cache entry yet
3766                assert!(self.accounts_cache.remove_slot(*remove_slot).is_some());
3767            } else {
3768                self.purge_slot_storage(*remove_slot, purge_stats);
3769            }
3770            // It should not be possible that a slot is neither in the cache or storage. Even in
3771            // a slot with all ticks, `Bank::new_from_parent()` immediately stores some sysvars
3772            // on bank creation.
3773        }
3774
3775        purge_stats
3776            .remove_cache_elapsed
3777            .fetch_add(remove_cache_elapsed_across_slots, Ordering::Relaxed);
3778        purge_stats
3779            .num_cached_slots_removed
3780            .fetch_add(num_cached_slots_removed, Ordering::Relaxed);
3781        purge_stats
3782            .total_removed_cached_bytes
3783            .fetch_add(total_removed_cached_bytes, Ordering::Relaxed);
3784    }
3785
3786    /// Purge the backing storage entries for the given slot, does not purge from
3787    /// the cache!
3788    fn purge_dead_slots_from_storage<'a>(
3789        &'a self,
3790        removed_slots: impl Iterator<Item = &'a Slot> + Clone,
3791        purge_stats: &PurgeStats,
3792    ) {
3793        // Check all slots `removed_slots` are no longer "relevant" roots.
3794        // Note that the slots here could have been rooted slots, but if they're passed here
3795        // for removal it means:
3796        // 1) All updates in that old root have been outdated by updates in newer roots
3797        // 2) Those slots/roots should have already been purged from the accounts index root
3798        // tracking metadata via `accounts_index.clean_dead_slot()`.
3799        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
3800        assert!(self
3801            .accounts_index
3802            .get_rooted_from_list(removed_slots.clone())
3803            .is_empty());
3804        safety_checks_elapsed.stop();
3805        purge_stats
3806            .safety_checks_elapsed
3807            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
3808
3809        let mut total_removed_storage_entries = 0;
3810        let mut total_removed_stored_bytes = 0;
3811        let mut all_removed_slot_storages = vec![];
3812
3813        let mut remove_storage_entries_elapsed = Measure::start("remove_storage_entries_elapsed");
3814        for remove_slot in removed_slots {
3815            // Remove the storage entries and collect some metrics
3816            if let Some((_, slot_storages_to_be_removed)) = self.storage.0.remove(remove_slot) {
3817                {
3818                    let r_slot_removed_storages = slot_storages_to_be_removed.read().unwrap();
3819                    total_removed_storage_entries += r_slot_removed_storages.len();
3820                    total_removed_stored_bytes += r_slot_removed_storages
3821                        .values()
3822                        .map(|i| i.accounts.capacity())
3823                        .sum::<u64>();
3824                }
3825                all_removed_slot_storages.push(slot_storages_to_be_removed.clone());
3826            }
3827        }
3828        remove_storage_entries_elapsed.stop();
3829        let num_stored_slots_removed = all_removed_slot_storages.len();
3830
3831        let recycle_stores_write_elapsed =
3832            self.recycle_slot_stores(total_removed_storage_entries, &all_removed_slot_storages);
3833
3834        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
3835        // Backing mmaps for removed storages entries explicitly dropped here outside
3836        // of any locks
3837        drop(all_removed_slot_storages);
3838        drop_storage_entries_elapsed.stop();
3839        purge_stats
3840            .remove_storage_entries_elapsed
3841            .fetch_add(remove_storage_entries_elapsed.as_us(), Ordering::Relaxed);
3842        purge_stats
3843            .drop_storage_entries_elapsed
3844            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
3845        purge_stats
3846            .num_stored_slots_removed
3847            .fetch_add(num_stored_slots_removed, Ordering::Relaxed);
3848        purge_stats
3849            .total_removed_storage_entries
3850            .fetch_add(total_removed_storage_entries, Ordering::Relaxed);
3851        purge_stats
3852            .total_removed_stored_bytes
3853            .fetch_add(total_removed_stored_bytes, Ordering::Relaxed);
3854        purge_stats
3855            .recycle_stores_write_elapsed
3856            .fetch_add(recycle_stores_write_elapsed, Ordering::Relaxed);
3857    }
3858
3859    fn purge_slot_cache(&self, purged_slot: Slot, slot_cache: SlotCache) {
3860        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
3861        let pubkey_to_slot_set: Vec<(Pubkey, Slot)> = slot_cache
3862            .iter()
3863            .map(|account| {
3864                purged_slot_pubkeys.insert((purged_slot, *account.key()));
3865                (*account.key(), purged_slot)
3866            })
3867            .collect();
3868        self.purge_slot_cache_pubkeys(purged_slot, purged_slot_pubkeys, pubkey_to_slot_set, true);
3869    }
3870
3871    fn purge_slot_cache_pubkeys(
3872        &self,
3873        purged_slot: Slot,
3874        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
3875        pubkey_to_slot_set: Vec<(Pubkey, Slot)>,
3876        is_dead: bool,
3877    ) {
3878        // Slot purged from cache should not exist in the backing store
3879        assert!(self.storage.get_slot_stores(purged_slot).is_none());
3880        let num_purged_keys = pubkey_to_slot_set.len();
3881        let reclaims = self.purge_keys_exact(pubkey_to_slot_set.iter());
3882        assert_eq!(reclaims.len(), num_purged_keys);
3883        if is_dead {
3884            self.remove_dead_slots_metadata(
3885                std::iter::once(&purged_slot),
3886                purged_slot_pubkeys,
3887                None,
3888            );
3889        }
3890    }
3891
3892    fn purge_slot_storage(&self, remove_slot: Slot, purge_stats: &PurgeStats) {
3893        // Because AccountsBackgroundService synchronously flushes from the accounts cache
3894        // and handles all Bank::drop() (the cleanup function that leads to this
3895        // function call), then we don't need to worry above an overlapping cache flush
3896        // with this function call. This means, if we get into this case, we can be
3897        // confident that the entire state for this slot has been flushed to the storage
3898        // already.
3899        let mut scan_storages_elasped = Measure::start("scan_storages_elasped");
3900        type ScanResult = ScanStorageResult<Pubkey, Arc<Mutex<HashSet<(Pubkey, Slot)>>>>;
3901        let scan_result: ScanResult = self.scan_account_storage(
3902            remove_slot,
3903            |loaded_account: LoadedAccount| Some(*loaded_account.pubkey()),
3904            |accum: &Arc<Mutex<HashSet<(Pubkey, Slot)>>>, loaded_account: LoadedAccount| {
3905                accum
3906                    .lock()
3907                    .unwrap()
3908                    .insert((*loaded_account.pubkey(), remove_slot));
3909            },
3910        );
3911        scan_storages_elasped.stop();
3912        purge_stats
3913            .scan_storages_elasped
3914            .fetch_add(scan_storages_elasped.as_us(), Ordering::Relaxed);
3915
3916        let mut purge_accounts_index_elapsed = Measure::start("purge_accounts_index_elapsed");
3917        let reclaims;
3918        match scan_result {
3919            ScanStorageResult::Cached(_) => {
3920                panic!("Should not see cached keys in this `else` branch, since we checked this slot did not exist in the cache above");
3921            }
3922            ScanStorageResult::Stored(stored_keys) => {
3923                // Purge this slot from the accounts index
3924                reclaims = self.purge_keys_exact(stored_keys.lock().unwrap().iter());
3925            }
3926        }
3927        purge_accounts_index_elapsed.stop();
3928        purge_stats
3929            .purge_accounts_index_elapsed
3930            .fetch_add(purge_accounts_index_elapsed.as_us(), Ordering::Relaxed);
3931
3932        // `handle_reclaims()` should remove all the account index entries and
3933        // storage entries
3934        let mut handle_reclaims_elapsed = Measure::start("handle_reclaims_elapsed");
3935        // Slot should be dead after removing all its account entries
3936        let expected_dead_slot = Some(remove_slot);
3937        self.handle_reclaims(
3938            &reclaims,
3939            expected_dead_slot,
3940            Some(purge_stats),
3941            Some(&mut ReclaimResult::default()),
3942            false,
3943        );
3944        handle_reclaims_elapsed.stop();
3945        purge_stats
3946            .handle_reclaims_elapsed
3947            .fetch_add(handle_reclaims_elapsed.as_us(), Ordering::Relaxed);
3948        // After handling the reclaimed entries, this slot's
3949        // storage entries should be purged from self.storage
3950        assert!(self.storage.get_slot_stores(remove_slot).is_none());
3951    }
3952
3953    #[allow(clippy::needless_collect)]
3954    fn purge_slots<'a>(&self, slots: impl Iterator<Item = &'a Slot>) {
3955        // `add_root()` should be called first
3956        let mut safety_checks_elapsed = Measure::start("safety_checks_elapsed");
3957        let non_roots = slots
3958            // Only safe to check when there are  duplciate versions of a slot
3959            // because ReplayStage will not make new roots before dumping the
3960            // duplicate slots first. Thus we will not be in a case where we
3961            // root slot `S`, then try to dump some other version of slot `S`, the
3962            // dumping has to finish first
3963            //
3964            // Also note roots are never removed via `remove_unrooted_slot()`, so
3965            // it's safe to filter them out here as they won't need deletion from
3966            // self.accounts_index.removed_bank_ids in `purge_slots_from_cache_and_store()`.
3967            .filter(|slot| !self.accounts_index.is_root(**slot));
3968        safety_checks_elapsed.stop();
3969        self.external_purge_slots_stats
3970            .safety_checks_elapsed
3971            .fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
3972        self.purge_slots_from_cache_and_store(non_roots, &self.external_purge_slots_stats);
3973        self.external_purge_slots_stats
3974            .report("external_purge_slots_stats", Some(1000));
3975    }
3976
3977    pub fn remove_unrooted_slots(&self, remove_slots: &[(Slot, BankId)]) {
3978        let rooted_slots = self
3979            .accounts_index
3980            .get_rooted_from_list(remove_slots.iter().map(|(slot, _)| slot));
3981        assert!(
3982            rooted_slots.is_empty(),
3983            "Trying to remove accounts for rooted slots {:?}",
3984            rooted_slots
3985        );
3986
3987        let RemoveUnrootedSlotsSynchronization {
3988            slots_under_contention,
3989            signal,
3990        } = &self.remove_unrooted_slots_synchronization;
3991
3992        {
3993            // Slots that are currently being flushed by flush_slot_cache()
3994
3995            let mut currently_contended_slots = slots_under_contention.lock().unwrap();
3996
3997            // Slots that are currently being flushed by flush_slot_cache() AND
3998            // we want to remove in this function
3999            let mut remaining_contended_flush_slots: Vec<Slot> = remove_slots
4000                .iter()
4001                .filter_map(|(remove_slot, _)| {
4002                    let is_being_flushed = currently_contended_slots.contains(remove_slot);
4003                    if !is_being_flushed {
4004                        // Reserve the slots that we want to purge that aren't currently
4005                        // being flushed to prevent cache from flushing those slots in
4006                        // the future.
4007                        //
4008                        // Note that the single replay thread has to remove a specific slot `N`
4009                        // before another version of the same slot can be replayed. This means
4010                        // multiple threads should not call `remove_unrooted_slots()` simultaneously
4011                        // with the same slot.
4012                        currently_contended_slots.insert(*remove_slot);
4013                    }
4014                    // If the cache is currently flushing this slot, add it to the list
4015                    Some(remove_slot).filter(|_| is_being_flushed)
4016                })
4017                .cloned()
4018                .collect();
4019
4020            // Wait for cache flushes to finish
4021            loop {
4022                if !remaining_contended_flush_slots.is_empty() {
4023                    // Wait for the signal that the cache has finished flushing a slot
4024                    //
4025                    // Don't wait if the remaining_contended_flush_slots is empty, otherwise
4026                    // we may never get a signal since there's no cache flush thread to
4027                    // do the signaling
4028                    currently_contended_slots = signal.wait(currently_contended_slots).unwrap();
4029                } else {
4030                    // There are no slots being flushed to wait on, so it's safe to continue
4031                    // to purging the slots we want to purge!
4032                    break;
4033                }
4034
4035                // For each slot the cache flush has finished, mark that we're about to start
4036                // purging these slots by reserving it in `currently_contended_slots`.
4037                remaining_contended_flush_slots.retain(|flush_slot| {
4038                    let is_being_flushed = currently_contended_slots.contains(flush_slot);
4039                    if !is_being_flushed {
4040                        // Mark that we're about to delete this slot now
4041                        currently_contended_slots.insert(*flush_slot);
4042                    }
4043                    is_being_flushed
4044                });
4045            }
4046        }
4047
4048        // Mark down these slots are about to be purged so that new attempts to scan these
4049        // banks fail, and any ongoing scans over these slots will detect that they should abort
4050        // their results
4051        {
4052            let mut locked_removed_bank_ids = self.accounts_index.removed_bank_ids.lock().unwrap();
4053            for (_slot, remove_bank_id) in remove_slots.iter() {
4054                locked_removed_bank_ids.insert(*remove_bank_id);
4055            }
4056        }
4057
4058        let remove_unrooted_purge_stats = PurgeStats::default();
4059        self.purge_slots_from_cache_and_store(
4060            remove_slots.iter().map(|(slot, _)| slot),
4061            &remove_unrooted_purge_stats,
4062        );
4063        remove_unrooted_purge_stats.report("remove_unrooted_slots_purge_slots_stats", Some(0));
4064
4065        let mut currently_contended_slots = slots_under_contention.lock().unwrap();
4066        for (remove_slot, _) in remove_slots {
4067            assert!(currently_contended_slots.remove(remove_slot));
4068        }
4069    }
4070
4071    pub fn hash_stored_account(slot: Slot, account: &StoredAccountMeta) -> Hash {
4072        Self::hash_account_data(
4073            slot,
4074            account.account_meta.carats,
4075            &account.account_meta.owner,
4076            account.account_meta.executable,
4077            account.account_meta.rent_epoch,
4078            account.data,
4079            &account.meta.pubkey,
4080        )
4081    }
4082
4083    pub fn hash_account<T: ReadableAccount>(slot: Slot, account: &T, pubkey: &Pubkey) -> Hash {
4084        Self::hash_account_data(
4085            slot,
4086            account.carats(),
4087            account.owner(),
4088            account.executable(),
4089            account.rent_epoch(),
4090            account.data(),
4091            pubkey,
4092        )
4093    }
4094
4095    fn hash_frozen_account_data(account: &AccountSharedData) -> Hash {
4096        let mut hasher = Hasher::default();
4097
4098        hasher.hash(account.data());
4099        hasher.hash(account.owner().as_ref());
4100
4101        if account.executable() {
4102            hasher.hash(&[1u8; 1]);
4103        } else {
4104            hasher.hash(&[0u8; 1]);
4105        }
4106
4107        hasher.result()
4108    }
4109
4110    fn hash_account_data(
4111        slot: Slot,
4112        carats: u64,
4113        owner: &Pubkey,
4114        executable: bool,
4115        rent_epoch: Epoch,
4116        data: &[u8],
4117        pubkey: &Pubkey,
4118    ) -> Hash {
4119        if carats == 0 {
4120            return Hash::default();
4121        }
4122
4123        let mut hasher = blake3::Hasher::new();
4124
4125        hasher.update(&carats.to_le_bytes());
4126
4127        hasher.update(&slot.to_le_bytes());
4128
4129        hasher.update(&rent_epoch.to_le_bytes());
4130
4131        hasher.update(data);
4132
4133        if executable {
4134            hasher.update(&[1u8; 1]);
4135        } else {
4136            hasher.update(&[0u8; 1]);
4137        }
4138
4139        hasher.update(owner.as_ref());
4140        hasher.update(pubkey.as_ref());
4141
4142        Hash(<[u8; gemachain_sdk::hash::HASH_BYTES]>::try_from(hasher.finalize().as_slice()).unwrap())
4143    }
4144
4145    fn bulk_assign_write_version(&self, count: usize) -> StoredMetaWriteVersion {
4146        self.write_version
4147            .fetch_add(count as StoredMetaWriteVersion, Ordering::Relaxed)
4148    }
4149
4150    fn write_accounts_to_storage<F: FnMut(Slot, usize) -> Arc<AccountStorageEntry>>(
4151        &self,
4152        slot: Slot,
4153        hashes: &[impl Borrow<Hash>],
4154        mut storage_finder: F,
4155        accounts_and_meta_to_store: &[(StoredMeta, Option<&impl ReadableAccount>)],
4156    ) -> Vec<AccountInfo> {
4157        assert_eq!(hashes.len(), accounts_and_meta_to_store.len());
4158        let mut infos: Vec<AccountInfo> = Vec::with_capacity(accounts_and_meta_to_store.len());
4159        let mut total_append_accounts_us = 0;
4160        let mut total_storage_find_us = 0;
4161        while infos.len() < accounts_and_meta_to_store.len() {
4162            let mut storage_find = Measure::start("storage_finder");
4163            let data_len = accounts_and_meta_to_store[infos.len()]
4164                .1
4165                .map(|account| account.data().len())
4166                .unwrap_or_default();
4167            let storage = storage_finder(slot, data_len + STORE_META_OVERHEAD);
4168            storage_find.stop();
4169            total_storage_find_us += storage_find.as_us();
4170            let mut append_accounts = Measure::start("append_accounts");
4171            let rvs = storage.accounts.append_accounts(
4172                &accounts_and_meta_to_store[infos.len()..],
4173                &hashes[infos.len()..],
4174            );
4175            assert!(!rvs.is_empty());
4176            append_accounts.stop();
4177            total_append_accounts_us += append_accounts.as_us();
4178            if rvs.len() == 1 {
4179                storage.set_status(AccountStorageStatus::Full);
4180
4181                // See if an account overflows the append vecs in the slot.
4182                let data_len = (data_len + STORE_META_OVERHEAD) as u64;
4183                if !self.has_space_available(slot, data_len) {
4184                    let special_store_size = std::cmp::max(data_len * 2, self.file_size);
4185                    if self
4186                        .try_recycle_and_insert_store(slot, special_store_size, std::u64::MAX)
4187                        .is_none()
4188                    {
4189                        self.stats
4190                            .create_store_count
4191                            .fetch_add(1, Ordering::Relaxed);
4192                        self.create_and_insert_store(slot, special_store_size, "large create");
4193                    } else {
4194                        self.stats
4195                            .recycle_store_count
4196                            .fetch_add(1, Ordering::Relaxed);
4197                    }
4198                }
4199                continue;
4200            }
4201
4202            for (offsets, (_, account)) in rvs
4203                .windows(2)
4204                .zip(&accounts_and_meta_to_store[infos.len()..])
4205            {
4206                let stored_size = offsets[1] - offsets[0];
4207                storage.add_account(stored_size);
4208                infos.push(AccountInfo {
4209                    store_id: storage.append_vec_id(),
4210                    offset: offsets[0],
4211                    stored_size,
4212                    carats: account
4213                        .map(|account| account.carats())
4214                        .unwrap_or_default(),
4215                });
4216            }
4217            // restore the state to available
4218            storage.set_status(AccountStorageStatus::Available);
4219        }
4220
4221        self.stats
4222            .store_append_accounts
4223            .fetch_add(total_append_accounts_us, Ordering::Relaxed);
4224        self.stats
4225            .store_find_store
4226            .fetch_add(total_storage_find_us, Ordering::Relaxed);
4227        infos
4228    }
4229
4230    pub fn mark_slot_frozen(&self, slot: Slot) {
4231        if let Some(slot_cache) = self.accounts_cache.slot_cache(slot) {
4232            slot_cache.mark_slot_frozen();
4233            slot_cache.report_slot_store_metrics();
4234        }
4235        self.accounts_cache.report_size();
4236    }
4237
4238    pub fn expire_old_recycle_stores(&self) {
4239        let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
4240        let recycle_stores = self.recycle_stores.write().unwrap().expire_old_entries();
4241        recycle_stores_write_elapsed.stop();
4242
4243        let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
4244        drop(recycle_stores);
4245        drop_storage_entries_elapsed.stop();
4246
4247        self.clean_accounts_stats
4248            .purge_stats
4249            .drop_storage_entries_elapsed
4250            .fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
4251        self.clean_accounts_stats
4252            .purge_stats
4253            .recycle_stores_write_elapsed
4254            .fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
4255    }
4256
4257    pub fn flush_accounts_cache_slot(&self, slot: Slot) {
4258        self.flush_slot_cache(slot, None::<&mut fn(&_, &_) -> bool>);
4259    }
4260
4261    // `force_flush` flushes all the cached roots `<= requested_flush_root`. It also then
4262    // flushes:
4263    // 1) Any remaining roots if there are > MAX_CACHE_SLOTS remaining slots in the cache,
4264    // 2) It there are still > MAX_CACHE_SLOTS remaining slots in the cache, the excess
4265    // unrooted slots
4266    pub fn flush_accounts_cache(&self, force_flush: bool, requested_flush_root: Option<Slot>) {
4267        #[cfg(not(test))]
4268        assert!(requested_flush_root.is_some());
4269
4270        if !force_flush && self.accounts_cache.num_slots() <= MAX_CACHE_SLOTS {
4271            return;
4272        }
4273
4274        // Flush only the roots <= requested_flush_root, so that snapshotting has all
4275        // the relevant roots in storage.
4276        let mut flush_roots_elapsed = Measure::start("flush_roots_elapsed");
4277        let mut account_bytes_saved = 0;
4278        let mut num_accounts_saved = 0;
4279
4280        // Note even if force_flush is false, we will still flush all roots <= the
4281        // given `requested_flush_root`, even if some of the later roots cannot be used for
4282        // cleaning due to an ongoing scan
4283        let (total_new_cleaned_roots, num_cleaned_roots_flushed) = self
4284            .flush_rooted_accounts_cache(
4285                requested_flush_root,
4286                Some((&mut account_bytes_saved, &mut num_accounts_saved)),
4287            );
4288        flush_roots_elapsed.stop();
4289
4290        // Note we don't purge unrooted slots here because there may be ongoing scans/references
4291        // for those slot, let the Bank::drop() implementation do cleanup instead on dead
4292        // banks
4293
4294        // If there are > MAX_CACHE_SLOTS, then flush the excess ones to storage
4295        let (total_new_excess_roots, num_excess_roots_flushed) =
4296            if self.accounts_cache.num_slots() > MAX_CACHE_SLOTS {
4297                // Start by flushing the roots
4298                //
4299                // Cannot do any cleaning on roots past `requested_flush_root` because future
4300                // snapshots may need updates from those later slots, hence we pass `None`
4301                // for `should_clean`.
4302                self.flush_rooted_accounts_cache(None, None)
4303            } else {
4304                (0, 0)
4305            };
4306        let old_slots = self.accounts_cache.find_older_frozen_slots(MAX_CACHE_SLOTS);
4307        let excess_slot_count = old_slots.len();
4308        let mut unflushable_unrooted_slot_count = 0;
4309        let max_flushed_root = self.accounts_cache.fetch_max_flush_root();
4310        let old_slot_flush_stats: Vec<_> = old_slots
4311            .into_iter()
4312            .filter_map(|old_slot| {
4313                // Don't flush slots that are known to be unrooted
4314                if old_slot > max_flushed_root {
4315                    Some(self.flush_slot_cache(old_slot, None::<&mut fn(&_, &_) -> bool>))
4316                } else {
4317                    unflushable_unrooted_slot_count += 1;
4318                    None
4319                }
4320            })
4321            .collect();
4322        info!(
4323            "req_flush_root: {:?} old_slot_flushes: {:?}",
4324            requested_flush_root, old_slot_flush_stats
4325        );
4326
4327        datapoint_info!(
4328            "accounts_db-flush_accounts_cache",
4329            ("total_new_cleaned_roots", total_new_cleaned_roots, i64),
4330            ("num_cleaned_roots_flushed", num_cleaned_roots_flushed, i64),
4331            ("total_new_excess_roots", total_new_excess_roots, i64),
4332            ("num_excess_roots_flushed", num_excess_roots_flushed, i64),
4333            ("excess_slot_count", excess_slot_count, i64),
4334            (
4335                "unflushable_unrooted_slot_count",
4336                unflushable_unrooted_slot_count,
4337                i64
4338            ),
4339            (
4340                "flush_roots_elapsed",
4341                flush_roots_elapsed.as_us() as i64,
4342                i64
4343            ),
4344            ("account_bytes_saved", account_bytes_saved, i64),
4345            ("num_accounts_saved", num_accounts_saved, i64),
4346        );
4347
4348        // Flush a random slot out after every force flush to catch any inconsistencies
4349        // between cache and written state (i.e. should cause a hash mismatch between validators
4350        // that flush and don't flush if such a bug exists).
4351        let num_slots_remaining = self.accounts_cache.num_slots();
4352        if force_flush && num_slots_remaining >= FLUSH_CACHE_RANDOM_THRESHOLD {
4353            // Don't flush slots that are known to be unrooted
4354            let mut frozen_slots = self.accounts_cache.find_older_frozen_slots(0);
4355            frozen_slots.retain(|s| *s > max_flushed_root);
4356            // Remove a random index 0 <= i < `frozen_slots.len()`
4357            let rand_slot = frozen_slots.choose(&mut thread_rng());
4358            if let Some(rand_slot) = rand_slot {
4359                let random_flush_stats =
4360                    self.flush_slot_cache(*rand_slot, None::<&mut fn(&_, &_) -> bool>);
4361                info!(
4362                    "Flushed random slot: num_remaining: {} {:?}",
4363                    num_slots_remaining, random_flush_stats,
4364                );
4365            }
4366        }
4367    }
4368
4369    fn flush_rooted_accounts_cache(
4370        &self,
4371        requested_flush_root: Option<Slot>,
4372        should_clean: Option<(&mut usize, &mut usize)>,
4373    ) -> (usize, usize) {
4374        let max_clean_root = should_clean.as_ref().and_then(|_| {
4375            // If there is a long running scan going on, this could prevent any cleaning
4376            // based on updates from slots > `max_clean_root`.
4377            self.max_clean_root(requested_flush_root)
4378        });
4379
4380        // Use HashMap because HashSet doesn't provide Entry api
4381        let mut written_accounts = HashMap::new();
4382
4383        // If `should_clean` is None, then`should_flush_f` is also None, which will cause
4384        // `flush_slot_cache` to flush all accounts to storage without cleaning any accounts.
4385        let mut should_flush_f = should_clean.map(|(account_bytes_saved, num_accounts_saved)| {
4386            move |&pubkey: &Pubkey, account: &AccountSharedData| {
4387                use std::collections::hash_map::Entry::{Occupied, Vacant};
4388                let should_flush = match written_accounts.entry(pubkey) {
4389                    Vacant(vacant_entry) => {
4390                        vacant_entry.insert(());
4391                        true
4392                    }
4393                    Occupied(_occupied_entry) => {
4394                        *account_bytes_saved += account.data().len();
4395                        *num_accounts_saved += 1;
4396                        // If a later root already wrote this account, no point
4397                        // in flushing it
4398                        false
4399                    }
4400                };
4401                should_flush
4402            }
4403        });
4404
4405        // Always flush up to `requested_flush_root`, which is necessary for things like snapshotting.
4406        let cached_roots: BTreeSet<Slot> = self.accounts_cache.clear_roots(requested_flush_root);
4407
4408        // Iterate from highest to lowest so that we don't need to flush earlier
4409        // outdated updates in earlier roots
4410        let mut num_roots_flushed = 0;
4411        for &root in cached_roots.iter().rev() {
4412            let should_flush_f = if let Some(max_clean_root) = max_clean_root {
4413                if root > max_clean_root {
4414                    // Only if the root is greater than the `max_clean_root` do we
4415                    // have to prevent cleaning, otherwise, just default to `should_flush_f`
4416                    // for any slots <= `max_clean_root`
4417                    None
4418                } else {
4419                    should_flush_f.as_mut()
4420                }
4421            } else {
4422                should_flush_f.as_mut()
4423            };
4424
4425            if self.flush_slot_cache(root, should_flush_f).is_some() {
4426                num_roots_flushed += 1;
4427            }
4428
4429            // Regardless of whether this slot was *just* flushed from the cache by the above
4430            // `flush_slot_cache()`, we should update the `max_flush_root`.
4431            // This is because some rooted slots may be flushed to storage *before* they are marked as root.
4432            // This can occur for instance when:
4433            // 1) The cache is overwhelmed, we we flushed some yet to be rooted frozen slots
4434            // 2) Random evictions
4435            // These slots may then *later* be marked as root, so we still need to handle updating the
4436            // `max_flush_root` in the accounts cache.
4437            self.accounts_cache.set_max_flush_root(root);
4438        }
4439
4440        // Only add to the uncleaned roots set *after* we've flushed the previous roots,
4441        // so that clean will actually be able to clean the slots.
4442        let num_new_roots = cached_roots.len();
4443        self.accounts_index.add_uncleaned_roots(cached_roots);
4444        (num_new_roots, num_roots_flushed)
4445    }
4446
4447    fn do_flush_slot_cache(
4448        &self,
4449        slot: Slot,
4450        slot_cache: &SlotCache,
4451        mut should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
4452    ) -> FlushStats {
4453        let mut num_purged = 0;
4454        let mut total_size = 0;
4455        let mut num_flushed = 0;
4456        let iter_items: Vec<_> = slot_cache.iter().collect();
4457        let mut purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = HashSet::new();
4458        let mut pubkey_to_slot_set: Vec<(Pubkey, Slot)> = vec![];
4459        let (accounts, hashes): (Vec<(&Pubkey, &AccountSharedData)>, Vec<Hash>) = iter_items
4460            .iter()
4461            .filter_map(|iter_item| {
4462                let key = iter_item.key();
4463                let account = &iter_item.value().account;
4464                let should_flush = should_flush_f
4465                    .as_mut()
4466                    .map(|should_flush_f| should_flush_f(key, account))
4467                    .unwrap_or(true);
4468                if should_flush {
4469                    let hash = iter_item.value().hash();
4470                    total_size += (account.data().len() + STORE_META_OVERHEAD) as u64;
4471                    num_flushed += 1;
4472                    Some(((key, account), hash))
4473                } else {
4474                    // If we don't flush, we have to remove the entry from the
4475                    // index, since it's equivalent to purging
4476                    purged_slot_pubkeys.insert((slot, *key));
4477                    pubkey_to_slot_set.push((*key, slot));
4478                    num_purged += 1;
4479                    None
4480                }
4481            })
4482            .unzip();
4483
4484        let is_dead_slot = accounts.is_empty();
4485        // Remove the account index entries from earlier roots that are outdated by later roots.
4486        // Safe because queries to the index will be reading updates from later roots.
4487        self.purge_slot_cache_pubkeys(slot, purged_slot_pubkeys, pubkey_to_slot_set, is_dead_slot);
4488
4489        if !is_dead_slot {
4490            let aligned_total_size = Self::page_align(total_size);
4491            // This ensures that all updates are written to an AppendVec, before any
4492            // updates to the index happen, so anybody that sees a real entry in the index,
4493            // will be able to find the account in storage
4494            let flushed_store =
4495                self.create_and_insert_store(slot, aligned_total_size, "flush_slot_cache");
4496            self.store_accounts_frozen(
4497                slot,
4498                &accounts,
4499                Some(&hashes),
4500                Some(Box::new(move |_, _| flushed_store.clone())),
4501                None,
4502            );
4503            // If the above sizing function is correct, just one AppendVec is enough to hold
4504            // all the data for the slot
4505            assert_eq!(
4506                self.storage
4507                    .get_slot_stores(slot)
4508                    .unwrap()
4509                    .read()
4510                    .unwrap()
4511                    .len(),
4512                1
4513            );
4514        }
4515
4516        // Remove this slot from the cache, which will to AccountsDb's new readers should look like an
4517        // atomic switch from the cache to storage.
4518        // There is some racy condition for existing readers who just has read exactly while
4519        // flushing. That case is handled by retry_to_get_account_accessor()
4520        assert!(self.accounts_cache.remove_slot(slot).is_some());
4521        FlushStats {
4522            slot,
4523            num_flushed,
4524            num_purged,
4525            total_size,
4526        }
4527    }
4528
4529    /// `should_flush_f` is an optional closure that determines whether a given
4530    /// account should be flushed. Passing `None` will by default flush all
4531    /// accounts
4532    fn flush_slot_cache(
4533        &self,
4534        slot: Slot,
4535        should_flush_f: Option<&mut impl FnMut(&Pubkey, &AccountSharedData) -> bool>,
4536    ) -> Option<FlushStats> {
4537        let is_being_purged = {
4538            let mut slots_under_contention = self
4539                .remove_unrooted_slots_synchronization
4540                .slots_under_contention
4541                .lock()
4542                .unwrap();
4543            // If we're purging this slot, don't flush it here
4544            if slots_under_contention.contains(&slot) {
4545                true
4546            } else {
4547                slots_under_contention.insert(slot);
4548                false
4549            }
4550        };
4551
4552        if !is_being_purged {
4553            let flush_stats = self.accounts_cache.slot_cache(slot).map(|slot_cache| {
4554                #[cfg(test)]
4555                {
4556                    // Give some time for cache flushing to occur here for unit tests
4557                    sleep(Duration::from_millis(self.load_delay));
4558                }
4559                // Since we added the slot to `slots_under_contention` AND this slot
4560                // still exists in the cache, we know the slot cannot be removed
4561                // by any other threads past this point. We are now responsible for
4562                // flushing this slot.
4563                self.do_flush_slot_cache(slot, &slot_cache, should_flush_f)
4564            });
4565
4566            // Nobody else should have been purging this slot, so should not have been removed
4567            // from `self.remove_unrooted_slots_synchronization`.
4568            assert!(self
4569                .remove_unrooted_slots_synchronization
4570                .slots_under_contention
4571                .lock()
4572                .unwrap()
4573                .remove(&slot));
4574
4575            // Signal to any threads blocked on `remove_unrooted_slots(slot)` that we have finished
4576            // flushing
4577            self.remove_unrooted_slots_synchronization
4578                .signal
4579                .notify_all();
4580            flush_stats
4581        } else {
4582            None
4583        }
4584    }
4585
4586    fn write_accounts_to_cache(
4587        &self,
4588        slot: Slot,
4589        hashes: Option<&[impl Borrow<Hash>]>,
4590        accounts_and_meta_to_store: &[(StoredMeta, Option<&impl ReadableAccount>)],
4591    ) -> Vec<AccountInfo> {
4592        let len = accounts_and_meta_to_store.len();
4593        let hashes = hashes.map(|hashes| {
4594            assert_eq!(hashes.len(), len);
4595            hashes
4596        });
4597
4598        accounts_and_meta_to_store
4599            .iter()
4600            .enumerate()
4601            .map(|(i, (meta, account))| {
4602                let hash = hashes.map(|hashes| hashes[i].borrow());
4603
4604                let account = account
4605                    .map(|account| account.to_account_shared_data())
4606                    .unwrap_or_default();
4607                let account_info = AccountInfo {
4608                    store_id: CACHE_VIRTUAL_STORAGE_ID,
4609                    offset: CACHE_VIRTUAL_OFFSET,
4610                    stored_size: CACHE_VIRTUAL_STORED_SIZE,
4611                    carats: account.carats(),
4612                };
4613
4614                let cached_account = self.accounts_cache.store(slot, &meta.pubkey, account, hash);
4615                // hash this account in the bg
4616                match &self.sender_bg_hasher {
4617                    Some(ref sender) => {
4618                        let _ = sender.send(cached_account);
4619                    }
4620                    None => (),
4621                };
4622                account_info
4623            })
4624            .collect()
4625    }
4626
4627    fn store_accounts_to<
4628        F: FnMut(Slot, usize) -> Arc<AccountStorageEntry>,
4629        P: Iterator<Item = u64>,
4630    >(
4631        &self,
4632        slot: Slot,
4633        accounts: &[(&Pubkey, &impl ReadableAccount)],
4634        hashes: Option<&[impl Borrow<Hash>]>,
4635        storage_finder: F,
4636        mut write_version_producer: P,
4637        is_cached_store: bool,
4638    ) -> Vec<AccountInfo> {
4639        let mut calc_stored_meta_time = Measure::start("calc_stored_meta");
4640        let accounts_and_meta_to_store: Vec<_> = accounts
4641            .iter()
4642            .map(|(pubkey, account)| {
4643                self.read_only_accounts_cache.remove(pubkey, slot);
4644                // this is the source of Some(Account) or None.
4645                // Some(Account) = store 'Account'
4646                // None = store a default/empty account with 0 carats
4647                let (account, data_len) = if account.carats() == 0 {
4648                    (None, 0)
4649                } else {
4650                    (Some(*account), account.data().len() as u64)
4651                };
4652                let meta = StoredMeta {
4653                    write_version: write_version_producer.next().unwrap(),
4654                    pubkey: **pubkey,
4655                    data_len,
4656                };
4657                (meta, account)
4658            })
4659            .collect();
4660        calc_stored_meta_time.stop();
4661        self.stats
4662            .calc_stored_meta
4663            .fetch_add(calc_stored_meta_time.as_us(), Ordering::Relaxed);
4664
4665        if self.caching_enabled && is_cached_store {
4666            self.write_accounts_to_cache(slot, hashes, &accounts_and_meta_to_store)
4667        } else {
4668            match hashes {
4669                Some(hashes) => self.write_accounts_to_storage(
4670                    slot,
4671                    hashes,
4672                    storage_finder,
4673                    &accounts_and_meta_to_store,
4674                ),
4675                None => {
4676                    // hash any accounts where we were lazy in calculating the hash
4677                    let mut hash_time = Measure::start("hash_accounts");
4678                    let mut stats = BankHashStats::default();
4679                    let len = accounts_and_meta_to_store.len();
4680                    let mut hashes = Vec::with_capacity(len);
4681                    for account in accounts {
4682                        stats.update(account.1);
4683                        let hash = Self::hash_account(slot, account.1, account.0);
4684                        hashes.push(hash);
4685                    }
4686                    hash_time.stop();
4687                    self.stats
4688                        .store_hash_accounts
4689                        .fetch_add(hash_time.as_us(), Ordering::Relaxed);
4690
4691                    self.write_accounts_to_storage(
4692                        slot,
4693                        &hashes,
4694                        storage_finder,
4695                        &accounts_and_meta_to_store,
4696                    )
4697                }
4698            }
4699        }
4700    }
4701
4702    fn report_store_stats(&self) {
4703        let mut total_count = 0;
4704        let mut min = std::usize::MAX;
4705        let mut min_slot = 0;
4706        let mut max = 0;
4707        let mut max_slot = 0;
4708        let mut newest_slot = 0;
4709        let mut oldest_slot = std::u64::MAX;
4710        let mut total_bytes = 0;
4711        let mut total_alive_bytes = 0;
4712        for iter_item in self.storage.0.iter() {
4713            let slot = iter_item.key();
4714            let slot_stores = iter_item.value().read().unwrap();
4715            total_count += slot_stores.len();
4716            if slot_stores.len() < min {
4717                min = slot_stores.len();
4718                min_slot = *slot;
4719            }
4720
4721            if slot_stores.len() > max {
4722                max = slot_stores.len();
4723                max_slot = *slot;
4724            }
4725            if *slot > newest_slot {
4726                newest_slot = *slot;
4727            }
4728
4729            if *slot < oldest_slot {
4730                oldest_slot = *slot;
4731            }
4732
4733            for store in slot_stores.values() {
4734                total_alive_bytes += Self::page_align(store.alive_bytes() as u64);
4735                total_bytes += store.total_bytes();
4736            }
4737        }
4738        info!("total_stores: {}, newest_slot: {}, oldest_slot: {}, max_slot: {} (num={}), min_slot: {} (num={})",
4739              total_count, newest_slot, oldest_slot, max_slot, max, min_slot, min);
4740
4741        let total_alive_ratio = if total_bytes > 0 {
4742            total_alive_bytes as f64 / total_bytes as f64
4743        } else {
4744            0.
4745        };
4746
4747        datapoint_info!(
4748            "accounts_db-stores",
4749            ("total_count", total_count, i64),
4750            (
4751                "recycle_count",
4752                self.recycle_stores.read().unwrap().entry_count() as u64,
4753                i64
4754            ),
4755            ("total_bytes", total_bytes, i64),
4756            ("total_alive_bytes", total_alive_bytes, i64),
4757            ("total_alive_ratio", total_alive_ratio, f64),
4758        );
4759        datapoint_info!(
4760            "accounts_db-perf-stats",
4761            (
4762                "delta_hash_num",
4763                self.stats.delta_hash_num.swap(0, Ordering::Relaxed),
4764                i64
4765            ),
4766            (
4767                "delta_hash_scan_us",
4768                self.stats
4769                    .delta_hash_scan_time_total_us
4770                    .swap(0, Ordering::Relaxed),
4771                i64
4772            ),
4773            (
4774                "delta_hash_accumulate_us",
4775                self.stats
4776                    .delta_hash_accumulate_time_total_us
4777                    .swap(0, Ordering::Relaxed),
4778                i64
4779            ),
4780        );
4781    }
4782
4783    pub fn checked_iterative_sum_for_capitalization(total_cap: u64, new_cap: u64) -> u64 {
4784        let new_total = total_cap as u128 + new_cap as u128;
4785        AccountsHash::checked_cast_for_capitalization(new_total)
4786    }
4787
4788    pub fn checked_sum_for_capitalization<T: Iterator<Item = u64>>(balances: T) -> u64 {
4789        AccountsHash::checked_cast_for_capitalization(balances.map(|b| b as u128).sum::<u128>())
4790    }
4791
4792    fn calculate_accounts_hash(
4793        &self,
4794        slot: Slot,
4795        ancestors: &Ancestors,
4796        check_hash: bool,
4797    ) -> Result<(Hash, u64), BankHashVerificationError> {
4798        use BankHashVerificationError::*;
4799        let mut collect = Measure::start("collect");
4800        let keys: Vec<_> = self
4801            .accounts_index
4802            .account_maps
4803            .iter()
4804            .map(|map| {
4805                let mut keys = map.read().unwrap().keys();
4806                keys.sort_unstable(); // hashmap is not ordered, but bins are relative to each other
4807                keys
4808            })
4809            .flatten()
4810            .collect();
4811        collect.stop();
4812
4813        let mut scan = Measure::start("scan");
4814        let mismatch_found = AtomicU64::new(0);
4815        // Pick a chunk size big enough to allow us to produce output vectors that are smaller than the overall size.
4816        // We'll also accumulate the carats within each chunk and fewer chunks results in less contention to accumulate the sum.
4817        let chunks = crate::accounts_hash::MERKLE_FANOUT.pow(4);
4818        let total_carats = Mutex::<u64>::new(0);
4819        let get_hashes = || {
4820            keys.par_chunks(chunks)
4821                .map(|pubkeys| {
4822                    let mut sum = 0u128;
4823                    let result: Vec<Hash> = pubkeys
4824                        .iter()
4825                        .filter_map(|pubkey| {
4826                            if let AccountIndexGetResult::Found(lock, index) =
4827                                self.accounts_index.get(pubkey, Some(ancestors), Some(slot))
4828                            {
4829                                let (slot, account_info) = &lock.slot_list()[index];
4830                                if account_info.carats != 0 {
4831                                    // Because we're keeping the `lock' here, there is no need
4832                                    // to use retry_to_get_account_accessor()
4833                                    // In other words, flusher/shrinker/cleaner is blocked to
4834                                    // cause any Accessor(None) situtation.
4835                                    // Anyway this race condition concern is currently a moot
4836                                    // point because calculate_accounts_hash() should not
4837                                    // currently race with clean/shrink because the full hash
4838                                    // is synchronous with clean/shrink in
4839                                    // AccountsBackgroundService
4840                                    self.get_account_accessor(
4841                                        *slot,
4842                                        pubkey,
4843                                        account_info.store_id,
4844                                        account_info.offset,
4845                                    )
4846                                    .get_loaded_account()
4847                                    .and_then(
4848                                        |loaded_account| {
4849                                            let loaded_hash = loaded_account.loaded_hash();
4850                                            let balance = account_info.carats;
4851                                            if check_hash {
4852                                                let computed_hash =
4853                                                    loaded_account.compute_hash(*slot, pubkey);
4854                                                if computed_hash != loaded_hash {
4855                                                    info!("hash mismatch found: computed: {}, loaded: {}, pubkey: {}", computed_hash, loaded_hash, pubkey);
4856                                                    mismatch_found
4857                                                        .fetch_add(1, Ordering::Relaxed);
4858                                                    return None;
4859                                                }
4860                                            }
4861
4862                                            sum += balance as u128;
4863                                            Some(loaded_hash)
4864                                        },
4865                                    )
4866                                } else {
4867                                    None
4868                                }
4869                            } else {
4870                                None
4871                            }
4872                        })
4873                        .collect();
4874                    let mut total = total_carats.lock().unwrap();
4875                    *total =
4876                        AccountsHash::checked_cast_for_capitalization(*total as u128 + sum);
4877                    result
4878                }).collect()
4879        };
4880
4881        let hashes: Vec<Vec<Hash>> = if check_hash {
4882            get_hashes()
4883        } else {
4884            self.thread_pool_clean.install(get_hashes)
4885        };
4886        if mismatch_found.load(Ordering::Relaxed) > 0 {
4887            warn!(
4888                "{} mismatched account hash(es) found",
4889                mismatch_found.load(Ordering::Relaxed)
4890            );
4891            return Err(MismatchedAccountHash);
4892        }
4893
4894        scan.stop();
4895        let total_carats = *total_carats.lock().unwrap();
4896
4897        let mut hash_time = Measure::start("hash");
4898        let (accumulated_hash, hash_total) = AccountsHash::calculate_hash(hashes);
4899        hash_time.stop();
4900        datapoint_info!(
4901            "update_accounts_hash",
4902            ("accounts_scan", scan.as_us(), i64),
4903            ("hash", hash_time.as_us(), i64),
4904            ("hash_total", hash_total, i64),
4905            ("collect", collect.as_us(), i64),
4906        );
4907        Ok((accumulated_hash, total_carats))
4908    }
4909
4910    pub fn get_accounts_hash(&self, slot: Slot) -> Hash {
4911        let bank_hashes = self.bank_hashes.read().unwrap();
4912        let bank_hash_info = bank_hashes.get(&slot).unwrap();
4913        bank_hash_info.snapshot_hash
4914    }
4915
4916    pub fn update_accounts_hash(&self, slot: Slot, ancestors: &Ancestors) -> (Hash, u64) {
4917        self.update_accounts_hash_with_index_option(true, false, slot, ancestors, None, false, None)
4918    }
4919
4920    pub fn update_accounts_hash_test(&self, slot: Slot, ancestors: &Ancestors) -> (Hash, u64) {
4921        self.update_accounts_hash_with_index_option(true, true, slot, ancestors, None, false, None)
4922    }
4923
4924    fn scan_multiple_account_storages_one_slot<F, B>(
4925        storages: &[Arc<AccountStorageEntry>],
4926        scan_func: &F,
4927        slot: Slot,
4928        retval: &mut B,
4929    ) where
4930        F: Fn(LoadedAccount, &mut B, Slot) + Send + Sync,
4931        B: Send + Default,
4932    {
4933        // we have to call the scan_func in order of write_version within a slot if there are multiple storages per slot
4934        let mut len = storages.len();
4935        let mut progress = Vec::with_capacity(len);
4936        let mut current = Vec::with_capacity(len);
4937        for storage in storages {
4938            let accounts = storage.accounts.accounts(0);
4939            let mut iterator: std::vec::IntoIter<StoredAccountMeta<'_>> = accounts.into_iter();
4940            if let Some(item) = iterator
4941                .next()
4942                .map(|stored_account| (stored_account.meta.write_version, Some(stored_account)))
4943            {
4944                current.push(item);
4945                progress.push(iterator);
4946            }
4947        }
4948        while !progress.is_empty() {
4949            let mut min = current[0].0;
4950            let mut min_index = 0;
4951            for (i, (item, _)) in current.iter().enumerate().take(len).skip(1) {
4952                if item < &min {
4953                    min_index = i;
4954                    min = *item;
4955                }
4956            }
4957            let mut account = (0, None);
4958            std::mem::swap(&mut account, &mut current[min_index]);
4959            scan_func(LoadedAccount::Stored(account.1.unwrap()), retval, slot);
4960            let next = progress[min_index]
4961                .next()
4962                .map(|stored_account| (stored_account.meta.write_version, Some(stored_account)));
4963            match next {
4964                Some(item) => {
4965                    current[min_index] = item;
4966                }
4967                None => {
4968                    current.remove(min_index);
4969                    progress.remove(min_index);
4970                    len -= 1;
4971                }
4972            }
4973        }
4974    }
4975
4976    /// Scan through all the account storage in parallel
4977    fn scan_account_storage_no_bank<F, F2>(
4978        cache_hash_data: &CacheHashData,
4979        accounts_cache_and_ancestors: Option<(
4980            &AccountsCache,
4981            &Ancestors,
4982            &AccountInfoAccountsIndex,
4983        )>,
4984        snapshot_storages: &SortedStorages,
4985        scan_func: F,
4986        after_func: F2,
4987        bin_range: &Range<usize>,
4988        bin_calculator: &PubkeyBinCalculator16,
4989    ) -> Vec<BinnedHashData>
4990    where
4991        F: Fn(LoadedAccount, &mut BinnedHashData, Slot) + Send + Sync,
4992        F2: Fn(BinnedHashData) -> BinnedHashData + Send + Sync,
4993    {
4994        let start_bin_index = bin_range.start;
4995
4996        let width = snapshot_storages.range_width();
4997        // 2 is for 2 special chunks - unaligned slots at the beginning and end
4998        let chunks = 2 + (width as Slot / MAX_ITEMS_PER_CHUNK);
4999        let range = snapshot_storages.range();
5000        let slot0 = range.start;
5001        let first_boundary =
5002            ((slot0 + MAX_ITEMS_PER_CHUNK) / MAX_ITEMS_PER_CHUNK) * MAX_ITEMS_PER_CHUNK;
5003        (0..chunks)
5004            .into_par_iter()
5005            .map(|chunk| {
5006                let mut retval = vec![];
5007                // calculate start, end
5008                let (start, mut end) = if chunk == 0 {
5009                    if slot0 == first_boundary {
5010                        return after_func(retval); // if we evenly divide, nothing for special chunk 0 to do
5011                    }
5012                    // otherwise first chunk is not 'full'
5013                    (slot0, first_boundary)
5014                } else {
5015                    // normal chunk in the middle or at the end
5016                    let start = first_boundary + MAX_ITEMS_PER_CHUNK * (chunk - 1);
5017                    let end = start + MAX_ITEMS_PER_CHUNK;
5018                    (start, end)
5019                };
5020                end = std::cmp::min(end, range.end);
5021                if start == end {
5022                    return after_func(retval);
5023                }
5024
5025                let mut file_name = String::default();
5026                if accounts_cache_and_ancestors.is_none()
5027                    && end.saturating_sub(start) == MAX_ITEMS_PER_CHUNK
5028                {
5029                    let mut load_from_cache = true;
5030                    let mut hasher = std::collections::hash_map::DefaultHasher::new(); // wrong one?
5031
5032                    for slot in start..end {
5033                        let sub_storages = snapshot_storages.get(slot);
5034                        bin_range.start.hash(&mut hasher);
5035                        bin_range.end.hash(&mut hasher);
5036                        if let Some(sub_storages) = sub_storages {
5037                            if sub_storages.len() > 1 {
5038                                load_from_cache = false;
5039                                break;
5040                            }
5041                            let storage_file = sub_storages.first().unwrap().accounts.get_path();
5042                            slot.hash(&mut hasher);
5043                            storage_file.hash(&mut hasher);
5044                            // check alive_bytes, etc. here?
5045                            let amod = std::fs::metadata(storage_file);
5046                            if amod.is_err() {
5047                                load_from_cache = false;
5048                                break;
5049                            }
5050                            let amod = amod.unwrap().modified();
5051                            if amod.is_err() {
5052                                load_from_cache = false;
5053                                break;
5054                            }
5055                            let amod = amod
5056                                .unwrap()
5057                                .duration_since(std::time::UNIX_EPOCH)
5058                                .unwrap()
5059                                .as_secs();
5060                            amod.hash(&mut hasher);
5061                        }
5062                    }
5063                    if load_from_cache {
5064                        // we have a hash value for all the storages in this slot
5065                        // so, build a file name:
5066                        let hash = hasher.finish();
5067                        file_name = format!(
5068                            "{}.{}.{}.{}.{}",
5069                            start, end, bin_range.start, bin_range.end, hash
5070                        );
5071                        if retval.is_empty() {
5072                            let range = bin_range.end - bin_range.start;
5073                            retval.append(&mut vec![Vec::new(); range]);
5074                        }
5075                        if cache_hash_data
5076                            .load(
5077                                &Path::new(&file_name),
5078                                &mut retval,
5079                                start_bin_index,
5080                                bin_calculator,
5081                            )
5082                            .is_ok()
5083                        {
5084                            return retval;
5085                        }
5086
5087                        // fall through and load normally - we failed to load
5088                    }
5089                }
5090
5091                for slot in start..end {
5092                    let sub_storages = snapshot_storages.get(slot);
5093                    let valid_slot = sub_storages.is_some();
5094                    if let Some((cache, ancestors, accounts_index)) = accounts_cache_and_ancestors {
5095                        if let Some(slot_cache) = cache.slot_cache(slot) {
5096                            if valid_slot
5097                                || ancestors.contains_key(&slot)
5098                                || accounts_index.is_root(slot)
5099                            {
5100                                let keys = slot_cache.get_all_pubkeys();
5101                                for key in keys {
5102                                    if let Some(cached_account) = slot_cache.get_cloned(&key) {
5103                                        let mut accessor = LoadedAccountAccessor::Cached(Some((
5104                                            key,
5105                                            Cow::Owned(cached_account),
5106                                        )));
5107                                        let account = accessor.get_loaded_account().unwrap();
5108                                        scan_func(account, &mut retval, slot);
5109                                    };
5110                                }
5111                            }
5112                        }
5113                    }
5114
5115                    if let Some(sub_storages) = sub_storages {
5116                        Self::scan_multiple_account_storages_one_slot(
5117                            sub_storages,
5118                            &scan_func,
5119                            slot,
5120                            &mut retval,
5121                        );
5122                    }
5123                }
5124                let r = after_func(retval);
5125                if !file_name.is_empty() {
5126                    let result = cache_hash_data.save(Path::new(&file_name), &r);
5127
5128                    if result.is_err() {
5129                        info!(
5130                            "FAILED_TO_SAVE: {}-{}, {}, first_boundary: {}, {:?}",
5131                            range.start, range.end, width, first_boundary, file_name,
5132                        );
5133                    }
5134                }
5135                r
5136            })
5137            .filter(|x| !x.is_empty())
5138            .collect()
5139    }
5140
5141    // storages are sorted by slot and have range info.
5142    // if we know slots_per_epoch, then add all stores older than slots_per_epoch to dirty_stores so clean visits these slots
5143    fn mark_old_slots_as_dirty(&self, storages: &SortedStorages, slots_per_epoch: Option<Slot>) {
5144        if let Some(slots_per_epoch) = slots_per_epoch {
5145            let max = storages.range().end;
5146            let acceptable_straggler_slot_count = 100; // do nothing special for these old stores which will likely get cleaned up shortly
5147            let sub = slots_per_epoch + acceptable_straggler_slot_count;
5148            let in_epoch_range_start = max.saturating_sub(sub);
5149            for slot in storages.range().start..in_epoch_range_start {
5150                if let Some(storages) = storages.get(slot) {
5151                    storages.iter().for_each(|store| {
5152                        self.dirty_stores
5153                            .insert((slot, store.id.load(Ordering::Relaxed)), store.clone());
5154                    });
5155                }
5156            }
5157        }
5158    }
5159
5160    fn calculate_accounts_hash_helper(
5161        &self,
5162        use_index: bool,
5163        slot: Slot,
5164        ancestors: &Ancestors,
5165        check_hash: bool,
5166        can_cached_slot_be_unflushed: bool,
5167        slots_per_epoch: Option<Slot>,
5168    ) -> Result<(Hash, u64), BankHashVerificationError> {
5169        if !use_index {
5170            let accounts_cache_and_ancestors = if can_cached_slot_be_unflushed {
5171                Some((&self.accounts_cache, ancestors, &self.accounts_index))
5172            } else {
5173                None
5174            };
5175
5176            let mut collect_time = Measure::start("collect");
5177            let (combined_maps, slots) = self.get_snapshot_storages(slot, None, Some(ancestors));
5178            collect_time.stop();
5179
5180            let mut sort_time = Measure::start("sort_storages");
5181            let min_root = self.accounts_index.min_root();
5182            let storages = SortedStorages::new_with_slots(
5183                combined_maps.iter().zip(slots.iter()),
5184                min_root,
5185                Some(slot),
5186            );
5187
5188            self.mark_old_slots_as_dirty(&storages, slots_per_epoch);
5189            sort_time.stop();
5190
5191            let timings = HashStats {
5192                collect_snapshots_us: collect_time.as_us(),
5193                storage_sort_us: sort_time.as_us(),
5194                ..HashStats::default()
5195            };
5196
5197            Self::calculate_accounts_hash_without_index(
5198                &self.accounts_hash_cache_path,
5199                &storages,
5200                Some(&self.thread_pool_clean),
5201                timings,
5202                check_hash,
5203                accounts_cache_and_ancestors,
5204            )
5205        } else {
5206            self.calculate_accounts_hash(slot, ancestors, check_hash)
5207        }
5208    }
5209
5210    fn calculate_accounts_hash_helper_with_verify(
5211        &self,
5212        use_index: bool,
5213        debug_verify: bool,
5214        slot: Slot,
5215        ancestors: &Ancestors,
5216        expected_capitalization: Option<u64>,
5217        can_cached_slot_be_unflushed: bool,
5218        check_hash: bool,
5219        slots_per_epoch: Option<Slot>,
5220    ) -> Result<(Hash, u64), BankHashVerificationError> {
5221        let (hash, total_carats) = self.calculate_accounts_hash_helper(
5222            use_index,
5223            slot,
5224            ancestors,
5225            check_hash,
5226            can_cached_slot_be_unflushed,
5227            slots_per_epoch,
5228        )?;
5229        if debug_verify {
5230            // calculate the other way (store or non-store) and verify results match.
5231            let (hash_other, total_carats_other) = self.calculate_accounts_hash_helper(
5232                !use_index,
5233                slot,
5234                ancestors,
5235                check_hash,
5236                can_cached_slot_be_unflushed,
5237                None,
5238            )?;
5239
5240            let success = hash == hash_other
5241                && total_carats == total_carats_other
5242                && total_carats == expected_capitalization.unwrap_or(total_carats);
5243            assert!(success, "update_accounts_hash_with_index_option mismatch. hashes: {}, {}; carats: {}, {}; expected carats: {:?}, using index: {}, slot: {}", hash, hash_other, total_carats, total_carats_other, expected_capitalization, use_index, slot);
5244        }
5245        Ok((hash, total_carats))
5246    }
5247
5248    pub fn update_accounts_hash_with_index_option(
5249        &self,
5250        use_index: bool,
5251        debug_verify: bool,
5252        slot: Slot,
5253        ancestors: &Ancestors,
5254        expected_capitalization: Option<u64>,
5255        can_cached_slot_be_unflushed: bool,
5256        slots_per_epoch: Option<Slot>,
5257    ) -> (Hash, u64) {
5258        let check_hash = false;
5259        let (hash, total_carats) = self
5260            .calculate_accounts_hash_helper_with_verify(
5261                use_index,
5262                debug_verify,
5263                slot,
5264                ancestors,
5265                expected_capitalization,
5266                can_cached_slot_be_unflushed,
5267                check_hash,
5268                slots_per_epoch,
5269            )
5270            .unwrap(); // unwrap here will never fail since check_hash = false
5271        let mut bank_hashes = self.bank_hashes.write().unwrap();
5272        let mut bank_hash_info = bank_hashes.get_mut(&slot).unwrap();
5273        bank_hash_info.snapshot_hash = hash;
5274        (hash, total_carats)
5275    }
5276
5277    fn scan_snapshot_stores_with_cache(
5278        cache_hash_data: &CacheHashData,
5279        storage: &SortedStorages,
5280        mut stats: &mut crate::accounts_hash::HashStats,
5281        bins: usize,
5282        bin_range: &Range<usize>,
5283        check_hash: bool,
5284        accounts_cache_and_ancestors: Option<(
5285            &AccountsCache,
5286            &Ancestors,
5287            &AccountInfoAccountsIndex,
5288        )>,
5289    ) -> Result<Vec<BinnedHashData>, BankHashVerificationError> {
5290        let bin_calculator = PubkeyBinCalculator16::new(bins);
5291        assert!(bin_range.start < bins && bin_range.end <= bins && bin_range.start < bin_range.end);
5292        let mut time = Measure::start("scan all accounts");
5293        stats.num_snapshot_storage = storage.slot_count();
5294        let mismatch_found = AtomicU64::new(0);
5295        let range = bin_range.end - bin_range.start;
5296        let sort_time = AtomicU64::new(0);
5297
5298        let result: Vec<BinnedHashData> = Self::scan_account_storage_no_bank(
5299            cache_hash_data,
5300            accounts_cache_and_ancestors,
5301            storage,
5302            |loaded_account: LoadedAccount, accum: &mut BinnedHashData, slot: Slot| {
5303                let pubkey = loaded_account.pubkey();
5304                let mut pubkey_to_bin_index = bin_calculator.bin_from_pubkey(pubkey);
5305                if !bin_range.contains(&pubkey_to_bin_index) {
5306                    return;
5307                }
5308
5309                // 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.
5310                pubkey_to_bin_index -= bin_range.start;
5311
5312                let raw_carats = loaded_account.carats();
5313                let zero_raw_carats = raw_carats == 0;
5314                let balance = if zero_raw_carats {
5315                    crate::accounts_hash::ZERO_RAW_CARATS_SENTINEL
5316                } else {
5317                    raw_carats
5318                };
5319
5320                let source_item =
5321                    CalculateHashIntermediate::new(loaded_account.loaded_hash(), balance, *pubkey);
5322
5323                if check_hash {
5324                    let computed_hash = loaded_account.compute_hash(slot, pubkey);
5325                    if computed_hash != source_item.hash {
5326                        info!(
5327                            "hash mismatch found: computed: {}, loaded: {}, pubkey: {}",
5328                            computed_hash, source_item.hash, pubkey
5329                        );
5330                        mismatch_found.fetch_add(1, Ordering::Relaxed);
5331                    }
5332                }
5333                if accum.is_empty() {
5334                    accum.append(&mut vec![Vec::new(); range]);
5335                }
5336                accum[pubkey_to_bin_index].push(source_item);
5337            },
5338            |x| {
5339                let (result, timing) = Self::sort_slot_storage_scan(x);
5340                sort_time.fetch_add(timing, Ordering::Relaxed);
5341                result
5342            },
5343            bin_range,
5344            &bin_calculator,
5345        );
5346
5347        stats.sort_time_total_us += sort_time.load(Ordering::Relaxed);
5348
5349        if check_hash && mismatch_found.load(Ordering::Relaxed) > 0 {
5350            warn!(
5351                "{} mismatched account hash(es) found",
5352                mismatch_found.load(Ordering::Relaxed)
5353            );
5354            return Err(BankHashVerificationError::MismatchedAccountHash);
5355        }
5356
5357        time.stop();
5358        stats.scan_time_total_us += time.as_us();
5359
5360        Ok(result)
5361    }
5362
5363    fn sort_slot_storage_scan(accum: BinnedHashData) -> (BinnedHashData, u64) {
5364        let time = AtomicU64::new(0);
5365        (
5366            accum
5367                .into_iter()
5368                .map(|mut items| {
5369                    let mut sort_time = Measure::start("sort");
5370                    {
5371                        // sort_by vs unstable because slot and write_version are already in order
5372                        items.sort_by(AccountsHash::compare_two_hash_entries);
5373                    }
5374                    sort_time.stop();
5375                    time.fetch_add(sort_time.as_us(), Ordering::Relaxed);
5376                    items
5377                })
5378                .collect(),
5379            time.load(Ordering::Relaxed),
5380        )
5381    }
5382
5383    // modeled after get_accounts_delta_hash
5384    // intended to be faster than calculate_accounts_hash
5385    pub fn calculate_accounts_hash_without_index(
5386        accounts_hash_cache_path: &Path,
5387        storages: &SortedStorages,
5388        thread_pool: Option<&ThreadPool>,
5389        mut stats: HashStats,
5390        check_hash: bool,
5391        accounts_cache_and_ancestors: Option<(
5392            &AccountsCache,
5393            &Ancestors,
5394            &AccountInfoAccountsIndex,
5395        )>,
5396    ) -> Result<(Hash, u64), BankHashVerificationError> {
5397        let mut scan_and_hash = move || {
5398            assert_eq!(
5399                BINS_PER_PASS * NUM_SCAN_PASSES,
5400                PUBKEY_BINS_FOR_CALCULATING_HASHES
5401            ); // evenly divisible
5402            let mut previous_pass = PreviousPass::default();
5403            let mut final_result = (Hash::default(), 0);
5404
5405            let cache_hash_data = CacheHashData::new(&accounts_hash_cache_path);
5406
5407            for pass in 0..NUM_SCAN_PASSES {
5408                let bounds = Range {
5409                    start: pass * BINS_PER_PASS,
5410                    end: (pass + 1) * BINS_PER_PASS,
5411                };
5412
5413                let result = Self::scan_snapshot_stores_with_cache(
5414                    &cache_hash_data,
5415                    storages,
5416                    &mut stats,
5417                    PUBKEY_BINS_FOR_CALCULATING_HASHES,
5418                    &bounds,
5419                    check_hash,
5420                    accounts_cache_and_ancestors,
5421                )?;
5422
5423                let (hash, carats, for_next_pass) = AccountsHash::rest_of_hash_calculation(
5424                    result,
5425                    &mut stats,
5426                    pass == NUM_SCAN_PASSES - 1,
5427                    previous_pass,
5428                    BINS_PER_PASS,
5429                );
5430                previous_pass = for_next_pass;
5431                final_result = (hash, carats);
5432            }
5433
5434            Ok(final_result)
5435        };
5436        if let Some(thread_pool) = thread_pool {
5437            thread_pool.install(scan_and_hash)
5438        } else {
5439            scan_and_hash()
5440        }
5441    }
5442
5443    pub fn verify_bank_hash_and_carats(
5444        &self,
5445        slot: Slot,
5446        ancestors: &Ancestors,
5447        total_carats: u64,
5448        test_hash_calculation: bool,
5449    ) -> Result<(), BankHashVerificationError> {
5450        use BankHashVerificationError::*;
5451
5452        let use_index = false;
5453        let check_hash = true;
5454        let can_cached_slot_be_unflushed = false;
5455        let (calculated_hash, calculated_carats) = self
5456            .calculate_accounts_hash_helper_with_verify(
5457                use_index,
5458                test_hash_calculation,
5459                slot,
5460                ancestors,
5461                None,
5462                can_cached_slot_be_unflushed,
5463                check_hash,
5464                None,
5465            )?;
5466
5467        if calculated_carats != total_carats {
5468            warn!(
5469                "Mismatched total carats: {} calculated: {}",
5470                total_carats, calculated_carats
5471            );
5472            return Err(MismatchedTotalCarats(calculated_carats, total_carats));
5473        }
5474
5475        let bank_hashes = self.bank_hashes.read().unwrap();
5476        if let Some(found_hash_info) = bank_hashes.get(&slot) {
5477            if calculated_hash == found_hash_info.snapshot_hash {
5478                Ok(())
5479            } else {
5480                warn!(
5481                    "mismatched bank hash for slot {}: {} (calculated) != {} (expected)",
5482                    slot, calculated_hash, found_hash_info.snapshot_hash
5483                );
5484                Err(MismatchedBankHash)
5485            }
5486        } else {
5487            Err(MissingBankHash)
5488        }
5489    }
5490
5491    /// Perform the scan for pubkeys that were written to in a slot
5492    fn do_scan_slot_for_dirty_pubkeys(
5493        &self,
5494        slot: Slot,
5495    ) -> ScanStorageResult<Pubkey, DashSet<Pubkey>> {
5496        self.scan_account_storage(
5497            slot,
5498            |loaded_account: LoadedAccount| Some(*loaded_account.pubkey()),
5499            |accum: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
5500                accum.insert(*loaded_account.pubkey());
5501            },
5502        )
5503    }
5504
5505    /// Reduce the scan result of dirty pubkeys after calling `scan_account_storage()` into a
5506    /// single vec of Pubkeys.
5507    fn do_reduce_scan_slot_for_dirty_pubkeys(
5508        scan_result: ScanStorageResult<Pubkey, DashSet<Pubkey>>,
5509    ) -> Vec<Pubkey> {
5510        match scan_result {
5511            ScanStorageResult::Cached(cached_result) => cached_result,
5512            ScanStorageResult::Stored(stored_result) => {
5513                stored_result.into_iter().collect::<Vec<_>>()
5514            }
5515        }
5516    }
5517
5518    /// Scan a slot for dirty pubkeys
5519    fn scan_slot_for_dirty_pubkeys(&self, slot: Slot) -> Vec<Pubkey> {
5520        let dirty_pubkeys = self.do_scan_slot_for_dirty_pubkeys(slot);
5521        Self::do_reduce_scan_slot_for_dirty_pubkeys(dirty_pubkeys)
5522    }
5523
5524    /// Scan a slot in the account storage for dirty pubkeys and insert them into the list of
5525    /// uncleaned pubkeys
5526    ///
5527    /// This function is called in Bank::drop() when the bank is _not_ frozen, so that its pubkeys
5528    /// are considered for cleanup.
5529    pub fn scan_slot_and_insert_dirty_pubkeys_into_uncleaned_pubkeys(&self, slot: Slot) {
5530        let dirty_pubkeys = self.scan_slot_for_dirty_pubkeys(slot);
5531        self.uncleaned_pubkeys.insert(slot, dirty_pubkeys);
5532    }
5533
5534    pub fn get_accounts_delta_hash(&self, slot: Slot) -> Hash {
5535        let mut scan = Measure::start("scan");
5536
5537        let scan_result: ScanStorageResult<(Pubkey, Hash), DashMapVersionHash> = self
5538            .scan_account_storage(
5539                slot,
5540                |loaded_account: LoadedAccount| {
5541                    // Cache only has one version per key, don't need to worry about versioning
5542                    Some((*loaded_account.pubkey(), loaded_account.loaded_hash()))
5543                },
5544                |accum: &DashMap<Pubkey, (u64, Hash)>, loaded_account: LoadedAccount| {
5545                    let loaded_write_version = loaded_account.write_version();
5546                    let loaded_hash = loaded_account.loaded_hash();
5547                    let should_insert =
5548                        if let Some(existing_entry) = accum.get(loaded_account.pubkey()) {
5549                            loaded_write_version > existing_entry.value().version()
5550                        } else {
5551                            true
5552                        };
5553                    if should_insert {
5554                        // Detected insertion is necessary, grabs the write lock to commit the write,
5555                        match accum.entry(*loaded_account.pubkey()) {
5556                            // Double check in case another thread interleaved a write between the read + write.
5557                            Occupied(mut occupied_entry) => {
5558                                if loaded_write_version > occupied_entry.get().version() {
5559                                    occupied_entry.insert((loaded_write_version, loaded_hash));
5560                                }
5561                            }
5562
5563                            Vacant(vacant_entry) => {
5564                                vacant_entry.insert((loaded_write_version, loaded_hash));
5565                            }
5566                        }
5567                    }
5568                },
5569            );
5570        scan.stop();
5571
5572        let mut accumulate = Measure::start("accumulate");
5573        let hashes: Vec<_> = match scan_result {
5574            ScanStorageResult::Cached(cached_result) => cached_result,
5575            ScanStorageResult::Stored(stored_result) => stored_result
5576                .into_iter()
5577                .map(|(pubkey, (_latest_write_version, hash))| (pubkey, hash))
5578                .collect(),
5579        };
5580        let dirty_keys = hashes.iter().map(|(pubkey, _hash)| *pubkey).collect();
5581
5582        let ret = AccountsHash::accumulate_account_hashes(hashes);
5583        accumulate.stop();
5584        let mut uncleaned_time = Measure::start("uncleaned_index");
5585        self.uncleaned_pubkeys.insert(slot, dirty_keys);
5586        uncleaned_time.stop();
5587        self.stats
5588            .store_uncleaned_update
5589            .fetch_add(uncleaned_time.as_us(), Ordering::Relaxed);
5590
5591        self.stats
5592            .delta_hash_scan_time_total_us
5593            .fetch_add(scan.as_us(), Ordering::Relaxed);
5594        self.stats
5595            .delta_hash_accumulate_time_total_us
5596            .fetch_add(accumulate.as_us(), Ordering::Relaxed);
5597        self.stats.delta_hash_num.fetch_add(1, Ordering::Relaxed);
5598        ret
5599    }
5600
5601    // previous_slot_entry_was_cached = true means we just need to assert that after this update is complete
5602    //  that there are no items we would have put in reclaims that are not cached
5603    fn update_index(
5604        &self,
5605        slot: Slot,
5606        infos: Vec<AccountInfo>,
5607        accounts: &[(&Pubkey, &impl ReadableAccount)],
5608        previous_slot_entry_was_cached: bool,
5609    ) -> SlotList<AccountInfo> {
5610        let mut reclaims = SlotList::<AccountInfo>::with_capacity(infos.len() * 2);
5611        for (info, pubkey_account) in infos.into_iter().zip(accounts.iter()) {
5612            let pubkey = pubkey_account.0;
5613            self.accounts_index.upsert(
5614                slot,
5615                pubkey,
5616                pubkey_account.1.owner(),
5617                pubkey_account.1.data(),
5618                &self.account_indexes,
5619                info,
5620                &mut reclaims,
5621                previous_slot_entry_was_cached,
5622            );
5623        }
5624        reclaims
5625    }
5626
5627    fn should_not_shrink(aligned_bytes: u64, total_bytes: u64, num_stores: usize) -> bool {
5628        aligned_bytes + PAGE_SIZE > total_bytes && num_stores == 1
5629    }
5630
5631    fn is_shrinking_productive(slot: Slot, stores: &[Arc<AccountStorageEntry>]) -> bool {
5632        let mut alive_count = 0;
5633        let mut stored_count = 0;
5634        let mut alive_bytes = 0;
5635        let mut total_bytes = 0;
5636
5637        for store in stores {
5638            alive_count += store.count();
5639            stored_count += store.approx_stored_count();
5640            alive_bytes += store.alive_bytes();
5641            total_bytes += store.total_bytes();
5642        }
5643
5644        let aligned_bytes = Self::page_align(alive_bytes as u64);
5645        if Self::should_not_shrink(aligned_bytes, total_bytes, stores.len()) {
5646            trace!(
5647                "shrink_slot_forced ({}, {}): not able to shrink at all: alive/stored: ({} / {}) ({}b / {}b) save: {}",
5648                slot,
5649                stores.len(),
5650                alive_count,
5651                stored_count,
5652                aligned_bytes,
5653                total_bytes,
5654                total_bytes.saturating_sub(aligned_bytes),
5655            );
5656            return false;
5657        }
5658
5659        true
5660    }
5661
5662    fn is_candidate_for_shrink(&self, store: &Arc<AccountStorageEntry>) -> bool {
5663        match self.shrink_ratio {
5664            AccountShrinkThreshold::TotalSpace { shrink_ratio: _ } => {
5665                Self::page_align(store.alive_bytes() as u64) < store.total_bytes()
5666            }
5667            AccountShrinkThreshold::IndividalStore { shrink_ratio } => {
5668                (Self::page_align(store.alive_bytes() as u64) as f64 / store.total_bytes() as f64)
5669                    < shrink_ratio
5670            }
5671        }
5672    }
5673
5674    fn remove_dead_accounts(
5675        &self,
5676        reclaims: SlotSlice<AccountInfo>,
5677        expected_slot: Option<Slot>,
5678        mut reclaimed_offsets: Option<&mut AppendVecOffsets>,
5679        reset_accounts: bool,
5680    ) -> HashSet<Slot> {
5681        let mut dead_slots = HashSet::new();
5682        let mut new_shrink_candidates: ShrinkCandidates = HashMap::new();
5683        for (slot, account_info) in reclaims {
5684            // No cached accounts should make it here
5685            assert_ne!(account_info.store_id, CACHE_VIRTUAL_STORAGE_ID);
5686            if let Some(ref mut reclaimed_offsets) = reclaimed_offsets {
5687                reclaimed_offsets
5688                    .entry(account_info.store_id)
5689                    .or_default()
5690                    .insert(account_info.offset);
5691            }
5692            if let Some(expected_slot) = expected_slot {
5693                assert_eq!(*slot, expected_slot);
5694            }
5695            if let Some(store) = self
5696                .storage
5697                .get_account_storage_entry(*slot, account_info.store_id)
5698            {
5699                assert_eq!(
5700                    *slot, store.slot(),
5701                    "AccountDB::accounts_index corrupted. Storage pointed to: {}, expected: {}, should only point to one slot",
5702                    store.slot(), *slot
5703                );
5704                let count = store.remove_account(account_info.stored_size, reset_accounts);
5705                if count == 0 {
5706                    self.dirty_stores
5707                        .insert((*slot, store.append_vec_id()), store.clone());
5708                    dead_slots.insert(*slot);
5709                } else if self.caching_enabled
5710                    && Self::is_shrinking_productive(*slot, &[store.clone()])
5711                    && self.is_candidate_for_shrink(&store)
5712                {
5713                    // Checking that this single storage entry is ready for shrinking,
5714                    // should be a sufficient indication that the slot is ready to be shrunk
5715                    // because slots should only have one storage entry, namely the one that was
5716                    // created by `flush_slot_cache()`.
5717                    {
5718                        new_shrink_candidates
5719                            .entry(*slot)
5720                            .or_default()
5721                            .insert(store.append_vec_id(), store);
5722                    }
5723                }
5724            }
5725        }
5726
5727        if self.caching_enabled {
5728            {
5729                let mut shrink_candidate_slots = self.shrink_candidate_slots.lock().unwrap();
5730                for (slot, slot_shrink_candidates) in new_shrink_candidates {
5731                    for (store_id, store) in slot_shrink_candidates {
5732                        // count could be == 0 if multiple accounts are removed
5733                        // at once
5734                        if store.count() != 0 {
5735                            debug!(
5736                                "adding: {} {} to shrink candidates: count: {}/{} bytes: {}/{}",
5737                                store_id,
5738                                slot,
5739                                store.approx_stored_count(),
5740                                store.count(),
5741                                store.alive_bytes(),
5742                                store.total_bytes()
5743                            );
5744
5745                            shrink_candidate_slots
5746                                .entry(slot)
5747                                .or_default()
5748                                .insert(store_id, store);
5749                        }
5750                    }
5751                }
5752            }
5753        }
5754
5755        dead_slots.retain(|slot| {
5756            if let Some(slot_stores) = self.storage.get_slot_stores(*slot) {
5757                for x in slot_stores.read().unwrap().values() {
5758                    if x.count() != 0 {
5759                        return false;
5760                    }
5761                }
5762            }
5763            true
5764        });
5765
5766        dead_slots
5767    }
5768
5769    fn remove_dead_slots_metadata<'a>(
5770        &'a self,
5771        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
5772        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
5773        // Should only be `Some` for non-cached slots
5774        purged_stored_account_slots: Option<&mut AccountSlots>,
5775    ) {
5776        self.clean_dead_slots_from_accounts_index(
5777            dead_slots_iter.clone(),
5778            purged_slot_pubkeys,
5779            purged_stored_account_slots,
5780        );
5781        {
5782            let mut bank_hashes = self.bank_hashes.write().unwrap();
5783            for slot in dead_slots_iter {
5784                bank_hashes.remove(slot);
5785            }
5786        }
5787    }
5788
5789    fn clean_dead_slots_from_accounts_index<'a>(
5790        &'a self,
5791        dead_slots_iter: impl Iterator<Item = &'a Slot> + Clone,
5792        purged_slot_pubkeys: HashSet<(Slot, Pubkey)>,
5793        // Should only be `Some` for non-cached slots
5794        purged_stored_account_slots: Option<&mut AccountSlots>,
5795    ) {
5796        if let Some(purged_stored_account_slots) = purged_stored_account_slots {
5797            for (slot, pubkey) in purged_slot_pubkeys {
5798                purged_stored_account_slots
5799                    .entry(pubkey)
5800                    .or_default()
5801                    .insert(slot);
5802                self.accounts_index.unref_from_storage(&pubkey);
5803            }
5804        }
5805
5806        let mut accounts_index_root_stats = AccountsIndexRootsStats::default();
5807        let mut rooted_cleaned_count = 0;
5808        let mut unrooted_cleaned_count = 0;
5809        let dead_slots: Vec<_> = dead_slots_iter
5810            .map(|slot| {
5811                if let Some(latest) = self.accounts_index.clean_dead_slot(*slot) {
5812                    rooted_cleaned_count += 1;
5813                    accounts_index_root_stats = latest;
5814                } else {
5815                    unrooted_cleaned_count += 1;
5816                }
5817                *slot
5818            })
5819            .collect();
5820        info!("remove_dead_slots_metadata: slots {:?}", dead_slots);
5821
5822        accounts_index_root_stats.rooted_cleaned_count += rooted_cleaned_count;
5823        accounts_index_root_stats.unrooted_cleaned_count += unrooted_cleaned_count;
5824
5825        self.clean_accounts_stats
5826            .latest_accounts_index_roots_stats
5827            .update(&accounts_index_root_stats);
5828    }
5829
5830    fn clean_stored_dead_slots(
5831        &self,
5832        dead_slots: &HashSet<Slot>,
5833        purged_account_slots: Option<&mut AccountSlots>,
5834    ) {
5835        let mut measure = Measure::start("clean_stored_dead_slots-ms");
5836        let mut stores: Vec<Arc<AccountStorageEntry>> = vec![];
5837        for slot in dead_slots.iter() {
5838            if let Some(slot_storage) = self.storage.get_slot_stores(*slot) {
5839                for store in slot_storage.read().unwrap().values() {
5840                    stores.push(store.clone());
5841                }
5842            }
5843        }
5844        let purged_slot_pubkeys: HashSet<(Slot, Pubkey)> = {
5845            self.thread_pool_clean.install(|| {
5846                stores
5847                    .into_par_iter()
5848                    .map(|store| {
5849                        let accounts = store.all_accounts();
5850                        accounts
5851                            .into_iter()
5852                            .map(|account| (store.slot(), account.meta.pubkey))
5853                            .collect::<HashSet<(Slot, Pubkey)>>()
5854                    })
5855                    .reduce(HashSet::new, |mut reduced, store_pubkeys| {
5856                        reduced.extend(store_pubkeys);
5857                        reduced
5858                    })
5859            })
5860        };
5861        self.remove_dead_slots_metadata(
5862            dead_slots.iter(),
5863            purged_slot_pubkeys,
5864            purged_account_slots,
5865        );
5866        measure.stop();
5867        inc_new_counter_info!("clean_stored_dead_slots-ms", measure.as_ms() as usize);
5868    }
5869
5870    pub(crate) fn freeze_accounts(&mut self, ancestors: &Ancestors, account_pubkeys: &[Pubkey]) {
5871        for account_pubkey in account_pubkeys {
5872            if let Some((account, _slot)) = self.load_without_fixed_root(ancestors, account_pubkey)
5873            {
5874                let frozen_account_info = FrozenAccountInfo {
5875                    hash: Self::hash_frozen_account_data(&account),
5876                    carats: account.carats(),
5877                };
5878                warn!(
5879                    "Account {} is now frozen at carats={}, hash={}",
5880                    account_pubkey, frozen_account_info.carats, frozen_account_info.hash
5881                );
5882                self.frozen_accounts
5883                    .insert(*account_pubkey, frozen_account_info);
5884            } else {
5885                panic!(
5886                    "Unable to freeze an account that does not exist: {}",
5887                    account_pubkey
5888                );
5889            }
5890        }
5891    }
5892
5893    /// Cause a panic if frozen accounts would be affected by data in `accounts`
5894    fn assert_frozen_accounts(&self, accounts: &[(&Pubkey, &AccountSharedData)]) {
5895        if self.frozen_accounts.is_empty() {
5896            return;
5897        }
5898        for (account_pubkey, account) in accounts.iter() {
5899            if let Some(frozen_account_info) = self.frozen_accounts.get(*account_pubkey) {
5900                if account.carats() < frozen_account_info.carats {
5901                    FROZEN_ACCOUNT_PANIC.store(true, Ordering::Relaxed);
5902                    panic!(
5903                        "Frozen account {} modified.  Carats decreased from {} to {}",
5904                        account_pubkey,
5905                        frozen_account_info.carats,
5906                        account.carats(),
5907                    )
5908                }
5909
5910                let hash = Self::hash_frozen_account_data(account);
5911                if hash != frozen_account_info.hash {
5912                    FROZEN_ACCOUNT_PANIC.store(true, Ordering::Relaxed);
5913                    panic!(
5914                        "Frozen account {} modified.  Hash changed from {} to {}",
5915                        account_pubkey, frozen_account_info.hash, hash,
5916                    )
5917                }
5918            }
5919        }
5920    }
5921
5922    pub fn store_cached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
5923        self.store(slot, accounts, self.caching_enabled);
5924    }
5925
5926    /// Store the account update.
5927    pub fn store_uncached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
5928        self.store(slot, accounts, false);
5929    }
5930
5931    fn store(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)], is_cached_store: bool) {
5932        // If all transactions in a batch are errored,
5933        // it's possible to get a store with no accounts.
5934        if accounts.is_empty() {
5935            return;
5936        }
5937        self.assert_frozen_accounts(accounts);
5938
5939        let mut stats = BankHashStats::default();
5940        let mut total_data = 0;
5941        accounts.iter().for_each(|(_pubkey, account)| {
5942            total_data += account.data().len();
5943            stats.update(*account);
5944        });
5945
5946        self.stats
5947            .store_total_data
5948            .fetch_add(total_data as u64, Ordering::Relaxed);
5949
5950        let mut bank_hashes = self.bank_hashes.write().unwrap();
5951        let slot_info = bank_hashes
5952            .entry(slot)
5953            .or_insert_with(BankHashInfo::default);
5954        slot_info.stats.merge(&stats);
5955
5956        // we use default hashes for now since the same account may be stored to the cache multiple times
5957        self.store_accounts_unfrozen(slot, accounts, None, is_cached_store);
5958        self.report_store_timings();
5959    }
5960
5961    fn report_store_timings(&self) {
5962        if self.stats.last_store_report.should_update(1000) {
5963            let (read_only_cache_hits, read_only_cache_misses) =
5964                self.read_only_accounts_cache.get_and_reset_stats();
5965            datapoint_info!(
5966                "accounts_db_store_timings",
5967                (
5968                    "hash_accounts",
5969                    self.stats.store_hash_accounts.swap(0, Ordering::Relaxed),
5970                    i64
5971                ),
5972                (
5973                    "store_accounts",
5974                    self.stats.store_accounts.swap(0, Ordering::Relaxed),
5975                    i64
5976                ),
5977                (
5978                    "update_index",
5979                    self.stats.store_update_index.swap(0, Ordering::Relaxed),
5980                    i64
5981                ),
5982                (
5983                    "handle_reclaims",
5984                    self.stats.store_handle_reclaims.swap(0, Ordering::Relaxed),
5985                    i64
5986                ),
5987                (
5988                    "append_accounts",
5989                    self.stats.store_append_accounts.swap(0, Ordering::Relaxed),
5990                    i64
5991                ),
5992                (
5993                    "find_storage",
5994                    self.stats.store_find_store.swap(0, Ordering::Relaxed),
5995                    i64
5996                ),
5997                (
5998                    "num_accounts",
5999                    self.stats.store_num_accounts.swap(0, Ordering::Relaxed),
6000                    i64
6001                ),
6002                (
6003                    "total_data",
6004                    self.stats.store_total_data.swap(0, Ordering::Relaxed),
6005                    i64
6006                ),
6007                (
6008                    "read_only_accounts_cache_entries",
6009                    self.read_only_accounts_cache.cache_len(),
6010                    i64
6011                ),
6012                (
6013                    "read_only_accounts_cache_data_size",
6014                    self.read_only_accounts_cache.data_size(),
6015                    i64
6016                ),
6017                ("read_only_accounts_cache_hits", read_only_cache_hits, i64),
6018                (
6019                    "read_only_accounts_cache_misses",
6020                    read_only_cache_misses,
6021                    i64
6022                ),
6023                (
6024                    "calc_stored_meta_us",
6025                    self.stats.calc_stored_meta.swap(0, Ordering::Relaxed),
6026                    i64
6027                ),
6028            );
6029
6030            let recycle_stores = self.recycle_stores.read().unwrap();
6031            datapoint_info!(
6032                "accounts_db_store_timings2",
6033                (
6034                    "recycle_store_count",
6035                    self.stats.recycle_store_count.swap(0, Ordering::Relaxed),
6036                    i64
6037                ),
6038                (
6039                    "current_recycle_store_count",
6040                    recycle_stores.entry_count(),
6041                    i64
6042                ),
6043                (
6044                    "current_recycle_store_bytes",
6045                    recycle_stores.total_bytes(),
6046                    i64
6047                ),
6048                (
6049                    "create_store_count",
6050                    self.stats.create_store_count.swap(0, Ordering::Relaxed),
6051                    i64
6052                ),
6053                (
6054                    "store_get_slot_store",
6055                    self.stats.store_get_slot_store.swap(0, Ordering::Relaxed),
6056                    i64
6057                ),
6058                (
6059                    "store_find_existing",
6060                    self.stats.store_find_existing.swap(0, Ordering::Relaxed),
6061                    i64
6062                ),
6063                (
6064                    "dropped_stores",
6065                    self.stats.dropped_stores.swap(0, Ordering::Relaxed),
6066                    i64
6067                ),
6068            );
6069        }
6070    }
6071
6072    fn store_accounts_unfrozen(
6073        &self,
6074        slot: Slot,
6075        accounts: &[(&Pubkey, &AccountSharedData)],
6076        hashes: Option<&[&Hash]>,
6077        is_cached_store: bool,
6078    ) {
6079        // This path comes from a store to a non-frozen slot.
6080        // If a store is dead here, then a newer update for
6081        // each pubkey in the store must exist in another
6082        // store in the slot. Thus it is safe to reset the store and
6083        // re-use it for a future store op. The pubkey ref counts should still
6084        // hold just 1 ref from this slot.
6085        let reset_accounts = true;
6086
6087        self.store_accounts_custom(
6088            slot,
6089            accounts,
6090            hashes,
6091            None::<StorageFinder>,
6092            None::<Box<dyn Iterator<Item = u64>>>,
6093            is_cached_store,
6094            reset_accounts,
6095        );
6096    }
6097
6098    fn store_accounts_frozen<'a>(
6099        &'a self,
6100        slot: Slot,
6101        accounts: &[(&Pubkey, &impl ReadableAccount)],
6102        hashes: Option<&[impl Borrow<Hash>]>,
6103        storage_finder: Option<StorageFinder<'a>>,
6104        write_version_producer: Option<Box<dyn Iterator<Item = StoredMetaWriteVersion>>>,
6105    ) -> StoreAccountsTiming {
6106        // stores on a frozen slot should not reset
6107        // the append vec so that hashing could happen on the store
6108        // and accounts in the append_vec can be unrefed correctly
6109        let reset_accounts = false;
6110        let is_cached_store = false;
6111        self.store_accounts_custom(
6112            slot,
6113            accounts,
6114            hashes,
6115            storage_finder,
6116            write_version_producer,
6117            is_cached_store,
6118            reset_accounts,
6119        )
6120    }
6121
6122    fn store_accounts_custom<'a>(
6123        &'a self,
6124        slot: Slot,
6125        accounts: &[(&Pubkey, &impl ReadableAccount)],
6126        hashes: Option<&[impl Borrow<Hash>]>,
6127        storage_finder: Option<StorageFinder<'a>>,
6128        write_version_producer: Option<Box<dyn Iterator<Item = u64>>>,
6129        is_cached_store: bool,
6130        reset_accounts: bool,
6131    ) -> StoreAccountsTiming {
6132        let storage_finder: StorageFinder<'a> = storage_finder
6133            .unwrap_or_else(|| Box::new(move |slot, size| self.find_storage_candidate(slot, size)));
6134
6135        let write_version_producer: Box<dyn Iterator<Item = u64>> = write_version_producer
6136            .unwrap_or_else(|| {
6137                let mut current_version = self.bulk_assign_write_version(accounts.len());
6138                Box::new(std::iter::from_fn(move || {
6139                    let ret = current_version;
6140                    current_version += 1;
6141                    Some(ret)
6142                }))
6143            });
6144
6145        self.stats
6146            .store_num_accounts
6147            .fetch_add(accounts.len() as u64, Ordering::Relaxed);
6148        let mut store_accounts_time = Measure::start("store_accounts");
6149        let infos = self.store_accounts_to(
6150            slot,
6151            accounts,
6152            hashes,
6153            storage_finder,
6154            write_version_producer,
6155            is_cached_store,
6156        );
6157        store_accounts_time.stop();
6158        self.stats
6159            .store_accounts
6160            .fetch_add(store_accounts_time.as_us(), Ordering::Relaxed);
6161        let mut update_index_time = Measure::start("update_index");
6162
6163        let previous_slot_entry_was_cached = self.caching_enabled && is_cached_store;
6164
6165        // If the cache was flushed, then because `update_index` occurs
6166        // after the account are stored by the above `store_accounts_to`
6167        // call and all the accounts are stored, all reads after this point
6168        // will know to not check the cache anymore
6169        let mut reclaims = self.update_index(slot, infos, accounts, previous_slot_entry_was_cached);
6170
6171        // For each updated account, `reclaims` should only have at most one
6172        // item (if the account was previously updated in this slot).
6173        // filter out the cached reclaims as those don't actually map
6174        // to anything that needs to be cleaned in the backing storage
6175        // entries
6176        if self.caching_enabled {
6177            reclaims.retain(|(_, r)| !r.is_cached());
6178
6179            if is_cached_store {
6180                assert!(reclaims.is_empty());
6181            }
6182        }
6183
6184        update_index_time.stop();
6185        self.stats
6186            .store_update_index
6187            .fetch_add(update_index_time.as_us(), Ordering::Relaxed);
6188
6189        // A store for a single slot should:
6190        // 1) Only make "reclaims" for the same slot
6191        // 2) Should not cause any slots to be removed from the storage
6192        // database because
6193        //    a) this slot  has at least one account (the one being stored),
6194        //    b)From 1) we know no other slots are included in the "reclaims"
6195        //
6196        // From 1) and 2) we guarantee passing `no_purge_stats` == None, which is
6197        // equivalent to asserting there will be no dead slots, is safe.
6198        let no_purge_stats = None;
6199        let mut handle_reclaims_time = Measure::start("handle_reclaims");
6200        self.handle_reclaims(&reclaims, Some(slot), no_purge_stats, None, reset_accounts);
6201        handle_reclaims_time.stop();
6202        self.stats
6203            .store_handle_reclaims
6204            .fetch_add(handle_reclaims_time.as_us(), Ordering::Relaxed);
6205
6206        StoreAccountsTiming {
6207            store_accounts_elapsed: store_accounts_time.as_us(),
6208            update_index_elapsed: update_index_time.as_us(),
6209            handle_reclaims_elapsed: handle_reclaims_time.as_us(),
6210        }
6211    }
6212
6213    pub fn add_root(&self, slot: Slot) {
6214        self.accounts_index.add_root(slot, self.caching_enabled);
6215        if self.caching_enabled {
6216            self.accounts_cache.add_root(slot);
6217        }
6218        if let Some(slot_stores) = self.storage.get_slot_stores(slot) {
6219            for (store_id, store) in slot_stores.read().unwrap().iter() {
6220                self.dirty_stores.insert((slot, *store_id), store.clone());
6221            }
6222        }
6223    }
6224
6225    pub fn get_snapshot_storages(
6226        &self,
6227        snapshot_slot: Slot,
6228        snapshot_base_slot: Option<Slot>,
6229        ancestors: Option<&Ancestors>,
6230    ) -> (SnapshotStorages, Vec<Slot>) {
6231        let mut m = Measure::start("get slots");
6232        let slots = self
6233            .storage
6234            .0
6235            .iter()
6236            .map(|k| *k.key() as Slot)
6237            .collect::<Vec<_>>();
6238        m.stop();
6239        let mut m2 = Measure::start("filter");
6240
6241        let chunk_size = 5_000;
6242        let wide = self.thread_pool_clean.install(|| {
6243            slots
6244                .par_chunks(chunk_size)
6245                .map(|slots| {
6246                    slots
6247                        .iter()
6248                        .filter_map(|slot| {
6249                            if *slot <= snapshot_slot
6250                                && snapshot_base_slot
6251                                    .map_or(true, |snapshot_base_slot| *slot > snapshot_base_slot)
6252                                && (self.accounts_index.is_root(*slot)
6253                                    || ancestors
6254                                        .map(|ancestors| ancestors.contains_key(slot))
6255                                        .unwrap_or_default())
6256                            {
6257                                self.storage.0.get(slot).map_or_else(
6258                                    || None,
6259                                    |item| {
6260                                        let storages = item
6261                                            .value()
6262                                            .read()
6263                                            .unwrap()
6264                                            .values()
6265                                            .filter(|x| x.has_accounts())
6266                                            .cloned()
6267                                            .collect::<Vec<_>>();
6268                                        if !storages.is_empty() {
6269                                            Some((storages, *slot))
6270                                        } else {
6271                                            None
6272                                        }
6273                                    },
6274                                )
6275                            } else {
6276                                None
6277                            }
6278                        })
6279                        .collect::<Vec<(SnapshotStorage, Slot)>>()
6280                })
6281                .collect::<Vec<_>>()
6282        });
6283        m2.stop();
6284        let mut m3 = Measure::start("flatten");
6285        // some slots we found above may not have been a root or met the slot # constraint.
6286        // So the resulting 'slots' vector we return will be a subset of the raw keys we got initially.
6287        let mut slots = Vec::with_capacity(slots.len());
6288        let result = wide
6289            .into_iter()
6290            .flatten()
6291            .map(|(storage, slot)| {
6292                slots.push(slot);
6293                storage
6294            })
6295            .collect::<Vec<_>>();
6296        m3.stop();
6297
6298        debug!(
6299            "hash_total: get slots: {}, filter: {}, flatten: {}",
6300            m.as_us(),
6301            m2.as_us(),
6302            m3.as_us()
6303        );
6304        (result, slots)
6305    }
6306
6307    fn process_storage_slot(
6308        storage_maps: &[Arc<AccountStorageEntry>],
6309    ) -> GenerateIndexAccountsMap<'_> {
6310        let num_accounts = storage_maps
6311            .iter()
6312            .map(|storage| storage.approx_stored_count())
6313            .sum();
6314        let mut accounts_map = GenerateIndexAccountsMap::with_capacity(num_accounts);
6315        storage_maps.iter().for_each(|storage| {
6316            let accounts = storage.all_accounts();
6317            accounts.into_iter().for_each(|stored_account| {
6318                let this_version = stored_account.meta.write_version;
6319                match accounts_map.entry(stored_account.meta.pubkey) {
6320                    std::collections::hash_map::Entry::Vacant(entry) => {
6321                        entry.insert(IndexAccountMapEntry {
6322                            write_version: this_version,
6323                            store_id: storage.append_vec_id(),
6324                            stored_account,
6325                        });
6326                    }
6327                    std::collections::hash_map::Entry::Occupied(mut entry) => {
6328                        let occupied_version = entry.get().write_version;
6329                        if occupied_version < this_version {
6330                            entry.insert(IndexAccountMapEntry {
6331                                write_version: this_version,
6332                                store_id: storage.append_vec_id(),
6333                                stored_account,
6334                            });
6335                        } else {
6336                            assert!(occupied_version != this_version);
6337                        }
6338                    }
6339                }
6340            })
6341        });
6342        accounts_map
6343    }
6344
6345    fn generate_index_for_slot<'a>(
6346        &self,
6347        accounts_map: GenerateIndexAccountsMap<'a>,
6348        slot: &Slot,
6349    ) -> u64 {
6350        if accounts_map.is_empty() {
6351            return 0;
6352        }
6353
6354        let secondary = !self.account_indexes.is_empty();
6355
6356        let len = accounts_map.len();
6357        let items = accounts_map.into_iter().map(
6358            |(
6359                pubkey,
6360                IndexAccountMapEntry {
6361                    write_version: _write_version,
6362                    store_id,
6363                    stored_account,
6364                },
6365            )| {
6366                if secondary {
6367                    self.accounts_index.update_secondary_indexes(
6368                        &pubkey,
6369                        &stored_account.account_meta.owner,
6370                        stored_account.data,
6371                        &self.account_indexes,
6372                    );
6373                }
6374
6375                (
6376                    pubkey,
6377                    AccountInfo {
6378                        store_id,
6379                        offset: stored_account.offset,
6380                        stored_size: stored_account.stored_size,
6381                        carats: stored_account.account_meta.carats,
6382                    },
6383                )
6384            },
6385        );
6386
6387        let (dirty_pubkeys, insert_us) = self
6388            .accounts_index
6389            .insert_new_if_missing_into_primary_index(*slot, len, items);
6390
6391        // dirty_pubkeys will contain a pubkey if an item has multiple rooted entries for
6392        // a given pubkey. If there is just a single item, there is no cleaning to
6393        // be done on that pubkey. Use only those pubkeys with multiple updates.
6394        if !dirty_pubkeys.is_empty() {
6395            self.uncleaned_pubkeys.insert(*slot, dirty_pubkeys);
6396        }
6397        insert_us
6398    }
6399
6400    #[allow(clippy::needless_collect)]
6401    pub fn generate_index(&self, limit_load_slot_count_from_snapshot: Option<usize>, verify: bool) {
6402        let mut slots = self.storage.all_slots();
6403        #[allow(clippy::stable_sort_primitive)]
6404        slots.sort();
6405        if let Some(limit) = limit_load_slot_count_from_snapshot {
6406            slots.truncate(limit); // get rid of the newer slots and keep just the older
6407        }
6408        // pass == 0 always runs and generates the index
6409        // pass == 1 only runs if verify == true.
6410        // verify checks that all the expected items are in the accounts index and measures how long it takes to look them all up
6411        let passes = if verify { 2 } else { 1 };
6412        for pass in 0..passes {
6413            if pass == 0 {
6414                self.accounts_index.set_startup(true);
6415            }
6416            let storage_info = StorageSizeAndCountMap::default();
6417            let total_processed_slots_across_all_threads = AtomicU64::new(0);
6418            let outer_slots_len = slots.len();
6419            let chunk_size = (outer_slots_len / 7) + 1; // approximately 400k slots in a snapshot
6420            let mut index_time = Measure::start("index");
6421            let insertion_time_us = AtomicU64::new(0);
6422            let storage_info_timings = Mutex::new(GenerateIndexTimings::default());
6423            let scan_time: u64 = slots
6424                .par_chunks(chunk_size)
6425                .map(|slots| {
6426                    let mut log_status = MultiThreadProgress::new(
6427                        &total_processed_slots_across_all_threads,
6428                        2,
6429                        outer_slots_len as u64,
6430                    );
6431                    let mut scan_time_sum = 0;
6432                    for (index, slot) in slots.iter().enumerate() {
6433                        let mut scan_time = Measure::start("scan");
6434                        log_status.report(index as u64);
6435                        let storage_maps: Vec<Arc<AccountStorageEntry>> = self
6436                            .storage
6437                            .get_slot_storage_entries(*slot)
6438                            .unwrap_or_default();
6439                        let accounts_map = Self::process_storage_slot(&storage_maps);
6440                        scan_time.stop();
6441                        scan_time_sum += scan_time.as_us();
6442                        Self::update_storage_info(
6443                            &storage_info,
6444                            &accounts_map,
6445                            &storage_info_timings,
6446                        );
6447
6448                        let insert_us = if pass == 0 {
6449                            // generate index
6450                            self.generate_index_for_slot(accounts_map, slot)
6451                        } else {
6452                            // verify index matches expected and measure the time to get all items
6453                            assert!(verify);
6454                            let mut lookup_time = Measure::start("lookup_time");
6455                            for account in accounts_map.into_iter() {
6456                                let (key, account_info) = account;
6457                                let lock = self.accounts_index.get_account_maps_read_lock(&key);
6458                                let x = lock.get(&key).unwrap();
6459                                let sl = x.slot_list.read().unwrap();
6460                                let mut count = 0;
6461                                for (slot2, account_info2) in sl.iter() {
6462                                    if slot2 == slot {
6463                                        count += 1;
6464                                        let ai = AccountInfo {
6465                                            store_id: account_info.store_id,
6466                                            offset: account_info.stored_account.offset,
6467                                            stored_size: account_info.stored_account.stored_size,
6468                                            carats: account_info
6469                                                .stored_account
6470                                                .account_meta
6471                                                .carats,
6472                                        };
6473                                        assert_eq!(&ai, account_info2);
6474                                    }
6475                                }
6476                                assert_eq!(1, count);
6477                            }
6478                            lookup_time.stop();
6479                            lookup_time.as_us()
6480                        };
6481                        insertion_time_us.fetch_add(insert_us, Ordering::Relaxed);
6482                    }
6483                    scan_time_sum
6484                })
6485                .sum();
6486            index_time.stop();
6487
6488            let mut min_bin_size = usize::MAX;
6489            let mut max_bin_size = usize::MIN;
6490            let total_items = self
6491                .accounts_index
6492                .account_maps
6493                .iter()
6494                .map(|map_bin| {
6495                    let len = map_bin.read().unwrap().len();
6496                    min_bin_size = std::cmp::min(min_bin_size, len);
6497                    max_bin_size = std::cmp::max(max_bin_size, len);
6498                    len
6499                })
6500                .sum();
6501
6502            let storage_info_timings = storage_info_timings.into_inner().unwrap();
6503
6504            let mut index_flush_us = 0;
6505            if pass == 0 {
6506                // tell accounts index we are done adding the initial accounts at startup
6507                let mut m = Measure::start("accounts_index_idle_us");
6508                self.accounts_index.set_startup(false);
6509                m.stop();
6510                index_flush_us = m.as_us();
6511            }
6512
6513            let mut timings = GenerateIndexTimings {
6514                index_flush_us,
6515                scan_time,
6516                index_time: index_time.as_us(),
6517                insertion_time_us: insertion_time_us.load(Ordering::Relaxed),
6518                min_bin_size,
6519                max_bin_size,
6520                total_items,
6521                storage_size_accounts_map_us: storage_info_timings.storage_size_accounts_map_us,
6522                storage_size_accounts_map_flatten_us: storage_info_timings
6523                    .storage_size_accounts_map_flatten_us,
6524                ..GenerateIndexTimings::default()
6525            };
6526
6527            if pass == 0 {
6528                // Need to add these last, otherwise older updates will be cleaned
6529                for slot in &slots {
6530                    self.accounts_index.add_root(*slot, false);
6531                }
6532
6533                self.set_storage_count_and_alive_bytes(storage_info, &mut timings);
6534            }
6535            timings.report();
6536        }
6537    }
6538
6539    fn update_storage_info(
6540        storage_info: &StorageSizeAndCountMap,
6541        accounts_map: &GenerateIndexAccountsMap<'_>,
6542        timings: &Mutex<GenerateIndexTimings>,
6543    ) {
6544        let mut storage_size_accounts_map_time = Measure::start("storage_size_accounts_map");
6545
6546        let mut storage_info_local = HashMap::<AppendVecId, StorageSizeAndCount>::default();
6547        // first collect into a local HashMap with no lock contention
6548        for (_, v) in accounts_map.iter() {
6549            let mut info = storage_info_local
6550                .entry(v.store_id)
6551                .or_insert_with(StorageSizeAndCount::default);
6552            info.stored_size += v.stored_account.stored_size;
6553            info.count += 1;
6554        }
6555        storage_size_accounts_map_time.stop();
6556        // second, collect into the shared DashMap once we've figured out all the info per store_id
6557        let mut storage_size_accounts_map_flatten_time =
6558            Measure::start("storage_size_accounts_map_flatten_time");
6559        for (store_id, v) in storage_info_local.into_iter() {
6560            let mut info = storage_info
6561                .entry(store_id)
6562                .or_insert_with(StorageSizeAndCount::default);
6563            info.stored_size += v.stored_size;
6564            info.count += v.count;
6565        }
6566        storage_size_accounts_map_flatten_time.stop();
6567
6568        let mut timings = timings.lock().unwrap();
6569        timings.storage_size_accounts_map_us += storage_size_accounts_map_time.as_us();
6570        timings.storage_size_accounts_map_flatten_us +=
6571            storage_size_accounts_map_flatten_time.as_us();
6572    }
6573    fn set_storage_count_and_alive_bytes(
6574        &self,
6575        stored_sizes_and_counts: StorageSizeAndCountMap,
6576        timings: &mut GenerateIndexTimings,
6577    ) {
6578        // store count and size for each storage
6579        let mut storage_size_storages_time = Measure::start("storage_size_storages");
6580        for slot_stores in self.storage.0.iter() {
6581            for (id, store) in slot_stores.value().read().unwrap().iter() {
6582                // Should be default at this point
6583                assert_eq!(store.alive_bytes(), 0);
6584                if let Some(entry) = stored_sizes_and_counts.get(id) {
6585                    trace!(
6586                        "id: {} setting count: {} cur: {}",
6587                        id,
6588                        entry.count,
6589                        store.count(),
6590                    );
6591                    store.count_and_status.write().unwrap().0 = entry.count;
6592                    store.alive_bytes.store(entry.stored_size, Ordering::SeqCst);
6593                } else {
6594                    trace!("id: {} clearing count", id);
6595                    store.count_and_status.write().unwrap().0 = 0;
6596                }
6597            }
6598        }
6599        storage_size_storages_time.stop();
6600        timings.storage_size_storages_us = storage_size_storages_time.as_us();
6601    }
6602
6603    pub(crate) fn print_accounts_stats(&self, label: &str) {
6604        self.print_index(label);
6605        self.print_count_and_status(label);
6606        info!("recycle_stores:");
6607        let recycle_stores = self.recycle_stores.read().unwrap();
6608        for (recycled_time, entry) in recycle_stores.iter() {
6609            info!(
6610                "  slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {} (recycled: {:?})",
6611                entry.slot(),
6612                entry.append_vec_id(),
6613                *entry.count_and_status.read().unwrap(),
6614                entry.approx_store_count.load(Ordering::Relaxed),
6615                entry.accounts.len(),
6616                entry.accounts.capacity(),
6617                recycled_time,
6618            );
6619        }
6620    }
6621
6622    fn print_index(&self, label: &str) {
6623        let mut roots: Vec<_> = self.accounts_index.all_roots();
6624        #[allow(clippy::stable_sort_primitive)]
6625        roots.sort();
6626        info!("{}: accounts_index roots: {:?}", label, roots,);
6627        self.accounts_index.account_maps.iter().for_each(|map| {
6628            for (pubkey, account_entry) in
6629                map.read().unwrap().items(&None::<&std::ops::Range<Pubkey>>)
6630            {
6631                info!("  key: {} ref_count: {}", pubkey, account_entry.ref_count(),);
6632                info!(
6633                    "      slots: {:?}",
6634                    *account_entry.slot_list.read().unwrap()
6635                );
6636            }
6637        });
6638    }
6639
6640    fn print_count_and_status(&self, label: &str) {
6641        let mut slots: Vec<_> = self.storage.all_slots();
6642        #[allow(clippy::stable_sort_primitive)]
6643        slots.sort();
6644        info!("{}: count_and status for {} slots:", label, slots.len());
6645        for slot in &slots {
6646            let slot_stores = self.storage.get_slot_stores(*slot).unwrap();
6647            let r_slot_stores = slot_stores.read().unwrap();
6648            let mut ids: Vec<_> = r_slot_stores.keys().cloned().collect();
6649            #[allow(clippy::stable_sort_primitive)]
6650            ids.sort();
6651            for id in &ids {
6652                let entry = r_slot_stores.get(id).unwrap();
6653                info!(
6654                    "  slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {}",
6655                    slot,
6656                    id,
6657                    *entry.count_and_status.read().unwrap(),
6658                    entry.approx_store_count.load(Ordering::Relaxed),
6659                    entry.accounts.len(),
6660                    entry.accounts.capacity(),
6661                );
6662            }
6663        }
6664    }
6665}
6666
6667#[cfg(test)]
6668impl AccountsDb {
6669    pub fn new(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
6670        Self::new_for_tests(paths, cluster_type)
6671    }
6672
6673    pub fn new_with_config_for_tests(
6674        paths: Vec<PathBuf>,
6675        cluster_type: &ClusterType,
6676        account_indexes: AccountSecondaryIndexes,
6677        caching_enabled: bool,
6678        shrink_ratio: AccountShrinkThreshold,
6679    ) -> Self {
6680        Self::new_with_config(
6681            paths,
6682            cluster_type,
6683            account_indexes,
6684            caching_enabled,
6685            shrink_ratio,
6686            Some(ACCOUNTS_DB_CONFIG_FOR_TESTING),
6687        )
6688    }
6689
6690    pub fn new_sized(paths: Vec<PathBuf>, file_size: u64) -> Self {
6691        AccountsDb {
6692            file_size,
6693            ..AccountsDb::new(paths, &ClusterType::Development)
6694        }
6695    }
6696
6697    pub fn new_sized_no_extra_stores(paths: Vec<PathBuf>, file_size: u64) -> Self {
6698        AccountsDb {
6699            file_size,
6700            min_num_stores: 0,
6701            ..AccountsDb::new(paths, &ClusterType::Development)
6702        }
6703    }
6704
6705    pub fn get_append_vec_id(&self, pubkey: &Pubkey, slot: Slot) -> Option<AppendVecId> {
6706        let ancestors = vec![(slot, 1)].into_iter().collect();
6707        let result = self.accounts_index.get(pubkey, Some(&ancestors), None);
6708        result.map(|(list, index)| list.slot_list()[index].1.store_id)
6709    }
6710
6711    pub fn alive_account_count_in_slot(&self, slot: Slot) -> usize {
6712        self.storage
6713            .get_slot_stores(slot)
6714            .map(|storages| storages.read().unwrap().values().map(|s| s.count()).sum())
6715            .unwrap_or(0)
6716    }
6717}
6718
6719/// Legacy shrink functions to support non-cached path.
6720/// Should be able to be deleted after cache path is the only path.
6721impl AccountsDb {
6722    // Reads all accounts in given slot's AppendVecs and filter only to alive,
6723    // then create a minimum AppendVec filled with the alive.
6724    // v1 path shrinks all stores in the slot
6725    //
6726    // Requires all stores in the slot to be re-written otherwise the accounts_index
6727    // store ref count could become incorrect.
6728    fn do_shrink_slot_v1(&self, slot: Slot, forced: bool) -> usize {
6729        trace!("shrink_stale_slot: slot: {}", slot);
6730
6731        if let Some(stores_lock) = self.storage.get_slot_stores(slot) {
6732            let stores: Vec<_> = stores_lock.read().unwrap().values().cloned().collect();
6733            let mut alive_count = 0;
6734            let mut stored_count = 0;
6735            let mut written_bytes = 0;
6736            let mut total_bytes = 0;
6737            for store in &stores {
6738                alive_count += store.count();
6739                stored_count += store.approx_stored_count();
6740                written_bytes += store.written_bytes();
6741                total_bytes += store.total_bytes();
6742            }
6743            if alive_count == stored_count && stores.len() == 1 {
6744                trace!(
6745                    "shrink_stale_slot ({}): not able to shrink at all: alive/stored: {} / {} {}",
6746                    slot,
6747                    alive_count,
6748                    stored_count,
6749                    if forced { " (forced)" } else { "" },
6750                );
6751                return 0;
6752            } else if !forced {
6753                let sparse_by_count = (alive_count as f32 / stored_count as f32) <= 0.8;
6754                let sparse_by_bytes = (written_bytes as f32 / total_bytes as f32) <= 0.8;
6755                let not_sparse = !sparse_by_count && !sparse_by_bytes;
6756                let too_small_to_shrink = total_bytes <= PAGE_SIZE;
6757                if not_sparse || too_small_to_shrink {
6758                    return 0;
6759                }
6760                info!(
6761                    "shrink_stale_slot ({}): not_sparse: {} count: {}/{} byte: {}/{}",
6762                    slot, not_sparse, alive_count, stored_count, written_bytes, total_bytes,
6763                );
6764            }
6765
6766            self.do_shrink_slot_stores(slot, stores.iter())
6767        } else {
6768            0
6769        }
6770    }
6771
6772    fn do_reset_uncleaned_roots_v1(
6773        &self,
6774        candidates: &mut MutexGuard<Vec<Slot>>,
6775        max_clean_root: Option<Slot>,
6776    ) {
6777        let previous_roots = self.accounts_index.reset_uncleaned_roots(max_clean_root);
6778        candidates.extend(previous_roots);
6779    }
6780
6781    #[cfg(test)]
6782    fn reset_uncleaned_roots_v1(&self) {
6783        self.do_reset_uncleaned_roots_v1(&mut self.shrink_candidate_slots_v1.lock().unwrap(), None);
6784    }
6785
6786    fn do_shrink_stale_slot_v1(&self, slot: Slot) -> usize {
6787        self.do_shrink_slot_v1(slot, false)
6788    }
6789    fn do_shrink_slot_forced_v1(&self, slot: Slot) {
6790        self.do_shrink_slot_v1(slot, true);
6791    }
6792
6793    fn shrink_stale_slot_v1(&self, candidates: &mut MutexGuard<Vec<Slot>>) -> usize {
6794        let mut shrunken_account_total = 0;
6795        let mut shrunk_slot_count = 0;
6796        let start = Instant::now();
6797        let num_roots = self.accounts_index.num_roots();
6798        loop {
6799            if let Some(slot) = self.do_next_shrink_slot_v1(candidates) {
6800                shrunken_account_total += self.do_shrink_stale_slot_v1(slot);
6801            } else {
6802                return 0;
6803            }
6804            if start.elapsed().as_millis() > 100 || shrunk_slot_count > num_roots / 10 {
6805                debug!(
6806                    "do_shrink_stale_slot_v1: {} {} {}us",
6807                    shrunk_slot_count,
6808                    candidates.len(),
6809                    start.elapsed().as_micros()
6810                );
6811                break;
6812            }
6813            shrunk_slot_count += 1;
6814        }
6815        shrunken_account_total
6816    }
6817
6818    // Infinitely returns rooted roots in cyclic order
6819    fn do_next_shrink_slot_v1(&self, candidates: &mut MutexGuard<Vec<Slot>>) -> Option<Slot> {
6820        // At this point, a lock (= candidates) is ensured to be held to keep
6821        // do_reset_uncleaned_roots() (in clean_accounts()) from updating candidates.
6822        // Also, candidates in the lock may be swapped here if it's empty.
6823        let next = candidates.pop();
6824
6825        if next.is_some() {
6826            next
6827        } else {
6828            let mut new_all_slots = self.all_root_slots_in_index();
6829            let next = new_all_slots.pop();
6830            // refresh candidates for later calls!
6831            **candidates = new_all_slots;
6832
6833            next
6834        }
6835    }
6836
6837    #[cfg(test)]
6838    fn next_shrink_slot_v1(&self) -> Option<Slot> {
6839        let mut candidates = self.shrink_candidate_slots_v1.lock().unwrap();
6840        self.do_next_shrink_slot_v1(&mut candidates)
6841    }
6842
6843    pub fn process_stale_slot_v1(&self) -> usize {
6844        let mut measure = Measure::start("stale_slot_shrink-ms");
6845        let candidates = self.shrink_candidate_slots_v1.try_lock();
6846        if candidates.is_err() {
6847            // skip and return immediately if locked by clean_accounts()
6848            // the calling background thread will just retry later.
6849            return 0;
6850        }
6851        // hold this lock as long as this shrinking process is running to avoid conflicts
6852        // with clean_accounts().
6853        let mut candidates = candidates.unwrap();
6854
6855        let count = self.shrink_stale_slot_v1(&mut candidates);
6856        measure.stop();
6857        inc_new_counter_info!("stale_slot_shrink-ms", measure.as_ms() as usize);
6858
6859        count
6860    }
6861
6862    #[cfg(test)]
6863    fn shrink_all_stale_slots_v1(&self) {
6864        for slot in self.all_slots_in_storage() {
6865            self.do_shrink_stale_slot_v1(slot);
6866        }
6867    }
6868}
6869
6870#[cfg(test)]
6871pub mod tests {
6872    use super::*;
6873    use crate::{
6874        accounts_hash::MERKLE_FANOUT,
6875        accounts_index::RefCount,
6876        accounts_index::{tests::*, AccountSecondaryIndexesIncludeExclude},
6877        append_vec::{test_utils::TempFile, AccountMeta},
6878        inline_spl_token_v2_0,
6879    };
6880    use assert_matches::assert_matches;
6881    use rand::{thread_rng, Rng};
6882    use gemachain_sdk::{
6883        account::{accounts_equal, Account, AccountSharedData, ReadableAccount, WritableAccount},
6884        hash::HASH_BYTES,
6885        pubkey::PUBKEY_BYTES,
6886    };
6887    use std::{
6888        iter::FromIterator,
6889        str::FromStr,
6890        thread::{self, sleep, Builder, JoinHandle},
6891        time::Duration,
6892    };
6893
6894    fn linear_ancestors(end_slot: u64) -> Ancestors {
6895        let mut ancestors: Ancestors = vec![(0, 0)].into_iter().collect();
6896        for i in 1..end_slot {
6897            ancestors.insert(i, (i - 1) as usize);
6898        }
6899        ancestors
6900    }
6901
6902    fn empty_storages<'a>() -> SortedStorages<'a> {
6903        SortedStorages::new(&[])
6904    }
6905
6906    impl AccountsDb {
6907        fn scan_snapshot_stores(
6908            storage: &SortedStorages,
6909            stats: &mut crate::accounts_hash::HashStats,
6910            bins: usize,
6911            bin_range: &Range<usize>,
6912            check_hash: bool,
6913        ) -> Result<Vec<BinnedHashData>, BankHashVerificationError> {
6914            let temp_dir = TempDir::new().unwrap();
6915            let accounts_hash_cache_path = temp_dir.path();
6916            Self::scan_snapshot_stores_with_cache(
6917                &CacheHashData::new(&accounts_hash_cache_path),
6918                storage,
6919                stats,
6920                bins,
6921                bin_range,
6922                check_hash,
6923                None,
6924            )
6925        }
6926    }
6927
6928    #[test]
6929    #[should_panic(
6930        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
6931    )]
6932    fn test_accountsdb_scan_snapshot_stores_illegal_range_start() {
6933        let mut stats = HashStats::default();
6934        let bounds = Range { start: 2, end: 2 };
6935
6936        AccountsDb::scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false).unwrap();
6937    }
6938    #[test]
6939    #[should_panic(
6940        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
6941    )]
6942    fn test_accountsdb_scan_snapshot_stores_illegal_range_end() {
6943        let mut stats = HashStats::default();
6944        let bounds = Range { start: 1, end: 3 };
6945
6946        AccountsDb::scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false).unwrap();
6947    }
6948
6949    #[test]
6950    #[should_panic(
6951        expected = "bin_range.start < bins && bin_range.end <= bins &&\\n    bin_range.start < bin_range.end"
6952    )]
6953    fn test_accountsdb_scan_snapshot_stores_illegal_range_inverse() {
6954        let mut stats = HashStats::default();
6955        let bounds = Range { start: 1, end: 0 };
6956
6957        AccountsDb::scan_snapshot_stores(&empty_storages(), &mut stats, 2, &bounds, false).unwrap();
6958    }
6959
6960    fn sample_storages_and_account_in_slot(
6961        slot: Slot,
6962    ) -> (SnapshotStorages, Vec<CalculateHashIntermediate>) {
6963        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
6964        let pubkey0 = Pubkey::new(&[0u8; 32]);
6965        let pubkey127 = Pubkey::new(&[0x7fu8; 32]);
6966        let pubkey128 = Pubkey::new(&[0x80u8; 32]);
6967        let pubkey255 = Pubkey::new(&[0xffu8; 32]);
6968
6969        let mut raw_expected = vec![
6970            CalculateHashIntermediate::new(Hash::default(), 1, pubkey0),
6971            CalculateHashIntermediate::new(Hash::default(), 128, pubkey127),
6972            CalculateHashIntermediate::new(Hash::default(), 129, pubkey128),
6973            CalculateHashIntermediate::new(Hash::default(), 256, pubkey255),
6974        ];
6975
6976        let expected_hashes = vec![
6977            Hash::from_str("5K3NW73xFHwgTWVe4LyCg4QfQda8f88uZj2ypDx2kmmH").unwrap(),
6978            Hash::from_str("84ozw83MZ8oeSF4hRAg7SeW1Tqs9LMXagX1BrDRjtZEx").unwrap(),
6979            Hash::from_str("5XqtnEJ41CG2JWNp7MAg9nxkRUAnyjLxfsKsdrLxQUbC").unwrap(),
6980            Hash::from_str("DpvwJcznzwULYh19Zu5CuAA4AT6WTBe4H6n15prATmqj").unwrap(),
6981        ];
6982
6983        let mut raw_accounts = Vec::default();
6984
6985        for i in 0..raw_expected.len() {
6986            raw_accounts.push(AccountSharedData::new(
6987                raw_expected[i].carats,
6988                1,
6989                AccountSharedData::default().owner(),
6990            ));
6991            let hash = AccountsDb::hash_account(slot, &raw_accounts[i], &raw_expected[i].pubkey);
6992            if slot == 1 {
6993                assert_eq!(hash, expected_hashes[i]);
6994            }
6995            raw_expected[i].hash = hash;
6996        }
6997
6998        let to_store = raw_accounts
6999            .iter()
7000            .zip(raw_expected.iter())
7001            .map(|(account, intermediate)| (&intermediate.pubkey, account))
7002            .collect::<Vec<_>>();
7003
7004        accounts.store_uncached(slot, &to_store[..]);
7005        accounts.add_root(slot);
7006
7007        let (storages, slots) = accounts.get_snapshot_storages(slot, None, None);
7008        assert_eq!(storages.len(), slots.len());
7009        storages
7010            .iter()
7011            .zip(slots.iter())
7012            .for_each(|(storages, slot)| {
7013                for storage in storages {
7014                    assert_eq!(&storage.slot(), slot);
7015                }
7016            });
7017        (storages, raw_expected)
7018    }
7019
7020    fn sample_storages_and_accounts() -> (SnapshotStorages, Vec<CalculateHashIntermediate>) {
7021        sample_storages_and_account_in_slot(1)
7022    }
7023
7024    fn get_storage_refs(input: &[SnapshotStorage]) -> SortedStorages {
7025        SortedStorages::new(input)
7026    }
7027
7028    #[test]
7029    fn test_accountsdb_scan_snapshot_stores() {
7030        gemachain_logger::setup();
7031        let (storages, raw_expected) = sample_storages_and_accounts();
7032
7033        let bins = 1;
7034        let mut stats = HashStats::default();
7035
7036        let result = AccountsDb::scan_snapshot_stores(
7037            &get_storage_refs(&storages),
7038            &mut stats,
7039            bins,
7040            &Range {
7041                start: 0,
7042                end: bins,
7043            },
7044            false,
7045        )
7046        .unwrap();
7047        assert_eq!(result, vec![vec![raw_expected.clone()]]);
7048
7049        let bins = 2;
7050        let result = AccountsDb::scan_snapshot_stores(
7051            &get_storage_refs(&storages),
7052            &mut stats,
7053            bins,
7054            &Range {
7055                start: 0,
7056                end: bins,
7057            },
7058            false,
7059        )
7060        .unwrap();
7061        let mut expected = vec![Vec::new(); bins];
7062        expected[0].push(raw_expected[0].clone());
7063        expected[0].push(raw_expected[1].clone());
7064        expected[bins - 1].push(raw_expected[2].clone());
7065        expected[bins - 1].push(raw_expected[3].clone());
7066        assert_eq!(result, vec![expected]);
7067
7068        let bins = 4;
7069        let result = AccountsDb::scan_snapshot_stores(
7070            &get_storage_refs(&storages),
7071            &mut stats,
7072            bins,
7073            &Range {
7074                start: 0,
7075                end: bins,
7076            },
7077            false,
7078        )
7079        .unwrap();
7080        let mut expected = vec![Vec::new(); bins];
7081        expected[0].push(raw_expected[0].clone());
7082        expected[1].push(raw_expected[1].clone());
7083        expected[2].push(raw_expected[2].clone());
7084        expected[bins - 1].push(raw_expected[3].clone());
7085        assert_eq!(result, vec![expected]);
7086
7087        let bins = 256;
7088        let result = AccountsDb::scan_snapshot_stores(
7089            &get_storage_refs(&storages),
7090            &mut stats,
7091            bins,
7092            &Range {
7093                start: 0,
7094                end: bins,
7095            },
7096            false,
7097        )
7098        .unwrap();
7099        let mut expected = vec![Vec::new(); bins];
7100        expected[0].push(raw_expected[0].clone());
7101        expected[127].push(raw_expected[1].clone());
7102        expected[128].push(raw_expected[2].clone());
7103        expected[bins - 1].push(raw_expected.last().unwrap().clone());
7104        assert_eq!(result, vec![expected]);
7105    }
7106
7107    #[test]
7108    fn test_accountsdb_scan_snapshot_stores_2nd_chunk() {
7109        // enough stores to get to 2nd chunk
7110        let bins = 1;
7111        let slot = MAX_ITEMS_PER_CHUNK as Slot;
7112        let (storages, raw_expected) = sample_storages_and_account_in_slot(slot);
7113        let storage_data = vec![(&storages[0], slot)];
7114
7115        let sorted_storages =
7116            SortedStorages::new_debug(&storage_data[..], 0, MAX_ITEMS_PER_CHUNK as usize + 1);
7117
7118        let mut stats = HashStats::default();
7119        let result = AccountsDb::scan_snapshot_stores(
7120            &sorted_storages,
7121            &mut stats,
7122            bins,
7123            &Range {
7124                start: 0,
7125                end: bins,
7126            },
7127            false,
7128        )
7129        .unwrap();
7130        assert_eq!(result.len(), 2); // 2 chunks
7131        assert_eq!(result[0].len(), bins);
7132        assert_eq!(0, result[0].iter().map(|x| x.len()).sum::<usize>()); // nothing found in bin 0
7133        assert_eq!(result[1].len(), bins);
7134        assert_eq!(result[1], vec![raw_expected]);
7135    }
7136
7137    #[test]
7138    fn test_accountsdb_scan_snapshot_stores_binning() {
7139        let mut stats = HashStats::default();
7140        let (storages, raw_expected) = sample_storages_and_accounts();
7141
7142        // just the first bin of 2
7143        let bins = 2;
7144        let half_bins = bins / 2;
7145        let result = AccountsDb::scan_snapshot_stores(
7146            &get_storage_refs(&storages),
7147            &mut stats,
7148            bins,
7149            &Range {
7150                start: 0,
7151                end: half_bins,
7152            },
7153            false,
7154        )
7155        .unwrap();
7156        let mut expected = vec![Vec::new(); half_bins];
7157        expected[0].push(raw_expected[0].clone());
7158        expected[0].push(raw_expected[1].clone());
7159        assert_eq!(result, vec![expected]);
7160
7161        // just the second bin of 2
7162        let result = AccountsDb::scan_snapshot_stores(
7163            &get_storage_refs(&storages),
7164            &mut stats,
7165            bins,
7166            &Range {
7167                start: 1,
7168                end: bins,
7169            },
7170            false,
7171        )
7172        .unwrap();
7173
7174        let mut expected = vec![Vec::new(); half_bins];
7175        let starting_bin_index = 0;
7176        expected[starting_bin_index].push(raw_expected[2].clone());
7177        expected[starting_bin_index].push(raw_expected[3].clone());
7178        assert_eq!(result, vec![expected]);
7179
7180        // 1 bin at a time of 4
7181        let bins = 4;
7182        for (bin, expected_item) in raw_expected.iter().enumerate().take(bins) {
7183            let result = AccountsDb::scan_snapshot_stores(
7184                &get_storage_refs(&storages),
7185                &mut stats,
7186                bins,
7187                &Range {
7188                    start: bin,
7189                    end: bin + 1,
7190                },
7191                false,
7192            )
7193            .unwrap();
7194            let mut expected = vec![Vec::new(); 1];
7195            expected[0].push(expected_item.clone());
7196            assert_eq!(result, vec![expected]);
7197        }
7198
7199        let bins = 256;
7200        let bin_locations = vec![0, 127, 128, 255];
7201        let range = 1;
7202        for bin in 0..bins {
7203            let result = AccountsDb::scan_snapshot_stores(
7204                &get_storage_refs(&storages),
7205                &mut stats,
7206                bins,
7207                &Range {
7208                    start: bin,
7209                    end: bin + range,
7210                },
7211                false,
7212            )
7213            .unwrap();
7214            let mut expected = vec![];
7215            if let Some(index) = bin_locations.iter().position(|&r| r == bin) {
7216                expected = vec![vec![Vec::new(); range]];
7217                expected[0][0].push(raw_expected[index].clone());
7218            }
7219            assert_eq!(result, expected);
7220        }
7221    }
7222
7223    #[test]
7224    fn test_accountsdb_scan_snapshot_stores_binning_2nd_chunk() {
7225        // enough stores to get to 2nd chunk
7226        // range is for only 1 bin out of 256.
7227        let bins = 256;
7228        let slot = MAX_ITEMS_PER_CHUNK as Slot;
7229        let (storages, raw_expected) = sample_storages_and_account_in_slot(slot);
7230        let storage_data = vec![(&storages[0], slot)];
7231
7232        let sorted_storages =
7233            SortedStorages::new_debug(&storage_data[..], 0, MAX_ITEMS_PER_CHUNK as usize + 1);
7234
7235        let mut stats = HashStats::default();
7236        let range = 1;
7237        let start = 127;
7238        let result = AccountsDb::scan_snapshot_stores(
7239            &sorted_storages,
7240            &mut stats,
7241            bins,
7242            &Range {
7243                start,
7244                end: start + range,
7245            },
7246            false,
7247        )
7248        .unwrap();
7249        assert_eq!(result.len(), 2); // 2 chunks
7250        assert_eq!(result[0].len(), range);
7251        assert_eq!(0, result[0].iter().map(|x| x.len()).sum::<usize>()); // nothing found in bin 0
7252        let mut expected = vec![Vec::new(); range];
7253        expected[0].push(raw_expected[1].clone());
7254        assert_eq!(result[1].len(), 1);
7255        assert_eq!(result[1], expected);
7256    }
7257
7258    #[test]
7259    fn test_accountsdb_calculate_accounts_hash_without_index_simple() {
7260        gemachain_logger::setup();
7261
7262        let (storages, _size, _slot_expected) = sample_storage();
7263        let result = AccountsDb::calculate_accounts_hash_without_index(
7264            TempDir::new().unwrap().path(),
7265            &get_storage_refs(&storages),
7266            None,
7267            HashStats::default(),
7268            false,
7269            None,
7270        )
7271        .unwrap();
7272        let expected_hash = Hash::from_str("GKot5hBsd81kMupNCXHaqbhv3huEbxAFMLnpcX2hniwn").unwrap();
7273        assert_eq!(result, (expected_hash, 0));
7274    }
7275
7276    #[test]
7277    fn test_accountsdb_calculate_accounts_hash_without_index() {
7278        gemachain_logger::setup();
7279
7280        let (storages, raw_expected) = sample_storages_and_accounts();
7281        let expected_hash =
7282            AccountsHash::compute_merkle_root_loop(raw_expected.clone(), MERKLE_FANOUT, |item| {
7283                item.hash
7284            });
7285        let sum = raw_expected.iter().map(|item| item.carats).sum();
7286        let result = AccountsDb::calculate_accounts_hash_without_index(
7287            TempDir::new().unwrap().path(),
7288            &get_storage_refs(&storages),
7289            None,
7290            HashStats::default(),
7291            false,
7292            None,
7293        )
7294        .unwrap();
7295
7296        assert_eq!(result, (expected_hash, sum));
7297    }
7298
7299    fn sample_storage() -> (SnapshotStorages, usize, Slot) {
7300        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
7301        let slot_expected: Slot = 0;
7302        let size: usize = 123;
7303        let data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
7304
7305        let arc = Arc::new(data);
7306        let storages = vec![vec![arc]];
7307        (storages, size, slot_expected)
7308    }
7309
7310    #[test]
7311    fn test_accountsdb_scan_account_storage_no_bank() {
7312        gemachain_logger::setup();
7313
7314        let expected = 1;
7315        let tf = crate::append_vec::test_utils::get_append_vec_path(
7316            "test_accountsdb_scan_account_storage_no_bank",
7317        );
7318        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
7319        let slot_expected: Slot = 0;
7320        let size: usize = 123;
7321        let mut data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
7322        let av = AppendVec::new(&tf.path, true, 1024 * 1024);
7323        data.accounts = av;
7324
7325        let arc = Arc::new(data);
7326        let storages = vec![vec![arc]];
7327        let pubkey = gemachain_sdk::pubkey::new_rand();
7328        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
7329        let sm = StoredMeta {
7330            data_len: 1,
7331            pubkey,
7332            write_version: 1,
7333        };
7334        storages[0][0]
7335            .accounts
7336            .append_accounts(&[(sm, Some(&acc))], &[&Hash::default()]);
7337
7338        let calls = AtomicU64::new(0);
7339        let temp_dir = TempDir::new().unwrap();
7340        let accounts_hash_cache_path = temp_dir.path();
7341        let result = AccountsDb::scan_account_storage_no_bank(
7342            &CacheHashData::new(&accounts_hash_cache_path),
7343            None,
7344            &get_storage_refs(&storages),
7345            |loaded_account: LoadedAccount, accum: &mut BinnedHashData, slot: Slot| {
7346                calls.fetch_add(1, Ordering::Relaxed);
7347                assert_eq!(loaded_account.pubkey(), &pubkey);
7348                assert_eq!(slot_expected, slot);
7349                accum.push(vec![CalculateHashIntermediate::new(
7350                    Hash::default(),
7351                    expected,
7352                    pubkey,
7353                )]);
7354            },
7355            |a| a,
7356            &Range { start: 0, end: 1 },
7357            &PubkeyBinCalculator16::new(1),
7358        );
7359        assert_eq!(calls.load(Ordering::Relaxed), 1);
7360        assert_eq!(
7361            result,
7362            vec![vec![vec![CalculateHashIntermediate::new(
7363                Hash::default(),
7364                expected,
7365                pubkey
7366            )]]]
7367        );
7368    }
7369
7370    #[test]
7371    fn test_accountsdb_scan_account_storage_no_bank_one_slot() {
7372        gemachain_logger::setup();
7373
7374        let expected = 1;
7375        let tf = crate::append_vec::test_utils::get_append_vec_path(
7376            "test_accountsdb_scan_account_storage_no_bank",
7377        );
7378        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
7379        let slot_expected: Slot = 0;
7380        let size: usize = 123;
7381        let mut data = AccountStorageEntry::new(&paths[0], slot_expected, 0, size as u64);
7382        let av = AppendVec::new(&tf.path, true, 1024 * 1024);
7383        data.accounts = av;
7384
7385        let arc = Arc::new(data);
7386        let storages = vec![vec![arc]];
7387        let pubkey = gemachain_sdk::pubkey::new_rand();
7388        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
7389        let sm = StoredMeta {
7390            data_len: 1,
7391            pubkey,
7392            write_version: 1,
7393        };
7394        storages[0][0]
7395            .accounts
7396            .append_accounts(&[(sm, Some(&acc))], &[&Hash::default()]);
7397
7398        let calls = AtomicU64::new(0);
7399        let mut accum = Vec::new();
7400        let scan_func = |loaded_account: LoadedAccount, accum: &mut Vec<u64>, slot: Slot| {
7401            calls.fetch_add(1, Ordering::Relaxed);
7402            assert_eq!(loaded_account.pubkey(), &pubkey);
7403            assert_eq!(slot_expected, slot);
7404            accum.push(expected);
7405        };
7406        AccountsDb::scan_multiple_account_storages_one_slot(
7407            &storages[0],
7408            &scan_func,
7409            slot_expected,
7410            &mut accum,
7411        );
7412        assert_eq!(calls.load(Ordering::Relaxed), 1);
7413        assert_eq!(accum, vec![expected]);
7414    }
7415
7416    fn sample_storage_with_entries(
7417        tf: &TempFile,
7418        write_version: StoredMetaWriteVersion,
7419        slot: Slot,
7420        pubkey: &Pubkey,
7421    ) -> SnapshotStorages {
7422        let (_temp_dirs, paths) = get_temp_accounts_paths(1).unwrap();
7423        let size: usize = 123;
7424        let mut data = AccountStorageEntry::new(&paths[0], slot, 0, size as u64);
7425        let av = AppendVec::new(&tf.path, true, 1024 * 1024);
7426        data.accounts = av;
7427
7428        let arc = Arc::new(data);
7429        let storages = vec![vec![arc]];
7430        let acc = AccountSharedData::new(1, 48, AccountSharedData::default().owner());
7431        let sm = StoredMeta {
7432            data_len: 1,
7433            pubkey: *pubkey,
7434            write_version,
7435        };
7436        storages[0][0]
7437            .accounts
7438            .append_accounts(&[(sm, Some(&acc))], &[&Hash::default()]);
7439        storages
7440    }
7441
7442    #[test]
7443    fn test_accountsdb_scan_multiple_account_storage_no_bank_one_slot() {
7444        gemachain_logger::setup();
7445
7446        let slot_expected: Slot = 0;
7447        let tf = crate::append_vec::test_utils::get_append_vec_path(
7448            "test_accountsdb_scan_account_storage_no_bank",
7449        );
7450        let write_version1 = 0;
7451        let write_version2 = 1;
7452        let pubkey1 = gemachain_sdk::pubkey::new_rand();
7453        let pubkey2 = gemachain_sdk::pubkey::new_rand();
7454        for swap in [false, true].iter() {
7455            let mut storages = [
7456                sample_storage_with_entries(&tf, write_version1, slot_expected, &pubkey1)
7457                    .remove(0)
7458                    .remove(0),
7459                sample_storage_with_entries(&tf, write_version2, slot_expected, &pubkey2)
7460                    .remove(0)
7461                    .remove(0),
7462            ];
7463            if *swap {
7464                storages[..].swap(0, 1);
7465            }
7466            let calls = AtomicU64::new(0);
7467            let scan_func = |loaded_account: LoadedAccount, accum: &mut Vec<u64>, slot: Slot| {
7468                calls.fetch_add(1, Ordering::Relaxed);
7469                let write_version = loaded_account.write_version();
7470                let first = loaded_account.pubkey() == &pubkey1 && write_version == write_version1;
7471                assert!(
7472                    first || loaded_account.pubkey() == &pubkey2 && write_version == write_version2
7473                );
7474                assert_eq!(slot_expected, slot);
7475                if first {
7476                    assert!(accum.is_empty());
7477                } else {
7478                    assert!(accum.len() == 1);
7479                }
7480                accum.push(write_version);
7481            };
7482            let mut accum = Vec::new();
7483            AccountsDb::scan_multiple_account_storages_one_slot(
7484                &storages,
7485                &scan_func,
7486                slot_expected,
7487                &mut accum,
7488            );
7489            assert_eq!(calls.load(Ordering::Relaxed), storages.len() as u64);
7490            assert_eq!(accum, vec![write_version1, write_version2]);
7491        }
7492    }
7493
7494    #[test]
7495    fn test_accountsdb_add_root() {
7496        gemachain_logger::setup();
7497        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7498        let key = Pubkey::default();
7499        let account0 = AccountSharedData::new(1, 0, &key);
7500
7501        db.store_uncached(0, &[(&key, &account0)]);
7502        db.add_root(0);
7503        let ancestors = vec![(1, 1)].into_iter().collect();
7504        assert_eq!(
7505            db.load_without_fixed_root(&ancestors, &key),
7506            Some((account0, 0))
7507        );
7508    }
7509
7510    const COLLECT_ALL_UNSORTED_FALSE: bool = false;
7511
7512    #[test]
7513    fn test_accountsdb_latest_ancestor() {
7514        gemachain_logger::setup();
7515        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7516        let key = Pubkey::default();
7517        let account0 = AccountSharedData::new(1, 0, &key);
7518
7519        db.store_uncached(0, &[(&key, &account0)]);
7520
7521        let account1 = AccountSharedData::new(0, 0, &key);
7522        db.store_uncached(1, &[(&key, &account1)]);
7523
7524        let ancestors = vec![(1, 1)].into_iter().collect();
7525        assert_eq!(
7526            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
7527            &account1
7528        );
7529
7530        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
7531        assert_eq!(
7532            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
7533            &account1
7534        );
7535
7536        let accounts: Vec<AccountSharedData> = db.unchecked_scan_accounts(
7537            "",
7538            &ancestors,
7539            |accounts: &mut Vec<AccountSharedData>, option| {
7540                accounts.push(option.1.take_account());
7541            },
7542            COLLECT_ALL_UNSORTED_FALSE,
7543        );
7544        assert_eq!(accounts, vec![account1]);
7545    }
7546
7547    #[test]
7548    fn test_accountsdb_latest_ancestor_with_root() {
7549        gemachain_logger::setup();
7550        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7551        let key = Pubkey::default();
7552        let account0 = AccountSharedData::new(1, 0, &key);
7553
7554        db.store_uncached(0, &[(&key, &account0)]);
7555
7556        let account1 = AccountSharedData::new(0, 0, &key);
7557        db.store_uncached(1, &[(&key, &account1)]);
7558        db.add_root(0);
7559
7560        let ancestors = vec![(1, 1)].into_iter().collect();
7561        assert_eq!(
7562            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
7563            &account1
7564        );
7565
7566        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
7567        assert_eq!(
7568            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
7569            &account1
7570        );
7571    }
7572
7573    #[test]
7574    fn test_accountsdb_root_one_slot() {
7575        gemachain_logger::setup();
7576        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7577
7578        let key = Pubkey::default();
7579        let account0 = AccountSharedData::new(1, 0, &key);
7580
7581        // store value 1 in the "root", i.e. db zero
7582        db.store_uncached(0, &[(&key, &account0)]);
7583
7584        // now we have:
7585        //
7586        //                       root0 -> key.carats==1
7587        //                        / \
7588        //                       /   \
7589        //  key.carats==0 <- slot1    \
7590        //                             slot2 -> key.carats==1
7591        //                                       (via root0)
7592
7593        // store value 0 in one child
7594        let account1 = AccountSharedData::new(0, 0, &key);
7595        db.store_uncached(1, &[(&key, &account1)]);
7596
7597        // masking accounts is done at the Accounts level, at accountsDB we see
7598        // original account (but could also accept "None", which is implemented
7599        // at the Accounts level)
7600        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
7601        assert_eq!(
7602            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
7603            &account1
7604        );
7605
7606        // we should see 1 token in slot 2
7607        let ancestors = vec![(0, 0), (2, 2)].into_iter().collect();
7608        assert_eq!(
7609            &db.load_without_fixed_root(&ancestors, &key).unwrap().0,
7610            &account0
7611        );
7612
7613        db.add_root(0);
7614
7615        let ancestors = vec![(1, 1)].into_iter().collect();
7616        assert_eq!(
7617            db.load_without_fixed_root(&ancestors, &key),
7618            Some((account1, 1))
7619        );
7620        let ancestors = vec![(2, 2)].into_iter().collect();
7621        assert_eq!(
7622            db.load_without_fixed_root(&ancestors, &key),
7623            Some((account0, 0))
7624        ); // original value
7625    }
7626
7627    #[test]
7628    fn test_accountsdb_add_root_many() {
7629        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7630
7631        let mut pubkeys: Vec<Pubkey> = vec![];
7632        create_account(&db, &mut pubkeys, 0, 100, 0, 0);
7633        for _ in 1..100 {
7634            let idx = thread_rng().gen_range(0, 99);
7635            let ancestors = vec![(0, 0)].into_iter().collect();
7636            let account = db
7637                .load_without_fixed_root(&ancestors, &pubkeys[idx])
7638                .unwrap();
7639            let default_account = AccountSharedData::from(Account {
7640                carats: (idx + 1) as u64,
7641                ..Account::default()
7642            });
7643            assert_eq!((default_account, 0), account);
7644        }
7645
7646        db.add_root(0);
7647
7648        // check that all the accounts appear with a new root
7649        for _ in 1..100 {
7650            let idx = thread_rng().gen_range(0, 99);
7651            let ancestors = vec![(0, 0)].into_iter().collect();
7652            let account0 = db
7653                .load_without_fixed_root(&ancestors, &pubkeys[idx])
7654                .unwrap();
7655            let ancestors = vec![(1, 1)].into_iter().collect();
7656            let account1 = db
7657                .load_without_fixed_root(&ancestors, &pubkeys[idx])
7658                .unwrap();
7659            let default_account = AccountSharedData::from(Account {
7660                carats: (idx + 1) as u64,
7661                ..Account::default()
7662            });
7663            assert_eq!(&default_account, &account0.0);
7664            assert_eq!(&default_account, &account1.0);
7665        }
7666    }
7667
7668    #[test]
7669    fn test_accountsdb_count_stores() {
7670        gemachain_logger::setup();
7671        let db = AccountsDb::new_single_for_tests();
7672
7673        let mut pubkeys: Vec<Pubkey> = vec![];
7674        create_account(&db, &mut pubkeys, 0, 2, DEFAULT_FILE_SIZE as usize / 3, 0);
7675        assert!(check_storage(&db, 0, 2));
7676
7677        let pubkey = gemachain_sdk::pubkey::new_rand();
7678        let account = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 3, &pubkey);
7679        db.store_uncached(1, &[(&pubkey, &account)]);
7680        db.store_uncached(1, &[(&pubkeys[0], &account)]);
7681        {
7682            let slot_0_stores = &db.storage.get_slot_stores(0).unwrap();
7683            let slot_1_stores = &db.storage.get_slot_stores(1).unwrap();
7684            let r_slot_0_stores = slot_0_stores.read().unwrap();
7685            let r_slot_1_stores = slot_1_stores.read().unwrap();
7686            assert_eq!(r_slot_0_stores.len(), 1);
7687            assert_eq!(r_slot_1_stores.len(), 1);
7688            assert_eq!(r_slot_0_stores.get(&0).unwrap().count(), 2);
7689            assert_eq!(r_slot_1_stores[&1].count(), 2);
7690            assert_eq!(r_slot_0_stores.get(&0).unwrap().approx_stored_count(), 2);
7691            assert_eq!(r_slot_1_stores[&1].approx_stored_count(), 2);
7692        }
7693
7694        // adding root doesn't change anything
7695        db.get_accounts_delta_hash(1);
7696        db.add_root(1);
7697        {
7698            let slot_0_stores = &db.storage.get_slot_stores(0).unwrap();
7699            let slot_1_stores = &db.storage.get_slot_stores(1).unwrap();
7700            let r_slot_0_stores = slot_0_stores.read().unwrap();
7701            let r_slot_1_stores = slot_1_stores.read().unwrap();
7702            assert_eq!(r_slot_0_stores.len(), 1);
7703            assert_eq!(r_slot_1_stores.len(), 1);
7704            assert_eq!(r_slot_0_stores.get(&0).unwrap().count(), 2);
7705            assert_eq!(r_slot_1_stores[&1].count(), 2);
7706            assert_eq!(r_slot_0_stores.get(&0).unwrap().approx_stored_count(), 2);
7707            assert_eq!(r_slot_1_stores[&1].approx_stored_count(), 2);
7708        }
7709
7710        // overwrite old rooted account version; only the r_slot_0_stores.count() should be
7711        // decremented
7712        db.store_uncached(2, &[(&pubkeys[0], &account)]);
7713        db.clean_accounts(None, false, None);
7714        {
7715            let slot_0_stores = &db.storage.get_slot_stores(0).unwrap();
7716            let slot_1_stores = &db.storage.get_slot_stores(1).unwrap();
7717            let r_slot_0_stores = slot_0_stores.read().unwrap();
7718            let r_slot_1_stores = slot_1_stores.read().unwrap();
7719            assert_eq!(r_slot_0_stores.len(), 1);
7720            assert_eq!(r_slot_1_stores.len(), 1);
7721            assert_eq!(r_slot_0_stores.get(&0).unwrap().count(), 1);
7722            assert_eq!(r_slot_1_stores[&1].count(), 2);
7723            assert_eq!(r_slot_0_stores.get(&0).unwrap().approx_stored_count(), 2);
7724            assert_eq!(r_slot_1_stores[&1].approx_stored_count(), 2);
7725        }
7726    }
7727
7728    #[test]
7729    fn test_accounts_unsquashed() {
7730        let key = Pubkey::default();
7731
7732        // 1 token in the "root", i.e. db zero
7733        let db0 = AccountsDb::new(Vec::new(), &ClusterType::Development);
7734        let account0 = AccountSharedData::new(1, 0, &key);
7735        db0.store_uncached(0, &[(&key, &account0)]);
7736
7737        // 0 carats in the child
7738        let account1 = AccountSharedData::new(0, 0, &key);
7739        db0.store_uncached(1, &[(&key, &account1)]);
7740
7741        // masking accounts is done at the Accounts level, at accountsDB we see
7742        // original account
7743        let ancestors = vec![(0, 0), (1, 1)].into_iter().collect();
7744        assert_eq!(
7745            db0.load_without_fixed_root(&ancestors, &key),
7746            Some((account1, 1))
7747        );
7748        let ancestors = vec![(0, 0)].into_iter().collect();
7749        assert_eq!(
7750            db0.load_without_fixed_root(&ancestors, &key),
7751            Some((account0, 0))
7752        );
7753    }
7754
7755    fn run_test_remove_unrooted_slot(is_cached: bool) {
7756        let unrooted_slot = 9;
7757        let unrooted_bank_id = 9;
7758        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7759        db.caching_enabled = true;
7760        let key = Pubkey::default();
7761        let account0 = AccountSharedData::new(1, 0, &key);
7762        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
7763        if is_cached {
7764            db.store_cached(unrooted_slot, &[(&key, &account0)]);
7765        } else {
7766            db.store_uncached(unrooted_slot, &[(&key, &account0)]);
7767        }
7768        db.bank_hashes
7769            .write()
7770            .unwrap()
7771            .insert(unrooted_slot, BankHashInfo::default());
7772        assert!(db
7773            .accounts_index
7774            .get(&key, Some(&ancestors), None)
7775            .is_some());
7776        assert_load_account(&db, unrooted_slot, key, 1);
7777
7778        // Purge the slot
7779        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
7780        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
7781        assert!(db.bank_hashes.read().unwrap().get(&unrooted_slot).is_none());
7782        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_none());
7783        assert!(db.storage.0.get(&unrooted_slot).is_none());
7784        assert!(db.accounts_index.get_account_read_entry(&key).is_none());
7785        assert!(db
7786            .accounts_index
7787            .get(&key, Some(&ancestors), None)
7788            .is_none());
7789
7790        // Test we can store for the same slot again and get the right information
7791        let account0 = AccountSharedData::new(2, 0, &key);
7792        db.store_uncached(unrooted_slot, &[(&key, &account0)]);
7793        assert_load_account(&db, unrooted_slot, key, 2);
7794    }
7795
7796    #[test]
7797    fn test_remove_unrooted_slot_cached() {
7798        run_test_remove_unrooted_slot(true);
7799    }
7800
7801    #[test]
7802    fn test_remove_unrooted_slot_storage() {
7803        run_test_remove_unrooted_slot(false);
7804    }
7805
7806    #[test]
7807    fn test_remove_unrooted_slot_snapshot() {
7808        gemachain_logger::setup();
7809        let unrooted_slot = 9;
7810        let unrooted_bank_id = 9;
7811        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
7812        let key = gemachain_sdk::pubkey::new_rand();
7813        let account0 = AccountSharedData::new(1, 0, &key);
7814        db.store_uncached(unrooted_slot, &[(&key, &account0)]);
7815
7816        // Purge the slot
7817        db.remove_unrooted_slots(&[(unrooted_slot, unrooted_bank_id)]);
7818
7819        // Add a new root
7820        let key2 = gemachain_sdk::pubkey::new_rand();
7821        let new_root = unrooted_slot + 1;
7822        db.store_uncached(new_root, &[(&key2, &account0)]);
7823        db.add_root(new_root);
7824
7825        // Simulate reconstruction from snapshot
7826        let db = reconstruct_accounts_db_via_serialization(&db, new_root);
7827
7828        // Check root account exists
7829        assert_load_account(&db, new_root, key2, 1);
7830
7831        // Check purged account stays gone
7832        let unrooted_slot_ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
7833        assert!(db
7834            .load_without_fixed_root(&unrooted_slot_ancestors, &key)
7835            .is_none());
7836    }
7837
7838    fn create_account(
7839        accounts: &AccountsDb,
7840        pubkeys: &mut Vec<Pubkey>,
7841        slot: Slot,
7842        num: usize,
7843        space: usize,
7844        num_vote: usize,
7845    ) {
7846        let ancestors = vec![(slot, 0)].into_iter().collect();
7847        for t in 0..num {
7848            let pubkey = gemachain_sdk::pubkey::new_rand();
7849            let account =
7850                AccountSharedData::new((t + 1) as u64, space, AccountSharedData::default().owner());
7851            pubkeys.push(pubkey);
7852            assert!(accounts
7853                .load_without_fixed_root(&ancestors, &pubkey)
7854                .is_none());
7855            accounts.store_uncached(slot, &[(&pubkey, &account)]);
7856        }
7857        for t in 0..num_vote {
7858            let pubkey = gemachain_sdk::pubkey::new_rand();
7859            let account =
7860                AccountSharedData::new((num + t + 1) as u64, space, &gemachain_vote_program::id());
7861            pubkeys.push(pubkey);
7862            let ancestors = vec![(slot, 0)].into_iter().collect();
7863            assert!(accounts
7864                .load_without_fixed_root(&ancestors, &pubkey)
7865                .is_none());
7866            accounts.store_uncached(slot, &[(&pubkey, &account)]);
7867        }
7868    }
7869
7870    fn update_accounts(accounts: &AccountsDb, pubkeys: &[Pubkey], slot: Slot, range: usize) {
7871        for _ in 1..1000 {
7872            let idx = thread_rng().gen_range(0, range);
7873            let ancestors = vec![(slot, 0)].into_iter().collect();
7874            if let Some((mut account, _)) =
7875                accounts.load_without_fixed_root(&ancestors, &pubkeys[idx])
7876            {
7877                account.checked_add_carats(1).unwrap();
7878                accounts.store_uncached(slot, &[(&pubkeys[idx], &account)]);
7879                if account.carats() == 0 {
7880                    let ancestors = vec![(slot, 0)].into_iter().collect();
7881                    assert!(accounts
7882                        .load_without_fixed_root(&ancestors, &pubkeys[idx])
7883                        .is_none());
7884                } else {
7885                    let default_account = AccountSharedData::from(Account {
7886                        carats: account.carats(),
7887                        ..Account::default()
7888                    });
7889                    assert_eq!(default_account, account);
7890                }
7891            }
7892        }
7893    }
7894
7895    fn check_storage(accounts: &AccountsDb, slot: Slot, count: usize) -> bool {
7896        assert_eq!(
7897            accounts
7898                .storage
7899                .get_slot_stores(slot)
7900                .unwrap()
7901                .read()
7902                .unwrap()
7903                .len(),
7904            1
7905        );
7906        let slot_storages = accounts.storage.get_slot_stores(slot).unwrap();
7907        let mut total_count: usize = 0;
7908        let r_slot_storages = slot_storages.read().unwrap();
7909        for store in r_slot_storages.values() {
7910            assert_eq!(store.status(), AccountStorageStatus::Available);
7911            total_count += store.count();
7912        }
7913        assert_eq!(total_count, count);
7914        let (expected_store_count, actual_store_count): (usize, usize) = (
7915            r_slot_storages
7916                .values()
7917                .map(|s| s.approx_stored_count())
7918                .sum(),
7919            r_slot_storages
7920                .values()
7921                .map(|s| s.all_accounts().len())
7922                .sum(),
7923        );
7924        assert_eq!(expected_store_count, actual_store_count);
7925        total_count == count
7926    }
7927
7928    fn check_accounts(
7929        accounts: &AccountsDb,
7930        pubkeys: &[Pubkey],
7931        slot: Slot,
7932        num: usize,
7933        count: usize,
7934    ) {
7935        let ancestors = vec![(slot, 0)].into_iter().collect();
7936        for _ in 0..num {
7937            let idx = thread_rng().gen_range(0, num);
7938            let account = accounts.load_without_fixed_root(&ancestors, &pubkeys[idx]);
7939            let account1 = Some((
7940                AccountSharedData::new(
7941                    (idx + count) as u64,
7942                    0,
7943                    AccountSharedData::default().owner(),
7944                ),
7945                slot,
7946            ));
7947            assert_eq!(account, account1);
7948        }
7949    }
7950
7951    #[allow(clippy::needless_range_loop)]
7952    fn modify_accounts(
7953        accounts: &AccountsDb,
7954        pubkeys: &[Pubkey],
7955        slot: Slot,
7956        num: usize,
7957        count: usize,
7958    ) {
7959        for idx in 0..num {
7960            let account = AccountSharedData::new(
7961                (idx + count) as u64,
7962                0,
7963                AccountSharedData::default().owner(),
7964            );
7965            accounts.store_uncached(slot, &[(&pubkeys[idx], &account)]);
7966        }
7967    }
7968
7969    #[test]
7970    fn test_account_one() {
7971        let (_accounts_dirs, paths) = get_temp_accounts_paths(1).unwrap();
7972        let db = AccountsDb::new(paths, &ClusterType::Development);
7973        let mut pubkeys: Vec<Pubkey> = vec![];
7974        create_account(&db, &mut pubkeys, 0, 1, 0, 0);
7975        let ancestors = vec![(0, 0)].into_iter().collect();
7976        let account = db.load_without_fixed_root(&ancestors, &pubkeys[0]).unwrap();
7977        let default_account = AccountSharedData::from(Account {
7978            carats: 1,
7979            ..Account::default()
7980        });
7981        assert_eq!((default_account, 0), account);
7982    }
7983
7984    #[test]
7985    fn test_account_many() {
7986        let (_accounts_dirs, paths) = get_temp_accounts_paths(2).unwrap();
7987        let db = AccountsDb::new(paths, &ClusterType::Development);
7988        let mut pubkeys: Vec<Pubkey> = vec![];
7989        create_account(&db, &mut pubkeys, 0, 100, 0, 0);
7990        check_accounts(&db, &pubkeys, 0, 100, 1);
7991    }
7992
7993    #[test]
7994    fn test_account_update() {
7995        let accounts = AccountsDb::new_single_for_tests();
7996        let mut pubkeys: Vec<Pubkey> = vec![];
7997        create_account(&accounts, &mut pubkeys, 0, 100, 0, 0);
7998        update_accounts(&accounts, &pubkeys, 0, 99);
7999        assert!(check_storage(&accounts, 0, 100));
8000    }
8001
8002    #[test]
8003    fn test_account_grow_many() {
8004        let (_accounts_dir, paths) = get_temp_accounts_paths(2).unwrap();
8005        let size = 4096;
8006        let accounts = AccountsDb::new_sized(paths, size);
8007        let mut keys = vec![];
8008        for i in 0..9 {
8009            let key = gemachain_sdk::pubkey::new_rand();
8010            let account = AccountSharedData::new(i + 1, size as usize / 4, &key);
8011            accounts.store_uncached(0, &[(&key, &account)]);
8012            keys.push(key);
8013        }
8014        let ancestors = vec![(0, 0)].into_iter().collect();
8015        for (i, key) in keys.iter().enumerate() {
8016            assert_eq!(
8017                accounts
8018                    .load_without_fixed_root(&ancestors, key)
8019                    .unwrap()
8020                    .0
8021                    .carats(),
8022                (i as u64) + 1
8023            );
8024        }
8025
8026        let mut append_vec_histogram = HashMap::new();
8027        let mut all_storages = vec![];
8028        for slot_storage in accounts.storage.0.iter() {
8029            all_storages.extend(slot_storage.read().unwrap().values().cloned())
8030        }
8031        for storage in all_storages {
8032            *append_vec_histogram.entry(storage.slot()).or_insert(0) += 1;
8033        }
8034        for count in append_vec_histogram.values() {
8035            assert!(*count >= 2);
8036        }
8037    }
8038
8039    #[test]
8040    fn test_account_grow() {
8041        let accounts = AccountsDb::new_single_for_tests();
8042
8043        let status = [AccountStorageStatus::Available, AccountStorageStatus::Full];
8044        let pubkey1 = gemachain_sdk::pubkey::new_rand();
8045        let account1 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey1);
8046        accounts.store_uncached(0, &[(&pubkey1, &account1)]);
8047        {
8048            let stores = &accounts.storage.get_slot_stores(0).unwrap();
8049            let r_stores = stores.read().unwrap();
8050            assert_eq!(r_stores.len(), 1);
8051            assert_eq!(r_stores[&0].count(), 1);
8052            assert_eq!(r_stores[&0].status(), AccountStorageStatus::Available);
8053        }
8054
8055        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8056        let account2 = AccountSharedData::new(1, DEFAULT_FILE_SIZE as usize / 2, &pubkey2);
8057        accounts.store_uncached(0, &[(&pubkey2, &account2)]);
8058        {
8059            assert_eq!(accounts.storage.0.len(), 1);
8060            let stores = &accounts.storage.get_slot_stores(0).unwrap();
8061            let r_stores = stores.read().unwrap();
8062            assert_eq!(r_stores.len(), 2);
8063            assert_eq!(r_stores[&0].count(), 1);
8064            assert_eq!(r_stores[&0].status(), AccountStorageStatus::Full);
8065            assert_eq!(r_stores[&1].count(), 1);
8066            assert_eq!(r_stores[&1].status(), AccountStorageStatus::Available);
8067        }
8068        let ancestors = vec![(0, 0)].into_iter().collect();
8069        assert_eq!(
8070            accounts
8071                .load_without_fixed_root(&ancestors, &pubkey1)
8072                .unwrap()
8073                .0,
8074            account1
8075        );
8076        assert_eq!(
8077            accounts
8078                .load_without_fixed_root(&ancestors, &pubkey2)
8079                .unwrap()
8080                .0,
8081            account2
8082        );
8083
8084        // lots of stores, but 7 storages should be enough for everything
8085        for _ in 0..25 {
8086            accounts.store_uncached(0, &[(&pubkey1, &account1)]);
8087            {
8088                assert_eq!(accounts.storage.0.len(), 1);
8089                let stores = &accounts.storage.get_slot_stores(0).unwrap();
8090                let r_stores = stores.read().unwrap();
8091                assert!(r_stores.len() <= 7);
8092                assert_eq!(r_stores[&0].status(), status[0]);
8093            }
8094            let ancestors = vec![(0, 0)].into_iter().collect();
8095            assert_eq!(
8096                accounts
8097                    .load_without_fixed_root(&ancestors, &pubkey1)
8098                    .unwrap()
8099                    .0,
8100                account1
8101            );
8102            assert_eq!(
8103                accounts
8104                    .load_without_fixed_root(&ancestors, &pubkey2)
8105                    .unwrap()
8106                    .0,
8107                account2
8108            );
8109        }
8110    }
8111
8112    #[test]
8113    fn test_lazy_gc_slot() {
8114        gemachain_logger::setup();
8115        //This test is pedantic
8116        //A slot is purged when a non root bank is cleaned up.  If a slot is behind root but it is
8117        //not root, it means we are retaining dead banks.
8118        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8119        let pubkey = gemachain_sdk::pubkey::new_rand();
8120        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8121        //store an account
8122        accounts.store_uncached(0, &[(&pubkey, &account)]);
8123        let ancestors = vec![(0, 0)].into_iter().collect();
8124        let id = {
8125            let (lock, idx) = accounts
8126                .accounts_index
8127                .get(&pubkey, Some(&ancestors), None)
8128                .unwrap();
8129            lock.slot_list()[idx].1.store_id
8130        };
8131        accounts.get_accounts_delta_hash(0);
8132        accounts.add_root(1);
8133
8134        //slot is still there, since gc is lazy
8135        assert!(accounts
8136            .storage
8137            .get_slot_stores(0)
8138            .unwrap()
8139            .read()
8140            .unwrap()
8141            .get(&id)
8142            .is_some());
8143
8144        //store causes clean
8145        accounts.store_uncached(1, &[(&pubkey, &account)]);
8146
8147        // generate delta state for slot 1, so clean operates on it.
8148        accounts.get_accounts_delta_hash(1);
8149
8150        //slot is gone
8151        accounts.print_accounts_stats("pre-clean");
8152        accounts.clean_accounts(None, false, None);
8153        assert!(accounts.storage.0.get(&0).is_none());
8154
8155        //new value is there
8156        let ancestors = vec![(1, 1)].into_iter().collect();
8157        assert_eq!(
8158            accounts.load_without_fixed_root(&ancestors, &pubkey),
8159            Some((account, 1))
8160        );
8161    }
8162
8163    impl AccountsDb {
8164        fn all_account_count_in_append_vec(&self, slot: Slot) -> usize {
8165            let slot_storage = self.storage.get_slot_stores(slot);
8166            if let Some(slot_storage) = slot_storage {
8167                let r_slot_storage = slot_storage.read().unwrap();
8168                let count = r_slot_storage
8169                    .values()
8170                    .map(|store| store.all_accounts().len())
8171                    .sum();
8172                let stored_count: usize = r_slot_storage
8173                    .values()
8174                    .map(|store| store.approx_stored_count())
8175                    .sum();
8176                assert_eq!(stored_count, count);
8177                count
8178            } else {
8179                0
8180            }
8181        }
8182
8183        pub fn ref_count_for_pubkey(&self, pubkey: &Pubkey) -> RefCount {
8184            self.accounts_index.ref_count_from_storage(pubkey)
8185        }
8186    }
8187
8188    #[test]
8189    fn test_clean_zero_carat_and_dead_slot() {
8190        gemachain_logger::setup();
8191
8192        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8193        let pubkey1 = gemachain_sdk::pubkey::new_rand();
8194        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8195        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
8196        let zero_carat_account =
8197            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8198
8199        // Store two accounts
8200        accounts.store_uncached(0, &[(&pubkey1, &account)]);
8201        accounts.store_uncached(0, &[(&pubkey2, &account)]);
8202
8203        // Make sure both accounts are in the same AppendVec in slot 0, which
8204        // will prevent pubkey1 from being cleaned up later even when it's a
8205        // zero-carat account
8206        let ancestors = vec![(0, 1)].into_iter().collect();
8207        let (slot1, account_info1) = accounts
8208            .accounts_index
8209            .get(&pubkey1, Some(&ancestors), None)
8210            .map(|(account_list1, index1)| account_list1.slot_list()[index1])
8211            .unwrap();
8212        let (slot2, account_info2) = accounts
8213            .accounts_index
8214            .get(&pubkey2, Some(&ancestors), None)
8215            .map(|(account_list2, index2)| account_list2.slot_list()[index2])
8216            .unwrap();
8217        assert_eq!(slot1, 0);
8218        assert_eq!(slot1, slot2);
8219        assert_eq!(account_info1.store_id, account_info2.store_id);
8220
8221        // Update account 1 in slot 1
8222        accounts.store_uncached(1, &[(&pubkey1, &account)]);
8223
8224        // Update account 1 as  zero carats account
8225        accounts.store_uncached(2, &[(&pubkey1, &zero_carat_account)]);
8226
8227        // Pubkey 1 was the only account in slot 1, and it was updated in slot 2, so
8228        // slot 1 should be purged
8229        accounts.add_root(0);
8230        accounts.add_root(1);
8231        accounts.add_root(2);
8232
8233        // Slot 1 should be removed, slot 0 cannot be removed because it still has
8234        // the latest update for pubkey 2
8235        accounts.clean_accounts(None, false, None);
8236        assert!(accounts.storage.get_slot_stores(0).is_some());
8237        assert!(accounts.storage.get_slot_stores(1).is_none());
8238
8239        // Slot 1 should be cleaned because all it's accounts are
8240        // zero carats, and are not present in any other slot's
8241        // storage entries
8242        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
8243    }
8244
8245    #[test]
8246    fn test_clean_multiple_zero_carat_decrements_index_ref_count() {
8247        gemachain_logger::setup();
8248
8249        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8250        let pubkey1 = gemachain_sdk::pubkey::new_rand();
8251        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8252        let zero_carat_account =
8253            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8254
8255        // Store 2 accounts in slot 0, then update account 1 in two more slots
8256        accounts.store_uncached(0, &[(&pubkey1, &zero_carat_account)]);
8257        accounts.store_uncached(0, &[(&pubkey2, &zero_carat_account)]);
8258        accounts.store_uncached(1, &[(&pubkey1, &zero_carat_account)]);
8259        accounts.store_uncached(2, &[(&pubkey1, &zero_carat_account)]);
8260        // Root all slots
8261        accounts.add_root(0);
8262        accounts.add_root(1);
8263        accounts.add_root(2);
8264
8265        // Account ref counts should match how many slots they were stored in
8266        // Account 1 = 3 slots; account 2 = 1 slot
8267        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 3);
8268        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 1);
8269
8270        accounts.clean_accounts(None, false, None);
8271        // Slots 0 and 1 should each have been cleaned because all of their
8272        // accounts are zero carats
8273        assert!(accounts.storage.get_slot_stores(0).is_none());
8274        assert!(accounts.storage.get_slot_stores(1).is_none());
8275        // Slot 2 only has a zero carat account as well. But, calc_delete_dependencies()
8276        // should exclude slot 2 from the clean due to changes in other slots
8277        assert!(accounts.storage.get_slot_stores(2).is_some());
8278        // Index ref counts should be consistent with the slot stores. Account 1 ref count
8279        // should be 1 since slot 2 is the only alive slot; account 2 should have a ref
8280        // count of 0 due to slot 0 being dead
8281        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 1);
8282        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey2), 0);
8283
8284        accounts.clean_accounts(None, false, None);
8285        // Slot 2 will now be cleaned, which will leave account 1 with a ref count of 0
8286        assert!(accounts.storage.get_slot_stores(2).is_none());
8287        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
8288    }
8289
8290    #[test]
8291    fn test_clean_zero_carat_and_old_roots() {
8292        gemachain_logger::setup();
8293
8294        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8295        let pubkey = gemachain_sdk::pubkey::new_rand();
8296        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8297        let zero_carat_account =
8298            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8299
8300        // Store a zero-carat account
8301        accounts.store_uncached(0, &[(&pubkey, &account)]);
8302        accounts.store_uncached(1, &[(&pubkey, &zero_carat_account)]);
8303
8304        // Simulate rooting the zero-carat account, should be a
8305        // candidate for cleaning
8306        accounts.add_root(0);
8307        accounts.add_root(1);
8308
8309        // Slot 0 should be removed, and
8310        // zero-carat account should be cleaned
8311        accounts.clean_accounts(None, false, None);
8312
8313        assert!(accounts.storage.get_slot_stores(0).is_none());
8314        assert!(accounts.storage.get_slot_stores(1).is_none());
8315
8316        // Slot 0 should be cleaned because all it's accounts have been
8317        // updated in the rooted slot 1
8318        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
8319
8320        // Slot 1 should be cleaned because all it's accounts are
8321        // zero carats, and are not present in any other slot's
8322        // storage entries
8323        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
8324
8325        // zero carat account, should no longer exist in accounts index
8326        // because it has been removed
8327        assert!(accounts.accounts_index.get(&pubkey, None, None).is_none());
8328    }
8329
8330    #[test]
8331    fn test_clean_old_with_normal_account() {
8332        gemachain_logger::setup();
8333
8334        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8335        let pubkey = gemachain_sdk::pubkey::new_rand();
8336        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8337        //store an account
8338        accounts.store_uncached(0, &[(&pubkey, &account)]);
8339        accounts.store_uncached(1, &[(&pubkey, &account)]);
8340
8341        // simulate slots are rooted after while
8342        accounts.get_accounts_delta_hash(0);
8343        accounts.add_root(0);
8344        accounts.get_accounts_delta_hash(1);
8345        accounts.add_root(1);
8346
8347        //even if rooted, old state isn't cleaned up
8348        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
8349        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
8350
8351        accounts.clean_accounts(None, false, None);
8352
8353        //now old state is cleaned up
8354        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
8355        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
8356    }
8357
8358    #[test]
8359    fn test_clean_old_with_zero_carat_account() {
8360        gemachain_logger::setup();
8361
8362        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8363        let pubkey1 = gemachain_sdk::pubkey::new_rand();
8364        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8365        let normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8366        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8367        //store an account
8368        accounts.store_uncached(0, &[(&pubkey1, &normal_account)]);
8369        accounts.store_uncached(1, &[(&pubkey1, &zero_account)]);
8370        accounts.store_uncached(0, &[(&pubkey2, &normal_account)]);
8371        accounts.store_uncached(1, &[(&pubkey2, &normal_account)]);
8372
8373        //simulate slots are rooted after while
8374        accounts.get_accounts_delta_hash(0);
8375        accounts.add_root(0);
8376        accounts.get_accounts_delta_hash(1);
8377        accounts.add_root(1);
8378
8379        //even if rooted, old state isn't cleaned up
8380        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
8381        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
8382
8383        accounts.print_accounts_stats("");
8384
8385        accounts.clean_accounts(None, false, None);
8386
8387        //Old state behind zero-carat account is cleaned up
8388        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
8389        assert_eq!(accounts.alive_account_count_in_slot(1), 2);
8390    }
8391
8392    #[test]
8393    fn test_clean_old_with_both_normal_and_zero_carat_accounts() {
8394        gemachain_logger::setup();
8395
8396        let mut accounts = AccountsDb::new_with_config_for_tests(
8397            Vec::new(),
8398            &ClusterType::Development,
8399            spl_token_mint_index_enabled(),
8400            false,
8401            AccountShrinkThreshold::default(),
8402        );
8403        let pubkey1 = gemachain_sdk::pubkey::new_rand();
8404        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8405
8406        // Set up account to be added to secondary index
8407        let mint_key = Pubkey::new_unique();
8408        let mut account_data_with_mint =
8409            vec![0; inline_spl_token_v2_0::state::Account::get_packed_len()];
8410        account_data_with_mint[..PUBKEY_BYTES].clone_from_slice(&(mint_key.to_bytes()));
8411
8412        let mut normal_account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8413        normal_account.set_owner(inline_spl_token_v2_0::id());
8414        normal_account.set_data(account_data_with_mint.clone());
8415        let mut zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8416        zero_account.set_owner(inline_spl_token_v2_0::id());
8417        zero_account.set_data(account_data_with_mint);
8418
8419        //store an account
8420        accounts.store_uncached(0, &[(&pubkey1, &normal_account)]);
8421        accounts.store_uncached(0, &[(&pubkey1, &normal_account)]);
8422        accounts.store_uncached(1, &[(&pubkey1, &zero_account)]);
8423        accounts.store_uncached(0, &[(&pubkey2, &normal_account)]);
8424        accounts.store_uncached(2, &[(&pubkey2, &normal_account)]);
8425
8426        //simulate slots are rooted after while
8427        accounts.get_accounts_delta_hash(0);
8428        accounts.add_root(0);
8429        accounts.get_accounts_delta_hash(1);
8430        accounts.add_root(1);
8431        accounts.get_accounts_delta_hash(2);
8432        accounts.add_root(2);
8433
8434        //even if rooted, old state isn't cleaned up
8435        assert_eq!(accounts.alive_account_count_in_slot(0), 2);
8436        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
8437        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
8438
8439        // Secondary index should still find both pubkeys
8440        let mut found_accounts = HashSet::new();
8441        let index_key = IndexKey::SplTokenMint(mint_key);
8442        let bank_id = 0;
8443        accounts
8444            .accounts_index
8445            .index_scan_accounts(&Ancestors::default(), bank_id, index_key, |key, _| {
8446                found_accounts.insert(*key);
8447            })
8448            .unwrap();
8449        assert_eq!(found_accounts.len(), 2);
8450        assert!(found_accounts.contains(&pubkey1));
8451        assert!(found_accounts.contains(&pubkey2));
8452
8453        {
8454            accounts.account_indexes.keys = Some(AccountSecondaryIndexesIncludeExclude {
8455                exclude: true,
8456                keys: [mint_key].iter().cloned().collect::<HashSet<Pubkey>>(),
8457            });
8458            // Secondary index can't be used - do normal scan: should still find both pubkeys
8459            let found_accounts = accounts
8460                .index_scan_accounts(
8461                    &Ancestors::default(),
8462                    bank_id,
8463                    index_key,
8464                    |collection: &mut HashSet<Pubkey>, account| {
8465                        collection.insert(*account.unwrap().0);
8466                    },
8467                )
8468                .unwrap();
8469            assert!(!found_accounts.1);
8470            assert_eq!(found_accounts.0.len(), 2);
8471            assert!(found_accounts.0.contains(&pubkey1));
8472            assert!(found_accounts.0.contains(&pubkey2));
8473
8474            accounts.account_indexes.keys = None;
8475
8476            // Secondary index can now be used since it isn't marked as excluded
8477            let found_accounts = accounts
8478                .index_scan_accounts(
8479                    &Ancestors::default(),
8480                    bank_id,
8481                    index_key,
8482                    |collection: &mut HashSet<Pubkey>, account| {
8483                        collection.insert(*account.unwrap().0);
8484                    },
8485                )
8486                .unwrap();
8487            assert!(found_accounts.1);
8488            assert_eq!(found_accounts.0.len(), 2);
8489            assert!(found_accounts.0.contains(&pubkey1));
8490            assert!(found_accounts.0.contains(&pubkey2));
8491
8492            accounts.account_indexes.keys = None;
8493        }
8494
8495        accounts.clean_accounts(None, false, None);
8496
8497        //both zero carat and normal accounts are cleaned up
8498        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
8499        // The only store to slot 1 was a zero carat account, should
8500        // be purged by zero-carat cleaning logic because slot 1 is
8501        // rooted
8502        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
8503        assert_eq!(accounts.alive_account_count_in_slot(2), 1);
8504
8505        // `pubkey1`, a zero carat account, should no longer exist in accounts index
8506        // because it has been removed by the clean
8507        assert!(accounts.accounts_index.get(&pubkey1, None, None).is_none());
8508
8509        // Secondary index should have purged `pubkey1` as well
8510        let mut found_accounts = vec![];
8511        accounts
8512            .accounts_index
8513            .index_scan_accounts(
8514                &Ancestors::default(),
8515                bank_id,
8516                IndexKey::SplTokenMint(mint_key),
8517                |key, _| found_accounts.push(*key),
8518            )
8519            .unwrap();
8520        assert_eq!(found_accounts, vec![pubkey2]);
8521    }
8522
8523    #[test]
8524    fn test_clean_max_slot_zero_carat_account() {
8525        gemachain_logger::setup();
8526
8527        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8528        let pubkey = gemachain_sdk::pubkey::new_rand();
8529        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8530        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8531
8532        // store an account, make it a zero carat account
8533        // in slot 1
8534        accounts.store_uncached(0, &[(&pubkey, &account)]);
8535        accounts.store_uncached(1, &[(&pubkey, &zero_account)]);
8536
8537        // simulate slots are rooted after while
8538        accounts.add_root(0);
8539        accounts.add_root(1);
8540
8541        // Only clean up to account 0, should not purge slot 0 based on
8542        // updates in later slots in slot 1
8543        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
8544        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
8545        accounts.clean_accounts(Some(0), false, None);
8546        assert_eq!(accounts.alive_account_count_in_slot(0), 1);
8547        assert_eq!(accounts.alive_account_count_in_slot(1), 1);
8548        assert!(accounts.accounts_index.get(&pubkey, None, None).is_some());
8549
8550        // Now the account can be cleaned up
8551        accounts.clean_accounts(Some(1), false, None);
8552        assert_eq!(accounts.alive_account_count_in_slot(0), 0);
8553        assert_eq!(accounts.alive_account_count_in_slot(1), 0);
8554
8555        // The zero carat account, should no longer exist in accounts index
8556        // because it has been removed
8557        assert!(accounts.accounts_index.get(&pubkey, None, None).is_none());
8558    }
8559
8560    #[test]
8561    fn test_uncleaned_roots_with_account() {
8562        gemachain_logger::setup();
8563
8564        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8565        let pubkey = gemachain_sdk::pubkey::new_rand();
8566        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
8567        //store an account
8568        accounts.store_uncached(0, &[(&pubkey, &account)]);
8569        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
8570
8571        // simulate slots are rooted after while
8572        accounts.add_root(0);
8573        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
8574
8575        //now uncleaned roots are cleaned up
8576        accounts.clean_accounts(None, false, None);
8577        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
8578    }
8579
8580    #[test]
8581    fn test_uncleaned_roots_with_no_account() {
8582        gemachain_logger::setup();
8583
8584        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
8585
8586        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
8587
8588        // simulate slots are rooted after while
8589        accounts.add_root(0);
8590        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 1);
8591
8592        //now uncleaned roots are cleaned up
8593        accounts.clean_accounts(None, false, None);
8594        assert_eq!(accounts.accounts_index.uncleaned_roots_len(), 0);
8595    }
8596
8597    #[test]
8598    fn test_accounts_db_serialize1() {
8599        gemachain_logger::setup();
8600        let accounts = AccountsDb::new_single_for_tests();
8601        let mut pubkeys: Vec<Pubkey> = vec![];
8602
8603        // Create 100 accounts in slot 0
8604        create_account(&accounts, &mut pubkeys, 0, 100, 0, 0);
8605        accounts.clean_accounts(None, false, None);
8606        check_accounts(&accounts, &pubkeys, 0, 100, 1);
8607
8608        // do some updates to those accounts and re-check
8609        modify_accounts(&accounts, &pubkeys, 0, 100, 2);
8610        assert!(check_storage(&accounts, 0, 100));
8611        check_accounts(&accounts, &pubkeys, 0, 100, 2);
8612        accounts.get_accounts_delta_hash(0);
8613        accounts.add_root(0);
8614
8615        let mut pubkeys1: Vec<Pubkey> = vec![];
8616
8617        // CREATE SLOT 1
8618        let latest_slot = 1;
8619
8620        // Modify the first 10 of the accounts from slot 0 in slot 1
8621        modify_accounts(&accounts, &pubkeys, latest_slot, 10, 3);
8622        // Overwrite account 30 from slot 0 with carats=0 into slot 1.
8623        // Slot 1 should now have 10 + 1 = 11 accounts
8624        let account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8625        accounts.store_uncached(latest_slot, &[(&pubkeys[30], &account)]);
8626
8627        // Create 10 new accounts in slot 1, should now have 11 + 10 = 21
8628        // accounts
8629        create_account(&accounts, &mut pubkeys1, latest_slot, 10, 0, 0);
8630
8631        accounts.get_accounts_delta_hash(latest_slot);
8632        accounts.add_root(latest_slot);
8633        assert!(check_storage(&accounts, 1, 21));
8634
8635        // CREATE SLOT 2
8636        let latest_slot = 2;
8637        let mut pubkeys2: Vec<Pubkey> = vec![];
8638
8639        // Modify first 20 of the accounts from slot 0 in slot 2
8640        modify_accounts(&accounts, &pubkeys, latest_slot, 20, 4);
8641        accounts.clean_accounts(None, false, None);
8642        // Overwrite account 31 from slot 0 with carats=0 into slot 2.
8643        // Slot 2 should now have 20 + 1 = 21 accounts
8644        let account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
8645        accounts.store_uncached(latest_slot, &[(&pubkeys[31], &account)]);
8646
8647        // Create 10 new accounts in slot 2. Slot 2 should now have
8648        // 21 + 10 = 31 accounts
8649        create_account(&accounts, &mut pubkeys2, latest_slot, 10, 0, 0);
8650
8651        accounts.get_accounts_delta_hash(latest_slot);
8652        accounts.add_root(latest_slot);
8653        assert!(check_storage(&accounts, 2, 31));
8654
8655        accounts.clean_accounts(None, false, None);
8656        // The first 20 accounts of slot 0 have been updated in slot 2, as well as
8657        // accounts 30 and  31 (overwritten with zero-carat accounts in slot 1 and
8658        // slot 2 respectively), so only 78 accounts are left in slot 0's storage entries.
8659        assert!(check_storage(&accounts, 0, 78));
8660        // 10 of the 21 accounts have been modified in slot 2, so only 11
8661        // accounts left in slot 1.
8662        assert!(check_storage(&accounts, 1, 11));
8663        assert!(check_storage(&accounts, 2, 31));
8664
8665        let daccounts = reconstruct_accounts_db_via_serialization(&accounts, latest_slot);
8666
8667        assert_eq!(
8668            daccounts.write_version.load(Ordering::Relaxed),
8669            accounts.write_version.load(Ordering::Relaxed)
8670        );
8671
8672        // Get the hash for the latest slot, which should be the only hash in the
8673        // bank_hashes map on the deserialized AccountsDb
8674        assert_eq!(daccounts.bank_hashes.read().unwrap().len(), 2);
8675        assert_eq!(
8676            daccounts.bank_hashes.read().unwrap().get(&latest_slot),
8677            accounts.bank_hashes.read().unwrap().get(&latest_slot)
8678        );
8679
8680        daccounts.print_count_and_status("daccounts");
8681
8682        // Don't check the first 35 accounts which have not been modified on slot 0
8683        check_accounts(&daccounts, &pubkeys[35..], 0, 65, 37);
8684        check_accounts(&daccounts, &pubkeys1, 1, 10, 1);
8685        assert!(check_storage(&daccounts, 0, 100));
8686        assert!(check_storage(&daccounts, 1, 21));
8687        assert!(check_storage(&daccounts, 2, 31));
8688
8689        let ancestors = linear_ancestors(latest_slot);
8690        assert_eq!(
8691            daccounts.update_accounts_hash(latest_slot, &ancestors),
8692            accounts.update_accounts_hash(latest_slot, &ancestors)
8693        );
8694    }
8695
8696    fn assert_load_account(
8697        accounts: &AccountsDb,
8698        slot: Slot,
8699        pubkey: Pubkey,
8700        expected_carats: u64,
8701    ) {
8702        let ancestors = vec![(slot, 0)].into_iter().collect();
8703        let (account, slot) = accounts
8704            .load_without_fixed_root(&ancestors, &pubkey)
8705            .unwrap();
8706        assert_eq!((account.carats(), slot), (expected_carats, slot));
8707    }
8708
8709    fn assert_not_load_account(accounts: &AccountsDb, slot: Slot, pubkey: Pubkey) {
8710        let ancestors = vec![(slot, 0)].into_iter().collect();
8711        assert!(accounts
8712            .load_without_fixed_root(&ancestors, &pubkey)
8713            .is_none());
8714    }
8715
8716    fn reconstruct_accounts_db_via_serialization(accounts: &AccountsDb, slot: Slot) -> AccountsDb {
8717        let daccounts =
8718            crate::serde_snapshot::reconstruct_accounts_db_via_serialization(accounts, slot);
8719        daccounts.print_count_and_status("daccounts");
8720        daccounts
8721    }
8722
8723    fn assert_no_stores(accounts: &AccountsDb, slot: Slot) {
8724        let slot_stores = accounts.storage.get_slot_stores(slot);
8725        let r_slot_stores = slot_stores.as_ref().map(|slot_stores| {
8726            let r_slot_stores = slot_stores.read().unwrap();
8727            info!("{:?}", *r_slot_stores);
8728            r_slot_stores
8729        });
8730        assert!(r_slot_stores.is_none() || r_slot_stores.unwrap().is_empty());
8731    }
8732
8733    #[test]
8734    fn test_accounts_db_purge_keep_live() {
8735        gemachain_logger::setup();
8736        let some_carat = 223;
8737        let zero_carat = 0;
8738        let no_data = 0;
8739        let owner = *AccountSharedData::default().owner();
8740
8741        let account = AccountSharedData::new(some_carat, no_data, &owner);
8742        let pubkey = gemachain_sdk::pubkey::new_rand();
8743
8744        let account2 = AccountSharedData::new(some_carat, no_data, &owner);
8745        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8746
8747        let zero_carat_account = AccountSharedData::new(zero_carat, no_data, &owner);
8748
8749        let accounts = AccountsDb::new_single_for_tests();
8750        accounts.add_root(0);
8751
8752        // Step A
8753        let mut current_slot = 1;
8754        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
8755        // Store another live account to slot 1 which will prevent any purge
8756        // since the store count will not be zero
8757        accounts.store_uncached(current_slot, &[(&pubkey2, &account2)]);
8758        accounts.add_root(current_slot);
8759        let (slot1, account_info1) = accounts
8760            .accounts_index
8761            .get(&pubkey, None, None)
8762            .map(|(account_list1, index1)| account_list1.slot_list()[index1])
8763            .unwrap();
8764        let (slot2, account_info2) = accounts
8765            .accounts_index
8766            .get(&pubkey2, None, None)
8767            .map(|(account_list2, index2)| account_list2.slot_list()[index2])
8768            .unwrap();
8769        assert_eq!(slot1, current_slot);
8770        assert_eq!(slot1, slot2);
8771        assert_eq!(account_info1.store_id, account_info2.store_id);
8772
8773        // Step B
8774        current_slot += 1;
8775        let zero_carat_slot = current_slot;
8776        accounts.store_uncached(current_slot, &[(&pubkey, &zero_carat_account)]);
8777        accounts.add_root(current_slot);
8778
8779        assert_load_account(&accounts, current_slot, pubkey, zero_carat);
8780
8781        current_slot += 1;
8782        accounts.add_root(current_slot);
8783
8784        accounts.print_accounts_stats("pre_purge");
8785
8786        accounts.clean_accounts(None, false, None);
8787
8788        accounts.print_accounts_stats("post_purge");
8789
8790        // The earlier entry for pubkey in the account index is purged,
8791        let (slot_list_len, index_slot) = {
8792            let account_entry = accounts
8793                .accounts_index
8794                .get_account_read_entry(&pubkey)
8795                .unwrap();
8796            let slot_list = account_entry.slot_list();
8797            (slot_list.len(), slot_list[0].0)
8798        };
8799        assert_eq!(slot_list_len, 1);
8800        // Zero carat entry was not the one purged
8801        assert_eq!(index_slot, zero_carat_slot);
8802        // The ref count should still be 2 because no slots were purged
8803        assert_eq!(accounts.ref_count_for_pubkey(&pubkey), 2);
8804
8805        // storage for slot 1 had 2 accounts, now has 1 after pubkey 1
8806        // was reclaimed
8807        check_storage(&accounts, 1, 1);
8808        // storage for slot 2 had 1 accounts, now has 1
8809        check_storage(&accounts, 2, 1);
8810    }
8811
8812    #[test]
8813    fn test_accounts_db_purge1() {
8814        gemachain_logger::setup();
8815        let some_carat = 223;
8816        let zero_carat = 0;
8817        let no_data = 0;
8818        let owner = *AccountSharedData::default().owner();
8819
8820        let account = AccountSharedData::new(some_carat, no_data, &owner);
8821        let pubkey = gemachain_sdk::pubkey::new_rand();
8822
8823        let zero_carat_account = AccountSharedData::new(zero_carat, no_data, &owner);
8824
8825        let accounts = AccountsDb::new_single_for_tests();
8826        accounts.add_root(0);
8827
8828        let mut current_slot = 1;
8829        accounts.set_hash(current_slot, current_slot - 1);
8830        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
8831        accounts.add_root(current_slot);
8832
8833        current_slot += 1;
8834        accounts.set_hash(current_slot, current_slot - 1);
8835        accounts.store_uncached(current_slot, &[(&pubkey, &zero_carat_account)]);
8836        accounts.add_root(current_slot);
8837
8838        assert_load_account(&accounts, current_slot, pubkey, zero_carat);
8839
8840        // Otherwise slot 2 will not be removed
8841        current_slot += 1;
8842        accounts.set_hash(current_slot, current_slot - 1);
8843        accounts.add_root(current_slot);
8844
8845        accounts.print_accounts_stats("pre_purge");
8846
8847        let ancestors = linear_ancestors(current_slot);
8848        info!("ancestors: {:?}", ancestors);
8849        let hash = accounts.update_accounts_hash_test(current_slot, &ancestors);
8850
8851        accounts.clean_accounts(None, false, None);
8852
8853        assert_eq!(
8854            accounts.update_accounts_hash_test(current_slot, &ancestors),
8855            hash
8856        );
8857
8858        accounts.print_accounts_stats("post_purge");
8859
8860        // Make sure the index is for pubkey cleared
8861        assert!(accounts
8862            .accounts_index
8863            .get_account_read_entry(&pubkey)
8864            .is_none());
8865
8866        // slot 1 & 2 should not have any stores
8867        assert_no_stores(&accounts, 1);
8868        assert_no_stores(&accounts, 2);
8869    }
8870
8871    #[test]
8872    fn test_accounts_db_serialize_zero_and_free() {
8873        gemachain_logger::setup();
8874
8875        let some_carat = 223;
8876        let zero_carat = 0;
8877        let no_data = 0;
8878        let owner = *AccountSharedData::default().owner();
8879
8880        let account = AccountSharedData::new(some_carat, no_data, &owner);
8881        let pubkey = gemachain_sdk::pubkey::new_rand();
8882        let zero_carat_account = AccountSharedData::new(zero_carat, no_data, &owner);
8883
8884        let account2 = AccountSharedData::new(some_carat + 1, no_data, &owner);
8885        let pubkey2 = gemachain_sdk::pubkey::new_rand();
8886
8887        let filler_account = AccountSharedData::new(some_carat, no_data, &owner);
8888        let filler_account_pubkey = gemachain_sdk::pubkey::new_rand();
8889
8890        let accounts = AccountsDb::new_single_for_tests();
8891
8892        let mut current_slot = 1;
8893        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
8894        accounts.add_root(current_slot);
8895
8896        current_slot += 1;
8897        accounts.store_uncached(current_slot, &[(&pubkey, &zero_carat_account)]);
8898        accounts.store_uncached(current_slot, &[(&pubkey2, &account2)]);
8899
8900        // Store enough accounts such that an additional store for slot 2 is created.
8901        while accounts
8902            .storage
8903            .get_slot_stores(current_slot)
8904            .unwrap()
8905            .read()
8906            .unwrap()
8907            .len()
8908            < 2
8909        {
8910            accounts.store_uncached(current_slot, &[(&filler_account_pubkey, &filler_account)]);
8911        }
8912        accounts.add_root(current_slot);
8913
8914        assert_load_account(&accounts, current_slot, pubkey, zero_carat);
8915
8916        accounts.print_accounts_stats("accounts");
8917
8918        accounts.clean_accounts(None, false, None);
8919
8920        accounts.print_accounts_stats("accounts_post_purge");
8921        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
8922
8923        accounts.print_accounts_stats("reconstructed");
8924
8925        assert_load_account(&accounts, current_slot, pubkey, zero_carat);
8926    }
8927
8928    fn with_chained_zero_carat_accounts<F>(f: F)
8929    where
8930        F: Fn(AccountsDb, Slot) -> AccountsDb,
8931    {
8932        let some_carat = 223;
8933        let zero_carat = 0;
8934        let dummy_carat = 999;
8935        let no_data = 0;
8936        let owner = *AccountSharedData::default().owner();
8937
8938        let account = AccountSharedData::new(some_carat, no_data, &owner);
8939        let account2 = AccountSharedData::new(some_carat + 100_001, no_data, &owner);
8940        let account3 = AccountSharedData::new(some_carat + 100_002, no_data, &owner);
8941        let zero_carat_account = AccountSharedData::new(zero_carat, no_data, &owner);
8942
8943        let pubkey = gemachain_sdk::pubkey::new_rand();
8944        let purged_pubkey1 = gemachain_sdk::pubkey::new_rand();
8945        let purged_pubkey2 = gemachain_sdk::pubkey::new_rand();
8946
8947        let dummy_account = AccountSharedData::new(dummy_carat, no_data, &owner);
8948        let dummy_pubkey = Pubkey::default();
8949
8950        let accounts = AccountsDb::new_single_for_tests();
8951
8952        let mut current_slot = 1;
8953        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
8954        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
8955        accounts.add_root(current_slot);
8956
8957        current_slot += 1;
8958        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &zero_carat_account)]);
8959        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &account3)]);
8960        accounts.add_root(current_slot);
8961
8962        current_slot += 1;
8963        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &zero_carat_account)]);
8964        accounts.add_root(current_slot);
8965
8966        current_slot += 1;
8967        accounts.store_uncached(current_slot, &[(&dummy_pubkey, &dummy_account)]);
8968        accounts.add_root(current_slot);
8969
8970        accounts.print_accounts_stats("pre_f");
8971        accounts.update_accounts_hash(4, &Ancestors::default());
8972
8973        let accounts = f(accounts, current_slot);
8974
8975        accounts.print_accounts_stats("post_f");
8976
8977        assert_load_account(&accounts, current_slot, pubkey, some_carat);
8978        assert_load_account(&accounts, current_slot, purged_pubkey1, 0);
8979        assert_load_account(&accounts, current_slot, purged_pubkey2, 0);
8980        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_carat);
8981
8982        accounts
8983            .verify_bank_hash_and_carats(4, &Ancestors::default(), 1222, true)
8984            .unwrap();
8985    }
8986
8987    #[test]
8988    fn test_accounts_purge_chained_purge_before_snapshot_restore() {
8989        gemachain_logger::setup();
8990        with_chained_zero_carat_accounts(|accounts, current_slot| {
8991            accounts.clean_accounts(None, false, None);
8992            reconstruct_accounts_db_via_serialization(&accounts, current_slot)
8993        });
8994    }
8995
8996    #[test]
8997    fn test_accounts_purge_chained_purge_after_snapshot_restore() {
8998        gemachain_logger::setup();
8999        with_chained_zero_carat_accounts(|accounts, current_slot| {
9000            let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
9001            accounts.print_accounts_stats("after_reconstruct");
9002            accounts.clean_accounts(None, false, None);
9003            reconstruct_accounts_db_via_serialization(&accounts, current_slot)
9004        });
9005    }
9006
9007    #[test]
9008    #[ignore]
9009    fn test_store_account_stress() {
9010        let slot = 42;
9011        let num_threads = 2;
9012
9013        let min_file_bytes = std::mem::size_of::<StoredMeta>()
9014            + std::mem::size_of::<crate::append_vec::AccountMeta>();
9015
9016        let db = Arc::new(AccountsDb::new_sized(Vec::new(), min_file_bytes as u64));
9017
9018        db.add_root(slot);
9019        let thread_hdls: Vec<_> = (0..num_threads)
9020            .map(|_| {
9021                let db = db.clone();
9022                std::thread::Builder::new()
9023                    .name("account-writers".to_string())
9024                    .spawn(move || {
9025                        let pubkey = gemachain_sdk::pubkey::new_rand();
9026                        let mut account = AccountSharedData::new(1, 0, &pubkey);
9027                        let mut i = 0;
9028                        loop {
9029                            let account_bal = thread_rng().gen_range(1, 99);
9030                            account.set_carats(account_bal);
9031                            db.store_uncached(slot, &[(&pubkey, &account)]);
9032
9033                            let (account, slot) = db
9034                                .load_without_fixed_root(&Ancestors::default(), &pubkey)
9035                                .unwrap_or_else(|| {
9036                                    panic!("Could not fetch stored account {}, iter {}", pubkey, i)
9037                                });
9038                            assert_eq!(slot, slot);
9039                            assert_eq!(account.carats(), account_bal);
9040                            i += 1;
9041                        }
9042                    })
9043                    .unwrap()
9044            })
9045            .collect();
9046
9047        for t in thread_hdls {
9048            t.join().unwrap();
9049        }
9050    }
9051
9052    #[test]
9053    fn test_accountsdb_scan_accounts() {
9054        gemachain_logger::setup();
9055        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9056        let key = Pubkey::default();
9057        let key0 = gemachain_sdk::pubkey::new_rand();
9058        let account0 = AccountSharedData::new(1, 0, &key);
9059
9060        db.store_uncached(0, &[(&key0, &account0)]);
9061
9062        let key1 = gemachain_sdk::pubkey::new_rand();
9063        let account1 = AccountSharedData::new(2, 0, &key);
9064        db.store_uncached(1, &[(&key1, &account1)]);
9065
9066        let ancestors = vec![(0, 0)].into_iter().collect();
9067        let accounts: Vec<AccountSharedData> = db.unchecked_scan_accounts(
9068            "",
9069            &ancestors,
9070            |accounts: &mut Vec<AccountSharedData>, option| {
9071                accounts.push(option.1.take_account());
9072            },
9073            COLLECT_ALL_UNSORTED_FALSE,
9074        );
9075        assert_eq!(accounts, vec![account0]);
9076
9077        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
9078        let accounts: Vec<AccountSharedData> = db.unchecked_scan_accounts(
9079            "",
9080            &ancestors,
9081            |accounts: &mut Vec<AccountSharedData>, option| {
9082                accounts.push(option.1.take_account());
9083            },
9084            COLLECT_ALL_UNSORTED_FALSE,
9085        );
9086        assert_eq!(accounts.len(), 2);
9087    }
9088
9089    #[test]
9090    fn test_cleanup_key_not_removed() {
9091        gemachain_logger::setup();
9092        let db = AccountsDb::new_single_for_tests();
9093
9094        let key = Pubkey::default();
9095        let key0 = gemachain_sdk::pubkey::new_rand();
9096        let account0 = AccountSharedData::new(1, 0, &key);
9097
9098        db.store_uncached(0, &[(&key0, &account0)]);
9099
9100        let key1 = gemachain_sdk::pubkey::new_rand();
9101        let account1 = AccountSharedData::new(2, 0, &key);
9102        db.store_uncached(1, &[(&key1, &account1)]);
9103
9104        db.print_accounts_stats("pre");
9105
9106        let slots: HashSet<Slot> = vec![1].into_iter().collect();
9107        let purge_keys = vec![(key1, slots)];
9108        db.purge_keys_exact(purge_keys.iter());
9109
9110        let account2 = AccountSharedData::new(3, 0, &key);
9111        db.store_uncached(2, &[(&key1, &account2)]);
9112
9113        db.print_accounts_stats("post");
9114        let ancestors = vec![(2, 0)].into_iter().collect();
9115        assert_eq!(
9116            db.load_without_fixed_root(&ancestors, &key1)
9117                .unwrap()
9118                .0
9119                .carats(),
9120            3
9121        );
9122    }
9123
9124    #[test]
9125    fn test_store_large_account() {
9126        gemachain_logger::setup();
9127        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9128
9129        let key = Pubkey::default();
9130        let data_len = DEFAULT_FILE_SIZE as usize + 7;
9131        let account = AccountSharedData::new(1, data_len, &key);
9132
9133        db.store_uncached(0, &[(&key, &account)]);
9134
9135        let ancestors = vec![(0, 0)].into_iter().collect();
9136        let ret = db.load_without_fixed_root(&ancestors, &key).unwrap();
9137        assert_eq!(ret.0.data().len(), data_len);
9138    }
9139
9140    #[test]
9141    fn test_hash_frozen_account_data() {
9142        let account = AccountSharedData::new(1, 42, &Pubkey::default());
9143
9144        let hash = AccountsDb::hash_frozen_account_data(&account);
9145        assert_ne!(hash, Hash::default()); // Better not be the default Hash
9146
9147        // Carats changes to not affect the hash
9148        let mut account_modified = account.clone();
9149        account_modified.checked_sub_carats(1).unwrap();
9150        assert_eq!(
9151            hash,
9152            AccountsDb::hash_frozen_account_data(&account_modified)
9153        );
9154
9155        // Rent epoch may changes to not affect the hash
9156        let mut account_modified = account.clone();
9157        account_modified.set_rent_epoch(account_modified.rent_epoch() + 1);
9158        assert_eq!(
9159            hash,
9160            AccountsDb::hash_frozen_account_data(&account_modified)
9161        );
9162
9163        // Account data may not be modified
9164        let mut account_modified = account.clone();
9165        account_modified.data_as_mut_slice()[0] = 42;
9166        assert_ne!(
9167            hash,
9168            AccountsDb::hash_frozen_account_data(&account_modified)
9169        );
9170
9171        // Owner may not be modified
9172        let mut account_modified = account.clone();
9173        account_modified
9174            .set_owner(Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap());
9175        assert_ne!(
9176            hash,
9177            AccountsDb::hash_frozen_account_data(&account_modified)
9178        );
9179
9180        // Executable may not be modified
9181        let mut account_modified = account;
9182        account_modified.set_executable(true);
9183        assert_ne!(
9184            hash,
9185            AccountsDb::hash_frozen_account_data(&account_modified)
9186        );
9187    }
9188
9189    #[test]
9190    fn test_frozen_account_carat_increase() {
9191        let frozen_pubkey =
9192            Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
9193        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9194
9195        let mut account = AccountSharedData::new(1, 42, &frozen_pubkey);
9196        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9197
9198        let ancestors = vec![(0, 0)].into_iter().collect();
9199        db.freeze_accounts(&ancestors, &[frozen_pubkey]);
9200
9201        // Store with no account changes is ok
9202        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9203
9204        // Store with an increase in carats is ok
9205        account.set_carats(2);
9206        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9207
9208        // Store with an decrease that does not go below the frozen amount of carats is tolerated
9209        account.set_carats(1);
9210        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9211
9212        // A store of any value over the frozen value of '1' across different slots is also ok
9213        account.set_carats(3);
9214        db.store_uncached(1, &[(&frozen_pubkey, &account)]);
9215        account.set_carats(2);
9216        db.store_uncached(2, &[(&frozen_pubkey, &account)]);
9217        account.set_carats(1);
9218        db.store_uncached(3, &[(&frozen_pubkey, &account)]);
9219    }
9220
9221    #[test]
9222    #[should_panic(
9223        expected = "Frozen account My11111111111111111111111111111111111111111 modified.  Carats decreased from 1 to 0"
9224    )]
9225    fn test_frozen_account_carat_decrease() {
9226        let frozen_pubkey =
9227            Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
9228        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9229
9230        let mut account = AccountSharedData::new(1, 42, &frozen_pubkey);
9231        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9232
9233        let ancestors = vec![(0, 0)].into_iter().collect();
9234        db.freeze_accounts(&ancestors, &[frozen_pubkey]);
9235
9236        // Store with a decrease below the frozen amount of carats is not ok
9237        account.checked_sub_carats(1).unwrap();
9238        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9239    }
9240
9241    #[test]
9242    #[should_panic(
9243        expected = "Unable to freeze an account that does not exist: My11111111111111111111111111111111111111111"
9244    )]
9245    fn test_frozen_account_nonexistent() {
9246        let frozen_pubkey =
9247            Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
9248        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9249
9250        let ancestors = vec![(0, 0)].into_iter().collect();
9251        db.freeze_accounts(&ancestors, &[frozen_pubkey]);
9252    }
9253
9254    #[test]
9255    #[should_panic(
9256        expected = "Frozen account My11111111111111111111111111111111111111111 modified.  Hash changed from 8wHcxDkjiwdrkPAsDnmNrF1UDGJFAtZzPQBSVweY3yRA to JdscGYB1uczVssmYuJusDD1Bfe6wpNeeho8XjcH8inN"
9257    )]
9258    fn test_frozen_account_data_modified() {
9259        let frozen_pubkey =
9260            Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
9261        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9262
9263        let mut account = AccountSharedData::new(1, 42, &frozen_pubkey);
9264        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9265
9266        let ancestors = vec![(0, 0)].into_iter().collect();
9267        db.freeze_accounts(&ancestors, &[frozen_pubkey]);
9268
9269        account.data_as_mut_slice()[0] = 42;
9270        db.store_uncached(0, &[(&frozen_pubkey, &account)]);
9271    }
9272
9273    #[test]
9274    fn test_stored_readable_account() {
9275        let carats = 1;
9276        let owner = Pubkey::new_unique();
9277        let executable = true;
9278        let rent_epoch = 2;
9279        let meta = StoredMeta {
9280            write_version: 5,
9281            pubkey: Pubkey::new_unique(),
9282            data_len: 7,
9283        };
9284        let account_meta = AccountMeta {
9285            carats,
9286            owner,
9287            executable,
9288            rent_epoch,
9289        };
9290        let data = Vec::new();
9291        let account = Account {
9292            carats,
9293            owner,
9294            executable,
9295            rent_epoch,
9296            data: data.clone(),
9297        };
9298        let offset = 99;
9299        let stored_size = 101;
9300        let hash = Hash::new_unique();
9301        let stored_account = StoredAccountMeta {
9302            meta: &meta,
9303            account_meta: &account_meta,
9304            data: &data,
9305            offset,
9306            stored_size,
9307            hash: &hash,
9308        };
9309        assert!(accounts_equal(&account, &stored_account));
9310    }
9311
9312    #[test]
9313    fn test_hash_stored_account() {
9314        // This test uses some UNSAFE trick to detect most of account's field
9315        // addition and deletion without changing the hash code
9316
9317        const ACCOUNT_DATA_LEN: usize = 3;
9318        // the type of InputTuple elements must not contain references;
9319        // they should be simple scalars or data blobs
9320        type InputTuple = (
9321            Slot,
9322            StoredMeta,
9323            AccountMeta,
9324            [u8; ACCOUNT_DATA_LEN],
9325            usize, // for StoredAccountMeta::offset
9326            Hash,
9327        );
9328        const INPUT_LEN: usize = std::mem::size_of::<InputTuple>();
9329        type InputBlob = [u8; INPUT_LEN];
9330        let mut blob: InputBlob = [0u8; INPUT_LEN];
9331
9332        // spray memory with decreasing counts so that, data layout can be detected.
9333        for (i, byte) in blob.iter_mut().enumerate() {
9334            *byte = (INPUT_LEN - i) as u8;
9335        }
9336
9337        //UNSAFE: forcibly cast the special byte pattern to actual account fields.
9338        let (slot, meta, account_meta, data, offset, hash): InputTuple =
9339            unsafe { std::mem::transmute::<InputBlob, InputTuple>(blob) };
9340
9341        let stored_account = StoredAccountMeta {
9342            meta: &meta,
9343            account_meta: &account_meta,
9344            data: &data,
9345            offset,
9346            stored_size: CACHE_VIRTUAL_STORED_SIZE,
9347            hash: &hash,
9348        };
9349        let account = stored_account.clone_account();
9350
9351        let expected_account_hash = if cfg!(debug_assertions) {
9352            Hash::from_str("4StuvYHFd7xuShVXB94uHHvpqGMCaacdZnYB74QQkPA1").unwrap()
9353        } else {
9354            Hash::from_str("33ruy7m3Xto7irYfsBSN74aAzQwCQxsfoZxXuZy2Rra3").unwrap()
9355        };
9356
9357        assert_eq!(
9358            AccountsDb::hash_stored_account(slot, &stored_account),
9359            expected_account_hash,
9360            "StoredAccountMeta's data layout might be changed; update hashing if needed."
9361        );
9362        assert_eq!(
9363            AccountsDb::hash_account(slot, &account, &stored_account.meta.pubkey),
9364            expected_account_hash,
9365            "Account-based hashing must be consistent with StoredAccountMeta-based one."
9366        );
9367    }
9368
9369    #[test]
9370    fn test_bank_hash_stats() {
9371        gemachain_logger::setup();
9372        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9373
9374        let key = Pubkey::default();
9375        let some_data_len = 5;
9376        let some_slot: Slot = 0;
9377        let account = AccountSharedData::new(1, some_data_len, &key);
9378        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9379
9380        db.store_uncached(some_slot, &[(&key, &account)]);
9381        let mut account = db.load_without_fixed_root(&ancestors, &key).unwrap().0;
9382        account.checked_sub_carats(1).unwrap();
9383        account.set_executable(true);
9384        db.store_uncached(some_slot, &[(&key, &account)]);
9385        db.add_root(some_slot);
9386
9387        let bank_hashes = db.bank_hashes.read().unwrap();
9388        let bank_hash = bank_hashes.get(&some_slot).unwrap();
9389        assert_eq!(bank_hash.stats.num_updated_accounts, 1);
9390        assert_eq!(bank_hash.stats.num_removed_accounts, 1);
9391        assert_eq!(bank_hash.stats.num_carats_stored, 1);
9392        assert_eq!(bank_hash.stats.total_data_len, 2 * some_data_len as u64);
9393        assert_eq!(bank_hash.stats.num_executable_accounts, 1);
9394    }
9395
9396    #[test]
9397    fn test_calculate_accounts_hash_check_hash_mismatch() {
9398        gemachain_logger::setup();
9399        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9400
9401        let key = gemachain_sdk::pubkey::new_rand();
9402        let some_data_len = 0;
9403        let some_slot: Slot = 0;
9404        let account = AccountSharedData::new(1, some_data_len, &key);
9405
9406        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9407
9408        // put wrong hash value in store so we get a mismatch
9409        db.store_accounts_unfrozen(
9410            some_slot,
9411            &[(&key, &account)],
9412            Some(&[&Hash::default()]),
9413            false,
9414        );
9415        db.add_root(some_slot);
9416        let check_hash = true;
9417        for use_index in [true, false] {
9418            assert!(db
9419                .calculate_accounts_hash_helper(
9420                    use_index, some_slot, &ancestors, check_hash, false, None
9421                )
9422                .is_err());
9423        }
9424    }
9425
9426    #[test]
9427    fn test_calculate_accounts_hash_check_hash() {
9428        gemachain_logger::setup();
9429        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9430
9431        let key = gemachain_sdk::pubkey::new_rand();
9432        let some_data_len = 0;
9433        let some_slot: Slot = 0;
9434        let account = AccountSharedData::new(1, some_data_len, &key);
9435
9436        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9437
9438        db.store_uncached(some_slot, &[(&key, &account)]);
9439        db.add_root(some_slot);
9440        let check_hash = true;
9441        assert_eq!(
9442            db.calculate_accounts_hash_helper(
9443                false, some_slot, &ancestors, check_hash, false, None
9444            )
9445            .unwrap(),
9446            db.calculate_accounts_hash_helper(true, some_slot, &ancestors, check_hash, false, None)
9447                .unwrap(),
9448        );
9449    }
9450
9451    #[test]
9452    fn test_verify_bank_hash() {
9453        use BankHashVerificationError::*;
9454        gemachain_logger::setup();
9455        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9456
9457        let key = gemachain_sdk::pubkey::new_rand();
9458        let some_data_len = 0;
9459        let some_slot: Slot = 0;
9460        let account = AccountSharedData::new(1, some_data_len, &key);
9461        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9462
9463        db.store_uncached(some_slot, &[(&key, &account)]);
9464        db.add_root(some_slot);
9465        db.update_accounts_hash_test(some_slot, &ancestors);
9466        assert_matches!(
9467            db.verify_bank_hash_and_carats(some_slot, &ancestors, 1, true),
9468            Ok(_)
9469        );
9470
9471        db.bank_hashes.write().unwrap().remove(&some_slot).unwrap();
9472        assert_matches!(
9473            db.verify_bank_hash_and_carats(some_slot, &ancestors, 1, true),
9474            Err(MissingBankHash)
9475        );
9476
9477        let some_bank_hash = Hash::new(&[0xca; HASH_BYTES]);
9478        let bank_hash_info = BankHashInfo {
9479            hash: some_bank_hash,
9480            snapshot_hash: Hash::new(&[0xca; HASH_BYTES]),
9481            stats: BankHashStats::default(),
9482        };
9483        db.bank_hashes
9484            .write()
9485            .unwrap()
9486            .insert(some_slot, bank_hash_info);
9487        assert_matches!(
9488            db.verify_bank_hash_and_carats(some_slot, &ancestors, 1, true),
9489            Err(MismatchedBankHash)
9490        );
9491    }
9492
9493    #[test]
9494    fn test_verify_bank_capitalization() {
9495        use BankHashVerificationError::*;
9496        gemachain_logger::setup();
9497        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9498
9499        let key = gemachain_sdk::pubkey::new_rand();
9500        let some_data_len = 0;
9501        let some_slot: Slot = 0;
9502        let account = AccountSharedData::new(1, some_data_len, &key);
9503        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9504
9505        db.store_uncached(some_slot, &[(&key, &account)]);
9506        db.add_root(some_slot);
9507        db.update_accounts_hash_test(some_slot, &ancestors);
9508        assert_matches!(
9509            db.verify_bank_hash_and_carats(some_slot, &ancestors, 1, true),
9510            Ok(_)
9511        );
9512
9513        let native_account_pubkey = gemachain_sdk::pubkey::new_rand();
9514        db.store_uncached(
9515            some_slot,
9516            &[(
9517                &native_account_pubkey,
9518                &gemachain_sdk::native_loader::create_loadable_account_for_test("foo"),
9519            )],
9520        );
9521        db.update_accounts_hash_test(some_slot, &ancestors);
9522        assert_matches!(
9523            db.verify_bank_hash_and_carats(some_slot, &ancestors, 2, true),
9524            Ok(_)
9525        );
9526
9527        assert_matches!(
9528            db.verify_bank_hash_and_carats(some_slot, &ancestors, 10, true),
9529            Err(MismatchedTotalCarats(expected, actual)) if expected == 2 && actual == 10
9530        );
9531    }
9532
9533    #[test]
9534    fn test_verify_bank_hash_no_account() {
9535        gemachain_logger::setup();
9536        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9537
9538        let some_slot: Slot = 0;
9539        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9540
9541        db.bank_hashes
9542            .write()
9543            .unwrap()
9544            .insert(some_slot, BankHashInfo::default());
9545        db.add_root(some_slot);
9546        db.update_accounts_hash_test(some_slot, &ancestors);
9547        assert_matches!(
9548            db.verify_bank_hash_and_carats(some_slot, &ancestors, 0, true),
9549            Ok(_)
9550        );
9551    }
9552
9553    #[test]
9554    fn test_verify_bank_hash_bad_account_hash() {
9555        use BankHashVerificationError::*;
9556        gemachain_logger::setup();
9557        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9558
9559        let key = Pubkey::default();
9560        let some_data_len = 0;
9561        let some_slot: Slot = 0;
9562        let account = AccountSharedData::new(1, some_data_len, &key);
9563        let ancestors = vec![(some_slot, 0)].into_iter().collect();
9564
9565        let accounts = &[(&key, &account)];
9566        // update AccountsDb's bank hash
9567        {
9568            let mut bank_hashes = db.bank_hashes.write().unwrap();
9569            bank_hashes
9570                .entry(some_slot)
9571                .or_insert_with(BankHashInfo::default);
9572        }
9573        // provide bogus account hashes
9574        let some_hash = Hash::new(&[0xca; HASH_BYTES]);
9575        db.store_accounts_unfrozen(some_slot, accounts, Some(&[&some_hash]), false);
9576        db.add_root(some_slot);
9577        assert_matches!(
9578            db.verify_bank_hash_and_carats(some_slot, &ancestors, 1, true),
9579            Err(MismatchedAccountHash)
9580        );
9581    }
9582
9583    #[test]
9584    fn test_storage_finder() {
9585        gemachain_logger::setup();
9586        let db = AccountsDb::new_sized(Vec::new(), 16 * 1024);
9587        let key = gemachain_sdk::pubkey::new_rand();
9588        let carats = 100;
9589        let data_len = 8190;
9590        let account = AccountSharedData::new(carats, data_len, &gemachain_sdk::pubkey::new_rand());
9591        // pre-populate with a smaller empty store
9592        db.create_and_insert_store(1, 8192, "test_storage_finder");
9593        db.store_uncached(1, &[(&key, &account)]);
9594    }
9595
9596    #[test]
9597    fn test_get_snapshot_storages_empty() {
9598        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9599        assert!(db.get_snapshot_storages(0, None, None).0.is_empty());
9600    }
9601
9602    #[test]
9603    fn test_get_snapshot_storages_only_older_than_or_equal_to_snapshot_slot() {
9604        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9605
9606        let key = Pubkey::default();
9607        let account = AccountSharedData::new(1, 0, &key);
9608        let before_slot = 0;
9609        let base_slot = before_slot + 1;
9610        let after_slot = base_slot + 1;
9611
9612        db.add_root(base_slot);
9613        db.store_uncached(base_slot, &[(&key, &account)]);
9614        assert!(db
9615            .get_snapshot_storages(before_slot, None, None)
9616            .0
9617            .is_empty());
9618
9619        assert_eq!(1, db.get_snapshot_storages(base_slot, None, None).0.len());
9620        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
9621    }
9622
9623    #[test]
9624    fn test_get_snapshot_storages_only_non_empty() {
9625        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9626
9627        let key = Pubkey::default();
9628        let account = AccountSharedData::new(1, 0, &key);
9629        let base_slot = 0;
9630        let after_slot = base_slot + 1;
9631
9632        db.store_uncached(base_slot, &[(&key, &account)]);
9633        db.storage
9634            .get_slot_stores(base_slot)
9635            .unwrap()
9636            .write()
9637            .unwrap()
9638            .clear();
9639        db.add_root(base_slot);
9640        assert!(db
9641            .get_snapshot_storages(after_slot, None, None)
9642            .0
9643            .is_empty());
9644
9645        db.store_uncached(base_slot, &[(&key, &account)]);
9646        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
9647    }
9648
9649    #[test]
9650    fn test_get_snapshot_storages_only_roots() {
9651        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9652
9653        let key = Pubkey::default();
9654        let account = AccountSharedData::new(1, 0, &key);
9655        let base_slot = 0;
9656        let after_slot = base_slot + 1;
9657
9658        db.store_uncached(base_slot, &[(&key, &account)]);
9659        assert!(db
9660            .get_snapshot_storages(after_slot, None, None)
9661            .0
9662            .is_empty());
9663
9664        db.add_root(base_slot);
9665        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
9666    }
9667
9668    #[test]
9669    fn test_get_snapshot_storages_exclude_empty() {
9670        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9671
9672        let key = Pubkey::default();
9673        let account = AccountSharedData::new(1, 0, &key);
9674        let base_slot = 0;
9675        let after_slot = base_slot + 1;
9676
9677        db.store_uncached(base_slot, &[(&key, &account)]);
9678        db.add_root(base_slot);
9679        assert_eq!(1, db.get_snapshot_storages(after_slot, None, None).0.len());
9680
9681        db.storage
9682            .get_slot_stores(0)
9683            .unwrap()
9684            .read()
9685            .unwrap()
9686            .values()
9687            .next()
9688            .unwrap()
9689            .remove_account(0, true);
9690        assert!(db
9691            .get_snapshot_storages(after_slot, None, None)
9692            .0
9693            .is_empty());
9694    }
9695
9696    #[test]
9697    fn test_get_snapshot_storages_with_base_slot() {
9698        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
9699
9700        let key = Pubkey::default();
9701        let account = AccountSharedData::new(1, 0, &key);
9702
9703        let slot = 10;
9704        db.store_uncached(slot, &[(&key, &account)]);
9705        db.add_root(slot);
9706        assert_eq!(
9707            0,
9708            db.get_snapshot_storages(slot + 1, Some(slot), None).0.len()
9709        );
9710        assert_eq!(
9711            1,
9712            db.get_snapshot_storages(slot + 1, Some(slot - 1), None)
9713                .0
9714                .len()
9715        );
9716    }
9717
9718    #[test]
9719    #[should_panic(expected = "double remove of account in slot: 0/store: 0!!")]
9720    fn test_storage_remove_account_double_remove() {
9721        let accounts = AccountsDb::new(Vec::new(), &ClusterType::Development);
9722        let pubkey = gemachain_sdk::pubkey::new_rand();
9723        let account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
9724        accounts.store_uncached(0, &[(&pubkey, &account)]);
9725        let storage_entry = accounts
9726            .storage
9727            .get_slot_stores(0)
9728            .unwrap()
9729            .read()
9730            .unwrap()
9731            .values()
9732            .next()
9733            .unwrap()
9734            .clone();
9735        storage_entry.remove_account(0, true);
9736        storage_entry.remove_account(0, true);
9737    }
9738
9739    #[test]
9740    fn test_accounts_purge_long_chained_after_snapshot_restore() {
9741        gemachain_logger::setup();
9742        let old_carat = 223;
9743        let zero_carat = 0;
9744        let no_data = 0;
9745        let owner = *AccountSharedData::default().owner();
9746
9747        let account = AccountSharedData::new(old_carat, no_data, &owner);
9748        let account2 = AccountSharedData::new(old_carat + 100_001, no_data, &owner);
9749        let account3 = AccountSharedData::new(old_carat + 100_002, no_data, &owner);
9750        let dummy_account = AccountSharedData::new(99_999_999, no_data, &owner);
9751        let zero_carat_account = AccountSharedData::new(zero_carat, no_data, &owner);
9752
9753        let pubkey = gemachain_sdk::pubkey::new_rand();
9754        let dummy_pubkey = gemachain_sdk::pubkey::new_rand();
9755        let purged_pubkey1 = gemachain_sdk::pubkey::new_rand();
9756        let purged_pubkey2 = gemachain_sdk::pubkey::new_rand();
9757
9758        let mut current_slot = 0;
9759        let accounts = AccountsDb::new_single_for_tests();
9760
9761        // create intermediate updates to purged_pubkey1 so that
9762        // generate_index must add slots as root last at once
9763        current_slot += 1;
9764        accounts.store_uncached(current_slot, &[(&pubkey, &account)]);
9765        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
9766        accounts.add_root(current_slot);
9767
9768        current_slot += 1;
9769        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
9770        accounts.add_root(current_slot);
9771
9772        current_slot += 1;
9773        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &account2)]);
9774        accounts.add_root(current_slot);
9775
9776        current_slot += 1;
9777        accounts.store_uncached(current_slot, &[(&purged_pubkey1, &zero_carat_account)]);
9778        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &account3)]);
9779        accounts.add_root(current_slot);
9780
9781        current_slot += 1;
9782        accounts.store_uncached(current_slot, &[(&purged_pubkey2, &zero_carat_account)]);
9783        accounts.add_root(current_slot);
9784
9785        current_slot += 1;
9786        accounts.store_uncached(current_slot, &[(&dummy_pubkey, &dummy_account)]);
9787        accounts.add_root(current_slot);
9788
9789        accounts.print_count_and_status("before reconstruct");
9790        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
9791        accounts.print_count_and_status("before purge zero");
9792        accounts.clean_accounts(None, false, None);
9793        accounts.print_count_and_status("after purge zero");
9794
9795        assert_load_account(&accounts, current_slot, pubkey, old_carat);
9796        assert_load_account(&accounts, current_slot, purged_pubkey1, 0);
9797        assert_load_account(&accounts, current_slot, purged_pubkey2, 0);
9798    }
9799
9800    fn do_full_clean_refcount(store1_first: bool, store_size: u64) {
9801        let pubkey1 = Pubkey::from_str("My11111111111111111111111111111111111111111").unwrap();
9802        let pubkey2 = Pubkey::from_str("My22211111111111111111111111111111111111111").unwrap();
9803        let pubkey3 = Pubkey::from_str("My33311111111111111111111111111111111111111").unwrap();
9804
9805        let old_carat = 223;
9806        let zero_carat = 0;
9807        let dummy_carat = 999_999;
9808
9809        // size data so only 1 fits in a 4k store
9810        let data_size = 2200;
9811
9812        let owner = *AccountSharedData::default().owner();
9813
9814        let account = AccountSharedData::new(old_carat, data_size, &owner);
9815        let account2 = AccountSharedData::new(old_carat + 100_001, data_size, &owner);
9816        let account3 = AccountSharedData::new(old_carat + 100_002, data_size, &owner);
9817        let account4 = AccountSharedData::new(dummy_carat, data_size, &owner);
9818        let zero_carat_account = AccountSharedData::new(zero_carat, data_size, &owner);
9819
9820        let mut current_slot = 0;
9821        let accounts = AccountsDb::new_sized_no_extra_stores(Vec::new(), store_size);
9822
9823        // A: Initialize AccountsDb with pubkey1 and pubkey2
9824        current_slot += 1;
9825        if store1_first {
9826            accounts.store_uncached(current_slot, &[(&pubkey1, &account)]);
9827            accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
9828        } else {
9829            accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
9830            accounts.store_uncached(current_slot, &[(&pubkey1, &account)]);
9831        }
9832        accounts.get_accounts_delta_hash(current_slot);
9833        accounts.add_root(current_slot);
9834
9835        info!("post A");
9836        accounts.print_accounts_stats("Post-A");
9837
9838        // B: Test multiple updates to pubkey1 in a single slot/storage
9839        current_slot += 1;
9840        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
9841        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
9842        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
9843        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
9844        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
9845        // Stores to same pubkey, same slot only count once towards the
9846        // ref count
9847        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
9848        accounts.get_accounts_delta_hash(current_slot);
9849        accounts.add_root(current_slot);
9850
9851        accounts.print_accounts_stats("Post-B pre-clean");
9852
9853        accounts.clean_accounts(None, false, None);
9854
9855        info!("post B");
9856        accounts.print_accounts_stats("Post-B");
9857
9858        // C: more updates to trigger clean of previous updates
9859        current_slot += 1;
9860        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
9861        accounts.store_uncached(current_slot, &[(&pubkey1, &account3)]);
9862        accounts.store_uncached(current_slot, &[(&pubkey2, &account3)]);
9863        accounts.store_uncached(current_slot, &[(&pubkey3, &account4)]);
9864        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
9865        accounts.get_accounts_delta_hash(current_slot);
9866        accounts.add_root(current_slot);
9867
9868        info!("post C");
9869
9870        accounts.print_accounts_stats("Post-C");
9871
9872        // D: Make all keys 0-carat, cleans all keys
9873        current_slot += 1;
9874        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
9875        accounts.store_uncached(current_slot, &[(&pubkey1, &zero_carat_account)]);
9876        accounts.store_uncached(current_slot, &[(&pubkey2, &zero_carat_account)]);
9877        accounts.store_uncached(current_slot, &[(&pubkey3, &zero_carat_account)]);
9878
9879        let snapshot_stores = accounts.get_snapshot_storages(current_slot, None, None).0;
9880        let total_accounts: usize = snapshot_stores
9881            .iter()
9882            .flatten()
9883            .map(|s| s.all_accounts().len())
9884            .sum();
9885        assert!(!snapshot_stores.is_empty());
9886        assert!(total_accounts > 0);
9887
9888        info!("post D");
9889        accounts.print_accounts_stats("Post-D");
9890
9891        accounts.get_accounts_delta_hash(current_slot);
9892        accounts.add_root(current_slot);
9893        accounts.clean_accounts(None, false, None);
9894
9895        accounts.print_accounts_stats("Post-D clean");
9896
9897        let total_accounts_post_clean: usize = snapshot_stores
9898            .iter()
9899            .flatten()
9900            .map(|s| s.all_accounts().len())
9901            .sum();
9902        assert_eq!(total_accounts, total_accounts_post_clean);
9903
9904        // should clean all 3 pubkeys
9905        assert_eq!(accounts.ref_count_for_pubkey(&pubkey1), 0);
9906        assert_eq!(accounts.ref_count_for_pubkey(&pubkey2), 0);
9907        assert_eq!(accounts.ref_count_for_pubkey(&pubkey3), 0);
9908    }
9909
9910    #[test]
9911    fn test_full_clean_refcount() {
9912        gemachain_logger::setup();
9913
9914        // Setup 3 scenarios which try to differentiate between pubkey1 being in an
9915        // Available slot or a Full slot which would cause a different reset behavior
9916        // when pubkey1 is cleaned and therefor cause the ref count to be incorrect
9917        // preventing a removal of that key.
9918        //
9919        // do stores with a 4mb size so only 1 store is created per slot
9920        do_full_clean_refcount(false, 4 * 1024 * 1024);
9921
9922        // do stores with a 4k size and store pubkey1 first
9923        do_full_clean_refcount(false, 4096);
9924
9925        // do stores with a 4k size and store pubkey1 2nd
9926        do_full_clean_refcount(true, 4096);
9927    }
9928
9929    #[test]
9930    fn test_accounts_clean_after_snapshot_restore_then_old_revives() {
9931        gemachain_logger::setup();
9932        let old_carat = 223;
9933        let zero_carat = 0;
9934        let no_data = 0;
9935        let dummy_carat = 999_999;
9936        let owner = *AccountSharedData::default().owner();
9937
9938        let account = AccountSharedData::new(old_carat, no_data, &owner);
9939        let account2 = AccountSharedData::new(old_carat + 100_001, no_data, &owner);
9940        let account3 = AccountSharedData::new(old_carat + 100_002, no_data, &owner);
9941        let dummy_account = AccountSharedData::new(dummy_carat, no_data, &owner);
9942        let zero_carat_account = AccountSharedData::new(zero_carat, no_data, &owner);
9943
9944        let pubkey1 = gemachain_sdk::pubkey::new_rand();
9945        let pubkey2 = gemachain_sdk::pubkey::new_rand();
9946        let dummy_pubkey = gemachain_sdk::pubkey::new_rand();
9947
9948        let mut current_slot = 0;
9949        let accounts = AccountsDb::new_single_for_tests();
9950
9951        // A: Initialize AccountsDb with pubkey1 and pubkey2
9952        current_slot += 1;
9953        accounts.store_uncached(current_slot, &[(&pubkey1, &account)]);
9954        accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
9955        accounts.get_accounts_delta_hash(current_slot);
9956        accounts.add_root(current_slot);
9957
9958        // B: Test multiple updates to pubkey1 in a single slot/storage
9959        current_slot += 1;
9960        assert_eq!(0, accounts.alive_account_count_in_slot(current_slot));
9961        assert_eq!(1, accounts.ref_count_for_pubkey(&pubkey1));
9962        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
9963        accounts.store_uncached(current_slot, &[(&pubkey1, &account2)]);
9964        assert_eq!(1, accounts.alive_account_count_in_slot(current_slot));
9965        // Stores to same pubkey, same slot only count once towards the
9966        // ref count
9967        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
9968        accounts.get_accounts_delta_hash(current_slot);
9969        accounts.add_root(current_slot);
9970
9971        // C: Yet more update to trigger lazy clean of step A
9972        current_slot += 1;
9973        assert_eq!(2, accounts.ref_count_for_pubkey(&pubkey1));
9974        accounts.store_uncached(current_slot, &[(&pubkey1, &account3)]);
9975        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
9976        accounts.get_accounts_delta_hash(current_slot);
9977        accounts.add_root(current_slot);
9978
9979        // D: Make pubkey1 0-carat; also triggers clean of step B
9980        current_slot += 1;
9981        assert_eq!(3, accounts.ref_count_for_pubkey(&pubkey1));
9982        accounts.store_uncached(current_slot, &[(&pubkey1, &zero_carat_account)]);
9983        accounts.clean_accounts(None, false, None);
9984
9985        assert_eq!(
9986            // Removed one reference from the dead slot (reference only counted once
9987            // even though there were two stores to the pubkey in that slot)
9988            3, /* == 3 - 1 + 1 */
9989            accounts.ref_count_for_pubkey(&pubkey1)
9990        );
9991        accounts.get_accounts_delta_hash(current_slot);
9992        accounts.add_root(current_slot);
9993
9994        // E: Avoid missing bank hash error
9995        current_slot += 1;
9996        accounts.store_uncached(current_slot, &[(&dummy_pubkey, &dummy_account)]);
9997        accounts.get_accounts_delta_hash(current_slot);
9998        accounts.add_root(current_slot);
9999
10000        assert_load_account(&accounts, current_slot, pubkey1, zero_carat);
10001        assert_load_account(&accounts, current_slot, pubkey2, old_carat);
10002        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_carat);
10003
10004        // At this point, there is no index entries for A and B
10005        // If step C and step D should be purged, snapshot restore would cause
10006        // pubkey1 to be revived as the state of step A.
10007        // So, prevent that from happening by introducing refcount
10008        accounts.clean_accounts(None, false, None);
10009        let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
10010        accounts.clean_accounts(None, false, None);
10011
10012        info!("pubkey: {}", pubkey1);
10013        accounts.print_accounts_stats("pre_clean");
10014        assert_load_account(&accounts, current_slot, pubkey1, zero_carat);
10015        assert_load_account(&accounts, current_slot, pubkey2, old_carat);
10016        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_carat);
10017
10018        // F: Finally, make Step A cleanable
10019        current_slot += 1;
10020        accounts.store_uncached(current_slot, &[(&pubkey2, &account)]);
10021        accounts.get_accounts_delta_hash(current_slot);
10022        accounts.add_root(current_slot);
10023
10024        // Do clean
10025        accounts.clean_accounts(None, false, None);
10026
10027        // 2nd clean needed to clean-up pubkey1
10028        accounts.clean_accounts(None, false, None);
10029
10030        // Ensure pubkey2 is cleaned from the index finally
10031        assert_not_load_account(&accounts, current_slot, pubkey1);
10032        assert_load_account(&accounts, current_slot, pubkey2, old_carat);
10033        assert_load_account(&accounts, current_slot, dummy_pubkey, dummy_carat);
10034    }
10035
10036    #[test]
10037    fn test_clean_stored_dead_slots_empty() {
10038        let accounts = AccountsDb::new_single_for_tests();
10039        let mut dead_slots = HashSet::new();
10040        dead_slots.insert(10);
10041        accounts.clean_stored_dead_slots(&dead_slots, None);
10042    }
10043
10044    #[test]
10045    fn test_shrink_all_slots_none() {
10046        for startup in &[false, true] {
10047            let accounts = AccountsDb::new_single_for_tests();
10048
10049            for _ in 0..10 {
10050                accounts.shrink_candidate_slots();
10051            }
10052
10053            accounts.shrink_all_slots(*startup, None);
10054        }
10055    }
10056
10057    #[test]
10058    fn test_shrink_next_slots() {
10059        let mut accounts = AccountsDb::new_single_for_tests();
10060        accounts.caching_enabled = false;
10061
10062        let mut current_slot = 7;
10063
10064        assert_eq!(
10065            vec![None, None, None],
10066            (0..3)
10067                .map(|_| accounts.next_shrink_slot_v1())
10068                .collect::<Vec<_>>()
10069        );
10070
10071        accounts.get_accounts_delta_hash(current_slot);
10072        accounts.add_root(current_slot);
10073
10074        assert_eq!(
10075            vec![Some(7), Some(7), Some(7)],
10076            (0..3)
10077                .map(|_| accounts.next_shrink_slot_v1())
10078                .collect::<Vec<_>>()
10079        );
10080
10081        current_slot += 1;
10082        accounts.get_accounts_delta_hash(current_slot);
10083        accounts.add_root(current_slot);
10084
10085        let slots = (0..6)
10086            .map(|_| accounts.next_shrink_slot_v1())
10087            .collect::<Vec<_>>();
10088
10089        // Because the origin of this data is HashMap (not BTreeMap), key order is arbitrary per cycle.
10090        assert!(
10091            vec![Some(7), Some(8), Some(7), Some(8), Some(7), Some(8)] == slots
10092                || vec![Some(8), Some(7), Some(8), Some(7), Some(8), Some(7)] == slots
10093        );
10094    }
10095
10096    #[test]
10097    fn test_shrink_reset_uncleaned_roots() {
10098        let mut accounts = AccountsDb::new_single_for_tests();
10099        accounts.caching_enabled = false;
10100
10101        accounts.reset_uncleaned_roots_v1();
10102        assert_eq!(
10103            *accounts.shrink_candidate_slots_v1.lock().unwrap(),
10104            vec![] as Vec<Slot>
10105        );
10106
10107        accounts.get_accounts_delta_hash(0);
10108        accounts.add_root(0);
10109        accounts.get_accounts_delta_hash(1);
10110        accounts.add_root(1);
10111        accounts.get_accounts_delta_hash(2);
10112        accounts.add_root(2);
10113
10114        accounts.reset_uncleaned_roots_v1();
10115        let actual_slots = accounts.shrink_candidate_slots_v1.lock().unwrap().clone();
10116        assert_eq!(actual_slots, vec![] as Vec<Slot>);
10117
10118        accounts.reset_uncleaned_roots_v1();
10119        let mut actual_slots = accounts.shrink_candidate_slots_v1.lock().unwrap().clone();
10120        actual_slots.sort_unstable();
10121        assert_eq!(actual_slots, vec![0, 1, 2]);
10122
10123        accounts.accounts_index.clear_roots();
10124        let mut actual_slots = (0..5)
10125            .map(|_| accounts.next_shrink_slot_v1())
10126            .collect::<Vec<_>>();
10127        actual_slots.sort();
10128        assert_eq!(actual_slots, vec![None, None, Some(0), Some(1), Some(2)],);
10129    }
10130
10131    #[test]
10132    fn test_shrink_stale_slots_processed() {
10133        gemachain_logger::setup();
10134
10135        for startup in &[false, true] {
10136            let accounts = AccountsDb::new_single_for_tests();
10137
10138            let pubkey_count = 100;
10139            let pubkeys: Vec<_> = (0..pubkey_count)
10140                .map(|_| gemachain_sdk::pubkey::new_rand())
10141                .collect();
10142
10143            let some_carat = 223;
10144            let no_data = 0;
10145            let owner = *AccountSharedData::default().owner();
10146
10147            let account = AccountSharedData::new(some_carat, no_data, &owner);
10148
10149            let mut current_slot = 0;
10150
10151            current_slot += 1;
10152            for pubkey in &pubkeys {
10153                accounts.store_uncached(current_slot, &[(pubkey, &account)]);
10154            }
10155            let shrink_slot = current_slot;
10156            accounts.get_accounts_delta_hash(current_slot);
10157            accounts.add_root(current_slot);
10158
10159            current_slot += 1;
10160            let pubkey_count_after_shrink = 10;
10161            let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
10162
10163            for pubkey in updated_pubkeys {
10164                accounts.store_uncached(current_slot, &[(pubkey, &account)]);
10165            }
10166            accounts.get_accounts_delta_hash(current_slot);
10167            accounts.add_root(current_slot);
10168
10169            accounts.clean_accounts(None, false, None);
10170
10171            assert_eq!(
10172                pubkey_count,
10173                accounts.all_account_count_in_append_vec(shrink_slot)
10174            );
10175            accounts.shrink_all_slots(*startup, None);
10176            assert_eq!(
10177                pubkey_count_after_shrink,
10178                accounts.all_account_count_in_append_vec(shrink_slot)
10179            );
10180
10181            let no_ancestors = Ancestors::default();
10182            accounts.update_accounts_hash(current_slot, &no_ancestors);
10183            accounts
10184                .verify_bank_hash_and_carats(current_slot, &no_ancestors, 22300, true)
10185                .unwrap();
10186
10187            let accounts = reconstruct_accounts_db_via_serialization(&accounts, current_slot);
10188            accounts
10189                .verify_bank_hash_and_carats(current_slot, &no_ancestors, 22300, true)
10190                .unwrap();
10191
10192            // repeating should be no-op
10193            accounts.shrink_all_slots(*startup, None);
10194            assert_eq!(
10195                pubkey_count_after_shrink,
10196                accounts.all_account_count_in_append_vec(shrink_slot)
10197            );
10198        }
10199    }
10200
10201    #[test]
10202    fn test_shrink_candidate_slots() {
10203        gemachain_logger::setup();
10204
10205        let accounts = AccountsDb::new_single_for_tests();
10206
10207        let pubkey_count = 30000;
10208        let pubkeys: Vec<_> = (0..pubkey_count)
10209            .map(|_| gemachain_sdk::pubkey::new_rand())
10210            .collect();
10211
10212        let some_carat = 223;
10213        let no_data = 0;
10214        let owner = *AccountSharedData::default().owner();
10215
10216        let account = AccountSharedData::new(some_carat, no_data, &owner);
10217
10218        let mut current_slot = 0;
10219
10220        current_slot += 1;
10221        for pubkey in &pubkeys {
10222            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
10223        }
10224        let shrink_slot = current_slot;
10225        accounts.get_accounts_delta_hash(current_slot);
10226        accounts.add_root(current_slot);
10227
10228        current_slot += 1;
10229        let pubkey_count_after_shrink = 25000;
10230        let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
10231
10232        for pubkey in updated_pubkeys {
10233            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
10234        }
10235        accounts.get_accounts_delta_hash(current_slot);
10236        accounts.add_root(current_slot);
10237        accounts.clean_accounts(None, false, None);
10238
10239        assert_eq!(
10240            pubkey_count,
10241            accounts.all_account_count_in_append_vec(shrink_slot)
10242        );
10243
10244        // Only, try to shrink stale slots, nothing happens because 90/100
10245        // is not small enough to do a shrink
10246        accounts.shrink_candidate_slots();
10247        assert_eq!(
10248            pubkey_count,
10249            accounts.all_account_count_in_append_vec(shrink_slot)
10250        );
10251
10252        // Now, do full-shrink.
10253        accounts.shrink_all_slots(false, None);
10254        assert_eq!(
10255            pubkey_count_after_shrink,
10256            accounts.all_account_count_in_append_vec(shrink_slot)
10257        );
10258    }
10259
10260    #[test]
10261    fn test_select_candidates_by_total_usage_no_candidates() {
10262        // no input candidates -- none should be selected
10263        gemachain_logger::setup();
10264        let accounts = AccountsDb::new_single_for_tests();
10265        let candidates: ShrinkCandidates = HashMap::new();
10266
10267        let (selected_candidates, next_candidates) =
10268            accounts.select_candidates_by_total_usage(&candidates, DEFAULT_ACCOUNTS_SHRINK_RATIO);
10269
10270        assert_eq!(0, selected_candidates.len());
10271        assert_eq!(0, next_candidates.len());
10272    }
10273
10274    #[test]
10275    fn test_select_candidates_by_total_usage_3_way_split_condition() {
10276        // three candidates, one selected for shrink, one is put back to the candidate list and one is ignored
10277        gemachain_logger::setup();
10278        let accounts = AccountsDb::new_single_for_tests();
10279        let mut candidates: ShrinkCandidates = HashMap::new();
10280
10281        let common_store_path = Path::new("");
10282        let common_slot_id = 12;
10283        let store_file_size = 2 * PAGE_SIZE;
10284
10285        let store1_id = 22;
10286        let store1 = Arc::new(AccountStorageEntry::new(
10287            common_store_path,
10288            common_slot_id,
10289            store1_id,
10290            store_file_size,
10291        ));
10292        store1.alive_bytes.store(0, Ordering::Relaxed);
10293
10294        candidates
10295            .entry(common_slot_id)
10296            .or_default()
10297            .insert(store1.append_vec_id(), store1.clone());
10298
10299        let store2_id = 44;
10300        let store2 = Arc::new(AccountStorageEntry::new(
10301            common_store_path,
10302            common_slot_id,
10303            store2_id,
10304            store_file_size,
10305        ));
10306
10307        // The store2's alive_ratio is 0.5: as its page aligned alive size is 1 page.
10308        let store2_alive_bytes = (PAGE_SIZE - 1) as usize;
10309        store2
10310            .alive_bytes
10311            .store(store2_alive_bytes, Ordering::Relaxed);
10312        candidates
10313            .entry(common_slot_id)
10314            .or_default()
10315            .insert(store2.append_vec_id(), store2.clone());
10316
10317        let store3_id = 55;
10318        let entry3 = Arc::new(AccountStorageEntry::new(
10319            common_store_path,
10320            common_slot_id,
10321            store3_id,
10322            store_file_size,
10323        ));
10324
10325        // The store3's alive ratio is 1.0 as its page-aligned alive size is 2 pages
10326        let store3_alive_bytes = (PAGE_SIZE + 1) as usize;
10327        entry3
10328            .alive_bytes
10329            .store(store3_alive_bytes, Ordering::Relaxed);
10330
10331        candidates
10332            .entry(common_slot_id)
10333            .or_default()
10334            .insert(entry3.append_vec_id(), entry3.clone());
10335
10336        // Set the target alive ratio to 0.6 so that we can just get rid of store1, the remaining two stores
10337        // alive ratio can be > the target ratio: the actual ratio is 0.75 because of 3 alive pages / 4 total pages.
10338        // The target ratio is also set to larger than store2's alive ratio: 0.5 so that it would be added
10339        // to the candidates list for next round.
10340        let target_alive_ratio = 0.6;
10341        let (selected_candidates, next_candidates) =
10342            accounts.select_candidates_by_total_usage(&candidates, target_alive_ratio);
10343        assert_eq!(1, selected_candidates.len());
10344        assert_eq!(1, selected_candidates[&common_slot_id].len());
10345        assert!(selected_candidates[&common_slot_id].contains(&store1.append_vec_id()));
10346        assert_eq!(1, next_candidates.len());
10347        assert!(next_candidates[&common_slot_id].contains(&store2.append_vec_id()));
10348    }
10349
10350    #[test]
10351    fn test_select_candidates_by_total_usage_2_way_split_condition() {
10352        // three candidates, 2 are selected for shrink, one is ignored
10353        gemachain_logger::setup();
10354        let accounts = AccountsDb::new_single_for_tests();
10355        let mut candidates: ShrinkCandidates = HashMap::new();
10356
10357        let common_store_path = Path::new("");
10358        let common_slot_id = 12;
10359        let store_file_size = 2 * PAGE_SIZE;
10360
10361        let store1_id = 22;
10362        let store1 = Arc::new(AccountStorageEntry::new(
10363            common_store_path,
10364            common_slot_id,
10365            store1_id,
10366            store_file_size,
10367        ));
10368        store1.alive_bytes.store(0, Ordering::Relaxed);
10369
10370        candidates
10371            .entry(common_slot_id)
10372            .or_default()
10373            .insert(store1.append_vec_id(), store1.clone());
10374
10375        let store2_id = 44;
10376        let store2 = Arc::new(AccountStorageEntry::new(
10377            common_store_path,
10378            common_slot_id,
10379            store2_id,
10380            store_file_size,
10381        ));
10382
10383        // The store2's alive_ratio is 0.5: as its page aligned alive size is 1 page.
10384        let store2_alive_bytes = (PAGE_SIZE - 1) as usize;
10385        store2
10386            .alive_bytes
10387            .store(store2_alive_bytes, Ordering::Relaxed);
10388        candidates
10389            .entry(common_slot_id)
10390            .or_default()
10391            .insert(store2.append_vec_id(), store2.clone());
10392
10393        let store3_id = 55;
10394        let entry3 = Arc::new(AccountStorageEntry::new(
10395            common_store_path,
10396            common_slot_id,
10397            store3_id,
10398            store_file_size,
10399        ));
10400
10401        // The store3's alive ratio is 1.0 as its page-aligned alive size is 2 pages
10402        let store3_alive_bytes = (PAGE_SIZE + 1) as usize;
10403        entry3
10404            .alive_bytes
10405            .store(store3_alive_bytes, Ordering::Relaxed);
10406
10407        candidates
10408            .entry(common_slot_id)
10409            .or_default()
10410            .insert(entry3.append_vec_id(), entry3.clone());
10411
10412        // Set the target ratio to default (0.8), both store1 and store2 must be selected and store3 is ignored.
10413        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
10414        let (selected_candidates, next_candidates) =
10415            accounts.select_candidates_by_total_usage(&candidates, target_alive_ratio);
10416        assert_eq!(1, selected_candidates.len());
10417        assert_eq!(2, selected_candidates[&common_slot_id].len());
10418        assert!(selected_candidates[&common_slot_id].contains(&store1.append_vec_id()));
10419        assert!(selected_candidates[&common_slot_id].contains(&store2.append_vec_id()));
10420        assert_eq!(0, next_candidates.len());
10421    }
10422
10423    #[test]
10424    fn test_select_candidates_by_total_usage_all_clean() {
10425        // 2 candidates, they must be selected to achieve the target alive ratio
10426        gemachain_logger::setup();
10427        let accounts = AccountsDb::new_single_for_tests();
10428        let mut candidates: ShrinkCandidates = HashMap::new();
10429
10430        let slot1 = 12;
10431        let common_store_path = Path::new("");
10432
10433        let store_file_size = 4 * PAGE_SIZE;
10434        let store1_id = 22;
10435        let store1 = Arc::new(AccountStorageEntry::new(
10436            common_store_path,
10437            slot1,
10438            store1_id,
10439            store_file_size,
10440        ));
10441
10442        // store1 has 1 page-aligned alive bytes, its alive ratio is 1/4: 0.25
10443        let store1_alive_bytes = (PAGE_SIZE - 1) as usize;
10444        store1
10445            .alive_bytes
10446            .store(store1_alive_bytes, Ordering::Relaxed);
10447
10448        candidates
10449            .entry(slot1)
10450            .or_default()
10451            .insert(store1.append_vec_id(), store1.clone());
10452
10453        let store2_id = 44;
10454        let slot2 = 44;
10455        let store2 = Arc::new(AccountStorageEntry::new(
10456            common_store_path,
10457            slot2,
10458            store2_id,
10459            store_file_size,
10460        ));
10461
10462        // store2 has 2 page-aligned bytes, its alive ratio is 2/4: 0.5
10463        let store2_alive_bytes = (PAGE_SIZE + 1) as usize;
10464        store2
10465            .alive_bytes
10466            .store(store2_alive_bytes, Ordering::Relaxed);
10467
10468        candidates
10469            .entry(slot2)
10470            .or_default()
10471            .insert(store2.append_vec_id(), store2.clone());
10472
10473        // Set the target ratio to default (0.8), both stores from the two different slots must be selected.
10474        let target_alive_ratio = DEFAULT_ACCOUNTS_SHRINK_RATIO;
10475        let (selected_candidates, next_candidates) =
10476            accounts.select_candidates_by_total_usage(&candidates, target_alive_ratio);
10477        assert_eq!(2, selected_candidates.len());
10478        assert_eq!(1, selected_candidates[&slot1].len());
10479        assert_eq!(1, selected_candidates[&slot2].len());
10480
10481        assert!(selected_candidates[&slot1].contains(&store1.append_vec_id()));
10482        assert!(selected_candidates[&slot2].contains(&store2.append_vec_id()));
10483        assert_eq!(0, next_candidates.len());
10484    }
10485
10486    #[test]
10487    fn test_shrink_stale_slots_skipped() {
10488        gemachain_logger::setup();
10489
10490        let mut accounts = AccountsDb::new_single_for_tests();
10491        accounts.caching_enabled = false;
10492
10493        let pubkey_count = 30000;
10494        let pubkeys: Vec<_> = (0..pubkey_count)
10495            .map(|_| gemachain_sdk::pubkey::new_rand())
10496            .collect();
10497
10498        let some_carat = 223;
10499        let no_data = 0;
10500        let owner = *AccountSharedData::default().owner();
10501
10502        let account = AccountSharedData::new(some_carat, no_data, &owner);
10503
10504        let mut current_slot = 0;
10505
10506        current_slot += 1;
10507        for pubkey in &pubkeys {
10508            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
10509        }
10510        let shrink_slot = current_slot;
10511        accounts.get_accounts_delta_hash(current_slot);
10512        accounts.add_root(current_slot);
10513
10514        current_slot += 1;
10515        let pubkey_count_after_shrink = 25000;
10516        let updated_pubkeys = &pubkeys[0..pubkey_count - pubkey_count_after_shrink];
10517
10518        for pubkey in updated_pubkeys {
10519            accounts.store_uncached(current_slot, &[(pubkey, &account)]);
10520        }
10521        accounts.get_accounts_delta_hash(current_slot);
10522        accounts.add_root(current_slot);
10523
10524        accounts.clean_accounts(None, false, None);
10525
10526        assert_eq!(
10527            pubkey_count,
10528            accounts.all_account_count_in_append_vec(shrink_slot)
10529        );
10530
10531        // Only, try to shrink stale slots.
10532        accounts.shrink_all_stale_slots_v1();
10533        assert_eq!(
10534            pubkey_count,
10535            accounts.all_account_count_in_append_vec(shrink_slot)
10536        );
10537
10538        // Now, do full-shrink.
10539        accounts.shrink_all_slots(false, None);
10540        assert_eq!(
10541            pubkey_count_after_shrink,
10542            accounts.all_account_count_in_append_vec(shrink_slot)
10543        );
10544    }
10545
10546    const UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE: bool = false;
10547
10548    #[test]
10549    fn test_delete_dependencies() {
10550        gemachain_logger::setup();
10551        let accounts_index = AccountsIndex::default_for_tests();
10552        let key0 = Pubkey::new_from_array([0u8; 32]);
10553        let key1 = Pubkey::new_from_array([1u8; 32]);
10554        let key2 = Pubkey::new_from_array([2u8; 32]);
10555        let info0 = AccountInfo {
10556            store_id: 0,
10557            offset: 0,
10558            stored_size: 0,
10559            carats: 0,
10560        };
10561        let info1 = AccountInfo {
10562            store_id: 1,
10563            offset: 0,
10564            stored_size: 0,
10565            carats: 0,
10566        };
10567        let info2 = AccountInfo {
10568            store_id: 2,
10569            offset: 0,
10570            stored_size: 0,
10571            carats: 0,
10572        };
10573        let info3 = AccountInfo {
10574            store_id: 3,
10575            offset: 0,
10576            stored_size: 0,
10577            carats: 0,
10578        };
10579        let mut reclaims = vec![];
10580        accounts_index.upsert(
10581            0,
10582            &key0,
10583            &Pubkey::default(),
10584            &[],
10585            &AccountSecondaryIndexes::default(),
10586            info0,
10587            &mut reclaims,
10588            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
10589        );
10590        accounts_index.upsert(
10591            1,
10592            &key0,
10593            &Pubkey::default(),
10594            &[],
10595            &AccountSecondaryIndexes::default(),
10596            info1,
10597            &mut reclaims,
10598            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
10599        );
10600        accounts_index.upsert(
10601            1,
10602            &key1,
10603            &Pubkey::default(),
10604            &[],
10605            &AccountSecondaryIndexes::default(),
10606            info1,
10607            &mut reclaims,
10608            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
10609        );
10610        accounts_index.upsert(
10611            2,
10612            &key1,
10613            &Pubkey::default(),
10614            &[],
10615            &AccountSecondaryIndexes::default(),
10616            info2,
10617            &mut reclaims,
10618            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
10619        );
10620        accounts_index.upsert(
10621            2,
10622            &key2,
10623            &Pubkey::default(),
10624            &[],
10625            &AccountSecondaryIndexes::default(),
10626            info2,
10627            &mut reclaims,
10628            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
10629        );
10630        accounts_index.upsert(
10631            3,
10632            &key2,
10633            &Pubkey::default(),
10634            &[],
10635            &AccountSecondaryIndexes::default(),
10636            info3,
10637            &mut reclaims,
10638            UPSERT_PREVIOUS_SLOT_ENTRY_WAS_CACHED_FALSE,
10639        );
10640        accounts_index.add_root(0, false);
10641        accounts_index.add_root(1, false);
10642        accounts_index.add_root(2, false);
10643        accounts_index.add_root(3, false);
10644        let mut purges = HashMap::new();
10645        let (key0_entry, _) = accounts_index.get(&key0, None, None).unwrap();
10646        purges.insert(key0, accounts_index.roots_and_ref_count(&key0_entry, None));
10647        let (key1_entry, _) = accounts_index.get(&key1, None, None).unwrap();
10648        purges.insert(key1, accounts_index.roots_and_ref_count(&key1_entry, None));
10649        let (key2_entry, _) = accounts_index.get(&key2, None, None).unwrap();
10650        purges.insert(key2, accounts_index.roots_and_ref_count(&key2_entry, None));
10651        for (key, (list, ref_count)) in &purges {
10652            info!(" purge {} ref_count {} =>", key, ref_count);
10653            for x in list {
10654                info!("  {:?}", x);
10655            }
10656        }
10657
10658        let mut store_counts = HashMap::new();
10659        store_counts.insert(0, (0, HashSet::from_iter(vec![key0])));
10660        store_counts.insert(1, (0, HashSet::from_iter(vec![key0, key1])));
10661        store_counts.insert(2, (0, HashSet::from_iter(vec![key1, key2])));
10662        store_counts.insert(3, (1, HashSet::from_iter(vec![key2])));
10663        AccountsDb::calc_delete_dependencies(&purges, &mut store_counts);
10664        let mut stores: Vec<_> = store_counts.keys().cloned().collect();
10665        stores.sort_unstable();
10666        for store in &stores {
10667            info!(
10668                "store: {:?} : {:?}",
10669                store,
10670                store_counts.get(store).unwrap()
10671            );
10672        }
10673        for x in 0..3 {
10674            assert!(store_counts[&x].0 >= 1);
10675        }
10676    }
10677
10678    #[test]
10679    fn test_account_balance_for_capitalization_sysvar() {
10680        let normal_sysvar = gemachain_sdk::account::create_account_for_test(
10681            &gemachain_sdk::slot_history::SlotHistory::default(),
10682        );
10683        assert_eq!(normal_sysvar.carats(), 1);
10684    }
10685
10686    #[test]
10687    fn test_account_balance_for_capitalization_native_program() {
10688        let normal_native_program =
10689            gemachain_sdk::native_loader::create_loadable_account_for_test("foo");
10690        assert_eq!(normal_native_program.carats(), 1);
10691    }
10692
10693    #[test]
10694    fn test_checked_sum_for_capitalization_normal() {
10695        assert_eq!(
10696            AccountsDb::checked_sum_for_capitalization(vec![1, 2].into_iter()),
10697            3
10698        );
10699    }
10700
10701    #[test]
10702    #[should_panic(expected = "overflow is detected while summing capitalization")]
10703    fn test_checked_sum_for_capitalization_overflow() {
10704        assert_eq!(
10705            AccountsDb::checked_sum_for_capitalization(vec![1, u64::max_value()].into_iter()),
10706            3
10707        );
10708    }
10709
10710    #[test]
10711    fn test_store_overhead() {
10712        gemachain_logger::setup();
10713        let accounts = AccountsDb::new_single_for_tests();
10714        let account = AccountSharedData::default();
10715        let pubkey = gemachain_sdk::pubkey::new_rand();
10716        accounts.store_uncached(0, &[(&pubkey, &account)]);
10717        let slot_stores = accounts.storage.get_slot_stores(0).unwrap();
10718        let mut total_len = 0;
10719        for (_id, store) in slot_stores.read().unwrap().iter() {
10720            total_len += store.accounts.len();
10721        }
10722        info!("total: {}", total_len);
10723        assert!(total_len < STORE_META_OVERHEAD);
10724    }
10725
10726    #[test]
10727    fn test_store_clean_after_shrink() {
10728        gemachain_logger::setup();
10729        let accounts = AccountsDb::new_with_config_for_tests(
10730            vec![],
10731            &ClusterType::Development,
10732            AccountSecondaryIndexes::default(),
10733            true,
10734            AccountShrinkThreshold::default(),
10735        );
10736
10737        let account = AccountSharedData::new(1, 16 * 4096, &Pubkey::default());
10738        let pubkey1 = gemachain_sdk::pubkey::new_rand();
10739        accounts.store_cached(0, &[(&pubkey1, &account)]);
10740
10741        let pubkey2 = gemachain_sdk::pubkey::new_rand();
10742        accounts.store_cached(0, &[(&pubkey2, &account)]);
10743
10744        let zero_account = AccountSharedData::new(0, 1, &Pubkey::default());
10745        accounts.store_cached(1, &[(&pubkey1, &zero_account)]);
10746
10747        // Add root 0 and flush separately
10748        accounts.get_accounts_delta_hash(0);
10749        accounts.add_root(0);
10750        accounts.flush_accounts_cache(true, None);
10751
10752        // clear out the dirty keys
10753        accounts.clean_accounts(None, false, None);
10754
10755        // flush 1
10756        accounts.get_accounts_delta_hash(1);
10757        accounts.add_root(1);
10758        accounts.flush_accounts_cache(true, None);
10759
10760        accounts.print_accounts_stats("pre-clean");
10761
10762        // clean to remove pubkey1 from 0,
10763        // shrink to shrink pubkey1 from 0
10764        // then another clean to remove pubkey1 from slot 1
10765        accounts.clean_accounts(None, false, None);
10766
10767        accounts.shrink_candidate_slots();
10768
10769        accounts.clean_accounts(None, false, None);
10770
10771        accounts.print_accounts_stats("post-clean");
10772        assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
10773    }
10774
10775    #[test]
10776    fn test_store_reuse() {
10777        gemachain_logger::setup();
10778        let accounts = AccountsDb::new_sized(vec![], 4096);
10779
10780        let size = 100;
10781        let num_accounts: usize = 100;
10782        let mut keys = Vec::new();
10783        for i in 0..num_accounts {
10784            let account = AccountSharedData::new((i + 1) as u64, size, &Pubkey::default());
10785            let pubkey = gemachain_sdk::pubkey::new_rand();
10786            accounts.store_uncached(0, &[(&pubkey, &account)]);
10787            keys.push(pubkey);
10788        }
10789        accounts.add_root(0);
10790
10791        for (i, key) in keys[1..].iter().enumerate() {
10792            let account =
10793                AccountSharedData::new((1 + i + num_accounts) as u64, size, &Pubkey::default());
10794            accounts.store_uncached(1, &[(key, &account)]);
10795        }
10796        accounts.add_root(1);
10797        accounts.clean_accounts(None, false, None);
10798        accounts.shrink_all_slots(false, None);
10799
10800        // Clean again to flush the dirty stores
10801        // and allow them to be recycled in the next step
10802        accounts.clean_accounts(None, false, None);
10803        accounts.print_accounts_stats("post-shrink");
10804        let num_stores = accounts.recycle_stores.read().unwrap().entry_count();
10805        assert!(num_stores > 0);
10806
10807        let mut account_refs = Vec::new();
10808        let num_to_store = 20;
10809        for (i, key) in keys[..num_to_store].iter().enumerate() {
10810            let account = AccountSharedData::new(
10811                (1 + i + 2 * num_accounts) as u64,
10812                i + 20,
10813                &Pubkey::default(),
10814            );
10815            accounts.store_uncached(2, &[(key, &account)]);
10816            account_refs.push(account);
10817        }
10818        assert!(accounts.recycle_stores.read().unwrap().entry_count() < num_stores);
10819
10820        accounts.print_accounts_stats("post-store");
10821
10822        let mut ancestors = Ancestors::default();
10823        ancestors.insert(1, 0);
10824        ancestors.insert(2, 1);
10825        for (key, account_ref) in keys[..num_to_store].iter().zip(account_refs) {
10826            assert_eq!(
10827                accounts.load_without_fixed_root(&ancestors, key).unwrap().0,
10828                account_ref
10829            );
10830        }
10831    }
10832
10833    #[test]
10834    fn test_zero_carat_new_root_not_cleaned() {
10835        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10836        let account_key = Pubkey::new_unique();
10837        let zero_carat_account =
10838            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
10839
10840        // Store zero carat account into slots 0 and 1, root both slots
10841        db.store_uncached(0, &[(&account_key, &zero_carat_account)]);
10842        db.store_uncached(1, &[(&account_key, &zero_carat_account)]);
10843        db.get_accounts_delta_hash(0);
10844        db.add_root(0);
10845        db.get_accounts_delta_hash(1);
10846        db.add_root(1);
10847
10848        // Only clean zero carat accounts up to slot 0
10849        db.clean_accounts(Some(0), false, None);
10850
10851        // Should still be able to find zero carat account in slot 1
10852        assert_eq!(
10853            db.load_without_fixed_root(&Ancestors::default(), &account_key),
10854            Some((zero_carat_account, 1))
10855        );
10856    }
10857
10858    #[test]
10859    fn test_store_load_cached() {
10860        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10861        db.caching_enabled = true;
10862        let key = Pubkey::default();
10863        let account0 = AccountSharedData::new(1, 0, &key);
10864        let slot = 0;
10865        db.store_cached(slot, &[(&key, &account0)]);
10866
10867        // Load with no ancestors and no root will return nothing
10868        assert!(db
10869            .load_without_fixed_root(&Ancestors::default(), &key)
10870            .is_none());
10871
10872        // Load with ancestors not equal to `slot` will return nothing
10873        let ancestors = vec![(slot + 1, 1)].into_iter().collect();
10874        assert!(db.load_without_fixed_root(&ancestors, &key).is_none());
10875
10876        // Load with ancestors equal to `slot` will return the account
10877        let ancestors = vec![(slot, 1)].into_iter().collect();
10878        assert_eq!(
10879            db.load_without_fixed_root(&ancestors, &key),
10880            Some((account0.clone(), slot))
10881        );
10882
10883        // Adding root will return the account even without ancestors
10884        db.add_root(slot);
10885        assert_eq!(
10886            db.load_without_fixed_root(&Ancestors::default(), &key),
10887            Some((account0, slot))
10888        );
10889    }
10890
10891    #[test]
10892    fn test_store_flush_load_cached() {
10893        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10894        db.caching_enabled = true;
10895        let key = Pubkey::default();
10896        let account0 = AccountSharedData::new(1, 0, &key);
10897        let slot = 0;
10898        db.store_cached(slot, &[(&key, &account0)]);
10899        db.mark_slot_frozen(slot);
10900
10901        // No root was added yet, requires an ancestor to find
10902        // the account
10903        db.flush_accounts_cache(true, None);
10904        let ancestors = vec![(slot, 1)].into_iter().collect();
10905        assert_eq!(
10906            db.load_without_fixed_root(&ancestors, &key),
10907            Some((account0.clone(), slot))
10908        );
10909
10910        // Add root then flush
10911        db.add_root(slot);
10912        db.flush_accounts_cache(true, None);
10913        assert_eq!(
10914            db.load_without_fixed_root(&Ancestors::default(), &key),
10915            Some((account0, slot))
10916        );
10917    }
10918
10919    #[test]
10920    fn test_flush_accounts_cache() {
10921        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10922        db.caching_enabled = true;
10923        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
10924
10925        let unrooted_slot = 4;
10926        let root5 = 5;
10927        let root6 = 6;
10928        let unrooted_key = gemachain_sdk::pubkey::new_rand();
10929        let key5 = gemachain_sdk::pubkey::new_rand();
10930        let key6 = gemachain_sdk::pubkey::new_rand();
10931        db.store_cached(unrooted_slot, &[(&unrooted_key, &account0)]);
10932        db.store_cached(root5, &[(&key5, &account0)]);
10933        db.store_cached(root6, &[(&key6, &account0)]);
10934        for slot in &[unrooted_slot, root5, root6] {
10935            db.mark_slot_frozen(*slot);
10936        }
10937        db.add_root(root5);
10938        db.add_root(root6);
10939
10940        // Unrooted slot should be able to be fetched before the flush
10941        let ancestors = vec![(unrooted_slot, 1)].into_iter().collect();
10942        assert_eq!(
10943            db.load_without_fixed_root(&ancestors, &unrooted_key),
10944            Some((account0.clone(), unrooted_slot))
10945        );
10946        db.flush_accounts_cache(true, None);
10947
10948        // After the flush, the unrooted slot is still in the cache
10949        assert!(db
10950            .load_without_fixed_root(&ancestors, &unrooted_key)
10951            .is_some());
10952        assert!(db
10953            .accounts_index
10954            .get_account_read_entry(&unrooted_key)
10955            .is_some());
10956        assert_eq!(db.accounts_cache.num_slots(), 1);
10957        assert!(db.accounts_cache.slot_cache(unrooted_slot).is_some());
10958        assert_eq!(
10959            db.load_without_fixed_root(&Ancestors::default(), &key5),
10960            Some((account0.clone(), root5))
10961        );
10962        assert_eq!(
10963            db.load_without_fixed_root(&Ancestors::default(), &key6),
10964            Some((account0, root6))
10965        );
10966    }
10967
10968    #[test]
10969    fn test_flush_accounts_cache_if_needed() {
10970        run_test_flush_accounts_cache_if_needed(0, 2 * MAX_CACHE_SLOTS);
10971        run_test_flush_accounts_cache_if_needed(2 * MAX_CACHE_SLOTS, 0);
10972        run_test_flush_accounts_cache_if_needed(MAX_CACHE_SLOTS - 1, 0);
10973        run_test_flush_accounts_cache_if_needed(0, MAX_CACHE_SLOTS - 1);
10974        run_test_flush_accounts_cache_if_needed(MAX_CACHE_SLOTS, 0);
10975        run_test_flush_accounts_cache_if_needed(0, MAX_CACHE_SLOTS);
10976        run_test_flush_accounts_cache_if_needed(2 * MAX_CACHE_SLOTS, 2 * MAX_CACHE_SLOTS);
10977        run_test_flush_accounts_cache_if_needed(MAX_CACHE_SLOTS - 1, MAX_CACHE_SLOTS - 1);
10978        run_test_flush_accounts_cache_if_needed(MAX_CACHE_SLOTS, MAX_CACHE_SLOTS);
10979    }
10980
10981    fn run_test_flush_accounts_cache_if_needed(num_roots: usize, num_unrooted: usize) {
10982        let mut db = AccountsDb::new(Vec::new(), &ClusterType::Development);
10983        db.caching_enabled = true;
10984        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
10985        let mut keys = vec![];
10986        let num_slots = 2 * MAX_CACHE_SLOTS;
10987        for i in 0..num_roots + num_unrooted {
10988            let key = Pubkey::new_unique();
10989            db.store_cached(i as Slot, &[(&key, &account0)]);
10990            keys.push(key);
10991            db.mark_slot_frozen(i as Slot);
10992            if i < num_roots {
10993                db.add_root(i as Slot);
10994            }
10995        }
10996
10997        db.flush_accounts_cache(false, None);
10998
10999        let total_slots = num_roots + num_unrooted;
11000        // If there's <= the max size, then nothing will be flushed from the slot
11001        if total_slots <= MAX_CACHE_SLOTS {
11002            assert_eq!(db.accounts_cache.num_slots(), total_slots);
11003        } else {
11004            // Otherwise, all the roots are flushed, and only at most MAX_CACHE_SLOTS
11005            // of the unrooted slots are kept in the cache
11006            let expected_size = std::cmp::min(num_unrooted, MAX_CACHE_SLOTS);
11007            if expected_size > 0 {
11008                for unrooted_slot in total_slots - expected_size..total_slots {
11009                    assert!(db
11010                        .accounts_cache
11011                        .slot_cache(unrooted_slot as Slot)
11012                        .is_some());
11013                }
11014            }
11015        }
11016
11017        // Should still be able to fetch all the accounts after flush
11018        for (slot, key) in (0..num_slots as Slot).zip(keys) {
11019            let ancestors = if slot < num_roots as Slot {
11020                Ancestors::default()
11021            } else {
11022                vec![(slot, 1)].into_iter().collect()
11023            };
11024            assert_eq!(
11025                db.load_without_fixed_root(&ancestors, &key),
11026                Some((account0.clone(), slot))
11027            );
11028        }
11029    }
11030
11031    fn slot_stores(db: &AccountsDb, slot: Slot) -> Vec<Arc<AccountStorageEntry>> {
11032        db.storage
11033            .get_slot_storage_entries(slot)
11034            .unwrap_or_default()
11035    }
11036
11037    #[test]
11038    fn test_read_only_accounts_cache() {
11039        let caching_enabled = true;
11040        let db = Arc::new(AccountsDb::new_with_config_for_tests(
11041            Vec::new(),
11042            &ClusterType::Development,
11043            AccountSecondaryIndexes::default(),
11044            caching_enabled,
11045            AccountShrinkThreshold::default(),
11046        ));
11047
11048        let account_key = Pubkey::new_unique();
11049        let zero_carat_account =
11050            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11051        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
11052        db.store_cached(0, &[(&account_key, &zero_carat_account)]);
11053        db.store_cached(1, &[(&account_key, &slot1_account)]);
11054
11055        db.add_root(0);
11056        db.add_root(1);
11057        db.clean_accounts(None, false, None);
11058        db.flush_accounts_cache(true, None);
11059        db.clean_accounts(None, false, None);
11060        db.add_root(2);
11061
11062        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
11063        let account = db
11064            .load_with_fixed_root(&Ancestors::default(), &account_key)
11065            .map(|(account, _)| account)
11066            .unwrap();
11067        assert_eq!(account.carats(), 1);
11068        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
11069        let account = db
11070            .load_with_fixed_root(&Ancestors::default(), &account_key)
11071            .map(|(account, _)| account)
11072            .unwrap();
11073        assert_eq!(account.carats(), 1);
11074        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
11075        db.store_cached(2, &[(&account_key, &zero_carat_account)]);
11076        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
11077        let account = db
11078            .load_with_fixed_root(&Ancestors::default(), &account_key)
11079            .map(|(account, _)| account)
11080            .unwrap();
11081        assert_eq!(account.carats(), 0);
11082        assert_eq!(db.read_only_accounts_cache.cache_len(), 1);
11083    }
11084
11085    #[test]
11086    fn test_flush_cache_clean() {
11087        let caching_enabled = true;
11088        let db = Arc::new(AccountsDb::new_with_config_for_tests(
11089            Vec::new(),
11090            &ClusterType::Development,
11091            AccountSecondaryIndexes::default(),
11092            caching_enabled,
11093            AccountShrinkThreshold::default(),
11094        ));
11095
11096        let account_key = Pubkey::new_unique();
11097        let zero_carat_account =
11098            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11099        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
11100        db.store_cached(0, &[(&account_key, &zero_carat_account)]);
11101        db.store_cached(1, &[(&account_key, &slot1_account)]);
11102
11103        db.add_root(0);
11104        db.add_root(1);
11105
11106        // Clean should not remove anything yet as nothing has been flushed
11107        db.clean_accounts(None, false, None);
11108        let account = db
11109            .do_load(
11110                &Ancestors::default(),
11111                &account_key,
11112                Some(0),
11113                LoadHint::Unspecified,
11114            )
11115            .unwrap();
11116        assert_eq!(account.0.carats(), 0);
11117        // since this item is in the cache, it should not be in the read only cache
11118        assert_eq!(db.read_only_accounts_cache.cache_len(), 0);
11119
11120        // Flush, then clean again. Should not need another root to initiate the cleaning
11121        // because `accounts_index.uncleaned_roots` should be correct
11122        db.flush_accounts_cache(true, None);
11123        db.clean_accounts(None, false, None);
11124        assert!(db
11125            .do_load(
11126                &Ancestors::default(),
11127                &account_key,
11128                Some(0),
11129                LoadHint::Unspecified
11130            )
11131            .is_none());
11132    }
11133
11134    #[test]
11135    fn test_flush_cache_dont_clean_zero_carat_account() {
11136        let caching_enabled = true;
11137        let db = Arc::new(AccountsDb::new_with_config_for_tests(
11138            Vec::new(),
11139            &ClusterType::Development,
11140            AccountSecondaryIndexes::default(),
11141            caching_enabled,
11142            AccountShrinkThreshold::default(),
11143        ));
11144
11145        let zero_carat_account_key = Pubkey::new_unique();
11146        let other_account_key = Pubkey::new_unique();
11147
11148        let original_carats = 1;
11149        let slot0_account =
11150            AccountSharedData::new(original_carats, 1, AccountSharedData::default().owner());
11151        let zero_carat_account =
11152            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11153
11154        // Store into slot 0, and then flush the slot to storage
11155        db.store_cached(0, &[(&zero_carat_account_key, &slot0_account)]);
11156        // Second key keeps other carat account entry for slot 0 alive,
11157        // preventing clean of the zero_carat_account in slot 1.
11158        db.store_cached(0, &[(&other_account_key, &slot0_account)]);
11159        db.add_root(0);
11160        db.flush_accounts_cache(true, None);
11161        assert!(!db.storage.get_slot_storage_entries(0).unwrap().is_empty());
11162
11163        // Store into slot 1, a dummy slot that will be dead and purged before flush
11164        db.store_cached(1, &[(&zero_carat_account_key, &zero_carat_account)]);
11165
11166        // Store into slot 2, which makes all updates from slot 1 outdated.
11167        // This means slot 1 is a dead slot. Later, slot 1 will be cleaned/purged
11168        // before it even reaches storage, but this purge of slot 1should not affect
11169        // the refcount of `zero_carat_account_key` because cached keys do not bump
11170        // the refcount in the index. This means clean should *not* remove
11171        // `zero_carat_account_key` from slot 2
11172        db.store_cached(2, &[(&zero_carat_account_key, &zero_carat_account)]);
11173        db.add_root(1);
11174        db.add_root(2);
11175
11176        // Flush, then clean. Should not need another root to initiate the cleaning
11177        // because `accounts_index.uncleaned_roots` should be correct
11178        db.flush_accounts_cache(true, None);
11179        db.clean_accounts(None, false, None);
11180
11181        // The `zero_carat_account_key` is still alive in slot 1, so refcount for the
11182        // pubkey should be 2
11183        assert_eq!(
11184            db.accounts_index
11185                .ref_count_from_storage(&zero_carat_account_key),
11186            2
11187        );
11188        assert_eq!(
11189            db.accounts_index.ref_count_from_storage(&other_account_key),
11190            1
11191        );
11192
11193        // The zero-carat account in slot 2 should not be purged yet, because the
11194        // entry in slot 1 is blocking cleanup of the zero-carat account.
11195        let max_root = None;
11196        // Fine to simulate a transaction load since we are not doing any out of band
11197        // removals, only using clean_accounts
11198        let load_hint = LoadHint::FixedMaxRoot;
11199        assert_eq!(
11200            db.do_load(
11201                &Ancestors::default(),
11202                &zero_carat_account_key,
11203                max_root,
11204                load_hint
11205            )
11206            .unwrap()
11207            .0
11208            .carats(),
11209            0
11210        );
11211    }
11212
11213    struct ScanTracker {
11214        t_scan: JoinHandle<()>,
11215        exit: Arc<AtomicBool>,
11216    }
11217
11218    impl ScanTracker {
11219        fn exit(self) -> thread::Result<()> {
11220            self.exit.store(true, Ordering::Relaxed);
11221            self.t_scan.join()
11222        }
11223    }
11224
11225    fn setup_scan(
11226        db: Arc<AccountsDb>,
11227        scan_ancestors: Arc<Ancestors>,
11228        bank_id: BankId,
11229        stall_key: Pubkey,
11230    ) -> ScanTracker {
11231        let exit = Arc::new(AtomicBool::new(false));
11232        let exit_ = exit.clone();
11233        let ready = Arc::new(AtomicBool::new(false));
11234        let ready_ = ready.clone();
11235
11236        let t_scan = Builder::new()
11237            .name("scan".to_string())
11238            .spawn(move || {
11239                db.scan_accounts(
11240                    &scan_ancestors,
11241                    bank_id,
11242                    |_collector: &mut Vec<(Pubkey, AccountSharedData)>, maybe_account| {
11243                        ready_.store(true, Ordering::Relaxed);
11244                        if let Some((pubkey, _, _)) = maybe_account {
11245                            if *pubkey == stall_key {
11246                                loop {
11247                                    if exit_.load(Ordering::Relaxed) {
11248                                        break;
11249                                    } else {
11250                                        sleep(Duration::from_millis(10));
11251                                    }
11252                                }
11253                            }
11254                        }
11255                    },
11256                )
11257                .unwrap();
11258            })
11259            .unwrap();
11260
11261        // Wait for scan to start
11262        while !ready.load(Ordering::Relaxed) {
11263            sleep(Duration::from_millis(10));
11264        }
11265
11266        ScanTracker { t_scan, exit }
11267    }
11268
11269    #[test]
11270    fn test_scan_flush_accounts_cache_then_clean_drop() {
11271        let caching_enabled = true;
11272        let db = Arc::new(AccountsDb::new_with_config_for_tests(
11273            Vec::new(),
11274            &ClusterType::Development,
11275            AccountSecondaryIndexes::default(),
11276            caching_enabled,
11277            AccountShrinkThreshold::default(),
11278        ));
11279        let account_key = Pubkey::new_unique();
11280        let account_key2 = Pubkey::new_unique();
11281        let zero_carat_account =
11282            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
11283        let slot1_account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
11284        let slot2_account = AccountSharedData::new(2, 1, AccountSharedData::default().owner());
11285
11286        /*
11287            Store zero carat account into slots 0, 1, 2 where
11288            root slots are 0, 2, and slot 1 is unrooted.
11289                                    0 (root)
11290                                /        \
11291                              1            2 (root)
11292        */
11293        db.store_cached(0, &[(&account_key, &zero_carat_account)]);
11294        db.store_cached(1, &[(&account_key, &slot1_account)]);
11295        // Fodder for the scan so that the lock on `account_key` is not held
11296        db.store_cached(1, &[(&account_key2, &slot1_account)]);
11297        db.store_cached(2, &[(&account_key, &slot2_account)]);
11298        db.get_accounts_delta_hash(0);
11299
11300        let max_scan_root = 0;
11301        db.add_root(max_scan_root);
11302        let scan_ancestors: Arc<Ancestors> = Arc::new(vec![(0, 1), (1, 1)].into_iter().collect());
11303        let bank_id = 0;
11304        let scan_tracker = setup_scan(db.clone(), scan_ancestors.clone(), bank_id, account_key2);
11305
11306        // Add a new root 2
11307        let new_root = 2;
11308        db.get_accounts_delta_hash(new_root);
11309        db.add_root(new_root);
11310
11311        // Check that the scan is properly set up
11312        assert_eq!(
11313            db.accounts_index.min_ongoing_scan_root().unwrap(),
11314            max_scan_root
11315        );
11316
11317        // If we specify a requested_flush_root == 2, then `slot 2 <= max_flush_slot` will
11318        // be flushed even though `slot 2 > max_scan_root`. The unrooted slot 1 should
11319        // remain in the cache
11320        db.flush_accounts_cache(true, Some(new_root));
11321        assert_eq!(db.accounts_cache.num_slots(), 1);
11322        assert!(db.accounts_cache.slot_cache(1).is_some());
11323
11324        // Intra cache cleaning should not clean the entry for `account_key` from slot 0,
11325        // even though it was updated in slot `2` because of the ongoing scan
11326        let account = db
11327            .do_load(
11328                &Ancestors::default(),
11329                &account_key,
11330                Some(0),
11331                LoadHint::Unspecified,
11332            )
11333            .unwrap();
11334        assert_eq!(account.0.carats(), zero_carat_account.carats());
11335
11336        // Run clean, unrooted slot 1 should not be purged, and still readable from the cache,
11337        // because we're still doing a scan on it.
11338        db.clean_accounts(None, false, None);
11339        let account = db
11340            .do_load(
11341                &scan_ancestors,
11342                &account_key,
11343                Some(max_scan_root),
11344                LoadHint::Unspecified,
11345            )
11346            .unwrap();
11347        assert_eq!(account.0.carats(), slot1_account.carats());
11348
11349        // When the scan is over, clean should not panic and should not purge something
11350        // still in the cache.
11351        scan_tracker.exit().unwrap();
11352        db.clean_accounts(None, false, None);
11353        let account = db
11354            .do_load(
11355                &scan_ancestors,
11356                &account_key,
11357                Some(max_scan_root),
11358                LoadHint::Unspecified,
11359            )
11360            .unwrap();
11361        assert_eq!(account.0.carats(), slot1_account.carats());
11362
11363        // Simulate dropping the bank, which finally removes the slot from the cache
11364        let bank_id = 1;
11365        db.purge_slot(1, bank_id, false);
11366        assert!(db
11367            .do_load(
11368                &scan_ancestors,
11369                &account_key,
11370                Some(max_scan_root),
11371                LoadHint::Unspecified
11372            )
11373            .is_none());
11374    }
11375
11376    #[test]
11377    fn test_alive_bytes() {
11378        let caching_enabled = true;
11379        let accounts_db = AccountsDb::new_with_config_for_tests(
11380            Vec::new(),
11381            &ClusterType::Development,
11382            AccountSecondaryIndexes::default(),
11383            caching_enabled,
11384            AccountShrinkThreshold::default(),
11385        );
11386        let slot: Slot = 0;
11387        let num_keys = 10;
11388
11389        for data_size in 0..num_keys {
11390            let account = AccountSharedData::new(1, data_size, &Pubkey::default());
11391            accounts_db.store_cached(slot, &[(&Pubkey::new_unique(), &account)]);
11392        }
11393
11394        accounts_db.add_root(slot);
11395        accounts_db.flush_accounts_cache(true, None);
11396
11397        let mut storage_maps: Vec<Arc<AccountStorageEntry>> = accounts_db
11398            .storage
11399            .get_slot_storage_entries(slot)
11400            .unwrap_or_default();
11401
11402        // Flushing cache should only create one storage entry
11403        assert_eq!(storage_maps.len(), 1);
11404        let storage0 = storage_maps.pop().unwrap();
11405        let accounts = storage0.all_accounts();
11406
11407        for account in accounts {
11408            let before_size = storage0.alive_bytes.load(Ordering::Relaxed);
11409            let account_info = accounts_db
11410                .accounts_index
11411                .get_account_read_entry(&account.meta.pubkey)
11412                .map(|locked_entry| {
11413                    // Should only be one entry per key, since every key was only stored to slot 0
11414                    locked_entry.slot_list()[0]
11415                })
11416                .unwrap();
11417            let removed_data_size = account_info.1.stored_size;
11418            // Fetching the account from storage should return the same
11419            // stored size as in the index.
11420            assert_eq!(removed_data_size, account.stored_size);
11421            assert_eq!(account_info.0, slot);
11422            let reclaims = vec![account_info];
11423            accounts_db.remove_dead_accounts(&reclaims, None, None, true);
11424            let after_size = storage0.alive_bytes.load(Ordering::Relaxed);
11425            assert_eq!(before_size, after_size + account.stored_size);
11426        }
11427    }
11428
11429    fn setup_accounts_db_cache_clean(
11430        num_slots: usize,
11431        scan_slot: Option<Slot>,
11432    ) -> (Arc<AccountsDb>, Vec<Pubkey>, Vec<Slot>, Option<ScanTracker>) {
11433        let caching_enabled = true;
11434        let accounts_db = Arc::new(AccountsDb::new_with_config_for_tests(
11435            Vec::new(),
11436            &ClusterType::Development,
11437            AccountSecondaryIndexes::default(),
11438            caching_enabled,
11439            AccountShrinkThreshold::default(),
11440        ));
11441        let slots: Vec<_> = (0..num_slots as Slot).into_iter().collect();
11442        let stall_slot = num_slots as Slot;
11443        let scan_stall_key = Pubkey::new_unique();
11444        let keys: Vec<Pubkey> = std::iter::repeat_with(Pubkey::new_unique)
11445            .take(num_slots)
11446            .collect();
11447        if scan_slot.is_some() {
11448            accounts_db.store_cached(
11449                // Store it in a slot that isn't returned in `slots`
11450                stall_slot,
11451                &[(
11452                    &scan_stall_key,
11453                    &AccountSharedData::new(1, 0, &Pubkey::default()),
11454                )],
11455            );
11456        }
11457
11458        // Store some subset of the keys in slots 0..num_slots
11459        let mut scan_tracker = None;
11460        for slot in &slots {
11461            for key in &keys[*slot as usize..] {
11462                accounts_db.store_cached(
11463                    *slot,
11464                    &[(key, &AccountSharedData::new(1, 0, &Pubkey::default()))],
11465                );
11466            }
11467            accounts_db.add_root(*slot as Slot);
11468            if Some(*slot) == scan_slot {
11469                let ancestors = Arc::new(vec![(stall_slot, 1), (*slot, 1)].into_iter().collect());
11470                let bank_id = 0;
11471                scan_tracker = Some(setup_scan(
11472                    accounts_db.clone(),
11473                    ancestors,
11474                    bank_id,
11475                    scan_stall_key,
11476                ));
11477                assert_eq!(
11478                    accounts_db.accounts_index.min_ongoing_scan_root().unwrap(),
11479                    *slot
11480                );
11481            }
11482        }
11483
11484        accounts_db.accounts_cache.remove_slot(stall_slot);
11485
11486        // If there's <= MAX_CACHE_SLOTS, no slots should be flushed
11487        if accounts_db.accounts_cache.num_slots() <= MAX_CACHE_SLOTS {
11488            accounts_db.flush_accounts_cache(false, None);
11489            assert_eq!(accounts_db.accounts_cache.num_slots(), num_slots);
11490        }
11491
11492        (accounts_db, keys, slots, scan_tracker)
11493    }
11494
11495    #[test]
11496    fn test_accounts_db_cache_clean_dead_slots() {
11497        let num_slots = 10;
11498        let (accounts_db, keys, mut slots, _) = setup_accounts_db_cache_clean(num_slots, None);
11499        let last_dead_slot = (num_slots - 1) as Slot;
11500        assert_eq!(*slots.last().unwrap(), last_dead_slot);
11501        let alive_slot = last_dead_slot as Slot + 1;
11502        slots.push(alive_slot);
11503        for key in &keys {
11504            // Store a slot that overwrites all previous keys, rendering all previous keys dead
11505            accounts_db.store_cached(
11506                alive_slot,
11507                &[(key, &AccountSharedData::new(1, 0, &Pubkey::default()))],
11508            );
11509            accounts_db.add_root(alive_slot);
11510        }
11511
11512        // Before the flush, we can find entries in the database for slots < alive_slot if we specify
11513        // a smaller max root
11514        for key in &keys {
11515            assert!(accounts_db
11516                .do_load(
11517                    &Ancestors::default(),
11518                    key,
11519                    Some(last_dead_slot),
11520                    LoadHint::Unspecified
11521                )
11522                .is_some());
11523        }
11524
11525        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
11526        accounts_db.flush_accounts_cache(true, None);
11527        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
11528        let mut uncleaned_roots = accounts_db
11529            .accounts_index
11530            .clear_uncleaned_roots(None)
11531            .into_iter()
11532            .collect::<Vec<_>>();
11533        uncleaned_roots.sort_unstable();
11534        assert_eq!(uncleaned_roots, slots);
11535        assert_eq!(
11536            accounts_db.accounts_cache.fetch_max_flush_root(),
11537            alive_slot,
11538        );
11539
11540        // Specifying a max_root < alive_slot, should not return any more entries,
11541        // as those have been purged from the accounts index for the dead slots.
11542        for key in &keys {
11543            assert!(accounts_db
11544                .do_load(
11545                    &Ancestors::default(),
11546                    key,
11547                    Some(last_dead_slot),
11548                    LoadHint::Unspecified
11549                )
11550                .is_none());
11551        }
11552        // Each slot should only have one entry in the storage, since all other accounts were
11553        // cleaned due to later updates
11554        for slot in &slots {
11555            if let ScanStorageResult::Stored(slot_accounts) = accounts_db.scan_account_storage(
11556                *slot as Slot,
11557                |_| Some(0),
11558                |slot_accounts: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
11559                    slot_accounts.insert(*loaded_account.pubkey());
11560                },
11561            ) {
11562                if *slot == alive_slot {
11563                    assert_eq!(slot_accounts.len(), keys.len());
11564                } else {
11565                    assert!(slot_accounts.is_empty());
11566                }
11567            } else {
11568                panic!("Expected slot to be in storage, not cache");
11569            }
11570        }
11571    }
11572
11573    #[test]
11574    fn test_accounts_db_cache_clean() {
11575        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(10, None);
11576
11577        // If no `max_clean_root` is specified, cleaning should purge all flushed slots
11578        accounts_db.flush_accounts_cache(true, None);
11579        assert_eq!(accounts_db.accounts_cache.num_slots(), 0);
11580        let mut uncleaned_roots = accounts_db
11581            .accounts_index
11582            .clear_uncleaned_roots(None)
11583            .into_iter()
11584            .collect::<Vec<_>>();
11585        uncleaned_roots.sort_unstable();
11586        assert_eq!(uncleaned_roots, slots);
11587        assert_eq!(
11588            accounts_db.accounts_cache.fetch_max_flush_root(),
11589            *slots.last().unwrap()
11590        );
11591
11592        // Each slot should only have one entry in the storage, since all other accounts were
11593        // cleaned due to later updates
11594        for slot in &slots {
11595            if let ScanStorageResult::Stored(slot_account) = accounts_db.scan_account_storage(
11596                *slot as Slot,
11597                |_| Some(0),
11598                |slot_account: &Arc<RwLock<Pubkey>>, loaded_account: LoadedAccount| {
11599                    *slot_account.write().unwrap() = *loaded_account.pubkey();
11600                },
11601            ) {
11602                assert_eq!(*slot_account.read().unwrap(), keys[*slot as usize]);
11603            } else {
11604                panic!("Everything should have been flushed")
11605            }
11606        }
11607    }
11608
11609    fn run_test_accounts_db_cache_clean_max_root(
11610        num_slots: usize,
11611        requested_flush_root: Slot,
11612        scan_root: Option<Slot>,
11613    ) {
11614        assert!(requested_flush_root < (num_slots as Slot));
11615        let (accounts_db, keys, slots, scan_tracker) =
11616            setup_accounts_db_cache_clean(num_slots, scan_root);
11617        let is_cache_at_limit = num_slots - requested_flush_root as usize - 1 > MAX_CACHE_SLOTS;
11618
11619        // If:
11620        // 1) `requested_flush_root` is specified,
11621        // 2) not at the cache limit, i.e. `is_cache_at_limit == false`, then
11622        // `flush_accounts_cache()` should clean and flush only slots <= requested_flush_root,
11623        accounts_db.flush_accounts_cache(true, Some(requested_flush_root));
11624
11625        if !is_cache_at_limit {
11626            // Should flush all slots between 0..=requested_flush_root
11627            assert_eq!(
11628                accounts_db.accounts_cache.num_slots(),
11629                slots.len() - requested_flush_root as usize - 1
11630            );
11631        } else {
11632            // Otherwise, if we are at the cache limit, all roots will be flushed
11633            assert_eq!(accounts_db.accounts_cache.num_slots(), 0,);
11634        }
11635
11636        let mut uncleaned_roots = accounts_db
11637            .accounts_index
11638            .clear_uncleaned_roots(None)
11639            .into_iter()
11640            .collect::<Vec<_>>();
11641        uncleaned_roots.sort_unstable();
11642
11643        let expected_max_flushed_root = if !is_cache_at_limit {
11644            // Should flush all slots between 0..=requested_flush_root
11645            requested_flush_root
11646        } else {
11647            // Otherwise, if we are at the cache limit, all roots will be flushed
11648            num_slots as Slot - 1
11649        };
11650
11651        assert_eq!(
11652            uncleaned_roots,
11653            slots[0..=expected_max_flushed_root as usize].to_vec()
11654        );
11655        assert_eq!(
11656            accounts_db.accounts_cache.fetch_max_flush_root(),
11657            expected_max_flushed_root,
11658        );
11659
11660        for slot in &slots {
11661            let slot_accounts = accounts_db.scan_account_storage(
11662                *slot as Slot,
11663                |loaded_account: LoadedAccount| {
11664                    if is_cache_at_limit {
11665                        panic!(
11666                            "When cache is at limit, all roots should have been flushed to storage"
11667                        );
11668                    }
11669                    // All slots <= requested_flush_root should have been flushed, regardless
11670                    // of ongoing scans
11671                    assert!(*slot > requested_flush_root);
11672                    Some(*loaded_account.pubkey())
11673                },
11674                |slot_accounts: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
11675                    slot_accounts.insert(*loaded_account.pubkey());
11676                    if !is_cache_at_limit {
11677                        // Only true when the limit hasn't been reached and there are still
11678                        // slots left in the cache
11679                        assert!(*slot <= requested_flush_root);
11680                    }
11681                },
11682            );
11683
11684            let slot_accounts = match slot_accounts {
11685                ScanStorageResult::Cached(slot_accounts) => {
11686                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
11687                }
11688                ScanStorageResult::Stored(slot_accounts) => {
11689                    slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
11690                }
11691            };
11692
11693            let expected_accounts =
11694                if *slot >= requested_flush_root || *slot >= scan_root.unwrap_or(Slot::MAX) {
11695                    // 1) If slot > `requested_flush_root`, then  either:
11696                    //   a) If `is_cache_at_limit == false`, still in the cache
11697                    //   b) if `is_cache_at_limit == true`, were not cleaned before being flushed to storage.
11698                    //
11699                    // In both cases all the *original* updates at index `slot` were uncleaned and thus
11700                    // should be discoverable by this scan.
11701                    //
11702                    // 2) If slot == `requested_flush_root`, the slot was not cleaned before being flushed to storage,
11703                    // so it also contains all the original updates.
11704                    //
11705                    // 3) If *slot >= scan_root, then we should not clean it either
11706                    keys[*slot as usize..]
11707                        .iter()
11708                        .cloned()
11709                        .collect::<HashSet<Pubkey>>()
11710                } else {
11711                    // Slots less than `requested_flush_root` and `scan_root` were cleaned in the cache before being flushed
11712                    // to storage, should only contain one account
11713                    std::iter::once(keys[*slot as usize])
11714                        .into_iter()
11715                        .collect::<HashSet<Pubkey>>()
11716                };
11717
11718            assert_eq!(slot_accounts, expected_accounts);
11719        }
11720
11721        if let Some(scan_tracker) = scan_tracker {
11722            scan_tracker.exit().unwrap();
11723        }
11724    }
11725
11726    #[test]
11727    fn test_accounts_db_cache_clean_max_root() {
11728        let requested_flush_root = 5;
11729        run_test_accounts_db_cache_clean_max_root(10, requested_flush_root, None);
11730    }
11731
11732    #[test]
11733    fn test_accounts_db_cache_clean_max_root_with_scan() {
11734        let requested_flush_root = 5;
11735        run_test_accounts_db_cache_clean_max_root(
11736            10,
11737            requested_flush_root,
11738            Some(requested_flush_root - 1),
11739        );
11740        run_test_accounts_db_cache_clean_max_root(
11741            10,
11742            requested_flush_root,
11743            Some(requested_flush_root + 1),
11744        );
11745    }
11746
11747    #[test]
11748    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit() {
11749        let requested_flush_root = 5;
11750        // Test that if there are > MAX_CACHE_SLOTS in the cache after flush, then more roots
11751        // will be flushed
11752        run_test_accounts_db_cache_clean_max_root(
11753            MAX_CACHE_SLOTS + requested_flush_root as usize + 2,
11754            requested_flush_root,
11755            None,
11756        );
11757    }
11758
11759    #[test]
11760    fn test_accounts_db_cache_clean_max_root_with_cache_limit_hit_and_scan() {
11761        let requested_flush_root = 5;
11762        // Test that if there are > MAX_CACHE_SLOTS in the cache after flush, then more roots
11763        // will be flushed
11764        run_test_accounts_db_cache_clean_max_root(
11765            MAX_CACHE_SLOTS + requested_flush_root as usize + 2,
11766            requested_flush_root,
11767            Some(requested_flush_root - 1),
11768        );
11769        run_test_accounts_db_cache_clean_max_root(
11770            MAX_CACHE_SLOTS + requested_flush_root as usize + 2,
11771            requested_flush_root,
11772            Some(requested_flush_root + 1),
11773        );
11774    }
11775
11776    fn run_flush_rooted_accounts_cache(should_clean: bool) {
11777        let num_slots = 10;
11778        let (accounts_db, keys, slots, _) = setup_accounts_db_cache_clean(num_slots, None);
11779        let mut cleaned_bytes = 0;
11780        let mut cleaned_accounts = 0;
11781        let should_clean_tracker = if should_clean {
11782            Some((&mut cleaned_bytes, &mut cleaned_accounts))
11783        } else {
11784            None
11785        };
11786
11787        // If no cleaning is specified, then flush everything
11788        accounts_db.flush_rooted_accounts_cache(None, should_clean_tracker);
11789        for slot in &slots {
11790            let slot_accounts = if let ScanStorageResult::Stored(slot_accounts) = accounts_db
11791                .scan_account_storage(
11792                    *slot as Slot,
11793                    |_| Some(0),
11794                    |slot_account: &DashSet<Pubkey>, loaded_account: LoadedAccount| {
11795                        slot_account.insert(*loaded_account.pubkey());
11796                    },
11797                ) {
11798                slot_accounts.into_iter().collect::<HashSet<Pubkey>>()
11799            } else {
11800                panic!("All roots should have been flushed to storage");
11801            };
11802            if !should_clean || slot == slots.last().unwrap() {
11803                // The slot was not cleaned before being flushed to storage,
11804                // so it also contains all the original updates.
11805                assert_eq!(
11806                    slot_accounts,
11807                    keys[*slot as usize..]
11808                        .iter()
11809                        .cloned()
11810                        .collect::<HashSet<Pubkey>>()
11811                );
11812            } else {
11813                // If clean was specified, only the latest slot should have all the updates.
11814                // All these other slots have been cleaned before flush
11815                assert_eq!(
11816                    slot_accounts,
11817                    std::iter::once(keys[*slot as usize])
11818                        .into_iter()
11819                        .collect::<HashSet<Pubkey>>()
11820                );
11821            }
11822        }
11823    }
11824
11825    #[test]
11826    fn test_flush_rooted_accounts_cache_with_clean() {
11827        run_flush_rooted_accounts_cache(true);
11828    }
11829
11830    #[test]
11831    fn test_flush_rooted_accounts_cache_without_clean() {
11832        run_flush_rooted_accounts_cache(false);
11833    }
11834
11835    fn run_test_shrink_unref(do_intra_cache_clean: bool) {
11836        // Enable caching so that we use the straightforward implementation
11837        // of shrink that will shrink all candidate slots
11838        let caching_enabled = true;
11839        let db = AccountsDb::new_with_config_for_tests(
11840            Vec::new(),
11841            &ClusterType::Development,
11842            AccountSecondaryIndexes::default(),
11843            caching_enabled,
11844            AccountShrinkThreshold::default(),
11845        );
11846        let account_key1 = Pubkey::new_unique();
11847        let account_key2 = Pubkey::new_unique();
11848        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11849
11850        // Store into slot 0
11851        db.store_cached(0, &[(&account_key1, &account1)]);
11852        db.store_cached(0, &[(&account_key2, &account1)]);
11853        db.add_root(0);
11854        if !do_intra_cache_clean {
11855            // If we don't want the cache doing purges before flush,
11856            // then we cannot flush multiple roots at once, otherwise the later
11857            // roots will clean the earlier roots before they are stored.
11858            // Thus flush the roots individually
11859            db.flush_accounts_cache(true, None);
11860
11861            // Add an additional ref within the same slot to pubkey 1
11862            db.store_uncached(0, &[(&account_key1, &account1)]);
11863        }
11864
11865        // Make account_key1 in slot 0 outdated by updating in rooted slot 1
11866        db.store_cached(1, &[(&account_key1, &account1)]);
11867        db.add_root(1);
11868        // Flushes all roots
11869        db.flush_accounts_cache(true, None);
11870        db.get_accounts_delta_hash(0);
11871        db.get_accounts_delta_hash(1);
11872
11873        // Clean to remove outdated entry from slot 0
11874        db.clean_accounts(Some(1), false, None);
11875
11876        // Shrink Slot 0
11877        let mut slot0_stores = db.storage.get_slot_storage_entries(0).unwrap();
11878        assert_eq!(slot0_stores.len(), 1);
11879        let slot0_store = slot0_stores.pop().unwrap();
11880        {
11881            let mut shrink_candidate_slots = db.shrink_candidate_slots.lock().unwrap();
11882            shrink_candidate_slots
11883                .entry(0)
11884                .or_default()
11885                .insert(slot0_store.append_vec_id(), slot0_store);
11886        }
11887        db.shrink_candidate_slots();
11888
11889        // Make slot 0 dead by updating the remaining key
11890        db.store_cached(2, &[(&account_key2, &account1)]);
11891        db.add_root(2);
11892
11893        // Flushes all roots
11894        db.flush_accounts_cache(true, None);
11895
11896        // Should be one store before clean for slot 0
11897        assert_eq!(db.storage.get_slot_storage_entries(0).unwrap().len(), 1);
11898        db.get_accounts_delta_hash(2);
11899        db.clean_accounts(Some(2), false, None);
11900
11901        // No stores should exist for slot 0 after clean
11902        assert!(db.storage.get_slot_storage_entries(0).is_none());
11903
11904        // Ref count for `account_key1` (account removed earlier by shrink)
11905        // should be 1, since it was only stored in slot 0 and 1, and slot 0
11906        // is now dead
11907        assert_eq!(db.accounts_index.ref_count_from_storage(&account_key1), 1);
11908    }
11909
11910    #[test]
11911    fn test_shrink_unref() {
11912        run_test_shrink_unref(false)
11913    }
11914
11915    #[test]
11916    fn test_shrink_unref_with_intra_slot_cleaning() {
11917        run_test_shrink_unref(true)
11918    }
11919
11920    #[test]
11921    fn test_partial_clean() {
11922        gemachain_logger::setup();
11923        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
11924        let account_key1 = Pubkey::new_unique();
11925        let account_key2 = Pubkey::new_unique();
11926        let account1 = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
11927        let account2 = AccountSharedData::new(2, 0, AccountSharedData::default().owner());
11928        let account3 = AccountSharedData::new(3, 0, AccountSharedData::default().owner());
11929        let account4 = AccountSharedData::new(4, 0, AccountSharedData::default().owner());
11930
11931        // Store accounts into slots 0 and 1
11932        db.store_uncached(0, &[(&account_key1, &account1)]);
11933        db.store_uncached(0, &[(&account_key2, &account1)]);
11934        db.store_uncached(1, &[(&account_key1, &account2)]);
11935        db.get_accounts_delta_hash(0);
11936        db.get_accounts_delta_hash(1);
11937
11938        db.print_accounts_stats("pre-clean1");
11939
11940        // clean accounts - no accounts should be cleaned, since no rooted slots
11941        //
11942        // Checking that the uncleaned_pubkeys are not pre-maturely removed
11943        // such that when the slots are rooted, and can actually be cleaned, then the
11944        // delta keys are still there.
11945        db.clean_accounts(None, false, None);
11946
11947        db.print_accounts_stats("post-clean1");
11948        // Check stores > 0
11949        assert!(!slot_stores(&db, 0).is_empty());
11950        assert!(!slot_stores(&db, 1).is_empty());
11951
11952        // root slot 0
11953        db.add_root(0);
11954
11955        // store into slot 2
11956        db.store_uncached(2, &[(&account_key2, &account3)]);
11957        db.store_uncached(2, &[(&account_key1, &account3)]);
11958        db.get_accounts_delta_hash(2);
11959
11960        db.clean_accounts(None, false, None);
11961        db.print_accounts_stats("post-clean2");
11962
11963        // root slots 1
11964        db.add_root(1);
11965        db.clean_accounts(None, false, None);
11966
11967        db.print_accounts_stats("post-clean3");
11968
11969        db.store_uncached(3, &[(&account_key2, &account4)]);
11970        db.get_accounts_delta_hash(3);
11971        db.add_root(3);
11972
11973        // Check that we can clean where max_root=3 and slot=2 is not rooted
11974        db.clean_accounts(None, false, None);
11975
11976        assert!(db.uncleaned_pubkeys.is_empty());
11977
11978        db.print_accounts_stats("post-clean4");
11979
11980        assert!(slot_stores(&db, 0).is_empty());
11981        assert!(!slot_stores(&db, 1).is_empty());
11982    }
11983
11984    #[test]
11985    fn test_recycle_stores_expiration() {
11986        gemachain_logger::setup();
11987
11988        let common_store_path = Path::new("");
11989        let common_slot_id = 12;
11990        let store_file_size = 1000;
11991
11992        let store1_id = 22;
11993        let entry1 = Arc::new(AccountStorageEntry::new(
11994            common_store_path,
11995            common_slot_id,
11996            store1_id,
11997            store_file_size,
11998        ));
11999
12000        let store2_id = 44;
12001        let entry2 = Arc::new(AccountStorageEntry::new(
12002            common_store_path,
12003            common_slot_id,
12004            store2_id,
12005            store_file_size,
12006        ));
12007
12008        let mut recycle_stores = RecycleStores::default();
12009        recycle_stores.add_entry(entry1);
12010        recycle_stores.add_entry(entry2);
12011        assert_eq!(recycle_stores.entry_count(), 2);
12012
12013        // no expiration for newly added entries
12014        let expired = recycle_stores.expire_old_entries();
12015        assert_eq!(
12016            expired
12017                .iter()
12018                .map(|e| e.append_vec_id())
12019                .collect::<Vec<_>>(),
12020            Vec::<AppendVecId>::new()
12021        );
12022        assert_eq!(
12023            recycle_stores
12024                .iter()
12025                .map(|(_, e)| e.append_vec_id())
12026                .collect::<Vec<_>>(),
12027            vec![store1_id, store2_id]
12028        );
12029        assert_eq!(recycle_stores.entry_count(), 2);
12030        assert_eq!(recycle_stores.total_bytes(), store_file_size * 2);
12031
12032        // expiration for only too old entries
12033        recycle_stores.entries[0].0 =
12034            Instant::now() - Duration::from_secs(EXPIRATION_TTL_SECONDS + 1);
12035        let expired = recycle_stores.expire_old_entries();
12036        assert_eq!(
12037            expired
12038                .iter()
12039                .map(|e| e.append_vec_id())
12040                .collect::<Vec<_>>(),
12041            vec![store1_id]
12042        );
12043        assert_eq!(
12044            recycle_stores
12045                .iter()
12046                .map(|(_, e)| e.append_vec_id())
12047                .collect::<Vec<_>>(),
12048            vec![store2_id]
12049        );
12050        assert_eq!(recycle_stores.entry_count(), 1);
12051        assert_eq!(recycle_stores.total_bytes(), store_file_size);
12052    }
12053
12054    const RACY_SLEEP_MS: u64 = 10;
12055    const RACE_TIME: u64 = 5;
12056
12057    fn start_load_thread(
12058        with_retry: bool,
12059        ancestors: Ancestors,
12060        db: Arc<AccountsDb>,
12061        exit: Arc<AtomicBool>,
12062        pubkey: Arc<Pubkey>,
12063        expected_carats: impl Fn(&(AccountSharedData, Slot)) -> u64 + Send + 'static,
12064    ) -> JoinHandle<()> {
12065        let load_hint = if with_retry {
12066            LoadHint::FixedMaxRoot
12067        } else {
12068            LoadHint::Unspecified
12069        };
12070
12071        std::thread::Builder::new()
12072            .name("account-do-load".to_string())
12073            .spawn(move || {
12074                loop {
12075                    if exit.load(Ordering::Relaxed) {
12076                        return;
12077                    }
12078                    // Meddle load_limit to cover all branches of implementation.
12079                    // There should absolutely no behaviorial difference; the load_limit triggered
12080                    // slow branch should only affect the performance.
12081                    // Ordering::Relaxed is ok because of no data dependencies; the modified field is
12082                    // completely free-standing cfg(test) control-flow knob.
12083                    db.load_limit
12084                        .store(thread_rng().gen_range(0, 10) as u64, Ordering::Relaxed);
12085
12086                    // Load should never be unable to find this key
12087                    let loaded_account = db.do_load(&ancestors, &pubkey, None, load_hint).unwrap();
12088                    // slot + 1 == account.carats because of the account-cache-flush thread
12089                    assert_eq!(
12090                        loaded_account.0.carats(),
12091                        expected_carats(&loaded_account)
12092                    );
12093                }
12094            })
12095            .unwrap()
12096    }
12097
12098    fn do_test_load_account_and_cache_flush_race(with_retry: bool) {
12099        gemachain_logger::setup();
12100
12101        let caching_enabled = true;
12102        let mut db = AccountsDb::new_with_config_for_tests(
12103            Vec::new(),
12104            &ClusterType::Development,
12105            AccountSecondaryIndexes::default(),
12106            caching_enabled,
12107            AccountShrinkThreshold::default(),
12108        );
12109        db.load_delay = RACY_SLEEP_MS;
12110        let db = Arc::new(db);
12111        let pubkey = Arc::new(Pubkey::new_unique());
12112        let exit = Arc::new(AtomicBool::new(false));
12113        db.store_cached(
12114            0,
12115            &[(
12116                &pubkey,
12117                &AccountSharedData::new(1, 0, AccountSharedData::default().owner()),
12118            )],
12119        );
12120        db.add_root(0);
12121        db.flush_accounts_cache(true, None);
12122
12123        let t_flush_accounts_cache = {
12124            let db = db.clone();
12125            let exit = exit.clone();
12126            let pubkey = pubkey.clone();
12127            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12128            std::thread::Builder::new()
12129                .name("account-cache-flush".to_string())
12130                .spawn(move || {
12131                    let mut slot = 1;
12132                    loop {
12133                        if exit.load(Ordering::Relaxed) {
12134                            return;
12135                        }
12136                        account.set_carats(slot + 1);
12137                        db.store_cached(slot, &[(&pubkey, &account)]);
12138                        db.add_root(slot);
12139                        sleep(Duration::from_millis(RACY_SLEEP_MS));
12140                        db.flush_accounts_cache(true, None);
12141                        slot += 1;
12142                    }
12143                })
12144                .unwrap()
12145        };
12146
12147        let t_do_load = start_load_thread(
12148            with_retry,
12149            Ancestors::default(),
12150            db,
12151            exit.clone(),
12152            pubkey,
12153            |(_, slot)| slot + 1,
12154        );
12155
12156        sleep(Duration::from_secs(RACE_TIME));
12157        exit.store(true, Ordering::Relaxed);
12158        t_flush_accounts_cache.join().unwrap();
12159        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
12160    }
12161
12162    #[test]
12163    fn test_load_account_and_cache_flush_race_with_retry() {
12164        do_test_load_account_and_cache_flush_race(true);
12165    }
12166
12167    #[test]
12168    fn test_load_account_and_cache_flush_race_without_retry() {
12169        do_test_load_account_and_cache_flush_race(false);
12170    }
12171
12172    fn do_test_load_account_and_shrink_race(with_retry: bool) {
12173        let caching_enabled = true;
12174        let mut db = AccountsDb::new_with_config_for_tests(
12175            Vec::new(),
12176            &ClusterType::Development,
12177            AccountSecondaryIndexes::default(),
12178            caching_enabled,
12179            AccountShrinkThreshold::default(),
12180        );
12181        db.load_delay = RACY_SLEEP_MS;
12182        let db = Arc::new(db);
12183        let pubkey = Arc::new(Pubkey::new_unique());
12184        let exit = Arc::new(AtomicBool::new(false));
12185        let slot = 1;
12186
12187        // Store an account
12188        let carats = 42;
12189        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12190        account.set_carats(carats);
12191        db.store_uncached(slot, &[(&pubkey, &account)]);
12192
12193        // Set the slot as a root so account loads will see the contents of this slot
12194        db.add_root(slot);
12195
12196        let t_shrink_accounts = {
12197            let db = db.clone();
12198            let exit = exit.clone();
12199
12200            std::thread::Builder::new()
12201                .name("account-shrink".to_string())
12202                .spawn(move || loop {
12203                    if exit.load(Ordering::Relaxed) {
12204                        return;
12205                    }
12206                    // Simulate adding shrink candidates from clean_accounts()
12207                    let stores = db.storage.get_slot_storage_entries(slot).unwrap();
12208                    assert_eq!(stores.len(), 1);
12209                    let store = &stores[0];
12210                    let store_id = store.append_vec_id();
12211                    db.shrink_candidate_slots
12212                        .lock()
12213                        .unwrap()
12214                        .entry(slot)
12215                        .or_default()
12216                        .insert(store_id, store.clone());
12217                    db.shrink_candidate_slots();
12218                })
12219                .unwrap()
12220        };
12221
12222        let t_do_load = start_load_thread(
12223            with_retry,
12224            Ancestors::default(),
12225            db,
12226            exit.clone(),
12227            pubkey,
12228            move |_| carats,
12229        );
12230
12231        sleep(Duration::from_secs(RACE_TIME));
12232        exit.store(true, Ordering::Relaxed);
12233        t_shrink_accounts.join().unwrap();
12234        t_do_load.join().map_err(std::panic::resume_unwind).unwrap()
12235    }
12236
12237    #[test]
12238    fn test_load_account_and_shrink_race_with_retry() {
12239        do_test_load_account_and_shrink_race(true);
12240    }
12241
12242    #[test]
12243    fn test_load_account_and_shrink_race_without_retry() {
12244        do_test_load_account_and_shrink_race(false);
12245    }
12246
12247    #[test]
12248    fn test_cache_flush_delayed_remove_unrooted_race() {
12249        let caching_enabled = true;
12250        let mut db = AccountsDb::new_with_config_for_tests(
12251            Vec::new(),
12252            &ClusterType::Development,
12253            AccountSecondaryIndexes::default(),
12254            caching_enabled,
12255            AccountShrinkThreshold::default(),
12256        );
12257        db.load_delay = RACY_SLEEP_MS;
12258        let db = Arc::new(db);
12259        let slot = 10;
12260        let bank_id = 10;
12261
12262        let carats = 42;
12263        let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12264        account.set_carats(carats);
12265
12266        // Start up a thread to flush the accounts cache
12267        let (flush_trial_start_sender, flush_trial_start_receiver) = unbounded();
12268        let (flush_done_sender, flush_done_receiver) = unbounded();
12269        let t_flush_cache = {
12270            let db = db.clone();
12271            std::thread::Builder::new()
12272                .name("account-cache-flush".to_string())
12273                .spawn(move || loop {
12274                    // Wait for the signal to start a trial
12275                    if flush_trial_start_receiver.recv().is_err() {
12276                        return;
12277                    }
12278                    db.flush_slot_cache(10, None::<&mut fn(&_, &_) -> bool>);
12279                    flush_done_sender.send(()).unwrap();
12280                })
12281                .unwrap()
12282        };
12283
12284        // Start up a thread remove the slot
12285        let (remove_trial_start_sender, remove_trial_start_receiver) = unbounded();
12286        let (remove_done_sender, remove_done_receiver) = unbounded();
12287        let t_remove = {
12288            let db = db.clone();
12289            std::thread::Builder::new()
12290                .name("account-remove".to_string())
12291                .spawn(move || loop {
12292                    // Wait for the signal to start a trial
12293                    if remove_trial_start_receiver.recv().is_err() {
12294                        return;
12295                    }
12296                    db.remove_unrooted_slots(&[(slot, bank_id)]);
12297                    remove_done_sender.send(()).unwrap();
12298                })
12299                .unwrap()
12300        };
12301
12302        let num_trials = 10;
12303        for _ in 0..num_trials {
12304            let pubkey = Pubkey::new_unique();
12305            db.store_cached(slot, &[(&pubkey, &account)]);
12306            // Wait for both threads to finish
12307            flush_trial_start_sender.send(()).unwrap();
12308            remove_trial_start_sender.send(()).unwrap();
12309            let _ = flush_done_receiver.recv();
12310            let _ = remove_done_receiver.recv();
12311        }
12312
12313        drop(flush_trial_start_sender);
12314        drop(remove_trial_start_sender);
12315        t_flush_cache.join().unwrap();
12316        t_remove.join().unwrap();
12317    }
12318
12319    #[test]
12320    fn test_cache_flush_remove_unrooted_race_multiple_slots() {
12321        let caching_enabled = true;
12322        let db = AccountsDb::new_with_config_for_tests(
12323            Vec::new(),
12324            &ClusterType::Development,
12325            AccountSecondaryIndexes::default(),
12326            caching_enabled,
12327            AccountShrinkThreshold::default(),
12328        );
12329        let db = Arc::new(db);
12330        let num_cached_slots = 100;
12331
12332        let num_trials = 100;
12333        let (new_trial_start_sender, new_trial_start_receiver) = unbounded();
12334        let (flush_done_sender, flush_done_receiver) = unbounded();
12335        // Start up a thread to flush the accounts cache
12336        let t_flush_cache = {
12337            let db = db.clone();
12338
12339            std::thread::Builder::new()
12340                .name("account-cache-flush".to_string())
12341                .spawn(move || loop {
12342                    // Wait for the signal to start a trial
12343                    if new_trial_start_receiver.recv().is_err() {
12344                        return;
12345                    }
12346                    for slot in 0..num_cached_slots {
12347                        db.flush_slot_cache(slot, None::<&mut fn(&_, &_) -> bool>);
12348                    }
12349                    flush_done_sender.send(()).unwrap();
12350                })
12351                .unwrap()
12352        };
12353
12354        let exit = Arc::new(AtomicBool::new(false));
12355
12356        let t_spurious_signal = {
12357            let db = db.clone();
12358            let exit = exit.clone();
12359            std::thread::Builder::new()
12360                .name("account-cache-flush".to_string())
12361                .spawn(move || loop {
12362                    if exit.load(Ordering::Relaxed) {
12363                        return;
12364                    }
12365                    // Simulate spurious wake-up that can happen, but is too rare to
12366                    // otherwise depend on in tests.
12367                    db.remove_unrooted_slots_synchronization.signal.notify_all();
12368                })
12369                .unwrap()
12370        };
12371
12372        // Run multiple trials. Has the added benefit of rewriting the same slots after we've
12373        // dumped them in previous trials.
12374        for _ in 0..num_trials {
12375            // Store an account
12376            let carats = 42;
12377            let mut account = AccountSharedData::new(1, 0, AccountSharedData::default().owner());
12378            account.set_carats(carats);
12379
12380            // Pick random 50% of the slots to pass to `remove_unrooted_slots()`
12381            let mut all_slots: Vec<(Slot, BankId)> = (0..num_cached_slots)
12382                .map(|slot| {
12383                    let bank_id = slot + 1;
12384                    (slot, bank_id)
12385                })
12386                .collect();
12387            all_slots.shuffle(&mut rand::thread_rng());
12388            let slots_to_dump = &all_slots[0..num_cached_slots as usize / 2];
12389            let slots_to_keep = &all_slots[num_cached_slots as usize / 2..];
12390
12391            // Set up a one account per slot across many different slots, track which
12392            // pubkey was stored in each slot.
12393            let slot_to_pubkey_map: HashMap<Slot, Pubkey> = (0..num_cached_slots)
12394                .map(|slot| {
12395                    let pubkey = Pubkey::new_unique();
12396                    db.store_cached(slot, &[(&pubkey, &account)]);
12397                    (slot, pubkey)
12398                })
12399                .collect();
12400
12401            // Signal the flushing shred to start flushing
12402            new_trial_start_sender.send(()).unwrap();
12403
12404            // Here we want to test both:
12405            // 1) Flush thread starts flushing a slot before we try dumping it.
12406            // 2) Flushing thread trying to flush while/after we're trying to dump the slot,
12407            // in which case flush should ignore/move past the slot to be dumped
12408            //
12409            // Hence, we split into chunks to get the dumping of each chunk to race with the
12410            // flushes. If we were to dump the entire chunk at once, then this reduces the possibility
12411            // of the flush occurring first since the dumping logic reserves all the slots it's about
12412            // to dump immediately.
12413
12414            for chunks in slots_to_dump.chunks(slots_to_dump.len() / 2) {
12415                db.remove_unrooted_slots(chunks);
12416            }
12417
12418            // Check that all the slots in `slots_to_dump` were completely removed from the
12419            // cache, storage, and index
12420
12421            for (slot, _) in slots_to_dump {
12422                assert!(db.storage.get_slot_storage_entries(*slot).is_none());
12423                assert!(db.accounts_cache.slot_cache(*slot).is_none());
12424                let account_in_slot = slot_to_pubkey_map[slot];
12425                let item = db.accounts_index.get_account_read_entry(&account_in_slot);
12426                assert!(item.is_none(), "item: {:?}", item);
12427            }
12428
12429            // Wait for flush to finish before starting next trial
12430
12431            flush_done_receiver.recv().unwrap();
12432
12433            for (slot, bank_id) in slots_to_keep {
12434                let account_in_slot = slot_to_pubkey_map[slot];
12435                assert!(db
12436                    .load(
12437                        &Ancestors::from(vec![(*slot, 0)]),
12438                        &account_in_slot,
12439                        LoadHint::FixedMaxRoot
12440                    )
12441                    .is_some());
12442                // Clear for next iteration so that `assert!(self.storage.get_slot_stores(purged_slot).is_none());`
12443                // in `purge_slot_pubkeys()` doesn't trigger
12444                db.remove_unrooted_slots(&[(*slot, *bank_id)]);
12445            }
12446        }
12447
12448        exit.store(true, Ordering::Relaxed);
12449        drop(new_trial_start_sender);
12450        t_flush_cache.join().unwrap();
12451
12452        t_spurious_signal.join().unwrap();
12453    }
12454
12455    #[test]
12456    fn test_collect_uncleaned_slots_up_to_slot() {
12457        gemachain_logger::setup();
12458        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12459
12460        let slot1 = 11;
12461        let slot2 = 222;
12462        let slot3 = 3333;
12463
12464        let pubkey1 = Pubkey::new_unique();
12465        let pubkey2 = Pubkey::new_unique();
12466        let pubkey3 = Pubkey::new_unique();
12467
12468        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
12469        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
12470        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
12471
12472        let mut uncleaned_slots1 = db.collect_uncleaned_slots_up_to_slot(slot1);
12473        let mut uncleaned_slots2 = db.collect_uncleaned_slots_up_to_slot(slot2);
12474        let mut uncleaned_slots3 = db.collect_uncleaned_slots_up_to_slot(slot3);
12475
12476        uncleaned_slots1.sort_unstable();
12477        uncleaned_slots2.sort_unstable();
12478        uncleaned_slots3.sort_unstable();
12479
12480        assert_eq!(uncleaned_slots1, [slot1]);
12481        assert_eq!(uncleaned_slots2, [slot1, slot2]);
12482        assert_eq!(uncleaned_slots3, [slot1, slot2, slot3]);
12483    }
12484
12485    #[test]
12486    fn test_remove_uncleaned_slots_and_collect_pubkeys() {
12487        gemachain_logger::setup();
12488        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12489
12490        let slot1 = 11;
12491        let slot2 = 222;
12492        let slot3 = 3333;
12493
12494        let pubkey1 = Pubkey::new_unique();
12495        let pubkey2 = Pubkey::new_unique();
12496        let pubkey3 = Pubkey::new_unique();
12497
12498        let account1 = AccountSharedData::new(0, 0, &pubkey1);
12499        let account2 = AccountSharedData::new(0, 0, &pubkey2);
12500        let account3 = AccountSharedData::new(0, 0, &pubkey3);
12501
12502        db.store_uncached(slot1, &[(&pubkey1, &account1)]);
12503        db.store_uncached(slot2, &[(&pubkey2, &account2)]);
12504        db.store_uncached(slot3, &[(&pubkey3, &account3)]);
12505
12506        db.add_root(slot1);
12507        // slot 2 is _not_ a root on purpose
12508        db.add_root(slot3);
12509
12510        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
12511        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
12512        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
12513
12514        let uncleaned_pubkeys1 = db
12515            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot1])
12516            .into_iter()
12517            .flatten()
12518            .collect::<Vec<_>>();
12519        let uncleaned_pubkeys2 = db
12520            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot2])
12521            .into_iter()
12522            .flatten()
12523            .collect::<Vec<_>>();
12524        let uncleaned_pubkeys3 = db
12525            .remove_uncleaned_slots_and_collect_pubkeys(vec![slot3])
12526            .into_iter()
12527            .flatten()
12528            .collect::<Vec<_>>();
12529
12530        assert!(uncleaned_pubkeys1.contains(&pubkey1));
12531        assert!(!uncleaned_pubkeys1.contains(&pubkey2));
12532        assert!(!uncleaned_pubkeys1.contains(&pubkey3));
12533
12534        assert!(!uncleaned_pubkeys2.contains(&pubkey1));
12535        assert!(uncleaned_pubkeys2.contains(&pubkey2));
12536        assert!(!uncleaned_pubkeys2.contains(&pubkey3));
12537
12538        assert!(!uncleaned_pubkeys3.contains(&pubkey1));
12539        assert!(!uncleaned_pubkeys3.contains(&pubkey2));
12540        assert!(uncleaned_pubkeys3.contains(&pubkey3));
12541    }
12542
12543    #[test]
12544    fn test_remove_uncleaned_slots_and_collect_pubkeys_up_to_slot() {
12545        gemachain_logger::setup();
12546        let db = AccountsDb::new(Vec::new(), &ClusterType::Development);
12547
12548        let slot1 = 11;
12549        let slot2 = 222;
12550        let slot3 = 3333;
12551
12552        let pubkey1 = Pubkey::new_unique();
12553        let pubkey2 = Pubkey::new_unique();
12554        let pubkey3 = Pubkey::new_unique();
12555
12556        let account1 = AccountSharedData::new(0, 0, &pubkey1);
12557        let account2 = AccountSharedData::new(0, 0, &pubkey2);
12558        let account3 = AccountSharedData::new(0, 0, &pubkey3);
12559
12560        db.store_uncached(slot1, &[(&pubkey1, &account1)]);
12561        db.store_uncached(slot2, &[(&pubkey2, &account2)]);
12562        db.store_uncached(slot3, &[(&pubkey3, &account3)]);
12563
12564        // slot 1 is _not_ a root on purpose
12565        db.add_root(slot2);
12566        db.add_root(slot3);
12567
12568        db.uncleaned_pubkeys.insert(slot1, vec![pubkey1]);
12569        db.uncleaned_pubkeys.insert(slot2, vec![pubkey2]);
12570        db.uncleaned_pubkeys.insert(slot3, vec![pubkey3]);
12571
12572        let uncleaned_pubkeys = db
12573            .remove_uncleaned_slots_and_collect_pubkeys_up_to_slot(slot3)
12574            .into_iter()
12575            .flatten()
12576            .collect::<Vec<_>>();
12577
12578        assert!(uncleaned_pubkeys.contains(&pubkey1));
12579        assert!(uncleaned_pubkeys.contains(&pubkey2));
12580        assert!(uncleaned_pubkeys.contains(&pubkey3));
12581    }
12582
12583    #[test]
12584    fn test_shrink_productive() {
12585        gemachain_logger::setup();
12586        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, 1024);
12587        let stores = vec![Arc::new(s1)];
12588        assert!(!AccountsDb::is_shrinking_productive(0, &stores));
12589
12590        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, PAGE_SIZE * 4);
12591        let stores = vec![Arc::new(s1)];
12592        stores[0].add_account((3 * PAGE_SIZE as usize) - 1);
12593        stores[0].add_account(10);
12594        stores[0].remove_account(10, false);
12595        assert!(AccountsDb::is_shrinking_productive(0, &stores));
12596
12597        stores[0].add_account(PAGE_SIZE as usize);
12598        assert!(!AccountsDb::is_shrinking_productive(0, &stores));
12599
12600        let s1 = AccountStorageEntry::new(Path::new("."), 0, 0, PAGE_SIZE + 1);
12601        s1.add_account(PAGE_SIZE as usize);
12602        let s2 = AccountStorageEntry::new(Path::new("."), 0, 1, PAGE_SIZE + 1);
12603        s2.add_account(PAGE_SIZE as usize);
12604        let stores = vec![Arc::new(s1), Arc::new(s2)];
12605        assert!(AccountsDb::is_shrinking_productive(0, &stores));
12606    }
12607
12608    #[test]
12609    fn test_is_candidate_for_shrink() {
12610        gemachain_logger::setup();
12611
12612        let mut accounts = AccountsDb::new_single_for_tests();
12613        let common_store_path = Path::new("");
12614        let store_file_size = 2 * PAGE_SIZE;
12615        let entry = Arc::new(AccountStorageEntry::new(
12616            common_store_path,
12617            0,
12618            1,
12619            store_file_size,
12620        ));
12621        match accounts.shrink_ratio {
12622            AccountShrinkThreshold::TotalSpace { shrink_ratio } => {
12623                assert_eq!(
12624                    (DEFAULT_ACCOUNTS_SHRINK_RATIO * 100.) as u64,
12625                    (shrink_ratio * 100.) as u64
12626                )
12627            }
12628            AccountShrinkThreshold::IndividalStore { shrink_ratio: _ } => {
12629                panic!("Expect the default to be TotalSpace")
12630            }
12631        }
12632        entry.alive_bytes.store(3000, Ordering::Relaxed);
12633        assert!(accounts.is_candidate_for_shrink(&entry));
12634        entry.alive_bytes.store(5000, Ordering::Relaxed);
12635        assert!(!accounts.is_candidate_for_shrink(&entry));
12636        accounts.shrink_ratio = AccountShrinkThreshold::TotalSpace { shrink_ratio: 0.3 };
12637        entry.alive_bytes.store(3000, Ordering::Relaxed);
12638        assert!(accounts.is_candidate_for_shrink(&entry));
12639        accounts.shrink_ratio = AccountShrinkThreshold::IndividalStore { shrink_ratio: 0.3 };
12640        assert!(!accounts.is_candidate_for_shrink(&entry));
12641    }
12642
12643    #[test]
12644    fn test_calculate_storage_count_and_alive_bytes() {
12645        let accounts = AccountsDb::new_single_for_tests();
12646        let shared_key = gemachain_sdk::pubkey::new_rand();
12647        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
12648        let slot0 = 0;
12649        accounts.store_uncached(slot0, &[(&shared_key, &account)]);
12650
12651        let storage_maps = accounts
12652            .storage
12653            .get_slot_storage_entries(slot0)
12654            .unwrap_or_default();
12655        let storage_info = StorageSizeAndCountMap::default();
12656        let accounts_map = AccountsDb::process_storage_slot(&storage_maps[..]);
12657        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
12658        assert_eq!(storage_info.len(), 1);
12659        for entry in storage_info.iter() {
12660            assert_eq!(
12661                (entry.key(), entry.value().count, entry.value().stored_size),
12662                (&0, 1, 144)
12663            );
12664        }
12665    }
12666
12667    #[test]
12668    fn test_calculate_storage_count_and_alive_bytes_0_accounts() {
12669        let storage_maps = vec![];
12670        let storage_info = StorageSizeAndCountMap::default();
12671        let accounts_map = AccountsDb::process_storage_slot(&storage_maps[..]);
12672        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
12673        assert!(storage_info.is_empty());
12674    }
12675
12676    #[test]
12677    fn test_calculate_storage_count_and_alive_bytes_2_accounts() {
12678        let accounts = AccountsDb::new_single_for_tests();
12679        let keys = [
12680            gemachain_sdk::pubkey::Pubkey::new(&[0; 32]),
12681            gemachain_sdk::pubkey::Pubkey::new(&[255; 32]),
12682        ];
12683        // make sure accounts are in 2 different bins
12684        assert!(
12685            (accounts.accounts_index.bins() == 1)
12686                ^ (accounts
12687                    .accounts_index
12688                    .bin_calculator
12689                    .bin_from_pubkey(&keys[0])
12690                    != accounts
12691                        .accounts_index
12692                        .bin_calculator
12693                        .bin_from_pubkey(&keys[1]))
12694        );
12695        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
12696        let account_big = AccountSharedData::new(1, 1000, AccountSharedData::default().owner());
12697        let slot0 = 0;
12698        accounts.store_uncached(slot0, &[(&keys[0], &account)]);
12699        accounts.store_uncached(slot0, &[(&keys[1], &account_big)]);
12700
12701        let storage_maps = accounts
12702            .storage
12703            .get_slot_storage_entries(slot0)
12704            .unwrap_or_default();
12705        let storage_info = StorageSizeAndCountMap::default();
12706        let accounts_map = AccountsDb::process_storage_slot(&storage_maps[..]);
12707        AccountsDb::update_storage_info(&storage_info, &accounts_map, &Mutex::default());
12708        assert_eq!(storage_info.len(), 1);
12709        for entry in storage_info.iter() {
12710            assert_eq!(
12711                (entry.key(), entry.value().count, entry.value().stored_size),
12712                (&0, 2, 1280)
12713            );
12714        }
12715    }
12716
12717    #[test]
12718    fn test_set_storage_count_and_alive_bytes() {
12719        let accounts = AccountsDb::new_single_for_tests();
12720
12721        // make sure we have storage 0
12722        let shared_key = gemachain_sdk::pubkey::new_rand();
12723        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
12724        let slot0 = 0;
12725        accounts.store_uncached(slot0, &[(&shared_key, &account)]);
12726
12727        // fake out the store count to avoid the assert
12728        for slot_stores in accounts.storage.0.iter() {
12729            for (_id, store) in slot_stores.value().read().unwrap().iter() {
12730                store.alive_bytes.store(0, Ordering::SeqCst);
12731            }
12732        }
12733
12734        // populate based on made up hash data
12735        let dashmap = DashMap::default();
12736        dashmap.insert(
12737            0,
12738            StorageSizeAndCount {
12739                stored_size: 2,
12740                count: 3,
12741            },
12742        );
12743        accounts.set_storage_count_and_alive_bytes(dashmap, &mut GenerateIndexTimings::default());
12744        assert_eq!(accounts.storage.0.len(), 1);
12745        for slot_stores in accounts.storage.0.iter() {
12746            for (id, store) in slot_stores.value().read().unwrap().iter() {
12747                assert_eq!(id, &0);
12748                assert_eq!(store.count_and_status.read().unwrap().0, 3);
12749                assert_eq!(store.alive_bytes.load(Ordering::SeqCst), 2);
12750            }
12751        }
12752    }
12753
12754    #[test]
12755    fn test_purge_alive_unrooted_slots_after_clean() {
12756        let accounts = AccountsDb::new_single_for_tests();
12757
12758        // Key shared between rooted and nonrooted slot
12759        let shared_key = gemachain_sdk::pubkey::new_rand();
12760        // Key to keep the storage entry for the unrooted slot alive
12761        let unrooted_key = gemachain_sdk::pubkey::new_rand();
12762        let slot0 = 0;
12763        let slot1 = 1;
12764
12765        // Store accounts with greater than 0 carats
12766        let account = AccountSharedData::new(1, 1, AccountSharedData::default().owner());
12767        accounts.store_uncached(slot0, &[(&shared_key, &account)]);
12768        accounts.store_uncached(slot0, &[(&unrooted_key, &account)]);
12769
12770        // Simulate adding dirty pubkeys on bank freeze. Note this is
12771        // not a rooted slot
12772        accounts.get_accounts_delta_hash(slot0);
12773
12774        // On the next *rooted* slot, update the `shared_key` account to zero carats
12775        let zero_carat_account =
12776            AccountSharedData::new(0, 0, AccountSharedData::default().owner());
12777        accounts.store_uncached(slot1, &[(&shared_key, &zero_carat_account)]);
12778
12779        // Simulate adding dirty pubkeys on bank freeze, set root
12780        accounts.get_accounts_delta_hash(slot1);
12781        accounts.add_root(slot1);
12782
12783        // The later rooted zero-carat update to `shared_key` cannot be cleaned
12784        // because it is kept alive by the unrooted slot.
12785        accounts.clean_accounts(None, false, None);
12786        assert!(accounts
12787            .accounts_index
12788            .get_account_read_entry(&shared_key)
12789            .is_some());
12790
12791        // Simulate purge_slot() all from AccountsBackgroundService
12792        let is_from_abs = true;
12793        accounts.purge_slot(slot0, 0, is_from_abs);
12794
12795        // Now clean should clean up the remaining key
12796        accounts.clean_accounts(None, false, None);
12797        assert!(accounts
12798            .accounts_index
12799            .get_account_read_entry(&shared_key)
12800            .is_none());
12801        assert!(accounts.storage.get_slot_storage_entries(slot0).is_none());
12802    }
12803
12804    /// Test to make sure `clean_accounts()` works properly with the `last_full_snapshot_slot`
12805    /// parameter.  Basically:
12806    ///
12807    /// - slot 1: set Account1's balance to non-zero
12808    /// - slot 2: set Account1's balance to a different non-zero amount
12809    /// - slot 3: set Account1's balance to zero
12810    /// - call `clean_accounts()` with `max_clean_root` set to 2
12811    ///     - ensure Account1 has *not* been purged
12812    ///     - ensure the store from slot 1 is cleaned up
12813    /// - call `clean_accounts()` with `last_full_snapshot_slot` set to 2
12814    ///     - ensure Account1 has *not* been purged
12815    /// - call `clean_accounts()` with `last_full_snapshot_slot` set to 3
12816    ///     - ensure Account1 *has* been purged
12817    #[test]
12818    fn test_clean_accounts_with_last_full_snapshot_slot() {
12819        gemachain_logger::setup();
12820        let accounts_db = AccountsDb::new_single_for_tests();
12821        let pubkey = gemachain_sdk::pubkey::new_rand();
12822        let owner = gemachain_sdk::pubkey::new_rand();
12823        let space = 0;
12824
12825        let slot1 = 1;
12826        let account = AccountSharedData::new(111, space, &owner);
12827        accounts_db.store_cached(slot1, &[(&pubkey, &account)]);
12828        accounts_db.get_accounts_delta_hash(slot1);
12829        accounts_db.add_root(slot1);
12830
12831        let slot2 = 2;
12832        let account = AccountSharedData::new(222, space, &owner);
12833        accounts_db.store_cached(slot2, &[(&pubkey, &account)]);
12834        accounts_db.get_accounts_delta_hash(slot2);
12835        accounts_db.add_root(slot2);
12836
12837        let slot3 = 3;
12838        let account = AccountSharedData::new(0, space, &owner);
12839        accounts_db.store_cached(slot3, &[(&pubkey, &account)]);
12840        accounts_db.get_accounts_delta_hash(slot3);
12841        accounts_db.add_root(slot3);
12842
12843        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 3);
12844
12845        accounts_db.clean_accounts(Some(slot2), false, Some(slot2));
12846        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 2);
12847
12848        accounts_db.clean_accounts(None, false, Some(slot2));
12849        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 1);
12850
12851        accounts_db.clean_accounts(None, false, Some(slot3));
12852        assert_eq!(accounts_db.ref_count_for_pubkey(&pubkey), 0);
12853    }
12854
12855    #[test]
12856    fn test_filter_zero_carat_clean_for_incremental_snapshots() {
12857        gemachain_logger::setup();
12858        let slot = 10;
12859
12860        struct TestParameters {
12861            last_full_snapshot_slot: Option<Slot>,
12862            max_clean_root: Option<Slot>,
12863            should_contain: bool,
12864        }
12865
12866        let do_test = |test_params: TestParameters| {
12867            let account_info = AccountInfo {
12868                store_id: 42,
12869                offset: 123,
12870                stored_size: 234,
12871                carats: 0,
12872            };
12873            let pubkey = gemachain_sdk::pubkey::new_rand();
12874            let mut key_set = HashSet::default();
12875            key_set.insert(pubkey);
12876            let store_count = 0;
12877            let mut store_counts = HashMap::default();
12878            store_counts.insert(account_info.store_id, (store_count, key_set));
12879            let mut purges_zero_carats = HashMap::default();
12880            purges_zero_carats.insert(pubkey, (vec![(slot, account_info)], 1));
12881
12882            let accounts_db = AccountsDb::new_single_for_tests();
12883            accounts_db.filter_zero_carat_clean_for_incremental_snapshots(
12884                test_params.max_clean_root,
12885                test_params.last_full_snapshot_slot,
12886                &store_counts,
12887                &mut purges_zero_carats,
12888            );
12889
12890            assert_eq!(
12891                purges_zero_carats.contains_key(&pubkey),
12892                test_params.should_contain
12893            );
12894        };
12895
12896        // Scenario 1: last full snapshot is NONE
12897        // In this scenario incremental snapshots are OFF, so always purge
12898        {
12899            let last_full_snapshot_slot = None;
12900
12901            do_test(TestParameters {
12902                last_full_snapshot_slot,
12903                max_clean_root: Some(slot),
12904                should_contain: true,
12905            });
12906
12907            do_test(TestParameters {
12908                last_full_snapshot_slot,
12909                max_clean_root: None,
12910                should_contain: true,
12911            });
12912        }
12913
12914        // Scenario 2: last full snapshot is GREATER THAN zero carat account slot
12915        // In this scenario always purge, and just test the various permutations of
12916        // `should_filter_for_incremental_snapshots` based on `max_clean_root`.
12917        {
12918            let last_full_snapshot_slot = Some(slot + 1);
12919
12920            do_test(TestParameters {
12921                last_full_snapshot_slot,
12922                max_clean_root: last_full_snapshot_slot,
12923                should_contain: true,
12924            });
12925
12926            do_test(TestParameters {
12927                last_full_snapshot_slot,
12928                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
12929                should_contain: true,
12930            });
12931
12932            do_test(TestParameters {
12933                last_full_snapshot_slot,
12934                max_clean_root: None,
12935                should_contain: true,
12936            });
12937        }
12938
12939        // Scenario 3: last full snapshot is EQUAL TO zero carat account slot
12940        // In this scenario always purge, as it's the same as Scenario 2.
12941        {
12942            let last_full_snapshot_slot = Some(slot);
12943
12944            do_test(TestParameters {
12945                last_full_snapshot_slot,
12946                max_clean_root: last_full_snapshot_slot,
12947                should_contain: true,
12948            });
12949
12950            do_test(TestParameters {
12951                last_full_snapshot_slot,
12952                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
12953                should_contain: true,
12954            });
12955
12956            do_test(TestParameters {
12957                last_full_snapshot_slot,
12958                max_clean_root: None,
12959                should_contain: true,
12960            });
12961        }
12962
12963        // Scenario 4: last full snapshot is LESS THAN zero carat account slot
12964        // In this scenario do *not* purge, except when `should_filter_for_incremental_snapshots`
12965        // is false
12966        {
12967            let last_full_snapshot_slot = Some(slot - 1);
12968
12969            do_test(TestParameters {
12970                last_full_snapshot_slot,
12971                max_clean_root: last_full_snapshot_slot,
12972                should_contain: true,
12973            });
12974
12975            do_test(TestParameters {
12976                last_full_snapshot_slot,
12977                max_clean_root: last_full_snapshot_slot.map(|s| s + 1),
12978                should_contain: false,
12979            });
12980
12981            do_test(TestParameters {
12982                last_full_snapshot_slot,
12983                max_clean_root: None,
12984                should_contain: false,
12985            });
12986        }
12987    }
12988}