solana_accounts_db/
accounts_index.rs

1mod account_map_entry;
2mod accounts_index_storage;
3mod bucket_map_holder;
4pub(crate) mod in_mem_accounts_index;
5mod iter;
6mod roots_tracker;
7mod secondary;
8mod stats;
9use {
10    crate::{
11        ancestors::Ancestors, contains::Contains, is_zero_lamport::IsZeroLamport,
12        pubkey_bins::PubkeyBinCalculator24, rolling_bit_field::RollingBitField,
13    },
14    account_map_entry::{AccountMapEntry, PreAllocatedAccountMapEntry, SlotListWriteGuard},
15    accounts_index_storage::AccountsIndexStorage,
16    bucket_map_holder::Age,
17    in_mem_accounts_index::{
18        ExistedLocation, InMemAccountsIndex, InsertNewEntryResults, StartupStats,
19    },
20    iter::{AccountsIndexPubkeyIterOrder, AccountsIndexPubkeyIterator},
21    log::*,
22    rand::{thread_rng, Rng},
23    rayon::iter::{IntoParallelIterator, ParallelIterator},
24    roots_tracker::RootsTracker,
25    secondary::{RwLockSecondaryIndexEntry, SecondaryIndex, SecondaryIndexEntry},
26    smallvec::SmallVec,
27    solana_account::ReadableAccount,
28    solana_clock::{BankId, Slot},
29    solana_measure::measure::Measure,
30    solana_pubkey::Pubkey,
31    stats::Stats,
32    std::{
33        collections::{btree_map::BTreeMap, HashSet},
34        fmt::Debug,
35        num::NonZeroUsize,
36        ops::{Bound, Range, RangeBounds},
37        path::PathBuf,
38        sync::{
39            atomic::{AtomicBool, AtomicU32, AtomicU64, AtomicUsize, Ordering},
40            Arc, Mutex, RwLock,
41        },
42    },
43    thiserror::Error,
44};
45pub use {
46    iter::ITER_BATCH_SIZE,
47    secondary::{
48        AccountIndex, AccountSecondaryIndexes, AccountSecondaryIndexesIncludeExclude, IndexKey,
49    },
50};
51
52pub const BINS_DEFAULT: usize = 8192;
53pub const BINS_FOR_TESTING: usize = 2; // we want > 1, but each bin is a few disk files with a disk based index, so fewer is better
54pub const BINS_FOR_BENCHMARKS: usize = 8192;
55// The unsafe is safe because we're using a fixed, known non-zero value
56pub const FLUSH_THREADS_TESTING: NonZeroUsize = NonZeroUsize::new(1).unwrap();
57pub const ACCOUNTS_INDEX_CONFIG_FOR_TESTING: AccountsIndexConfig = AccountsIndexConfig {
58    bins: Some(BINS_FOR_TESTING),
59    num_flush_threads: Some(FLUSH_THREADS_TESTING),
60    drives: None,
61    index_limit_mb: IndexLimitMb::InMemOnly,
62    ages_to_stay_in_cache: None,
63    scan_results_limit_bytes: None,
64    num_initial_accounts: None,
65};
66pub const ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS: AccountsIndexConfig = AccountsIndexConfig {
67    bins: Some(BINS_FOR_BENCHMARKS),
68    num_flush_threads: Some(FLUSH_THREADS_TESTING),
69    drives: None,
70    index_limit_mb: IndexLimitMb::InMemOnly,
71    ages_to_stay_in_cache: None,
72    scan_results_limit_bytes: None,
73    num_initial_accounts: None,
74};
75pub type ScanResult<T> = Result<T, ScanError>;
76pub type SlotList<T> = SmallVec<[(Slot, T); 1]>;
77pub type ReclaimsSlotList<T> = Vec<(Slot, T)>;
78
79// The ref count cannot be higher than the total number of storages, and we should never have more
80// than 1 million storages. A 32-bit ref count should be *significantly* more than enough.
81// (We already effectively limit the number of storages to 2^32 since the storage ID type is a u32.)
82// The majority of accounts should only exist in one storage, so the most common ref count is '1'.
83// Heavily updated accounts should still have a ref count that is < 100.
84pub type RefCount = u32;
85pub type AtomicRefCount = AtomicU32;
86
87/// values returned from `insert_new_if_missing_into_primary_index()`
88#[derive(Default, Debug, PartialEq, Eq)]
89pub(crate) struct InsertNewIfMissingIntoPrimaryIndexInfo {
90    /// number of accounts inserted in the index
91    pub count: usize,
92    /// Number of accounts added to the index that didn't already exist in the index
93    pub num_did_not_exist: u64,
94    /// Number of accounts added to the index that already existed, and were in-mem
95    pub num_existed_in_mem: u64,
96    /// Number of accounts added to the index that already existed, and were on-disk
97    pub num_existed_on_disk: u64,
98}
99
100#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
101/// which accounts `scan` should load from disk
102pub enum ScanFilter {
103    /// Scan both in-memory and on-disk index
104    #[default]
105    All,
106
107    /// abnormal = ref_count != 1 or slot list.len() != 1
108    /// Scan only in-memory index and skip on-disk index
109    OnlyAbnormal,
110
111    /// Similar to `OnlyAbnormal but also check on-disk index to verify the
112    /// entry on-disk is indeed normal.
113    OnlyAbnormalWithVerify,
114
115    /// Similar to `OnlyAbnormal but mark entries in memory as not found
116    /// if they are normal
117    /// This removes the possibility of any race conditions with index
118    /// flushing and simulates the system running an uncached disk index
119    /// where nothing 'normal' is ever held in the in memory index as far as
120    /// callers are concerned. This could also be a  correct/ideal future api
121    /// to similarly provide consistency and remove race condition behavior.
122    OnlyAbnormalTest,
123}
124
125#[derive(Debug, Clone, Copy, PartialEq, Eq)]
126/// how accounts index 'upsert' should handle reclaims
127pub enum UpsertReclaim {
128    /// previous entry for this slot in the index is expected to be cached, so irrelevant to reclaims
129    PreviousSlotEntryWasCached,
130    /// previous entry for this slot in the index may need to be reclaimed, so return it.
131    /// reclaims is the only output of upsert, requiring a synchronous execution
132    PopulateReclaims,
133    /// overwrite existing data in the same slot and do not return in 'reclaims'
134    IgnoreReclaims,
135    // Reclaim all older versions of the account from the index and return
136    // in the 'reclaims'
137    ReclaimOldSlots,
138}
139
140#[derive(Debug)]
141pub struct ScanConfig {
142    /// checked by the scan. When true, abort scan.
143    pub abort: Option<Arc<AtomicBool>>,
144
145    /// In what order should items be scanned?
146    pub scan_order: ScanOrder,
147}
148
149impl Default for ScanConfig {
150    fn default() -> Self {
151        Self {
152            abort: None,
153            scan_order: ScanOrder::Unsorted,
154        }
155    }
156}
157
158impl ScanConfig {
159    pub fn new(scan_order: ScanOrder) -> Self {
160        Self {
161            scan_order,
162            ..Default::default()
163        }
164    }
165
166    /// mark the scan as aborted
167    pub fn abort(&self) {
168        if let Some(abort) = self.abort.as_ref() {
169            abort.store(true, Ordering::Relaxed)
170        }
171    }
172
173    /// use existing 'abort' if available, otherwise allocate one
174    pub fn recreate_with_abort(&self) -> Self {
175        ScanConfig {
176            abort: Some(self.abort.clone().unwrap_or_default()),
177            scan_order: self.scan_order,
178        }
179    }
180
181    /// true if scan should abort
182    pub fn is_aborted(&self) -> bool {
183        if let Some(abort) = self.abort.as_ref() {
184            abort.load(Ordering::Relaxed)
185        } else {
186            false
187        }
188    }
189}
190
191/// In what order should items be scanned?
192///
193/// Users should prefer `Unsorted`, unless required otherwise,
194/// as sorting incurs additional runtime cost.
195#[derive(Debug, Copy, Clone, Eq, PartialEq)]
196pub enum ScanOrder {
197    /// Scan items in any order
198    Unsorted,
199    /// Scan items in sorted order
200    Sorted,
201}
202
203pub trait IsCached {
204    fn is_cached(&self) -> bool;
205}
206
207pub trait IndexValue: 'static + IsCached + IsZeroLamport + DiskIndexValue {}
208
209pub trait DiskIndexValue:
210    'static + Clone + Debug + PartialEq + Copy + Default + Sync + Send
211{
212}
213
214#[derive(Error, Debug, PartialEq, Eq)]
215pub enum ScanError {
216    #[error(
217        "Node detected it replayed bad version of slot {slot:?} with id {bank_id:?}, thus the \
218         scan on said slot was aborted"
219    )]
220    SlotRemoved { slot: Slot, bank_id: BankId },
221    #[error("scan aborted: {0}")]
222    Aborted(String),
223}
224
225enum ScanTypes<R: RangeBounds<Pubkey>> {
226    Unindexed(Option<R>),
227    Indexed(IndexKey),
228}
229
230/// specification of how much memory in-mem portion of account index can use
231#[derive(Debug, Copy, Clone)]
232pub enum IndexLimitMb {
233    /// use disk index while keeping a minimal amount in-mem
234    Minimal,
235    /// in-mem-only was specified, no disk index
236    InMemOnly,
237}
238
239#[derive(Debug, Clone)]
240pub struct AccountsIndexConfig {
241    pub bins: Option<usize>,
242    pub num_flush_threads: Option<NonZeroUsize>,
243    pub drives: Option<Vec<PathBuf>>,
244    pub index_limit_mb: IndexLimitMb,
245    pub ages_to_stay_in_cache: Option<Age>,
246    pub scan_results_limit_bytes: Option<usize>,
247    /// Initial number of accounts, used to pre-allocate HashMap capacity at startup.
248    pub num_initial_accounts: Option<usize>,
249}
250
251impl Default for AccountsIndexConfig {
252    fn default() -> Self {
253        Self {
254            bins: None,
255            num_flush_threads: None,
256            drives: None,
257            index_limit_mb: IndexLimitMb::InMemOnly,
258            ages_to_stay_in_cache: None,
259            scan_results_limit_bytes: None,
260            num_initial_accounts: None,
261        }
262    }
263}
264
265pub fn default_num_flush_threads() -> NonZeroUsize {
266    NonZeroUsize::new(std::cmp::max(2, num_cpus::get() / 4)).expect("non-zero system threads")
267}
268
269#[derive(Debug, Default)]
270pub struct AccountsIndexRootsStats {
271    pub roots_len: Option<usize>,
272    pub uncleaned_roots_len: Option<usize>,
273    pub roots_range: Option<u64>,
274    pub rooted_cleaned_count: usize,
275    pub unrooted_cleaned_count: usize,
276    pub clean_unref_from_storage_us: u64,
277    pub clean_dead_slot_us: u64,
278}
279
280#[derive(Copy, Clone)]
281pub enum AccountsIndexScanResult {
282    /// if the entry is not in the in-memory index, do not add it unless the entry becomes dirty
283    OnlyKeepInMemoryIfDirty,
284    /// keep the entry in the in-memory index
285    KeepInMemory,
286    /// reduce refcount by 1
287    Unref,
288    /// reduce refcount by 1 and assert that ref_count = 0 after unref
289    UnrefAssert0,
290    /// reduce refcount by 1 and log if ref_count != 0 after unref
291    UnrefLog0,
292}
293
294#[derive(Debug)]
295/// T: account info type to interact in in-memory items
296/// U: account info type to be persisted to disk
297pub struct AccountsIndex<T: IndexValue, U: DiskIndexValue + From<T> + Into<T>> {
298    pub account_maps: Box<[Arc<InMemAccountsIndex<T, U>>]>,
299    pub bin_calculator: PubkeyBinCalculator24,
300    program_id_index: SecondaryIndex<RwLockSecondaryIndexEntry>,
301    spl_token_mint_index: SecondaryIndex<RwLockSecondaryIndexEntry>,
302    spl_token_owner_index: SecondaryIndex<RwLockSecondaryIndexEntry>,
303    pub roots_tracker: RwLock<RootsTracker>,
304    ongoing_scan_roots: RwLock<BTreeMap<Slot, u64>>,
305    // Each scan has some latest slot `S` that is the tip of the fork the scan
306    // is iterating over. The unique id of that slot `S` is recorded here (note we don't use
307    // `S` as the id because there can be more than one version of a slot `S`). If a fork
308    // is abandoned, all of the slots on that fork up to `S` will be removed via
309    // `AccountsDb::remove_unrooted_slots()`. When the scan finishes, it'll realize that the
310    // results of the scan may have been corrupted by `remove_unrooted_slots` and abort its results.
311    //
312    // `removed_bank_ids` tracks all the slot ids that were removed via `remove_unrooted_slots()` so any attempted scans
313    // on any of these slots fails. This is safe to purge once the associated Bank is dropped and
314    // scanning the fork with that Bank at the tip is no longer possible.
315    pub removed_bank_ids: Mutex<HashSet<BankId>>,
316
317    storage: AccountsIndexStorage<T, U>,
318
319    /// when a scan's accumulated data exceeds this limit, abort the scan
320    pub scan_results_limit_bytes: Option<usize>,
321
322    pub purge_older_root_entries_one_slot_list: AtomicUsize,
323
324    /// # roots added since last check
325    pub roots_added: AtomicUsize,
326    /// # roots removed since last check
327    pub roots_removed: AtomicUsize,
328    /// # scans active currently
329    pub active_scans: AtomicUsize,
330    /// # of slots between latest max and latest scan
331    pub max_distance_to_min_scan_slot: AtomicU64,
332}
333
334impl<T: IndexValue, U: DiskIndexValue + From<T> + Into<T>> AccountsIndex<T, U> {
335    pub fn default_for_tests() -> Self {
336        Self::new(&ACCOUNTS_INDEX_CONFIG_FOR_TESTING, Arc::default())
337    }
338
339    pub fn new(config: &AccountsIndexConfig, exit: Arc<AtomicBool>) -> Self {
340        let scan_results_limit_bytes = config.scan_results_limit_bytes;
341        let (account_maps, bin_calculator, storage) = Self::allocate_accounts_index(config, exit);
342        Self {
343            purge_older_root_entries_one_slot_list: AtomicUsize::default(),
344            account_maps,
345            bin_calculator,
346            program_id_index: SecondaryIndex::<RwLockSecondaryIndexEntry>::new(
347                "program_id_index_stats",
348            ),
349            spl_token_mint_index: SecondaryIndex::<RwLockSecondaryIndexEntry>::new(
350                "spl_token_mint_index_stats",
351            ),
352            spl_token_owner_index: SecondaryIndex::<RwLockSecondaryIndexEntry>::new(
353                "spl_token_owner_index_stats",
354            ),
355            roots_tracker: RwLock::<RootsTracker>::default(),
356            ongoing_scan_roots: RwLock::<BTreeMap<Slot, u64>>::default(),
357            removed_bank_ids: Mutex::<HashSet<BankId>>::default(),
358            storage,
359            scan_results_limit_bytes,
360            roots_added: AtomicUsize::default(),
361            roots_removed: AtomicUsize::default(),
362            active_scans: AtomicUsize::default(),
363            max_distance_to_min_scan_slot: AtomicU64::default(),
364        }
365    }
366
367    /// return the bin index for a given pubkey
368    fn bin_from_pubkey(&self, pubkey: &Pubkey) -> usize {
369        self.bin_calculator.bin_from_pubkey(pubkey)
370    }
371
372    /// returns the start bin and the end bin (inclusive) to scan.
373    ///
374    /// Note that start_bin maybe larger than highest bin index. Therefore, the
375    /// caller should not assume that start_bin is a valid bin index. So don't
376    /// index into `account_maps` with start_bin. Use `start_bin..=end_bin` to
377    /// iterate over the bins.
378    fn bin_start_end_inclusive<R>(&self, range: &R) -> (usize, usize)
379    where
380        R: RangeBounds<Pubkey>,
381    {
382        let start_bin = match range.start_bound() {
383            Bound::Included(start) => self.bin_from_pubkey(start),
384            Bound::Excluded(start) => {
385                // check if start == self.account_maps[start_bin].highest_pubkey(), then
386                // we should return start_bin + 1
387                let start_bin = self.bin_from_pubkey(start);
388                if start == &self.account_maps[start_bin].highest_pubkey {
389                    start_bin + 1
390                } else {
391                    start_bin
392                }
393            }
394            Bound::Unbounded => 0,
395        };
396
397        let end_bin_inclusive = match range.end_bound() {
398            Bound::Included(end) => self.bin_from_pubkey(end),
399            Bound::Excluded(end) => {
400                // check if end == self.account_maps[end_bin].lowest_pubkey(), then
401                // we should return end_bin - 1
402                let end_bin = self.bin_from_pubkey(end);
403                if end == &self.account_maps[end_bin].lowest_pubkey {
404                    end_bin.saturating_sub(1)
405                } else {
406                    end_bin
407                }
408            }
409            Bound::Unbounded => self.account_maps.len().saturating_sub(1),
410        };
411
412        (start_bin, end_bin_inclusive)
413    }
414
415    #[allow(clippy::type_complexity)]
416    fn allocate_accounts_index(
417        config: &AccountsIndexConfig,
418        exit: Arc<AtomicBool>,
419    ) -> (
420        Box<[Arc<InMemAccountsIndex<T, U>>]>,
421        PubkeyBinCalculator24,
422        AccountsIndexStorage<T, U>,
423    ) {
424        let bins = config.bins.unwrap_or(BINS_DEFAULT);
425        // create bin_calculator early to verify # bins is reasonable
426        let bin_calculator = PubkeyBinCalculator24::new(bins);
427        let storage = AccountsIndexStorage::new(bins, config, exit);
428
429        let account_maps: Box<_> = (0..bins)
430            .map(|bin| Arc::clone(&storage.in_mem[bin]))
431            .collect();
432        (account_maps, bin_calculator, storage)
433    }
434
435    fn iter<'a, R>(
436        &'a self,
437        range: Option<&'a R>,
438        iter_order: AccountsIndexPubkeyIterOrder,
439    ) -> AccountsIndexPubkeyIterator<'a, T, U>
440    where
441        R: RangeBounds<Pubkey>,
442    {
443        AccountsIndexPubkeyIterator::new(self, range, iter_order)
444    }
445
446    /// is the accounts index using disk as a backing store
447    pub fn is_disk_index_enabled(&self) -> bool {
448        self.storage.storage.is_disk_index_enabled()
449    }
450
451    fn min_ongoing_scan_root_from_btree(ongoing_scan_roots: &BTreeMap<Slot, u64>) -> Option<Slot> {
452        ongoing_scan_roots.keys().next().cloned()
453    }
454
455    fn do_checked_scan_accounts<F, R>(
456        &self,
457        metric_name: &'static str,
458        ancestors: &Ancestors,
459        scan_bank_id: BankId,
460        func: F,
461        scan_type: ScanTypes<R>,
462        config: &ScanConfig,
463    ) -> Result<(), ScanError>
464    where
465        F: FnMut(&Pubkey, (&T, Slot)),
466        R: RangeBounds<Pubkey> + std::fmt::Debug,
467    {
468        {
469            let locked_removed_bank_ids = self.removed_bank_ids.lock().unwrap();
470            if locked_removed_bank_ids.contains(&scan_bank_id) {
471                return Err(ScanError::SlotRemoved {
472                    slot: ancestors.max_slot(),
473                    bank_id: scan_bank_id,
474                });
475            }
476        }
477
478        self.active_scans.fetch_add(1, Ordering::Relaxed);
479        let max_root = {
480            let mut w_ongoing_scan_roots = self
481                // This lock is also grabbed by clean_accounts(), so clean
482                // has at most cleaned up to the current `max_root` (since
483                // clean only happens *after* BankForks::set_root() which sets
484                // the `max_root`)
485                .ongoing_scan_roots
486                .write()
487                .unwrap();
488            // `max_root()` grabs a lock while
489            // the `ongoing_scan_roots` lock is held,
490            // make sure inverse doesn't happen to avoid
491            // deadlock
492            let max_root_inclusive = self.max_root_inclusive();
493            if let Some(min_ongoing_scan_root) =
494                Self::min_ongoing_scan_root_from_btree(&w_ongoing_scan_roots)
495            {
496                if min_ongoing_scan_root < max_root_inclusive {
497                    let current = max_root_inclusive - min_ongoing_scan_root;
498                    self.max_distance_to_min_scan_slot
499                        .fetch_max(current, Ordering::Relaxed);
500                }
501            }
502            *w_ongoing_scan_roots.entry(max_root_inclusive).or_default() += 1;
503
504            max_root_inclusive
505        };
506
507        // First we show that for any bank `B` that is a descendant of
508        // the current `max_root`, it must be true that and `B.ancestors.contains(max_root)`,
509        // regardless of the pattern of `squash()` behavior, where `ancestors` is the set
510        // of ancestors that is tracked in each bank.
511        //
512        // Proof: At startup, if starting from a snapshot, generate_index() adds all banks
513        // in the snapshot to the index via `add_root()` and so `max_root` will be the
514        // greatest of these. Thus, so the claim holds at startup since there are no
515        // descendants of `max_root`.
516        //
517        // Now we proceed by induction on each `BankForks::set_root()`.
518        // Assume the claim holds when the `max_root` is `R`. Call the set of
519        // descendants of `R` present in BankForks `R_descendants`.
520        //
521        // Then for any banks `B` in `R_descendants`, it must be that `B.ancestors.contains(S)`,
522        // where `S` is any ancestor of `B` such that `S >= R`.
523        //
524        // For example:
525        //          `R` -> `A` -> `C` -> `B`
526        // Then `B.ancestors == {R, A, C}`
527        //
528        // Next we call `BankForks::set_root()` at some descendant of `R`, `R_new`,
529        // where `R_new > R`.
530        //
531        // When we squash `R_new`, `max_root` in the AccountsIndex here is now set to `R_new`,
532        // and all nondescendants of `R_new` are pruned.
533        //
534        // Now consider any outstanding references to banks in the system that are descended from
535        // `max_root == R_new`. Take any one of these references and call it `B`. Because `B` is
536        // a descendant of `R_new`, this means `B` was also a descendant of `R`. Thus `B`
537        // must be a member of `R_descendants` because `B` was constructed and added to
538        // BankForks before the `set_root`.
539        //
540        // This means by the guarantees of `R_descendants` described above, because
541        // `R_new` is an ancestor of `B`, and `R < R_new < B`, then `B.ancestors.contains(R_new)`.
542        //
543        // Now until the next `set_root`, any new banks constructed from `new_from_parent` will
544        // also have `max_root == R_new` in their ancestor set, so the claim holds for those descendants
545        // as well. Once the next `set_root` happens, we once again update `max_root` and the same
546        // inductive argument can be applied again to show the claim holds.
547
548        // Check that the `max_root` is present in `ancestors`. From the proof above, if
549        // `max_root` is not present in `ancestors`, this means the bank `B` with the
550        // given `ancestors` is not descended from `max_root, which means
551        // either:
552        // 1) `B` is on a different fork or
553        // 2) `B` is an ancestor of `max_root`.
554        // In both cases we can ignore the given ancestors and instead just rely on the roots
555        // present as `max_root` indicates the roots present in the index are more up to date
556        // than the ancestors given.
557        let empty = Ancestors::default();
558        let ancestors = if ancestors.contains_key(&max_root) {
559            ancestors
560        } else {
561            /*
562            This takes of edge cases like:
563
564            Diagram 1:
565
566                        slot 0
567                          |
568                        slot 1
569                      /        \
570                 slot 2         |
571                    |       slot 3 (max root)
572            slot 4 (scan)
573
574            By the time the scan on slot 4 is called, slot 2 may already have been
575            cleaned by a clean on slot 3, but slot 4 may not have been cleaned.
576            The state in slot 2 would have been purged and is not saved in any roots.
577            In this case, a scan on slot 4 wouldn't accurately reflect the state when bank 4
578            was frozen. In cases like this, we default to a scan on the latest roots by
579            removing all `ancestors`.
580            */
581            &empty
582        };
583
584        /*
585        Now there are two cases, either `ancestors` is empty or nonempty:
586
587        1) If ancestors is empty, then this is the same as a scan on a rooted bank,
588        and `ongoing_scan_roots` provides protection against cleanup of roots necessary
589        for the scan, and  passing `Some(max_root)` to `do_scan_accounts()` ensures newer
590        roots don't appear in the scan.
591
592        2) If ancestors is non-empty, then from the `ancestors_contains(&max_root)` above, we know
593        that the fork structure must look something like:
594
595        Diagram 2:
596
597                Build fork structure:
598                        slot 0
599                          |
600                    slot 1 (max_root)
601                    /            \
602             slot 2              |
603                |            slot 3 (potential newer max root)
604              slot 4
605                |
606             slot 5 (scan)
607
608        Consider both types of ancestors, ancestor <= `max_root` and
609        ancestor > `max_root`, where `max_root == 1` as illustrated above.
610
611        a) The set of `ancestors <= max_root` are all rooted, which means their state
612        is protected by the same guarantees as 1).
613
614        b) As for the `ancestors > max_root`, those banks have at least one reference discoverable
615        through the chain of `Bank::BankRc::parent` starting from the calling bank. For instance
616        bank 5's parent reference keeps bank 4 alive, which will prevent the `Bank::drop()` from
617        running and cleaning up bank 4. Furthermore, no cleans can happen past the saved max_root == 1,
618        so a potential newer max root at 3 will not clean up any of the ancestors > 1, so slot 4
619        will not be cleaned in the middle of the scan either. (NOTE similar reasoning is employed for
620        assert!() justification in AccountsDb::retry_to_get_account_accessor)
621        */
622        match scan_type {
623            ScanTypes::Unindexed(range) => {
624                // Pass "" not to log metrics, so RPC doesn't get spammy
625                self.do_scan_accounts(metric_name, ancestors, func, range, Some(max_root), config);
626            }
627            ScanTypes::Indexed(IndexKey::ProgramId(program_id)) => {
628                self.do_scan_secondary_index(
629                    ancestors,
630                    func,
631                    &self.program_id_index,
632                    &program_id,
633                    Some(max_root),
634                    config,
635                );
636            }
637            ScanTypes::Indexed(IndexKey::SplTokenMint(mint_key)) => {
638                self.do_scan_secondary_index(
639                    ancestors,
640                    func,
641                    &self.spl_token_mint_index,
642                    &mint_key,
643                    Some(max_root),
644                    config,
645                );
646            }
647            ScanTypes::Indexed(IndexKey::SplTokenOwner(owner_key)) => {
648                self.do_scan_secondary_index(
649                    ancestors,
650                    func,
651                    &self.spl_token_owner_index,
652                    &owner_key,
653                    Some(max_root),
654                    config,
655                );
656            }
657        }
658
659        {
660            self.active_scans.fetch_sub(1, Ordering::Relaxed);
661            let mut ongoing_scan_roots = self.ongoing_scan_roots.write().unwrap();
662            let count = ongoing_scan_roots.get_mut(&max_root).unwrap();
663            *count -= 1;
664            if *count == 0 {
665                ongoing_scan_roots.remove(&max_root);
666            }
667        }
668
669        // If the fork with tip at bank `scan_bank_id` was removed during our scan, then the scan
670        // may have been corrupted, so abort the results.
671        let was_scan_corrupted = self
672            .removed_bank_ids
673            .lock()
674            .unwrap()
675            .contains(&scan_bank_id);
676
677        if was_scan_corrupted {
678            Err(ScanError::SlotRemoved {
679                slot: ancestors.max_slot(),
680                bank_id: scan_bank_id,
681            })
682        } else {
683            Ok(())
684        }
685    }
686
687    // Scan accounts and return latest version of each account that is either:
688    // 1) rooted or
689    // 2) present in ancestors
690    fn do_scan_accounts<F, R>(
691        &self,
692        metric_name: &'static str,
693        ancestors: &Ancestors,
694        mut func: F,
695        range: Option<R>,
696        max_root: Option<Slot>,
697        config: &ScanConfig,
698    ) where
699        F: FnMut(&Pubkey, (&T, Slot)),
700        R: RangeBounds<Pubkey> + std::fmt::Debug,
701    {
702        let returns_items = match config.scan_order {
703            ScanOrder::Unsorted => AccountsIndexPubkeyIterOrder::Unsorted,
704            ScanOrder::Sorted => AccountsIndexPubkeyIterOrder::Sorted,
705        };
706
707        // TODO: expand to use mint index to find the `pubkey_list` below more efficiently
708        // instead of scanning the entire range
709        let mut total_elapsed_timer = Measure::start("total");
710        let mut num_keys_iterated = 0;
711        let mut latest_slot_elapsed = 0;
712        let mut load_account_elapsed = 0;
713        let mut read_lock_elapsed = 0;
714        let mut iterator_elapsed = 0;
715        let mut iterator_timer = Measure::start("iterator_elapsed");
716
717        for pubkeys in self.iter(range.as_ref(), returns_items) {
718            iterator_timer.stop();
719            iterator_elapsed += iterator_timer.as_us();
720            for pubkey in pubkeys {
721                num_keys_iterated += 1;
722                self.get_and_then(&pubkey, |entry| {
723                    if let Some(list) = entry {
724                        let mut read_lock_timer = Measure::start("read_lock");
725                        let list_r = &list.slot_list_read_lock();
726                        read_lock_timer.stop();
727                        read_lock_elapsed += read_lock_timer.as_us();
728                        let mut latest_slot_timer = Measure::start("latest_slot");
729                        if let Some(index) = self.latest_slot(Some(ancestors), list_r, max_root) {
730                            latest_slot_timer.stop();
731                            latest_slot_elapsed += latest_slot_timer.as_us();
732                            let mut load_account_timer = Measure::start("load_account");
733                            func(&pubkey, (&list_r[index].1, list_r[index].0));
734                            load_account_timer.stop();
735                            load_account_elapsed += load_account_timer.as_us();
736                        }
737                    }
738                    let add_to_in_mem_cache = false;
739                    (add_to_in_mem_cache, ())
740                });
741                if config.is_aborted() {
742                    return;
743                }
744            }
745            iterator_timer = Measure::start("iterator_elapsed");
746        }
747
748        total_elapsed_timer.stop();
749        if !metric_name.is_empty() {
750            datapoint_info!(
751                metric_name,
752                ("total_elapsed", total_elapsed_timer.as_us(), i64),
753                ("latest_slot_elapsed", latest_slot_elapsed, i64),
754                ("read_lock_elapsed", read_lock_elapsed, i64),
755                ("load_account_elapsed", load_account_elapsed, i64),
756                ("iterator_elapsed", iterator_elapsed, i64),
757                ("num_keys_iterated", num_keys_iterated, i64),
758            )
759        }
760    }
761
762    fn do_scan_secondary_index<
763        F,
764        SecondaryIndexEntryType: SecondaryIndexEntry + Default + Sync + Send,
765    >(
766        &self,
767        ancestors: &Ancestors,
768        mut func: F,
769        index: &SecondaryIndex<SecondaryIndexEntryType>,
770        index_key: &Pubkey,
771        max_root: Option<Slot>,
772        config: &ScanConfig,
773    ) where
774        F: FnMut(&Pubkey, (&T, Slot)),
775    {
776        for pubkey in index.get(index_key) {
777            if config.is_aborted() {
778                break;
779            }
780            self.get_with_and_then(
781                &pubkey,
782                Some(ancestors),
783                max_root,
784                true,
785                |(slot, account_info)| func(&pubkey, (&account_info, slot)),
786            );
787        }
788    }
789
790    /// Gets the index's entry for `pubkey` and applies `callback` to it
791    ///
792    /// If `callback`'s boolean return value is true, add this entry to the in-mem cache.
793    pub fn get_and_then<R>(
794        &self,
795        pubkey: &Pubkey,
796        callback: impl FnOnce(Option<&AccountMapEntry<T>>) -> (bool, R),
797    ) -> R {
798        self.get_bin(pubkey).get_internal_inner(pubkey, callback)
799    }
800
801    /// Gets the index's entry for `pubkey`, with `ancestors` and `max_root`,
802    /// and applies `callback` to it
803    pub(crate) fn get_with_and_then<R>(
804        &self,
805        pubkey: &Pubkey,
806        ancestors: Option<&Ancestors>,
807        max_root: Option<Slot>,
808        should_add_to_in_mem_cache: bool,
809        callback: impl FnOnce((Slot, T)) -> R,
810    ) -> Option<R> {
811        self.get_and_then(pubkey, |entry| {
812            let callback_result = entry.and_then(|entry| {
813                self.get_account_info_with_and_then(entry, ancestors, max_root, callback)
814            });
815            (should_add_to_in_mem_cache, callback_result)
816        })
817    }
818
819    /// Gets the account info (and slot) in `entry`, with `ancestors` and `max_root`,
820    /// and applies `callback` to it
821    pub(crate) fn get_account_info_with_and_then<R>(
822        &self,
823        entry: &AccountMapEntry<T>,
824        ancestors: Option<&Ancestors>,
825        max_root: Option<Slot>,
826        callback: impl FnOnce((Slot, T)) -> R,
827    ) -> Option<R> {
828        let slot_list = entry.slot_list_read_lock();
829        self.latest_slot(ancestors, &slot_list, max_root)
830            .map(|found_index| callback(slot_list[found_index]))
831    }
832
833    /// Is `pubkey` in the index?
834    pub fn contains(&self, pubkey: &Pubkey) -> bool {
835        self.get_and_then(pubkey, |entry| (false, entry.is_some()))
836    }
837
838    /// Is `pubkey`, with `ancestors` and `max_root`, in the index?
839    #[cfg(test)]
840    pub(crate) fn contains_with(
841        &self,
842        pubkey: &Pubkey,
843        ancestors: Option<&Ancestors>,
844        max_root: Option<Slot>,
845    ) -> bool {
846        self.get_with_and_then(pubkey, ancestors, max_root, false, |_| ())
847            .is_some()
848    }
849
850    fn slot_list_mut<RT>(
851        &self,
852        pubkey: &Pubkey,
853        user_fn: impl FnOnce(SlotListWriteGuard<T>) -> RT,
854    ) -> Option<RT> {
855        let read_lock = self.get_bin(pubkey);
856        read_lock.slot_list_mut(pubkey, user_fn)
857    }
858
859    /// Remove keys from the account index if the key's slot list is empty.
860    /// Returns the keys that were removed from the index. These keys should not be accessed again in the current code path.
861    #[must_use]
862    pub fn handle_dead_keys(
863        &self,
864        dead_keys: &[Pubkey],
865        account_indexes: &AccountSecondaryIndexes,
866    ) -> HashSet<Pubkey> {
867        let mut pubkeys_removed_from_accounts_index = HashSet::default();
868        if !dead_keys.is_empty() {
869            for key in dead_keys.iter() {
870                let w_index = self.get_bin(key);
871                if w_index.remove_if_slot_list_empty(*key) {
872                    pubkeys_removed_from_accounts_index.insert(*key);
873                    // Note it's only safe to remove all the entries for this key
874                    // because we have the lock for this key's entry in the AccountsIndex,
875                    // so no other thread is also updating the index
876                    self.purge_secondary_indexes_by_inner_key(key, account_indexes);
877                }
878            }
879        }
880        pubkeys_removed_from_accounts_index
881    }
882
883    /// call func with every pubkey and index visible from a given set of ancestors
884    pub(crate) fn scan_accounts<F>(
885        &self,
886        ancestors: &Ancestors,
887        scan_bank_id: BankId,
888        func: F,
889        config: &ScanConfig,
890    ) -> Result<(), ScanError>
891    where
892        F: FnMut(&Pubkey, (&T, Slot)),
893    {
894        // Pass "" not to log metrics, so RPC doesn't get spammy
895        self.do_checked_scan_accounts(
896            "",
897            ancestors,
898            scan_bank_id,
899            func,
900            ScanTypes::Unindexed(None::<Range<Pubkey>>),
901            config,
902        )
903    }
904
905    /// call func with every pubkey and index visible from a given set of ancestors
906    pub(crate) fn index_scan_accounts<F>(
907        &self,
908        ancestors: &Ancestors,
909        scan_bank_id: BankId,
910        index_key: IndexKey,
911        func: F,
912        config: &ScanConfig,
913    ) -> Result<(), ScanError>
914    where
915        F: FnMut(&Pubkey, (&T, Slot)),
916    {
917        // Pass "" not to log metrics, so RPC doesn't get spammy
918        self.do_checked_scan_accounts(
919            "",
920            ancestors,
921            scan_bank_id,
922            func,
923            ScanTypes::<Range<Pubkey>>::Indexed(index_key),
924            config,
925        )
926    }
927
928    pub fn get_rooted_entries(
929        &self,
930        slot_list: &[(Slot, T)],
931        max_inclusive: Option<Slot>,
932    ) -> SlotList<T> {
933        let max_inclusive = max_inclusive.unwrap_or(Slot::MAX);
934        let lock = &self.roots_tracker.read().unwrap().alive_roots;
935        slot_list
936            .iter()
937            .filter(|(slot, _)| *slot <= max_inclusive && lock.contains(slot))
938            .cloned()
939            .collect()
940    }
941
942    /// returns true if, after this fn call:
943    /// accounts index entry for `pubkey` has an empty slot list
944    /// or `pubkey` does not exist in accounts index
945    pub(crate) fn purge_exact(
946        &self,
947        pubkey: &Pubkey,
948        slots_to_purge: impl for<'a> Contains<'a, Slot>,
949        reclaims: &mut ReclaimsSlotList<T>,
950    ) -> bool {
951        self.slot_list_mut(pubkey, |mut slot_list| {
952            slot_list.retain_and_count(|(slot, item)| {
953                let should_purge = slots_to_purge.contains(slot);
954                if should_purge {
955                    reclaims.push((*slot, *item));
956                    false
957                } else {
958                    true
959                }
960            }) == 0
961        })
962        .unwrap_or(true)
963    }
964
965    pub fn min_ongoing_scan_root(&self) -> Option<Slot> {
966        Self::min_ongoing_scan_root_from_btree(&self.ongoing_scan_roots.read().unwrap())
967    }
968
969    // Given a SlotList `L`, a list of ancestors and a maximum slot, find the latest element
970    // in `L`, where the slot `S` is an ancestor or root, and if `S` is a root, then `S <= max_root`
971    pub(crate) fn latest_slot(
972        &self,
973        ancestors: Option<&Ancestors>,
974        slot_list: &[(Slot, T)],
975        max_root_inclusive: Option<Slot>,
976    ) -> Option<usize> {
977        let mut current_max = 0;
978        let mut rv = None;
979        if let Some(ancestors) = ancestors {
980            if !ancestors.is_empty() {
981                for (i, (slot, _t)) in slot_list.iter().rev().enumerate() {
982                    if (rv.is_none() || *slot > current_max) && ancestors.contains_key(slot) {
983                        rv = Some(i);
984                        current_max = *slot;
985                    }
986                }
987            }
988        }
989
990        let max_root_inclusive = max_root_inclusive.unwrap_or(Slot::MAX);
991        let mut tracker = None;
992
993        for (i, (slot, _t)) in slot_list.iter().rev().enumerate() {
994            if (rv.is_none() || *slot > current_max) && *slot <= max_root_inclusive {
995                let lock = match tracker {
996                    Some(inner) => inner,
997                    None => self.roots_tracker.read().unwrap(),
998                };
999                if lock.alive_roots.contains(slot) {
1000                    rv = Some(i);
1001                    current_max = *slot;
1002                }
1003                tracker = Some(lock);
1004            }
1005        }
1006
1007        rv.map(|index| slot_list.len() - 1 - index)
1008    }
1009
1010    pub(crate) fn stats(&self) -> &Stats {
1011        &self.storage.storage.stats
1012    }
1013
1014    /// get stats related to startup
1015    pub(crate) fn get_startup_stats(&self) -> &StartupStats {
1016        &self.storage.storage.startup_stats
1017    }
1018
1019    pub(crate) fn set_startup(&self, value: Startup) {
1020        self.storage.set_startup(value);
1021    }
1022
1023    pub fn get_startup_remaining_items_to_flush_estimate(&self) -> usize {
1024        self.storage.get_startup_remaining_items_to_flush_estimate()
1025    }
1026
1027    /// Scan AccountsIndex for a given iterator of Pubkeys.
1028    ///
1029    /// This fn takes 4 arguments.
1030    ///  - an iterator of pubkeys to scan
1031    ///  - callback fn to run for each pubkey in the accounts index
1032    ///  - avoid_callback_result. If it is Some(default), then callback is ignored and
1033    ///    default is returned instead.
1034    ///  - provide_entry_in_callback. If true, populate the ref of the Arc of the
1035    ///    index entry to `callback` fn. Otherwise, provide None.
1036    ///
1037    /// The `callback` fn must return `AccountsIndexScanResult`, which is
1038    /// used to indicates whether the AccountIndex Entry should be added to
1039    /// in-memory cache. The `callback` fn takes in 3 arguments:
1040    ///   - the first an immutable ref of the pubkey,
1041    ///   - the second an option of the SlotList and RefCount
1042    ///   - the third an option of the AccountMapEntry, which is only populated
1043    ///     when `provide_entry_in_callback` is true. Otherwise, it will be
1044    ///     None.
1045    pub(crate) fn scan<'a, F, I>(
1046        &self,
1047        pubkeys: I,
1048        mut callback: F,
1049        avoid_callback_result: Option<AccountsIndexScanResult>,
1050        filter: ScanFilter,
1051    ) where
1052        F: FnMut(&'a Pubkey, Option<(&[(Slot, T)], RefCount)>) -> AccountsIndexScanResult,
1053        I: Iterator<Item = &'a Pubkey>,
1054    {
1055        let mut lock = None;
1056        let mut last_bin = self.bins(); // too big, won't match
1057        pubkeys.into_iter().for_each(|pubkey| {
1058            let bin = self.bin_calculator.bin_from_pubkey(pubkey);
1059            if bin != last_bin {
1060                // cannot re-use lock since next pubkey is in a different bin than previous one
1061                lock = Some(&self.account_maps[bin]);
1062                last_bin = bin;
1063            }
1064
1065            let mut internal_callback = |entry: Option<&AccountMapEntry<T>>| {
1066                let mut cache = false;
1067                match entry {
1068                    Some(locked_entry) => {
1069                        let result = if let Some(result) = avoid_callback_result.as_ref() {
1070                            *result
1071                        } else {
1072                            let slot_list = locked_entry.slot_list_read_lock();
1073                            callback(pubkey, Some((slot_list.as_ref(), locked_entry.ref_count())))
1074                        };
1075                        cache = match result {
1076                            AccountsIndexScanResult::Unref => {
1077                                locked_entry.unref();
1078                                true
1079                            }
1080                            AccountsIndexScanResult::UnrefAssert0 => {
1081                                assert_eq!(
1082                                    locked_entry.unref(),
1083                                    1,
1084                                    "ref count expected to be zero, but is {}! {pubkey}, {:?}",
1085                                    locked_entry.ref_count(),
1086                                    locked_entry.slot_list_read_lock(),
1087                                );
1088                                true
1089                            }
1090                            AccountsIndexScanResult::UnrefLog0 => {
1091                                let old_ref = locked_entry.unref();
1092                                if old_ref != 1 {
1093                                    info!(
1094                                        "Unexpected unref {pubkey} with {old_ref} {:?}, expect \
1095                                         old_ref to be 1",
1096                                        locked_entry.slot_list_read_lock()
1097                                    );
1098                                    datapoint_warn!(
1099                                        "accounts_db-unexpected-unref-zero",
1100                                        ("old_ref", old_ref, i64),
1101                                        ("pubkey", pubkey.to_string(), String),
1102                                    );
1103                                }
1104                                true
1105                            }
1106                            AccountsIndexScanResult::KeepInMemory => true,
1107                            AccountsIndexScanResult::OnlyKeepInMemoryIfDirty => false,
1108                        };
1109                    }
1110                    None => {
1111                        avoid_callback_result.unwrap_or_else(|| callback(pubkey, None));
1112                    }
1113                }
1114                (cache, ())
1115            };
1116
1117            match filter {
1118                ScanFilter::All => {
1119                    // SAFETY: The caller must ensure that if `provide_entry_in_callback` is true, and
1120                    // if it's possible for `callback` to clone the entry Arc, then it must also add
1121                    // the entry to the in-mem cache if the entry is made dirty.
1122                    lock.as_ref()
1123                        .unwrap()
1124                        .get_internal_inner(pubkey, internal_callback);
1125                }
1126                ScanFilter::OnlyAbnormal
1127                | ScanFilter::OnlyAbnormalWithVerify
1128                | ScanFilter::OnlyAbnormalTest => {
1129                    let found =
1130                        lock.as_ref()
1131                            .unwrap()
1132                            .get_only_in_mem(pubkey, false, |mut entry| {
1133                                if entry.is_some() && matches!(filter, ScanFilter::OnlyAbnormalTest)
1134                                {
1135                                    let local_entry = entry.unwrap();
1136                                    if local_entry.ref_count() == 1
1137                                        && local_entry.slot_list_lock_read_len() == 1
1138                                    {
1139                                        // Account was found in memory, but is a single ref single slot account
1140                                        // For testing purposes, return None as this can be treated like
1141                                        // a normal account that was flushed to storage.
1142                                        entry = None;
1143                                    }
1144                                }
1145                                internal_callback(entry);
1146                                entry.is_some()
1147                            });
1148                    if !found && matches!(filter, ScanFilter::OnlyAbnormalWithVerify) {
1149                        lock.as_ref().unwrap().get_internal_inner(pubkey, |entry| {
1150                            assert!(entry.is_some(), "{pubkey}, entry: {entry:?}");
1151                            let entry = entry.unwrap();
1152                            assert_eq!(entry.ref_count(), 1, "{pubkey}");
1153                            assert_eq!(entry.slot_list_lock_read_len(), 1, "{pubkey}");
1154                            (false, ())
1155                        });
1156                    }
1157                }
1158            }
1159        });
1160    }
1161
1162    // Get the maximum root <= `max_allowed_root` from the given `slot_list`
1163    fn get_newest_root_in_slot_list(
1164        alive_roots: &RollingBitField,
1165        slot_list: &[(Slot, T)],
1166        max_allowed_root_inclusive: Option<Slot>,
1167    ) -> Slot {
1168        slot_list
1169            .iter()
1170            .map(|(slot, _)| slot)
1171            .filter(|slot| max_allowed_root_inclusive.is_none_or(|max_root| **slot <= max_root))
1172            .filter(|slot| alive_roots.contains(slot))
1173            .max()
1174            .copied()
1175            .unwrap_or(0)
1176    }
1177
1178    fn update_spl_token_secondary_indexes<G: spl_generic_token::token::GenericTokenAccount>(
1179        &self,
1180        token_id: &Pubkey,
1181        pubkey: &Pubkey,
1182        account_owner: &Pubkey,
1183        account_data: &[u8],
1184        account_indexes: &AccountSecondaryIndexes,
1185    ) {
1186        if *account_owner == *token_id {
1187            if account_indexes.contains(&AccountIndex::SplTokenOwner) {
1188                if let Some(owner_key) = G::unpack_account_owner(account_data) {
1189                    if account_indexes.include_key(owner_key) {
1190                        self.spl_token_owner_index.insert(owner_key, pubkey);
1191                    }
1192                }
1193            }
1194
1195            if account_indexes.contains(&AccountIndex::SplTokenMint) {
1196                if let Some(mint_key) = G::unpack_account_mint(account_data) {
1197                    if account_indexes.include_key(mint_key) {
1198                        self.spl_token_mint_index.insert(mint_key, pubkey);
1199                    }
1200                }
1201            }
1202        }
1203    }
1204
1205    pub fn get_index_key_size(&self, index: &AccountIndex, index_key: &Pubkey) -> Option<usize> {
1206        match index {
1207            AccountIndex::ProgramId => self.program_id_index.index.get(index_key).map(|x| x.len()),
1208            AccountIndex::SplTokenOwner => self
1209                .spl_token_owner_index
1210                .index
1211                .get(index_key)
1212                .map(|x| x.len()),
1213            AccountIndex::SplTokenMint => self
1214                .spl_token_mint_index
1215                .index
1216                .get(index_key)
1217                .map(|x| x.len()),
1218        }
1219    }
1220
1221    /// log any secondary index counts, if non-zero
1222    pub(crate) fn log_secondary_indexes(&self) {
1223        if !self.program_id_index.index.is_empty() {
1224            info!("secondary index: {:?}", AccountIndex::ProgramId);
1225            self.program_id_index.log_contents();
1226        }
1227        if !self.spl_token_mint_index.index.is_empty() {
1228            info!("secondary index: {:?}", AccountIndex::SplTokenMint);
1229            self.spl_token_mint_index.log_contents();
1230        }
1231        if !self.spl_token_owner_index.index.is_empty() {
1232            info!("secondary index: {:?}", AccountIndex::SplTokenOwner);
1233            self.spl_token_owner_index.log_contents();
1234        }
1235    }
1236
1237    pub(crate) fn update_secondary_indexes(
1238        &self,
1239        pubkey: &Pubkey,
1240        account: &impl ReadableAccount,
1241        account_indexes: &AccountSecondaryIndexes,
1242    ) {
1243        if account_indexes.is_empty() {
1244            return;
1245        }
1246
1247        let account_owner = account.owner();
1248        let account_data = account.data();
1249
1250        if account_indexes.contains(&AccountIndex::ProgramId)
1251            && account_indexes.include_key(account_owner)
1252        {
1253            self.program_id_index.insert(account_owner, pubkey);
1254        }
1255        // Note because of the below check below on the account data length, when an
1256        // account hits zero lamports and is reset to AccountSharedData::Default, then we skip
1257        // the below updates to the secondary indexes.
1258        //
1259        // Skipping means not updating secondary index to mark the account as missing.
1260        // This doesn't introduce false positives during a scan because the caller to scan
1261        // provides the ancestors to check. So even if a zero-lamport account is not yet
1262        // removed from the secondary index, the scan function will:
1263        // 1) consult the primary index via `get(&pubkey, Some(ancestors), max_root)`
1264        // and find the zero-lamport version
1265        // 2) When the fetch from storage occurs, it will return AccountSharedData::Default
1266        // (as persisted tombstone for snapshots). This will then ultimately be
1267        // filtered out by post-scan filters, like in `get_filtered_spl_token_accounts_by_owner()`.
1268
1269        self.update_spl_token_secondary_indexes::<spl_generic_token::token::Account>(
1270            &spl_generic_token::token::id(),
1271            pubkey,
1272            account_owner,
1273            account_data,
1274            account_indexes,
1275        );
1276        self.update_spl_token_secondary_indexes::<spl_generic_token::token_2022::Account>(
1277            &spl_generic_token::token_2022::id(),
1278            pubkey,
1279            account_owner,
1280            account_data,
1281            account_indexes,
1282        );
1283    }
1284
1285    pub(crate) fn get_bin(&self, pubkey: &Pubkey) -> &InMemAccountsIndex<T, U> {
1286        &self.account_maps[self.bin_calculator.bin_from_pubkey(pubkey)]
1287    }
1288
1289    pub fn bins(&self) -> usize {
1290        self.account_maps.len()
1291    }
1292
1293    // Same functionally to upsert, but:
1294    // 1. operates on a batch of items
1295    // 2. holds the write lock for the duration of adding the items
1296    // Can save time when inserting lots of new keys.
1297    // But, does NOT update secondary index
1298    // This is designed to be called at startup time.
1299    // returns (insertion_time_us, InsertNewIfMissingIntoPrimaryIndexInfo)
1300    pub(crate) fn insert_new_if_missing_into_primary_index(
1301        &self,
1302        slot: Slot,
1303        mut items: Vec<(Pubkey, T)>,
1304    ) -> (u64, InsertNewIfMissingIntoPrimaryIndexInfo) {
1305        let mut insert_time = Measure::start("insert_into_primary_index");
1306
1307        let use_disk = self.storage.storage.is_disk_index_enabled();
1308
1309        let mut count = 0;
1310
1311        // accumulated stats after inserting pubkeys into the index
1312        let mut num_did_not_exist = 0;
1313        let mut num_existed_in_mem = 0;
1314        let mut num_existed_on_disk = 0;
1315
1316        // offset bin processing in the 'binned' array by a random amount.
1317        // This results in calls to insert_new_entry_if_missing_with_lock from different threads starting at different bins to avoid
1318        // lock contention.
1319        let bins = self.bins();
1320        let random_bin_offset = thread_rng().gen_range(0..bins);
1321        let bin_calc = self.bin_calculator;
1322        items.sort_unstable_by(|(pubkey_a, _), (pubkey_b, _)| {
1323            ((bin_calc.bin_from_pubkey(pubkey_a) + random_bin_offset) % bins)
1324                .cmp(&((bin_calc.bin_from_pubkey(pubkey_b) + random_bin_offset) % bins))
1325                .then_with(|| pubkey_a.cmp(pubkey_b))
1326        });
1327
1328        let storage = self.storage.storage.as_ref();
1329        while !items.is_empty() {
1330            let mut start_index = items.len() - 1;
1331            let mut last_pubkey = &items[start_index].0;
1332            let pubkey_bin = bin_calc.bin_from_pubkey(last_pubkey);
1333            // Find the smallest index with the same pubkey bin
1334            while start_index > 0 {
1335                let next = start_index - 1;
1336                let next_pubkey = &items[next].0;
1337                assert_ne!(
1338                    next_pubkey, last_pubkey,
1339                    "Accounts may only be stored once per slot: {slot}"
1340                );
1341                if bin_calc.bin_from_pubkey(next_pubkey) != pubkey_bin {
1342                    break;
1343                }
1344                start_index = next;
1345                last_pubkey = next_pubkey;
1346            }
1347
1348            let r_account_maps = self.account_maps[pubkey_bin].as_ref();
1349            // count only considers non-duplicate accounts
1350            count += items.len() - start_index;
1351
1352            let items = items.drain(start_index..);
1353            if use_disk {
1354                r_account_maps.startup_insert_only(slot, items);
1355            } else {
1356                // not using disk buckets, so just write to in-mem
1357                // this is no longer the default case
1358                let mut duplicates_from_in_memory = vec![];
1359                items.for_each(|(pubkey, account_info)| {
1360                    let new_entry =
1361                        PreAllocatedAccountMapEntry::new(slot, account_info, storage, use_disk);
1362                    match r_account_maps.insert_new_entry_if_missing_with_lock(pubkey, new_entry) {
1363                        InsertNewEntryResults::DidNotExist => {
1364                            num_did_not_exist += 1;
1365                        }
1366                        InsertNewEntryResults::Existed {
1367                            other_slot,
1368                            location,
1369                        } => {
1370                            if let Some(other_slot) = other_slot {
1371                                duplicates_from_in_memory.push((other_slot, pubkey));
1372                            }
1373                            duplicates_from_in_memory.push((slot, pubkey));
1374
1375                            match location {
1376                                ExistedLocation::InMem => {
1377                                    num_existed_in_mem += 1;
1378                                }
1379                                ExistedLocation::OnDisk => {
1380                                    num_existed_on_disk += 1;
1381                                }
1382                            }
1383                        }
1384                    }
1385                });
1386
1387                r_account_maps
1388                    .startup_update_duplicates_from_in_memory_only(duplicates_from_in_memory);
1389            }
1390        }
1391        insert_time.stop();
1392
1393        (
1394            insert_time.as_us(),
1395            InsertNewIfMissingIntoPrimaryIndexInfo {
1396                count,
1397                num_did_not_exist,
1398                num_existed_in_mem,
1399                num_existed_on_disk,
1400            },
1401        )
1402    }
1403
1404    /// use Vec<> because the internal vecs are already allocated per bin
1405    pub(crate) fn populate_and_retrieve_duplicate_keys_from_startup(
1406        &self,
1407        f: impl Fn(Vec<(Slot, Pubkey)>) + Sync + Send,
1408    ) {
1409        (0..self.bins())
1410            .into_par_iter()
1411            .map(|pubkey_bin| {
1412                let r_account_maps = &self.account_maps[pubkey_bin];
1413                if self.storage.storage.is_disk_index_enabled() {
1414                    r_account_maps.populate_and_retrieve_duplicate_keys_from_startup()
1415                } else {
1416                    r_account_maps.startup_take_duplicates_from_in_memory_only()
1417                }
1418            })
1419            .for_each(f);
1420    }
1421
1422    /// Updates the given pubkey at the given slot with the new account information.
1423    /// on return, the index's previous account info may be returned in 'reclaims' depending on 'previous_slot_entry_was_cached'
1424    pub fn upsert(
1425        &self,
1426        new_slot: Slot,
1427        old_slot: Slot,
1428        pubkey: &Pubkey,
1429        account: &impl ReadableAccount,
1430        account_indexes: &AccountSecondaryIndexes,
1431        account_info: T,
1432        reclaims: &mut ReclaimsSlotList<T>,
1433        reclaim: UpsertReclaim,
1434    ) {
1435        // vast majority of updates are to item already in accounts index, so store as raw to avoid unnecessary allocations
1436        let store_raw = true;
1437
1438        // We don't atomically update both primary index and secondary index together.
1439        // This certainly creates a small time window with inconsistent state across the two indexes.
1440        // However, this is acceptable because:
1441        //
1442        //  - A strict consistent view at any given moment of time is not necessary, because the only
1443        //  use case for the secondary index is `scan`, and `scans` are only supported/require consistency
1444        //  on frozen banks, and this inconsistency is only possible on working banks.
1445        //
1446        //  - The secondary index is never consulted as primary source of truth for gets/stores.
1447        //  So, what the accounts_index sees alone is sufficient as a source of truth for other non-scan
1448        //  account operations.
1449        let new_item = PreAllocatedAccountMapEntry::new(
1450            new_slot,
1451            account_info,
1452            &self.storage.storage,
1453            store_raw,
1454        );
1455        let map = self.get_bin(pubkey);
1456
1457        map.upsert(pubkey, new_item, Some(old_slot), reclaims, reclaim);
1458        self.update_secondary_indexes(pubkey, account, account_indexes);
1459    }
1460
1461    pub fn ref_count_from_storage(&self, pubkey: &Pubkey) -> RefCount {
1462        let map = self.get_bin(pubkey);
1463        map.get_internal_inner(pubkey, |entry| {
1464            (
1465                false,
1466                entry.map(|entry| entry.ref_count()).unwrap_or_default(),
1467            )
1468        })
1469    }
1470
1471    fn purge_secondary_indexes_by_inner_key(
1472        &self,
1473        inner_key: &Pubkey,
1474        account_indexes: &AccountSecondaryIndexes,
1475    ) {
1476        if account_indexes.contains(&AccountIndex::ProgramId) {
1477            self.program_id_index.remove_by_inner_key(inner_key);
1478        }
1479
1480        if account_indexes.contains(&AccountIndex::SplTokenOwner) {
1481            self.spl_token_owner_index.remove_by_inner_key(inner_key);
1482        }
1483
1484        if account_indexes.contains(&AccountIndex::SplTokenMint) {
1485            self.spl_token_mint_index.remove_by_inner_key(inner_key);
1486        }
1487    }
1488
1489    /// Returns true if the slot list was completely purged (is empty at the end).
1490    fn purge_older_root_entries(
1491        &self,
1492        slot_list: &mut SlotListWriteGuard<T>,
1493        reclaims: &mut ReclaimsSlotList<T>,
1494        max_clean_root_inclusive: Option<Slot>,
1495    ) -> bool {
1496        if slot_list.len() <= 1 {
1497            self.purge_older_root_entries_one_slot_list
1498                .fetch_add(1, Ordering::Relaxed);
1499        }
1500        let newest_root_in_slot_list;
1501        let max_clean_root_inclusive = {
1502            let roots_tracker = &self.roots_tracker.read().unwrap();
1503            newest_root_in_slot_list = Self::get_newest_root_in_slot_list(
1504                &roots_tracker.alive_roots,
1505                slot_list,
1506                max_clean_root_inclusive,
1507            );
1508            max_clean_root_inclusive.unwrap_or_else(|| roots_tracker.alive_roots.max_inclusive())
1509        };
1510
1511        slot_list.retain_and_count(|(slot, value)| {
1512            let should_purge = Self::can_purge_older_entries(
1513                // Note that we have a root that is inclusive here.
1514                // Calling a function that expects 'exclusive'
1515                // This is expected behavior for this call.
1516                max_clean_root_inclusive,
1517                newest_root_in_slot_list,
1518                *slot,
1519            ) && !value.is_cached();
1520            if should_purge {
1521                reclaims.push((*slot, *value));
1522            }
1523            !should_purge
1524        }) == 0
1525    }
1526
1527    /// return true if pubkey was removed from the accounts index
1528    ///  or does not exist in the accounts index
1529    /// This means it should NOT be unref'd later.
1530    #[must_use]
1531    pub fn clean_rooted_entries(
1532        &self,
1533        pubkey: &Pubkey,
1534        reclaims: &mut ReclaimsSlotList<T>,
1535        max_clean_root_inclusive: Option<Slot>,
1536    ) -> bool {
1537        let mut is_slot_list_empty = false;
1538        let missing_in_accounts_index = self
1539            .slot_list_mut(pubkey, |mut slot_list| {
1540                is_slot_list_empty = self.purge_older_root_entries(
1541                    &mut slot_list,
1542                    reclaims,
1543                    max_clean_root_inclusive,
1544                );
1545            })
1546            .is_none();
1547
1548        let mut removed = false;
1549        // If the slot list is empty, remove the pubkey from `account_maps`. Make sure to grab the
1550        // lock and double check the slot list is still empty, because another writer could have
1551        // locked and inserted the pubkey in-between when `is_slot_list_empty=true` and the call to
1552        // remove() below.
1553        if is_slot_list_empty {
1554            let w_maps = self.get_bin(pubkey);
1555            removed = w_maps.remove_if_slot_list_empty(*pubkey);
1556        }
1557        removed || missing_in_accounts_index
1558    }
1559
1560    /// Clean the slot list by removing all slot_list items older than the max_slot
1561    /// Decrease the reference count of the entry by the number of removed accounts.
1562    /// Returns the slot and account_info of the remaining entry in the slot list
1563    /// Note: This must only be called on startup, and reclaims
1564    /// must be reclaimed.
1565    fn clean_and_unref_slot_list_on_startup(
1566        &self,
1567        entry: &AccountMapEntry<T>,
1568        reclaims: &mut ReclaimsSlotList<T>,
1569    ) -> (u64, T) {
1570        let mut slot_list = entry.slot_list_write_lock();
1571        let max_slot = slot_list
1572            .iter()
1573            .map(|(slot, _account)| *slot)
1574            .max()
1575            .expect("Slot list has entries");
1576
1577        let mut reclaim_count = 0;
1578
1579        slot_list.retain_and_count(|(slot, value)| {
1580            // keep the newest entry, and reclaim all others
1581            if *slot < max_slot {
1582                assert!(!value.is_cached(), "Unsafe to reclaim cached entries");
1583                reclaims.push((*slot, *value));
1584                reclaim_count += 1;
1585                false
1586            } else {
1587                true
1588            }
1589        });
1590
1591        // Unref
1592        entry.unref_by_count(reclaim_count);
1593        assert_eq!(
1594            entry.ref_count(),
1595            1,
1596            "ref count should be one after cleaning all entries"
1597        );
1598
1599        entry.set_dirty(true);
1600
1601        // Return the last entry in the slot list, which is the only one
1602        *slot_list
1603            .last()
1604            .expect("Slot list should have at least one entry after cleaning")
1605    }
1606
1607    /// Cleans and unrefs all older rooted entries for each pubkey in the accounts index.
1608    /// Calls passed in callback on the remaining slot entry
1609    /// All pubkeys must be from a single bin
1610    pub fn clean_and_unref_rooted_entries_by_bin(
1611        &self,
1612        pubkeys_by_bin: &[Pubkey],
1613    ) -> ReclaimsSlotList<T> {
1614        let mut reclaims = ReclaimsSlotList::new();
1615
1616        let map = match pubkeys_by_bin.first() {
1617            Some(pubkey) => self.get_bin(pubkey),
1618            None => return reclaims, // no pubkeys to process, return
1619        };
1620
1621        for pubkey in pubkeys_by_bin {
1622            map.get_internal_inner(pubkey, |entry| {
1623                let entry = entry.expect("Expected entry to exist in accounts index");
1624                self.clean_and_unref_slot_list_on_startup(entry, &mut reclaims);
1625                (false, ())
1626            });
1627        }
1628        reclaims
1629    }
1630
1631    /// When can an entry be purged?
1632    ///
1633    /// If we get a slot update where slot != newest_root_in_slot_list for an account where slot <
1634    /// max_clean_root_exclusive, then we know it's safe to delete because:
1635    ///
1636    /// a) If slot < newest_root_in_slot_list, then we know the update is outdated by a later rooted
1637    /// update, namely the one in newest_root_in_slot_list
1638    ///
1639    /// b) If slot > newest_root_in_slot_list, then because slot < max_clean_root_exclusive and we know there are
1640    /// no roots in the slot list between newest_root_in_slot_list and max_clean_root_exclusive, (otherwise there
1641    /// would be a bigger newest_root_in_slot_list, which is a contradiction), then we know slot must be
1642    /// an unrooted slot less than max_clean_root_exclusive and thus safe to clean as well.
1643    fn can_purge_older_entries(
1644        max_clean_root_exclusive: Slot,
1645        newest_root_in_slot_list: Slot,
1646        slot: Slot,
1647    ) -> bool {
1648        slot < max_clean_root_exclusive && slot != newest_root_in_slot_list
1649    }
1650
1651    /// Given a list of slots, return a new list of only the slots that are rooted
1652    pub fn get_rooted_from_list<'a>(&self, slots: impl Iterator<Item = &'a Slot>) -> Vec<Slot> {
1653        let roots_tracker = self.roots_tracker.read().unwrap();
1654        slots
1655            .filter_map(|s| {
1656                if roots_tracker.alive_roots.contains(s) {
1657                    Some(*s)
1658                } else {
1659                    None
1660                }
1661            })
1662            .collect()
1663    }
1664
1665    pub fn is_alive_root(&self, slot: Slot) -> bool {
1666        self.roots_tracker
1667            .read()
1668            .unwrap()
1669            .alive_roots
1670            .contains(&slot)
1671    }
1672
1673    pub fn add_root(&self, slot: Slot) {
1674        self.roots_added.fetch_add(1, Ordering::Relaxed);
1675        let mut w_roots_tracker = self.roots_tracker.write().unwrap();
1676        // `AccountsDb::flush_accounts_cache()` relies on roots being added in order
1677        assert!(
1678            slot >= w_roots_tracker.alive_roots.max_inclusive(),
1679            "Roots must be added in order: {} < {}",
1680            slot,
1681            w_roots_tracker.alive_roots.max_inclusive()
1682        );
1683        // 'slot' is a root, so it is both 'root' and 'original'
1684        w_roots_tracker.alive_roots.insert(slot);
1685    }
1686
1687    pub fn max_root_inclusive(&self) -> Slot {
1688        self.roots_tracker
1689            .read()
1690            .unwrap()
1691            .alive_roots
1692            .max_inclusive()
1693    }
1694
1695    /// Remove the slot when the storage for the slot is freed
1696    /// Accounts no longer reference this slot.
1697    /// return true if slot was a root
1698    pub fn clean_dead_slot(&self, slot: Slot) -> bool {
1699        let mut w_roots_tracker = self.roots_tracker.write().unwrap();
1700        if !w_roots_tracker.alive_roots.remove(&slot) {
1701            false
1702        } else {
1703            drop(w_roots_tracker);
1704            self.roots_removed.fetch_add(1, Ordering::Relaxed);
1705            true
1706        }
1707    }
1708
1709    pub(crate) fn update_roots_stats(&self, stats: &mut AccountsIndexRootsStats) {
1710        let roots_tracker = self.roots_tracker.read().unwrap();
1711        stats.roots_len = Some(roots_tracker.alive_roots.len());
1712        stats.roots_range = Some(roots_tracker.alive_roots.range_width());
1713    }
1714
1715    pub fn all_alive_roots(&self) -> Vec<Slot> {
1716        let tracker = self.roots_tracker.read().unwrap();
1717        tracker.alive_roots.get_all()
1718    }
1719
1720    // These functions/fields are only usable from a dev context (i.e. tests and benches)
1721    #[cfg(feature = "dev-context-only-utils")]
1722    // filter any rooted entries and return them along with a bool that indicates
1723    // if this account has no more entries. Note this does not update the secondary
1724    // indexes!
1725    pub fn purge_roots(&self, pubkey: &Pubkey) -> (SlotList<T>, bool) {
1726        self.slot_list_mut(pubkey, |mut slot_list| {
1727            let reclaims = self.get_rooted_entries(&slot_list, None);
1728            let is_empty = slot_list.retain_and_count(|(slot, _)| !self.is_alive_root(*slot)) == 0;
1729            (reclaims, is_empty)
1730        })
1731        .unwrap()
1732    }
1733}
1734
1735/// modes the system can be in
1736#[allow(clippy::enum_variant_names)]
1737#[derive(Debug, Eq, PartialEq)]
1738pub(crate) enum Startup {
1739    /// not startup, but steady state execution
1740    Normal,
1741    /// startup (not steady state execution)
1742    /// requesting 'startup'-like behavior where in-mem acct idx items are flushed asap
1743    #[cfg(test)]
1744    Startup,
1745    /// startup (not steady state execution)
1746    /// but also requesting additional threads to be running to flush the acct idx to disk asap
1747    /// The idea is that the best perf to ssds will be with multiple threads,
1748    ///  but during steady state, we can't allocate as many threads because we'd starve the rest of the system.
1749    StartupWithExtraThreads,
1750}
1751
1752#[cfg(test)]
1753pub mod tests {
1754    use {
1755        super::{bucket_map_holder::BucketMapHolder, *},
1756        crate::accounts_index::account_map_entry::AccountMapEntryMeta,
1757        solana_account::{AccountSharedData, WritableAccount},
1758        solana_pubkey::PUBKEY_BYTES,
1759        spl_generic_token::{spl_token_ids, token::SPL_TOKEN_ACCOUNT_OWNER_OFFSET},
1760        std::ops::{
1761            Bound::{Excluded, Included, Unbounded},
1762            RangeInclusive,
1763        },
1764        test_case::test_matrix,
1765    };
1766
1767    pub enum SecondaryIndexTypes<'a> {
1768        RwLock(&'a SecondaryIndex<RwLockSecondaryIndexEntry>),
1769    }
1770
1771    pub fn spl_token_mint_index_enabled() -> AccountSecondaryIndexes {
1772        let mut account_indexes = HashSet::new();
1773        account_indexes.insert(AccountIndex::SplTokenMint);
1774        AccountSecondaryIndexes {
1775            indexes: account_indexes,
1776            keys: None,
1777        }
1778    }
1779
1780    pub fn spl_token_owner_index_enabled() -> AccountSecondaryIndexes {
1781        let mut account_indexes = HashSet::new();
1782        account_indexes.insert(AccountIndex::SplTokenOwner);
1783        AccountSecondaryIndexes {
1784            indexes: account_indexes,
1785            keys: None,
1786        }
1787    }
1788
1789    fn create_spl_token_mint_secondary_index_state() -> (usize, usize, AccountSecondaryIndexes) {
1790        {
1791            // Check that we're actually testing the correct variant
1792            let index = AccountsIndex::<bool, bool>::default_for_tests();
1793            let _type_check = SecondaryIndexTypes::RwLock(&index.spl_token_mint_index);
1794        }
1795
1796        (0, PUBKEY_BYTES, spl_token_mint_index_enabled())
1797    }
1798
1799    fn create_spl_token_owner_secondary_index_state() -> (usize, usize, AccountSecondaryIndexes) {
1800        {
1801            // Check that we're actually testing the correct variant
1802            let index = AccountsIndex::<bool, bool>::default_for_tests();
1803            let _type_check = SecondaryIndexTypes::RwLock(&index.spl_token_owner_index);
1804        }
1805
1806        (
1807            SPL_TOKEN_ACCOUNT_OWNER_OFFSET,
1808            SPL_TOKEN_ACCOUNT_OWNER_OFFSET + PUBKEY_BYTES,
1809            spl_token_owner_index_enabled(),
1810        )
1811    }
1812
1813    #[test]
1814    fn test_get_empty() {
1815        let key = solana_pubkey::new_rand();
1816        let index = AccountsIndex::<bool, bool>::default_for_tests();
1817        let ancestors = Ancestors::default();
1818        let key = &key;
1819        assert!(!index.contains_with(key, Some(&ancestors), None));
1820        assert!(!index.contains_with(key, None, None));
1821
1822        let mut num = 0;
1823        index
1824            .scan_accounts(
1825                &ancestors,
1826                0,
1827                |_pubkey, _index| num += 1,
1828                &ScanConfig::default(),
1829            )
1830            .expect("scan should succeed");
1831        assert_eq!(num, 0);
1832    }
1833
1834    #[test]
1835    fn test_secondary_index_include_exclude() {
1836        let pk1 = Pubkey::new_unique();
1837        let pk2 = Pubkey::new_unique();
1838        let mut index = AccountSecondaryIndexes::default();
1839
1840        assert!(!index.contains(&AccountIndex::ProgramId));
1841        index.indexes.insert(AccountIndex::ProgramId);
1842        assert!(index.contains(&AccountIndex::ProgramId));
1843        assert!(index.include_key(&pk1));
1844        assert!(index.include_key(&pk2));
1845
1846        let exclude = false;
1847        index.keys = Some(AccountSecondaryIndexesIncludeExclude {
1848            keys: [pk1].iter().cloned().collect::<HashSet<_>>(),
1849            exclude,
1850        });
1851        assert!(index.include_key(&pk1));
1852        assert!(!index.include_key(&pk2));
1853
1854        let exclude = true;
1855        index.keys = Some(AccountSecondaryIndexesIncludeExclude {
1856            keys: [pk1].iter().cloned().collect::<HashSet<_>>(),
1857            exclude,
1858        });
1859        assert!(!index.include_key(&pk1));
1860        assert!(index.include_key(&pk2));
1861
1862        let exclude = true;
1863        index.keys = Some(AccountSecondaryIndexesIncludeExclude {
1864            keys: [pk1, pk2].iter().cloned().collect::<HashSet<_>>(),
1865            exclude,
1866        });
1867        assert!(!index.include_key(&pk1));
1868        assert!(!index.include_key(&pk2));
1869
1870        let exclude = false;
1871        index.keys = Some(AccountSecondaryIndexesIncludeExclude {
1872            keys: [pk1, pk2].iter().cloned().collect::<HashSet<_>>(),
1873            exclude,
1874        });
1875        assert!(index.include_key(&pk1));
1876        assert!(index.include_key(&pk2));
1877    }
1878
1879    const UPSERT_RECLAIM_TEST_DEFAULT: UpsertReclaim = UpsertReclaim::PopulateReclaims;
1880
1881    #[test]
1882    fn test_insert_no_ancestors() {
1883        let key = solana_pubkey::new_rand();
1884        let index = AccountsIndex::<bool, bool>::default_for_tests();
1885        let mut gc = ReclaimsSlotList::new();
1886        index.upsert(
1887            0,
1888            0,
1889            &key,
1890            &AccountSharedData::default(),
1891            &AccountSecondaryIndexes::default(),
1892            true,
1893            &mut gc,
1894            UPSERT_RECLAIM_TEST_DEFAULT,
1895        );
1896        assert!(gc.is_empty());
1897
1898        let ancestors = Ancestors::default();
1899        assert!(!index.contains_with(&key, Some(&ancestors), None));
1900        assert!(!index.contains_with(&key, None, None));
1901
1902        let mut num = 0;
1903        index
1904            .scan_accounts(
1905                &ancestors,
1906                0,
1907                |_pubkey, _index| num += 1,
1908                &ScanConfig::default(),
1909            )
1910            .expect("scan should succeed");
1911        assert_eq!(num, 0);
1912    }
1913
1914    type AccountInfoTest = f64;
1915
1916    impl IndexValue for AccountInfoTest {}
1917    impl DiskIndexValue for AccountInfoTest {}
1918    impl IsCached for AccountInfoTest {
1919        fn is_cached(&self) -> bool {
1920            true
1921        }
1922    }
1923
1924    impl IsZeroLamport for AccountInfoTest {
1925        fn is_zero_lamport(&self) -> bool {
1926            true
1927        }
1928    }
1929
1930    #[test]
1931    #[should_panic(expected = "Accounts may only be stored once per slot:")]
1932    fn test_insert_duplicates() {
1933        let key = solana_pubkey::new_rand();
1934        let pubkey = &key;
1935        let slot = 0;
1936        let mut ancestors = Ancestors::default();
1937        ancestors.insert(slot, 0);
1938
1939        let account_info = true;
1940        let index = AccountsIndex::<bool, bool>::default_for_tests();
1941        let account_info2: bool = !account_info;
1942        let items = vec![(*pubkey, account_info), (*pubkey, account_info2)];
1943        index.set_startup(Startup::Startup);
1944        let (_, _result) = index.insert_new_if_missing_into_primary_index(slot, items);
1945    }
1946
1947    #[test]
1948    fn test_insert_new_with_lock_no_ancestors() {
1949        let key = solana_pubkey::new_rand();
1950        let pubkey = &key;
1951        let slot = 0;
1952
1953        let index = AccountsIndex::<bool, bool>::default_for_tests();
1954        let account_info = true;
1955        let items = vec![(*pubkey, account_info)];
1956        index.set_startup(Startup::Startup);
1957        let expected_len = items.len();
1958        let (_, result) = index.insert_new_if_missing_into_primary_index(slot, items);
1959        assert_eq!(result.count, expected_len);
1960        index.set_startup(Startup::Normal);
1961
1962        let mut ancestors = Ancestors::default();
1963        assert!(!index.contains_with(pubkey, Some(&ancestors), None));
1964        assert!(!index.contains_with(pubkey, None, None));
1965
1966        let mut num = 0;
1967        index
1968            .scan_accounts(
1969                &ancestors,
1970                0,
1971                |_pubkey, _index| num += 1,
1972                &ScanConfig::default(),
1973            )
1974            .expect("scan should succeed");
1975        assert_eq!(num, 0);
1976        ancestors.insert(slot, 0);
1977        assert!(index.contains_with(pubkey, Some(&ancestors), None));
1978        assert_eq!(index.ref_count_from_storage(pubkey), 1);
1979        index
1980            .scan_accounts(
1981                &ancestors,
1982                0,
1983                |_pubkey, _index| num += 1,
1984                &ScanConfig::default(),
1985            )
1986            .expect("scan should succeed");
1987        assert_eq!(num, 1);
1988
1989        // not zero lamports
1990        let index = AccountsIndex::<bool, bool>::default_for_tests();
1991        let account_info = false;
1992        let items = vec![(*pubkey, account_info)];
1993        index.set_startup(Startup::Startup);
1994        let expected_len = items.len();
1995        let (_, result) = index.insert_new_if_missing_into_primary_index(slot, items);
1996        assert_eq!(result.count, expected_len);
1997        index.set_startup(Startup::Normal);
1998
1999        let mut ancestors = Ancestors::default();
2000        assert!(!index.contains_with(pubkey, Some(&ancestors), None));
2001        assert!(!index.contains_with(pubkey, None, None));
2002
2003        let mut num = 0;
2004        index
2005            .scan_accounts(
2006                &ancestors,
2007                0,
2008                |_pubkey, _index| num += 1,
2009                &ScanConfig::default(),
2010            )
2011            .expect("scan should succeed");
2012        assert_eq!(num, 0);
2013        ancestors.insert(slot, 0);
2014        assert!(index.contains_with(pubkey, Some(&ancestors), None));
2015        assert_eq!(index.ref_count_from_storage(pubkey), 1);
2016        index
2017            .scan_accounts(
2018                &ancestors,
2019                0,
2020                |_pubkey, _index| num += 1,
2021                &ScanConfig::default(),
2022            )
2023            .expect("scan should succeed");
2024        assert_eq!(num, 1);
2025    }
2026
2027    fn get_pre_allocated<T: IndexValue>(
2028        slot: Slot,
2029        account_info: T,
2030        storage: &Arc<BucketMapHolder<T, T>>,
2031        store_raw: bool,
2032        to_raw_first: bool,
2033    ) -> PreAllocatedAccountMapEntry<T> {
2034        let entry = PreAllocatedAccountMapEntry::new(slot, account_info, storage, store_raw);
2035
2036        if to_raw_first {
2037            // convert to raw
2038            let (slot2, account_info2) = entry.into();
2039            // recreate using extracted raw
2040            PreAllocatedAccountMapEntry::new(slot2, account_info2, storage, store_raw)
2041        } else {
2042            entry
2043        }
2044    }
2045
2046    #[test]
2047    fn test_clean_and_unref_rooted_entries_by_bin_empty() {
2048        let index: AccountsIndex<bool, bool> = AccountsIndex::<bool, bool>::default_for_tests();
2049        let pubkeys_by_bin: Vec<Pubkey> = vec![];
2050
2051        let reclaims = index.clean_and_unref_rooted_entries_by_bin(&pubkeys_by_bin);
2052
2053        assert!(reclaims.is_empty());
2054    }
2055
2056    #[test]
2057    fn test_clean_and_unref_rooted_entries_by_bin_single_entry() {
2058        let index = AccountsIndex::<bool, bool>::default_for_tests();
2059        let pubkey = solana_pubkey::new_rand();
2060        let slot = 0;
2061        let account_info = true;
2062
2063        let mut gc = ReclaimsSlotList::new();
2064        index.upsert(
2065            slot,
2066            slot,
2067            &pubkey,
2068            &AccountSharedData::default(),
2069            &AccountSecondaryIndexes::default(),
2070            account_info,
2071            &mut gc,
2072            UPSERT_RECLAIM_TEST_DEFAULT,
2073        );
2074
2075        assert!(gc.is_empty());
2076
2077        let reclaims = index.clean_and_unref_rooted_entries_by_bin(&[pubkey]);
2078
2079        assert_eq!(reclaims.len(), 0);
2080    }
2081
2082    #[test]
2083    fn test_clean_and_unref_rooted_entries_by_bin_with_reclaim() {
2084        let index = AccountsIndex::<u64, u64>::default_for_tests();
2085        let pubkey = solana_pubkey::new_rand();
2086        let slot1 = 0;
2087        let slot2 = 1;
2088        let account_info1 = 0;
2089        let account_info2 = 1;
2090
2091        let mut gc = ReclaimsSlotList::new();
2092        for (slot, account_info) in [(slot1, account_info1), (slot2, account_info2)] {
2093            index.upsert(
2094                slot,
2095                slot,
2096                &pubkey,
2097                &AccountSharedData::default(),
2098                &AccountSecondaryIndexes::default(),
2099                account_info,
2100                &mut gc,
2101                UpsertReclaim::IgnoreReclaims,
2102            );
2103        }
2104
2105        assert!(gc.is_empty());
2106
2107        let reclaims = index.clean_and_unref_rooted_entries_by_bin(&[pubkey]);
2108        assert_eq!(reclaims, ReclaimsSlotList::from([(slot1, account_info1)]));
2109    }
2110
2111    #[test]
2112    fn test_clean_and_unref_rooted_entries_by_bin_multiple_pubkeys() {
2113        let index: AccountsIndex<bool, bool> = AccountsIndex::<bool, bool>::default_for_tests();
2114        let bin_index = 0;
2115        let mut pubkeys = Vec::new();
2116        let mut expected_reclaims = ReclaimsSlotList::new();
2117        let mut gc = ReclaimsSlotList::new();
2118
2119        while pubkeys.len() < 10 {
2120            let new_pubkey = solana_pubkey::new_rand();
2121
2122            // Ensure the pubkey is in the desired bin
2123            if index.bin_calculator.bin_from_pubkey(&new_pubkey) == bin_index {
2124                pubkeys.push(new_pubkey);
2125            }
2126        }
2127
2128        for (i, pubkey) in pubkeys.iter().enumerate() {
2129            let num_inserts: u64 = i as u64 % 4 + 1;
2130
2131            for slot in 0..num_inserts {
2132                if slot > 0 {
2133                    expected_reclaims.push((slot - 1, true));
2134                }
2135                index.upsert(
2136                    slot,
2137                    slot,
2138                    pubkey,
2139                    &AccountSharedData::default(),
2140                    &AccountSecondaryIndexes::default(),
2141                    true,
2142                    &mut gc,
2143                    UpsertReclaim::IgnoreReclaims,
2144                );
2145            }
2146        }
2147
2148        assert!(gc.is_empty());
2149
2150        let mut reclaims = index.clean_and_unref_rooted_entries_by_bin(&pubkeys);
2151        reclaims.sort_unstable();
2152        expected_reclaims.sort_unstable();
2153
2154        assert!(!reclaims.is_empty());
2155        assert_eq!(reclaims, expected_reclaims);
2156    }
2157
2158    #[test]
2159    fn test_new_entry() {
2160        for store_raw in [false, true] {
2161            for to_raw_first in [false, true] {
2162                let slot = 0;
2163                // account_info type that IS cached
2164                let account_info = AccountInfoTest::default();
2165                let index = AccountsIndex::default_for_tests();
2166
2167                let new_entry = get_pre_allocated(
2168                    slot,
2169                    account_info,
2170                    &index.storage.storage,
2171                    store_raw,
2172                    to_raw_first,
2173                )
2174                .into_account_map_entry(&index.storage.storage);
2175                assert_eq!(new_entry.ref_count(), 0);
2176                assert_eq!(new_entry.slot_list_lock_read_len(), 1);
2177                assert_eq!(
2178                    new_entry.slot_list_read_lock().to_vec(),
2179                    vec![(slot, account_info)]
2180                );
2181
2182                // account_info type that is NOT cached
2183                let account_info = true;
2184                let index = AccountsIndex::default_for_tests();
2185
2186                let new_entry = get_pre_allocated(
2187                    slot,
2188                    account_info,
2189                    &index.storage.storage,
2190                    store_raw,
2191                    to_raw_first,
2192                )
2193                .into_account_map_entry(&index.storage.storage);
2194                assert_eq!(new_entry.ref_count(), 1);
2195                assert_eq!(new_entry.slot_list_lock_read_len(), 1);
2196                assert_eq!(
2197                    new_entry.slot_list_read_lock().to_vec(),
2198                    vec![(slot, account_info)]
2199                );
2200            }
2201        }
2202    }
2203
2204    #[test]
2205    fn test_batch_insert() {
2206        let slot0 = 0;
2207        let key0 = solana_pubkey::new_rand();
2208        let key1 = solana_pubkey::new_rand();
2209
2210        let index = AccountsIndex::<bool, bool>::default_for_tests();
2211        let account_infos = [true, false];
2212
2213        index.set_startup(Startup::Startup);
2214        let items = vec![(key0, account_infos[0]), (key1, account_infos[1])];
2215        let expected_len = items.len();
2216        let (_, result) = index.insert_new_if_missing_into_primary_index(slot0, items);
2217        assert_eq!(result.count, expected_len);
2218        index.set_startup(Startup::Normal);
2219
2220        for (i, key) in [key0, key1].iter().enumerate() {
2221            index.get_and_then(key, |entry| {
2222                assert_eq!(entry.unwrap().ref_count(), 1);
2223                assert_eq!(
2224                    entry.unwrap().slot_list_read_lock().as_ref(),
2225                    &[(slot0, account_infos[i])],
2226                );
2227                (false, ())
2228            });
2229        }
2230    }
2231
2232    fn test_new_entry_code_paths_helper<T: IndexValue>(
2233        account_infos: [T; 2],
2234        is_cached: bool,
2235        upsert_method: Option<UpsertReclaim>,
2236        use_disk: bool,
2237    ) {
2238        if is_cached && upsert_method.is_none() {
2239            // This is an illegal combination when we are using queued lazy inserts.
2240            // Cached items don't ever leave the in-mem cache.
2241            // But the queued lazy insert code relies on there being nothing in the in-mem cache.
2242            return;
2243        }
2244
2245        let slot0 = 0;
2246        let slot1 = 1;
2247        let key = solana_pubkey::new_rand();
2248
2249        let mut config = ACCOUNTS_INDEX_CONFIG_FOR_TESTING;
2250        config.index_limit_mb = if use_disk {
2251            IndexLimitMb::Minimal
2252        } else {
2253            IndexLimitMb::InMemOnly // in-mem only
2254        };
2255        let index = AccountsIndex::<T, T>::new(&config, Arc::default());
2256        let mut gc = ReclaimsSlotList::new();
2257
2258        match upsert_method {
2259            Some(upsert_method) => {
2260                // insert first entry for pubkey. This will use new_entry_after_update and not call update.
2261                index.upsert(
2262                    slot0,
2263                    slot0,
2264                    &key,
2265                    &AccountSharedData::default(),
2266                    &AccountSecondaryIndexes::default(),
2267                    account_infos[0],
2268                    &mut gc,
2269                    upsert_method,
2270                );
2271            }
2272            None => {
2273                let items = vec![(key, account_infos[0])];
2274                index.set_startup(Startup::Startup);
2275                let expected_len = items.len();
2276                let (_, result) = index.insert_new_if_missing_into_primary_index(slot0, items);
2277                assert_eq!(result.count, expected_len);
2278                index.set_startup(Startup::Normal);
2279            }
2280        }
2281        assert!(gc.is_empty());
2282
2283        // verify the added entry matches expected
2284        index.get_and_then(&key, |entry| {
2285            let entry = entry.unwrap();
2286            let slot_list = entry.slot_list_read_lock();
2287            assert_eq!(entry.ref_count(), RefCount::from(!is_cached));
2288            assert_eq!(slot_list.as_ref(), &[(slot0, account_infos[0])]);
2289            let new_entry = PreAllocatedAccountMapEntry::new(
2290                slot0,
2291                account_infos[0],
2292                &index.storage.storage,
2293                false,
2294            )
2295            .into_account_map_entry(&index.storage.storage);
2296            assert_eq!(slot_list.as_ref(), new_entry.slot_list_read_lock().as_ref(),);
2297            (false, ())
2298        });
2299
2300        match upsert_method {
2301            Some(upsert_method) => {
2302                // insert second entry for pubkey. This will use update and NOT use new_entry_after_update.
2303                index.upsert(
2304                    slot1,
2305                    slot1,
2306                    &key,
2307                    &AccountSharedData::default(),
2308                    &AccountSecondaryIndexes::default(),
2309                    account_infos[1],
2310                    &mut gc,
2311                    upsert_method,
2312                );
2313            }
2314            None => {
2315                // this has the effect of aging out everything in the in-mem cache
2316                for _ in 0..5 {
2317                    index.set_startup(Startup::Startup);
2318                    index.set_startup(Startup::Normal);
2319                }
2320
2321                let items = vec![(key, account_infos[1])];
2322                index.set_startup(Startup::Startup);
2323                let expected_len = items.len();
2324                let (_, result) = index.insert_new_if_missing_into_primary_index(slot1, items);
2325                assert_eq!(result.count, expected_len);
2326                index.set_startup(Startup::Normal);
2327            }
2328        }
2329
2330        // There should be reclaims if entries are uncached and old slots are being reclaimed
2331        let should_have_reclaims =
2332            upsert_method == Some(UpsertReclaim::ReclaimOldSlots) && !is_cached;
2333
2334        if should_have_reclaims {
2335            assert!(!gc.is_empty());
2336            assert_eq!(gc.len(), 1);
2337            assert_eq!(gc[0], (slot0, account_infos[0]));
2338        } else {
2339            assert!(gc.is_empty());
2340        }
2341
2342        index.populate_and_retrieve_duplicate_keys_from_startup(|_slot_keys| {});
2343
2344        let last_item = index.get_and_then(&key, |entry| {
2345            let entry = entry.unwrap();
2346            let slot_list = entry.slot_list_read_lock();
2347
2348            if should_have_reclaims {
2349                assert_eq!(entry.ref_count(), 1);
2350                assert_eq!(slot_list.as_ref(), &[(slot1, account_infos[1])],);
2351            } else {
2352                assert_eq!(entry.ref_count(), if is_cached { 0 } else { 2 });
2353                assert_eq!(
2354                    slot_list.as_ref(),
2355                    &[(slot0, account_infos[0]), (slot1, account_infos[1])],
2356                );
2357            }
2358            (false, *slot_list.last().unwrap())
2359        });
2360
2361        let new_entry = PreAllocatedAccountMapEntry::new(
2362            slot1,
2363            account_infos[1],
2364            &index.storage.storage,
2365            false,
2366        );
2367
2368        assert_eq!(last_item, new_entry.into());
2369    }
2370
2371    #[test_matrix(
2372        [false, true],
2373        [None, Some(UpsertReclaim::PopulateReclaims), Some(UpsertReclaim::ReclaimOldSlots)],
2374        [true, false]
2375    )]
2376    fn test_new_entry_and_update_code_paths(
2377        use_disk: bool,
2378        upsert_method: Option<UpsertReclaim>,
2379        is_cached: bool,
2380    ) {
2381        if is_cached {
2382            // account_info type that IS cached
2383            test_new_entry_code_paths_helper([1.0, 2.0], true, upsert_method, use_disk);
2384        } else {
2385            // account_info type that is NOT cached
2386            test_new_entry_code_paths_helper([1, 2], false, upsert_method, use_disk);
2387        }
2388    }
2389
2390    #[test]
2391    fn test_insert_with_lock_no_ancestors() {
2392        let key = solana_pubkey::new_rand();
2393        let index = AccountsIndex::<bool, bool>::default_for_tests();
2394        let slot = 0;
2395        let account_info = true;
2396
2397        let new_entry =
2398            PreAllocatedAccountMapEntry::new(slot, account_info, &index.storage.storage, false);
2399        assert_eq!(0, account_maps_stats_len(&index));
2400
2401        assert_eq!(0, account_maps_stats_len(&index));
2402        let r_account_maps = index.get_bin(&key);
2403        r_account_maps.upsert(
2404            &key,
2405            new_entry,
2406            None,
2407            &mut ReclaimsSlotList::default(),
2408            UPSERT_RECLAIM_TEST_DEFAULT,
2409        );
2410        assert_eq!(1, account_maps_stats_len(&index));
2411
2412        let mut ancestors = Ancestors::default();
2413        assert!(!index.contains_with(&key, Some(&ancestors), None));
2414        assert!(!index.contains_with(&key, None, None));
2415        index.get_and_then(&key, |entry| {
2416            let (stored_slot, value) = entry.unwrap().slot_list_read_lock()[0];
2417            assert_eq!(stored_slot, slot);
2418            assert_eq!(value, account_info);
2419            (false, ())
2420        });
2421
2422        let mut num = 0;
2423        index
2424            .scan_accounts(
2425                &ancestors,
2426                0,
2427                |_pubkey, _index| num += 1,
2428                &ScanConfig::default(),
2429            )
2430            .expect("scan should succeed");
2431        assert_eq!(num, 0);
2432        ancestors.insert(slot, 0);
2433        assert!(index.contains_with(&key, Some(&ancestors), None));
2434        index
2435            .scan_accounts(
2436                &ancestors,
2437                0,
2438                |_pubkey, _index| num += 1,
2439                &ScanConfig::default(),
2440            )
2441            .expect("scan should succeed");
2442        assert_eq!(num, 1);
2443    }
2444
2445    #[test]
2446    fn test_insert_wrong_ancestors() {
2447        let key = solana_pubkey::new_rand();
2448        let index = AccountsIndex::<bool, bool>::default_for_tests();
2449        let mut gc = ReclaimsSlotList::new();
2450        index.upsert(
2451            0,
2452            0,
2453            &key,
2454            &AccountSharedData::default(),
2455            &AccountSecondaryIndexes::default(),
2456            true,
2457            &mut gc,
2458            UPSERT_RECLAIM_TEST_DEFAULT,
2459        );
2460        assert!(gc.is_empty());
2461
2462        let ancestors = vec![(1, 1)].into_iter().collect();
2463        assert!(!index.contains_with(&key, Some(&ancestors), None));
2464
2465        let mut num = 0;
2466        index
2467            .scan_accounts(
2468                &ancestors,
2469                0,
2470                |_pubkey, _index| num += 1,
2471                &ScanConfig::default(),
2472            )
2473            .expect("scan should succeed");
2474        assert_eq!(num, 0);
2475    }
2476    #[test]
2477    fn test_insert_ignore_reclaims() {
2478        {
2479            // non-cached
2480            let key = solana_pubkey::new_rand();
2481            let index = AccountsIndex::<u64, u64>::default_for_tests();
2482            let mut reclaims = ReclaimsSlotList::new();
2483            let slot = 0;
2484            let value = 1;
2485            assert!(!value.is_cached());
2486            index.upsert(
2487                slot,
2488                slot,
2489                &key,
2490                &AccountSharedData::default(),
2491                &AccountSecondaryIndexes::default(),
2492                value,
2493                &mut reclaims,
2494                UpsertReclaim::PopulateReclaims,
2495            );
2496            assert!(reclaims.is_empty());
2497            index.upsert(
2498                slot,
2499                slot,
2500                &key,
2501                &AccountSharedData::default(),
2502                &AccountSecondaryIndexes::default(),
2503                value,
2504                &mut reclaims,
2505                UpsertReclaim::PopulateReclaims,
2506            );
2507            // reclaimed
2508            assert!(!reclaims.is_empty());
2509            reclaims.clear();
2510            index.upsert(
2511                slot,
2512                slot,
2513                &key,
2514                &AccountSharedData::default(),
2515                &AccountSecondaryIndexes::default(),
2516                value,
2517                &mut reclaims,
2518                // since IgnoreReclaims, we should expect reclaims to be empty
2519                UpsertReclaim::IgnoreReclaims,
2520            );
2521            // reclaims is ignored
2522            assert!(reclaims.is_empty());
2523        }
2524        {
2525            // cached
2526            let key = solana_pubkey::new_rand();
2527            let index = AccountsIndex::<AccountInfoTest, AccountInfoTest>::default_for_tests();
2528            let mut reclaims = ReclaimsSlotList::new();
2529            let slot = 0;
2530            let value = 1.0;
2531            assert!(value.is_cached());
2532            index.upsert(
2533                slot,
2534                slot,
2535                &key,
2536                &AccountSharedData::default(),
2537                &AccountSecondaryIndexes::default(),
2538                value,
2539                &mut reclaims,
2540                UpsertReclaim::PopulateReclaims,
2541            );
2542            assert!(reclaims.is_empty());
2543            index.upsert(
2544                slot,
2545                slot,
2546                &key,
2547                &AccountSharedData::default(),
2548                &AccountSecondaryIndexes::default(),
2549                value,
2550                &mut reclaims,
2551                UpsertReclaim::PopulateReclaims,
2552            );
2553            // No reclaims, since the entry replaced was cached
2554            assert!(reclaims.is_empty());
2555            index.upsert(
2556                slot,
2557                slot,
2558                &key,
2559                &AccountSharedData::default(),
2560                &AccountSecondaryIndexes::default(),
2561                value,
2562                &mut reclaims,
2563                // since IgnoreReclaims, we should expect reclaims to be empty
2564                UpsertReclaim::IgnoreReclaims,
2565            );
2566            // reclaims is ignored
2567            assert!(reclaims.is_empty());
2568        }
2569    }
2570
2571    #[test]
2572    fn test_insert_with_ancestors() {
2573        let key = solana_pubkey::new_rand();
2574        let index = AccountsIndex::<bool, bool>::default_for_tests();
2575        let mut gc = ReclaimsSlotList::new();
2576        index.upsert(
2577            0,
2578            0,
2579            &key,
2580            &AccountSharedData::default(),
2581            &AccountSecondaryIndexes::default(),
2582            true,
2583            &mut gc,
2584            UPSERT_RECLAIM_TEST_DEFAULT,
2585        );
2586        assert!(gc.is_empty());
2587
2588        let ancestors = vec![(0, 0)].into_iter().collect();
2589        index
2590            .get_with_and_then(
2591                &key,
2592                Some(&ancestors),
2593                None,
2594                false,
2595                |(slot, account_info)| {
2596                    assert_eq!(slot, 0);
2597                    assert!(account_info);
2598                },
2599            )
2600            .unwrap();
2601
2602        let mut num = 0;
2603        let mut found_key = false;
2604        index
2605            .scan_accounts(
2606                &ancestors,
2607                0,
2608                |pubkey, _index| {
2609                    if pubkey == &key {
2610                        found_key = true
2611                    };
2612                    num += 1
2613                },
2614                &ScanConfig::default(),
2615            )
2616            .expect("scan should succeed");
2617
2618        assert_eq!(num, 1);
2619        assert!(found_key);
2620    }
2621
2622    fn setup_accounts_index_keys(num_pubkeys: usize) -> (AccountsIndex<bool, bool>, Vec<Pubkey>) {
2623        let index = AccountsIndex::<bool, bool>::default_for_tests();
2624        let root_slot = 0;
2625
2626        let mut pubkeys: Vec<Pubkey> = std::iter::repeat_with(|| {
2627            let new_pubkey = solana_pubkey::new_rand();
2628            index.upsert(
2629                root_slot,
2630                root_slot,
2631                &new_pubkey,
2632                &AccountSharedData::default(),
2633                &AccountSecondaryIndexes::default(),
2634                true,
2635                &mut ReclaimsSlotList::new(),
2636                UPSERT_RECLAIM_TEST_DEFAULT,
2637            );
2638            new_pubkey
2639        })
2640        .take(num_pubkeys.saturating_sub(1))
2641        .collect();
2642
2643        if num_pubkeys != 0 {
2644            pubkeys.push(Pubkey::default());
2645            index.upsert(
2646                root_slot,
2647                root_slot,
2648                &Pubkey::default(),
2649                &AccountSharedData::default(),
2650                &AccountSecondaryIndexes::default(),
2651                true,
2652                &mut ReclaimsSlotList::new(),
2653                UPSERT_RECLAIM_TEST_DEFAULT,
2654            );
2655        }
2656
2657        index.add_root(root_slot);
2658
2659        (index, pubkeys)
2660    }
2661
2662    fn run_test_scan_accounts(num_pubkeys: usize) {
2663        let (index, _) = setup_accounts_index_keys(num_pubkeys);
2664
2665        let mut scanned_keys = HashSet::new();
2666        index
2667            .scan_accounts(
2668                &Ancestors::default(),
2669                0,
2670                |pubkey, _index| {
2671                    scanned_keys.insert(*pubkey);
2672                },
2673                &ScanConfig::default(),
2674            )
2675            .expect("scan should succeed");
2676        assert_eq!(scanned_keys.len(), num_pubkeys);
2677    }
2678
2679    #[test]
2680    fn test_scan_accounts() {
2681        run_test_scan_accounts(0);
2682        run_test_scan_accounts(1);
2683        run_test_scan_accounts(ITER_BATCH_SIZE * 10);
2684        run_test_scan_accounts(ITER_BATCH_SIZE * 10 - 1);
2685        run_test_scan_accounts(ITER_BATCH_SIZE * 10 + 1);
2686    }
2687
2688    #[test]
2689    fn test_is_alive_root() {
2690        let index = AccountsIndex::<bool, bool>::default_for_tests();
2691        assert!(!index.is_alive_root(0));
2692        index.add_root(0);
2693        assert!(index.is_alive_root(0));
2694    }
2695
2696    #[test]
2697    fn test_insert_with_root() {
2698        let key = solana_pubkey::new_rand();
2699        let index = AccountsIndex::<bool, bool>::default_for_tests();
2700        let mut gc = ReclaimsSlotList::new();
2701        index.upsert(
2702            0,
2703            0,
2704            &key,
2705            &AccountSharedData::default(),
2706            &AccountSecondaryIndexes::default(),
2707            true,
2708            &mut gc,
2709            UPSERT_RECLAIM_TEST_DEFAULT,
2710        );
2711        assert!(gc.is_empty());
2712
2713        index.add_root(0);
2714        index
2715            .get_with_and_then(&key, None, None, false, |(slot, account_info)| {
2716                assert_eq!(slot, 0);
2717                assert!(account_info);
2718            })
2719            .unwrap();
2720    }
2721
2722    #[test]
2723    fn test_clean_first() {
2724        let index = AccountsIndex::<bool, bool>::default_for_tests();
2725        index.add_root(0);
2726        index.add_root(1);
2727        index.clean_dead_slot(0);
2728        assert!(index.is_alive_root(1));
2729        assert!(!index.is_alive_root(0));
2730    }
2731
2732    #[test]
2733    fn test_clean_last() {
2734        //this behavior might be undefined, clean up should only occur on older slots
2735        let index = AccountsIndex::<bool, bool>::default_for_tests();
2736        index.add_root(0);
2737        index.add_root(1);
2738        index.clean_dead_slot(1);
2739        assert!(!index.is_alive_root(1));
2740        assert!(index.is_alive_root(0));
2741    }
2742
2743    #[test]
2744    fn test_update_last_wins() {
2745        let key = solana_pubkey::new_rand();
2746        let index = AccountsIndex::<u64, u64>::default_for_tests();
2747        let ancestors = vec![(0, 0)].into_iter().collect();
2748        let mut gc = ReclaimsSlotList::new();
2749        index.upsert(
2750            0,
2751            0,
2752            &key,
2753            &AccountSharedData::default(),
2754            &AccountSecondaryIndexes::default(),
2755            1,
2756            &mut gc,
2757            UPSERT_RECLAIM_TEST_DEFAULT,
2758        );
2759        assert!(gc.is_empty());
2760        index
2761            .get_with_and_then(
2762                &key,
2763                Some(&ancestors),
2764                None,
2765                false,
2766                |(slot, account_info)| {
2767                    assert_eq!(slot, 0);
2768                    assert_eq!(account_info, 1);
2769                },
2770            )
2771            .unwrap();
2772
2773        let mut gc = ReclaimsSlotList::new();
2774        index.upsert(
2775            0,
2776            0,
2777            &key,
2778            &AccountSharedData::default(),
2779            &AccountSecondaryIndexes::default(),
2780            0,
2781            &mut gc,
2782            UPSERT_RECLAIM_TEST_DEFAULT,
2783        );
2784        assert_eq!(gc, ReclaimsSlotList::from([(0, 1)]));
2785        index
2786            .get_with_and_then(
2787                &key,
2788                Some(&ancestors),
2789                None,
2790                false,
2791                |(slot, account_info)| {
2792                    assert_eq!(slot, 0);
2793                    assert_eq!(account_info, 0);
2794                },
2795            )
2796            .unwrap();
2797    }
2798
2799    #[test]
2800    fn test_update_new_slot() {
2801        agave_logger::setup();
2802        let key = solana_pubkey::new_rand();
2803        let index = AccountsIndex::<bool, bool>::default_for_tests();
2804        let ancestors = vec![(0, 0)].into_iter().collect();
2805        let mut gc = ReclaimsSlotList::new();
2806        index.upsert(
2807            0,
2808            0,
2809            &key,
2810            &AccountSharedData::default(),
2811            &AccountSecondaryIndexes::default(),
2812            true,
2813            &mut gc,
2814            UpsertReclaim::PopulateReclaims,
2815        );
2816        assert!(gc.is_empty());
2817        index.upsert(
2818            1,
2819            1,
2820            &key,
2821            &AccountSharedData::default(),
2822            &AccountSecondaryIndexes::default(),
2823            false,
2824            &mut gc,
2825            UpsertReclaim::PopulateReclaims,
2826        );
2827        assert!(gc.is_empty());
2828        index
2829            .get_with_and_then(
2830                &key,
2831                Some(&ancestors),
2832                None,
2833                false,
2834                |(slot, account_info)| {
2835                    assert_eq!(slot, 0);
2836                    assert!(account_info);
2837                },
2838            )
2839            .unwrap();
2840        let ancestors = vec![(1, 0)].into_iter().collect();
2841        index
2842            .get_with_and_then(
2843                &key,
2844                Some(&ancestors),
2845                None,
2846                false,
2847                |(slot, account_info)| {
2848                    assert_eq!(slot, 1);
2849                    assert!(!account_info);
2850                },
2851            )
2852            .unwrap();
2853    }
2854
2855    #[test]
2856    fn test_update_gc_purged_slot() {
2857        let key = solana_pubkey::new_rand();
2858        let index = AccountsIndex::<bool, bool>::default_for_tests();
2859        let mut gc = ReclaimsSlotList::new();
2860        index.upsert(
2861            0,
2862            0,
2863            &key,
2864            &AccountSharedData::default(),
2865            &AccountSecondaryIndexes::default(),
2866            true,
2867            &mut gc,
2868            UpsertReclaim::PopulateReclaims,
2869        );
2870        assert!(gc.is_empty());
2871        index.upsert(
2872            1,
2873            1,
2874            &key,
2875            &AccountSharedData::default(),
2876            &AccountSecondaryIndexes::default(),
2877            false,
2878            &mut gc,
2879            UpsertReclaim::PopulateReclaims,
2880        );
2881        index.upsert(
2882            2,
2883            2,
2884            &key,
2885            &AccountSharedData::default(),
2886            &AccountSecondaryIndexes::default(),
2887            true,
2888            &mut gc,
2889            UpsertReclaim::PopulateReclaims,
2890        );
2891        index.upsert(
2892            3,
2893            3,
2894            &key,
2895            &AccountSharedData::default(),
2896            &AccountSecondaryIndexes::default(),
2897            true,
2898            &mut gc,
2899            UpsertReclaim::PopulateReclaims,
2900        );
2901        index.add_root(0);
2902        index.add_root(1);
2903        index.add_root(3);
2904        index.upsert(
2905            4,
2906            4,
2907            &key,
2908            &AccountSharedData::default(),
2909            &AccountSecondaryIndexes::default(),
2910            true,
2911            &mut gc,
2912            UpsertReclaim::PopulateReclaims,
2913        );
2914
2915        // Updating index should not purge older roots, only purges
2916        // previous updates within the same slot
2917        assert_eq!(gc, ReclaimsSlotList::new());
2918        index
2919            .get_with_and_then(&key, None, None, false, |(slot, account_info)| {
2920                assert_eq!(slot, 3);
2921                assert!(account_info);
2922            })
2923            .unwrap();
2924
2925        let mut num = 0;
2926        let mut found_key = false;
2927        index
2928            .scan_accounts(
2929                &Ancestors::default(),
2930                0,
2931                |pubkey, index| {
2932                    if pubkey == &key {
2933                        found_key = true;
2934                        assert_eq!(index, (&true, 3));
2935                    };
2936                    num += 1
2937                },
2938                &ScanConfig::default(),
2939            )
2940            .expect("scan should succeed");
2941        assert_eq!(num, 1);
2942        assert!(found_key);
2943    }
2944
2945    #[test]
2946    fn test_upsert_reclaims() {
2947        let key = solana_pubkey::new_rand();
2948        let index =
2949            AccountsIndex::<CacheableIndexValueTest, CacheableIndexValueTest>::default_for_tests();
2950        let mut reclaims = ReclaimsSlotList::new();
2951        index.upsert(
2952            0,
2953            0,
2954            &key,
2955            &AccountSharedData::default(),
2956            &AccountSecondaryIndexes::default(),
2957            CacheableIndexValueTest(true),
2958            &mut reclaims,
2959            UPSERT_RECLAIM_TEST_DEFAULT,
2960        );
2961        // No reclaims should be returned on the first item
2962        assert!(reclaims.is_empty());
2963
2964        index.upsert(
2965            0,
2966            0,
2967            &key,
2968            &AccountSharedData::default(),
2969            &AccountSecondaryIndexes::default(),
2970            CacheableIndexValueTest(false),
2971            &mut reclaims,
2972            UPSERT_RECLAIM_TEST_DEFAULT,
2973        );
2974        // Cached item should not be reclaimed
2975        assert!(reclaims.is_empty());
2976
2977        // Slot list should only have a single entry
2978        let slot_list_len = index.get_and_then(&key, |entry| {
2979            (false, entry.unwrap().slot_list_lock_read_len())
2980        });
2981        assert_eq!(slot_list_len, 1);
2982
2983        index.upsert(
2984            0,
2985            0,
2986            &key,
2987            &AccountSharedData::default(),
2988            &AccountSecondaryIndexes::default(),
2989            CacheableIndexValueTest(false),
2990            &mut reclaims,
2991            UPSERT_RECLAIM_TEST_DEFAULT,
2992        );
2993
2994        // Uncached item should be returned as reclaim
2995        assert!(!reclaims.is_empty());
2996
2997        // Slot list should only have a single entry
2998        let slot_list_len = index.get_and_then(&key, |entry| {
2999            (false, entry.unwrap().slot_list_lock_read_len())
3000        });
3001        assert_eq!(slot_list_len, 1);
3002    }
3003
3004    fn account_maps_stats_len<T: IndexValue>(index: &AccountsIndex<T, T>) -> usize {
3005        index.storage.storage.stats.total_count()
3006    }
3007
3008    #[test]
3009    fn test_purge() {
3010        let key = solana_pubkey::new_rand();
3011        let index = AccountsIndex::<u64, u64>::default_for_tests();
3012        let mut gc = ReclaimsSlotList::new();
3013        assert_eq!(0, account_maps_stats_len(&index));
3014        index.upsert(
3015            1,
3016            1,
3017            &key,
3018            &AccountSharedData::default(),
3019            &AccountSecondaryIndexes::default(),
3020            12,
3021            &mut gc,
3022            UPSERT_RECLAIM_TEST_DEFAULT,
3023        );
3024        assert_eq!(1, account_maps_stats_len(&index));
3025
3026        index.upsert(
3027            1,
3028            1,
3029            &key,
3030            &AccountSharedData::default(),
3031            &AccountSecondaryIndexes::default(),
3032            10,
3033            &mut gc,
3034            UPSERT_RECLAIM_TEST_DEFAULT,
3035        );
3036        assert_eq!(1, account_maps_stats_len(&index));
3037
3038        let purges = index.purge_roots(&key);
3039        assert_eq!(purges, (SlotList::new(), false));
3040        index.add_root(1);
3041
3042        let purges = index.purge_roots(&key);
3043        assert_eq!(purges, (SlotList::from([(1, 10)]), true));
3044
3045        assert_eq!(1, account_maps_stats_len(&index));
3046        index.upsert(
3047            1,
3048            1,
3049            &key,
3050            &AccountSharedData::default(),
3051            &AccountSecondaryIndexes::default(),
3052            9,
3053            &mut gc,
3054            UPSERT_RECLAIM_TEST_DEFAULT,
3055        );
3056        assert_eq!(1, account_maps_stats_len(&index));
3057    }
3058
3059    #[test]
3060    fn test_latest_slot() {
3061        let slot_slice = vec![(0, true), (5, true), (3, true), (7, true)];
3062        let index = AccountsIndex::<bool, bool>::default_for_tests();
3063
3064        // No ancestors, no root, should return None
3065        assert!(index.latest_slot(None, &slot_slice, None).is_none());
3066
3067        // Given a root, should return the root
3068        index.add_root(5);
3069        assert_eq!(index.latest_slot(None, &slot_slice, None).unwrap(), 1);
3070
3071        // Given a max_root == root, should still return the root
3072        assert_eq!(index.latest_slot(None, &slot_slice, Some(5)).unwrap(), 1);
3073
3074        // Given a max_root < root, should filter out the root
3075        assert!(index.latest_slot(None, &slot_slice, Some(4)).is_none());
3076
3077        // Given a max_root, should filter out roots < max_root, but specified
3078        // ancestors should not be affected
3079        let ancestors = vec![(3, 1), (7, 1)].into_iter().collect();
3080        assert_eq!(
3081            index
3082                .latest_slot(Some(&ancestors), &slot_slice, Some(4))
3083                .unwrap(),
3084            3
3085        );
3086        assert_eq!(
3087            index
3088                .latest_slot(Some(&ancestors), &slot_slice, Some(7))
3089                .unwrap(),
3090            3
3091        );
3092
3093        // Given no max_root, should just return the greatest ancestor or root
3094        assert_eq!(
3095            index
3096                .latest_slot(Some(&ancestors), &slot_slice, None)
3097                .unwrap(),
3098            3
3099        );
3100    }
3101
3102    fn make_empty_token_account_data() -> Vec<u8> {
3103        const SPL_TOKEN_INITIALIZED_OFFSET: usize = 108;
3104        let mut data = vec![0; spl_generic_token::token::Account::get_packed_len()];
3105        data[SPL_TOKEN_INITIALIZED_OFFSET] = 1;
3106        data
3107    }
3108
3109    fn run_test_purge_exact_secondary_index<
3110        SecondaryIndexEntryType: SecondaryIndexEntry + Default + Sync + Send,
3111    >(
3112        index: &AccountsIndex<bool, bool>,
3113        secondary_index: &SecondaryIndex<SecondaryIndexEntryType>,
3114        key_start: usize,
3115        key_end: usize,
3116        secondary_indexes: &AccountSecondaryIndexes,
3117    ) {
3118        // No roots, should be no reclaims
3119        let slots = vec![1, 2, 5, 9];
3120        let index_key = Pubkey::new_unique();
3121        let account_key = Pubkey::new_unique();
3122
3123        let mut account_data = make_empty_token_account_data();
3124        account_data[key_start..key_end].clone_from_slice(&(index_key.to_bytes()));
3125
3126        // Insert slots into secondary index
3127        for slot in &slots {
3128            index.upsert(
3129                *slot,
3130                *slot,
3131                &account_key,
3132                // Make sure these accounts are added to secondary index
3133                &AccountSharedData::create(
3134                    0,
3135                    account_data.to_vec(),
3136                    spl_generic_token::token::id(),
3137                    false,
3138                    0,
3139                ),
3140                secondary_indexes,
3141                true,
3142                &mut ReclaimsSlotList::new(),
3143                UPSERT_RECLAIM_TEST_DEFAULT,
3144            );
3145        }
3146
3147        // Only one top level index entry exists
3148        assert_eq!(secondary_index.index.get(&index_key).unwrap().len(), 1);
3149
3150        // In the reverse index, one account maps across multiple slots
3151        // to the same top level key
3152        assert_eq!(
3153            secondary_index
3154                .reverse_index
3155                .get(&account_key)
3156                .unwrap()
3157                .value()
3158                .read()
3159                .unwrap()
3160                .len(),
3161            1
3162        );
3163
3164        index.purge_exact(
3165            &account_key,
3166            slots.into_iter().collect::<HashSet<Slot>>(),
3167            &mut ReclaimsSlotList::new(),
3168        );
3169
3170        let _ = index.handle_dead_keys(&[account_key], secondary_indexes);
3171        assert!(secondary_index.index.is_empty());
3172        assert!(secondary_index.reverse_index.is_empty());
3173    }
3174
3175    #[test]
3176    fn test_reclaim_older_items_in_slot_list() {
3177        agave_logger::setup();
3178        let key = solana_pubkey::new_rand();
3179        let index = AccountsIndex::<u64, u64>::default_for_tests();
3180        let mut gc = ReclaimsSlotList::new();
3181        let reclaim_slot = 5;
3182        let account_value = 50;
3183
3184        // Insert multiple older items into the slot list
3185        for slot in 0..reclaim_slot {
3186            index.upsert(
3187                slot,
3188                slot,
3189                &key,
3190                &AccountSharedData::default(),
3191                &AccountSecondaryIndexes::default(),
3192                slot,
3193                &mut gc,
3194                UpsertReclaim::IgnoreReclaims,
3195            );
3196        }
3197        let slot_list_len = index.get_and_then(&key, |entry| {
3198            (false, entry.unwrap().slot_list_lock_read_len())
3199        });
3200        assert_eq!(slot_list_len, reclaim_slot as usize);
3201
3202        // Insert an item newer than the one that we will reclaim old slots on
3203        index.upsert(
3204            reclaim_slot + 1,
3205            reclaim_slot + 1,
3206            &key,
3207            &AccountSharedData::default(),
3208            &AccountSecondaryIndexes::default(),
3209            account_value + 1,
3210            &mut gc,
3211            UpsertReclaim::IgnoreReclaims,
3212        );
3213        let slot_list_len = index.get_and_then(&key, |entry| {
3214            (false, entry.unwrap().slot_list_lock_read_len())
3215        });
3216        assert_eq!(slot_list_len, (reclaim_slot + 1) as usize);
3217
3218        // Reclaim all older slots
3219        index.upsert(
3220            reclaim_slot,
3221            reclaim_slot,
3222            &key,
3223            &AccountSharedData::default(),
3224            &AccountSecondaryIndexes::default(),
3225            account_value,
3226            &mut gc,
3227            UpsertReclaim::ReclaimOldSlots,
3228        );
3229
3230        // Verify that older items are reclaimed
3231        assert_eq!(gc.len(), reclaim_slot as usize);
3232        for (slot, value) in gc.iter() {
3233            assert!(*slot < reclaim_slot);
3234            assert_eq!(*value, *slot);
3235        }
3236
3237        // Verify that the item added is in in the slot list
3238        let ancestors = vec![(reclaim_slot, 0)].into_iter().collect();
3239        index
3240            .get_with_and_then(
3241                &key,
3242                Some(&ancestors),
3243                None,
3244                false,
3245                |(slot, account_info)| {
3246                    assert_eq!(slot, reclaim_slot);
3247                    assert_eq!(account_info, account_value);
3248                },
3249            )
3250            .unwrap();
3251
3252        // Verify that the newer item remains in the slot list
3253        let ancestors = vec![((reclaim_slot + 1), 0)].into_iter().collect();
3254        index
3255            .get_with_and_then(
3256                &key,
3257                Some(&ancestors),
3258                None,
3259                false,
3260                |(slot, account_info)| {
3261                    assert_eq!(slot, reclaim_slot + 1);
3262                    assert_eq!(account_info, account_value + 1);
3263                },
3264            )
3265            .unwrap();
3266    }
3267
3268    #[test]
3269    fn test_reclaim_do_not_reclaim_cached_other_slot() {
3270        agave_logger::setup();
3271        let key = solana_pubkey::new_rand();
3272        let index =
3273            AccountsIndex::<CacheableIndexValueTest, CacheableIndexValueTest>::default_for_tests();
3274        let mut gc = ReclaimsSlotList::new();
3275
3276        // Insert an uncached account at slot 0 and an cached account at slot 1
3277        index.upsert(
3278            0,
3279            0,
3280            &key,
3281            &AccountSharedData::default(),
3282            &AccountSecondaryIndexes::default(),
3283            CacheableIndexValueTest(false),
3284            &mut gc,
3285            UpsertReclaim::IgnoreReclaims,
3286        );
3287
3288        index.upsert(
3289            1,
3290            1,
3291            &key,
3292            &AccountSharedData::default(),
3293            &AccountSecondaryIndexes::default(),
3294            CacheableIndexValueTest(true),
3295            &mut gc,
3296            UpsertReclaim::IgnoreReclaims,
3297        );
3298
3299        // Now insert a cached account at slot 2
3300        index.upsert(
3301            2,
3302            2,
3303            &key,
3304            &AccountSharedData::default(),
3305            &AccountSecondaryIndexes::default(),
3306            CacheableIndexValueTest(true),
3307            &mut gc,
3308            UpsertReclaim::IgnoreReclaims,
3309        );
3310
3311        // Replace the cached account at slot 2 with a uncached account
3312        index.upsert(
3313            2,
3314            2,
3315            &key,
3316            &AccountSharedData::default(),
3317            &AccountSecondaryIndexes::default(),
3318            CacheableIndexValueTest(false),
3319            &mut gc,
3320            UpsertReclaim::ReclaimOldSlots,
3321        );
3322
3323        // Verify that the slot list is length two and consists of the cached account at slot 1
3324        // and the uncached account at slot 2
3325        index.get_and_then(&key, |entry| {
3326            let entry = entry.unwrap();
3327            assert_eq!(entry.slot_list_lock_read_len(), 2);
3328            assert_eq!(
3329                entry.slot_list_read_lock()[0],
3330                PreAllocatedAccountMapEntry::new(
3331                    1,
3332                    CacheableIndexValueTest(true),
3333                    &index.storage.storage,
3334                    false
3335                )
3336                .into()
3337            );
3338            assert_eq!(
3339                entry.slot_list_read_lock()[1],
3340                PreAllocatedAccountMapEntry::new(
3341                    2,
3342                    CacheableIndexValueTest(false),
3343                    &index.storage.storage,
3344                    false
3345                )
3346                .into()
3347            );
3348            // Verify that the uncached account at slot 0 was reclaimed
3349            assert_eq!(gc.len(), 1);
3350            assert_eq!(gc[0], (0, CacheableIndexValueTest(false)));
3351            (false, ())
3352        });
3353    }
3354
3355    #[test]
3356    fn test_purge_exact_spl_token_mint_secondary_index() {
3357        let (key_start, key_end, secondary_indexes) = create_spl_token_mint_secondary_index_state();
3358        let index = AccountsIndex::<bool, bool>::default_for_tests();
3359        run_test_purge_exact_secondary_index(
3360            &index,
3361            &index.spl_token_mint_index,
3362            key_start,
3363            key_end,
3364            &secondary_indexes,
3365        );
3366    }
3367
3368    #[test]
3369    fn test_purge_exact_spl_token_owner_secondary_index() {
3370        let (key_start, key_end, secondary_indexes) =
3371            create_spl_token_owner_secondary_index_state();
3372        let index = AccountsIndex::<bool, bool>::default_for_tests();
3373        run_test_purge_exact_secondary_index(
3374            &index,
3375            &index.spl_token_owner_index,
3376            key_start,
3377            key_end,
3378            &secondary_indexes,
3379        );
3380    }
3381
3382    #[test]
3383    fn test_purge_older_root_entries() {
3384        // No roots, should be no reclaims
3385        let index = AccountsIndex::<bool, bool>::default_for_tests();
3386        let entry = AccountMapEntry::new(
3387            SlotList::from_iter([(1, true), (2, true), (5, true), (9, true)]),
3388            1,
3389            AccountMapEntryMeta::default(),
3390        );
3391        let mut slot_list = entry.slot_list_write_lock();
3392        let mut reclaims = ReclaimsSlotList::new();
3393        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, None));
3394        assert!(reclaims.is_empty());
3395        assert_eq!(
3396            slot_list.clone_list(),
3397            SlotList::from_iter([(1, true), (2, true), (5, true), (9, true)])
3398        );
3399
3400        // Add a later root, earlier slots should be reclaimed
3401        slot_list.assign([(1, true), (2, true), (5, true), (9, true)]);
3402        index.add_root(1);
3403        // Note 2 is not a root
3404        index.add_root(5);
3405        reclaims = ReclaimsSlotList::new();
3406        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, None));
3407        assert_eq!(reclaims, ReclaimsSlotList::from([(1, true), (2, true)]));
3408        assert_eq!(
3409            slot_list.clone_list(),
3410            SlotList::from_iter([(5, true), (9, true)])
3411        );
3412        // Add a later root that is not in the list, should not affect the outcome
3413        slot_list.assign([(1 as Slot, true), (2, true), (5, true), (9, true)]);
3414        index.add_root(6);
3415        reclaims = ReclaimsSlotList::new();
3416        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, None));
3417        assert_eq!(reclaims, ReclaimsSlotList::from([(1, true), (2, true)]));
3418        assert_eq!(
3419            slot_list.clone_list(),
3420            SlotList::from_iter([(5, true), (9, true)])
3421        );
3422
3423        // Pass a max root >= than any root in the slot list, should not affect
3424        // outcome
3425        slot_list.assign([(1, true), (2, true), (5, true), (9, true)]);
3426        reclaims = ReclaimsSlotList::new();
3427        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, Some(6)));
3428        assert_eq!(reclaims, ReclaimsSlotList::from([(1, true), (2, true)]));
3429        assert_eq!(
3430            slot_list.clone_list(),
3431            SlotList::from_iter([(5, true), (9, true)])
3432        );
3433
3434        // Pass a max root, earlier slots should be reclaimed
3435        slot_list.assign([(1, true), (2, true), (5, true), (9, true)]);
3436        reclaims = ReclaimsSlotList::new();
3437        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, Some(5)));
3438        assert_eq!(reclaims, ReclaimsSlotList::from([(1, true), (2, true)]));
3439        assert_eq!(
3440            slot_list.clone_list(),
3441            SlotList::from_iter([(5, true), (9, true)])
3442        );
3443
3444        // Pass a max root 2. This means the latest root < 2 is 1 because 2 is not a root
3445        // so nothing will be purged
3446        slot_list.assign([(1, true), (2, true), (5, true), (9, true)]);
3447        reclaims = ReclaimsSlotList::new();
3448        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, Some(2)));
3449        assert!(reclaims.is_empty());
3450        assert_eq!(
3451            slot_list.clone_list(),
3452            SlotList::from_iter([(1, true), (2, true), (5, true), (9, true)])
3453        );
3454
3455        // Pass a max root 1. This means the latest root < 3 is 1 because 2 is not a root
3456        // so nothing will be purged
3457        slot_list.assign([(1, true), (2, true), (5, true), (9, true)]);
3458        reclaims = ReclaimsSlotList::new();
3459        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, Some(1)));
3460        assert!(reclaims.is_empty());
3461        assert_eq!(
3462            slot_list.clone_list(),
3463            SlotList::from_iter([(1, true), (2, true), (5, true), (9, true)])
3464        );
3465
3466        // Pass a max root that doesn't exist in the list but is greater than
3467        // some of the roots in the list, shouldn't return those smaller roots
3468        slot_list.assign([(1, true), (2, true), (5, true), (9, true)]);
3469        reclaims = ReclaimsSlotList::new();
3470        assert!(!index.purge_older_root_entries(&mut slot_list, &mut reclaims, Some(7)));
3471        assert_eq!(reclaims, ReclaimsSlotList::from([(1, true), (2, true)]));
3472        assert_eq!(
3473            slot_list.clone_list(),
3474            SlotList::from_iter([(5, true), (9, true)])
3475        );
3476    }
3477
3478    fn check_secondary_index_mapping_correct<SecondaryIndexEntryType>(
3479        secondary_index: &SecondaryIndex<SecondaryIndexEntryType>,
3480        secondary_index_keys: &[Pubkey],
3481        account_key: &Pubkey,
3482    ) where
3483        SecondaryIndexEntryType: SecondaryIndexEntry + Default + Sync + Send,
3484    {
3485        // Check secondary index has unique mapping from secondary index key
3486        // to the account key and slot
3487        for secondary_index_key in secondary_index_keys {
3488            assert_eq!(secondary_index.index.len(), secondary_index_keys.len());
3489            let account_key_map = secondary_index.get(secondary_index_key);
3490            assert_eq!(account_key_map.len(), 1);
3491            assert_eq!(account_key_map, vec![*account_key]);
3492        }
3493        // Check reverse index contains all of the `secondary_index_keys`
3494        let secondary_index_key_map = secondary_index.reverse_index.get(account_key).unwrap();
3495        assert_eq!(
3496            &*secondary_index_key_map.value().read().unwrap(),
3497            secondary_index_keys
3498        );
3499    }
3500
3501    fn run_test_spl_token_secondary_indexes<
3502        SecondaryIndexEntryType: SecondaryIndexEntry + Default + Sync + Send,
3503    >(
3504        token_id: &Pubkey,
3505        index: &AccountsIndex<bool, bool>,
3506        secondary_index: &SecondaryIndex<SecondaryIndexEntryType>,
3507        key_start: usize,
3508        key_end: usize,
3509        secondary_indexes: &AccountSecondaryIndexes,
3510    ) {
3511        let mut secondary_indexes = secondary_indexes.clone();
3512        let account_key = Pubkey::new_unique();
3513        let index_key = Pubkey::new_unique();
3514        let mut account_data = make_empty_token_account_data();
3515        account_data[key_start..key_end].clone_from_slice(&(index_key.to_bytes()));
3516
3517        // Wrong program id
3518        index.upsert(
3519            0,
3520            0,
3521            &account_key,
3522            &AccountSharedData::create(0, account_data.to_vec(), Pubkey::default(), false, 0),
3523            &secondary_indexes,
3524            true,
3525            &mut ReclaimsSlotList::new(),
3526            UPSERT_RECLAIM_TEST_DEFAULT,
3527        );
3528        assert!(secondary_index.index.is_empty());
3529        assert!(secondary_index.reverse_index.is_empty());
3530
3531        // Wrong account data size
3532        index.upsert(
3533            0,
3534            0,
3535            &account_key,
3536            &AccountSharedData::create(0, account_data[1..].to_vec(), *token_id, false, 0),
3537            &secondary_indexes,
3538            true,
3539            &mut ReclaimsSlotList::new(),
3540            UPSERT_RECLAIM_TEST_DEFAULT,
3541        );
3542        assert!(secondary_index.index.is_empty());
3543        assert!(secondary_index.reverse_index.is_empty());
3544
3545        secondary_indexes.keys = None;
3546
3547        // Just right. Inserting the same index multiple times should be ok
3548        for _ in 0..2 {
3549            index.update_secondary_indexes(
3550                &account_key,
3551                &AccountSharedData::create(0, account_data.to_vec(), *token_id, false, 0),
3552                &secondary_indexes,
3553            );
3554            check_secondary_index_mapping_correct(secondary_index, &[index_key], &account_key);
3555        }
3556
3557        // included
3558        assert!(!secondary_index.index.is_empty());
3559        assert!(!secondary_index.reverse_index.is_empty());
3560
3561        secondary_indexes.keys = Some(AccountSecondaryIndexesIncludeExclude {
3562            keys: [index_key].iter().cloned().collect::<HashSet<_>>(),
3563            exclude: false,
3564        });
3565        secondary_index.index.clear();
3566        secondary_index.reverse_index.clear();
3567        index.update_secondary_indexes(
3568            &account_key,
3569            &AccountSharedData::create(0, account_data.to_vec(), *token_id, false, 0),
3570            &secondary_indexes,
3571        );
3572        assert!(!secondary_index.index.is_empty());
3573        assert!(!secondary_index.reverse_index.is_empty());
3574        check_secondary_index_mapping_correct(secondary_index, &[index_key], &account_key);
3575
3576        // not-excluded
3577        secondary_indexes.keys = Some(AccountSecondaryIndexesIncludeExclude {
3578            keys: [].iter().cloned().collect::<HashSet<_>>(),
3579            exclude: true,
3580        });
3581        secondary_index.index.clear();
3582        secondary_index.reverse_index.clear();
3583        index.update_secondary_indexes(
3584            &account_key,
3585            &AccountSharedData::create(0, account_data.to_vec(), *token_id, false, 0),
3586            &secondary_indexes,
3587        );
3588        assert!(!secondary_index.index.is_empty());
3589        assert!(!secondary_index.reverse_index.is_empty());
3590        check_secondary_index_mapping_correct(secondary_index, &[index_key], &account_key);
3591
3592        secondary_indexes.keys = None;
3593
3594        index.slot_list_mut(&account_key, |mut slot_list| slot_list.clear());
3595
3596        // Everything should be deleted
3597        let _ = index.handle_dead_keys(&[account_key], &secondary_indexes);
3598        assert!(secondary_index.index.is_empty());
3599        assert!(secondary_index.reverse_index.is_empty());
3600    }
3601
3602    #[test]
3603    fn test_spl_token_mint_secondary_index() {
3604        let (key_start, key_end, secondary_indexes) = create_spl_token_mint_secondary_index_state();
3605        let index = AccountsIndex::<bool, bool>::default_for_tests();
3606        for token_id in &spl_token_ids() {
3607            run_test_spl_token_secondary_indexes(
3608                token_id,
3609                &index,
3610                &index.spl_token_mint_index,
3611                key_start,
3612                key_end,
3613                &secondary_indexes,
3614            );
3615        }
3616    }
3617
3618    #[test]
3619    fn test_spl_token_owner_secondary_index() {
3620        let (key_start, key_end, secondary_indexes) =
3621            create_spl_token_owner_secondary_index_state();
3622        let index = AccountsIndex::<bool, bool>::default_for_tests();
3623        for token_id in &spl_token_ids() {
3624            run_test_spl_token_secondary_indexes(
3625                token_id,
3626                &index,
3627                &index.spl_token_owner_index,
3628                key_start,
3629                key_end,
3630                &secondary_indexes,
3631            );
3632        }
3633    }
3634
3635    fn run_test_secondary_indexes_same_slot_and_forks<
3636        SecondaryIndexEntryType: SecondaryIndexEntry + Default + Sync + Send,
3637    >(
3638        token_id: &Pubkey,
3639        index: &AccountsIndex<bool, bool>,
3640        secondary_index: &SecondaryIndex<SecondaryIndexEntryType>,
3641        index_key_start: usize,
3642        index_key_end: usize,
3643        secondary_indexes: &AccountSecondaryIndexes,
3644    ) {
3645        let account_key = Pubkey::new_unique();
3646        let secondary_key1 = Pubkey::new_unique();
3647        let secondary_key2 = Pubkey::new_unique();
3648        let slot = 1;
3649        let mut account_data1 = make_empty_token_account_data();
3650        account_data1[index_key_start..index_key_end]
3651            .clone_from_slice(&(secondary_key1.to_bytes()));
3652        let mut account_data2 = make_empty_token_account_data();
3653        account_data2[index_key_start..index_key_end]
3654            .clone_from_slice(&(secondary_key2.to_bytes()));
3655
3656        // First write one mint index
3657        index.upsert(
3658            slot,
3659            slot,
3660            &account_key,
3661            &AccountSharedData::create(0, account_data1.to_vec(), *token_id, false, 0),
3662            secondary_indexes,
3663            true,
3664            &mut ReclaimsSlotList::new(),
3665            UPSERT_RECLAIM_TEST_DEFAULT,
3666        );
3667
3668        // Now write a different mint index for the same account
3669        index.upsert(
3670            slot,
3671            slot,
3672            &account_key,
3673            &AccountSharedData::create(0, account_data2.to_vec(), *token_id, false, 0),
3674            secondary_indexes,
3675            true,
3676            &mut ReclaimsSlotList::new(),
3677            UPSERT_RECLAIM_TEST_DEFAULT,
3678        );
3679
3680        // Both pubkeys will now be present in the index
3681        check_secondary_index_mapping_correct(
3682            secondary_index,
3683            &[secondary_key1, secondary_key2],
3684            &account_key,
3685        );
3686
3687        // If a later slot also introduces secondary_key1, then it should still exist in the index
3688        let later_slot = slot + 1;
3689        index.upsert(
3690            later_slot,
3691            later_slot,
3692            &account_key,
3693            &AccountSharedData::create(0, account_data1.to_vec(), *token_id, false, 0),
3694            secondary_indexes,
3695            true,
3696            &mut ReclaimsSlotList::new(),
3697            UPSERT_RECLAIM_TEST_DEFAULT,
3698        );
3699        assert_eq!(secondary_index.get(&secondary_key1), vec![account_key]);
3700
3701        // If we set a root at `later_slot`, and clean, then even though the account with secondary_key1
3702        // was outdated by the update in the later slot, the primary account key is still alive,
3703        // so both secondary keys will still be kept alive.
3704        index.add_root(later_slot);
3705        index.slot_list_mut(&account_key, |mut slot_list| {
3706            index.purge_older_root_entries(&mut slot_list, &mut ReclaimsSlotList::new(), None)
3707        });
3708
3709        check_secondary_index_mapping_correct(
3710            secondary_index,
3711            &[secondary_key1, secondary_key2],
3712            &account_key,
3713        );
3714
3715        // Removing the remaining entry for this pubkey in the index should mark the
3716        // pubkey as dead and finally remove all the secondary indexes
3717        let mut reclaims = ReclaimsSlotList::new();
3718        index.purge_exact(&account_key, later_slot, &mut reclaims);
3719        let _ = index.handle_dead_keys(&[account_key], secondary_indexes);
3720        assert!(secondary_index.index.is_empty());
3721        assert!(secondary_index.reverse_index.is_empty());
3722    }
3723
3724    #[test]
3725    fn test_spl_token_mint_secondary_index_same_slot_and_forks() {
3726        let (key_start, key_end, account_index) = create_spl_token_mint_secondary_index_state();
3727        let index = AccountsIndex::<bool, bool>::default_for_tests();
3728        for token_id in &spl_token_ids() {
3729            run_test_secondary_indexes_same_slot_and_forks(
3730                token_id,
3731                &index,
3732                &index.spl_token_mint_index,
3733                key_start,
3734                key_end,
3735                &account_index,
3736            );
3737        }
3738    }
3739
3740    #[test]
3741    fn test_rwlock_secondary_index_same_slot_and_forks() {
3742        let (key_start, key_end, account_index) = create_spl_token_owner_secondary_index_state();
3743        let index = AccountsIndex::<bool, bool>::default_for_tests();
3744        for token_id in &spl_token_ids() {
3745            run_test_secondary_indexes_same_slot_and_forks(
3746                token_id,
3747                &index,
3748                &index.spl_token_owner_index,
3749                key_start,
3750                key_end,
3751                &account_index,
3752            );
3753        }
3754    }
3755
3756    impl IndexValue for bool {}
3757    impl IndexValue for u64 {}
3758    impl DiskIndexValue for bool {}
3759    impl DiskIndexValue for u64 {}
3760    impl IsCached for bool {
3761        fn is_cached(&self) -> bool {
3762            false
3763        }
3764    }
3765    impl IsCached for u64 {
3766        fn is_cached(&self) -> bool {
3767            false
3768        }
3769    }
3770    impl IsZeroLamport for bool {
3771        fn is_zero_lamport(&self) -> bool {
3772            false
3773        }
3774    }
3775
3776    impl IsZeroLamport for u64 {
3777        fn is_zero_lamport(&self) -> bool {
3778            false
3779        }
3780    }
3781
3782    /// Type that supports caching for tests. Used to test upsert behaviour
3783    /// when the slot list has mixed cached and uncached items.
3784    #[derive(Default, Debug, Clone, Copy, PartialEq, Eq)]
3785    struct CacheableIndexValueTest(bool);
3786    impl IndexValue for CacheableIndexValueTest {}
3787    impl DiskIndexValue for CacheableIndexValueTest {}
3788    impl IsCached for CacheableIndexValueTest {
3789        fn is_cached(&self) -> bool {
3790            // Return self value as whether the item is cached or not
3791            self.0
3792        }
3793    }
3794    impl IsZeroLamport for CacheableIndexValueTest {
3795        fn is_zero_lamport(&self) -> bool {
3796            false
3797        }
3798    }
3799
3800    #[test]
3801    fn test_get_newest_root_in_slot_list() {
3802        let index = AccountsIndex::<bool, bool>::default_for_tests();
3803        let return_0 = 0;
3804        let slot1 = 1;
3805        let slot2 = 2;
3806        let slot99 = 99;
3807
3808        // no roots, so always 0
3809        {
3810            let roots_tracker = &index.roots_tracker.read().unwrap();
3811            let slot_list = Vec::<(Slot, bool)>::default();
3812            assert_eq!(
3813                return_0,
3814                AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
3815                    &roots_tracker.alive_roots,
3816                    &slot_list,
3817                    Some(slot1),
3818                )
3819            );
3820            assert_eq!(
3821                return_0,
3822                AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
3823                    &roots_tracker.alive_roots,
3824                    &slot_list,
3825                    Some(slot2),
3826                )
3827            );
3828            assert_eq!(
3829                return_0,
3830                AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
3831                    &roots_tracker.alive_roots,
3832                    &slot_list,
3833                    Some(slot99),
3834                )
3835            );
3836        }
3837
3838        index.add_root(slot2);
3839
3840        {
3841            let roots_tracker = &index.roots_tracker.read().unwrap();
3842            let slot_list = vec![(slot2, true)];
3843            assert_eq!(
3844                slot2,
3845                AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
3846                    &roots_tracker.alive_roots,
3847                    &slot_list,
3848                    Some(slot2),
3849                )
3850            );
3851            // no newest root
3852            assert_eq!(
3853                return_0,
3854                AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
3855                    &roots_tracker.alive_roots,
3856                    &slot_list,
3857                    Some(slot1),
3858                )
3859            );
3860            assert_eq!(
3861                slot2,
3862                AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
3863                    &roots_tracker.alive_roots,
3864                    &slot_list,
3865                    Some(slot99),
3866                )
3867            );
3868        }
3869    }
3870
3871    impl<T: IndexValue> AccountsIndex<T, T> {
3872        fn upsert_simple_test(&self, key: &Pubkey, slot: Slot, value: T) {
3873            let mut gc = ReclaimsSlotList::new();
3874
3875            // It is invalid to reclaim older slots if the slot being upserted
3876            // is unrooted
3877            let reclaim_method = if self.is_alive_root(slot) {
3878                UPSERT_RECLAIM_TEST_DEFAULT
3879            } else {
3880                UpsertReclaim::IgnoreReclaims
3881            };
3882
3883            self.upsert(
3884                slot,
3885                slot,
3886                key,
3887                &AccountSharedData::default(),
3888                &AccountSecondaryIndexes::default(),
3889                value,
3890                &mut gc,
3891                reclaim_method,
3892            );
3893            assert!(gc.is_empty());
3894        }
3895
3896        pub fn clear_roots(&self) {
3897            self.roots_tracker.write().unwrap().alive_roots.clear()
3898        }
3899    }
3900
3901    #[test]
3902    fn test_unref() {
3903        let value = true;
3904        let key = solana_pubkey::new_rand();
3905        let index = AccountsIndex::<bool, bool>::default_for_tests();
3906        let slot1 = 1;
3907
3908        index.upsert_simple_test(&key, slot1, value);
3909
3910        index.get_and_then(&key, |entry| {
3911            let entry = entry.unwrap();
3912            // check refcount BEFORE the unref
3913            assert_eq!(entry.ref_count(), 1);
3914            // first time, ref count was at 1, we can unref once. Unref should return 1.
3915            assert_eq!(entry.unref(), 1);
3916            // check refcount AFTER the unref
3917            assert_eq!(entry.ref_count(), 0);
3918            (false, ())
3919        });
3920    }
3921
3922    #[test]
3923    #[should_panic(expected = "decremented ref count below zero")]
3924    fn test_illegal_unref() {
3925        let value = true;
3926        let key = solana_pubkey::new_rand();
3927        let index = AccountsIndex::<bool, bool>::default_for_tests();
3928        let slot1 = 1;
3929
3930        index.upsert_simple_test(&key, slot1, value);
3931
3932        index.get_and_then(&key, |entry| {
3933            let entry = entry.unwrap();
3934            // make ref count be zero
3935            assert_eq!(entry.unref(), 1);
3936            assert_eq!(entry.ref_count(), 0);
3937
3938            // unref when already at zero should panic
3939            entry.unref();
3940            (false, ())
3941        });
3942    }
3943
3944    #[test]
3945    fn test_clean_rooted_entries_return() {
3946        agave_logger::setup();
3947        let value = true;
3948        let key = solana_pubkey::new_rand();
3949        let key_unknown = solana_pubkey::new_rand();
3950        let index = AccountsIndex::<bool, bool>::default_for_tests();
3951        let slot1 = 1;
3952
3953        let mut gc = ReclaimsSlotList::new();
3954        // return true if we don't know anything about 'key_unknown'
3955        // the item did not exist in the accounts index at all, so index is up to date
3956        assert!(index.clean_rooted_entries(&key_unknown, &mut gc, None));
3957
3958        index.upsert_simple_test(&key, slot1, value);
3959
3960        let slot2 = 2;
3961        // none for max root because we don't want to delete the entry yet
3962        assert!(!index.clean_rooted_entries(&key, &mut gc, None));
3963        // this is because of inclusive vs exclusive in the call to can_purge_older_entries
3964        assert!(!index.clean_rooted_entries(&key, &mut gc, Some(slot1)));
3965        // this will delete the entry because it is <= max_root_inclusive and NOT a root
3966        // note this has to be slot2 because of inclusive vs exclusive in the call to can_purge_older_entries
3967        {
3968            let mut gc = ReclaimsSlotList::new();
3969            assert!(index.clean_rooted_entries(&key, &mut gc, Some(slot2)));
3970            assert_eq!(gc, ReclaimsSlotList::from([(slot1, value)]));
3971        }
3972
3973        // re-add it
3974        index.upsert_simple_test(&key, slot1, value);
3975
3976        index.add_root(slot1);
3977        assert!(!index.clean_rooted_entries(&key, &mut gc, Some(slot2)));
3978        index.upsert_simple_test(&key, slot2, value);
3979
3980        index.get_and_then(&key, |entry| {
3981            let account_map_entry = entry.unwrap();
3982            let slot_list = account_map_entry.slot_list_read_lock();
3983            assert_eq!(2, slot_list.len());
3984            assert_eq!(&[(slot1, value), (slot2, value)], slot_list.as_ref());
3985            (false, ())
3986        });
3987        assert!(!index.clean_rooted_entries(&key, &mut gc, Some(slot2)));
3988        assert_eq!(
3989            2,
3990            index.get_and_then(&key, |entry| (
3991                false,
3992                entry.unwrap().slot_list_lock_read_len()
3993            ))
3994        );
3995        assert!(gc.is_empty());
3996        {
3997            {
3998                let roots_tracker = &index.roots_tracker.read().unwrap();
3999                let slot_list = SlotList::from([(slot2, value)]);
4000                assert_eq!(
4001                    0,
4002                    AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
4003                        &roots_tracker.alive_roots,
4004                        &slot_list,
4005                        None,
4006                    )
4007                );
4008            }
4009            index.add_root(slot2);
4010            {
4011                let roots_tracker = &index.roots_tracker.read().unwrap();
4012                let slot_list = SlotList::from([(slot2, value)]);
4013                assert_eq!(
4014                    slot2,
4015                    AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
4016                        &roots_tracker.alive_roots,
4017                        &slot_list,
4018                        None,
4019                    )
4020                );
4021                assert_eq!(
4022                    0,
4023                    AccountsIndex::<bool, bool>::get_newest_root_in_slot_list(
4024                        &roots_tracker.alive_roots,
4025                        &slot_list,
4026                        Some(0),
4027                    )
4028                );
4029            }
4030        }
4031
4032        assert!(gc.is_empty());
4033        assert!(!index.clean_rooted_entries(&key, &mut gc, Some(slot2)));
4034        assert_eq!(gc, ReclaimsSlotList::from([(slot1, value)]));
4035        gc.clear();
4036        index.clean_dead_slot(slot2);
4037        let slot3 = 3;
4038        assert!(index.clean_rooted_entries(&key, &mut gc, Some(slot3)));
4039        assert_eq!(gc, ReclaimsSlotList::from([(slot2, value)]));
4040    }
4041
4042    #[test]
4043    fn test_handle_dead_keys_return() {
4044        let key = solana_pubkey::new_rand();
4045        let index = AccountsIndex::<bool, bool>::default_for_tests();
4046
4047        assert_eq!(
4048            index.handle_dead_keys(&[key], &AccountSecondaryIndexes::default()),
4049            vec![key].into_iter().collect::<HashSet<_>>()
4050        );
4051    }
4052
4053    #[test]
4054    fn test_start_end_bin() {
4055        let index = AccountsIndex::<bool, bool>::default_for_tests();
4056        assert_eq!(index.bins(), BINS_FOR_TESTING);
4057
4058        let range = (Unbounded::<Pubkey>, Unbounded);
4059        let (start, end) = index.bin_start_end_inclusive(&range);
4060        assert_eq!(start, 0); // no range, so 0
4061        assert_eq!(end, BINS_FOR_TESTING - 1); // no range, so last bin
4062
4063        let key = Pubkey::from([0; 32]);
4064        let range = RangeInclusive::new(key, key);
4065        let (start, end) = index.bin_start_end_inclusive(&range);
4066        assert_eq!(start, 0); // start at pubkey 0, so 0
4067        assert_eq!(end, 0); // end at pubkey 0, so 0
4068
4069        let range = (Included(key), Excluded(key));
4070        let (start, end) = index.bin_start_end_inclusive(&range);
4071        assert_eq!(start, 0); // start at pubkey 0, so 0
4072        assert_eq!(end, 0); // end at pubkey 0, so 0
4073
4074        let range = (Excluded(key), Excluded(key));
4075        let (start, end) = index.bin_start_end_inclusive(&range);
4076        assert_eq!(start, 0); // start at pubkey 0, so 0
4077        assert_eq!(end, 0); // end at pubkey 0, so 0
4078
4079        let key = Pubkey::from([0xff; 32]);
4080        let range = RangeInclusive::new(key, key);
4081        let (start, end) = index.bin_start_end_inclusive(&range);
4082        let bins = index.bins();
4083        assert_eq!(start, bins - 1); // start at highest possible pubkey, so bins - 1
4084        assert_eq!(end, bins - 1);
4085
4086        let range = (Included(key), Excluded(key));
4087        let (start, end) = index.bin_start_end_inclusive(&range);
4088        assert_eq!(start, bins - 1); // start at highest possible pubkey, so bins - 1
4089        assert_eq!(end, bins - 1);
4090
4091        let range = (Excluded(key), Excluded(key));
4092        let (start, end) = index.bin_start_end_inclusive(&range);
4093        assert_eq!(start, bins); // Exclude the highest possible pubkey, so start should be "bins"
4094        assert_eq!(end, bins - 1); // End should be the last bin index
4095    }
4096
4097    #[test]
4098    #[should_panic(expected = "bins.is_power_of_two()")]
4099    #[allow(clippy::field_reassign_with_default)]
4100    fn test_illegal_bins() {
4101        let mut config = AccountsIndexConfig::default();
4102        config.bins = Some(3);
4103        AccountsIndex::<bool, bool>::new(&config, Arc::default());
4104    }
4105
4106    #[test]
4107    fn test_scan_config() {
4108        for scan_order in [ScanOrder::Sorted, ScanOrder::Unsorted] {
4109            let config = ScanConfig::new(scan_order);
4110            assert_eq!(config.scan_order, scan_order);
4111            assert!(config.abort.is_none()); // not allocated
4112            assert!(!config.is_aborted());
4113            config.abort(); // has no effect
4114            assert!(!config.is_aborted());
4115        }
4116
4117        let config = ScanConfig::new(ScanOrder::Sorted);
4118        assert_eq!(config.scan_order, ScanOrder::Sorted);
4119        assert!(config.abort.is_none());
4120
4121        let config = ScanConfig::default();
4122        assert_eq!(config.scan_order, ScanOrder::Unsorted);
4123        assert!(config.abort.is_none());
4124
4125        let config = config.recreate_with_abort();
4126        assert!(config.abort.is_some());
4127        assert!(!config.is_aborted());
4128        config.abort();
4129        assert!(config.is_aborted());
4130
4131        let config = config.recreate_with_abort();
4132        assert!(config.is_aborted());
4133    }
4134}