solana_runtime/
accounts_index.rs

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