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