solana_accounts_db/
accounts.rs

1use {
2    crate::{
3        account_locks::{validate_account_locks, AccountLocks},
4        accounts_db::{
5            AccountStorageEntry, AccountsAddRootTiming, AccountsDb, LoadHint, LoadedAccount,
6            ScanAccountStorageData, ScanStorageResult, VerifyAccountsHashAndLamportsConfig,
7        },
8        accounts_index::{IndexKey, ScanConfig, ScanError, ScanResult},
9        ancestors::Ancestors,
10        storable_accounts::StorableAccounts,
11    },
12    dashmap::DashMap,
13    log::*,
14    solana_pubkey::Pubkey,
15    solana_sdk::{
16        account::{AccountSharedData, ReadableAccount},
17        address_lookup_table::{self, error::AddressLookupError, state::AddressLookupTable},
18        clock::{BankId, Slot},
19        message::v0::LoadedAddresses,
20        slot_hashes::SlotHashes,
21        transaction::{Result, SanitizedTransaction},
22        transaction_context::TransactionAccount,
23    },
24    solana_svm_transaction::{
25        message_address_table_lookup::SVMMessageAddressTableLookup, svm_message::SVMMessage,
26    },
27    std::{
28        cmp::Reverse,
29        collections::{BinaryHeap, HashSet},
30        ops::RangeBounds,
31        sync::{
32            atomic::{AtomicUsize, Ordering},
33            Arc, Mutex,
34        },
35    },
36};
37
38pub type PubkeyAccountSlot = (Pubkey, AccountSharedData, Slot);
39
40struct TransactionAccountLocksIterator<'a, T: SVMMessage> {
41    transaction: &'a T,
42}
43
44impl<'a, T: SVMMessage> TransactionAccountLocksIterator<'a, T> {
45    pub(crate) fn new(transaction: &'a T) -> Self {
46        Self { transaction }
47    }
48
49    pub(crate) fn accounts_with_is_writable(
50        &self,
51    ) -> impl Iterator<Item = (&'a Pubkey, bool)> + Clone {
52        self.transaction
53            .account_keys()
54            .iter()
55            .enumerate()
56            .map(|(index, key)| (key, self.transaction.is_writable(index)))
57    }
58}
59
60/// This structure handles synchronization for db
61#[derive(Debug)]
62pub struct Accounts {
63    /// Single global AccountsDb
64    pub accounts_db: Arc<AccountsDb>,
65
66    /// set of read-only and writable accounts which are currently
67    /// being processed by banking/replay threads
68    pub(crate) account_locks: Mutex<AccountLocks>,
69}
70
71pub enum AccountAddressFilter {
72    Exclude, // exclude all addresses matching the filter
73    Include, // only include addresses matching the filter
74}
75
76impl Accounts {
77    pub fn new(accounts_db: Arc<AccountsDb>) -> Self {
78        Self {
79            accounts_db,
80            account_locks: Mutex::new(AccountLocks::default()),
81        }
82    }
83
84    /// Return loaded addresses and the deactivation slot.
85    /// If the table hasn't been deactivated, the deactivation slot is `u64::MAX`.
86    pub fn load_lookup_table_addresses(
87        &self,
88        ancestors: &Ancestors,
89        address_table_lookup: SVMMessageAddressTableLookup,
90        slot_hashes: &SlotHashes,
91    ) -> std::result::Result<(LoadedAddresses, Slot), AddressLookupError> {
92        let mut loaded_addresses = LoadedAddresses::default();
93        self.load_lookup_table_addresses_into(
94            ancestors,
95            address_table_lookup,
96            slot_hashes,
97            &mut loaded_addresses,
98        )
99        .map(|deactivation_slot| (loaded_addresses, deactivation_slot))
100    }
101
102    /// Fill `loaded_addresses` and return the deactivation slot.
103    /// If no tables are de-activating, the deactivation slot is `u64::MAX`.
104    pub fn load_lookup_table_addresses_into(
105        &self,
106        ancestors: &Ancestors,
107        address_table_lookup: SVMMessageAddressTableLookup,
108        slot_hashes: &SlotHashes,
109        loaded_addresses: &mut LoadedAddresses,
110    ) -> std::result::Result<Slot, AddressLookupError> {
111        let table_account = self
112            .accounts_db
113            .load_with_fixed_root(ancestors, address_table_lookup.account_key)
114            .map(|(account, _rent)| account)
115            .ok_or(AddressLookupError::LookupTableAccountNotFound)?;
116
117        if table_account.owner() == &address_lookup_table::program::id() {
118            let current_slot = ancestors.max_slot();
119            let lookup_table = AddressLookupTable::deserialize(table_account.data())
120                .map_err(|_ix_err| AddressLookupError::InvalidAccountData)?;
121
122            // Load iterators for addresses.
123            let writable_addresses = lookup_table.lookup_iter(
124                current_slot,
125                address_table_lookup.writable_indexes,
126                slot_hashes,
127            )?;
128            let readonly_addresses = lookup_table.lookup_iter(
129                current_slot,
130                address_table_lookup.readonly_indexes,
131                slot_hashes,
132            )?;
133
134            // Reserve space in vectors to avoid reallocations.
135            // If `loaded_addresses` is pre-allocated, this only does a simple
136            // bounds check.
137            loaded_addresses
138                .writable
139                .reserve(address_table_lookup.writable_indexes.len());
140            loaded_addresses
141                .readonly
142                .reserve(address_table_lookup.readonly_indexes.len());
143
144            // Append to the loaded addresses.
145            // Check if **any** of the addresses are not available.
146            for address in writable_addresses {
147                loaded_addresses
148                    .writable
149                    .push(address.ok_or(AddressLookupError::InvalidLookupIndex)?);
150            }
151            for address in readonly_addresses {
152                loaded_addresses
153                    .readonly
154                    .push(address.ok_or(AddressLookupError::InvalidLookupIndex)?);
155            }
156
157            Ok(lookup_table.meta.deactivation_slot)
158        } else {
159            Err(AddressLookupError::InvalidAccountOwner)
160        }
161    }
162    /// Slow because lock is held for 1 operation instead of many
163    /// This always returns None for zero-lamport accounts.
164    fn load_slow(
165        &self,
166        ancestors: &Ancestors,
167        pubkey: &Pubkey,
168        load_hint: LoadHint,
169    ) -> Option<(AccountSharedData, Slot)> {
170        self.accounts_db.load(ancestors, pubkey, load_hint)
171    }
172
173    pub fn load_with_fixed_root(
174        &self,
175        ancestors: &Ancestors,
176        pubkey: &Pubkey,
177    ) -> Option<(AccountSharedData, Slot)> {
178        self.load_slow(ancestors, pubkey, LoadHint::FixedMaxRoot)
179    }
180
181    /// same as `load_with_fixed_root` except:
182    /// if the account is not already in the read cache, it is NOT put in the read cache on successful load
183    pub fn load_with_fixed_root_do_not_populate_read_cache(
184        &self,
185        ancestors: &Ancestors,
186        pubkey: &Pubkey,
187    ) -> Option<(AccountSharedData, Slot)> {
188        self.load_slow(
189            ancestors,
190            pubkey,
191            LoadHint::FixedMaxRootDoNotPopulateReadCache,
192        )
193    }
194
195    pub fn load_without_fixed_root(
196        &self,
197        ancestors: &Ancestors,
198        pubkey: &Pubkey,
199    ) -> Option<(AccountSharedData, Slot)> {
200        self.load_slow(ancestors, pubkey, LoadHint::Unspecified)
201    }
202
203    /// scans underlying accounts_db for this delta (slot) with a map function
204    ///   from LoadedAccount to B
205    /// returns only the latest/current version of B for this slot
206    pub fn scan_slot<F, B>(&self, slot: Slot, func: F) -> Vec<B>
207    where
208        F: Fn(&LoadedAccount) -> Option<B> + Send + Sync,
209        B: Sync + Send + Default + std::cmp::Eq,
210    {
211        let scan_result = self.accounts_db.scan_account_storage(
212            slot,
213            |loaded_account: &LoadedAccount| {
214                // Cache only has one version per key, don't need to worry about versioning
215                func(loaded_account)
216            },
217            |accum: &DashMap<Pubkey, B>, loaded_account: &LoadedAccount, _data| {
218                let loaded_account_pubkey = *loaded_account.pubkey();
219                if let Some(val) = func(loaded_account) {
220                    accum.insert(loaded_account_pubkey, val);
221                }
222            },
223            ScanAccountStorageData::NoData,
224        );
225
226        match scan_result {
227            ScanStorageResult::Cached(cached_result) => cached_result,
228            ScanStorageResult::Stored(stored_result) => stored_result
229                .into_iter()
230                .map(|(_pubkey, val)| val)
231                .collect(),
232        }
233    }
234
235    /// Returns all the accounts from `slot`
236    ///
237    /// If `program_id` is `Some`, filter the results to those whose owner matches `program_id`
238    pub fn load_by_program_slot(
239        &self,
240        slot: Slot,
241        program_id: Option<&Pubkey>,
242    ) -> Vec<TransactionAccount> {
243        self.scan_slot(slot, |stored_account| {
244            program_id
245                .map(|program_id| program_id == stored_account.owner())
246                .unwrap_or(true)
247                .then(|| (*stored_account.pubkey(), stored_account.take_account()))
248        })
249    }
250
251    pub fn load_largest_accounts(
252        &self,
253        ancestors: &Ancestors,
254        bank_id: BankId,
255        num: usize,
256        filter_by_address: &HashSet<Pubkey>,
257        filter: AccountAddressFilter,
258        sort_results: bool,
259    ) -> ScanResult<Vec<(Pubkey, u64)>> {
260        if num == 0 {
261            return Ok(vec![]);
262        }
263        let mut account_balances = BinaryHeap::new();
264        self.accounts_db.scan_accounts(
265            ancestors,
266            bank_id,
267            |option| {
268                if let Some((pubkey, account, _slot)) = option {
269                    if account.lamports() == 0 {
270                        return;
271                    }
272                    let contains_address = filter_by_address.contains(pubkey);
273                    let collect = match filter {
274                        AccountAddressFilter::Exclude => !contains_address,
275                        AccountAddressFilter::Include => contains_address,
276                    };
277                    if !collect {
278                        return;
279                    }
280                    if account_balances.len() == num {
281                        let Reverse(entry) = account_balances
282                            .peek()
283                            .expect("BinaryHeap::peek should succeed when len > 0");
284                        if *entry >= (account.lamports(), *pubkey) {
285                            return;
286                        }
287                        account_balances.pop();
288                    }
289                    account_balances.push(Reverse((account.lamports(), *pubkey)));
290                }
291            },
292            &ScanConfig::new(!sort_results),
293        )?;
294        Ok(account_balances
295            .into_sorted_vec()
296            .into_iter()
297            .map(|Reverse((balance, pubkey))| (pubkey, balance))
298            .collect())
299    }
300
301    /// Only called from startup or test code.
302    #[must_use]
303    pub fn verify_accounts_hash_and_lamports(
304        &self,
305        snapshot_storages_and_slots: (&[Arc<AccountStorageEntry>], &[Slot]),
306        slot: Slot,
307        total_lamports: u64,
308        base: Option<(Slot, /*capitalization*/ u64)>,
309        config: VerifyAccountsHashAndLamportsConfig,
310    ) -> bool {
311        if let Err(err) = self.accounts_db.verify_accounts_hash_and_lamports(
312            snapshot_storages_and_slots,
313            slot,
314            total_lamports,
315            base,
316            config,
317        ) {
318            warn!("verify_accounts_hash failed: {err:?}, slot: {slot}");
319            false
320        } else {
321            true
322        }
323    }
324
325    pub fn is_loadable(lamports: u64) -> bool {
326        // Don't ever load zero lamport accounts into runtime because
327        // the existence of zero-lamport accounts are never deterministic!!
328        lamports > 0
329    }
330
331    fn load_while_filtering<F: Fn(&AccountSharedData) -> bool>(
332        collector: &mut Vec<TransactionAccount>,
333        some_account_tuple: Option<(&Pubkey, AccountSharedData, Slot)>,
334        filter: F,
335    ) {
336        if let Some(mapped_account_tuple) = some_account_tuple
337            .filter(|(_, account, _)| Self::is_loadable(account.lamports()) && filter(account))
338            .map(|(pubkey, account, _slot)| (*pubkey, account))
339        {
340            collector.push(mapped_account_tuple)
341        }
342    }
343
344    fn load_with_slot(
345        collector: &mut Vec<PubkeyAccountSlot>,
346        some_account_tuple: Option<(&Pubkey, AccountSharedData, Slot)>,
347    ) {
348        if let Some(mapped_account_tuple) = some_account_tuple
349            .filter(|(_, account, _)| Self::is_loadable(account.lamports()))
350            .map(|(pubkey, account, slot)| (*pubkey, account, slot))
351        {
352            collector.push(mapped_account_tuple)
353        }
354    }
355
356    pub fn load_by_program(
357        &self,
358        ancestors: &Ancestors,
359        bank_id: BankId,
360        program_id: &Pubkey,
361        config: &ScanConfig,
362    ) -> ScanResult<Vec<TransactionAccount>> {
363        let mut collector = Vec::new();
364        self.accounts_db
365            .scan_accounts(
366                ancestors,
367                bank_id,
368                |some_account_tuple| {
369                    Self::load_while_filtering(&mut collector, some_account_tuple, |account| {
370                        account.owner() == program_id
371                    })
372                },
373                config,
374            )
375            .map(|_| collector)
376    }
377
378    pub fn load_by_program_with_filter<F: Fn(&AccountSharedData) -> bool>(
379        &self,
380        ancestors: &Ancestors,
381        bank_id: BankId,
382        program_id: &Pubkey,
383        filter: F,
384        config: &ScanConfig,
385    ) -> ScanResult<Vec<TransactionAccount>> {
386        let mut collector = Vec::new();
387        self.accounts_db
388            .scan_accounts(
389                ancestors,
390                bank_id,
391                |some_account_tuple| {
392                    Self::load_while_filtering(&mut collector, some_account_tuple, |account| {
393                        account.owner() == program_id && filter(account)
394                    })
395                },
396                config,
397            )
398            .map(|_| collector)
399    }
400
401    fn calc_scan_result_size(account: &AccountSharedData) -> usize {
402        account.data().len()
403            + std::mem::size_of::<AccountSharedData>()
404            + std::mem::size_of::<Pubkey>()
405    }
406
407    /// Accumulate size of (pubkey + account) into sum.
408    /// Return true iff sum > 'byte_limit_for_scan'
409    fn accumulate_and_check_scan_result_size(
410        sum: &AtomicUsize,
411        account: &AccountSharedData,
412        byte_limit_for_scan: &Option<usize>,
413    ) -> bool {
414        if let Some(byte_limit_for_scan) = byte_limit_for_scan.as_ref() {
415            let added = Self::calc_scan_result_size(account);
416            sum.fetch_add(added, Ordering::Relaxed)
417                .saturating_add(added)
418                > *byte_limit_for_scan
419        } else {
420            false
421        }
422    }
423
424    fn maybe_abort_scan(
425        result: ScanResult<Vec<TransactionAccount>>,
426        config: &ScanConfig,
427    ) -> ScanResult<Vec<TransactionAccount>> {
428        if config.is_aborted() {
429            ScanResult::Err(ScanError::Aborted(
430                "The accumulated scan results exceeded the limit".to_string(),
431            ))
432        } else {
433            result
434        }
435    }
436
437    pub fn load_by_index_key_with_filter<F: Fn(&AccountSharedData) -> bool>(
438        &self,
439        ancestors: &Ancestors,
440        bank_id: BankId,
441        index_key: &IndexKey,
442        filter: F,
443        config: &ScanConfig,
444        byte_limit_for_scan: Option<usize>,
445    ) -> ScanResult<Vec<TransactionAccount>> {
446        let sum = AtomicUsize::default();
447        let config = config.recreate_with_abort();
448        let mut collector = Vec::new();
449        let result = self
450            .accounts_db
451            .index_scan_accounts(
452                ancestors,
453                bank_id,
454                *index_key,
455                |some_account_tuple| {
456                    Self::load_while_filtering(&mut collector, some_account_tuple, |account| {
457                        let use_account = filter(account);
458                        if use_account
459                            && Self::accumulate_and_check_scan_result_size(
460                                &sum,
461                                account,
462                                &byte_limit_for_scan,
463                            )
464                        {
465                            // total size of results exceeds size limit, so abort scan
466                            config.abort();
467                        }
468                        use_account
469                    });
470                },
471                &config,
472            )
473            .map(|_| collector);
474        Self::maybe_abort_scan(result, &config)
475    }
476
477    pub fn account_indexes_include_key(&self, key: &Pubkey) -> bool {
478        self.accounts_db.account_indexes.include_key(key)
479    }
480
481    pub fn load_all(
482        &self,
483        ancestors: &Ancestors,
484        bank_id: BankId,
485        sort_results: bool,
486    ) -> ScanResult<Vec<PubkeyAccountSlot>> {
487        let mut collector = Vec::new();
488        self.accounts_db
489            .scan_accounts(
490                ancestors,
491                bank_id,
492                |some_account_tuple| {
493                    if let Some((pubkey, account, slot)) = some_account_tuple
494                        .filter(|(_, account, _)| Self::is_loadable(account.lamports()))
495                    {
496                        collector.push((*pubkey, account, slot))
497                    }
498                },
499                &ScanConfig::new(!sort_results),
500            )
501            .map(|_| collector)
502    }
503
504    pub fn scan_all<F>(
505        &self,
506        ancestors: &Ancestors,
507        bank_id: BankId,
508        scan_func: F,
509        sort_results: bool,
510    ) -> ScanResult<()>
511    where
512        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
513    {
514        self.accounts_db.scan_accounts(
515            ancestors,
516            bank_id,
517            scan_func,
518            &ScanConfig::new(!sort_results),
519        )
520    }
521
522    pub fn hold_range_in_memory<R>(
523        &self,
524        range: &R,
525        start_holding: bool,
526        thread_pool: &rayon::ThreadPool,
527    ) where
528        R: RangeBounds<Pubkey> + std::fmt::Debug + Sync,
529    {
530        self.accounts_db
531            .accounts_index
532            .hold_range_in_memory(range, start_holding, thread_pool)
533    }
534
535    pub fn load_to_collect_rent_eagerly<R: RangeBounds<Pubkey> + std::fmt::Debug>(
536        &self,
537        ancestors: &Ancestors,
538        range: R,
539    ) -> Vec<PubkeyAccountSlot> {
540        let mut collector = Vec::new();
541        self.accounts_db.range_scan_accounts(
542            "", // disable logging of this. We now parallelize it and this results in multiple parallel logs
543            ancestors,
544            range,
545            &ScanConfig::default(),
546            |option| Self::load_with_slot(&mut collector, option),
547        );
548        collector
549    }
550
551    /// Slow because lock is held for 1 operation instead of many.
552    /// WARNING: This noncached version is only to be used for tests/benchmarking
553    /// as bypassing the cache in general is not supported
554    pub fn store_slow_uncached(&self, slot: Slot, pubkey: &Pubkey, account: &AccountSharedData) {
555        self.accounts_db.store_uncached(slot, &[(pubkey, account)]);
556    }
557
558    /// This function will prevent multiple threads from modifying the same account state at the
559    /// same time
560    #[must_use]
561    pub fn lock_accounts<'a, Tx: SVMMessage + 'a>(
562        &self,
563        txs: impl Iterator<Item = &'a Tx>,
564        tx_account_lock_limit: usize,
565    ) -> Vec<Result<()>> {
566        // Validate the account locks, then get iterator if successful validation.
567        let tx_account_locks_results: Vec<Result<_>> = txs
568            .map(|tx| {
569                validate_account_locks(tx.account_keys(), tx_account_lock_limit)
570                    .map(|_| TransactionAccountLocksIterator::new(tx))
571            })
572            .collect();
573        self.lock_accounts_inner(tx_account_locks_results)
574    }
575
576    #[must_use]
577    pub fn lock_accounts_with_results<'a>(
578        &self,
579        txs: impl Iterator<Item = &'a (impl SVMMessage + 'a)>,
580        results: impl Iterator<Item = Result<()>>,
581        tx_account_lock_limit: usize,
582    ) -> Vec<Result<()>> {
583        // Validate the account locks, then get iterator if successful validation.
584        let tx_account_locks_results: Vec<Result<_>> = txs
585            .zip(results)
586            .map(|(tx, result)| match result {
587                Ok(()) => validate_account_locks(tx.account_keys(), tx_account_lock_limit)
588                    .map(|_| TransactionAccountLocksIterator::new(tx)),
589                Err(err) => Err(err),
590            })
591            .collect();
592        self.lock_accounts_inner(tx_account_locks_results)
593    }
594
595    #[must_use]
596    fn lock_accounts_inner(
597        &self,
598        tx_account_locks_results: Vec<Result<TransactionAccountLocksIterator<impl SVMMessage>>>,
599    ) -> Vec<Result<()>> {
600        let account_locks = &mut self.account_locks.lock().unwrap();
601        tx_account_locks_results
602            .into_iter()
603            .map(|tx_account_locks_result| match tx_account_locks_result {
604                Ok(tx_account_locks) => {
605                    account_locks.try_lock_accounts(tx_account_locks.accounts_with_is_writable())
606                }
607                Err(err) => Err(err),
608            })
609            .collect()
610    }
611
612    /// Once accounts are unlocked, new transactions that modify that state can enter the pipeline
613    pub fn unlock_accounts<'a, Tx: SVMMessage + 'a>(
614        &self,
615        txs_and_results: impl Iterator<Item = (&'a Tx, &'a Result<()>)> + Clone,
616    ) {
617        if !txs_and_results.clone().any(|(_, res)| res.is_ok()) {
618            return;
619        }
620
621        let mut account_locks = self.account_locks.lock().unwrap();
622        debug!("bank unlock accounts");
623        for (tx, res) in txs_and_results {
624            if res.is_ok() {
625                let tx_account_locks = TransactionAccountLocksIterator::new(tx);
626                account_locks.unlock_accounts(tx_account_locks.accounts_with_is_writable());
627            }
628        }
629    }
630
631    /// Store the accounts into the DB
632    pub fn store_cached<'a>(
633        &self,
634        accounts: impl StorableAccounts<'a>,
635        transactions: Option<&'a [&'a SanitizedTransaction]>,
636    ) {
637        self.accounts_db
638            .store_cached_inline_update_index(accounts, transactions);
639    }
640
641    pub fn store_accounts_cached<'a>(&self, accounts: impl StorableAccounts<'a>) {
642        self.accounts_db.store_cached(accounts, None)
643    }
644
645    /// Add a slot to root.  Root slots cannot be purged
646    pub fn add_root(&self, slot: Slot) -> AccountsAddRootTiming {
647        self.accounts_db.add_root(slot)
648    }
649}
650
651#[cfg(test)]
652mod tests {
653    use {
654        super::*,
655        solana_sdk::{
656            account::{AccountSharedData, WritableAccount},
657            address_lookup_table::state::LookupTableMeta,
658            hash::Hash,
659            instruction::CompiledInstruction,
660            message::{v0::MessageAddressTableLookup, Message, MessageHeader},
661            native_loader,
662            signature::{signers::Signers, Keypair, Signer},
663            transaction::{Transaction, TransactionError, MAX_TX_ACCOUNT_LOCKS},
664        },
665        std::{
666            borrow::Cow,
667            iter,
668            sync::atomic::{AtomicBool, AtomicU64, Ordering},
669            thread, time,
670        },
671    };
672
673    fn new_sanitized_tx<T: Signers>(
674        from_keypairs: &T,
675        message: Message,
676        recent_blockhash: Hash,
677    ) -> SanitizedTransaction {
678        SanitizedTransaction::from_transaction_for_tests(Transaction::new(
679            from_keypairs,
680            message,
681            recent_blockhash,
682        ))
683    }
684
685    #[test]
686    fn test_hold_range_in_memory() {
687        let accounts_db = AccountsDb::default_for_tests();
688        let accts = Accounts::new(Arc::new(accounts_db));
689        let range = Pubkey::from([0; 32])..=Pubkey::from([0xff; 32]);
690        accts.hold_range_in_memory(&range, true, &test_thread_pool());
691        accts.hold_range_in_memory(&range, false, &test_thread_pool());
692        accts.hold_range_in_memory(&range, true, &test_thread_pool());
693        accts.hold_range_in_memory(&range, true, &test_thread_pool());
694        accts.hold_range_in_memory(&range, false, &test_thread_pool());
695        accts.hold_range_in_memory(&range, false, &test_thread_pool());
696    }
697
698    #[test]
699    fn test_hold_range_in_memory2() {
700        let accounts_db = AccountsDb::default_for_tests();
701        let accts = Accounts::new(Arc::new(accounts_db));
702        let range = Pubkey::from([0; 32])..=Pubkey::from([0xff; 32]);
703        let idx = &accts.accounts_db.accounts_index;
704        let bins = idx.account_maps.len();
705        // use bins * 2 to get the first half of the range within bin 0
706        let bins_2 = bins * 2;
707        let binner = crate::pubkey_bins::PubkeyBinCalculator24::new(bins_2);
708        let range2 =
709            binner.lowest_pubkey_from_bin(0, bins_2)..binner.lowest_pubkey_from_bin(1, bins_2);
710        let range2_inclusive = range2.start..=range2.end;
711        assert_eq!(0, idx.bin_calculator.bin_from_pubkey(&range2.start));
712        assert_eq!(0, idx.bin_calculator.bin_from_pubkey(&range2.end));
713        accts.hold_range_in_memory(&range, true, &test_thread_pool());
714        idx.account_maps.iter().for_each(|map| {
715            assert_eq!(
716                map.cache_ranges_held.read().unwrap().to_vec(),
717                vec![range.clone()]
718            );
719        });
720        accts.hold_range_in_memory(&range2, true, &test_thread_pool());
721        idx.account_maps.iter().enumerate().for_each(|(bin, map)| {
722            let expected = if bin == 0 {
723                vec![range.clone(), range2_inclusive.clone()]
724            } else {
725                vec![range.clone()]
726            };
727            assert_eq!(
728                map.cache_ranges_held.read().unwrap().to_vec(),
729                expected,
730                "bin: {bin}"
731            );
732        });
733        accts.hold_range_in_memory(&range, false, &test_thread_pool());
734        accts.hold_range_in_memory(&range2, false, &test_thread_pool());
735    }
736
737    fn test_thread_pool() -> rayon::ThreadPool {
738        crate::accounts_db::make_min_priority_thread_pool()
739    }
740
741    #[test]
742    fn test_load_lookup_table_addresses_account_not_found() {
743        let ancestors = vec![(0, 0)].into_iter().collect();
744        let accounts_db = AccountsDb::new_single_for_tests();
745        let accounts = Accounts::new(Arc::new(accounts_db));
746
747        let invalid_table_key = Pubkey::new_unique();
748        let address_table_lookup = MessageAddressTableLookup {
749            account_key: invalid_table_key,
750            writable_indexes: vec![],
751            readonly_indexes: vec![],
752        };
753
754        assert_eq!(
755            accounts.load_lookup_table_addresses(
756                &ancestors,
757                SVMMessageAddressTableLookup::from(&address_table_lookup),
758                &SlotHashes::default(),
759            ),
760            Err(AddressLookupError::LookupTableAccountNotFound),
761        );
762    }
763
764    #[test]
765    fn test_load_lookup_table_addresses_invalid_account_owner() {
766        let ancestors = vec![(0, 0)].into_iter().collect();
767        let accounts_db = AccountsDb::new_single_for_tests();
768        let accounts = Accounts::new(Arc::new(accounts_db));
769
770        let invalid_table_key = Pubkey::new_unique();
771        let mut invalid_table_account = AccountSharedData::default();
772        invalid_table_account.set_lamports(1);
773        accounts.store_slow_uncached(0, &invalid_table_key, &invalid_table_account);
774
775        let address_table_lookup = MessageAddressTableLookup {
776            account_key: invalid_table_key,
777            writable_indexes: vec![],
778            readonly_indexes: vec![],
779        };
780
781        assert_eq!(
782            accounts.load_lookup_table_addresses(
783                &ancestors,
784                SVMMessageAddressTableLookup::from(&address_table_lookup),
785                &SlotHashes::default(),
786            ),
787            Err(AddressLookupError::InvalidAccountOwner),
788        );
789    }
790
791    #[test]
792    fn test_load_lookup_table_addresses_invalid_account_data() {
793        let ancestors = vec![(0, 0)].into_iter().collect();
794        let accounts_db = AccountsDb::new_single_for_tests();
795        let accounts = Accounts::new(Arc::new(accounts_db));
796
797        let invalid_table_key = Pubkey::new_unique();
798        let invalid_table_account =
799            AccountSharedData::new(1, 0, &address_lookup_table::program::id());
800        accounts.store_slow_uncached(0, &invalid_table_key, &invalid_table_account);
801
802        let address_table_lookup = MessageAddressTableLookup {
803            account_key: invalid_table_key,
804            writable_indexes: vec![],
805            readonly_indexes: vec![],
806        };
807
808        assert_eq!(
809            accounts.load_lookup_table_addresses(
810                &ancestors,
811                SVMMessageAddressTableLookup::from(&address_table_lookup),
812                &SlotHashes::default(),
813            ),
814            Err(AddressLookupError::InvalidAccountData),
815        );
816    }
817
818    #[test]
819    fn test_load_lookup_table_addresses() {
820        let ancestors = vec![(1, 1), (0, 0)].into_iter().collect();
821        let accounts_db = AccountsDb::new_single_for_tests();
822        let accounts = Accounts::new(Arc::new(accounts_db));
823
824        let table_key = Pubkey::new_unique();
825        let table_addresses = vec![Pubkey::new_unique(), Pubkey::new_unique()];
826        let table_account = {
827            let table_state = AddressLookupTable {
828                meta: LookupTableMeta::default(),
829                addresses: Cow::Owned(table_addresses.clone()),
830            };
831            AccountSharedData::create(
832                1,
833                table_state.serialize_for_tests().unwrap(),
834                address_lookup_table::program::id(),
835                false,
836                0,
837            )
838        };
839        accounts.store_slow_uncached(0, &table_key, &table_account);
840
841        let address_table_lookup = MessageAddressTableLookup {
842            account_key: table_key,
843            writable_indexes: vec![0],
844            readonly_indexes: vec![1],
845        };
846
847        assert_eq!(
848            accounts.load_lookup_table_addresses(
849                &ancestors,
850                SVMMessageAddressTableLookup::from(&address_table_lookup),
851                &SlotHashes::default(),
852            ),
853            Ok((
854                LoadedAddresses {
855                    writable: vec![table_addresses[0]],
856                    readonly: vec![table_addresses[1]],
857                },
858                u64::MAX
859            )),
860        );
861    }
862
863    #[test]
864    fn test_load_by_program_slot() {
865        let accounts_db = AccountsDb::new_single_for_tests();
866        let accounts = Accounts::new(Arc::new(accounts_db));
867
868        // Load accounts owned by various programs into AccountsDb
869        let pubkey0 = solana_pubkey::new_rand();
870        let account0 = AccountSharedData::new(1, 0, &Pubkey::from([2; 32]));
871        accounts.store_slow_uncached(0, &pubkey0, &account0);
872        let pubkey1 = solana_pubkey::new_rand();
873        let account1 = AccountSharedData::new(1, 0, &Pubkey::from([2; 32]));
874        accounts.store_slow_uncached(0, &pubkey1, &account1);
875        let pubkey2 = solana_pubkey::new_rand();
876        let account2 = AccountSharedData::new(1, 0, &Pubkey::from([3; 32]));
877        accounts.store_slow_uncached(0, &pubkey2, &account2);
878
879        let loaded = accounts.load_by_program_slot(0, Some(&Pubkey::from([2; 32])));
880        assert_eq!(loaded.len(), 2);
881        let loaded = accounts.load_by_program_slot(0, Some(&Pubkey::from([3; 32])));
882        assert_eq!(loaded, vec![(pubkey2, account2)]);
883        let loaded = accounts.load_by_program_slot(0, Some(&Pubkey::from([4; 32])));
884        assert_eq!(loaded, vec![]);
885    }
886
887    #[test]
888    fn test_lock_accounts_with_duplicates() {
889        let accounts_db = AccountsDb::new_single_for_tests();
890        let accounts = Accounts::new(Arc::new(accounts_db));
891
892        let keypair = Keypair::new();
893        let message = Message {
894            header: MessageHeader {
895                num_required_signatures: 1,
896                ..MessageHeader::default()
897            },
898            account_keys: vec![keypair.pubkey(), keypair.pubkey()],
899            ..Message::default()
900        };
901
902        let tx = new_sanitized_tx(&[&keypair], message, Hash::default());
903        let results = accounts.lock_accounts([tx].iter(), MAX_TX_ACCOUNT_LOCKS);
904        assert_eq!(results[0], Err(TransactionError::AccountLoadedTwice));
905    }
906
907    #[test]
908    fn test_lock_accounts_with_too_many_accounts() {
909        let accounts_db = AccountsDb::new_single_for_tests();
910        let accounts = Accounts::new(Arc::new(accounts_db));
911
912        let keypair = Keypair::new();
913
914        // Allow up to MAX_TX_ACCOUNT_LOCKS
915        {
916            let num_account_keys = MAX_TX_ACCOUNT_LOCKS;
917            let mut account_keys: Vec<_> = (0..num_account_keys)
918                .map(|_| Pubkey::new_unique())
919                .collect();
920            account_keys[0] = keypair.pubkey();
921            let message = Message {
922                header: MessageHeader {
923                    num_required_signatures: 1,
924                    ..MessageHeader::default()
925                },
926                account_keys,
927                ..Message::default()
928            };
929
930            let txs = vec![new_sanitized_tx(&[&keypair], message, Hash::default())];
931            let results = accounts.lock_accounts(txs.iter(), MAX_TX_ACCOUNT_LOCKS);
932            assert_eq!(results, vec![Ok(())]);
933            accounts.unlock_accounts(txs.iter().zip(&results));
934        }
935
936        // Disallow over MAX_TX_ACCOUNT_LOCKS
937        {
938            let num_account_keys = MAX_TX_ACCOUNT_LOCKS + 1;
939            let mut account_keys: Vec<_> = (0..num_account_keys)
940                .map(|_| Pubkey::new_unique())
941                .collect();
942            account_keys[0] = keypair.pubkey();
943            let message = Message {
944                header: MessageHeader {
945                    num_required_signatures: 1,
946                    ..MessageHeader::default()
947                },
948                account_keys,
949                ..Message::default()
950            };
951
952            let txs = vec![new_sanitized_tx(&[&keypair], message, Hash::default())];
953            let results = accounts.lock_accounts(txs.iter(), MAX_TX_ACCOUNT_LOCKS);
954            assert_eq!(results[0], Err(TransactionError::TooManyAccountLocks));
955        }
956    }
957
958    #[test]
959    fn test_accounts_locks() {
960        let keypair0 = Keypair::new();
961        let keypair1 = Keypair::new();
962        let keypair2 = Keypair::new();
963        let keypair3 = Keypair::new();
964
965        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
966        let account1 = AccountSharedData::new(2, 0, &Pubkey::default());
967        let account2 = AccountSharedData::new(3, 0, &Pubkey::default());
968        let account3 = AccountSharedData::new(4, 0, &Pubkey::default());
969
970        let accounts_db = AccountsDb::new_single_for_tests();
971        let accounts = Accounts::new(Arc::new(accounts_db));
972        accounts.store_for_tests(0, &keypair0.pubkey(), &account0);
973        accounts.store_for_tests(0, &keypair1.pubkey(), &account1);
974        accounts.store_for_tests(0, &keypair2.pubkey(), &account2);
975        accounts.store_for_tests(0, &keypair3.pubkey(), &account3);
976
977        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
978        let message = Message::new_with_compiled_instructions(
979            1,
980            0,
981            2,
982            vec![keypair0.pubkey(), keypair1.pubkey(), native_loader::id()],
983            Hash::default(),
984            instructions,
985        );
986        let tx = new_sanitized_tx(&[&keypair0], message, Hash::default());
987        let results0 = accounts.lock_accounts([tx.clone()].iter(), MAX_TX_ACCOUNT_LOCKS);
988
989        assert_eq!(results0, vec![Ok(())]);
990        assert!(accounts
991            .account_locks
992            .lock()
993            .unwrap()
994            .is_locked_readonly(&keypair1.pubkey()));
995
996        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
997        let message = Message::new_with_compiled_instructions(
998            1,
999            0,
1000            2,
1001            vec![keypair2.pubkey(), keypair1.pubkey(), native_loader::id()],
1002            Hash::default(),
1003            instructions,
1004        );
1005        let tx0 = new_sanitized_tx(&[&keypair2], message, Hash::default());
1006        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1007        let message = Message::new_with_compiled_instructions(
1008            1,
1009            0,
1010            2,
1011            vec![keypair1.pubkey(), keypair3.pubkey(), native_loader::id()],
1012            Hash::default(),
1013            instructions,
1014        );
1015        let tx1 = new_sanitized_tx(&[&keypair1], message, Hash::default());
1016        let txs = vec![tx0, tx1];
1017        let results1 = accounts.lock_accounts(txs.iter(), MAX_TX_ACCOUNT_LOCKS);
1018        assert_eq!(
1019            results1,
1020            vec![
1021                Ok(()), // Read-only account (keypair1) can be referenced multiple times
1022                Err(TransactionError::AccountInUse), // Read-only account (keypair1) cannot also be locked as writable
1023            ],
1024        );
1025        assert!(accounts
1026            .account_locks
1027            .lock()
1028            .unwrap()
1029            .is_locked_readonly(&keypair1.pubkey()));
1030
1031        accounts.unlock_accounts(iter::once(&tx).zip(&results0));
1032        accounts.unlock_accounts(txs.iter().zip(&results1));
1033        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1034        let message = Message::new_with_compiled_instructions(
1035            1,
1036            0,
1037            2,
1038            vec![keypair1.pubkey(), keypair3.pubkey(), native_loader::id()],
1039            Hash::default(),
1040            instructions,
1041        );
1042        let tx = new_sanitized_tx(&[&keypair1], message, Hash::default());
1043        let results2 = accounts.lock_accounts([tx].iter(), MAX_TX_ACCOUNT_LOCKS);
1044        assert_eq!(
1045            results2,
1046            vec![Ok(())] // Now keypair1 account can be locked as writable
1047        );
1048
1049        // Check that read-only lock with zero references is deleted
1050        assert!(!accounts
1051            .account_locks
1052            .lock()
1053            .unwrap()
1054            .is_locked_readonly(&keypair1.pubkey()));
1055    }
1056
1057    #[test]
1058    fn test_accounts_locks_multithreaded() {
1059        let counter = Arc::new(AtomicU64::new(0));
1060        let exit = Arc::new(AtomicBool::new(false));
1061
1062        let keypair0 = Keypair::new();
1063        let keypair1 = Keypair::new();
1064        let keypair2 = Keypair::new();
1065
1066        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
1067        let account1 = AccountSharedData::new(2, 0, &Pubkey::default());
1068        let account2 = AccountSharedData::new(3, 0, &Pubkey::default());
1069
1070        let accounts_db = AccountsDb::new_single_for_tests();
1071        let accounts = Accounts::new(Arc::new(accounts_db));
1072        accounts.store_for_tests(0, &keypair0.pubkey(), &account0);
1073        accounts.store_for_tests(0, &keypair1.pubkey(), &account1);
1074        accounts.store_for_tests(0, &keypair2.pubkey(), &account2);
1075
1076        let accounts_arc = Arc::new(accounts);
1077
1078        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1079        let readonly_message = Message::new_with_compiled_instructions(
1080            1,
1081            0,
1082            2,
1083            vec![keypair0.pubkey(), keypair1.pubkey(), native_loader::id()],
1084            Hash::default(),
1085            instructions,
1086        );
1087        let readonly_tx = new_sanitized_tx(&[&keypair0], readonly_message, Hash::default());
1088
1089        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1090        let writable_message = Message::new_with_compiled_instructions(
1091            1,
1092            0,
1093            2,
1094            vec![keypair1.pubkey(), keypair2.pubkey(), native_loader::id()],
1095            Hash::default(),
1096            instructions,
1097        );
1098        let writable_tx = new_sanitized_tx(&[&keypair1], writable_message, Hash::default());
1099
1100        let counter_clone = counter.clone();
1101        let accounts_clone = accounts_arc.clone();
1102        let exit_clone = exit.clone();
1103        thread::spawn(move || loop {
1104            let txs = vec![writable_tx.clone()];
1105            let results = accounts_clone
1106                .clone()
1107                .lock_accounts(txs.iter(), MAX_TX_ACCOUNT_LOCKS);
1108            for result in results.iter() {
1109                if result.is_ok() {
1110                    counter_clone.clone().fetch_add(1, Ordering::Release);
1111                }
1112            }
1113            accounts_clone.unlock_accounts(txs.iter().zip(&results));
1114            if exit_clone.clone().load(Ordering::Relaxed) {
1115                break;
1116            }
1117        });
1118        let counter_clone = counter;
1119        for _ in 0..5 {
1120            let txs = vec![readonly_tx.clone()];
1121            let results = accounts_arc
1122                .clone()
1123                .lock_accounts(txs.iter(), MAX_TX_ACCOUNT_LOCKS);
1124            if results[0].is_ok() {
1125                let counter_value = counter_clone.clone().load(Ordering::Acquire);
1126                thread::sleep(time::Duration::from_millis(50));
1127                assert_eq!(counter_value, counter_clone.clone().load(Ordering::Acquire));
1128            }
1129            accounts_arc.unlock_accounts(txs.iter().zip(&results));
1130            thread::sleep(time::Duration::from_millis(50));
1131        }
1132        exit.store(true, Ordering::Relaxed);
1133    }
1134
1135    #[test]
1136    fn test_demote_program_write_locks() {
1137        let keypair0 = Keypair::new();
1138        let keypair1 = Keypair::new();
1139        let keypair2 = Keypair::new();
1140        let keypair3 = Keypair::new();
1141
1142        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
1143        let account1 = AccountSharedData::new(2, 0, &Pubkey::default());
1144        let account2 = AccountSharedData::new(3, 0, &Pubkey::default());
1145        let account3 = AccountSharedData::new(4, 0, &Pubkey::default());
1146
1147        let accounts_db = AccountsDb::new_single_for_tests();
1148        let accounts = Accounts::new(Arc::new(accounts_db));
1149        accounts.store_for_tests(0, &keypair0.pubkey(), &account0);
1150        accounts.store_for_tests(0, &keypair1.pubkey(), &account1);
1151        accounts.store_for_tests(0, &keypair2.pubkey(), &account2);
1152        accounts.store_for_tests(0, &keypair3.pubkey(), &account3);
1153
1154        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1155        let message = Message::new_with_compiled_instructions(
1156            1,
1157            0,
1158            0, // All accounts marked as writable
1159            vec![keypair0.pubkey(), keypair1.pubkey(), native_loader::id()],
1160            Hash::default(),
1161            instructions,
1162        );
1163        let tx = new_sanitized_tx(&[&keypair0], message, Hash::default());
1164        let results0 = accounts.lock_accounts([tx].iter(), MAX_TX_ACCOUNT_LOCKS);
1165
1166        assert!(results0[0].is_ok());
1167        // Instruction program-id account demoted to readonly
1168        assert!(accounts
1169            .account_locks
1170            .lock()
1171            .unwrap()
1172            .is_locked_readonly(&native_loader::id()));
1173        // Non-program accounts remain writable
1174        assert!(accounts
1175            .account_locks
1176            .lock()
1177            .unwrap()
1178            .is_locked_write(&keypair0.pubkey()));
1179        assert!(accounts
1180            .account_locks
1181            .lock()
1182            .unwrap()
1183            .is_locked_write(&keypair1.pubkey()));
1184    }
1185
1186    impl Accounts {
1187        /// callers used to call store_uncached. But, this is not allowed anymore.
1188        pub fn store_for_tests(&self, slot: Slot, pubkey: &Pubkey, account: &AccountSharedData) {
1189            self.accounts_db.store_for_tests(slot, &[(pubkey, account)])
1190        }
1191
1192        /// useful to adapt tests written prior to introduction of the write cache
1193        /// to use the write cache
1194        pub fn add_root_and_flush_write_cache(&self, slot: Slot) {
1195            self.add_root(slot);
1196            self.accounts_db.flush_accounts_cache_slot_for_tests(slot);
1197        }
1198    }
1199
1200    #[test]
1201    fn test_accounts_locks_with_results() {
1202        let keypair0 = Keypair::new();
1203        let keypair1 = Keypair::new();
1204        let keypair2 = Keypair::new();
1205        let keypair3 = Keypair::new();
1206
1207        let account0 = AccountSharedData::new(1, 0, &Pubkey::default());
1208        let account1 = AccountSharedData::new(2, 0, &Pubkey::default());
1209        let account2 = AccountSharedData::new(3, 0, &Pubkey::default());
1210        let account3 = AccountSharedData::new(4, 0, &Pubkey::default());
1211
1212        let accounts_db = AccountsDb::new_single_for_tests();
1213        let accounts = Accounts::new(Arc::new(accounts_db));
1214        accounts.store_for_tests(0, &keypair0.pubkey(), &account0);
1215        accounts.store_for_tests(0, &keypair1.pubkey(), &account1);
1216        accounts.store_for_tests(0, &keypair2.pubkey(), &account2);
1217        accounts.store_for_tests(0, &keypair3.pubkey(), &account3);
1218
1219        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1220        let message = Message::new_with_compiled_instructions(
1221            1,
1222            0,
1223            2,
1224            vec![keypair1.pubkey(), keypair0.pubkey(), native_loader::id()],
1225            Hash::default(),
1226            instructions,
1227        );
1228        let tx0 = new_sanitized_tx(&[&keypair1], message, Hash::default());
1229        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1230        let message = Message::new_with_compiled_instructions(
1231            1,
1232            0,
1233            2,
1234            vec![keypair2.pubkey(), keypair0.pubkey(), native_loader::id()],
1235            Hash::default(),
1236            instructions,
1237        );
1238        let tx1 = new_sanitized_tx(&[&keypair2], message, Hash::default());
1239        let instructions = vec![CompiledInstruction::new(2, &(), vec![0, 1])];
1240        let message = Message::new_with_compiled_instructions(
1241            1,
1242            0,
1243            2,
1244            vec![keypair3.pubkey(), keypair0.pubkey(), native_loader::id()],
1245            Hash::default(),
1246            instructions,
1247        );
1248        let tx2 = new_sanitized_tx(&[&keypair3], message, Hash::default());
1249        let txs = vec![tx0, tx1, tx2];
1250
1251        let qos_results = vec![
1252            Ok(()),
1253            Err(TransactionError::WouldExceedMaxBlockCostLimit),
1254            Ok(()),
1255        ];
1256
1257        let results = accounts.lock_accounts_with_results(
1258            txs.iter(),
1259            qos_results.into_iter(),
1260            MAX_TX_ACCOUNT_LOCKS,
1261        );
1262
1263        assert_eq!(
1264            results,
1265            vec![
1266                Ok(()), // Read-only account (keypair0) can be referenced multiple times
1267                Err(TransactionError::WouldExceedMaxBlockCostLimit), // is not locked due to !qos_results[1].is_ok()
1268                Ok(()), // Read-only account (keypair0) can be referenced multiple times
1269            ],
1270        );
1271
1272        // verify that keypair0 read-only locked
1273        assert!(accounts
1274            .account_locks
1275            .lock()
1276            .unwrap()
1277            .is_locked_readonly(&keypair0.pubkey()));
1278        // verify that keypair2 (for tx1) is not write-locked
1279        assert!(!accounts
1280            .account_locks
1281            .lock()
1282            .unwrap()
1283            .is_locked_write(&keypair2.pubkey()));
1284    }
1285
1286    #[test]
1287    fn huge_clean() {
1288        solana_logger::setup();
1289        let accounts_db = AccountsDb::new_single_for_tests();
1290        let accounts = Accounts::new(Arc::new(accounts_db));
1291        let mut old_pubkey = Pubkey::default();
1292        let zero_account = AccountSharedData::new(0, 0, AccountSharedData::default().owner());
1293        info!("storing..");
1294        for i in 0..2_000 {
1295            let pubkey = solana_pubkey::new_rand();
1296            let account = AccountSharedData::new(i + 1, 0, AccountSharedData::default().owner());
1297            accounts.store_for_tests(i, &pubkey, &account);
1298            accounts.store_for_tests(i, &old_pubkey, &zero_account);
1299            old_pubkey = pubkey;
1300            accounts.add_root_and_flush_write_cache(i);
1301
1302            if i % 1_000 == 0 {
1303                info!("  store {}", i);
1304            }
1305        }
1306        info!("done..cleaning..");
1307        accounts.accounts_db.clean_accounts_for_tests();
1308    }
1309
1310    #[test]
1311    fn test_load_largest_accounts() {
1312        let accounts_db = AccountsDb::new_single_for_tests();
1313        let accounts = Accounts::new(Arc::new(accounts_db));
1314
1315        /* This test assumes pubkey0 < pubkey1 < pubkey2.
1316         * But the keys created with new_unique() does not guarantee this
1317         * order because of the endianness.  new_unique() calls add 1 at each
1318         * key generaration as the little endian integer.  A pubkey stores its
1319         * value in a 32-byte array bytes, and its eq-partial trait considers
1320         * the lower-address bytes more significant, which is the big-endian
1321         * order.
1322         * So, sort first to ensure the order assumption holds.
1323         */
1324        let mut keys = vec![];
1325        for _idx in 0..3 {
1326            keys.push(Pubkey::new_unique());
1327        }
1328        keys.sort();
1329        let pubkey2 = keys.pop().unwrap();
1330        let pubkey1 = keys.pop().unwrap();
1331        let pubkey0 = keys.pop().unwrap();
1332        let account0 = AccountSharedData::new(42, 0, &Pubkey::default());
1333        accounts.store_for_tests(0, &pubkey0, &account0);
1334        let account1 = AccountSharedData::new(42, 0, &Pubkey::default());
1335        accounts.store_for_tests(0, &pubkey1, &account1);
1336        let account2 = AccountSharedData::new(41, 0, &Pubkey::default());
1337        accounts.store_for_tests(0, &pubkey2, &account2);
1338
1339        let ancestors = vec![(0, 0)].into_iter().collect();
1340        let all_pubkeys: HashSet<_> = vec![pubkey0, pubkey1, pubkey2].into_iter().collect();
1341
1342        // num == 0 should always return empty set
1343        let bank_id = 0;
1344        assert_eq!(
1345            accounts
1346                .load_largest_accounts(
1347                    &ancestors,
1348                    bank_id,
1349                    0,
1350                    &HashSet::new(),
1351                    AccountAddressFilter::Exclude,
1352                    false
1353                )
1354                .unwrap(),
1355            vec![]
1356        );
1357        assert_eq!(
1358            accounts
1359                .load_largest_accounts(
1360                    &ancestors,
1361                    bank_id,
1362                    0,
1363                    &all_pubkeys,
1364                    AccountAddressFilter::Include,
1365                    false
1366                )
1367                .unwrap(),
1368            vec![]
1369        );
1370
1371        // list should be sorted by balance, then pubkey, descending
1372        assert!(pubkey1 > pubkey0);
1373        assert_eq!(
1374            accounts
1375                .load_largest_accounts(
1376                    &ancestors,
1377                    bank_id,
1378                    1,
1379                    &HashSet::new(),
1380                    AccountAddressFilter::Exclude,
1381                    false
1382                )
1383                .unwrap(),
1384            vec![(pubkey1, 42)]
1385        );
1386        assert_eq!(
1387            accounts
1388                .load_largest_accounts(
1389                    &ancestors,
1390                    bank_id,
1391                    2,
1392                    &HashSet::new(),
1393                    AccountAddressFilter::Exclude,
1394                    false
1395                )
1396                .unwrap(),
1397            vec![(pubkey1, 42), (pubkey0, 42)]
1398        );
1399        assert_eq!(
1400            accounts
1401                .load_largest_accounts(
1402                    &ancestors,
1403                    bank_id,
1404                    3,
1405                    &HashSet::new(),
1406                    AccountAddressFilter::Exclude,
1407                    false
1408                )
1409                .unwrap(),
1410            vec![(pubkey1, 42), (pubkey0, 42), (pubkey2, 41)]
1411        );
1412
1413        // larger num should not affect results
1414        assert_eq!(
1415            accounts
1416                .load_largest_accounts(
1417                    &ancestors,
1418                    bank_id,
1419                    6,
1420                    &HashSet::new(),
1421                    AccountAddressFilter::Exclude,
1422                    false
1423                )
1424                .unwrap(),
1425            vec![(pubkey1, 42), (pubkey0, 42), (pubkey2, 41)]
1426        );
1427
1428        // AccountAddressFilter::Exclude should exclude entry
1429        let exclude1: HashSet<_> = vec![pubkey1].into_iter().collect();
1430        assert_eq!(
1431            accounts
1432                .load_largest_accounts(
1433                    &ancestors,
1434                    bank_id,
1435                    1,
1436                    &exclude1,
1437                    AccountAddressFilter::Exclude,
1438                    false
1439                )
1440                .unwrap(),
1441            vec![(pubkey0, 42)]
1442        );
1443        assert_eq!(
1444            accounts
1445                .load_largest_accounts(
1446                    &ancestors,
1447                    bank_id,
1448                    2,
1449                    &exclude1,
1450                    AccountAddressFilter::Exclude,
1451                    false
1452                )
1453                .unwrap(),
1454            vec![(pubkey0, 42), (pubkey2, 41)]
1455        );
1456        assert_eq!(
1457            accounts
1458                .load_largest_accounts(
1459                    &ancestors,
1460                    bank_id,
1461                    3,
1462                    &exclude1,
1463                    AccountAddressFilter::Exclude,
1464                    false
1465                )
1466                .unwrap(),
1467            vec![(pubkey0, 42), (pubkey2, 41)]
1468        );
1469
1470        // AccountAddressFilter::Include should limit entries
1471        let include1_2: HashSet<_> = vec![pubkey1, pubkey2].into_iter().collect();
1472        assert_eq!(
1473            accounts
1474                .load_largest_accounts(
1475                    &ancestors,
1476                    bank_id,
1477                    1,
1478                    &include1_2,
1479                    AccountAddressFilter::Include,
1480                    false
1481                )
1482                .unwrap(),
1483            vec![(pubkey1, 42)]
1484        );
1485        assert_eq!(
1486            accounts
1487                .load_largest_accounts(
1488                    &ancestors,
1489                    bank_id,
1490                    2,
1491                    &include1_2,
1492                    AccountAddressFilter::Include,
1493                    false
1494                )
1495                .unwrap(),
1496            vec![(pubkey1, 42), (pubkey2, 41)]
1497        );
1498        assert_eq!(
1499            accounts
1500                .load_largest_accounts(
1501                    &ancestors,
1502                    bank_id,
1503                    3,
1504                    &include1_2,
1505                    AccountAddressFilter::Include,
1506                    false
1507                )
1508                .unwrap(),
1509            vec![(pubkey1, 42), (pubkey2, 41)]
1510        );
1511    }
1512
1513    fn zero_len_account_size() -> usize {
1514        std::mem::size_of::<AccountSharedData>() + std::mem::size_of::<Pubkey>()
1515    }
1516
1517    #[test]
1518    fn test_calc_scan_result_size() {
1519        for len in 0..3 {
1520            assert_eq!(
1521                Accounts::calc_scan_result_size(&AccountSharedData::new(
1522                    0,
1523                    len,
1524                    &Pubkey::default()
1525                )),
1526                zero_len_account_size() + len
1527            );
1528        }
1529    }
1530
1531    #[test]
1532    fn test_maybe_abort_scan() {
1533        assert!(Accounts::maybe_abort_scan(ScanResult::Ok(vec![]), &ScanConfig::default()).is_ok());
1534        assert!(
1535            Accounts::maybe_abort_scan(ScanResult::Ok(vec![]), &ScanConfig::new(false)).is_ok()
1536        );
1537        let config = ScanConfig::new(false).recreate_with_abort();
1538        assert!(Accounts::maybe_abort_scan(ScanResult::Ok(vec![]), &config).is_ok());
1539        config.abort();
1540        assert!(Accounts::maybe_abort_scan(ScanResult::Ok(vec![]), &config).is_err());
1541    }
1542
1543    #[test]
1544    fn test_accumulate_and_check_scan_result_size() {
1545        for (account, byte_limit_for_scan, result) in [
1546            (AccountSharedData::default(), zero_len_account_size(), false),
1547            (
1548                AccountSharedData::new(0, 1, &Pubkey::default()),
1549                zero_len_account_size(),
1550                true,
1551            ),
1552            (
1553                AccountSharedData::new(0, 2, &Pubkey::default()),
1554                zero_len_account_size() + 3,
1555                false,
1556            ),
1557        ] {
1558            let sum = AtomicUsize::default();
1559            assert_eq!(
1560                result,
1561                Accounts::accumulate_and_check_scan_result_size(
1562                    &sum,
1563                    &account,
1564                    &Some(byte_limit_for_scan)
1565                )
1566            );
1567            // calling a second time should accumulate above the threshold
1568            assert!(Accounts::accumulate_and_check_scan_result_size(
1569                &sum,
1570                &account,
1571                &Some(byte_limit_for_scan)
1572            ));
1573            assert!(!Accounts::accumulate_and_check_scan_result_size(
1574                &sum, &account, &None
1575            ));
1576        }
1577    }
1578}