solana_accounts_db/
accounts_index.rs

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