solana_accounts_db/
accounts.rs

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