solana_runtime/
bank.rs

1//! The `bank` module tracks client accounts and the progress of on-chain
2//! programs.
3//!
4//! A single bank relates to a block produced by a single leader and each bank
5//! except for the genesis bank points back to a parent bank.
6//!
7//! The bank is the main entrypoint for processing verified transactions with the function
8//! `Bank::process_transactions`
9//!
10//! It does this by loading the accounts using the reference it holds on the account store,
11//! and then passing those to an InvokeContext which handles loading the programs specified
12//! by the Transaction and executing it.
13//!
14//! The bank then stores the results to the accounts store.
15//!
16//! It then has APIs for retrieving if a transaction has been processed and it's status.
17//! See `get_signature_status` et al.
18//!
19//! Bank lifecycle:
20//!
21//! A bank is newly created and open to transactions. Transactions are applied
22//! until either the bank reached the tick count when the node is the leader for that slot, or the
23//! node has applied all transactions present in all `Entry`s in the slot.
24//!
25//! Once it is complete, the bank can then be frozen. After frozen, no more transactions can
26//! be applied or state changes made. At the frozen step, rent will be applied and various
27//! sysvar special accounts update to the new state of the system.
28//!
29//! After frozen, and the bank has had the appropriate number of votes on it, then it can become
30//! rooted. At this point, it will not be able to be removed from the chain and the
31//! state is finalized.
32//!
33//! It offers a high-level API that signs transactions
34//! on behalf of the caller, and a low-level API for when they have
35//! already been signed and verified.
36use {
37    crate::{
38        account_saver::collect_accounts_to_store,
39        bank::{
40            metrics::*,
41            partitioned_epoch_rewards::{EpochRewardStatus, VoteRewardsAccounts},
42        },
43        bank_forks::BankForks,
44        epoch_stakes::{NodeVoteAccounts, VersionedEpochStakes},
45        inflation_rewards::points::InflationPointCalculationEvent,
46        installed_scheduler_pool::{BankWithScheduler, InstalledSchedulerRwLock},
47        rent_collector::RentCollector,
48        runtime_config::RuntimeConfig,
49        stake_account::StakeAccount,
50        stake_history::StakeHistory as CowStakeHistory,
51        stake_weighted_timestamp::{
52            calculate_stake_weighted_timestamp, MaxAllowableDrift,
53            MAX_ALLOWABLE_DRIFT_PERCENTAGE_FAST, MAX_ALLOWABLE_DRIFT_PERCENTAGE_SLOW_V2,
54        },
55        stakes::{SerdeStakesToStakeFormat, Stakes, StakesCache},
56        status_cache::{SlotDelta, StatusCache},
57        transaction_batch::{OwnedOrBorrowed, TransactionBatch},
58    },
59    accounts_lt_hash::{CacheValue as AccountsLtHashCacheValue, Stats as AccountsLtHashStats},
60    agave_feature_set::{
61        self as feature_set, increase_cpi_account_info_limit, raise_cpi_nesting_limit_to_8,
62        FeatureSet,
63    },
64    agave_precompiles::{get_precompile, get_precompiles, is_precompile},
65    agave_reserved_account_keys::ReservedAccountKeys,
66    agave_snapshots::snapshot_hash::SnapshotHash,
67    agave_syscalls::{
68        create_program_runtime_environment_v1, create_program_runtime_environment_v2,
69    },
70    ahash::AHashSet,
71    dashmap::DashMap,
72    log::*,
73    partitioned_epoch_rewards::PartitionedRewardsCalculation,
74    rayon::{ThreadPool, ThreadPoolBuilder},
75    serde::{Deserialize, Serialize},
76    solana_account::{
77        create_account_shared_data_with_fields as create_account, from_account, Account,
78        AccountSharedData, InheritableAccountFields, ReadableAccount, WritableAccount,
79    },
80    solana_accounts_db::{
81        account_locks::validate_account_locks,
82        accounts::{AccountAddressFilter, Accounts, PubkeyAccountSlot},
83        accounts_db::{AccountStorageEntry, AccountsDb, AccountsDbConfig},
84        accounts_hash::AccountsLtHash,
85        accounts_index::{IndexKey, ScanConfig, ScanResult},
86        accounts_update_notifier_interface::AccountsUpdateNotifier,
87        ancestors::{Ancestors, AncestorsForSerialization},
88        blockhash_queue::BlockhashQueue,
89        storable_accounts::StorableAccounts,
90        utils::create_account_shared_data,
91    },
92    solana_builtins::{BUILTINS, STATELESS_BUILTINS},
93    solana_clock::{
94        BankId, Epoch, Slot, SlotIndex, UnixTimestamp, INITIAL_RENT_EPOCH, MAX_PROCESSING_AGE,
95        MAX_TRANSACTION_FORWARDING_DELAY,
96    },
97    solana_cluster_type::ClusterType,
98    solana_compute_budget::compute_budget::ComputeBudget,
99    solana_compute_budget_instruction::instructions_processor::process_compute_budget_instructions,
100    solana_cost_model::{block_cost_limits::simd_0286_block_limits, cost_tracker::CostTracker},
101    solana_epoch_info::EpochInfo,
102    solana_epoch_schedule::EpochSchedule,
103    solana_feature_gate_interface as feature,
104    solana_fee::FeeFeatures,
105    solana_fee_calculator::FeeRateGovernor,
106    solana_fee_structure::{FeeBudgetLimits, FeeDetails, FeeStructure},
107    solana_genesis_config::GenesisConfig,
108    solana_hard_forks::HardForks,
109    solana_hash::Hash,
110    solana_inflation::Inflation,
111    solana_keypair::Keypair,
112    solana_lattice_hash::lt_hash::LtHash,
113    solana_measure::{measure::Measure, measure_time, measure_us},
114    solana_message::{inner_instruction::InnerInstructions, AccountKeys, SanitizedMessage},
115    solana_packet::PACKET_DATA_SIZE,
116    solana_precompile_error::PrecompileError,
117    solana_program_runtime::{
118        invoke_context::BuiltinFunctionWithContext,
119        loaded_programs::{ProgramCacheEntry, ProgramRuntimeEnvironments},
120    },
121    solana_pubkey::{Pubkey, PubkeyHasherBuilder},
122    solana_reward_info::RewardInfo,
123    solana_runtime_transaction::{
124        runtime_transaction::RuntimeTransaction, transaction_with_meta::TransactionWithMeta,
125    },
126    solana_sdk_ids::{bpf_loader_upgradeable, incinerator, native_loader},
127    solana_sha256_hasher::hashv,
128    solana_signature::Signature,
129    solana_slot_hashes::SlotHashes,
130    solana_slot_history::{Check, SlotHistory},
131    solana_stake_interface::{
132        stake_history::StakeHistory, state::Delegation, sysvar::stake_history,
133    },
134    solana_svm::{
135        account_loader::LoadedTransaction,
136        account_overrides::AccountOverrides,
137        program_loader::load_program_with_pubkey,
138        transaction_balances::{BalanceCollector, SvmTokenInfo},
139        transaction_commit_result::{CommittedTransaction, TransactionCommitResult},
140        transaction_error_metrics::TransactionErrorMetrics,
141        transaction_execution_result::{
142            TransactionExecutionDetails, TransactionLoadedAccountsStats,
143        },
144        transaction_processing_result::{
145            ProcessedTransaction, TransactionProcessingResult,
146            TransactionProcessingResultExtensions,
147        },
148        transaction_processor::{
149            ExecutionRecordingConfig, TransactionBatchProcessor, TransactionLogMessages,
150            TransactionProcessingConfig, TransactionProcessingEnvironment,
151        },
152    },
153    solana_svm_callback::{AccountState, InvokeContextCallback, TransactionProcessingCallback},
154    solana_svm_timings::{ExecuteTimingType, ExecuteTimings},
155    solana_svm_transaction::svm_message::SVMMessage,
156    solana_system_transaction as system_transaction,
157    solana_sysvar::{self as sysvar, last_restart_slot::LastRestartSlot, SysvarSerialize},
158    solana_sysvar_id::SysvarId,
159    solana_time_utils::years_as_slots,
160    solana_transaction::{
161        sanitized::{MessageHash, SanitizedTransaction, MAX_TX_ACCOUNT_LOCKS},
162        versioned::VersionedTransaction,
163        Transaction, TransactionVerificationMode,
164    },
165    solana_transaction_context::{
166        transaction_accounts::KeyedAccountSharedData, TransactionReturnData,
167    },
168    solana_transaction_error::{TransactionError, TransactionResult as Result},
169    solana_vote::vote_account::{VoteAccount, VoteAccounts, VoteAccountsHashMap},
170    std::{
171        collections::{HashMap, HashSet},
172        fmt,
173        ops::AddAssign,
174        path::PathBuf,
175        slice,
176        sync::{
177            atomic::{
178                AtomicBool, AtomicI64, AtomicU64,
179                Ordering::{self, AcqRel, Acquire, Relaxed},
180            },
181            Arc, LockResult, Mutex, RwLock, RwLockReadGuard, RwLockWriteGuard, Weak,
182        },
183        time::{Duration, Instant},
184    },
185};
186#[cfg(feature = "dev-context-only-utils")]
187use {
188    dashmap::DashSet,
189    rayon::iter::{IntoParallelRefIterator, ParallelIterator},
190    solana_accounts_db::accounts_db::{
191        ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS, ACCOUNTS_DB_CONFIG_FOR_TESTING,
192    },
193    solana_nonce as nonce,
194    solana_nonce_account::{get_system_account_kind, SystemAccountKind},
195    solana_program_runtime::sysvar_cache::SysvarCache,
196};
197pub use {partitioned_epoch_rewards::KeyedRewardsAndNumPartitions, solana_reward_info::RewardType};
198
199/// params to `verify_accounts_hash`
200struct VerifyAccountsHashConfig {
201    require_rooted_bank: bool,
202}
203
204mod accounts_lt_hash;
205mod address_lookup_table;
206pub mod bank_hash_details;
207pub mod builtins;
208mod check_transactions;
209mod fee_distribution;
210mod metrics;
211pub(crate) mod partitioned_epoch_rewards;
212mod recent_blockhashes_account;
213mod serde_snapshot;
214mod sysvar_cache;
215pub(crate) mod tests;
216
217pub const SECONDS_PER_YEAR: f64 = 365.25 * 24.0 * 60.0 * 60.0;
218
219pub const MAX_LEADER_SCHEDULE_STAKES: Epoch = 5;
220
221pub type BankStatusCache = StatusCache<Result<()>>;
222#[cfg_attr(
223    feature = "frozen-abi",
224    frozen_abi(digest = "FUttxQbsCnX5VMRuj8c2sUxZKNARUTaomdgsbg8wM3D6")
225)]
226pub type BankSlotDelta = SlotDelta<Result<()>>;
227
228#[derive(Default, Copy, Clone, Debug, PartialEq, Eq)]
229pub struct SquashTiming {
230    pub squash_accounts_ms: u64,
231    pub squash_accounts_cache_ms: u64,
232    pub squash_accounts_index_ms: u64,
233    pub squash_cache_ms: u64,
234}
235
236impl AddAssign for SquashTiming {
237    fn add_assign(&mut self, rhs: Self) {
238        self.squash_accounts_ms += rhs.squash_accounts_ms;
239        self.squash_accounts_cache_ms += rhs.squash_accounts_cache_ms;
240        self.squash_accounts_index_ms += rhs.squash_accounts_index_ms;
241        self.squash_cache_ms += rhs.squash_cache_ms;
242    }
243}
244
245#[derive(Clone, Debug, Default, PartialEq)]
246pub struct CollectorFeeDetails {
247    transaction_fee: u64,
248    priority_fee: u64,
249}
250
251impl CollectorFeeDetails {
252    pub(crate) fn accumulate(&mut self, fee_details: &FeeDetails) {
253        self.transaction_fee = self
254            .transaction_fee
255            .saturating_add(fee_details.transaction_fee());
256        self.priority_fee = self
257            .priority_fee
258            .saturating_add(fee_details.prioritization_fee());
259    }
260
261    pub fn total_transaction_fee(&self) -> u64 {
262        self.transaction_fee.saturating_add(self.priority_fee)
263    }
264
265    pub fn total_priority_fee(&self) -> u64 {
266        self.priority_fee
267    }
268}
269
270impl From<FeeDetails> for CollectorFeeDetails {
271    fn from(fee_details: FeeDetails) -> Self {
272        CollectorFeeDetails {
273            transaction_fee: fee_details.transaction_fee(),
274            priority_fee: fee_details.prioritization_fee(),
275        }
276    }
277}
278
279#[derive(Debug)]
280pub struct BankRc {
281    /// where all the Accounts are stored
282    pub accounts: Arc<Accounts>,
283
284    /// Previous checkpoint of this bank
285    pub(crate) parent: RwLock<Option<Arc<Bank>>>,
286
287    pub(crate) bank_id_generator: Arc<AtomicU64>,
288}
289
290impl BankRc {
291    pub(crate) fn new(accounts: Accounts) -> Self {
292        Self {
293            accounts: Arc::new(accounts),
294            parent: RwLock::new(None),
295            bank_id_generator: Arc::new(AtomicU64::new(0)),
296        }
297    }
298}
299
300pub struct LoadAndExecuteTransactionsOutput {
301    // Vector of results indicating whether a transaction was processed or could not
302    // be processed. Note processed transactions can still have failed!
303    pub processing_results: Vec<TransactionProcessingResult>,
304    // Processed transaction counts used to update bank transaction counts and
305    // for metrics reporting.
306    pub processed_counts: ProcessedTransactionCounts,
307    // Balances accumulated for TransactionStatusSender when transaction
308    // balance recording is enabled.
309    pub balance_collector: Option<BalanceCollector>,
310}
311
312#[derive(Debug, PartialEq)]
313pub struct TransactionSimulationResult {
314    pub result: Result<()>,
315    pub logs: TransactionLogMessages,
316    pub post_simulation_accounts: Vec<KeyedAccountSharedData>,
317    pub units_consumed: u64,
318    pub loaded_accounts_data_size: u32,
319    pub return_data: Option<TransactionReturnData>,
320    pub inner_instructions: Option<Vec<InnerInstructions>>,
321    pub fee: Option<u64>,
322    pub pre_balances: Option<Vec<u64>>,
323    pub post_balances: Option<Vec<u64>>,
324    pub pre_token_balances: Option<Vec<SvmTokenInfo>>,
325    pub post_token_balances: Option<Vec<SvmTokenInfo>>,
326}
327
328impl TransactionSimulationResult {
329    pub fn new_error(err: TransactionError) -> Self {
330        Self {
331            fee: None,
332            inner_instructions: None,
333            loaded_accounts_data_size: 0,
334            logs: vec![],
335            post_balances: None,
336            post_simulation_accounts: vec![],
337            post_token_balances: None,
338            pre_balances: None,
339            pre_token_balances: None,
340            result: Err(err),
341            return_data: None,
342            units_consumed: 0,
343        }
344    }
345}
346
347#[derive(Clone, Debug)]
348pub struct TransactionBalancesSet {
349    pub pre_balances: TransactionBalances,
350    pub post_balances: TransactionBalances,
351}
352
353impl TransactionBalancesSet {
354    pub fn new(pre_balances: TransactionBalances, post_balances: TransactionBalances) -> Self {
355        assert_eq!(pre_balances.len(), post_balances.len());
356        Self {
357            pre_balances,
358            post_balances,
359        }
360    }
361}
362pub type TransactionBalances = Vec<Vec<u64>>;
363
364pub type PreCommitResult<'a> = Result<Option<RwLockReadGuard<'a, Hash>>>;
365
366#[derive(Serialize, Deserialize, Debug, PartialEq, Eq, Default)]
367pub enum TransactionLogCollectorFilter {
368    All,
369    AllWithVotes,
370    #[default]
371    None,
372    OnlyMentionedAddresses,
373}
374
375#[derive(Debug, Default)]
376pub struct TransactionLogCollectorConfig {
377    pub mentioned_addresses: HashSet<Pubkey>,
378    pub filter: TransactionLogCollectorFilter,
379}
380
381#[derive(Clone, Debug, PartialEq, Eq)]
382pub struct TransactionLogInfo {
383    pub signature: Signature,
384    pub result: Result<()>,
385    pub is_vote: bool,
386    pub log_messages: TransactionLogMessages,
387}
388
389#[derive(Default, Debug)]
390pub struct TransactionLogCollector {
391    // All the logs collected for from this Bank.  Exact contents depend on the
392    // active `TransactionLogCollectorFilter`
393    pub logs: Vec<TransactionLogInfo>,
394
395    // For each `mentioned_addresses`, maintain a list of indices into `logs` to easily
396    // locate the logs from transactions that included the mentioned addresses.
397    pub mentioned_address_map: HashMap<Pubkey, Vec<usize>>,
398}
399
400impl TransactionLogCollector {
401    pub fn get_logs_for_address(
402        &self,
403        address: Option<&Pubkey>,
404    ) -> Option<Vec<TransactionLogInfo>> {
405        match address {
406            None => Some(self.logs.clone()),
407            Some(address) => self.mentioned_address_map.get(address).map(|log_indices| {
408                log_indices
409                    .iter()
410                    .filter_map(|i| self.logs.get(*i).cloned())
411                    .collect()
412            }),
413        }
414    }
415}
416
417/// Bank's common fields shared by all supported snapshot versions for deserialization.
418/// Sync fields with BankFieldsToSerialize! This is paired with it.
419/// All members are made public to remain Bank's members private and to make versioned deserializer workable on this
420/// Note that some fields are missing from the serializer struct. This is because of fields added later.
421/// Since it is difficult to insert fields to serialize/deserialize against existing code already deployed,
422/// new fields can be optionally serialized and optionally deserialized. At some point, the serialization and
423/// deserialization will use a new mechanism or otherwise be in sync more clearly.
424#[derive(Clone, Debug)]
425#[cfg_attr(feature = "dev-context-only-utils", derive(PartialEq))]
426pub struct BankFieldsToDeserialize {
427    pub(crate) blockhash_queue: BlockhashQueue,
428    pub(crate) ancestors: AncestorsForSerialization,
429    pub(crate) hash: Hash,
430    pub(crate) parent_hash: Hash,
431    pub(crate) parent_slot: Slot,
432    pub(crate) hard_forks: HardForks,
433    pub(crate) transaction_count: u64,
434    pub(crate) tick_height: u64,
435    pub(crate) signature_count: u64,
436    pub(crate) capitalization: u64,
437    pub(crate) max_tick_height: u64,
438    pub(crate) hashes_per_tick: Option<u64>,
439    pub(crate) ticks_per_slot: u64,
440    pub(crate) ns_per_slot: u128,
441    pub(crate) genesis_creation_time: UnixTimestamp,
442    pub(crate) slots_per_year: f64,
443    pub(crate) slot: Slot,
444    pub(crate) epoch: Epoch,
445    pub(crate) block_height: u64,
446    pub(crate) collector_id: Pubkey,
447    pub(crate) collector_fees: u64,
448    pub(crate) fee_rate_governor: FeeRateGovernor,
449    pub(crate) rent_collector: RentCollector,
450    pub(crate) epoch_schedule: EpochSchedule,
451    pub(crate) inflation: Inflation,
452    pub(crate) stakes: Stakes<Delegation>,
453    pub(crate) versioned_epoch_stakes: HashMap<Epoch, VersionedEpochStakes>,
454    pub(crate) is_delta: bool,
455    pub(crate) accounts_data_len: u64,
456    pub(crate) accounts_lt_hash: AccountsLtHash,
457    pub(crate) bank_hash_stats: BankHashStats,
458}
459
460/// Bank's common fields shared by all supported snapshot versions for serialization.
461/// This was separated from BankFieldsToDeserialize to avoid cloning by using refs.
462/// So, sync fields with BankFieldsToDeserialize!
463/// all members are made public to keep Bank private and to make versioned serializer workable on this.
464/// Note that some fields are missing from the serializer struct. This is because of fields added later.
465/// Since it is difficult to insert fields to serialize/deserialize against existing code already deployed,
466/// new fields can be optionally serialized and optionally deserialized. At some point, the serialization and
467/// deserialization will use a new mechanism or otherwise be in sync more clearly.
468#[derive(Debug)]
469pub struct BankFieldsToSerialize {
470    pub blockhash_queue: BlockhashQueue,
471    pub ancestors: AncestorsForSerialization,
472    pub hash: Hash,
473    pub parent_hash: Hash,
474    pub parent_slot: Slot,
475    pub hard_forks: HardForks,
476    pub transaction_count: u64,
477    pub tick_height: u64,
478    pub signature_count: u64,
479    pub capitalization: u64,
480    pub max_tick_height: u64,
481    pub hashes_per_tick: Option<u64>,
482    pub ticks_per_slot: u64,
483    pub ns_per_slot: u128,
484    pub genesis_creation_time: UnixTimestamp,
485    pub slots_per_year: f64,
486    pub slot: Slot,
487    pub epoch: Epoch,
488    pub block_height: u64,
489    pub collector_id: Pubkey,
490    pub collector_fees: u64,
491    pub fee_rate_governor: FeeRateGovernor,
492    pub rent_collector: RentCollector,
493    pub epoch_schedule: EpochSchedule,
494    pub inflation: Inflation,
495    pub stakes: Stakes<StakeAccount<Delegation>>,
496    pub is_delta: bool,
497    pub accounts_data_len: u64,
498    pub versioned_epoch_stakes: HashMap<u64, VersionedEpochStakes>,
499    pub accounts_lt_hash: AccountsLtHash,
500}
501
502// Can't derive PartialEq because RwLock doesn't implement PartialEq
503#[cfg(feature = "dev-context-only-utils")]
504impl PartialEq for Bank {
505    fn eq(&self, other: &Self) -> bool {
506        if std::ptr::eq(self, other) {
507            return true;
508        }
509        // Suppress rustfmt until https://github.com/rust-lang/rustfmt/issues/5920 is fixed ...
510        #[rustfmt::skip]
511        let Self {
512            rc: _,
513            status_cache: _,
514            blockhash_queue,
515            ancestors,
516            hash,
517            parent_hash,
518            parent_slot,
519            hard_forks,
520            transaction_count,
521            non_vote_transaction_count_since_restart: _,
522            transaction_error_count: _,
523            transaction_entries_count: _,
524            transactions_per_entry_max: _,
525            tick_height,
526            signature_count,
527            capitalization,
528            max_tick_height,
529            hashes_per_tick,
530            ticks_per_slot,
531            ns_per_slot,
532            genesis_creation_time,
533            slots_per_year,
534            slot,
535            bank_id: _,
536            epoch,
537            block_height,
538            collector_id,
539            collector_fees,
540            fee_rate_governor,
541            rent_collector,
542            epoch_schedule,
543            inflation,
544            stakes_cache,
545            epoch_stakes,
546            is_delta,
547            #[cfg(feature = "dev-context-only-utils")]
548            hash_overrides,
549            accounts_lt_hash,
550            // TODO: Confirm if all these fields are intentionally ignored!
551            rewards: _,
552            cluster_type: _,
553            transaction_debug_keys: _,
554            transaction_log_collector_config: _,
555            transaction_log_collector: _,
556            feature_set: _,
557            reserved_account_keys: _,
558            drop_callback: _,
559            freeze_started: _,
560            vote_only_bank: _,
561            cost_tracker: _,
562            accounts_data_size_initial: _,
563            accounts_data_size_delta_on_chain: _,
564            accounts_data_size_delta_off_chain: _,
565            epoch_reward_status: _,
566            transaction_processor: _,
567            check_program_modification_slot: _,
568            collector_fee_details: _,
569            compute_budget: _,
570            transaction_account_lock_limit: _,
571            fee_structure: _,
572            cache_for_accounts_lt_hash: _,
573            stats_for_accounts_lt_hash: _,
574            block_id,
575            bank_hash_stats: _,
576            epoch_rewards_calculation_cache: _,
577            // Ignore new fields explicitly if they do not impact PartialEq.
578            // Adding ".." will remove compile-time checks that if a new field
579            // is added to the struct, this PartialEq is accordingly updated.
580        } = self;
581        *blockhash_queue.read().unwrap() == *other.blockhash_queue.read().unwrap()
582            && ancestors == &other.ancestors
583            && *hash.read().unwrap() == *other.hash.read().unwrap()
584            && parent_hash == &other.parent_hash
585            && parent_slot == &other.parent_slot
586            && *hard_forks.read().unwrap() == *other.hard_forks.read().unwrap()
587            && transaction_count.load(Relaxed) == other.transaction_count.load(Relaxed)
588            && tick_height.load(Relaxed) == other.tick_height.load(Relaxed)
589            && signature_count.load(Relaxed) == other.signature_count.load(Relaxed)
590            && capitalization.load(Relaxed) == other.capitalization.load(Relaxed)
591            && max_tick_height == &other.max_tick_height
592            && hashes_per_tick == &other.hashes_per_tick
593            && ticks_per_slot == &other.ticks_per_slot
594            && ns_per_slot == &other.ns_per_slot
595            && genesis_creation_time == &other.genesis_creation_time
596            && slots_per_year == &other.slots_per_year
597            && slot == &other.slot
598            && epoch == &other.epoch
599            && block_height == &other.block_height
600            && collector_id == &other.collector_id
601            && collector_fees.load(Relaxed) == other.collector_fees.load(Relaxed)
602            && fee_rate_governor == &other.fee_rate_governor
603            && rent_collector == &other.rent_collector
604            && epoch_schedule == &other.epoch_schedule
605            && *inflation.read().unwrap() == *other.inflation.read().unwrap()
606            && *stakes_cache.stakes() == *other.stakes_cache.stakes()
607            && epoch_stakes == &other.epoch_stakes
608            && is_delta.load(Relaxed) == other.is_delta.load(Relaxed)
609            // No deadlock is possbile, when Arc::ptr_eq() returns false, because of being
610            // different Mutexes.
611            && (Arc::ptr_eq(hash_overrides, &other.hash_overrides) ||
612                *hash_overrides.lock().unwrap() == *other.hash_overrides.lock().unwrap())
613            && *accounts_lt_hash.lock().unwrap() == *other.accounts_lt_hash.lock().unwrap()
614            && *block_id.read().unwrap() == *other.block_id.read().unwrap()
615    }
616}
617
618#[cfg(feature = "dev-context-only-utils")]
619impl BankFieldsToSerialize {
620    /// Create a new BankFieldsToSerialize where basically every field is defaulted.
621    /// Only use for tests; many of the fields are invalid!
622    pub fn default_for_tests() -> Self {
623        Self {
624            blockhash_queue: BlockhashQueue::default(),
625            ancestors: AncestorsForSerialization::default(),
626            hash: Hash::default(),
627            parent_hash: Hash::default(),
628            parent_slot: Slot::default(),
629            hard_forks: HardForks::default(),
630            transaction_count: u64::default(),
631            tick_height: u64::default(),
632            signature_count: u64::default(),
633            capitalization: u64::default(),
634            max_tick_height: u64::default(),
635            hashes_per_tick: Option::default(),
636            ticks_per_slot: u64::default(),
637            ns_per_slot: u128::default(),
638            genesis_creation_time: UnixTimestamp::default(),
639            slots_per_year: f64::default(),
640            slot: Slot::default(),
641            epoch: Epoch::default(),
642            block_height: u64::default(),
643            collector_id: Pubkey::default(),
644            collector_fees: u64::default(),
645            fee_rate_governor: FeeRateGovernor::default(),
646            rent_collector: RentCollector::default(),
647            epoch_schedule: EpochSchedule::default(),
648            inflation: Inflation::default(),
649            stakes: Stakes::<StakeAccount<Delegation>>::default(),
650            is_delta: bool::default(),
651            accounts_data_len: u64::default(),
652            versioned_epoch_stakes: HashMap::default(),
653            accounts_lt_hash: AccountsLtHash(LtHash([0x7E57; LtHash::NUM_ELEMENTS])),
654        }
655    }
656}
657
658#[derive(Debug)]
659pub enum RewardCalculationEvent<'a, 'b> {
660    Staking(&'a Pubkey, &'b InflationPointCalculationEvent),
661}
662
663/// type alias is not supported for trait in rust yet. As a workaround, we define the
664/// `RewardCalcTracer` trait explicitly and implement it on any type that implement
665/// `Fn(&RewardCalculationEvent) + Send + Sync`.
666pub trait RewardCalcTracer: Fn(&RewardCalculationEvent) + Send + Sync {}
667
668impl<T: Fn(&RewardCalculationEvent) + Send + Sync> RewardCalcTracer for T {}
669
670fn null_tracer() -> Option<impl RewardCalcTracer> {
671    None::<fn(&RewardCalculationEvent)>
672}
673
674pub trait DropCallback: fmt::Debug {
675    fn callback(&self, b: &Bank);
676    fn clone_box(&self) -> Box<dyn DropCallback + Send + Sync>;
677}
678
679#[derive(Debug, Default)]
680pub struct OptionalDropCallback(Option<Box<dyn DropCallback + Send + Sync>>);
681
682#[derive(Default, Debug, Clone, PartialEq)]
683#[cfg(feature = "dev-context-only-utils")]
684pub struct HashOverrides {
685    hashes: HashMap<Slot, HashOverride>,
686}
687
688#[cfg(feature = "dev-context-only-utils")]
689impl HashOverrides {
690    fn get_hash_override(&self, slot: Slot) -> Option<&HashOverride> {
691        self.hashes.get(&slot)
692    }
693
694    fn get_blockhash_override(&self, slot: Slot) -> Option<&Hash> {
695        self.get_hash_override(slot)
696            .map(|hash_override| &hash_override.blockhash)
697    }
698
699    fn get_bank_hash_override(&self, slot: Slot) -> Option<&Hash> {
700        self.get_hash_override(slot)
701            .map(|hash_override| &hash_override.bank_hash)
702    }
703
704    pub fn add_override(&mut self, slot: Slot, blockhash: Hash, bank_hash: Hash) {
705        let is_new = self
706            .hashes
707            .insert(
708                slot,
709                HashOverride {
710                    blockhash,
711                    bank_hash,
712                },
713            )
714            .is_none();
715        assert!(is_new);
716    }
717}
718
719#[derive(Debug, Clone, PartialEq)]
720#[cfg(feature = "dev-context-only-utils")]
721struct HashOverride {
722    blockhash: Hash,
723    bank_hash: Hash,
724}
725
726/// Manager for the state of all accounts and programs after processing its entries.
727pub struct Bank {
728    /// References to accounts, parent and signature status
729    pub rc: BankRc,
730
731    /// A cache of signature statuses
732    pub status_cache: Arc<RwLock<BankStatusCache>>,
733
734    /// FIFO queue of `recent_blockhash` items
735    blockhash_queue: RwLock<BlockhashQueue>,
736
737    /// The set of parents including this bank
738    pub ancestors: Ancestors,
739
740    /// Hash of this Bank's state. Only meaningful after freezing.
741    hash: RwLock<Hash>,
742
743    /// Hash of this Bank's parent's state
744    parent_hash: Hash,
745
746    /// parent's slot
747    parent_slot: Slot,
748
749    /// slots to hard fork at
750    hard_forks: Arc<RwLock<HardForks>>,
751
752    /// The number of committed transactions since genesis.
753    transaction_count: AtomicU64,
754
755    /// The number of non-vote transactions committed since the most
756    /// recent boot from snapshot or genesis. This value is only stored in
757    /// blockstore for the RPC method "getPerformanceSamples". It is not
758    /// retained within snapshots, but is preserved in `Bank::new_from_parent`.
759    non_vote_transaction_count_since_restart: AtomicU64,
760
761    /// The number of transaction errors in this slot
762    transaction_error_count: AtomicU64,
763
764    /// The number of transaction entries in this slot
765    transaction_entries_count: AtomicU64,
766
767    /// The max number of transaction in an entry in this slot
768    transactions_per_entry_max: AtomicU64,
769
770    /// Bank tick height
771    tick_height: AtomicU64,
772
773    /// The number of signatures from valid transactions in this slot
774    signature_count: AtomicU64,
775
776    /// Total capitalization, used to calculate inflation
777    capitalization: AtomicU64,
778
779    // Bank max_tick_height
780    max_tick_height: u64,
781
782    /// The number of hashes in each tick. None value means hashing is disabled.
783    hashes_per_tick: Option<u64>,
784
785    /// The number of ticks in each slot.
786    ticks_per_slot: u64,
787
788    /// length of a slot in ns
789    pub ns_per_slot: u128,
790
791    /// genesis time, used for computed clock
792    genesis_creation_time: UnixTimestamp,
793
794    /// The number of slots per year, used for inflation
795    slots_per_year: f64,
796
797    /// Bank slot (i.e. block)
798    slot: Slot,
799
800    bank_id: BankId,
801
802    /// Bank epoch
803    epoch: Epoch,
804
805    /// Bank block_height
806    block_height: u64,
807
808    /// The pubkey to send transactions fees to.
809    collector_id: Pubkey,
810
811    /// Fees that have been collected
812    collector_fees: AtomicU64,
813
814    /// Track cluster signature throughput and adjust fee rate
815    pub(crate) fee_rate_governor: FeeRateGovernor,
816
817    /// latest rent collector, knows the epoch
818    rent_collector: RentCollector,
819
820    /// initialized from genesis
821    pub(crate) epoch_schedule: EpochSchedule,
822
823    /// inflation specs
824    inflation: Arc<RwLock<Inflation>>,
825
826    /// cache of vote_account and stake_account state for this fork
827    stakes_cache: StakesCache,
828
829    /// staked nodes on epoch boundaries, saved off when a bank.slot() is at
830    ///   a leader schedule calculation boundary
831    epoch_stakes: HashMap<Epoch, VersionedEpochStakes>,
832
833    /// A boolean reflecting whether any entries were recorded into the PoH
834    /// stream for the slot == self.slot
835    is_delta: AtomicBool,
836
837    /// Protocol-level rewards that were distributed by this bank
838    pub rewards: RwLock<Vec<(Pubkey, RewardInfo)>>,
839
840    pub cluster_type: Option<ClusterType>,
841
842    transaction_debug_keys: Option<Arc<HashSet<Pubkey>>>,
843
844    // Global configuration for how transaction logs should be collected across all banks
845    pub transaction_log_collector_config: Arc<RwLock<TransactionLogCollectorConfig>>,
846
847    // Logs from transactions that this Bank executed collected according to the criteria in
848    // `transaction_log_collector_config`
849    pub transaction_log_collector: Arc<RwLock<TransactionLogCollector>>,
850
851    pub feature_set: Arc<FeatureSet>,
852
853    /// Set of reserved account keys that cannot be write locked
854    reserved_account_keys: Arc<ReservedAccountKeys>,
855
856    /// callback function only to be called when dropping and should only be called once
857    pub drop_callback: RwLock<OptionalDropCallback>,
858
859    pub freeze_started: AtomicBool,
860
861    vote_only_bank: bool,
862
863    cost_tracker: RwLock<CostTracker>,
864
865    /// The initial accounts data size at the start of this Bank, before processing any transactions/etc
866    accounts_data_size_initial: u64,
867    /// The change to accounts data size in this Bank, due on-chain events (i.e. transactions)
868    accounts_data_size_delta_on_chain: AtomicI64,
869    /// The change to accounts data size in this Bank, due to off-chain events (i.e. rent collection)
870    accounts_data_size_delta_off_chain: AtomicI64,
871
872    epoch_reward_status: EpochRewardStatus,
873
874    transaction_processor: TransactionBatchProcessor<BankForks>,
875
876    check_program_modification_slot: bool,
877
878    /// Collected fee details
879    collector_fee_details: RwLock<CollectorFeeDetails>,
880
881    /// The compute budget to use for transaction execution.
882    compute_budget: Option<ComputeBudget>,
883
884    /// The max number of accounts that a transaction may lock.
885    transaction_account_lock_limit: Option<usize>,
886
887    /// Fee structure to use for assessing transaction fees.
888    fee_structure: FeeStructure,
889
890    /// blockhash and bank_hash overrides keyed by slot for simulated block production.
891    /// This _field_ was needed to be DCOU-ed to avoid 2 locks per bank freezing...
892    #[cfg(feature = "dev-context-only-utils")]
893    hash_overrides: Arc<Mutex<HashOverrides>>,
894
895    /// The lattice hash of all accounts
896    ///
897    /// The value is only meaningful after freezing.
898    accounts_lt_hash: Mutex<AccountsLtHash>,
899
900    /// A cache of *the initial state* of accounts modified in this slot
901    ///
902    /// The accounts lt hash needs both the initial and final state of each
903    /// account that was modified in this slot.  Cache the initial state here.
904    ///
905    /// Note: The initial state must be strictly from an ancestor,
906    /// and not an intermediate state within this slot.
907    cache_for_accounts_lt_hash: DashMap<Pubkey, AccountsLtHashCacheValue, ahash::RandomState>,
908
909    /// Stats related to the accounts lt hash
910    stats_for_accounts_lt_hash: AccountsLtHashStats,
911
912    /// The unique identifier for the corresponding block for this bank.
913    /// None for banks that have not yet completed replay or for leader banks as we cannot populate block_id
914    /// until bankless leader. Can be computed directly from shreds without needing to execute transactions.
915    block_id: RwLock<Option<Hash>>,
916
917    /// Accounts stats for computing the bank hash
918    bank_hash_stats: AtomicBankHashStats,
919
920    /// The cache of epoch rewards calculation results
921    /// This is used to avoid recalculating the same epoch rewards at epoch boundary.
922    /// The hashmap is keyed by parent_hash.
923    epoch_rewards_calculation_cache: Arc<Mutex<HashMap<Hash, Arc<PartitionedRewardsCalculation>>>>,
924}
925
926#[derive(Debug)]
927struct VoteReward {
928    vote_account: AccountSharedData,
929    commission: u8,
930    vote_rewards: u64,
931}
932
933type VoteRewards = HashMap<Pubkey, VoteReward, PubkeyHasherBuilder>;
934
935#[derive(Debug, Default)]
936pub struct NewBankOptions {
937    pub vote_only_bank: bool,
938}
939
940#[cfg(feature = "dev-context-only-utils")]
941#[derive(Debug)]
942pub struct BankTestConfig {
943    pub accounts_db_config: AccountsDbConfig,
944}
945
946#[cfg(feature = "dev-context-only-utils")]
947impl Default for BankTestConfig {
948    fn default() -> Self {
949        Self {
950            accounts_db_config: ACCOUNTS_DB_CONFIG_FOR_TESTING,
951        }
952    }
953}
954
955#[derive(Debug)]
956struct PrevEpochInflationRewards {
957    validator_rewards: u64,
958    prev_epoch_duration_in_years: f64,
959    validator_rate: f64,
960    foundation_rate: f64,
961}
962
963#[derive(Debug, Default, PartialEq)]
964pub struct ProcessedTransactionCounts {
965    pub processed_transactions_count: u64,
966    pub processed_non_vote_transactions_count: u64,
967    pub processed_with_successful_result_count: u64,
968    pub signature_count: u64,
969}
970
971/// Account stats for computing the bank hash
972/// This struct is serialized and stored in the snapshot.
973#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
974#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, Eq)]
975pub struct BankHashStats {
976    pub num_updated_accounts: u64,
977    pub num_removed_accounts: u64,
978    pub num_lamports_stored: u64,
979    pub total_data_len: u64,
980    pub num_executable_accounts: u64,
981}
982
983impl BankHashStats {
984    pub fn update<T: ReadableAccount>(&mut self, account: &T) {
985        if account.lamports() == 0 {
986            self.num_removed_accounts += 1;
987        } else {
988            self.num_updated_accounts += 1;
989        }
990        self.total_data_len = self
991            .total_data_len
992            .wrapping_add(account.data().len() as u64);
993        if account.executable() {
994            self.num_executable_accounts += 1;
995        }
996        self.num_lamports_stored = self.num_lamports_stored.wrapping_add(account.lamports());
997    }
998    pub fn accumulate(&mut self, other: &BankHashStats) {
999        self.num_updated_accounts += other.num_updated_accounts;
1000        self.num_removed_accounts += other.num_removed_accounts;
1001        self.total_data_len = self.total_data_len.wrapping_add(other.total_data_len);
1002        self.num_lamports_stored = self
1003            .num_lamports_stored
1004            .wrapping_add(other.num_lamports_stored);
1005        self.num_executable_accounts += other.num_executable_accounts;
1006    }
1007}
1008
1009#[derive(Debug, Default)]
1010pub struct AtomicBankHashStats {
1011    pub num_updated_accounts: AtomicU64,
1012    pub num_removed_accounts: AtomicU64,
1013    pub num_lamports_stored: AtomicU64,
1014    pub total_data_len: AtomicU64,
1015    pub num_executable_accounts: AtomicU64,
1016}
1017
1018impl AtomicBankHashStats {
1019    pub fn new(stat: &BankHashStats) -> Self {
1020        AtomicBankHashStats {
1021            num_updated_accounts: AtomicU64::new(stat.num_updated_accounts),
1022            num_removed_accounts: AtomicU64::new(stat.num_removed_accounts),
1023            num_lamports_stored: AtomicU64::new(stat.num_lamports_stored),
1024            total_data_len: AtomicU64::new(stat.total_data_len),
1025            num_executable_accounts: AtomicU64::new(stat.num_executable_accounts),
1026        }
1027    }
1028
1029    pub fn accumulate(&self, other: &BankHashStats) {
1030        self.num_updated_accounts
1031            .fetch_add(other.num_updated_accounts, Relaxed);
1032        self.num_removed_accounts
1033            .fetch_add(other.num_removed_accounts, Relaxed);
1034        self.total_data_len.fetch_add(other.total_data_len, Relaxed);
1035        self.num_lamports_stored
1036            .fetch_add(other.num_lamports_stored, Relaxed);
1037        self.num_executable_accounts
1038            .fetch_add(other.num_executable_accounts, Relaxed);
1039    }
1040
1041    pub fn load(&self) -> BankHashStats {
1042        BankHashStats {
1043            num_updated_accounts: self.num_updated_accounts.load(Relaxed),
1044            num_removed_accounts: self.num_removed_accounts.load(Relaxed),
1045            num_lamports_stored: self.num_lamports_stored.load(Relaxed),
1046            total_data_len: self.total_data_len.load(Relaxed),
1047            num_executable_accounts: self.num_executable_accounts.load(Relaxed),
1048        }
1049    }
1050}
1051
1052struct NewEpochBundle {
1053    stake_history: CowStakeHistory,
1054    vote_accounts: VoteAccounts,
1055    rewards_calculation: Arc<PartitionedRewardsCalculation>,
1056    calculate_activated_stake_time_us: u64,
1057    update_rewards_with_thread_pool_time_us: u64,
1058}
1059
1060impl Bank {
1061    fn default_with_accounts(accounts: Accounts) -> Self {
1062        let mut bank = Self {
1063            rc: BankRc::new(accounts),
1064            status_cache: Arc::<RwLock<BankStatusCache>>::default(),
1065            blockhash_queue: RwLock::<BlockhashQueue>::default(),
1066            ancestors: Ancestors::default(),
1067            hash: RwLock::<Hash>::default(),
1068            parent_hash: Hash::default(),
1069            parent_slot: Slot::default(),
1070            hard_forks: Arc::<RwLock<HardForks>>::default(),
1071            transaction_count: AtomicU64::default(),
1072            non_vote_transaction_count_since_restart: AtomicU64::default(),
1073            transaction_error_count: AtomicU64::default(),
1074            transaction_entries_count: AtomicU64::default(),
1075            transactions_per_entry_max: AtomicU64::default(),
1076            tick_height: AtomicU64::default(),
1077            signature_count: AtomicU64::default(),
1078            capitalization: AtomicU64::default(),
1079            max_tick_height: u64::default(),
1080            hashes_per_tick: Option::<u64>::default(),
1081            ticks_per_slot: u64::default(),
1082            ns_per_slot: u128::default(),
1083            genesis_creation_time: UnixTimestamp::default(),
1084            slots_per_year: f64::default(),
1085            slot: Slot::default(),
1086            bank_id: BankId::default(),
1087            epoch: Epoch::default(),
1088            block_height: u64::default(),
1089            collector_id: Pubkey::default(),
1090            collector_fees: AtomicU64::default(),
1091            fee_rate_governor: FeeRateGovernor::default(),
1092            rent_collector: RentCollector::default(),
1093            epoch_schedule: EpochSchedule::default(),
1094            inflation: Arc::<RwLock<Inflation>>::default(),
1095            stakes_cache: StakesCache::default(),
1096            epoch_stakes: HashMap::<Epoch, VersionedEpochStakes>::default(),
1097            is_delta: AtomicBool::default(),
1098            rewards: RwLock::<Vec<(Pubkey, RewardInfo)>>::default(),
1099            cluster_type: Option::<ClusterType>::default(),
1100            transaction_debug_keys: Option::<Arc<HashSet<Pubkey>>>::default(),
1101            transaction_log_collector_config: Arc::<RwLock<TransactionLogCollectorConfig>>::default(
1102            ),
1103            transaction_log_collector: Arc::<RwLock<TransactionLogCollector>>::default(),
1104            feature_set: Arc::<FeatureSet>::default(),
1105            reserved_account_keys: Arc::<ReservedAccountKeys>::default(),
1106            drop_callback: RwLock::new(OptionalDropCallback(None)),
1107            freeze_started: AtomicBool::default(),
1108            vote_only_bank: false,
1109            cost_tracker: RwLock::<CostTracker>::default(),
1110            accounts_data_size_initial: 0,
1111            accounts_data_size_delta_on_chain: AtomicI64::new(0),
1112            accounts_data_size_delta_off_chain: AtomicI64::new(0),
1113            epoch_reward_status: EpochRewardStatus::default(),
1114            transaction_processor: TransactionBatchProcessor::default(),
1115            check_program_modification_slot: false,
1116            collector_fee_details: RwLock::new(CollectorFeeDetails::default()),
1117            compute_budget: None,
1118            transaction_account_lock_limit: None,
1119            fee_structure: FeeStructure::default(),
1120            #[cfg(feature = "dev-context-only-utils")]
1121            hash_overrides: Arc::new(Mutex::new(HashOverrides::default())),
1122            accounts_lt_hash: Mutex::new(AccountsLtHash(LtHash::identity())),
1123            cache_for_accounts_lt_hash: DashMap::default(),
1124            stats_for_accounts_lt_hash: AccountsLtHashStats::default(),
1125            block_id: RwLock::new(None),
1126            bank_hash_stats: AtomicBankHashStats::default(),
1127            epoch_rewards_calculation_cache: Arc::new(Mutex::new(HashMap::default())),
1128        };
1129
1130        bank.transaction_processor =
1131            TransactionBatchProcessor::new_uninitialized(bank.slot, bank.epoch);
1132
1133        bank.accounts_data_size_initial = bank.calculate_accounts_data_size().unwrap();
1134
1135        bank
1136    }
1137
1138    #[allow(clippy::too_many_arguments)]
1139    pub fn new_from_genesis(
1140        genesis_config: &GenesisConfig,
1141        runtime_config: Arc<RuntimeConfig>,
1142        paths: Vec<PathBuf>,
1143        debug_keys: Option<Arc<HashSet<Pubkey>>>,
1144        accounts_db_config: AccountsDbConfig,
1145        accounts_update_notifier: Option<AccountsUpdateNotifier>,
1146        #[allow(unused)] collector_id_for_tests: Option<Pubkey>,
1147        exit: Arc<AtomicBool>,
1148        #[allow(unused)] genesis_hash: Option<Hash>,
1149        #[allow(unused)] feature_set: Option<FeatureSet>,
1150    ) -> Self {
1151        let accounts_db =
1152            AccountsDb::new_with_config(paths, accounts_db_config, accounts_update_notifier, exit);
1153        let accounts = Accounts::new(Arc::new(accounts_db));
1154        let mut bank = Self::default_with_accounts(accounts);
1155        bank.ancestors = Ancestors::from(vec![bank.slot()]);
1156        bank.compute_budget = runtime_config.compute_budget;
1157        if let Some(compute_budget) = &bank.compute_budget {
1158            bank.transaction_processor
1159                .set_execution_cost(compute_budget.to_cost());
1160        }
1161        bank.transaction_account_lock_limit = runtime_config.transaction_account_lock_limit;
1162        bank.transaction_debug_keys = debug_keys;
1163        bank.cluster_type = Some(genesis_config.cluster_type);
1164
1165        #[cfg(feature = "dev-context-only-utils")]
1166        {
1167            bank.feature_set = Arc::new(feature_set.unwrap_or_default());
1168        }
1169
1170        #[cfg(not(feature = "dev-context-only-utils"))]
1171        bank.process_genesis_config(genesis_config);
1172        #[cfg(feature = "dev-context-only-utils")]
1173        bank.process_genesis_config(genesis_config, collector_id_for_tests, genesis_hash);
1174
1175        bank.compute_and_apply_genesis_features();
1176
1177        // genesis needs stakes for all epochs up to the epoch implied by
1178        //  slot = 0 and genesis configuration
1179        {
1180            let stakes = bank.stakes_cache.stakes().clone();
1181            let stakes = SerdeStakesToStakeFormat::from(stakes);
1182            for epoch in 0..=bank.get_leader_schedule_epoch(bank.slot) {
1183                bank.epoch_stakes
1184                    .insert(epoch, VersionedEpochStakes::new(stakes.clone(), epoch));
1185            }
1186            bank.update_stake_history(None);
1187        }
1188        bank.update_clock(None);
1189        bank.update_rent();
1190        bank.update_epoch_schedule();
1191        bank.update_recent_blockhashes();
1192        bank.update_last_restart_slot();
1193        bank.transaction_processor
1194            .fill_missing_sysvar_cache_entries(&bank);
1195        bank
1196    }
1197
1198    /// Create a new bank that points to an immutable checkpoint of another bank.
1199    pub fn new_from_parent(parent: Arc<Bank>, collector_id: &Pubkey, slot: Slot) -> Self {
1200        Self::_new_from_parent(
1201            parent,
1202            collector_id,
1203            slot,
1204            null_tracer(),
1205            NewBankOptions::default(),
1206        )
1207    }
1208
1209    pub fn new_from_parent_with_options(
1210        parent: Arc<Bank>,
1211        collector_id: &Pubkey,
1212        slot: Slot,
1213        new_bank_options: NewBankOptions,
1214    ) -> Self {
1215        Self::_new_from_parent(parent, collector_id, slot, null_tracer(), new_bank_options)
1216    }
1217
1218    pub fn new_from_parent_with_tracer(
1219        parent: Arc<Bank>,
1220        collector_id: &Pubkey,
1221        slot: Slot,
1222        reward_calc_tracer: impl RewardCalcTracer,
1223    ) -> Self {
1224        Self::_new_from_parent(
1225            parent,
1226            collector_id,
1227            slot,
1228            Some(reward_calc_tracer),
1229            NewBankOptions::default(),
1230        )
1231    }
1232
1233    fn get_rent_collector_from(rent_collector: &RentCollector, epoch: Epoch) -> RentCollector {
1234        rent_collector.clone_with_epoch(epoch)
1235    }
1236
1237    fn _new_from_parent(
1238        parent: Arc<Bank>,
1239        collector_id: &Pubkey,
1240        slot: Slot,
1241        reward_calc_tracer: Option<impl RewardCalcTracer>,
1242        new_bank_options: NewBankOptions,
1243    ) -> Self {
1244        let mut time = Measure::start("bank::new_from_parent");
1245        let NewBankOptions { vote_only_bank } = new_bank_options;
1246
1247        parent.freeze();
1248        assert_ne!(slot, parent.slot());
1249
1250        let epoch_schedule = parent.epoch_schedule().clone();
1251        let epoch = epoch_schedule.get_epoch(slot);
1252
1253        let (rc, bank_rc_creation_time_us) = measure_us!({
1254            let accounts_db = Arc::clone(&parent.rc.accounts.accounts_db);
1255            BankRc {
1256                accounts: Arc::new(Accounts::new(accounts_db)),
1257                parent: RwLock::new(Some(Arc::clone(&parent))),
1258                bank_id_generator: Arc::clone(&parent.rc.bank_id_generator),
1259            }
1260        });
1261
1262        let (status_cache, status_cache_time_us) = measure_us!(Arc::clone(&parent.status_cache));
1263
1264        let (fee_rate_governor, fee_components_time_us) = measure_us!(
1265            FeeRateGovernor::new_derived(&parent.fee_rate_governor, parent.signature_count())
1266        );
1267
1268        let bank_id = rc.bank_id_generator.fetch_add(1, Relaxed) + 1;
1269        let (blockhash_queue, blockhash_queue_time_us) =
1270            measure_us!(RwLock::new(parent.blockhash_queue.read().unwrap().clone()));
1271
1272        let (stakes_cache, stakes_cache_time_us) =
1273            measure_us!(StakesCache::new(parent.stakes_cache.stakes().clone()));
1274
1275        let (epoch_stakes, epoch_stakes_time_us) = measure_us!(parent.epoch_stakes.clone());
1276
1277        let (transaction_processor, builtin_program_ids_time_us) = measure_us!(
1278            TransactionBatchProcessor::new_from(&parent.transaction_processor, slot, epoch)
1279        );
1280
1281        let (transaction_debug_keys, transaction_debug_keys_time_us) =
1282            measure_us!(parent.transaction_debug_keys.clone());
1283
1284        let (transaction_log_collector_config, transaction_log_collector_config_time_us) =
1285            measure_us!(parent.transaction_log_collector_config.clone());
1286
1287        let (feature_set, feature_set_time_us) = measure_us!(parent.feature_set.clone());
1288
1289        let accounts_data_size_initial = parent.load_accounts_data_size();
1290        let mut new = Self {
1291            rc,
1292            status_cache,
1293            slot,
1294            bank_id,
1295            epoch,
1296            blockhash_queue,
1297
1298            // TODO: clean this up, so much special-case copying...
1299            hashes_per_tick: parent.hashes_per_tick,
1300            ticks_per_slot: parent.ticks_per_slot,
1301            ns_per_slot: parent.ns_per_slot,
1302            genesis_creation_time: parent.genesis_creation_time,
1303            slots_per_year: parent.slots_per_year,
1304            epoch_schedule,
1305            rent_collector: Self::get_rent_collector_from(&parent.rent_collector, epoch),
1306            max_tick_height: slot
1307                .checked_add(1)
1308                .expect("max tick height addition overflowed")
1309                .checked_mul(parent.ticks_per_slot)
1310                .expect("max tick height multiplication overflowed"),
1311            block_height: parent
1312                .block_height
1313                .checked_add(1)
1314                .expect("block height addition overflowed"),
1315            fee_rate_governor,
1316            capitalization: AtomicU64::new(parent.capitalization()),
1317            vote_only_bank,
1318            inflation: parent.inflation.clone(),
1319            transaction_count: AtomicU64::new(parent.transaction_count()),
1320            non_vote_transaction_count_since_restart: AtomicU64::new(
1321                parent.non_vote_transaction_count_since_restart(),
1322            ),
1323            transaction_error_count: AtomicU64::new(0),
1324            transaction_entries_count: AtomicU64::new(0),
1325            transactions_per_entry_max: AtomicU64::new(0),
1326            // we will .clone_with_epoch() this soon after stake data update; so just .clone() for now
1327            stakes_cache,
1328            epoch_stakes,
1329            parent_hash: parent.hash(),
1330            parent_slot: parent.slot(),
1331            collector_id: *collector_id,
1332            collector_fees: AtomicU64::new(0),
1333            ancestors: Ancestors::default(),
1334            hash: RwLock::new(Hash::default()),
1335            is_delta: AtomicBool::new(false),
1336            tick_height: AtomicU64::new(parent.tick_height.load(Relaxed)),
1337            signature_count: AtomicU64::new(0),
1338            hard_forks: parent.hard_forks.clone(),
1339            rewards: RwLock::new(vec![]),
1340            cluster_type: parent.cluster_type,
1341            transaction_debug_keys,
1342            transaction_log_collector_config,
1343            transaction_log_collector: Arc::new(RwLock::new(TransactionLogCollector::default())),
1344            feature_set: Arc::clone(&feature_set),
1345            reserved_account_keys: parent.reserved_account_keys.clone(),
1346            drop_callback: RwLock::new(OptionalDropCallback(
1347                parent
1348                    .drop_callback
1349                    .read()
1350                    .unwrap()
1351                    .0
1352                    .as_ref()
1353                    .map(|drop_callback| drop_callback.clone_box()),
1354            )),
1355            freeze_started: AtomicBool::new(false),
1356            cost_tracker: RwLock::new(parent.read_cost_tracker().unwrap().new_from_parent_limits()),
1357            accounts_data_size_initial,
1358            accounts_data_size_delta_on_chain: AtomicI64::new(0),
1359            accounts_data_size_delta_off_chain: AtomicI64::new(0),
1360            epoch_reward_status: parent.epoch_reward_status.clone(),
1361            transaction_processor,
1362            check_program_modification_slot: false,
1363            collector_fee_details: RwLock::new(CollectorFeeDetails::default()),
1364            compute_budget: parent.compute_budget,
1365            transaction_account_lock_limit: parent.transaction_account_lock_limit,
1366            fee_structure: parent.fee_structure.clone(),
1367            #[cfg(feature = "dev-context-only-utils")]
1368            hash_overrides: parent.hash_overrides.clone(),
1369            accounts_lt_hash: Mutex::new(parent.accounts_lt_hash.lock().unwrap().clone()),
1370            cache_for_accounts_lt_hash: DashMap::default(),
1371            stats_for_accounts_lt_hash: AccountsLtHashStats::default(),
1372            block_id: RwLock::new(None),
1373            bank_hash_stats: AtomicBankHashStats::default(),
1374            epoch_rewards_calculation_cache: parent.epoch_rewards_calculation_cache.clone(),
1375        };
1376
1377        let (_, ancestors_time_us) = measure_us!({
1378            let mut ancestors = Vec::with_capacity(1 + new.parents().len());
1379            ancestors.push(new.slot());
1380            new.parents().iter().for_each(|p| {
1381                ancestors.push(p.slot());
1382            });
1383            new.ancestors = Ancestors::from(ancestors);
1384        });
1385
1386        // Following code may touch AccountsDb, requiring proper ancestors
1387        let (_, update_epoch_time_us) = measure_us!({
1388            if parent.epoch() < new.epoch() {
1389                new.process_new_epoch(
1390                    parent.epoch(),
1391                    parent.slot(),
1392                    parent.block_height(),
1393                    reward_calc_tracer,
1394                );
1395            } else {
1396                // Save a snapshot of stakes for use in consensus and stake weighted networking
1397                let leader_schedule_epoch = new.epoch_schedule().get_leader_schedule_epoch(slot);
1398                new.update_epoch_stakes(leader_schedule_epoch);
1399            }
1400            new.distribute_partitioned_epoch_rewards();
1401        });
1402
1403        let (_, cache_preparation_time_us) =
1404            measure_us!(new.prepare_program_cache_for_upcoming_feature_set());
1405
1406        // Update sysvars before processing transactions
1407        let (_, update_sysvars_time_us) = measure_us!({
1408            new.update_slot_hashes();
1409            new.update_stake_history(Some(parent.epoch()));
1410            new.update_clock(Some(parent.epoch()));
1411            new.update_last_restart_slot()
1412        });
1413
1414        let (_, fill_sysvar_cache_time_us) = measure_us!(new
1415            .transaction_processor
1416            .fill_missing_sysvar_cache_entries(&new));
1417
1418        let (num_accounts_modified_this_slot, populate_cache_for_accounts_lt_hash_us) =
1419            measure_us!({
1420                // The cache for accounts lt hash needs to be made aware of accounts modified
1421                // before transaction processing begins.  Otherwise we may calculate the wrong
1422                // accounts lt hash due to having the wrong initial state of the account.  The
1423                // lt hash cache's initial state must always be from an ancestor, and cannot be
1424                // an intermediate state within this Bank's slot.  If the lt hash cache has the
1425                // wrong initial account state, we'll mix out the wrong lt hash value, and thus
1426                // have the wrong overall accounts lt hash, and diverge.
1427                let accounts_modified_this_slot =
1428                    new.rc.accounts.accounts_db.get_pubkeys_for_slot(slot);
1429                let num_accounts_modified_this_slot = accounts_modified_this_slot.len();
1430                for pubkey in accounts_modified_this_slot {
1431                    new.cache_for_accounts_lt_hash
1432                        .entry(pubkey)
1433                        .or_insert(AccountsLtHashCacheValue::BankNew);
1434                }
1435                num_accounts_modified_this_slot
1436            });
1437
1438        time.stop();
1439        report_new_bank_metrics(
1440            slot,
1441            parent.slot(),
1442            new.block_height,
1443            num_accounts_modified_this_slot,
1444            NewBankTimings {
1445                bank_rc_creation_time_us,
1446                total_elapsed_time_us: time.as_us(),
1447                status_cache_time_us,
1448                fee_components_time_us,
1449                blockhash_queue_time_us,
1450                stakes_cache_time_us,
1451                epoch_stakes_time_us,
1452                builtin_program_ids_time_us,
1453                executor_cache_time_us: 0,
1454                transaction_debug_keys_time_us,
1455                transaction_log_collector_config_time_us,
1456                feature_set_time_us,
1457                ancestors_time_us,
1458                update_epoch_time_us,
1459                cache_preparation_time_us,
1460                update_sysvars_time_us,
1461                fill_sysvar_cache_time_us,
1462                populate_cache_for_accounts_lt_hash_us,
1463            },
1464        );
1465
1466        report_loaded_programs_stats(
1467            &parent
1468                .transaction_processor
1469                .global_program_cache
1470                .read()
1471                .unwrap()
1472                .stats,
1473            parent.slot(),
1474        );
1475
1476        new.transaction_processor
1477            .global_program_cache
1478            .write()
1479            .unwrap()
1480            .stats
1481            .reset();
1482
1483        new
1484    }
1485
1486    pub fn set_fork_graph_in_program_cache(&self, fork_graph: Weak<RwLock<BankForks>>) {
1487        self.transaction_processor
1488            .global_program_cache
1489            .write()
1490            .unwrap()
1491            .set_fork_graph(fork_graph);
1492    }
1493
1494    fn prepare_program_cache_for_upcoming_feature_set(&self) {
1495        let (_epoch, slot_index) = self.epoch_schedule.get_epoch_and_slot_index(self.slot);
1496        let slots_in_epoch = self.epoch_schedule.get_slots_in_epoch(self.epoch);
1497        let (upcoming_feature_set, _newly_activated) = self.compute_active_feature_set(true);
1498
1499        // Recompile loaded programs one at a time before the next epoch hits
1500        let slots_in_recompilation_phase =
1501            (solana_program_runtime::loaded_programs::MAX_LOADED_ENTRY_COUNT as u64)
1502                .min(slots_in_epoch)
1503                .checked_div(2)
1504                .unwrap();
1505
1506        let program_cache = self
1507            .transaction_processor
1508            .global_program_cache
1509            .read()
1510            .unwrap();
1511        let mut epoch_boundary_preparation = self
1512            .transaction_processor
1513            .epoch_boundary_preparation
1514            .write()
1515            .unwrap();
1516
1517        if let Some(upcoming_environments) =
1518            epoch_boundary_preparation.upcoming_environments.as_ref()
1519        {
1520            let upcoming_environments = upcoming_environments.clone();
1521            if let Some((key, program_to_recompile)) =
1522                epoch_boundary_preparation.programs_to_recompile.pop()
1523            {
1524                drop(epoch_boundary_preparation);
1525                drop(program_cache);
1526                if let Some(recompiled) = load_program_with_pubkey(
1527                    self,
1528                    &upcoming_environments,
1529                    &key,
1530                    self.slot,
1531                    &mut ExecuteTimings::default(),
1532                    false,
1533                ) {
1534                    recompiled.tx_usage_counter.fetch_add(
1535                        program_to_recompile
1536                            .tx_usage_counter
1537                            .load(Ordering::Relaxed),
1538                        Ordering::Relaxed,
1539                    );
1540                    let mut program_cache = self
1541                        .transaction_processor
1542                        .global_program_cache
1543                        .write()
1544                        .unwrap();
1545                    program_cache.assign_program(&upcoming_environments, key, recompiled);
1546                }
1547            }
1548        } else if slot_index.saturating_add(slots_in_recompilation_phase) >= slots_in_epoch {
1549            // Anticipate the upcoming program runtime environment for the next epoch,
1550            // so we can try to recompile loaded programs before the feature transition hits.
1551            let new_environments = self.create_program_runtime_environments(&upcoming_feature_set);
1552            let mut upcoming_environments = self.transaction_processor.environments.clone();
1553            let changed_program_runtime_v1 =
1554                *upcoming_environments.program_runtime_v1 != *new_environments.program_runtime_v1;
1555            let changed_program_runtime_v2 =
1556                *upcoming_environments.program_runtime_v2 != *new_environments.program_runtime_v2;
1557            if changed_program_runtime_v1 {
1558                upcoming_environments.program_runtime_v1 = new_environments.program_runtime_v1;
1559            }
1560            if changed_program_runtime_v2 {
1561                upcoming_environments.program_runtime_v2 = new_environments.program_runtime_v2;
1562            }
1563            epoch_boundary_preparation.upcoming_epoch = self.epoch.saturating_add(1);
1564            epoch_boundary_preparation.upcoming_environments = Some(upcoming_environments);
1565            epoch_boundary_preparation.programs_to_recompile = program_cache
1566                .get_flattened_entries(changed_program_runtime_v1, changed_program_runtime_v2);
1567            epoch_boundary_preparation
1568                .programs_to_recompile
1569                .sort_by_cached_key(|(_id, program)| program.decayed_usage_counter(self.slot));
1570        }
1571    }
1572
1573    pub fn prune_program_cache(&self, new_root_slot: Slot, new_root_epoch: Epoch) {
1574        let upcoming_environments = self
1575            .transaction_processor
1576            .epoch_boundary_preparation
1577            .write()
1578            .unwrap()
1579            .reroot(new_root_epoch);
1580        self.transaction_processor
1581            .global_program_cache
1582            .write()
1583            .unwrap()
1584            .prune(new_root_slot, upcoming_environments);
1585    }
1586
1587    pub fn prune_program_cache_by_deployment_slot(&self, deployment_slot: Slot) {
1588        self.transaction_processor
1589            .global_program_cache
1590            .write()
1591            .unwrap()
1592            .prune_by_deployment_slot(deployment_slot);
1593    }
1594
1595    /// Epoch in which the new cooldown warmup rate for stake was activated
1596    pub fn new_warmup_cooldown_rate_epoch(&self) -> Option<Epoch> {
1597        self.feature_set
1598            .new_warmup_cooldown_rate_epoch(&self.epoch_schedule)
1599    }
1600
1601    /// Returns updated stake history and vote accounts that includes new
1602    /// activated stake from the last epoch.
1603    fn compute_new_epoch_caches_and_rewards(
1604        &self,
1605        thread_pool: &ThreadPool,
1606        parent_epoch: Epoch,
1607        reward_calc_tracer: Option<impl RewardCalcTracer>,
1608        rewards_metrics: &mut RewardsMetrics,
1609    ) -> NewEpochBundle {
1610        // Add new entry to stakes.stake_history, set appropriate epoch and
1611        // update vote accounts with warmed up stakes before saving a
1612        // snapshot of stakes in epoch stakes
1613        let stakes = self.stakes_cache.stakes();
1614        let stake_delegations = stakes.stake_delegations_vec();
1615        let ((stake_history, vote_accounts), calculate_activated_stake_time_us) =
1616            measure_us!(stakes.calculate_activated_stake(
1617                self.epoch(),
1618                thread_pool,
1619                self.new_warmup_cooldown_rate_epoch(),
1620                &stake_delegations
1621            ));
1622        // Apply stake rewards and commission using new snapshots.
1623        let (rewards_calculation, update_rewards_with_thread_pool_time_us) = measure_us!(self
1624            .calculate_rewards(
1625                &stake_history,
1626                stake_delegations,
1627                &vote_accounts,
1628                parent_epoch,
1629                reward_calc_tracer,
1630                thread_pool,
1631                rewards_metrics,
1632            ));
1633        NewEpochBundle {
1634            stake_history,
1635            vote_accounts,
1636            rewards_calculation,
1637            calculate_activated_stake_time_us,
1638            update_rewards_with_thread_pool_time_us,
1639        }
1640    }
1641
1642    /// process for the start of a new epoch
1643    fn process_new_epoch(
1644        &mut self,
1645        parent_epoch: Epoch,
1646        parent_slot: Slot,
1647        parent_height: u64,
1648        reward_calc_tracer: Option<impl RewardCalcTracer>,
1649    ) {
1650        let epoch = self.epoch();
1651        let slot = self.slot();
1652        let (thread_pool, thread_pool_time_us) = measure_us!(ThreadPoolBuilder::new()
1653            .thread_name(|i| format!("solBnkNewEpch{i:02}"))
1654            .build()
1655            .expect("new rayon threadpool"));
1656
1657        let (_, apply_feature_activations_time_us) = measure_us!(
1658            thread_pool.install(|| { self.compute_and_apply_new_feature_activations() })
1659        );
1660
1661        let mut rewards_metrics = RewardsMetrics::default();
1662        let NewEpochBundle {
1663            stake_history,
1664            vote_accounts,
1665            rewards_calculation,
1666            calculate_activated_stake_time_us,
1667            update_rewards_with_thread_pool_time_us,
1668        } = self.compute_new_epoch_caches_and_rewards(
1669            &thread_pool,
1670            parent_epoch,
1671            reward_calc_tracer,
1672            &mut rewards_metrics,
1673        );
1674
1675        self.stakes_cache
1676            .activate_epoch(epoch, stake_history, vote_accounts);
1677
1678        // Save a snapshot of stakes for use in consensus and stake weighted networking
1679        let leader_schedule_epoch = self.epoch_schedule.get_leader_schedule_epoch(slot);
1680        let (_, update_epoch_stakes_time_us) =
1681            measure_us!(self.update_epoch_stakes(leader_schedule_epoch));
1682
1683        // Distribute rewards commission to vote accounts and cache stake rewards
1684        // for partitioned distribution in the upcoming slots.
1685        self.begin_partitioned_rewards(
1686            parent_epoch,
1687            parent_slot,
1688            parent_height,
1689            &rewards_calculation,
1690            &rewards_metrics,
1691        );
1692
1693        report_new_epoch_metrics(
1694            epoch,
1695            slot,
1696            parent_slot,
1697            NewEpochTimings {
1698                thread_pool_time_us,
1699                apply_feature_activations_time_us,
1700                calculate_activated_stake_time_us,
1701                update_epoch_stakes_time_us,
1702                update_rewards_with_thread_pool_time_us,
1703            },
1704            rewards_metrics,
1705        );
1706
1707        let new_environments = self.create_program_runtime_environments(&self.feature_set);
1708        self.transaction_processor
1709            .set_environments(new_environments);
1710    }
1711
1712    pub fn byte_limit_for_scans(&self) -> Option<usize> {
1713        self.rc
1714            .accounts
1715            .accounts_db
1716            .accounts_index
1717            .scan_results_limit_bytes
1718    }
1719
1720    pub fn proper_ancestors_set(&self) -> HashSet<Slot> {
1721        HashSet::from_iter(self.proper_ancestors())
1722    }
1723
1724    /// Returns all ancestors excluding self.slot.
1725    pub(crate) fn proper_ancestors(&self) -> impl Iterator<Item = Slot> + '_ {
1726        self.ancestors
1727            .keys()
1728            .into_iter()
1729            .filter(move |slot| *slot != self.slot)
1730    }
1731
1732    pub fn set_callback(&self, callback: Option<Box<dyn DropCallback + Send + Sync>>) {
1733        *self.drop_callback.write().unwrap() = OptionalDropCallback(callback);
1734    }
1735
1736    pub fn vote_only_bank(&self) -> bool {
1737        self.vote_only_bank
1738    }
1739
1740    /// Like `new_from_parent` but additionally:
1741    /// * Doesn't assume that the parent is anywhere near `slot`, parent could be millions of slots
1742    ///   in the past
1743    /// * Adjusts the new bank's tick height to avoid having to run PoH for millions of slots
1744    /// * Freezes the new bank, assuming that the user will `Bank::new_from_parent` from this bank
1745    pub fn warp_from_parent(parent: Arc<Bank>, collector_id: &Pubkey, slot: Slot) -> Self {
1746        parent.freeze();
1747        let parent_timestamp = parent.clock().unix_timestamp;
1748        let mut new = Bank::new_from_parent(parent, collector_id, slot);
1749        new.update_epoch_stakes(new.epoch_schedule().get_epoch(slot));
1750        new.tick_height.store(new.max_tick_height(), Relaxed);
1751
1752        let mut clock = new.clock();
1753        clock.epoch_start_timestamp = parent_timestamp;
1754        clock.unix_timestamp = parent_timestamp;
1755        new.update_sysvar_account(&sysvar::clock::id(), |account| {
1756            create_account(
1757                &clock,
1758                new.inherit_specially_retained_account_fields(account),
1759            )
1760        });
1761        new.transaction_processor
1762            .fill_missing_sysvar_cache_entries(&new);
1763        new.freeze();
1764        new
1765    }
1766
1767    /// Create a bank from explicit arguments and deserialized fields from snapshot
1768    pub(crate) fn new_from_snapshot(
1769        bank_rc: BankRc,
1770        genesis_config: &GenesisConfig,
1771        runtime_config: Arc<RuntimeConfig>,
1772        fields: BankFieldsToDeserialize,
1773        debug_keys: Option<Arc<HashSet<Pubkey>>>,
1774        accounts_data_size_initial: u64,
1775    ) -> Self {
1776        let now = Instant::now();
1777        let ancestors = Ancestors::from(&fields.ancestors);
1778        // For backward compatibility, we can only serialize and deserialize
1779        // Stakes<Delegation> in BankFieldsTo{Serialize,Deserialize}. But Bank
1780        // caches Stakes<StakeAccount>. Below Stakes<StakeAccount> is obtained
1781        // from Stakes<Delegation> by reading the full account state from
1782        // accounts-db. Note that it is crucial that these accounts are loaded
1783        // at the right slot and match precisely with serialized Delegations.
1784        //
1785        // Note that we are disabling the read cache while we populate the stakes cache.
1786        // The stakes accounts will not be expected to be loaded again.
1787        // If we populate the read cache with these loads, then we'll just soon have to evict these.
1788        let (stakes, stakes_time) = measure_time!(Stakes::new(&fields.stakes, |pubkey| {
1789            let (account, _slot) = bank_rc
1790                .accounts
1791                .load_with_fixed_root_do_not_populate_read_cache(&ancestors, pubkey)?;
1792            Some(account)
1793        })
1794        .expect(
1795            "Stakes cache is inconsistent with accounts-db. This can indicate a corrupted \
1796             snapshot or bugs in cached accounts or accounts-db.",
1797        ));
1798        info!("Loading Stakes took: {stakes_time}");
1799        let stakes_accounts_load_duration = now.elapsed();
1800        let mut bank = Self {
1801            rc: bank_rc,
1802            status_cache: Arc::<RwLock<BankStatusCache>>::default(),
1803            blockhash_queue: RwLock::new(fields.blockhash_queue),
1804            ancestors,
1805            hash: RwLock::new(fields.hash),
1806            parent_hash: fields.parent_hash,
1807            parent_slot: fields.parent_slot,
1808            hard_forks: Arc::new(RwLock::new(fields.hard_forks)),
1809            transaction_count: AtomicU64::new(fields.transaction_count),
1810            non_vote_transaction_count_since_restart: AtomicU64::default(),
1811            transaction_error_count: AtomicU64::default(),
1812            transaction_entries_count: AtomicU64::default(),
1813            transactions_per_entry_max: AtomicU64::default(),
1814            tick_height: AtomicU64::new(fields.tick_height),
1815            signature_count: AtomicU64::new(fields.signature_count),
1816            capitalization: AtomicU64::new(fields.capitalization),
1817            max_tick_height: fields.max_tick_height,
1818            hashes_per_tick: fields.hashes_per_tick,
1819            ticks_per_slot: fields.ticks_per_slot,
1820            ns_per_slot: fields.ns_per_slot,
1821            genesis_creation_time: fields.genesis_creation_time,
1822            slots_per_year: fields.slots_per_year,
1823            slot: fields.slot,
1824            bank_id: 0,
1825            epoch: fields.epoch,
1826            block_height: fields.block_height,
1827            collector_id: fields.collector_id,
1828            collector_fees: AtomicU64::new(fields.collector_fees),
1829            fee_rate_governor: fields.fee_rate_governor,
1830            // clone()-ing is needed to consider a gated behavior in rent_collector
1831            rent_collector: Self::get_rent_collector_from(&fields.rent_collector, fields.epoch),
1832            epoch_schedule: fields.epoch_schedule,
1833            inflation: Arc::new(RwLock::new(fields.inflation)),
1834            stakes_cache: StakesCache::new(stakes),
1835            epoch_stakes: fields.versioned_epoch_stakes,
1836            is_delta: AtomicBool::new(fields.is_delta),
1837            rewards: RwLock::new(vec![]),
1838            cluster_type: Some(genesis_config.cluster_type),
1839            transaction_debug_keys: debug_keys,
1840            transaction_log_collector_config: Arc::<RwLock<TransactionLogCollectorConfig>>::default(
1841            ),
1842            transaction_log_collector: Arc::<RwLock<TransactionLogCollector>>::default(),
1843            feature_set: Arc::<FeatureSet>::default(),
1844            reserved_account_keys: Arc::<ReservedAccountKeys>::default(),
1845            drop_callback: RwLock::new(OptionalDropCallback(None)),
1846            freeze_started: AtomicBool::new(fields.hash != Hash::default()),
1847            vote_only_bank: false,
1848            cost_tracker: RwLock::new(CostTracker::default()),
1849            accounts_data_size_initial,
1850            accounts_data_size_delta_on_chain: AtomicI64::new(0),
1851            accounts_data_size_delta_off_chain: AtomicI64::new(0),
1852            epoch_reward_status: EpochRewardStatus::default(),
1853            transaction_processor: TransactionBatchProcessor::default(),
1854            check_program_modification_slot: false,
1855            // collector_fee_details is not serialized to snapshot
1856            collector_fee_details: RwLock::new(CollectorFeeDetails::default()),
1857            compute_budget: runtime_config.compute_budget,
1858            transaction_account_lock_limit: runtime_config.transaction_account_lock_limit,
1859            fee_structure: FeeStructure::default(),
1860            #[cfg(feature = "dev-context-only-utils")]
1861            hash_overrides: Arc::new(Mutex::new(HashOverrides::default())),
1862            accounts_lt_hash: Mutex::new(fields.accounts_lt_hash),
1863            cache_for_accounts_lt_hash: DashMap::default(),
1864            stats_for_accounts_lt_hash: AccountsLtHashStats::default(),
1865            block_id: RwLock::new(None),
1866            bank_hash_stats: AtomicBankHashStats::new(&fields.bank_hash_stats),
1867            epoch_rewards_calculation_cache: Arc::new(Mutex::new(HashMap::default())),
1868        };
1869
1870        // Sanity assertions between bank snapshot and genesis config
1871        // Consider removing from serializable bank state
1872        // (BankFieldsToSerialize/BankFieldsToDeserialize) and initializing
1873        // from the passed in genesis_config instead (as new()/new_from_genesis() already do)
1874        assert_eq!(
1875            bank.genesis_creation_time, genesis_config.creation_time,
1876            "Bank snapshot genesis creation time does not match genesis.bin creation time. The \
1877             snapshot and genesis.bin might pertain to different clusters"
1878        );
1879        assert_eq!(bank.ticks_per_slot, genesis_config.ticks_per_slot);
1880        assert_eq!(
1881            bank.ns_per_slot,
1882            genesis_config.poh_config.target_tick_duration.as_nanos()
1883                * genesis_config.ticks_per_slot as u128
1884        );
1885        assert_eq!(bank.max_tick_height, (bank.slot + 1) * bank.ticks_per_slot);
1886        assert_eq!(
1887            bank.slots_per_year,
1888            years_as_slots(
1889                1.0,
1890                &genesis_config.poh_config.target_tick_duration,
1891                bank.ticks_per_slot,
1892            )
1893        );
1894        assert_eq!(bank.epoch_schedule, genesis_config.epoch_schedule);
1895        assert_eq!(bank.epoch, bank.epoch_schedule.get_epoch(bank.slot));
1896
1897        bank.initialize_after_snapshot_restore(|| {
1898            ThreadPoolBuilder::new()
1899                .thread_name(|i| format!("solBnkClcRwds{i:02}"))
1900                .build()
1901                .expect("new rayon threadpool")
1902        });
1903
1904        datapoint_info!(
1905            "bank-new-from-fields",
1906            (
1907                "accounts_data_len-from-snapshot",
1908                fields.accounts_data_len as i64,
1909                i64
1910            ),
1911            (
1912                "accounts_data_len-from-generate_index",
1913                accounts_data_size_initial as i64,
1914                i64
1915            ),
1916            (
1917                "stakes_accounts_load_duration_us",
1918                stakes_accounts_load_duration.as_micros(),
1919                i64
1920            ),
1921        );
1922        bank
1923    }
1924
1925    /// Return subset of bank fields representing serializable state
1926    pub(crate) fn get_fields_to_serialize(&self) -> BankFieldsToSerialize {
1927        BankFieldsToSerialize {
1928            blockhash_queue: self.blockhash_queue.read().unwrap().clone(),
1929            ancestors: AncestorsForSerialization::from(&self.ancestors),
1930            hash: *self.hash.read().unwrap(),
1931            parent_hash: self.parent_hash,
1932            parent_slot: self.parent_slot,
1933            hard_forks: self.hard_forks.read().unwrap().clone(),
1934            transaction_count: self.transaction_count.load(Relaxed),
1935            tick_height: self.tick_height.load(Relaxed),
1936            signature_count: self.signature_count.load(Relaxed),
1937            capitalization: self.capitalization.load(Relaxed),
1938            max_tick_height: self.max_tick_height,
1939            hashes_per_tick: self.hashes_per_tick,
1940            ticks_per_slot: self.ticks_per_slot,
1941            ns_per_slot: self.ns_per_slot,
1942            genesis_creation_time: self.genesis_creation_time,
1943            slots_per_year: self.slots_per_year,
1944            slot: self.slot,
1945            epoch: self.epoch,
1946            block_height: self.block_height,
1947            collector_id: self.collector_id,
1948            collector_fees: self.collector_fees.load(Relaxed),
1949            fee_rate_governor: self.fee_rate_governor.clone(),
1950            rent_collector: self.rent_collector.clone(),
1951            epoch_schedule: self.epoch_schedule.clone(),
1952            inflation: *self.inflation.read().unwrap(),
1953            stakes: self.stakes_cache.stakes().clone(),
1954            is_delta: self.is_delta.load(Relaxed),
1955            accounts_data_len: self.load_accounts_data_size(),
1956            versioned_epoch_stakes: self.epoch_stakes.clone(),
1957            accounts_lt_hash: self.accounts_lt_hash.lock().unwrap().clone(),
1958        }
1959    }
1960
1961    pub fn collector_id(&self) -> &Pubkey {
1962        &self.collector_id
1963    }
1964
1965    pub fn genesis_creation_time(&self) -> UnixTimestamp {
1966        self.genesis_creation_time
1967    }
1968
1969    pub fn slot(&self) -> Slot {
1970        self.slot
1971    }
1972
1973    pub fn bank_id(&self) -> BankId {
1974        self.bank_id
1975    }
1976
1977    pub fn epoch(&self) -> Epoch {
1978        self.epoch
1979    }
1980
1981    pub fn first_normal_epoch(&self) -> Epoch {
1982        self.epoch_schedule().first_normal_epoch
1983    }
1984
1985    pub fn freeze_lock(&self) -> RwLockReadGuard<'_, Hash> {
1986        self.hash.read().unwrap()
1987    }
1988
1989    pub fn hash(&self) -> Hash {
1990        *self.hash.read().unwrap()
1991    }
1992
1993    pub fn is_frozen(&self) -> bool {
1994        *self.hash.read().unwrap() != Hash::default()
1995    }
1996
1997    pub fn freeze_started(&self) -> bool {
1998        self.freeze_started.load(Relaxed)
1999    }
2000
2001    pub fn status_cache_ancestors(&self) -> Vec<u64> {
2002        let mut roots = self.status_cache.read().unwrap().roots().clone();
2003        let min = roots.iter().min().cloned().unwrap_or(0);
2004        for ancestor in self.ancestors.keys() {
2005            if ancestor >= min {
2006                roots.insert(ancestor);
2007            }
2008        }
2009
2010        let mut ancestors: Vec<_> = roots.into_iter().collect();
2011        #[allow(clippy::stable_sort_primitive)]
2012        ancestors.sort();
2013        ancestors
2014    }
2015
2016    /// computed unix_timestamp at this slot height
2017    pub fn unix_timestamp_from_genesis(&self) -> i64 {
2018        self.genesis_creation_time.saturating_add(
2019            (self.slot as u128)
2020                .saturating_mul(self.ns_per_slot)
2021                .saturating_div(1_000_000_000) as i64,
2022        )
2023    }
2024
2025    fn update_sysvar_account<F>(&self, pubkey: &Pubkey, updater: F)
2026    where
2027        F: Fn(&Option<AccountSharedData>) -> AccountSharedData,
2028    {
2029        let old_account = self.get_account_with_fixed_root(pubkey);
2030        let mut new_account = updater(&old_account);
2031
2032        // When new sysvar comes into existence (with RENT_UNADJUSTED_INITIAL_BALANCE lamports),
2033        // this code ensures that the sysvar's balance is adjusted to be rent-exempt.
2034        //
2035        // More generally, this code always re-calculates for possible sysvar data size change,
2036        // although there is no such sysvars currently.
2037        self.adjust_sysvar_balance_for_rent(&mut new_account);
2038        self.store_account_and_update_capitalization(pubkey, &new_account);
2039    }
2040
2041    fn inherit_specially_retained_account_fields(
2042        &self,
2043        old_account: &Option<AccountSharedData>,
2044    ) -> InheritableAccountFields {
2045        const RENT_UNADJUSTED_INITIAL_BALANCE: u64 = 1;
2046
2047        (
2048            old_account
2049                .as_ref()
2050                .map(|a| a.lamports())
2051                .unwrap_or(RENT_UNADJUSTED_INITIAL_BALANCE),
2052            old_account
2053                .as_ref()
2054                .map(|a| a.rent_epoch())
2055                .unwrap_or(INITIAL_RENT_EPOCH),
2056        )
2057    }
2058
2059    pub fn clock(&self) -> sysvar::clock::Clock {
2060        from_account(&self.get_account(&sysvar::clock::id()).unwrap_or_default())
2061            .unwrap_or_default()
2062    }
2063
2064    fn update_clock(&self, parent_epoch: Option<Epoch>) {
2065        let mut unix_timestamp = self.clock().unix_timestamp;
2066        // set epoch_start_timestamp to None to warp timestamp
2067        let epoch_start_timestamp = {
2068            let epoch = if let Some(epoch) = parent_epoch {
2069                epoch
2070            } else {
2071                self.epoch()
2072            };
2073            let first_slot_in_epoch = self.epoch_schedule().get_first_slot_in_epoch(epoch);
2074            Some((first_slot_in_epoch, self.clock().epoch_start_timestamp))
2075        };
2076        let max_allowable_drift = MaxAllowableDrift {
2077            fast: MAX_ALLOWABLE_DRIFT_PERCENTAGE_FAST,
2078            slow: MAX_ALLOWABLE_DRIFT_PERCENTAGE_SLOW_V2,
2079        };
2080
2081        let ancestor_timestamp = self.clock().unix_timestamp;
2082        if let Some(timestamp_estimate) =
2083            self.get_timestamp_estimate(max_allowable_drift, epoch_start_timestamp)
2084        {
2085            unix_timestamp = timestamp_estimate;
2086            if timestamp_estimate < ancestor_timestamp {
2087                unix_timestamp = ancestor_timestamp;
2088            }
2089        }
2090        datapoint_info!(
2091            "bank-timestamp-correction",
2092            ("slot", self.slot(), i64),
2093            ("from_genesis", self.unix_timestamp_from_genesis(), i64),
2094            ("corrected", unix_timestamp, i64),
2095            ("ancestor_timestamp", ancestor_timestamp, i64),
2096        );
2097        let mut epoch_start_timestamp =
2098            // On epoch boundaries, update epoch_start_timestamp
2099            if parent_epoch.is_some() && parent_epoch.unwrap() != self.epoch() {
2100                unix_timestamp
2101            } else {
2102                self.clock().epoch_start_timestamp
2103            };
2104        if self.slot == 0 {
2105            unix_timestamp = self.unix_timestamp_from_genesis();
2106            epoch_start_timestamp = self.unix_timestamp_from_genesis();
2107        }
2108        let clock = sysvar::clock::Clock {
2109            slot: self.slot,
2110            epoch_start_timestamp,
2111            epoch: self.epoch_schedule().get_epoch(self.slot),
2112            leader_schedule_epoch: self.epoch_schedule().get_leader_schedule_epoch(self.slot),
2113            unix_timestamp,
2114        };
2115        self.update_sysvar_account(&sysvar::clock::id(), |account| {
2116            create_account(
2117                &clock,
2118                self.inherit_specially_retained_account_fields(account),
2119            )
2120        });
2121    }
2122
2123    pub fn update_last_restart_slot(&self) {
2124        let feature_flag = self
2125            .feature_set
2126            .is_active(&feature_set::last_restart_slot_sysvar::id());
2127
2128        if feature_flag {
2129            // First, see what the currently stored last restart slot is. This
2130            // account may not exist yet if the feature was just activated.
2131            let current_last_restart_slot = self
2132                .get_account(&sysvar::last_restart_slot::id())
2133                .and_then(|account| {
2134                    let lrs: Option<LastRestartSlot> = from_account(&account);
2135                    lrs
2136                })
2137                .map(|account| account.last_restart_slot);
2138
2139            let last_restart_slot = {
2140                let slot = self.slot;
2141                let hard_forks_r = self.hard_forks.read().unwrap();
2142
2143                // Only consider hard forks <= this bank's slot to avoid prematurely applying
2144                // a hard fork that is set to occur in the future.
2145                hard_forks_r
2146                    .iter()
2147                    .rev()
2148                    .find(|(hard_fork, _)| *hard_fork <= slot)
2149                    .map(|(slot, _)| *slot)
2150                    .unwrap_or(0)
2151            };
2152
2153            // Only need to write if the last restart has changed
2154            if current_last_restart_slot != Some(last_restart_slot) {
2155                self.update_sysvar_account(&sysvar::last_restart_slot::id(), |account| {
2156                    create_account(
2157                        &LastRestartSlot { last_restart_slot },
2158                        self.inherit_specially_retained_account_fields(account),
2159                    )
2160                });
2161            }
2162        }
2163    }
2164
2165    pub fn set_sysvar_for_tests<T>(&self, sysvar: &T)
2166    where
2167        T: SysvarSerialize + SysvarId,
2168    {
2169        self.update_sysvar_account(&T::id(), |account| {
2170            create_account(
2171                sysvar,
2172                self.inherit_specially_retained_account_fields(account),
2173            )
2174        });
2175        // Simply force fill sysvar cache rather than checking which sysvar was
2176        // actually updated since tests don't need to be optimized for performance.
2177        self.transaction_processor.reset_sysvar_cache();
2178        self.transaction_processor
2179            .fill_missing_sysvar_cache_entries(self);
2180    }
2181
2182    fn update_slot_history(&self) {
2183        self.update_sysvar_account(&sysvar::slot_history::id(), |account| {
2184            let mut slot_history = account
2185                .as_ref()
2186                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
2187                .unwrap_or_default();
2188            slot_history.add(self.slot());
2189            create_account(
2190                &slot_history,
2191                self.inherit_specially_retained_account_fields(account),
2192            )
2193        });
2194    }
2195
2196    fn update_slot_hashes(&self) {
2197        self.update_sysvar_account(&sysvar::slot_hashes::id(), |account| {
2198            let mut slot_hashes = account
2199                .as_ref()
2200                .map(|account| from_account::<SlotHashes, _>(account).unwrap())
2201                .unwrap_or_default();
2202            slot_hashes.add(self.parent_slot, self.parent_hash);
2203            create_account(
2204                &slot_hashes,
2205                self.inherit_specially_retained_account_fields(account),
2206            )
2207        });
2208    }
2209
2210    pub fn get_slot_history(&self) -> SlotHistory {
2211        from_account(&self.get_account(&sysvar::slot_history::id()).unwrap()).unwrap()
2212    }
2213
2214    fn update_epoch_stakes(&mut self, leader_schedule_epoch: Epoch) {
2215        // update epoch_stakes cache
2216        //  if my parent didn't populate for this staker's epoch, we've
2217        //  crossed a boundary
2218        if !self.epoch_stakes.contains_key(&leader_schedule_epoch) {
2219            self.epoch_stakes.retain(|&epoch, _| {
2220                // Note the greater-than-or-equal (and the `- 1`) is needed here
2221                // to ensure we retain the oldest epoch, if that epoch is 0.
2222                epoch >= leader_schedule_epoch.saturating_sub(MAX_LEADER_SCHEDULE_STAKES - 1)
2223            });
2224            let stakes = self.stakes_cache.stakes().clone();
2225            let stakes = SerdeStakesToStakeFormat::from(stakes);
2226            let new_epoch_stakes = VersionedEpochStakes::new(stakes, leader_schedule_epoch);
2227            info!(
2228                "new epoch stakes, epoch: {}, total_stake: {}",
2229                leader_schedule_epoch,
2230                new_epoch_stakes.total_stake(),
2231            );
2232
2233            // It is expensive to log the details of epoch stakes. Only log them at "trace"
2234            // level for debugging purpose.
2235            if log::log_enabled!(log::Level::Trace) {
2236                let vote_stakes: HashMap<_, _> = self
2237                    .stakes_cache
2238                    .stakes()
2239                    .vote_accounts()
2240                    .delegated_stakes()
2241                    .map(|(pubkey, stake)| (*pubkey, stake))
2242                    .collect();
2243                trace!("new epoch stakes, stakes: {vote_stakes:#?}");
2244            }
2245            self.epoch_stakes
2246                .insert(leader_schedule_epoch, new_epoch_stakes);
2247        }
2248    }
2249
2250    #[cfg(feature = "dev-context-only-utils")]
2251    pub fn set_epoch_stakes_for_test(&mut self, epoch: Epoch, stakes: VersionedEpochStakes) {
2252        self.epoch_stakes.insert(epoch, stakes);
2253    }
2254
2255    fn update_rent(&self) {
2256        self.update_sysvar_account(&sysvar::rent::id(), |account| {
2257            create_account(
2258                &self.rent_collector.rent,
2259                self.inherit_specially_retained_account_fields(account),
2260            )
2261        });
2262    }
2263
2264    fn update_epoch_schedule(&self) {
2265        self.update_sysvar_account(&sysvar::epoch_schedule::id(), |account| {
2266            create_account(
2267                self.epoch_schedule(),
2268                self.inherit_specially_retained_account_fields(account),
2269            )
2270        });
2271    }
2272
2273    fn update_stake_history(&self, epoch: Option<Epoch>) {
2274        if epoch == Some(self.epoch()) {
2275            return;
2276        }
2277        // if I'm the first Bank in an epoch, ensure stake_history is updated
2278        self.update_sysvar_account(&stake_history::id(), |account| {
2279            create_account::<StakeHistory>(
2280                self.stakes_cache.stakes().history(),
2281                self.inherit_specially_retained_account_fields(account),
2282            )
2283        });
2284    }
2285
2286    pub fn epoch_duration_in_years(&self, prev_epoch: Epoch) -> f64 {
2287        // period: time that has passed as a fraction of a year, basically the length of
2288        //  an epoch as a fraction of a year
2289        //  calculated as: slots_elapsed / (slots / year)
2290        self.epoch_schedule().get_slots_in_epoch(prev_epoch) as f64 / self.slots_per_year
2291    }
2292
2293    // Calculates the starting-slot for inflation from the activation slot.
2294    // This method assumes that `pico_inflation` will be enabled before `full_inflation`, giving
2295    // precedence to the latter. However, since `pico_inflation` is fixed-rate Inflation, should
2296    // `pico_inflation` be enabled 2nd, the incorrect start slot provided here should have no
2297    // effect on the inflation calculation.
2298    fn get_inflation_start_slot(&self) -> Slot {
2299        let mut slots = self
2300            .feature_set
2301            .full_inflation_features_enabled()
2302            .iter()
2303            .filter_map(|id| self.feature_set.activated_slot(id))
2304            .collect::<Vec<_>>();
2305        slots.sort_unstable();
2306        slots.first().cloned().unwrap_or_else(|| {
2307            self.feature_set
2308                .activated_slot(&feature_set::pico_inflation::id())
2309                .unwrap_or(0)
2310        })
2311    }
2312
2313    fn get_inflation_num_slots(&self) -> u64 {
2314        let inflation_activation_slot = self.get_inflation_start_slot();
2315        // Normalize inflation_start to align with the start of rewards accrual.
2316        let inflation_start_slot = self.epoch_schedule().get_first_slot_in_epoch(
2317            self.epoch_schedule()
2318                .get_epoch(inflation_activation_slot)
2319                .saturating_sub(1),
2320        );
2321        self.epoch_schedule().get_first_slot_in_epoch(self.epoch()) - inflation_start_slot
2322    }
2323
2324    pub fn slot_in_year_for_inflation(&self) -> f64 {
2325        let num_slots = self.get_inflation_num_slots();
2326
2327        // calculated as: num_slots / (slots / year)
2328        num_slots as f64 / self.slots_per_year
2329    }
2330
2331    fn calculate_previous_epoch_inflation_rewards(
2332        &self,
2333        prev_epoch_capitalization: u64,
2334        prev_epoch: Epoch,
2335    ) -> PrevEpochInflationRewards {
2336        let slot_in_year = self.slot_in_year_for_inflation();
2337        let (validator_rate, foundation_rate) = {
2338            let inflation = self.inflation.read().unwrap();
2339            (
2340                (*inflation).validator(slot_in_year),
2341                (*inflation).foundation(slot_in_year),
2342            )
2343        };
2344
2345        let prev_epoch_duration_in_years = self.epoch_duration_in_years(prev_epoch);
2346        let validator_rewards = (validator_rate
2347            * prev_epoch_capitalization as f64
2348            * prev_epoch_duration_in_years) as u64;
2349
2350        PrevEpochInflationRewards {
2351            validator_rewards,
2352            prev_epoch_duration_in_years,
2353            validator_rate,
2354            foundation_rate,
2355        }
2356    }
2357
2358    /// Convert computed VoteRewards to VoteRewardsAccounts for storing.
2359    ///
2360    /// This function processes vote rewards and consolidates them into a single
2361    /// structure containing the pubkey, reward info, and updated account data
2362    /// for each vote account. The resulting structure is optimized for storage
2363    /// by combining previously separate rewards and accounts vectors into a
2364    /// single accounts_with_rewards vector.
2365    fn calc_vote_accounts_to_store(vote_account_rewards: VoteRewards) -> VoteRewardsAccounts {
2366        let len = vote_account_rewards.len();
2367        let mut result = VoteRewardsAccounts {
2368            accounts_with_rewards: Vec::with_capacity(len),
2369            total_vote_rewards_lamports: 0,
2370        };
2371        vote_account_rewards.into_iter().for_each(
2372            |(
2373                vote_pubkey,
2374                VoteReward {
2375                    mut vote_account,
2376                    commission,
2377                    vote_rewards,
2378                },
2379            )| {
2380                if let Err(err) = vote_account.checked_add_lamports(vote_rewards) {
2381                    debug!("reward redemption failed for {vote_pubkey}: {err:?}");
2382                    return;
2383                }
2384
2385                result.accounts_with_rewards.push((
2386                    vote_pubkey,
2387                    RewardInfo {
2388                        reward_type: RewardType::Voting,
2389                        lamports: vote_rewards as i64,
2390                        post_balance: vote_account.lamports(),
2391                        commission: Some(commission),
2392                    },
2393                    vote_account,
2394                ));
2395                result.total_vote_rewards_lamports += vote_rewards;
2396            },
2397        );
2398        result
2399    }
2400
2401    fn update_vote_rewards(&self, vote_rewards: &VoteRewardsAccounts) {
2402        let mut rewards = self.rewards.write().unwrap();
2403        rewards.reserve(vote_rewards.accounts_with_rewards.len());
2404        vote_rewards
2405            .accounts_with_rewards
2406            .iter()
2407            .for_each(|(vote_pubkey, vote_reward, _)| {
2408                rewards.push((*vote_pubkey, *vote_reward));
2409            });
2410    }
2411
2412    fn update_recent_blockhashes_locked(&self, locked_blockhash_queue: &BlockhashQueue) {
2413        #[allow(deprecated)]
2414        self.update_sysvar_account(&sysvar::recent_blockhashes::id(), |account| {
2415            let recent_blockhash_iter = locked_blockhash_queue.get_recent_blockhashes();
2416            recent_blockhashes_account::create_account_with_data_and_fields(
2417                recent_blockhash_iter,
2418                self.inherit_specially_retained_account_fields(account),
2419            )
2420        });
2421    }
2422
2423    pub fn update_recent_blockhashes(&self) {
2424        let blockhash_queue = self.blockhash_queue.read().unwrap();
2425        self.update_recent_blockhashes_locked(&blockhash_queue);
2426    }
2427
2428    fn get_timestamp_estimate(
2429        &self,
2430        max_allowable_drift: MaxAllowableDrift,
2431        epoch_start_timestamp: Option<(Slot, UnixTimestamp)>,
2432    ) -> Option<UnixTimestamp> {
2433        let mut get_timestamp_estimate_time = Measure::start("get_timestamp_estimate");
2434        let slots_per_epoch = self.epoch_schedule().slots_per_epoch;
2435        let vote_accounts = self.vote_accounts();
2436        let recent_timestamps = vote_accounts.iter().filter_map(|(pubkey, (_, account))| {
2437            let vote_state = account.vote_state_view();
2438            let last_timestamp = vote_state.last_timestamp();
2439            let slot_delta = self.slot().checked_sub(last_timestamp.slot)?;
2440            (slot_delta <= slots_per_epoch)
2441                .then_some((*pubkey, (last_timestamp.slot, last_timestamp.timestamp)))
2442        });
2443        let slot_duration = Duration::from_nanos(self.ns_per_slot as u64);
2444        let epoch = self.epoch_schedule().get_epoch(self.slot());
2445        let stakes = self.epoch_vote_accounts(epoch)?;
2446        let stake_weighted_timestamp = calculate_stake_weighted_timestamp(
2447            recent_timestamps,
2448            stakes,
2449            self.slot(),
2450            slot_duration,
2451            epoch_start_timestamp,
2452            max_allowable_drift,
2453            self.feature_set
2454                .is_active(&feature_set::warp_timestamp_again::id()),
2455        );
2456        get_timestamp_estimate_time.stop();
2457        datapoint_info!(
2458            "bank-timestamp",
2459            (
2460                "get_timestamp_estimate_us",
2461                get_timestamp_estimate_time.as_us(),
2462                i64
2463            ),
2464        );
2465        stake_weighted_timestamp
2466    }
2467
2468    /// Recalculates the bank hash
2469    ///
2470    /// This is used by ledger-tool when creating a snapshot, which
2471    /// recalcuates the bank hash.
2472    ///
2473    /// Note that the account state is *not* allowed to change by rehashing.
2474    /// If modifying accounts in ledger-tool is needed, create a new bank.
2475    pub fn rehash(&self) {
2476        let mut hash = self.hash.write().unwrap();
2477        let new = self.hash_internal_state();
2478        if new != *hash {
2479            warn!("Updating bank hash to {new}");
2480            *hash = new;
2481        }
2482    }
2483
2484    pub fn freeze(&self) {
2485        // This lock prevents any new commits from BankingStage
2486        // `Consumer::execute_and_commit_transactions_locked()` from
2487        // coming in after the last tick is observed. This is because in
2488        // BankingStage, any transaction successfully recorded in
2489        // `record_transactions()` is recorded after this `hash` lock
2490        // is grabbed. At the time of the successful record,
2491        // this means the PoH has not yet reached the last tick,
2492        // so this means freeze() hasn't been called yet. And because
2493        // BankingStage doesn't release this hash lock until both
2494        // record and commit are finished, those transactions will be
2495        // committed before this write lock can be obtained here.
2496        let mut hash = self.hash.write().unwrap();
2497        if *hash == Hash::default() {
2498            // finish up any deferred changes to account state
2499            self.distribute_transaction_fee_details();
2500            self.update_slot_history();
2501            self.run_incinerator();
2502
2503            // freeze is a one-way trip, idempotent
2504            self.freeze_started.store(true, Relaxed);
2505            // updating the accounts lt hash must happen *outside* of hash_internal_state() so
2506            // that rehash() can be called and *not* modify self.accounts_lt_hash.
2507            self.update_accounts_lt_hash();
2508            *hash = self.hash_internal_state();
2509            self.rc.accounts.accounts_db.mark_slot_frozen(self.slot());
2510        }
2511    }
2512
2513    // dangerous; don't use this; this is only needed for ledger-tool's special command
2514    #[cfg(feature = "dev-context-only-utils")]
2515    pub fn unfreeze_for_ledger_tool(&self) {
2516        self.freeze_started.store(false, Relaxed);
2517    }
2518
2519    pub fn epoch_schedule(&self) -> &EpochSchedule {
2520        &self.epoch_schedule
2521    }
2522
2523    /// squash the parent's state up into this Bank,
2524    ///   this Bank becomes a root
2525    /// Note that this function is not thread-safe. If it is called concurrently on the same bank
2526    /// by multiple threads, the end result could be inconsistent.
2527    /// Calling code does not currently call this concurrently.
2528    pub fn squash(&self) -> SquashTiming {
2529        self.freeze();
2530
2531        //this bank and all its parents are now on the rooted path
2532        let mut roots = vec![self.slot()];
2533        roots.append(&mut self.parents().iter().map(|p| p.slot()).collect());
2534
2535        let mut total_index_us = 0;
2536        let mut total_cache_us = 0;
2537
2538        let mut squash_accounts_time = Measure::start("squash_accounts_time");
2539        for slot in roots.iter().rev() {
2540            // root forks cannot be purged
2541            let add_root_timing = self.rc.accounts.add_root(*slot);
2542            total_index_us += add_root_timing.index_us;
2543            total_cache_us += add_root_timing.cache_us;
2544        }
2545        squash_accounts_time.stop();
2546
2547        *self.rc.parent.write().unwrap() = None;
2548
2549        let mut squash_cache_time = Measure::start("squash_cache_time");
2550        roots
2551            .iter()
2552            .for_each(|slot| self.status_cache.write().unwrap().add_root(*slot));
2553        squash_cache_time.stop();
2554
2555        SquashTiming {
2556            squash_accounts_ms: squash_accounts_time.as_ms(),
2557            squash_accounts_index_ms: total_index_us / 1000,
2558            squash_accounts_cache_ms: total_cache_us / 1000,
2559            squash_cache_ms: squash_cache_time.as_ms(),
2560        }
2561    }
2562
2563    /// Return the more recent checkpoint of this bank instance.
2564    pub fn parent(&self) -> Option<Arc<Bank>> {
2565        self.rc.parent.read().unwrap().clone()
2566    }
2567
2568    pub fn parent_slot(&self) -> Slot {
2569        self.parent_slot
2570    }
2571
2572    pub fn parent_hash(&self) -> Hash {
2573        self.parent_hash
2574    }
2575
2576    fn process_genesis_config(
2577        &mut self,
2578        genesis_config: &GenesisConfig,
2579        #[cfg(feature = "dev-context-only-utils")] collector_id_for_tests: Option<Pubkey>,
2580        #[cfg(feature = "dev-context-only-utils")] genesis_hash: Option<Hash>,
2581    ) {
2582        // Bootstrap validator collects fees until `new_from_parent` is called.
2583        self.fee_rate_governor = genesis_config.fee_rate_governor.clone();
2584
2585        for (pubkey, account) in genesis_config.accounts.iter() {
2586            assert!(
2587                self.get_account(pubkey).is_none(),
2588                "{pubkey} repeated in genesis config"
2589            );
2590            let account_shared_data = create_account_shared_data(account);
2591            self.store_account(pubkey, &account_shared_data);
2592            self.capitalization.fetch_add(account.lamports(), Relaxed);
2593            self.accounts_data_size_initial += account.data().len() as u64;
2594        }
2595
2596        for (pubkey, account) in genesis_config.rewards_pools.iter() {
2597            assert!(
2598                self.get_account(pubkey).is_none(),
2599                "{pubkey} repeated in genesis config"
2600            );
2601            let account_shared_data = create_account_shared_data(account);
2602            self.store_account(pubkey, &account_shared_data);
2603            self.accounts_data_size_initial += account.data().len() as u64;
2604        }
2605
2606        // After storing genesis accounts, the bank stakes cache will be warmed
2607        // up and can be used to set the collector id to the highest staked
2608        // node. If no staked nodes exist, allow fallback to an unstaked test
2609        // collector id during tests.
2610        let collector_id = self.stakes_cache.stakes().highest_staked_node().copied();
2611        #[cfg(feature = "dev-context-only-utils")]
2612        let collector_id = collector_id.or(collector_id_for_tests);
2613        self.collector_id =
2614            collector_id.expect("genesis processing failed because no staked nodes exist");
2615
2616        #[cfg(not(feature = "dev-context-only-utils"))]
2617        let genesis_hash = genesis_config.hash();
2618        #[cfg(feature = "dev-context-only-utils")]
2619        let genesis_hash = genesis_hash.unwrap_or(genesis_config.hash());
2620
2621        self.blockhash_queue.write().unwrap().genesis_hash(
2622            &genesis_hash,
2623            genesis_config.fee_rate_governor.lamports_per_signature,
2624        );
2625
2626        self.hashes_per_tick = genesis_config.hashes_per_tick();
2627        self.ticks_per_slot = genesis_config.ticks_per_slot();
2628        self.ns_per_slot = genesis_config.ns_per_slot();
2629        self.genesis_creation_time = genesis_config.creation_time;
2630        self.max_tick_height = (self.slot + 1) * self.ticks_per_slot;
2631        self.slots_per_year = genesis_config.slots_per_year();
2632
2633        self.epoch_schedule = genesis_config.epoch_schedule.clone();
2634
2635        self.inflation = Arc::new(RwLock::new(genesis_config.inflation));
2636
2637        self.rent_collector = RentCollector::new(
2638            self.epoch,
2639            self.epoch_schedule().clone(),
2640            self.slots_per_year,
2641            genesis_config.rent.clone(),
2642        );
2643    }
2644
2645    fn burn_and_purge_account(&self, program_id: &Pubkey, mut account: AccountSharedData) {
2646        let old_data_size = account.data().len();
2647        self.capitalization.fetch_sub(account.lamports(), Relaxed);
2648        // Both resetting account balance to 0 and zeroing the account data
2649        // is needed to really purge from AccountsDb and flush the Stakes cache
2650        account.set_lamports(0);
2651        account.data_as_mut_slice().fill(0);
2652        self.store_account(program_id, &account);
2653        self.calculate_and_update_accounts_data_size_delta_off_chain(old_data_size, 0);
2654    }
2655
2656    /// Add a precompiled program account
2657    pub fn add_precompiled_account(&self, program_id: &Pubkey) {
2658        self.add_precompiled_account_with_owner(program_id, native_loader::id())
2659    }
2660
2661    // Used by tests to simulate clusters with precompiles that aren't owned by the native loader
2662    fn add_precompiled_account_with_owner(&self, program_id: &Pubkey, owner: Pubkey) {
2663        if let Some(account) = self.get_account_with_fixed_root(program_id) {
2664            if account.executable() {
2665                return;
2666            } else {
2667                // malicious account is pre-occupying at program_id
2668                self.burn_and_purge_account(program_id, account);
2669            }
2670        };
2671
2672        assert!(
2673            !self.freeze_started(),
2674            "Can't change frozen bank by adding not-existing new precompiled program \
2675             ({program_id}). Maybe, inconsistent program activation is detected on snapshot \
2676             restore?"
2677        );
2678
2679        // Add a bogus executable account, which will be loaded and ignored.
2680        let (lamports, rent_epoch) = self.inherit_specially_retained_account_fields(&None);
2681
2682        let account = AccountSharedData::from(Account {
2683            lamports,
2684            owner,
2685            data: vec![],
2686            executable: true,
2687            rent_epoch,
2688        });
2689        self.store_account_and_update_capitalization(program_id, &account);
2690    }
2691
2692    pub fn set_rent_burn_percentage(&mut self, burn_percent: u8) {
2693        self.rent_collector.rent.burn_percent = burn_percent;
2694    }
2695
2696    pub fn set_hashes_per_tick(&mut self, hashes_per_tick: Option<u64>) {
2697        self.hashes_per_tick = hashes_per_tick;
2698    }
2699
2700    /// Return the last block hash registered.
2701    pub fn last_blockhash(&self) -> Hash {
2702        self.blockhash_queue.read().unwrap().last_hash()
2703    }
2704
2705    pub fn last_blockhash_and_lamports_per_signature(&self) -> (Hash, u64) {
2706        let blockhash_queue = self.blockhash_queue.read().unwrap();
2707        let last_hash = blockhash_queue.last_hash();
2708        let last_lamports_per_signature = blockhash_queue
2709            .get_lamports_per_signature(&last_hash)
2710            .unwrap(); // safe so long as the BlockhashQueue is consistent
2711        (last_hash, last_lamports_per_signature)
2712    }
2713
2714    pub fn is_blockhash_valid(&self, hash: &Hash) -> bool {
2715        let blockhash_queue = self.blockhash_queue.read().unwrap();
2716        blockhash_queue.is_hash_valid_for_age(hash, MAX_PROCESSING_AGE)
2717    }
2718
2719    pub fn get_minimum_balance_for_rent_exemption(&self, data_len: usize) -> u64 {
2720        self.rent_collector.rent.minimum_balance(data_len).max(1)
2721    }
2722
2723    pub fn get_lamports_per_signature(&self) -> u64 {
2724        self.fee_rate_governor.lamports_per_signature
2725    }
2726
2727    pub fn get_lamports_per_signature_for_blockhash(&self, hash: &Hash) -> Option<u64> {
2728        let blockhash_queue = self.blockhash_queue.read().unwrap();
2729        blockhash_queue.get_lamports_per_signature(hash)
2730    }
2731
2732    pub fn get_fee_for_message(&self, message: &SanitizedMessage) -> Option<u64> {
2733        let lamports_per_signature = {
2734            let blockhash_queue = self.blockhash_queue.read().unwrap();
2735            blockhash_queue.get_lamports_per_signature(message.recent_blockhash())
2736        }
2737        .or_else(|| {
2738            self.load_message_nonce_account(message).map(
2739                |(_nonce_address, _nonce_account, nonce_data)| {
2740                    nonce_data.get_lamports_per_signature()
2741                },
2742            )
2743        })?;
2744        Some(self.get_fee_for_message_with_lamports_per_signature(message, lamports_per_signature))
2745    }
2746
2747    pub fn get_fee_for_message_with_lamports_per_signature(
2748        &self,
2749        message: &impl SVMMessage,
2750        lamports_per_signature: u64,
2751    ) -> u64 {
2752        let fee_budget_limits = FeeBudgetLimits::from(
2753            process_compute_budget_instructions(
2754                message.program_instructions_iter(),
2755                &self.feature_set,
2756            )
2757            .unwrap_or_default(),
2758        );
2759        solana_fee::calculate_fee(
2760            message,
2761            lamports_per_signature == 0,
2762            self.fee_structure().lamports_per_signature,
2763            fee_budget_limits.prioritization_fee,
2764            FeeFeatures::from(self.feature_set.as_ref()),
2765        )
2766    }
2767
2768    pub fn get_blockhash_last_valid_block_height(&self, blockhash: &Hash) -> Option<Slot> {
2769        let blockhash_queue = self.blockhash_queue.read().unwrap();
2770        // This calculation will need to be updated to consider epoch boundaries if BlockhashQueue
2771        // length is made variable by epoch
2772        blockhash_queue
2773            .get_hash_age(blockhash)
2774            .map(|age| self.block_height + MAX_PROCESSING_AGE as u64 - age)
2775    }
2776
2777    pub fn confirmed_last_blockhash(&self) -> Hash {
2778        const NUM_BLOCKHASH_CONFIRMATIONS: usize = 3;
2779
2780        let parents = self.parents();
2781        if parents.is_empty() {
2782            self.last_blockhash()
2783        } else {
2784            let index = NUM_BLOCKHASH_CONFIRMATIONS.min(parents.len() - 1);
2785            parents[index].last_blockhash()
2786        }
2787    }
2788
2789    /// Forget all signatures. Useful for benchmarking.
2790    #[cfg(feature = "dev-context-only-utils")]
2791    pub fn clear_signatures(&self) {
2792        self.status_cache.write().unwrap().clear();
2793    }
2794
2795    pub fn clear_slot_signatures(&self, slot: Slot) {
2796        self.status_cache.write().unwrap().clear_slot_entries(slot);
2797    }
2798
2799    fn update_transaction_statuses(
2800        &self,
2801        sanitized_txs: &[impl TransactionWithMeta],
2802        processing_results: &[TransactionProcessingResult],
2803    ) {
2804        let mut status_cache = self.status_cache.write().unwrap();
2805        assert_eq!(sanitized_txs.len(), processing_results.len());
2806        for (tx, processing_result) in sanitized_txs.iter().zip(processing_results) {
2807            if let Ok(processed_tx) = &processing_result {
2808                // Add the message hash to the status cache to ensure that this message
2809                // won't be processed again with a different signature.
2810                status_cache.insert(
2811                    tx.recent_blockhash(),
2812                    tx.message_hash(),
2813                    self.slot(),
2814                    processed_tx.status(),
2815                );
2816                // Add the transaction signature to the status cache so that transaction status
2817                // can be queried by transaction signature over RPC. In the future, this should
2818                // only be added for API nodes because voting validators don't need to do this.
2819                status_cache.insert(
2820                    tx.recent_blockhash(),
2821                    tx.signature(),
2822                    self.slot(),
2823                    processed_tx.status(),
2824                );
2825            }
2826        }
2827    }
2828
2829    /// Register a new recent blockhash in the bank's recent blockhash queue. Called when a bank
2830    /// reaches its max tick height. Can be called by tests to get new blockhashes for transaction
2831    /// processing without advancing to a new bank slot.
2832    fn register_recent_blockhash(&self, blockhash: &Hash, scheduler: &InstalledSchedulerRwLock) {
2833        // This is needed because recent_blockhash updates necessitate synchronizations for
2834        // consistent tx check_age handling.
2835        BankWithScheduler::wait_for_paused_scheduler(self, scheduler);
2836
2837        // Only acquire the write lock for the blockhash queue on block boundaries because
2838        // readers can starve this write lock acquisition and ticks would be slowed down too
2839        // much if the write lock is acquired for each tick.
2840        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
2841
2842        #[cfg(feature = "dev-context-only-utils")]
2843        let blockhash_override = self
2844            .hash_overrides
2845            .lock()
2846            .unwrap()
2847            .get_blockhash_override(self.slot())
2848            .copied()
2849            .inspect(|blockhash_override| {
2850                if blockhash_override != blockhash {
2851                    info!(
2852                        "bank: slot: {}: overrode blockhash: {} with {}",
2853                        self.slot(),
2854                        blockhash,
2855                        blockhash_override
2856                    );
2857                }
2858            });
2859        #[cfg(feature = "dev-context-only-utils")]
2860        let blockhash = blockhash_override.as_ref().unwrap_or(blockhash);
2861
2862        w_blockhash_queue.register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
2863        self.update_recent_blockhashes_locked(&w_blockhash_queue);
2864    }
2865
2866    // gating this under #[cfg(feature = "dev-context-only-utils")] isn't easy due to
2867    // solana-program-test's usage...
2868    pub fn register_unique_recent_blockhash_for_test(&self) {
2869        self.register_recent_blockhash(
2870            &Hash::new_unique(),
2871            &BankWithScheduler::no_scheduler_available(),
2872        )
2873    }
2874
2875    #[cfg(feature = "dev-context-only-utils")]
2876    pub fn register_recent_blockhash_for_test(
2877        &self,
2878        blockhash: &Hash,
2879        lamports_per_signature: Option<u64>,
2880    ) {
2881        // Only acquire the write lock for the blockhash queue on block boundaries because
2882        // readers can starve this write lock acquisition and ticks would be slowed down too
2883        // much if the write lock is acquired for each tick.
2884        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
2885        if let Some(lamports_per_signature) = lamports_per_signature {
2886            w_blockhash_queue.register_hash(blockhash, lamports_per_signature);
2887        } else {
2888            w_blockhash_queue
2889                .register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
2890        }
2891    }
2892
2893    /// Tell the bank which Entry IDs exist on the ledger. This function assumes subsequent calls
2894    /// correspond to later entries, and will boot the oldest ones once its internal cache is full.
2895    /// Once boot, the bank will reject transactions using that `hash`.
2896    ///
2897    /// This is NOT thread safe because if tick height is updated by two different threads, the
2898    /// block boundary condition could be missed.
2899    pub fn register_tick(&self, hash: &Hash, scheduler: &InstalledSchedulerRwLock) {
2900        assert!(
2901            !self.freeze_started(),
2902            "register_tick() working on a bank that is already frozen or is undergoing freezing!"
2903        );
2904
2905        if self.is_block_boundary(self.tick_height.load(Relaxed) + 1) {
2906            self.register_recent_blockhash(hash, scheduler);
2907        }
2908
2909        // ReplayStage will start computing the accounts delta hash when it
2910        // detects the tick height has reached the boundary, so the system
2911        // needs to guarantee all account updates for the slot have been
2912        // committed before this tick height is incremented (like the blockhash
2913        // sysvar above)
2914        self.tick_height.fetch_add(1, Relaxed);
2915    }
2916
2917    #[cfg(feature = "dev-context-only-utils")]
2918    pub fn register_tick_for_test(&self, hash: &Hash) {
2919        self.register_tick(hash, &BankWithScheduler::no_scheduler_available())
2920    }
2921
2922    #[cfg(feature = "dev-context-only-utils")]
2923    pub fn register_default_tick_for_test(&self) {
2924        self.register_tick_for_test(&Hash::default())
2925    }
2926
2927    pub fn is_complete(&self) -> bool {
2928        self.tick_height() == self.max_tick_height()
2929    }
2930
2931    pub fn is_block_boundary(&self, tick_height: u64) -> bool {
2932        tick_height == self.max_tick_height
2933    }
2934
2935    /// Get the max number of accounts that a transaction may lock in this block
2936    pub fn get_transaction_account_lock_limit(&self) -> usize {
2937        if let Some(transaction_account_lock_limit) = self.transaction_account_lock_limit {
2938            transaction_account_lock_limit
2939        } else if self
2940            .feature_set
2941            .is_active(&feature_set::increase_tx_account_lock_limit::id())
2942        {
2943            MAX_TX_ACCOUNT_LOCKS
2944        } else {
2945            64
2946        }
2947    }
2948
2949    /// Prepare a transaction batch from a list of versioned transactions from
2950    /// an entry. Used for tests only.
2951    pub fn prepare_entry_batch(
2952        &self,
2953        txs: Vec<VersionedTransaction>,
2954    ) -> Result<TransactionBatch<'_, '_, RuntimeTransaction<SanitizedTransaction>>> {
2955        let enable_static_instruction_limit = self
2956            .feature_set
2957            .is_active(&agave_feature_set::static_instruction_limit::id());
2958        let sanitized_txs = txs
2959            .into_iter()
2960            .map(|tx| {
2961                RuntimeTransaction::try_create(
2962                    tx,
2963                    MessageHash::Compute,
2964                    None,
2965                    self,
2966                    self.get_reserved_account_keys(),
2967                    enable_static_instruction_limit,
2968                )
2969            })
2970            .collect::<Result<Vec<_>>>()?;
2971        Ok(TransactionBatch::new(
2972            self.try_lock_accounts(&sanitized_txs),
2973            self,
2974            OwnedOrBorrowed::Owned(sanitized_txs),
2975        ))
2976    }
2977
2978    /// Attempt to take locks on the accounts in a transaction batch
2979    pub fn try_lock_accounts(&self, txs: &[impl TransactionWithMeta]) -> Vec<Result<()>> {
2980        self.try_lock_accounts_with_results(txs, txs.iter().map(|_| Ok(())))
2981    }
2982
2983    /// Attempt to take locks on the accounts in a transaction batch, and their cost
2984    /// limited packing status and duplicate transaction conflict status
2985    pub fn try_lock_accounts_with_results(
2986        &self,
2987        txs: &[impl TransactionWithMeta],
2988        tx_results: impl Iterator<Item = Result<()>>,
2989    ) -> Vec<Result<()>> {
2990        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
2991        let relax_intrabatch_account_locks = self
2992            .feature_set
2993            .is_active(&feature_set::relax_intrabatch_account_locks::id());
2994
2995        // with simd83 enabled, we must fail transactions that duplicate a prior message hash
2996        // previously, conflicting account locks would fail such transactions as a side effect
2997        let mut batch_message_hashes = AHashSet::with_capacity(txs.len());
2998        let tx_results = tx_results
2999            .enumerate()
3000            .map(|(i, tx_result)| match tx_result {
3001                Ok(()) if relax_intrabatch_account_locks => {
3002                    // `HashSet::insert()` returns `true` when the value does *not* already exist
3003                    if batch_message_hashes.insert(txs[i].message_hash()) {
3004                        Ok(())
3005                    } else {
3006                        Err(TransactionError::AlreadyProcessed)
3007                    }
3008                }
3009                Ok(()) => Ok(()),
3010                Err(e) => Err(e),
3011            });
3012
3013        self.rc.accounts.lock_accounts(
3014            txs.iter(),
3015            tx_results,
3016            tx_account_lock_limit,
3017            relax_intrabatch_account_locks,
3018        )
3019    }
3020
3021    /// Prepare a locked transaction batch from a list of sanitized transactions.
3022    pub fn prepare_sanitized_batch<'a, 'b, Tx: TransactionWithMeta>(
3023        &'a self,
3024        txs: &'b [Tx],
3025    ) -> TransactionBatch<'a, 'b, Tx> {
3026        self.prepare_sanitized_batch_with_results(txs, txs.iter().map(|_| Ok(())))
3027    }
3028
3029    /// Prepare a locked transaction batch from a list of sanitized transactions, and their cost
3030    /// limited packing status
3031    pub fn prepare_sanitized_batch_with_results<'a, 'b, Tx: TransactionWithMeta>(
3032        &'a self,
3033        transactions: &'b [Tx],
3034        transaction_results: impl Iterator<Item = Result<()>>,
3035    ) -> TransactionBatch<'a, 'b, Tx> {
3036        // this lock_results could be: Ok, AccountInUse, WouldExceedBlockMaxLimit or WouldExceedAccountMaxLimit
3037        TransactionBatch::new(
3038            self.try_lock_accounts_with_results(transactions, transaction_results),
3039            self,
3040            OwnedOrBorrowed::Borrowed(transactions),
3041        )
3042    }
3043
3044    /// Prepare a transaction batch from a single transaction without locking accounts
3045    pub fn prepare_unlocked_batch_from_single_tx<'a, Tx: SVMMessage>(
3046        &'a self,
3047        transaction: &'a Tx,
3048    ) -> TransactionBatch<'a, 'a, Tx> {
3049        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3050        let lock_result = validate_account_locks(transaction.account_keys(), tx_account_lock_limit);
3051        let mut batch = TransactionBatch::new(
3052            vec![lock_result],
3053            self,
3054            OwnedOrBorrowed::Borrowed(slice::from_ref(transaction)),
3055        );
3056        batch.set_needs_unlock(false);
3057        batch
3058    }
3059
3060    /// Prepare a transaction batch from a single transaction after locking accounts
3061    pub fn prepare_locked_batch_from_single_tx<'a, Tx: TransactionWithMeta>(
3062        &'a self,
3063        transaction: &'a Tx,
3064    ) -> TransactionBatch<'a, 'a, Tx> {
3065        self.prepare_sanitized_batch(slice::from_ref(transaction))
3066    }
3067
3068    /// Run transactions against a frozen bank without committing the results
3069    pub fn simulate_transaction(
3070        &self,
3071        transaction: &impl TransactionWithMeta,
3072        enable_cpi_recording: bool,
3073    ) -> TransactionSimulationResult {
3074        assert!(self.is_frozen(), "simulation bank must be frozen");
3075
3076        self.simulate_transaction_unchecked(transaction, enable_cpi_recording)
3077    }
3078
3079    /// Run transactions against a bank without committing the results; does not check if the bank
3080    /// is frozen, enabling use in single-Bank test frameworks
3081    pub fn simulate_transaction_unchecked(
3082        &self,
3083        transaction: &impl TransactionWithMeta,
3084        enable_cpi_recording: bool,
3085    ) -> TransactionSimulationResult {
3086        let account_keys = transaction.account_keys();
3087        let number_of_accounts = account_keys.len();
3088        let account_overrides = self.get_account_overrides_for_simulation(&account_keys);
3089        let batch = self.prepare_unlocked_batch_from_single_tx(transaction);
3090        let mut timings = ExecuteTimings::default();
3091
3092        let LoadAndExecuteTransactionsOutput {
3093            mut processing_results,
3094            balance_collector,
3095            ..
3096        } = self.load_and_execute_transactions(
3097            &batch,
3098            // After simulation, transactions will need to be forwarded to the leader
3099            // for processing. During forwarding, the transaction could expire if the
3100            // delay is not accounted for.
3101            MAX_PROCESSING_AGE - MAX_TRANSACTION_FORWARDING_DELAY,
3102            &mut timings,
3103            &mut TransactionErrorMetrics::default(),
3104            TransactionProcessingConfig {
3105                account_overrides: Some(&account_overrides),
3106                check_program_modification_slot: self.check_program_modification_slot,
3107                log_messages_bytes_limit: None,
3108                limit_to_load_programs: true,
3109                recording_config: ExecutionRecordingConfig {
3110                    enable_cpi_recording,
3111                    enable_log_recording: true,
3112                    enable_return_data_recording: true,
3113                    enable_transaction_balance_recording: true,
3114                },
3115            },
3116        );
3117
3118        debug!("simulate_transaction: {timings:?}");
3119
3120        let processing_result = processing_results
3121            .pop()
3122            .unwrap_or(Err(TransactionError::InvalidProgramForExecution));
3123        let (
3124            post_simulation_accounts,
3125            result,
3126            fee,
3127            logs,
3128            return_data,
3129            inner_instructions,
3130            units_consumed,
3131            loaded_accounts_data_size,
3132        ) = match processing_result {
3133            Ok(processed_tx) => {
3134                let executed_units = processed_tx.executed_units();
3135                let loaded_accounts_data_size = processed_tx.loaded_accounts_data_size();
3136
3137                match processed_tx {
3138                    ProcessedTransaction::Executed(executed_tx) => {
3139                        let details = executed_tx.execution_details;
3140                        let post_simulation_accounts = executed_tx
3141                            .loaded_transaction
3142                            .accounts
3143                            .into_iter()
3144                            .take(number_of_accounts)
3145                            .collect::<Vec<_>>();
3146                        (
3147                            post_simulation_accounts,
3148                            details.status,
3149                            Some(executed_tx.loaded_transaction.fee_details.total_fee()),
3150                            details.log_messages,
3151                            details.return_data,
3152                            details.inner_instructions,
3153                            executed_units,
3154                            loaded_accounts_data_size,
3155                        )
3156                    }
3157                    ProcessedTransaction::FeesOnly(fees_only_tx) => (
3158                        vec![],
3159                        Err(fees_only_tx.load_error),
3160                        Some(fees_only_tx.fee_details.total_fee()),
3161                        None,
3162                        None,
3163                        None,
3164                        executed_units,
3165                        loaded_accounts_data_size,
3166                    ),
3167                }
3168            }
3169            Err(error) => (vec![], Err(error), None, None, None, None, 0, 0),
3170        };
3171        let logs = logs.unwrap_or_default();
3172
3173        let (pre_balances, post_balances, pre_token_balances, post_token_balances) =
3174            match balance_collector {
3175                Some(balance_collector) => {
3176                    let (mut native_pre, mut native_post, mut token_pre, mut token_post) =
3177                        balance_collector.into_vecs();
3178
3179                    (
3180                        native_pre.pop(),
3181                        native_post.pop(),
3182                        token_pre.pop(),
3183                        token_post.pop(),
3184                    )
3185                }
3186                None => (None, None, None, None),
3187            };
3188
3189        TransactionSimulationResult {
3190            result,
3191            logs,
3192            post_simulation_accounts,
3193            units_consumed,
3194            loaded_accounts_data_size,
3195            return_data,
3196            inner_instructions,
3197            fee,
3198            pre_balances,
3199            post_balances,
3200            pre_token_balances,
3201            post_token_balances,
3202        }
3203    }
3204
3205    fn get_account_overrides_for_simulation(&self, account_keys: &AccountKeys) -> AccountOverrides {
3206        let mut account_overrides = AccountOverrides::default();
3207        let slot_history_id = sysvar::slot_history::id();
3208        if account_keys.iter().any(|pubkey| *pubkey == slot_history_id) {
3209            let current_account = self.get_account_with_fixed_root(&slot_history_id);
3210            let slot_history = current_account
3211                .as_ref()
3212                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
3213                .unwrap_or_default();
3214            if slot_history.check(self.slot()) == Check::Found {
3215                let ancestors = Ancestors::from(self.proper_ancestors().collect::<Vec<_>>());
3216                if let Some((account, _)) =
3217                    self.load_slow_with_fixed_root(&ancestors, &slot_history_id)
3218                {
3219                    account_overrides.set_slot_history(Some(account));
3220                }
3221            }
3222        }
3223        account_overrides
3224    }
3225
3226    pub fn unlock_accounts<'a, Tx: SVMMessage + 'a>(
3227        &self,
3228        txs_and_results: impl Iterator<Item = (&'a Tx, &'a Result<()>)> + Clone,
3229    ) {
3230        self.rc.accounts.unlock_accounts(txs_and_results)
3231    }
3232
3233    pub fn remove_unrooted_slots(&self, slots: &[(Slot, BankId)]) {
3234        self.rc.accounts.accounts_db.remove_unrooted_slots(slots)
3235    }
3236
3237    pub fn get_hash_age(&self, hash: &Hash) -> Option<u64> {
3238        self.blockhash_queue.read().unwrap().get_hash_age(hash)
3239    }
3240
3241    pub fn is_hash_valid_for_age(&self, hash: &Hash, max_age: usize) -> bool {
3242        self.blockhash_queue
3243            .read()
3244            .unwrap()
3245            .is_hash_valid_for_age(hash, max_age)
3246    }
3247
3248    pub fn collect_balances(
3249        &self,
3250        batch: &TransactionBatch<impl SVMMessage>,
3251    ) -> TransactionBalances {
3252        let mut balances: TransactionBalances = vec![];
3253        for transaction in batch.sanitized_transactions() {
3254            let mut transaction_balances: Vec<u64> = vec![];
3255            for account_key in transaction.account_keys().iter() {
3256                transaction_balances.push(self.get_balance(account_key));
3257            }
3258            balances.push(transaction_balances);
3259        }
3260        balances
3261    }
3262
3263    pub fn load_and_execute_transactions(
3264        &self,
3265        batch: &TransactionBatch<impl TransactionWithMeta>,
3266        max_age: usize,
3267        timings: &mut ExecuteTimings,
3268        error_counters: &mut TransactionErrorMetrics,
3269        processing_config: TransactionProcessingConfig,
3270    ) -> LoadAndExecuteTransactionsOutput {
3271        let sanitized_txs = batch.sanitized_transactions();
3272
3273        let (check_results, check_us) = measure_us!(self.check_transactions(
3274            sanitized_txs,
3275            batch.lock_results(),
3276            max_age,
3277            error_counters,
3278        ));
3279        timings.saturating_add_in_place(ExecuteTimingType::CheckUs, check_us);
3280
3281        let (blockhash, blockhash_lamports_per_signature) =
3282            self.last_blockhash_and_lamports_per_signature();
3283        let effective_epoch_of_deployments =
3284            self.epoch_schedule().get_epoch(self.slot.saturating_add(
3285                solana_program_runtime::loaded_programs::DELAY_VISIBILITY_SLOT_OFFSET,
3286            ));
3287        let processing_environment = TransactionProcessingEnvironment {
3288            blockhash,
3289            blockhash_lamports_per_signature,
3290            epoch_total_stake: self.get_current_epoch_total_stake(),
3291            feature_set: self.feature_set.runtime_features(),
3292            program_runtime_environments_for_execution: self
3293                .transaction_processor
3294                .environments
3295                .clone(),
3296            program_runtime_environments_for_deployment: self
3297                .transaction_processor
3298                .get_environments_for_epoch(effective_epoch_of_deployments),
3299            rent: self.rent_collector.rent.clone(),
3300        };
3301
3302        let sanitized_output = self
3303            .transaction_processor
3304            .load_and_execute_sanitized_transactions(
3305                self,
3306                sanitized_txs,
3307                check_results,
3308                &processing_environment,
3309                &processing_config,
3310            );
3311
3312        // Accumulate the errors returned by the batch processor.
3313        error_counters.accumulate(&sanitized_output.error_metrics);
3314
3315        // Accumulate the transaction batch execution timings.
3316        timings.accumulate(&sanitized_output.execute_timings);
3317
3318        let ((), collect_logs_us) =
3319            measure_us!(self.collect_logs(sanitized_txs, &sanitized_output.processing_results));
3320        timings.saturating_add_in_place(ExecuteTimingType::CollectLogsUs, collect_logs_us);
3321
3322        let mut processed_counts = ProcessedTransactionCounts::default();
3323        let err_count = &mut error_counters.total;
3324
3325        for (processing_result, tx) in sanitized_output
3326            .processing_results
3327            .iter()
3328            .zip(sanitized_txs)
3329        {
3330            if let Some(debug_keys) = &self.transaction_debug_keys {
3331                for key in tx.account_keys().iter() {
3332                    if debug_keys.contains(key) {
3333                        let result = processing_result.flattened_result();
3334                        info!("slot: {} result: {:?} tx: {:?}", self.slot, result, tx);
3335                        break;
3336                    }
3337                }
3338            }
3339
3340            if processing_result.was_processed() {
3341                // Signature count must be accumulated only if the transaction
3342                // is processed, otherwise a mismatched count between banking
3343                // and replay could occur
3344                processed_counts.signature_count +=
3345                    tx.signature_details().num_transaction_signatures();
3346                processed_counts.processed_transactions_count += 1;
3347
3348                if !tx.is_simple_vote_transaction() {
3349                    processed_counts.processed_non_vote_transactions_count += 1;
3350                }
3351            }
3352
3353            match processing_result.flattened_result() {
3354                Ok(()) => {
3355                    processed_counts.processed_with_successful_result_count += 1;
3356                }
3357                Err(err) => {
3358                    if err_count.0 == 0 {
3359                        debug!("tx error: {err:?} {tx:?}");
3360                    }
3361                    *err_count += 1;
3362                }
3363            }
3364        }
3365
3366        LoadAndExecuteTransactionsOutput {
3367            processing_results: sanitized_output.processing_results,
3368            processed_counts,
3369            balance_collector: sanitized_output.balance_collector,
3370        }
3371    }
3372
3373    fn collect_logs(
3374        &self,
3375        transactions: &[impl TransactionWithMeta],
3376        processing_results: &[TransactionProcessingResult],
3377    ) {
3378        let transaction_log_collector_config =
3379            self.transaction_log_collector_config.read().unwrap();
3380        if transaction_log_collector_config.filter == TransactionLogCollectorFilter::None {
3381            return;
3382        }
3383
3384        let collected_logs: Vec<_> = processing_results
3385            .iter()
3386            .zip(transactions)
3387            .filter_map(|(processing_result, transaction)| {
3388                // Skip log collection for unprocessed transactions
3389                let processed_tx = processing_result.processed_transaction()?;
3390                // Skip log collection for unexecuted transactions
3391                let execution_details = processed_tx.execution_details()?;
3392                Self::collect_transaction_logs(
3393                    &transaction_log_collector_config,
3394                    transaction,
3395                    execution_details,
3396                )
3397            })
3398            .collect();
3399
3400        if !collected_logs.is_empty() {
3401            let mut transaction_log_collector = self.transaction_log_collector.write().unwrap();
3402            for (log, filtered_mentioned_addresses) in collected_logs {
3403                let transaction_log_index = transaction_log_collector.logs.len();
3404                transaction_log_collector.logs.push(log);
3405                for key in filtered_mentioned_addresses.into_iter() {
3406                    transaction_log_collector
3407                        .mentioned_address_map
3408                        .entry(key)
3409                        .or_default()
3410                        .push(transaction_log_index);
3411                }
3412            }
3413        }
3414    }
3415
3416    fn collect_transaction_logs(
3417        transaction_log_collector_config: &TransactionLogCollectorConfig,
3418        transaction: &impl TransactionWithMeta,
3419        execution_details: &TransactionExecutionDetails,
3420    ) -> Option<(TransactionLogInfo, Vec<Pubkey>)> {
3421        // Skip log collection if no log messages were recorded
3422        let log_messages = execution_details.log_messages.as_ref()?;
3423
3424        let mut filtered_mentioned_addresses = Vec::new();
3425        if !transaction_log_collector_config
3426            .mentioned_addresses
3427            .is_empty()
3428        {
3429            for key in transaction.account_keys().iter() {
3430                if transaction_log_collector_config
3431                    .mentioned_addresses
3432                    .contains(key)
3433                {
3434                    filtered_mentioned_addresses.push(*key);
3435                }
3436            }
3437        }
3438
3439        let is_vote = transaction.is_simple_vote_transaction();
3440        let store = match transaction_log_collector_config.filter {
3441            TransactionLogCollectorFilter::All => {
3442                !is_vote || !filtered_mentioned_addresses.is_empty()
3443            }
3444            TransactionLogCollectorFilter::AllWithVotes => true,
3445            TransactionLogCollectorFilter::None => false,
3446            TransactionLogCollectorFilter::OnlyMentionedAddresses => {
3447                !filtered_mentioned_addresses.is_empty()
3448            }
3449        };
3450
3451        if store {
3452            Some((
3453                TransactionLogInfo {
3454                    signature: *transaction.signature(),
3455                    result: execution_details.status.clone(),
3456                    is_vote,
3457                    log_messages: log_messages.clone(),
3458                },
3459                filtered_mentioned_addresses,
3460            ))
3461        } else {
3462            None
3463        }
3464    }
3465
3466    /// Load the accounts data size, in bytes
3467    pub fn load_accounts_data_size(&self) -> u64 {
3468        self.accounts_data_size_initial
3469            .saturating_add_signed(self.load_accounts_data_size_delta())
3470    }
3471
3472    /// Load the change in accounts data size in this Bank, in bytes
3473    pub fn load_accounts_data_size_delta(&self) -> i64 {
3474        let delta_on_chain = self.load_accounts_data_size_delta_on_chain();
3475        let delta_off_chain = self.load_accounts_data_size_delta_off_chain();
3476        delta_on_chain.saturating_add(delta_off_chain)
3477    }
3478
3479    /// Load the change in accounts data size in this Bank, in bytes, from on-chain events
3480    /// i.e. transactions
3481    pub fn load_accounts_data_size_delta_on_chain(&self) -> i64 {
3482        self.accounts_data_size_delta_on_chain.load(Acquire)
3483    }
3484
3485    /// Load the change in accounts data size in this Bank, in bytes, from off-chain events
3486    /// i.e. rent collection
3487    pub fn load_accounts_data_size_delta_off_chain(&self) -> i64 {
3488        self.accounts_data_size_delta_off_chain.load(Acquire)
3489    }
3490
3491    /// Update the accounts data size delta from on-chain events by adding `amount`.
3492    /// The arithmetic saturates.
3493    fn update_accounts_data_size_delta_on_chain(&self, amount: i64) {
3494        if amount == 0 {
3495            return;
3496        }
3497
3498        self.accounts_data_size_delta_on_chain
3499            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_on_chain| {
3500                Some(accounts_data_size_delta_on_chain.saturating_add(amount))
3501            })
3502            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3503            .unwrap();
3504    }
3505
3506    /// Update the accounts data size delta from off-chain events by adding `amount`.
3507    /// The arithmetic saturates.
3508    fn update_accounts_data_size_delta_off_chain(&self, amount: i64) {
3509        if amount == 0 {
3510            return;
3511        }
3512
3513        self.accounts_data_size_delta_off_chain
3514            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_off_chain| {
3515                Some(accounts_data_size_delta_off_chain.saturating_add(amount))
3516            })
3517            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3518            .unwrap();
3519    }
3520
3521    /// Calculate the data size delta and update the off-chain accounts data size delta
3522    fn calculate_and_update_accounts_data_size_delta_off_chain(
3523        &self,
3524        old_data_size: usize,
3525        new_data_size: usize,
3526    ) {
3527        let data_size_delta = calculate_data_size_delta(old_data_size, new_data_size);
3528        self.update_accounts_data_size_delta_off_chain(data_size_delta);
3529    }
3530
3531    fn filter_program_errors_and_collect_fee_details(
3532        &self,
3533        processing_results: &[TransactionProcessingResult],
3534    ) {
3535        let mut accumulated_fee_details = FeeDetails::default();
3536
3537        processing_results.iter().for_each(|processing_result| {
3538            if let Ok(processed_tx) = processing_result {
3539                accumulated_fee_details.accumulate(&processed_tx.fee_details());
3540            }
3541        });
3542
3543        self.collector_fee_details
3544            .write()
3545            .unwrap()
3546            .accumulate(&accumulated_fee_details);
3547    }
3548
3549    fn update_bank_hash_stats<'a>(&self, accounts: &impl StorableAccounts<'a>) {
3550        let mut stats = BankHashStats::default();
3551        (0..accounts.len()).for_each(|i| {
3552            accounts.account(i, |account| {
3553                stats.update(&account);
3554            })
3555        });
3556        self.bank_hash_stats.accumulate(&stats);
3557    }
3558
3559    pub fn commit_transactions(
3560        &self,
3561        sanitized_txs: &[impl TransactionWithMeta],
3562        processing_results: Vec<TransactionProcessingResult>,
3563        processed_counts: &ProcessedTransactionCounts,
3564        timings: &mut ExecuteTimings,
3565    ) -> Vec<TransactionCommitResult> {
3566        assert!(
3567            !self.freeze_started(),
3568            "commit_transactions() working on a bank that is already frozen or is undergoing \
3569             freezing!"
3570        );
3571
3572        let ProcessedTransactionCounts {
3573            processed_transactions_count,
3574            processed_non_vote_transactions_count,
3575            processed_with_successful_result_count,
3576            signature_count,
3577        } = *processed_counts;
3578
3579        self.increment_transaction_count(processed_transactions_count);
3580        self.increment_non_vote_transaction_count_since_restart(
3581            processed_non_vote_transactions_count,
3582        );
3583        self.increment_signature_count(signature_count);
3584
3585        let processed_with_failure_result_count =
3586            processed_transactions_count.saturating_sub(processed_with_successful_result_count);
3587        self.transaction_error_count
3588            .fetch_add(processed_with_failure_result_count, Relaxed);
3589
3590        if processed_transactions_count > 0 {
3591            self.is_delta.store(true, Relaxed);
3592            self.transaction_entries_count.fetch_add(1, Relaxed);
3593            self.transactions_per_entry_max
3594                .fetch_max(processed_transactions_count, Relaxed);
3595        }
3596
3597        let ((), store_accounts_us) = measure_us!({
3598            // If geyser is present, we must collect `SanitizedTransaction`
3599            // references in order to comply with that interface - until it
3600            // is changed.
3601            let maybe_transaction_refs = self
3602                .accounts()
3603                .accounts_db
3604                .has_accounts_update_notifier()
3605                .then(|| {
3606                    sanitized_txs
3607                        .iter()
3608                        .map(|tx| tx.as_sanitized_transaction())
3609                        .collect::<Vec<_>>()
3610                });
3611
3612            let (accounts_to_store, transactions) = collect_accounts_to_store(
3613                sanitized_txs,
3614                &maybe_transaction_refs,
3615                &processing_results,
3616            );
3617
3618            let to_store = (self.slot(), accounts_to_store.as_slice());
3619            self.update_bank_hash_stats(&to_store);
3620            // See https://github.com/solana-labs/solana/pull/31455 for discussion
3621            // on *not* updating the index within a threadpool.
3622            self.rc
3623                .accounts
3624                .store_accounts_seq(to_store, transactions.as_deref());
3625        });
3626
3627        // Cached vote and stake accounts are synchronized with accounts-db
3628        // after each transaction.
3629        let ((), update_stakes_cache_us) =
3630            measure_us!(self.update_stakes_cache(sanitized_txs, &processing_results));
3631
3632        let ((), update_executors_us) = measure_us!({
3633            let mut cache = None;
3634            for processing_result in &processing_results {
3635                if let Some(ProcessedTransaction::Executed(executed_tx)) =
3636                    processing_result.processed_transaction()
3637                {
3638                    let programs_modified_by_tx = &executed_tx.programs_modified_by_tx;
3639                    if executed_tx.was_successful() && !programs_modified_by_tx.is_empty() {
3640                        cache
3641                            .get_or_insert_with(|| {
3642                                self.transaction_processor
3643                                    .global_program_cache
3644                                    .write()
3645                                    .unwrap()
3646                            })
3647                            .merge(
3648                                &self.transaction_processor.environments,
3649                                programs_modified_by_tx,
3650                            );
3651                    }
3652                }
3653            }
3654        });
3655
3656        let accounts_data_len_delta = processing_results
3657            .iter()
3658            .filter_map(|processing_result| processing_result.processed_transaction())
3659            .filter_map(|processed_tx| processed_tx.execution_details())
3660            .filter_map(|details| {
3661                details
3662                    .status
3663                    .is_ok()
3664                    .then_some(details.accounts_data_len_delta)
3665            })
3666            .sum();
3667        self.update_accounts_data_size_delta_on_chain(accounts_data_len_delta);
3668
3669        let ((), update_transaction_statuses_us) =
3670            measure_us!(self.update_transaction_statuses(sanitized_txs, &processing_results));
3671
3672        self.filter_program_errors_and_collect_fee_details(&processing_results);
3673
3674        timings.saturating_add_in_place(ExecuteTimingType::StoreUs, store_accounts_us);
3675        timings.saturating_add_in_place(
3676            ExecuteTimingType::UpdateStakesCacheUs,
3677            update_stakes_cache_us,
3678        );
3679        timings.saturating_add_in_place(ExecuteTimingType::UpdateExecutorsUs, update_executors_us);
3680        timings.saturating_add_in_place(
3681            ExecuteTimingType::UpdateTransactionStatuses,
3682            update_transaction_statuses_us,
3683        );
3684
3685        Self::create_commit_results(processing_results)
3686    }
3687
3688    fn create_commit_results(
3689        processing_results: Vec<TransactionProcessingResult>,
3690    ) -> Vec<TransactionCommitResult> {
3691        processing_results
3692            .into_iter()
3693            .map(|processing_result| {
3694                let processing_result = processing_result?;
3695                let executed_units = processing_result.executed_units();
3696                let loaded_accounts_data_size = processing_result.loaded_accounts_data_size();
3697
3698                match processing_result {
3699                    ProcessedTransaction::Executed(executed_tx) => {
3700                        let successful = executed_tx.was_successful();
3701                        let execution_details = executed_tx.execution_details;
3702                        let LoadedTransaction {
3703                            accounts: loaded_accounts,
3704                            fee_details,
3705                            rollback_accounts,
3706                            ..
3707                        } = executed_tx.loaded_transaction;
3708
3709                        // Rollback value is used for failure.
3710                        let fee_payer_post_balance = if successful {
3711                            loaded_accounts[0].1.lamports()
3712                        } else {
3713                            rollback_accounts.fee_payer().1.lamports()
3714                        };
3715
3716                        Ok(CommittedTransaction {
3717                            status: execution_details.status,
3718                            log_messages: execution_details.log_messages,
3719                            inner_instructions: execution_details.inner_instructions,
3720                            return_data: execution_details.return_data,
3721                            executed_units,
3722                            fee_details,
3723                            loaded_account_stats: TransactionLoadedAccountsStats {
3724                                loaded_accounts_count: loaded_accounts.len(),
3725                                loaded_accounts_data_size,
3726                            },
3727                            fee_payer_post_balance,
3728                        })
3729                    }
3730                    ProcessedTransaction::FeesOnly(fees_only_tx) => Ok(CommittedTransaction {
3731                        status: Err(fees_only_tx.load_error),
3732                        log_messages: None,
3733                        inner_instructions: None,
3734                        return_data: None,
3735                        executed_units,
3736                        fee_details: fees_only_tx.fee_details,
3737                        loaded_account_stats: TransactionLoadedAccountsStats {
3738                            loaded_accounts_count: fees_only_tx.rollback_accounts.count(),
3739                            loaded_accounts_data_size,
3740                        },
3741                        fee_payer_post_balance: fees_only_tx
3742                            .rollback_accounts
3743                            .fee_payer()
3744                            .1
3745                            .lamports(),
3746                    }),
3747                }
3748            })
3749            .collect()
3750    }
3751
3752    fn run_incinerator(&self) {
3753        if let Some((account, _)) =
3754            self.get_account_modified_since_parent_with_fixed_root(&incinerator::id())
3755        {
3756            self.capitalization.fetch_sub(account.lamports(), Relaxed);
3757            self.store_account(&incinerator::id(), &AccountSharedData::default());
3758        }
3759    }
3760
3761    /// Returns the accounts, sorted by pubkey, that were part of accounts lt hash calculation
3762    /// This is used when writing a bank hash details file.
3763    pub(crate) fn get_accounts_for_bank_hash_details(&self) -> Vec<(Pubkey, AccountSharedData)> {
3764        let mut accounts = self
3765            .rc
3766            .accounts
3767            .accounts_db
3768            .get_pubkey_account_for_slot(self.slot());
3769        // Sort the accounts by pubkey to make diff deterministic.
3770        accounts.sort_unstable_by(|a, b| a.0.cmp(&b.0));
3771        accounts
3772    }
3773
3774    pub fn cluster_type(&self) -> ClusterType {
3775        // unwrap is safe; self.cluster_type is ensured to be Some() always...
3776        // we only using Option here for ABI compatibility...
3777        self.cluster_type.unwrap()
3778    }
3779
3780    /// Process a batch of transactions.
3781    #[must_use]
3782    pub fn load_execute_and_commit_transactions(
3783        &self,
3784        batch: &TransactionBatch<impl TransactionWithMeta>,
3785        max_age: usize,
3786        recording_config: ExecutionRecordingConfig,
3787        timings: &mut ExecuteTimings,
3788        log_messages_bytes_limit: Option<usize>,
3789    ) -> (Vec<TransactionCommitResult>, Option<BalanceCollector>) {
3790        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
3791            batch,
3792            max_age,
3793            recording_config,
3794            timings,
3795            log_messages_bytes_limit,
3796            None::<fn(&mut _, &_) -> _>,
3797        )
3798        .unwrap()
3799    }
3800
3801    pub fn load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
3802        &'a self,
3803        batch: &TransactionBatch<impl TransactionWithMeta>,
3804        max_age: usize,
3805        recording_config: ExecutionRecordingConfig,
3806        timings: &mut ExecuteTimings,
3807        log_messages_bytes_limit: Option<usize>,
3808        pre_commit_callback: impl FnOnce(
3809            &mut ExecuteTimings,
3810            &[TransactionProcessingResult],
3811        ) -> PreCommitResult<'a>,
3812    ) -> Result<(Vec<TransactionCommitResult>, Option<BalanceCollector>)> {
3813        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
3814            batch,
3815            max_age,
3816            recording_config,
3817            timings,
3818            log_messages_bytes_limit,
3819            Some(pre_commit_callback),
3820        )
3821    }
3822
3823    fn do_load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
3824        &'a self,
3825        batch: &TransactionBatch<impl TransactionWithMeta>,
3826        max_age: usize,
3827        recording_config: ExecutionRecordingConfig,
3828        timings: &mut ExecuteTimings,
3829        log_messages_bytes_limit: Option<usize>,
3830        pre_commit_callback: Option<
3831            impl FnOnce(&mut ExecuteTimings, &[TransactionProcessingResult]) -> PreCommitResult<'a>,
3832        >,
3833    ) -> Result<(Vec<TransactionCommitResult>, Option<BalanceCollector>)> {
3834        let LoadAndExecuteTransactionsOutput {
3835            processing_results,
3836            processed_counts,
3837            balance_collector,
3838        } = self.load_and_execute_transactions(
3839            batch,
3840            max_age,
3841            timings,
3842            &mut TransactionErrorMetrics::default(),
3843            TransactionProcessingConfig {
3844                account_overrides: None,
3845                check_program_modification_slot: self.check_program_modification_slot,
3846                log_messages_bytes_limit,
3847                limit_to_load_programs: false,
3848                recording_config,
3849            },
3850        );
3851
3852        // pre_commit_callback could initiate an atomic operation (i.e. poh recording with block
3853        // producing unified scheduler). in that case, it returns Some(freeze_lock), which should
3854        // unlocked only after calling commit_transactions() immediately after calling the
3855        // callback.
3856        let freeze_lock = if let Some(pre_commit_callback) = pre_commit_callback {
3857            pre_commit_callback(timings, &processing_results)?
3858        } else {
3859            None
3860        };
3861        let commit_results = self.commit_transactions(
3862            batch.sanitized_transactions(),
3863            processing_results,
3864            &processed_counts,
3865            timings,
3866        );
3867        drop(freeze_lock);
3868        Ok((commit_results, balance_collector))
3869    }
3870
3871    /// Process a Transaction. This is used for unit tests and simply calls the vector
3872    /// Bank::process_transactions method.
3873    pub fn process_transaction(&self, tx: &Transaction) -> Result<()> {
3874        self.try_process_transactions(std::iter::once(tx))?[0].clone()
3875    }
3876
3877    /// Process a Transaction and store metadata. This is used for tests and the banks services. It
3878    /// replicates the vector Bank::process_transaction method with metadata recording enabled.
3879    pub fn process_transaction_with_metadata(
3880        &self,
3881        tx: impl Into<VersionedTransaction>,
3882    ) -> Result<CommittedTransaction> {
3883        let txs = vec![tx.into()];
3884        let batch = self.prepare_entry_batch(txs)?;
3885
3886        let (mut commit_results, ..) = self.load_execute_and_commit_transactions(
3887            &batch,
3888            MAX_PROCESSING_AGE,
3889            ExecutionRecordingConfig {
3890                enable_cpi_recording: false,
3891                enable_log_recording: true,
3892                enable_return_data_recording: true,
3893                enable_transaction_balance_recording: false,
3894            },
3895            &mut ExecuteTimings::default(),
3896            Some(1000 * 1000),
3897        );
3898
3899        commit_results.remove(0)
3900    }
3901
3902    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
3903    /// Short circuits if any of the transactions do not pass sanitization checks.
3904    pub fn try_process_transactions<'a>(
3905        &self,
3906        txs: impl Iterator<Item = &'a Transaction>,
3907    ) -> Result<Vec<Result<()>>> {
3908        let txs = txs
3909            .map(|tx| VersionedTransaction::from(tx.clone()))
3910            .collect();
3911        self.try_process_entry_transactions(txs)
3912    }
3913
3914    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
3915    /// Short circuits if any of the transactions do not pass sanitization checks.
3916    pub fn try_process_entry_transactions(
3917        &self,
3918        txs: Vec<VersionedTransaction>,
3919    ) -> Result<Vec<Result<()>>> {
3920        let batch = self.prepare_entry_batch(txs)?;
3921        Ok(self.process_transaction_batch(&batch))
3922    }
3923
3924    #[must_use]
3925    fn process_transaction_batch(
3926        &self,
3927        batch: &TransactionBatch<impl TransactionWithMeta>,
3928    ) -> Vec<Result<()>> {
3929        self.load_execute_and_commit_transactions(
3930            batch,
3931            MAX_PROCESSING_AGE,
3932            ExecutionRecordingConfig::new_single_setting(false),
3933            &mut ExecuteTimings::default(),
3934            None,
3935        )
3936        .0
3937        .into_iter()
3938        .map(|commit_result| commit_result.and_then(|committed_tx| committed_tx.status))
3939        .collect()
3940    }
3941
3942    /// Create, sign, and process a Transaction from `keypair` to `to` of
3943    /// `n` lamports where `blockhash` is the last Entry ID observed by the client.
3944    pub fn transfer(&self, n: u64, keypair: &Keypair, to: &Pubkey) -> Result<Signature> {
3945        let blockhash = self.last_blockhash();
3946        let tx = system_transaction::transfer(keypair, to, n, blockhash);
3947        let signature = tx.signatures[0];
3948        self.process_transaction(&tx).map(|_| signature)
3949    }
3950
3951    pub fn read_balance(account: &AccountSharedData) -> u64 {
3952        account.lamports()
3953    }
3954    /// Each program would need to be able to introspect its own state
3955    /// this is hard-coded to the Budget language
3956    pub fn get_balance(&self, pubkey: &Pubkey) -> u64 {
3957        self.get_account(pubkey)
3958            .map(|x| Self::read_balance(&x))
3959            .unwrap_or(0)
3960    }
3961
3962    /// Compute all the parents of the bank in order
3963    pub fn parents(&self) -> Vec<Arc<Bank>> {
3964        let mut parents = vec![];
3965        let mut bank = self.parent();
3966        while let Some(parent) = bank {
3967            parents.push(parent.clone());
3968            bank = parent.parent();
3969        }
3970        parents
3971    }
3972
3973    /// Compute all the parents of the bank including this bank itself
3974    pub fn parents_inclusive(self: Arc<Self>) -> Vec<Arc<Bank>> {
3975        let mut parents = self.parents();
3976        parents.insert(0, self);
3977        parents
3978    }
3979
3980    /// fn store the single `account` with `pubkey`.
3981    /// Uses `store_accounts`, which works on a vector of accounts.
3982    pub fn store_account(&self, pubkey: &Pubkey, account: &AccountSharedData) {
3983        self.store_accounts((self.slot(), &[(pubkey, account)][..]))
3984    }
3985
3986    pub fn store_accounts<'a>(&self, accounts: impl StorableAccounts<'a>) {
3987        assert!(!self.freeze_started());
3988        let mut m = Measure::start("stakes_cache.check_and_store");
3989        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
3990
3991        (0..accounts.len()).for_each(|i| {
3992            accounts.account(i, |account| {
3993                self.stakes_cache.check_and_store(
3994                    account.pubkey(),
3995                    &account,
3996                    new_warmup_cooldown_rate_epoch,
3997                )
3998            })
3999        });
4000        self.update_bank_hash_stats(&accounts);
4001        self.rc.accounts.store_accounts_par(accounts, None);
4002        m.stop();
4003        self.rc
4004            .accounts
4005            .accounts_db
4006            .stats
4007            .stakes_cache_check_and_store_us
4008            .fetch_add(m.as_us(), Relaxed);
4009    }
4010
4011    pub fn force_flush_accounts_cache(&self) {
4012        self.rc
4013            .accounts
4014            .accounts_db
4015            .flush_accounts_cache(true, Some(self.slot()))
4016    }
4017
4018    pub fn flush_accounts_cache_if_needed(&self) {
4019        self.rc
4020            .accounts
4021            .accounts_db
4022            .flush_accounts_cache(false, Some(self.slot()))
4023    }
4024
4025    /// Technically this issues (or even burns!) new lamports,
4026    /// so be extra careful for its usage
4027    fn store_account_and_update_capitalization(
4028        &self,
4029        pubkey: &Pubkey,
4030        new_account: &AccountSharedData,
4031    ) {
4032        let old_account_data_size = if let Some(old_account) =
4033            self.get_account_with_fixed_root_no_cache(pubkey)
4034        {
4035            match new_account.lamports().cmp(&old_account.lamports()) {
4036                std::cmp::Ordering::Greater => {
4037                    let diff = new_account.lamports() - old_account.lamports();
4038                    trace!("store_account_and_update_capitalization: increased: {pubkey} {diff}");
4039                    self.capitalization.fetch_add(diff, Relaxed);
4040                }
4041                std::cmp::Ordering::Less => {
4042                    let diff = old_account.lamports() - new_account.lamports();
4043                    trace!("store_account_and_update_capitalization: decreased: {pubkey} {diff}");
4044                    self.capitalization.fetch_sub(diff, Relaxed);
4045                }
4046                std::cmp::Ordering::Equal => {}
4047            }
4048            old_account.data().len()
4049        } else {
4050            trace!(
4051                "store_account_and_update_capitalization: created: {pubkey} {}",
4052                new_account.lamports()
4053            );
4054            self.capitalization
4055                .fetch_add(new_account.lamports(), Relaxed);
4056            0
4057        };
4058
4059        self.store_account(pubkey, new_account);
4060        self.calculate_and_update_accounts_data_size_delta_off_chain(
4061            old_account_data_size,
4062            new_account.data().len(),
4063        );
4064    }
4065
4066    pub fn accounts(&self) -> Arc<Accounts> {
4067        self.rc.accounts.clone()
4068    }
4069
4070    fn apply_simd_0306_cost_tracker_changes(&mut self) {
4071        let mut cost_tracker = self.write_cost_tracker().unwrap();
4072        let block_cost_limit = cost_tracker.get_block_limit();
4073        let vote_cost_limit = cost_tracker.get_vote_limit();
4074        // SIMD-0306 makes account cost limit 40% of the block cost limit.
4075        let account_cost_limit = block_cost_limit.saturating_mul(40).saturating_div(100);
4076        cost_tracker.set_limits(account_cost_limit, block_cost_limit, vote_cost_limit);
4077    }
4078
4079    fn apply_simd_0339_invoke_cost_changes(&mut self) {
4080        let simd_0268_active = self
4081            .feature_set
4082            .is_active(&raise_cpi_nesting_limit_to_8::id());
4083        let simd_0339_active = self
4084            .feature_set
4085            .is_active(&increase_cpi_account_info_limit::id());
4086        let compute_budget = self
4087            .compute_budget()
4088            .as_ref()
4089            .unwrap_or(&ComputeBudget::new_with_defaults(
4090                simd_0268_active,
4091                simd_0339_active,
4092            ))
4093            .to_cost();
4094
4095        self.transaction_processor
4096            .set_execution_cost(compute_budget);
4097    }
4098
4099    /// This is called from genesis and snapshot restore
4100    fn apply_activated_features(&mut self) {
4101        // Update active set of reserved account keys which are not allowed to be write locked
4102        self.reserved_account_keys = {
4103            let mut reserved_keys = ReservedAccountKeys::clone(&self.reserved_account_keys);
4104            reserved_keys.update_active_set(&self.feature_set);
4105            Arc::new(reserved_keys)
4106        };
4107
4108        // Update the transaction processor with all active built-in programs
4109        self.add_active_builtin_programs();
4110
4111        // Cost-Tracker is not serialized in snapshot or any configs.
4112        // We must apply previously activated features related to limits here
4113        // so that the initial bank state is consistent with the feature set.
4114        // Cost-tracker limits are propagated through children banks.
4115        if self
4116            .feature_set
4117            .is_active(&feature_set::raise_block_limits_to_100m::id())
4118        {
4119            let block_cost_limit = simd_0286_block_limits();
4120            let mut cost_tracker = self.write_cost_tracker().unwrap();
4121            let account_cost_limit = cost_tracker.get_account_limit();
4122            let vote_cost_limit = cost_tracker.get_vote_limit();
4123            cost_tracker.set_limits(account_cost_limit, block_cost_limit, vote_cost_limit);
4124        }
4125
4126        if self
4127            .feature_set
4128            .is_active(&feature_set::raise_account_cu_limit::id())
4129        {
4130            self.apply_simd_0306_cost_tracker_changes();
4131        }
4132
4133        if self
4134            .feature_set
4135            .is_active(&feature_set::increase_cpi_account_info_limit::id())
4136        {
4137            self.apply_simd_0339_invoke_cost_changes();
4138        }
4139
4140        let environments = self.create_program_runtime_environments(&self.feature_set);
4141        self.transaction_processor
4142            .global_program_cache
4143            .write()
4144            .unwrap()
4145            .latest_root_slot = self.slot;
4146        self.transaction_processor
4147            .epoch_boundary_preparation
4148            .write()
4149            .unwrap()
4150            .upcoming_epoch = self.epoch;
4151        self.transaction_processor.environments = environments;
4152    }
4153
4154    fn create_program_runtime_environments(
4155        &self,
4156        feature_set: &FeatureSet,
4157    ) -> ProgramRuntimeEnvironments {
4158        let simd_0268_active = feature_set.is_active(&raise_cpi_nesting_limit_to_8::id());
4159        let simd_0339_active = feature_set.is_active(&increase_cpi_account_info_limit::id());
4160        let compute_budget = self
4161            .compute_budget()
4162            .as_ref()
4163            .unwrap_or(&ComputeBudget::new_with_defaults(
4164                simd_0268_active,
4165                simd_0339_active,
4166            ))
4167            .to_budget();
4168        ProgramRuntimeEnvironments {
4169            program_runtime_v1: Arc::new(
4170                create_program_runtime_environment_v1(
4171                    &feature_set.runtime_features(),
4172                    &compute_budget,
4173                    false, /* deployment */
4174                    false, /* debugging_features */
4175                )
4176                .unwrap(),
4177            ),
4178            program_runtime_v2: Arc::new(create_program_runtime_environment_v2(
4179                &compute_budget,
4180                false, /* debugging_features */
4181            )),
4182        }
4183    }
4184
4185    pub fn set_tick_height(&self, tick_height: u64) {
4186        self.tick_height.store(tick_height, Relaxed)
4187    }
4188
4189    pub fn set_inflation(&self, inflation: Inflation) {
4190        *self.inflation.write().unwrap() = inflation;
4191    }
4192
4193    /// Get a snapshot of the current set of hard forks
4194    pub fn hard_forks(&self) -> HardForks {
4195        self.hard_forks.read().unwrap().clone()
4196    }
4197
4198    pub fn register_hard_fork(&self, new_hard_fork_slot: Slot) {
4199        let bank_slot = self.slot();
4200
4201        let lock = self.freeze_lock();
4202        let bank_frozen = *lock != Hash::default();
4203        if new_hard_fork_slot < bank_slot {
4204            warn!(
4205                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is older than the \
4206                 bank at slot {bank_slot} that attempted to register it."
4207            );
4208        } else if (new_hard_fork_slot == bank_slot) && bank_frozen {
4209            warn!(
4210                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is the same slot \
4211                 as the bank at slot {bank_slot} that attempted to register it, but that bank is \
4212                 already frozen."
4213            );
4214        } else {
4215            self.hard_forks
4216                .write()
4217                .unwrap()
4218                .register(new_hard_fork_slot);
4219        }
4220    }
4221
4222    pub fn get_account_with_fixed_root_no_cache(
4223        &self,
4224        pubkey: &Pubkey,
4225    ) -> Option<AccountSharedData> {
4226        self.load_account_with(pubkey, false)
4227            .map(|(acc, _slot)| acc)
4228    }
4229
4230    fn load_account_with(
4231        &self,
4232        pubkey: &Pubkey,
4233        should_put_in_read_cache: bool,
4234    ) -> Option<(AccountSharedData, Slot)> {
4235        self.rc.accounts.accounts_db.load_account_with(
4236            &self.ancestors,
4237            pubkey,
4238            should_put_in_read_cache,
4239        )
4240    }
4241
4242    // Hi! leaky abstraction here....
4243    // try to use get_account_with_fixed_root() if it's called ONLY from on-chain runtime account
4244    // processing. That alternative fn provides more safety.
4245    pub fn get_account(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
4246        self.get_account_modified_slot(pubkey)
4247            .map(|(acc, _slot)| acc)
4248    }
4249
4250    // Hi! leaky abstraction here....
4251    // use this over get_account() if it's called ONLY from on-chain runtime account
4252    // processing (i.e. from in-band replay/banking stage; that ensures root is *fixed* while
4253    // running).
4254    // pro: safer assertion can be enabled inside AccountsDb
4255    // con: panics!() if called from off-chain processing
4256    pub fn get_account_with_fixed_root(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
4257        self.get_account_modified_slot_with_fixed_root(pubkey)
4258            .map(|(acc, _slot)| acc)
4259    }
4260
4261    // See note above get_account_with_fixed_root() about when to prefer this function
4262    pub fn get_account_modified_slot_with_fixed_root(
4263        &self,
4264        pubkey: &Pubkey,
4265    ) -> Option<(AccountSharedData, Slot)> {
4266        self.load_slow_with_fixed_root(&self.ancestors, pubkey)
4267    }
4268
4269    pub fn get_account_modified_slot(&self, pubkey: &Pubkey) -> Option<(AccountSharedData, Slot)> {
4270        self.load_slow(&self.ancestors, pubkey)
4271    }
4272
4273    fn load_slow(
4274        &self,
4275        ancestors: &Ancestors,
4276        pubkey: &Pubkey,
4277    ) -> Option<(AccountSharedData, Slot)> {
4278        // get_account (= primary this fn caller) may be called from on-chain Bank code even if we
4279        // try hard to use get_account_with_fixed_root for that purpose...
4280        // so pass safer LoadHint:Unspecified here as a fallback
4281        self.rc.accounts.load_without_fixed_root(ancestors, pubkey)
4282    }
4283
4284    fn load_slow_with_fixed_root(
4285        &self,
4286        ancestors: &Ancestors,
4287        pubkey: &Pubkey,
4288    ) -> Option<(AccountSharedData, Slot)> {
4289        self.rc.accounts.load_with_fixed_root(ancestors, pubkey)
4290    }
4291
4292    pub fn get_program_accounts(
4293        &self,
4294        program_id: &Pubkey,
4295        config: &ScanConfig,
4296    ) -> ScanResult<Vec<KeyedAccountSharedData>> {
4297        self.rc
4298            .accounts
4299            .load_by_program(&self.ancestors, self.bank_id, program_id, config)
4300    }
4301
4302    pub fn get_filtered_program_accounts<F: Fn(&AccountSharedData) -> bool>(
4303        &self,
4304        program_id: &Pubkey,
4305        filter: F,
4306        config: &ScanConfig,
4307    ) -> ScanResult<Vec<KeyedAccountSharedData>> {
4308        self.rc.accounts.load_by_program_with_filter(
4309            &self.ancestors,
4310            self.bank_id,
4311            program_id,
4312            filter,
4313            config,
4314        )
4315    }
4316
4317    pub fn get_filtered_indexed_accounts<F: Fn(&AccountSharedData) -> bool>(
4318        &self,
4319        index_key: &IndexKey,
4320        filter: F,
4321        config: &ScanConfig,
4322        byte_limit_for_scan: Option<usize>,
4323    ) -> ScanResult<Vec<KeyedAccountSharedData>> {
4324        self.rc.accounts.load_by_index_key_with_filter(
4325            &self.ancestors,
4326            self.bank_id,
4327            index_key,
4328            filter,
4329            config,
4330            byte_limit_for_scan,
4331        )
4332    }
4333
4334    pub fn account_indexes_include_key(&self, key: &Pubkey) -> bool {
4335        self.rc.accounts.account_indexes_include_key(key)
4336    }
4337
4338    /// Returns all the accounts this bank can load
4339    pub fn get_all_accounts(&self, sort_results: bool) -> ScanResult<Vec<PubkeyAccountSlot>> {
4340        self.rc
4341            .accounts
4342            .load_all(&self.ancestors, self.bank_id, sort_results)
4343    }
4344
4345    // Scans all the accounts this bank can load, applying `scan_func`
4346    pub fn scan_all_accounts<F>(&self, scan_func: F, sort_results: bool) -> ScanResult<()>
4347    where
4348        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4349    {
4350        self.rc
4351            .accounts
4352            .scan_all(&self.ancestors, self.bank_id, scan_func, sort_results)
4353    }
4354
4355    pub fn get_program_accounts_modified_since_parent(
4356        &self,
4357        program_id: &Pubkey,
4358    ) -> Vec<KeyedAccountSharedData> {
4359        self.rc
4360            .accounts
4361            .load_by_program_slot(self.slot(), Some(program_id))
4362    }
4363
4364    pub fn get_transaction_logs(
4365        &self,
4366        address: Option<&Pubkey>,
4367    ) -> Option<Vec<TransactionLogInfo>> {
4368        self.transaction_log_collector
4369            .read()
4370            .unwrap()
4371            .get_logs_for_address(address)
4372    }
4373
4374    /// Returns all the accounts stored in this slot
4375    pub fn get_all_accounts_modified_since_parent(&self) -> Vec<KeyedAccountSharedData> {
4376        self.rc.accounts.load_by_program_slot(self.slot(), None)
4377    }
4378
4379    // if you want get_account_modified_since_parent without fixed_root, please define so...
4380    fn get_account_modified_since_parent_with_fixed_root(
4381        &self,
4382        pubkey: &Pubkey,
4383    ) -> Option<(AccountSharedData, Slot)> {
4384        let just_self: Ancestors = Ancestors::from(vec![self.slot()]);
4385        if let Some((account, slot)) = self.load_slow_with_fixed_root(&just_self, pubkey) {
4386            if slot == self.slot() {
4387                return Some((account, slot));
4388            }
4389        }
4390        None
4391    }
4392
4393    pub fn get_largest_accounts(
4394        &self,
4395        num: usize,
4396        filter_by_address: &HashSet<Pubkey>,
4397        filter: AccountAddressFilter,
4398        sort_results: bool,
4399    ) -> ScanResult<Vec<(Pubkey, u64)>> {
4400        self.rc.accounts.load_largest_accounts(
4401            &self.ancestors,
4402            self.bank_id,
4403            num,
4404            filter_by_address,
4405            filter,
4406            sort_results,
4407        )
4408    }
4409
4410    /// Return the accumulated executed transaction count
4411    pub fn transaction_count(&self) -> u64 {
4412        self.transaction_count.load(Relaxed)
4413    }
4414
4415    /// Returns the number of non-vote transactions processed without error
4416    /// since the most recent boot from snapshot or genesis.
4417    /// This value is not shared though the network, nor retained
4418    /// within snapshots, but is preserved in `Bank::new_from_parent`.
4419    pub fn non_vote_transaction_count_since_restart(&self) -> u64 {
4420        self.non_vote_transaction_count_since_restart.load(Relaxed)
4421    }
4422
4423    /// Return the transaction count executed only in this bank
4424    pub fn executed_transaction_count(&self) -> u64 {
4425        self.transaction_count()
4426            .saturating_sub(self.parent().map_or(0, |parent| parent.transaction_count()))
4427    }
4428
4429    pub fn transaction_error_count(&self) -> u64 {
4430        self.transaction_error_count.load(Relaxed)
4431    }
4432
4433    pub fn transaction_entries_count(&self) -> u64 {
4434        self.transaction_entries_count.load(Relaxed)
4435    }
4436
4437    pub fn transactions_per_entry_max(&self) -> u64 {
4438        self.transactions_per_entry_max.load(Relaxed)
4439    }
4440
4441    fn increment_transaction_count(&self, tx_count: u64) {
4442        self.transaction_count.fetch_add(tx_count, Relaxed);
4443    }
4444
4445    fn increment_non_vote_transaction_count_since_restart(&self, tx_count: u64) {
4446        self.non_vote_transaction_count_since_restart
4447            .fetch_add(tx_count, Relaxed);
4448    }
4449
4450    pub fn signature_count(&self) -> u64 {
4451        self.signature_count.load(Relaxed)
4452    }
4453
4454    fn increment_signature_count(&self, signature_count: u64) {
4455        self.signature_count.fetch_add(signature_count, Relaxed);
4456    }
4457
4458    pub fn get_signature_status_processed_since_parent(
4459        &self,
4460        signature: &Signature,
4461    ) -> Option<Result<()>> {
4462        if let Some((slot, status)) = self.get_signature_status_slot(signature) {
4463            if slot <= self.slot() {
4464                return Some(status);
4465            }
4466        }
4467        None
4468    }
4469
4470    pub fn get_signature_status_with_blockhash(
4471        &self,
4472        signature: &Signature,
4473        blockhash: &Hash,
4474    ) -> Option<Result<()>> {
4475        let rcache = self.status_cache.read().unwrap();
4476        rcache
4477            .get_status(signature, blockhash, &self.ancestors)
4478            .map(|v| v.1)
4479    }
4480
4481    pub fn get_committed_transaction_status_and_slot(
4482        &self,
4483        message_hash: &Hash,
4484        transaction_blockhash: &Hash,
4485    ) -> Option<(Slot, bool)> {
4486        let rcache = self.status_cache.read().unwrap();
4487        rcache
4488            .get_status(message_hash, transaction_blockhash, &self.ancestors)
4489            .map(|(slot, status)| (slot, status.is_ok()))
4490    }
4491
4492    pub fn get_signature_status_slot(&self, signature: &Signature) -> Option<(Slot, Result<()>)> {
4493        let rcache = self.status_cache.read().unwrap();
4494        rcache.get_status_any_blockhash(signature, &self.ancestors)
4495    }
4496
4497    pub fn get_signature_status(&self, signature: &Signature) -> Option<Result<()>> {
4498        self.get_signature_status_slot(signature).map(|v| v.1)
4499    }
4500
4501    pub fn has_signature(&self, signature: &Signature) -> bool {
4502        self.get_signature_status_slot(signature).is_some()
4503    }
4504
4505    /// Hash the `accounts` HashMap. This represents a validator's interpretation
4506    ///  of the delta of the ledger since the last vote and up to now
4507    fn hash_internal_state(&self) -> Hash {
4508        let measure_total = Measure::start("");
4509        let slot = self.slot();
4510
4511        let mut hash = hashv(&[
4512            self.parent_hash.as_ref(),
4513            &self.signature_count().to_le_bytes(),
4514            self.last_blockhash().as_ref(),
4515        ]);
4516
4517        let accounts_lt_hash_checksum = {
4518            let accounts_lt_hash = &*self.accounts_lt_hash.lock().unwrap();
4519            let lt_hash_bytes = bytemuck::must_cast_slice(&accounts_lt_hash.0 .0);
4520            hash = hashv(&[hash.as_ref(), lt_hash_bytes]);
4521            accounts_lt_hash.0.checksum()
4522        };
4523
4524        let buf = self
4525            .hard_forks
4526            .read()
4527            .unwrap()
4528            .get_hash_data(slot, self.parent_slot());
4529        if let Some(buf) = buf {
4530            let hard_forked_hash = hashv(&[hash.as_ref(), &buf]);
4531            warn!("hard fork at slot {slot} by hashing {buf:?}: {hash} => {hard_forked_hash}");
4532            hash = hard_forked_hash;
4533        }
4534
4535        #[cfg(feature = "dev-context-only-utils")]
4536        let hash_override = self
4537            .hash_overrides
4538            .lock()
4539            .unwrap()
4540            .get_bank_hash_override(slot)
4541            .copied()
4542            .inspect(|&hash_override| {
4543                if hash_override != hash {
4544                    info!(
4545                        "bank: slot: {}: overrode bank hash: {} with {}",
4546                        self.slot(),
4547                        hash,
4548                        hash_override
4549                    );
4550                }
4551            });
4552        // Avoid to optimize out `hash` along with the whole computation by super smart rustc.
4553        // hash_override is used by ledger-tool's simulate-block-production, which prefers
4554        // the actual bank freezing processing for accurate simulation.
4555        #[cfg(feature = "dev-context-only-utils")]
4556        let hash = hash_override.unwrap_or(std::hint::black_box(hash));
4557
4558        let bank_hash_stats = self.bank_hash_stats.load();
4559
4560        let total_us = measure_total.end_as_us();
4561
4562        datapoint_info!(
4563            "bank-hash_internal_state",
4564            ("slot", slot, i64),
4565            ("total_us", total_us, i64),
4566        );
4567        info!(
4568            "bank frozen: {slot} hash: {hash} signature_count: {} last_blockhash: {} \
4569             capitalization: {}, accounts_lt_hash checksum: {accounts_lt_hash_checksum}, stats: \
4570             {bank_hash_stats:?}",
4571            self.signature_count(),
4572            self.last_blockhash(),
4573            self.capitalization(),
4574        );
4575        hash
4576    }
4577
4578    pub fn collector_fees(&self) -> u64 {
4579        self.collector_fees.load(Relaxed)
4580    }
4581
4582    /// Used by ledger tool to run a final hash calculation once all ledger replay has completed.
4583    /// This should not be called by validator code.
4584    pub fn run_final_hash_calc(&self) {
4585        self.force_flush_accounts_cache();
4586        // note that this slot may not be a root
4587        _ = self.verify_accounts(
4588            VerifyAccountsHashConfig {
4589                require_rooted_bank: false,
4590            },
4591            None,
4592        );
4593    }
4594
4595    /// Verify the account state as part of startup, typically from a snapshot.
4596    ///
4597    /// This fn compares the calculated accounts lt hash against the stored value in the bank.
4598    ///
4599    /// Normal validator operation will calculate the accounts lt hash during index generation.
4600    /// Tests/ledger-tool may not have the calculated value from index generation (or the bank
4601    /// being verified is different from the snapshot/startup bank), and thus will be calculated in
4602    /// this function, using the accounts index for input, running in the foreground.
4603    ///
4604    /// Returns true if all is good.
4605    ///
4606    /// Only intended to be called at startup, or from tests/ledger-tool.
4607    #[must_use]
4608    fn verify_accounts(
4609        &self,
4610        config: VerifyAccountsHashConfig,
4611        calculated_accounts_lt_hash: Option<&AccountsLtHash>,
4612    ) -> bool {
4613        let accounts_db = &self.rc.accounts.accounts_db;
4614
4615        let slot = self.slot();
4616
4617        if config.require_rooted_bank && !accounts_db.accounts_index.is_alive_root(slot) {
4618            if let Some(parent) = self.parent() {
4619                info!(
4620                    "slot {slot} is not a root, so verify accounts hash on parent bank at slot {}",
4621                    parent.slot(),
4622                );
4623                // The calculated_accounts_lt_hash parameter is only valid for the current slot, so
4624                // we must fall back to calculating the accounts lt hash with the index.
4625                return parent.verify_accounts(config, None);
4626            } else {
4627                // this will result in mismatch errors
4628                // accounts hash calc doesn't include unrooted slots
4629                panic!("cannot verify accounts hash because slot {slot} is not a root");
4630            }
4631        }
4632
4633        fn check_lt_hash(
4634            expected_accounts_lt_hash: &AccountsLtHash,
4635            calculated_accounts_lt_hash: &AccountsLtHash,
4636        ) -> bool {
4637            let is_ok = calculated_accounts_lt_hash == expected_accounts_lt_hash;
4638            if !is_ok {
4639                let expected = expected_accounts_lt_hash.0.checksum();
4640                let calculated = calculated_accounts_lt_hash.0.checksum();
4641                error!(
4642                    "Verifying accounts failed: accounts lattice hashes do not match, expected: \
4643                     {expected}, calculated: {calculated}",
4644                );
4645            }
4646            is_ok
4647        }
4648
4649        info!("Verifying accounts...");
4650        let start = Instant::now();
4651        let expected_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap().clone();
4652        let is_ok = if let Some(calculated_accounts_lt_hash) = calculated_accounts_lt_hash {
4653            check_lt_hash(&expected_accounts_lt_hash, calculated_accounts_lt_hash)
4654        } else {
4655            let calculated_accounts_lt_hash =
4656                accounts_db.calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, slot);
4657            check_lt_hash(&expected_accounts_lt_hash, &calculated_accounts_lt_hash)
4658        };
4659        info!("Verifying accounts... Done in {:?}", start.elapsed());
4660        is_ok
4661    }
4662
4663    /// Get this bank's storages to use for snapshots.
4664    ///
4665    /// If a base slot is provided, return only the storages that are *higher* than this slot.
4666    pub fn get_snapshot_storages(&self, base_slot: Option<Slot>) -> Vec<Arc<AccountStorageEntry>> {
4667        // if a base slot is provided, request storages starting at the slot *after*
4668        let start_slot = base_slot.map_or(0, |slot| slot.saturating_add(1));
4669        // we want to *include* the storage at our slot
4670        let requested_slots = start_slot..=self.slot();
4671
4672        self.rc.accounts.accounts_db.get_storages(requested_slots).0
4673    }
4674
4675    #[must_use]
4676    fn verify_hash(&self) -> bool {
4677        assert!(self.is_frozen());
4678        let calculated_hash = self.hash_internal_state();
4679        let expected_hash = self.hash();
4680
4681        if calculated_hash == expected_hash {
4682            true
4683        } else {
4684            warn!(
4685                "verify failed: slot: {}, {} (calculated) != {} (expected)",
4686                self.slot(),
4687                calculated_hash,
4688                expected_hash
4689            );
4690            false
4691        }
4692    }
4693
4694    pub fn verify_transaction(
4695        &self,
4696        tx: VersionedTransaction,
4697        verification_mode: TransactionVerificationMode,
4698    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
4699        let enable_static_instruction_limit = self
4700            .feature_set
4701            .is_active(&agave_feature_set::static_instruction_limit::id());
4702        let sanitized_tx = {
4703            let size =
4704                bincode::serialized_size(&tx).map_err(|_| TransactionError::SanitizeFailure)?;
4705            if size > PACKET_DATA_SIZE as u64 {
4706                return Err(TransactionError::SanitizeFailure);
4707            }
4708            let message_hash = if verification_mode == TransactionVerificationMode::FullVerification
4709            {
4710                // SIMD-0160, check instruction limit before signature verificaton
4711                if enable_static_instruction_limit
4712                    && tx.message.instructions().len()
4713                        > solana_transaction_context::MAX_INSTRUCTION_TRACE_LENGTH
4714                {
4715                    return Err(solana_transaction_error::TransactionError::SanitizeFailure);
4716                }
4717                tx.verify_and_hash_message()?
4718            } else {
4719                tx.message.hash()
4720            };
4721
4722            RuntimeTransaction::try_create(
4723                tx,
4724                MessageHash::Precomputed(message_hash),
4725                None,
4726                self,
4727                self.get_reserved_account_keys(),
4728                enable_static_instruction_limit,
4729            )
4730        }?;
4731
4732        Ok(sanitized_tx)
4733    }
4734
4735    pub fn fully_verify_transaction(
4736        &self,
4737        tx: VersionedTransaction,
4738    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
4739        self.verify_transaction(tx, TransactionVerificationMode::FullVerification)
4740    }
4741
4742    /// Checks if the transaction violates the bank's reserved keys.
4743    /// This needs to be checked upon epoch boundary crosses because the
4744    /// reserved key set may have changed since the initial sanitization.
4745    pub fn check_reserved_keys(&self, tx: &impl SVMMessage) -> Result<()> {
4746        // Check keys against the reserved set - these failures simply require us
4747        // to re-sanitize the transaction. We do not need to drop the transaction.
4748        let reserved_keys = self.get_reserved_account_keys();
4749        for (index, key) in tx.account_keys().iter().enumerate() {
4750            if tx.is_writable(index) && reserved_keys.contains(key) {
4751                return Err(TransactionError::ResanitizationNeeded);
4752            }
4753        }
4754
4755        Ok(())
4756    }
4757
4758    /// Calculates and returns the capitalization.
4759    ///
4760    /// Panics if capitalization overflows a u64.
4761    ///
4762    /// Note, this is *very* expensive!  It walks the whole accounts index,
4763    /// account-by-account, summing each account's balance.
4764    ///
4765    /// Only intended to be called at startup by ledger-tool or tests.
4766    /// (cannot be made DCOU due to solana-program-test)
4767    pub fn calculate_capitalization_for_tests(&self) -> u64 {
4768        self.rc
4769            .accounts
4770            .accounts_db
4771            .calculate_capitalization_at_startup_from_index(&self.ancestors, self.slot())
4772    }
4773
4774    /// Sets the capitalization.
4775    ///
4776    /// Only intended to be called by ledger-tool or tests.
4777    /// (cannot be made DCOU due to solana-program-test)
4778    pub fn set_capitalization_for_tests(&self, capitalization: u64) {
4779        self.capitalization.store(capitalization, Relaxed);
4780    }
4781
4782    /// Returns the `SnapshotHash` for this bank's slot
4783    ///
4784    /// This fn is used at startup to verify the bank was rebuilt correctly.
4785    pub fn get_snapshot_hash(&self) -> SnapshotHash {
4786        SnapshotHash::new(self.accounts_lt_hash.lock().unwrap().0.checksum())
4787    }
4788
4789    pub fn load_account_into_read_cache(&self, key: &Pubkey) {
4790        self.rc
4791            .accounts
4792            .accounts_db
4793            .load_account_into_read_cache(&self.ancestors, key);
4794    }
4795
4796    /// A snapshot bank should be purged of 0 lamport accounts which are not part of the hash
4797    /// calculation and could shield other real accounts.
4798    pub fn verify_snapshot_bank(
4799        &self,
4800        skip_shrink: bool,
4801        force_clean: bool,
4802        latest_full_snapshot_slot: Slot,
4803        calculated_accounts_lt_hash: Option<&AccountsLtHash>,
4804    ) -> bool {
4805        let (verified_accounts, verify_accounts_time_us) = measure_us!({
4806            let should_verify_accounts = !self.rc.accounts.accounts_db.skip_initial_hash_calc;
4807            if should_verify_accounts {
4808                self.verify_accounts(
4809                    VerifyAccountsHashConfig {
4810                        require_rooted_bank: false,
4811                    },
4812                    calculated_accounts_lt_hash,
4813                )
4814            } else {
4815                info!("Verifying accounts... Skipped.");
4816                true
4817            }
4818        });
4819
4820        let (_, clean_time_us) = measure_us!({
4821            let should_clean = force_clean || (!skip_shrink && self.slot() > 0);
4822            if should_clean {
4823                info!("Cleaning...");
4824                // We cannot clean past the latest full snapshot's slot because we are about to
4825                // perform an accounts hash calculation *up to that slot*.  If we cleaned *past*
4826                // that slot, then accounts could be removed from older storages, which would
4827                // change the accounts hash.
4828                self.rc.accounts.accounts_db.clean_accounts(
4829                    Some(latest_full_snapshot_slot),
4830                    true,
4831                    self.epoch_schedule(),
4832                );
4833                info!("Cleaning... Done.");
4834            } else {
4835                info!("Cleaning... Skipped.");
4836            }
4837        });
4838
4839        let (_, shrink_time_us) = measure_us!({
4840            let should_shrink = !skip_shrink && self.slot() > 0;
4841            if should_shrink {
4842                info!("Shrinking...");
4843                self.rc.accounts.accounts_db.shrink_all_slots(
4844                    true,
4845                    self.epoch_schedule(),
4846                    // we cannot allow the snapshot slot to be shrunk
4847                    Some(self.slot()),
4848                );
4849                info!("Shrinking... Done.");
4850            } else {
4851                info!("Shrinking... Skipped.");
4852            }
4853        });
4854
4855        info!("Verifying bank...");
4856        let (verified_bank, verify_bank_time_us) = measure_us!(self.verify_hash());
4857        info!("Verifying bank... Done.");
4858
4859        datapoint_info!(
4860            "verify_snapshot_bank",
4861            ("clean_us", clean_time_us, i64),
4862            ("shrink_us", shrink_time_us, i64),
4863            ("verify_accounts_us", verify_accounts_time_us, i64),
4864            ("verify_bank_us", verify_bank_time_us, i64),
4865        );
4866
4867        verified_accounts && verified_bank
4868    }
4869
4870    /// Return the number of hashes per tick
4871    pub fn hashes_per_tick(&self) -> &Option<u64> {
4872        &self.hashes_per_tick
4873    }
4874
4875    /// Return the number of ticks per slot
4876    pub fn ticks_per_slot(&self) -> u64 {
4877        self.ticks_per_slot
4878    }
4879
4880    /// Return the number of slots per year
4881    pub fn slots_per_year(&self) -> f64 {
4882        self.slots_per_year
4883    }
4884
4885    /// Return the number of ticks since genesis.
4886    pub fn tick_height(&self) -> u64 {
4887        self.tick_height.load(Relaxed)
4888    }
4889
4890    /// Return the inflation parameters of the Bank
4891    pub fn inflation(&self) -> Inflation {
4892        *self.inflation.read().unwrap()
4893    }
4894
4895    /// Return the rent collector for this Bank
4896    pub fn rent_collector(&self) -> &RentCollector {
4897        &self.rent_collector
4898    }
4899
4900    /// Return the total capitalization of the Bank
4901    pub fn capitalization(&self) -> u64 {
4902        self.capitalization.load(Relaxed)
4903    }
4904
4905    /// Return this bank's max_tick_height
4906    pub fn max_tick_height(&self) -> u64 {
4907        self.max_tick_height
4908    }
4909
4910    /// Return the block_height of this bank
4911    pub fn block_height(&self) -> u64 {
4912        self.block_height
4913    }
4914
4915    /// Return the number of slots per epoch for the given epoch
4916    pub fn get_slots_in_epoch(&self, epoch: Epoch) -> u64 {
4917        self.epoch_schedule().get_slots_in_epoch(epoch)
4918    }
4919
4920    /// returns the epoch for which this bank's leader_schedule_slot_offset and slot would
4921    ///  need to cache leader_schedule
4922    pub fn get_leader_schedule_epoch(&self, slot: Slot) -> Epoch {
4923        self.epoch_schedule().get_leader_schedule_epoch(slot)
4924    }
4925
4926    /// Returns whether the specified epoch should use the new vote account
4927    /// keyed leader schedule
4928    pub fn should_use_vote_keyed_leader_schedule(&self, epoch: Epoch) -> Option<bool> {
4929        let effective_epoch = self
4930            .feature_set
4931            .activated_slot(&agave_feature_set::enable_vote_address_leader_schedule::id())
4932            .map(|activation_slot| {
4933                // If the feature was activated at genesis, then the new leader
4934                // schedule should be effective immediately in the first epoch
4935                if activation_slot == 0 {
4936                    return 0;
4937                }
4938
4939                // Calculate the epoch that the feature became activated in
4940                let activation_epoch = self.epoch_schedule.get_epoch(activation_slot);
4941
4942                // The effective epoch is the epoch immediately after the
4943                // activation epoch
4944                activation_epoch.wrapping_add(1)
4945            });
4946
4947        // Starting from the effective epoch, always use the new leader schedule
4948        if let Some(effective_epoch) = effective_epoch {
4949            return Some(epoch >= effective_epoch);
4950        }
4951
4952        // Calculate the max epoch we can cache a leader schedule for
4953        let max_cached_leader_schedule = self.get_leader_schedule_epoch(self.slot());
4954        if epoch <= max_cached_leader_schedule {
4955            // The feature cannot be effective by the specified epoch
4956            Some(false)
4957        } else {
4958            // Cannot determine if an epoch should use the new leader schedule if the
4959            // the epoch is too far in the future because we won't know if the feature
4960            // will have been activated by then or not.
4961            None
4962        }
4963    }
4964
4965    /// a bank-level cache of vote accounts and stake delegation info
4966    fn update_stakes_cache(
4967        &self,
4968        txs: &[impl SVMMessage],
4969        processing_results: &[TransactionProcessingResult],
4970    ) {
4971        debug_assert_eq!(txs.len(), processing_results.len());
4972        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
4973        txs.iter()
4974            .zip(processing_results)
4975            .filter_map(|(tx, processing_result)| {
4976                processing_result
4977                    .processed_transaction()
4978                    .map(|processed_tx| (tx, processed_tx))
4979            })
4980            .filter_map(|(tx, processed_tx)| {
4981                processed_tx
4982                    .executed_transaction()
4983                    .map(|executed_tx| (tx, executed_tx))
4984            })
4985            .filter(|(_, executed_tx)| executed_tx.was_successful())
4986            .flat_map(|(tx, executed_tx)| {
4987                let num_account_keys = tx.account_keys().len();
4988                let loaded_tx = &executed_tx.loaded_transaction;
4989                loaded_tx.accounts.iter().take(num_account_keys)
4990            })
4991            .for_each(|(pubkey, account)| {
4992                // note that this could get timed to: self.rc.accounts.accounts_db.stats.stakes_cache_check_and_store_us,
4993                //  but this code path is captured separately in ExecuteTimingType::UpdateStakesCacheUs
4994                self.stakes_cache
4995                    .check_and_store(pubkey, account, new_warmup_cooldown_rate_epoch);
4996            });
4997    }
4998
4999    /// current vote accounts for this bank along with the stake
5000    ///   attributed to each account
5001    pub fn vote_accounts(&self) -> Arc<VoteAccountsHashMap> {
5002        let stakes = self.stakes_cache.stakes();
5003        Arc::from(stakes.vote_accounts())
5004    }
5005
5006    /// Vote account for the given vote account pubkey.
5007    pub fn get_vote_account(&self, vote_account: &Pubkey) -> Option<VoteAccount> {
5008        let stakes = self.stakes_cache.stakes();
5009        let vote_account = stakes.vote_accounts().get(vote_account)?;
5010        Some(vote_account.clone())
5011    }
5012
5013    /// Get the EpochStakes for the current Bank::epoch
5014    pub fn current_epoch_stakes(&self) -> &VersionedEpochStakes {
5015        // The stakes for a given epoch (E) in self.epoch_stakes are keyed by leader schedule epoch
5016        // (E + 1) so the stakes for the current epoch are stored at self.epoch_stakes[E + 1]
5017        self.epoch_stakes
5018            .get(&self.epoch.saturating_add(1))
5019            .expect("Current epoch stakes must exist")
5020    }
5021
5022    /// Get the EpochStakes for a given epoch
5023    pub fn epoch_stakes(&self, epoch: Epoch) -> Option<&VersionedEpochStakes> {
5024        self.epoch_stakes.get(&epoch)
5025    }
5026
5027    pub fn epoch_stakes_map(&self) -> &HashMap<Epoch, VersionedEpochStakes> {
5028        &self.epoch_stakes
5029    }
5030
5031    /// Get the staked nodes map for the current Bank::epoch
5032    pub fn current_epoch_staked_nodes(&self) -> Arc<HashMap<Pubkey, u64>> {
5033        self.current_epoch_stakes().stakes().staked_nodes()
5034    }
5035
5036    pub fn epoch_staked_nodes(&self, epoch: Epoch) -> Option<Arc<HashMap<Pubkey, u64>>> {
5037        Some(self.epoch_stakes.get(&epoch)?.stakes().staked_nodes())
5038    }
5039
5040    /// Get the total epoch stake for the given epoch.
5041    pub fn epoch_total_stake(&self, epoch: Epoch) -> Option<u64> {
5042        self.epoch_stakes
5043            .get(&epoch)
5044            .map(|epoch_stakes| epoch_stakes.total_stake())
5045    }
5046
5047    /// Get the total epoch stake for the current Bank::epoch
5048    pub fn get_current_epoch_total_stake(&self) -> u64 {
5049        self.current_epoch_stakes().total_stake()
5050    }
5051
5052    /// vote accounts for the specific epoch along with the stake
5053    ///   attributed to each account
5054    pub fn epoch_vote_accounts(&self, epoch: Epoch) -> Option<&VoteAccountsHashMap> {
5055        let epoch_stakes = self.epoch_stakes.get(&epoch)?.stakes();
5056        Some(epoch_stakes.vote_accounts().as_ref())
5057    }
5058
5059    /// Get the vote accounts along with the stake attributed to each account
5060    /// for the current Bank::epoch
5061    pub fn get_current_epoch_vote_accounts(&self) -> &VoteAccountsHashMap {
5062        self.current_epoch_stakes()
5063            .stakes()
5064            .vote_accounts()
5065            .as_ref()
5066    }
5067
5068    /// Get the fixed authorized voter for the given vote account for the
5069    /// current epoch
5070    pub fn epoch_authorized_voter(&self, vote_account: &Pubkey) -> Option<&Pubkey> {
5071        self.epoch_stakes
5072            .get(&self.epoch)
5073            .expect("Epoch stakes for bank's own epoch must exist")
5074            .epoch_authorized_voters()
5075            .get(vote_account)
5076    }
5077
5078    /// Get the fixed set of vote accounts for the given node id for the
5079    /// current epoch
5080    pub fn epoch_vote_accounts_for_node_id(&self, node_id: &Pubkey) -> Option<&NodeVoteAccounts> {
5081        self.epoch_stakes
5082            .get(&self.epoch)
5083            .expect("Epoch stakes for bank's own epoch must exist")
5084            .node_id_to_vote_accounts()
5085            .get(node_id)
5086    }
5087
5088    /// Get the total stake belonging to vote accounts associated with the given node id for the
5089    /// given epoch.
5090    pub fn epoch_node_id_to_stake(&self, epoch: Epoch, node_id: &Pubkey) -> Option<u64> {
5091        self.epoch_stakes(epoch)
5092            .and_then(|epoch_stakes| epoch_stakes.node_id_to_stake(node_id))
5093    }
5094
5095    /// Get the fixed total stake of all vote accounts for current epoch
5096    pub fn total_epoch_stake(&self) -> u64 {
5097        self.epoch_stakes
5098            .get(&self.epoch)
5099            .expect("Epoch stakes for bank's own epoch must exist")
5100            .total_stake()
5101    }
5102
5103    /// Get the fixed stake of the given vote account for the current epoch
5104    pub fn epoch_vote_account_stake(&self, vote_account: &Pubkey) -> u64 {
5105        *self
5106            .epoch_vote_accounts(self.epoch())
5107            .expect("Bank epoch vote accounts must contain entry for the bank's own epoch")
5108            .get(vote_account)
5109            .map(|(stake, _)| stake)
5110            .unwrap_or(&0)
5111    }
5112
5113    /// given a slot, return the epoch and offset into the epoch this slot falls
5114    /// e.g. with a fixed number for slots_per_epoch, the calculation is simply:
5115    ///
5116    ///  ( slot/slots_per_epoch, slot % slots_per_epoch )
5117    ///
5118    pub fn get_epoch_and_slot_index(&self, slot: Slot) -> (Epoch, SlotIndex) {
5119        self.epoch_schedule().get_epoch_and_slot_index(slot)
5120    }
5121
5122    pub fn get_epoch_info(&self) -> EpochInfo {
5123        let absolute_slot = self.slot();
5124        let block_height = self.block_height();
5125        let (epoch, slot_index) = self.get_epoch_and_slot_index(absolute_slot);
5126        let slots_in_epoch = self.get_slots_in_epoch(epoch);
5127        let transaction_count = Some(self.transaction_count());
5128        EpochInfo {
5129            epoch,
5130            slot_index,
5131            slots_in_epoch,
5132            absolute_slot,
5133            block_height,
5134            transaction_count,
5135        }
5136    }
5137
5138    pub fn is_empty(&self) -> bool {
5139        !self.is_delta.load(Relaxed)
5140    }
5141
5142    pub fn add_mockup_builtin(
5143        &mut self,
5144        program_id: Pubkey,
5145        builtin_function: BuiltinFunctionWithContext,
5146    ) {
5147        self.add_builtin(
5148            program_id,
5149            "mockup",
5150            ProgramCacheEntry::new_builtin(self.slot, 0, builtin_function),
5151        );
5152    }
5153
5154    pub fn add_precompile(&mut self, program_id: &Pubkey) {
5155        debug!("Adding precompiled program {program_id}");
5156        self.add_precompiled_account(program_id);
5157        debug!("Added precompiled program {program_id:?}");
5158    }
5159
5160    // Call AccountsDb::clean_accounts()
5161    //
5162    // This fn is meant to be called by the snapshot handler in Accounts Background Service.  If
5163    // calling from elsewhere, ensure the same invariants hold/expectations are met.
5164    pub(crate) fn clean_accounts(&self) {
5165        // Don't clean the slot we're snapshotting because it may have zero-lamport
5166        // accounts that were included in the bank delta hash when the bank was frozen,
5167        // and if we clean them here, any newly created snapshot's hash for this bank
5168        // may not match the frozen hash.
5169        //
5170        // So when we're snapshotting, the highest slot to clean is lowered by one.
5171        let highest_slot_to_clean = self.slot().saturating_sub(1);
5172
5173        self.rc.accounts.accounts_db.clean_accounts(
5174            Some(highest_slot_to_clean),
5175            false,
5176            self.epoch_schedule(),
5177        );
5178    }
5179
5180    pub fn print_accounts_stats(&self) {
5181        self.rc.accounts.accounts_db.print_accounts_stats("");
5182    }
5183
5184    pub fn shrink_candidate_slots(&self) -> usize {
5185        self.rc
5186            .accounts
5187            .accounts_db
5188            .shrink_candidate_slots(self.epoch_schedule())
5189    }
5190
5191    pub(crate) fn shrink_ancient_slots(&self) {
5192        self.rc
5193            .accounts
5194            .accounts_db
5195            .shrink_ancient_slots(self.epoch_schedule())
5196    }
5197
5198    pub fn read_cost_tracker(&self) -> LockResult<RwLockReadGuard<'_, CostTracker>> {
5199        self.cost_tracker.read()
5200    }
5201
5202    pub fn write_cost_tracker(&self) -> LockResult<RwLockWriteGuard<'_, CostTracker>> {
5203        self.cost_tracker.write()
5204    }
5205
5206    // Check if the wallclock time from bank creation to now has exceeded the allotted
5207    // time for transaction processing
5208    pub fn should_bank_still_be_processing_txs(
5209        bank_creation_time: &Instant,
5210        max_tx_ingestion_nanos: u128,
5211    ) -> bool {
5212        // Do this check outside of the PoH lock, hence not a method on PohRecorder
5213        bank_creation_time.elapsed().as_nanos() <= max_tx_ingestion_nanos
5214    }
5215
5216    pub fn deactivate_feature(&mut self, id: &Pubkey) {
5217        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
5218        feature_set.active_mut().remove(id);
5219        feature_set.inactive_mut().insert(*id);
5220        self.feature_set = Arc::new(feature_set);
5221    }
5222
5223    pub fn activate_feature(&mut self, id: &Pubkey) {
5224        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
5225        feature_set.inactive_mut().remove(id);
5226        feature_set.active_mut().insert(*id, 0);
5227        self.feature_set = Arc::new(feature_set);
5228    }
5229
5230    pub fn fill_bank_with_ticks_for_tests(&self) {
5231        self.do_fill_bank_with_ticks_for_tests(&BankWithScheduler::no_scheduler_available())
5232    }
5233
5234    pub(crate) fn do_fill_bank_with_ticks_for_tests(&self, scheduler: &InstalledSchedulerRwLock) {
5235        if self.tick_height.load(Relaxed) < self.max_tick_height {
5236            let last_blockhash = self.last_blockhash();
5237            while self.last_blockhash() == last_blockhash {
5238                self.register_tick(&Hash::new_unique(), scheduler)
5239            }
5240        } else {
5241            warn!("Bank already reached max tick height, cannot fill it with more ticks");
5242        }
5243    }
5244
5245    /// Get a set of all actively reserved account keys that are not allowed to
5246    /// be write-locked during transaction processing.
5247    pub fn get_reserved_account_keys(&self) -> &HashSet<Pubkey> {
5248        &self.reserved_account_keys.active
5249    }
5250
5251    /// Compute and apply all activated features, initialize the transaction
5252    /// processor, and recalculate partitioned rewards if needed
5253    fn initialize_after_snapshot_restore<F, TP>(&mut self, rewards_thread_pool_builder: F)
5254    where
5255        F: FnOnce() -> TP,
5256        TP: std::borrow::Borrow<ThreadPool>,
5257    {
5258        self.transaction_processor =
5259            TransactionBatchProcessor::new_uninitialized(self.slot, self.epoch);
5260        if let Some(compute_budget) = &self.compute_budget {
5261            self.transaction_processor
5262                .set_execution_cost(compute_budget.to_cost());
5263        }
5264
5265        self.compute_and_apply_features_after_snapshot_restore();
5266
5267        self.recalculate_partitioned_rewards_if_active(rewards_thread_pool_builder);
5268
5269        self.transaction_processor
5270            .fill_missing_sysvar_cache_entries(self);
5271    }
5272
5273    /// Compute and apply all activated features and also add accounts for builtins
5274    fn compute_and_apply_genesis_features(&mut self) {
5275        // Update the feature set to include all features active at this slot
5276        let feature_set = self.compute_active_feature_set(false).0;
5277        self.feature_set = Arc::new(feature_set);
5278
5279        // Add built-in program accounts to the bank if they don't already exist
5280        self.add_builtin_program_accounts();
5281
5282        self.apply_activated_features();
5283    }
5284
5285    /// Compute and apply all activated features but do not add built-in
5286    /// accounts because we shouldn't modify accounts db for a completed bank
5287    fn compute_and_apply_features_after_snapshot_restore(&mut self) {
5288        // Update the feature set to include all features active at this slot
5289        let feature_set = self.compute_active_feature_set(false).0;
5290        self.feature_set = Arc::new(feature_set);
5291
5292        self.apply_activated_features();
5293    }
5294
5295    /// This is called from each epoch boundary
5296    fn compute_and_apply_new_feature_activations(&mut self) {
5297        let include_pending = true;
5298        let (feature_set, new_feature_activations) =
5299            self.compute_active_feature_set(include_pending);
5300        self.feature_set = Arc::new(feature_set);
5301
5302        // Update activation slot of features in `new_feature_activations`
5303        for feature_id in new_feature_activations.iter() {
5304            if let Some(mut account) = self.get_account_with_fixed_root(feature_id) {
5305                if let Some(mut feature) = feature::state::from_account(&account) {
5306                    feature.activated_at = Some(self.slot());
5307                    if feature::state::to_account(&feature, &mut account).is_some() {
5308                        self.store_account(feature_id, &account);
5309                    }
5310                    info!("Feature {} activated at slot {}", feature_id, self.slot());
5311                }
5312            }
5313        }
5314
5315        // Update active set of reserved account keys which are not allowed to be write locked
5316        self.reserved_account_keys = {
5317            let mut reserved_keys = ReservedAccountKeys::clone(&self.reserved_account_keys);
5318            reserved_keys.update_active_set(&self.feature_set);
5319            Arc::new(reserved_keys)
5320        };
5321
5322        if new_feature_activations.contains(&feature_set::deprecate_rent_exemption_threshold::id())
5323        {
5324            self.rent_collector.rent.lamports_per_byte_year =
5325                (self.rent_collector.rent.lamports_per_byte_year as f64
5326                    * self.rent_collector.rent.exemption_threshold) as u64;
5327            self.rent_collector.rent.exemption_threshold = 1.0;
5328            self.update_rent();
5329        }
5330
5331        if new_feature_activations.contains(&feature_set::pico_inflation::id()) {
5332            *self.inflation.write().unwrap() = Inflation::pico();
5333            self.fee_rate_governor.burn_percent = solana_fee_calculator::DEFAULT_BURN_PERCENT; // 50% fee burn
5334            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
5335        }
5336
5337        if !new_feature_activations.is_disjoint(&self.feature_set.full_inflation_features_enabled())
5338        {
5339            *self.inflation.write().unwrap() = Inflation::full();
5340            self.fee_rate_governor.burn_percent = solana_fee_calculator::DEFAULT_BURN_PERCENT; // 50% fee burn
5341            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
5342        }
5343
5344        self.apply_new_builtin_program_feature_transitions(&new_feature_activations);
5345
5346        if new_feature_activations.contains(&feature_set::raise_block_limits_to_100m::id()) {
5347            let block_cost_limit = simd_0286_block_limits();
5348            let mut cost_tracker = self.write_cost_tracker().unwrap();
5349            let account_cost_limit = cost_tracker.get_account_limit();
5350            let vote_cost_limit = cost_tracker.get_vote_limit();
5351            cost_tracker.set_limits(account_cost_limit, block_cost_limit, vote_cost_limit);
5352            drop(cost_tracker);
5353
5354            if self
5355                .feature_set
5356                .is_active(&feature_set::raise_account_cu_limit::id())
5357            {
5358                self.apply_simd_0306_cost_tracker_changes();
5359            }
5360        }
5361
5362        if new_feature_activations.contains(&feature_set::raise_account_cu_limit::id()) {
5363            self.apply_simd_0306_cost_tracker_changes();
5364        }
5365
5366        if new_feature_activations.contains(&feature_set::vote_state_v4::id()) {
5367            if let Err(e) = self.upgrade_core_bpf_program(
5368                &solana_sdk_ids::stake::id(),
5369                &feature_set::vote_state_v4::stake_program_buffer::id(),
5370                "upgrade_stake_program_for_vote_state_v4",
5371            ) {
5372                error!("Failed to upgrade Core BPF Stake program: {e}");
5373            }
5374        }
5375        if new_feature_activations.contains(&feature_set::increase_cpi_account_info_limit::id()) {
5376            self.apply_simd_0339_invoke_cost_changes();
5377        }
5378
5379        if new_feature_activations.contains(&feature_set::replace_spl_token_with_p_token::id()) {
5380            if let Err(e) = self.upgrade_loader_v2_program_with_loader_v3_program(
5381                &feature_set::replace_spl_token_with_p_token::SPL_TOKEN_PROGRAM_ID,
5382                &feature_set::replace_spl_token_with_p_token::PTOKEN_PROGRAM_BUFFER,
5383                "replace_spl_token_with_p_token",
5384            ) {
5385                warn!(
5386                    "Failed to replace SPL Token with p-token buffer '{}': {e}",
5387                    feature_set::replace_spl_token_with_p_token::PTOKEN_PROGRAM_BUFFER,
5388                );
5389            }
5390        }
5391    }
5392
5393    fn apply_new_builtin_program_feature_transitions(
5394        &mut self,
5395        new_feature_activations: &AHashSet<Pubkey>,
5396    ) {
5397        for builtin in BUILTINS.iter() {
5398            if let Some(feature_id) = builtin.enable_feature_id {
5399                if new_feature_activations.contains(&feature_id) {
5400                    self.add_builtin(
5401                        builtin.program_id,
5402                        builtin.name,
5403                        ProgramCacheEntry::new_builtin(
5404                            self.feature_set.activated_slot(&feature_id).unwrap_or(0),
5405                            builtin.name.len(),
5406                            builtin.entrypoint,
5407                        ),
5408                    );
5409                }
5410            }
5411
5412            if let Some(core_bpf_migration_config) = &builtin.core_bpf_migration_config {
5413                // If the builtin is set to be migrated to Core BPF on feature
5414                // activation, perform the migration which will remove it from
5415                // the builtins list and the cache.
5416                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
5417                    if let Err(e) = self
5418                        .migrate_builtin_to_core_bpf(&builtin.program_id, core_bpf_migration_config)
5419                    {
5420                        warn!(
5421                            "Failed to migrate builtin {} to Core BPF: {}",
5422                            builtin.name, e
5423                        );
5424                    }
5425                }
5426            };
5427        }
5428
5429        // Migrate any necessary stateless builtins to core BPF.
5430        // Stateless builtins do not have an `enable_feature_id` since they
5431        // do not exist on-chain.
5432        for stateless_builtin in STATELESS_BUILTINS.iter() {
5433            if let Some(core_bpf_migration_config) = &stateless_builtin.core_bpf_migration_config {
5434                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
5435                    if let Err(e) = self.migrate_builtin_to_core_bpf(
5436                        &stateless_builtin.program_id,
5437                        core_bpf_migration_config,
5438                    ) {
5439                        warn!(
5440                            "Failed to migrate stateless builtin {} to Core BPF: {}",
5441                            stateless_builtin.name, e
5442                        );
5443                    }
5444                }
5445            }
5446        }
5447
5448        for precompile in get_precompiles() {
5449            if let Some(feature_id) = &precompile.feature {
5450                if new_feature_activations.contains(feature_id) {
5451                    self.add_precompile(&precompile.program_id);
5452                }
5453            }
5454        }
5455    }
5456
5457    fn adjust_sysvar_balance_for_rent(&self, account: &mut AccountSharedData) {
5458        account.set_lamports(
5459            self.get_minimum_balance_for_rent_exemption(account.data().len())
5460                .max(account.lamports()),
5461        );
5462    }
5463
5464    /// Compute the active feature set based on the current bank state,
5465    /// and return it together with the set of newly activated features.
5466    fn compute_active_feature_set(&self, include_pending: bool) -> (FeatureSet, AHashSet<Pubkey>) {
5467        let mut active = self.feature_set.active().clone();
5468        let mut inactive = AHashSet::new();
5469        let mut pending = AHashSet::new();
5470        let slot = self.slot();
5471
5472        for feature_id in self.feature_set.inactive() {
5473            let mut activated = None;
5474            if let Some(account) = self.get_account_with_fixed_root(feature_id) {
5475                if let Some(feature) = feature::state::from_account(&account) {
5476                    match feature.activated_at {
5477                        None if include_pending => {
5478                            // Feature activation is pending
5479                            pending.insert(*feature_id);
5480                            activated = Some(slot);
5481                        }
5482                        Some(activation_slot) if slot >= activation_slot => {
5483                            // Feature has been activated already
5484                            activated = Some(activation_slot);
5485                        }
5486                        _ => {}
5487                    }
5488                }
5489            }
5490            if let Some(slot) = activated {
5491                active.insert(*feature_id, slot);
5492            } else {
5493                inactive.insert(*feature_id);
5494            }
5495        }
5496
5497        (FeatureSet::new(active, inactive), pending)
5498    }
5499
5500    /// If `feature_id` is pending to be activated at the next epoch boundary, return
5501    /// the first slot at which it will be active (the epoch boundary).
5502    pub fn compute_pending_activation_slot(&self, feature_id: &Pubkey) -> Option<Slot> {
5503        let account = self.get_account_with_fixed_root(feature_id)?;
5504        let feature = feature::from_account(&account)?;
5505        if feature.activated_at.is_some() {
5506            // Feature is already active
5507            return None;
5508        }
5509        // Feature will be active at the next epoch boundary
5510        let active_epoch = self.epoch + 1;
5511        Some(self.epoch_schedule.get_first_slot_in_epoch(active_epoch))
5512    }
5513
5514    fn add_active_builtin_programs(&mut self) {
5515        for builtin in BUILTINS.iter() {
5516            // The `builtin_is_bpf` flag is used to handle the case where a
5517            // builtin is scheduled to be enabled by one feature gate and
5518            // later migrated to Core BPF by another.
5519            //
5520            // There should never be a case where a builtin is set to be
5521            // migrated to Core BPF and is also set to be enabled on feature
5522            // activation on the same feature gate. However, the
5523            // `builtin_is_bpf` flag will handle this case as well, electing
5524            // to first attempt the migration to Core BPF.
5525            //
5526            // The migration to Core BPF will fail gracefully because the
5527            // program account will not exist. The builtin will subsequently
5528            // be enabled, but it will never be migrated to Core BPF.
5529            //
5530            // Using the same feature gate for both enabling and migrating a
5531            // builtin to Core BPF should be strictly avoided.
5532            let builtin_is_bpf = builtin.core_bpf_migration_config.is_some() && {
5533                self.get_account(&builtin.program_id)
5534                    .map(|a| a.owner() == &bpf_loader_upgradeable::id())
5535                    .unwrap_or(false)
5536            };
5537
5538            // If the builtin has already been migrated to Core BPF, do not
5539            // add it to the bank's builtins.
5540            if builtin_is_bpf {
5541                continue;
5542            }
5543
5544            let builtin_is_active = builtin
5545                .enable_feature_id
5546                .map(|feature_id| self.feature_set.is_active(&feature_id))
5547                .unwrap_or(true);
5548
5549            if builtin_is_active {
5550                let activation_slot = builtin
5551                    .enable_feature_id
5552                    .and_then(|feature_id| self.feature_set.activated_slot(&feature_id))
5553                    .unwrap_or(0);
5554                self.transaction_processor.add_builtin(
5555                    builtin.program_id,
5556                    ProgramCacheEntry::new_builtin(
5557                        activation_slot,
5558                        builtin.name.len(),
5559                        builtin.entrypoint,
5560                    ),
5561                );
5562            }
5563        }
5564    }
5565
5566    fn add_builtin_program_accounts(&mut self) {
5567        for builtin in BUILTINS.iter() {
5568            // The `builtin_is_bpf` flag is used to handle the case where a
5569            // builtin is scheduled to be enabled by one feature gate and
5570            // later migrated to Core BPF by another.
5571            //
5572            // There should never be a case where a builtin is set to be
5573            // migrated to Core BPF and is also set to be enabled on feature
5574            // activation on the same feature gate. However, the
5575            // `builtin_is_bpf` flag will handle this case as well, electing
5576            // to first attempt the migration to Core BPF.
5577            //
5578            // The migration to Core BPF will fail gracefully because the
5579            // program account will not exist. The builtin will subsequently
5580            // be enabled, but it will never be migrated to Core BPF.
5581            //
5582            // Using the same feature gate for both enabling and migrating a
5583            // builtin to Core BPF should be strictly avoided.
5584            let builtin_is_bpf = builtin.core_bpf_migration_config.is_some() && {
5585                self.get_account(&builtin.program_id)
5586                    .map(|a| a.owner() == &bpf_loader_upgradeable::id())
5587                    .unwrap_or(false)
5588            };
5589
5590            // If the builtin has already been migrated to Core BPF, do not
5591            // add it to the bank's builtins.
5592            if builtin_is_bpf {
5593                continue;
5594            }
5595
5596            let builtin_is_active = builtin
5597                .enable_feature_id
5598                .map(|feature_id| self.feature_set.is_active(&feature_id))
5599                .unwrap_or(true);
5600
5601            if builtin_is_active {
5602                self.add_builtin_account(builtin.name, &builtin.program_id);
5603            }
5604        }
5605
5606        for precompile in get_precompiles() {
5607            let precompile_is_active = precompile
5608                .feature
5609                .as_ref()
5610                .map(|feature_id| self.feature_set.is_active(feature_id))
5611                .unwrap_or(true);
5612
5613            if precompile_is_active {
5614                self.add_precompile(&precompile.program_id);
5615            }
5616        }
5617    }
5618
5619    /// Use to replace programs by feature activation
5620    #[allow(dead_code)]
5621    fn replace_program_account(
5622        &mut self,
5623        old_address: &Pubkey,
5624        new_address: &Pubkey,
5625        datapoint_name: &'static str,
5626    ) {
5627        if let Some(old_account) = self.get_account_with_fixed_root(old_address) {
5628            if let Some(new_account) = self.get_account_with_fixed_root(new_address) {
5629                datapoint_info!(datapoint_name, ("slot", self.slot, i64));
5630
5631                // Burn lamports in the old account
5632                self.capitalization
5633                    .fetch_sub(old_account.lamports(), Relaxed);
5634
5635                // Transfer new account to old account
5636                self.store_account(old_address, &new_account);
5637
5638                // Clear new account
5639                self.store_account(new_address, &AccountSharedData::default());
5640
5641                // Unload a program from the bank's cache
5642                self.transaction_processor
5643                    .global_program_cache
5644                    .write()
5645                    .unwrap()
5646                    .remove_programs([*old_address].into_iter());
5647
5648                self.calculate_and_update_accounts_data_size_delta_off_chain(
5649                    old_account.data().len(),
5650                    new_account.data().len(),
5651                );
5652            }
5653        }
5654    }
5655
5656    /// Calculates the accounts data size of all accounts
5657    ///
5658    /// Panics if total overflows a u64.
5659    ///
5660    /// Note, this may be *very* expensive, as *all* accounts are collected
5661    /// into a Vec before summing each account's data size.
5662    ///
5663    /// Only intended to be called by tests or when the number of accounts is small.
5664    pub fn calculate_accounts_data_size(&self) -> ScanResult<u64> {
5665        let accounts = self.get_all_accounts(false)?;
5666        let accounts_data_size = accounts
5667            .into_iter()
5668            .map(|(_pubkey, account, _slot)| account.data().len() as u64)
5669            .try_fold(0, u64::checked_add)
5670            .expect("accounts data size cannot overflow");
5671        Ok(accounts_data_size)
5672    }
5673
5674    pub fn is_in_slot_hashes_history(&self, slot: &Slot) -> bool {
5675        if slot < &self.slot {
5676            if let Ok(slot_hashes) = self.transaction_processor.sysvar_cache().get_slot_hashes() {
5677                return slot_hashes.get(slot).is_some();
5678            }
5679        }
5680        false
5681    }
5682
5683    pub fn check_program_modification_slot(&self) -> bool {
5684        self.check_program_modification_slot
5685    }
5686
5687    pub fn set_check_program_modification_slot(&mut self, check: bool) {
5688        self.check_program_modification_slot = check;
5689    }
5690
5691    pub fn fee_structure(&self) -> &FeeStructure {
5692        &self.fee_structure
5693    }
5694
5695    pub fn parent_block_id(&self) -> Option<Hash> {
5696        self.parent().and_then(|p| p.block_id())
5697    }
5698
5699    pub fn block_id(&self) -> Option<Hash> {
5700        *self.block_id.read().unwrap()
5701    }
5702
5703    pub fn set_block_id(&self, block_id: Option<Hash>) {
5704        *self.block_id.write().unwrap() = block_id;
5705    }
5706
5707    pub fn compute_budget(&self) -> Option<ComputeBudget> {
5708        self.compute_budget
5709    }
5710
5711    pub fn add_builtin(&self, program_id: Pubkey, name: &str, builtin: ProgramCacheEntry) {
5712        debug!("Adding program {name} under {program_id:?}");
5713        self.add_builtin_account(name, &program_id);
5714        self.transaction_processor.add_builtin(program_id, builtin);
5715        debug!("Added program {name} under {program_id:?}");
5716    }
5717
5718    // NOTE: must hold idempotent for the same set of arguments
5719    /// Add a builtin program account
5720    fn add_builtin_account(&self, name: &str, program_id: &Pubkey) {
5721        let existing_genuine_program =
5722            self.get_account_with_fixed_root(program_id)
5723                .and_then(|account| {
5724                    // it's very unlikely to be squatted at program_id as non-system account because of burden to
5725                    // find victim's pubkey/hash. So, when account.owner is indeed native_loader's, it's
5726                    // safe to assume it's a genuine program.
5727                    if native_loader::check_id(account.owner()) {
5728                        Some(account)
5729                    } else {
5730                        // malicious account is pre-occupying at program_id
5731                        self.burn_and_purge_account(program_id, account);
5732                        None
5733                    }
5734                });
5735
5736        // introducing builtin program
5737        if existing_genuine_program.is_some() {
5738            // The existing account is sufficient
5739            return;
5740        }
5741
5742        assert!(
5743            !self.freeze_started(),
5744            "Can't change frozen bank by adding not-existing new builtin program ({name}, \
5745             {program_id}). Maybe, inconsistent program activation is detected on snapshot \
5746             restore?"
5747        );
5748
5749        // Add a bogus executable builtin account, which will be loaded and ignored.
5750        let (lamports, rent_epoch) =
5751            self.inherit_specially_retained_account_fields(&existing_genuine_program);
5752        let account: AccountSharedData = AccountSharedData::from(Account {
5753            lamports,
5754            data: name.as_bytes().to_vec(),
5755            owner: solana_sdk_ids::native_loader::id(),
5756            executable: true,
5757            rent_epoch,
5758        });
5759        self.store_account_and_update_capitalization(program_id, &account);
5760    }
5761
5762    pub fn get_bank_hash_stats(&self) -> BankHashStats {
5763        self.bank_hash_stats.load()
5764    }
5765
5766    pub fn clear_epoch_rewards_cache(&self) {
5767        self.epoch_rewards_calculation_cache.lock().unwrap().clear();
5768    }
5769
5770    /// Sets the accounts lt hash, only to be used by SnapshotMinimizer
5771    pub fn set_accounts_lt_hash_for_snapshot_minimizer(&self, accounts_lt_hash: AccountsLtHash) {
5772        *self.accounts_lt_hash.lock().unwrap() = accounts_lt_hash;
5773    }
5774
5775    /// Return total transaction fee collected
5776    pub fn get_collector_fee_details(&self) -> CollectorFeeDetails {
5777        self.collector_fee_details.read().unwrap().clone()
5778    }
5779}
5780
5781impl InvokeContextCallback for Bank {
5782    fn get_epoch_stake(&self) -> u64 {
5783        self.get_current_epoch_total_stake()
5784    }
5785
5786    fn get_epoch_stake_for_vote_account(&self, vote_address: &Pubkey) -> u64 {
5787        self.get_current_epoch_vote_accounts()
5788            .get(vote_address)
5789            .map(|(stake, _)| *stake)
5790            .unwrap_or(0)
5791    }
5792
5793    fn is_precompile(&self, program_id: &Pubkey) -> bool {
5794        is_precompile(program_id, |feature_id: &Pubkey| {
5795            self.feature_set.is_active(feature_id)
5796        })
5797    }
5798
5799    fn process_precompile(
5800        &self,
5801        program_id: &Pubkey,
5802        data: &[u8],
5803        instruction_datas: Vec<&[u8]>,
5804    ) -> std::result::Result<(), PrecompileError> {
5805        if let Some(precompile) = get_precompile(program_id, |feature_id: &Pubkey| {
5806            self.feature_set.is_active(feature_id)
5807        }) {
5808            precompile.verify(data, &instruction_datas, &self.feature_set)
5809        } else {
5810            Err(PrecompileError::InvalidPublicKey)
5811        }
5812    }
5813}
5814
5815impl TransactionProcessingCallback for Bank {
5816    fn get_account_shared_data(&self, pubkey: &Pubkey) -> Option<(AccountSharedData, Slot)> {
5817        self.rc
5818            .accounts
5819            .accounts_db
5820            .load_with_fixed_root(&self.ancestors, pubkey)
5821    }
5822
5823    fn inspect_account(&self, address: &Pubkey, account_state: AccountState, is_writable: bool) {
5824        self.inspect_account_for_accounts_lt_hash(address, &account_state, is_writable);
5825    }
5826}
5827
5828impl fmt::Debug for Bank {
5829    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
5830        f.debug_struct("Bank")
5831            .field("slot", &self.slot)
5832            .field("bank_id", &self.bank_id)
5833            .field("block_height", &self.block_height)
5834            .field("parent_slot", &self.parent_slot)
5835            .field("capitalization", &self.capitalization())
5836            .finish_non_exhaustive()
5837    }
5838}
5839
5840#[cfg(feature = "dev-context-only-utils")]
5841impl Bank {
5842    pub fn wrap_with_bank_forks_for_tests(self) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5843        let bank_forks = BankForks::new_rw_arc(self);
5844        let bank = bank_forks.read().unwrap().root_bank();
5845        (bank, bank_forks)
5846    }
5847
5848    pub fn default_for_tests() -> Self {
5849        let accounts_db = AccountsDb::default_for_tests();
5850        let accounts = Accounts::new(Arc::new(accounts_db));
5851        Self::default_with_accounts(accounts)
5852    }
5853
5854    pub fn new_with_bank_forks_for_tests(
5855        genesis_config: &GenesisConfig,
5856    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5857        let bank = Self::new_for_tests(genesis_config);
5858        bank.wrap_with_bank_forks_for_tests()
5859    }
5860
5861    pub fn new_for_tests(genesis_config: &GenesisConfig) -> Self {
5862        Self::new_with_config_for_tests(genesis_config, BankTestConfig::default())
5863    }
5864
5865    pub fn new_with_mockup_builtin_for_tests(
5866        genesis_config: &GenesisConfig,
5867        program_id: Pubkey,
5868        builtin_function: BuiltinFunctionWithContext,
5869    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5870        let mut bank = Self::new_for_tests(genesis_config);
5871        bank.add_mockup_builtin(program_id, builtin_function);
5872        bank.wrap_with_bank_forks_for_tests()
5873    }
5874
5875    pub fn new_no_wallclock_throttle_for_tests(
5876        genesis_config: &GenesisConfig,
5877    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5878        let mut bank = Self::new_for_tests(genesis_config);
5879
5880        bank.ns_per_slot = u128::MAX;
5881        bank.wrap_with_bank_forks_for_tests()
5882    }
5883
5884    pub fn new_with_config_for_tests(
5885        genesis_config: &GenesisConfig,
5886        test_config: BankTestConfig,
5887    ) -> Self {
5888        Self::new_with_paths_for_tests(
5889            genesis_config,
5890            Arc::new(RuntimeConfig::default()),
5891            test_config,
5892            Vec::new(),
5893        )
5894    }
5895
5896    pub fn new_with_paths_for_tests(
5897        genesis_config: &GenesisConfig,
5898        runtime_config: Arc<RuntimeConfig>,
5899        test_config: BankTestConfig,
5900        paths: Vec<PathBuf>,
5901    ) -> Self {
5902        Self::new_from_genesis(
5903            genesis_config,
5904            runtime_config,
5905            paths,
5906            None,
5907            test_config.accounts_db_config,
5908            None,
5909            Some(Pubkey::new_unique()),
5910            Arc::default(),
5911            None,
5912            None,
5913        )
5914    }
5915
5916    pub fn new_for_benches(genesis_config: &GenesisConfig) -> Self {
5917        Self::new_with_paths_for_benches(genesis_config, Vec::new())
5918    }
5919
5920    /// Intended for use by benches only.
5921    /// create new bank with the given config and paths.
5922    pub fn new_with_paths_for_benches(genesis_config: &GenesisConfig, paths: Vec<PathBuf>) -> Self {
5923        Self::new_from_genesis(
5924            genesis_config,
5925            Arc::<RuntimeConfig>::default(),
5926            paths,
5927            None,
5928            ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS,
5929            None,
5930            Some(Pubkey::new_unique()),
5931            Arc::default(),
5932            None,
5933            None,
5934        )
5935    }
5936
5937    pub fn new_from_parent_with_bank_forks(
5938        bank_forks: &RwLock<BankForks>,
5939        parent: Arc<Bank>,
5940        collector_id: &Pubkey,
5941        slot: Slot,
5942    ) -> Arc<Self> {
5943        let bank = Bank::new_from_parent(parent, collector_id, slot);
5944        bank_forks
5945            .write()
5946            .unwrap()
5947            .insert(bank)
5948            .clone_without_scheduler()
5949    }
5950
5951    /// Prepare a transaction batch from a list of legacy transactions. Used for tests only.
5952    pub fn prepare_batch_for_tests(
5953        &self,
5954        txs: Vec<Transaction>,
5955    ) -> TransactionBatch<'_, '_, RuntimeTransaction<SanitizedTransaction>> {
5956        let sanitized_txs = txs
5957            .into_iter()
5958            .map(RuntimeTransaction::from_transaction_for_tests)
5959            .collect::<Vec<_>>();
5960        TransactionBatch::new(
5961            self.try_lock_accounts(&sanitized_txs),
5962            self,
5963            OwnedOrBorrowed::Owned(sanitized_txs),
5964        )
5965    }
5966
5967    /// Set the initial accounts data size
5968    /// NOTE: This fn is *ONLY FOR TESTS*
5969    pub fn set_accounts_data_size_initial_for_tests(&mut self, amount: u64) {
5970        self.accounts_data_size_initial = amount;
5971    }
5972
5973    /// Update the accounts data size off-chain delta
5974    /// NOTE: This fn is *ONLY FOR TESTS*
5975    pub fn update_accounts_data_size_delta_off_chain_for_tests(&self, amount: i64) {
5976        self.update_accounts_data_size_delta_off_chain(amount)
5977    }
5978
5979    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
5980    ///
5981    /// # Panics
5982    ///
5983    /// Panics if any of the transactions do not pass sanitization checks.
5984    #[must_use]
5985    pub fn process_transactions<'a>(
5986        &self,
5987        txs: impl Iterator<Item = &'a Transaction>,
5988    ) -> Vec<Result<()>> {
5989        self.try_process_transactions(txs).unwrap()
5990    }
5991
5992    /// Process entry transactions in a single batch. This is used for benches and unit tests.
5993    ///
5994    /// # Panics
5995    ///
5996    /// Panics if any of the transactions do not pass sanitization checks.
5997    #[must_use]
5998    pub fn process_entry_transactions(&self, txs: Vec<VersionedTransaction>) -> Vec<Result<()>> {
5999        self.try_process_entry_transactions(txs).unwrap()
6000    }
6001
6002    #[cfg(test)]
6003    pub fn flush_accounts_cache_slot_for_tests(&self) {
6004        self.rc
6005            .accounts
6006            .accounts_db
6007            .flush_accounts_cache_slot_for_tests(self.slot())
6008    }
6009
6010    pub fn get_sysvar_cache_for_tests(&self) -> SysvarCache {
6011        self.transaction_processor.get_sysvar_cache_for_tests()
6012    }
6013
6014    pub fn calculate_accounts_lt_hash_for_tests(&self) -> AccountsLtHash {
6015        self.rc
6016            .accounts
6017            .accounts_db
6018            .calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, self.slot)
6019    }
6020
6021    pub fn get_transaction_processor(&self) -> &TransactionBatchProcessor<BankForks> {
6022        &self.transaction_processor
6023    }
6024
6025    pub fn set_fee_structure(&mut self, fee_structure: &FeeStructure) {
6026        self.fee_structure = fee_structure.clone();
6027    }
6028
6029    pub fn load_program(
6030        &self,
6031        pubkey: &Pubkey,
6032        reload: bool,
6033        effective_epoch: Epoch,
6034    ) -> Option<Arc<ProgramCacheEntry>> {
6035        let environments = self
6036            .transaction_processor
6037            .get_environments_for_epoch(effective_epoch);
6038        load_program_with_pubkey(
6039            self,
6040            &environments,
6041            pubkey,
6042            self.slot(),
6043            &mut ExecuteTimings::default(), // Called by ledger-tool, metrics not accumulated.
6044            reload,
6045        )
6046    }
6047
6048    pub fn withdraw(&self, pubkey: &Pubkey, lamports: u64) -> Result<()> {
6049        match self.get_account_with_fixed_root(pubkey) {
6050            Some(mut account) => {
6051                let min_balance = match get_system_account_kind(&account) {
6052                    Some(SystemAccountKind::Nonce) => self
6053                        .rent_collector
6054                        .rent
6055                        .minimum_balance(nonce::state::State::size()),
6056                    _ => 0,
6057                };
6058
6059                lamports
6060                    .checked_add(min_balance)
6061                    .filter(|required_balance| *required_balance <= account.lamports())
6062                    .ok_or(TransactionError::InsufficientFundsForFee)?;
6063                account
6064                    .checked_sub_lamports(lamports)
6065                    .map_err(|_| TransactionError::InsufficientFundsForFee)?;
6066                self.store_account(pubkey, &account);
6067
6068                Ok(())
6069            }
6070            None => Err(TransactionError::AccountNotFound),
6071        }
6072    }
6073
6074    pub fn set_hash_overrides(&self, hash_overrides: HashOverrides) {
6075        *self.hash_overrides.lock().unwrap() = hash_overrides;
6076    }
6077
6078    /// Get stake and stake node accounts
6079    pub(crate) fn get_stake_accounts(&self, minimized_account_set: &DashSet<Pubkey>) {
6080        self.stakes_cache
6081            .stakes()
6082            .stake_delegations()
6083            .iter()
6084            .for_each(|(pubkey, _)| {
6085                minimized_account_set.insert(*pubkey);
6086            });
6087
6088        self.stakes_cache
6089            .stakes()
6090            .staked_nodes()
6091            .par_iter()
6092            .for_each(|(pubkey, _)| {
6093                minimized_account_set.insert(*pubkey);
6094            });
6095    }
6096}
6097
6098/// Compute how much an account has changed size.  This function is useful when the data size delta
6099/// needs to be computed and passed to an `update_accounts_data_size_delta` function.
6100fn calculate_data_size_delta(old_data_size: usize, new_data_size: usize) -> i64 {
6101    assert!(old_data_size <= i64::MAX as usize);
6102    assert!(new_data_size <= i64::MAX as usize);
6103    let old_data_size = old_data_size as i64;
6104    let new_data_size = new_data_size as i64;
6105
6106    new_data_size.saturating_sub(old_data_size)
6107}
6108
6109impl Drop for Bank {
6110    fn drop(&mut self) {
6111        if let Some(drop_callback) = self.drop_callback.read().unwrap().0.as_ref() {
6112            drop_callback.callback(self);
6113        } else {
6114            // Default case for tests
6115            self.rc
6116                .accounts
6117                .accounts_db
6118                .purge_slot(self.slot(), self.bank_id(), false);
6119        }
6120    }
6121}
6122
6123/// utility function used for testing and benchmarking.
6124pub mod test_utils {
6125    use {
6126        super::Bank,
6127        crate::installed_scheduler_pool::BankWithScheduler,
6128        solana_account::{state_traits::StateMut, ReadableAccount, WritableAccount},
6129        solana_instruction::error::LamportsError,
6130        solana_pubkey::Pubkey,
6131        solana_sha256_hasher::hashv,
6132        solana_vote_interface::state::VoteStateV4,
6133        solana_vote_program::vote_state::{BlockTimestamp, VoteStateVersions},
6134        std::sync::Arc,
6135    };
6136    pub fn goto_end_of_slot(bank: Arc<Bank>) {
6137        goto_end_of_slot_with_scheduler(&BankWithScheduler::new_without_scheduler(bank))
6138    }
6139
6140    pub fn goto_end_of_slot_with_scheduler(bank: &BankWithScheduler) {
6141        let mut tick_hash = bank.last_blockhash();
6142        loop {
6143            tick_hash = hashv(&[tick_hash.as_ref(), &[42]]);
6144            bank.register_tick(&tick_hash);
6145            if tick_hash == bank.last_blockhash() {
6146                bank.freeze();
6147                return;
6148            }
6149        }
6150    }
6151
6152    pub fn update_vote_account_timestamp(
6153        timestamp: BlockTimestamp,
6154        bank: &Bank,
6155        vote_pubkey: &Pubkey,
6156    ) {
6157        let mut vote_account = bank.get_account(vote_pubkey).unwrap_or_default();
6158        let mut vote_state = VoteStateV4::deserialize(vote_account.data(), vote_pubkey)
6159            .ok()
6160            .unwrap_or_default();
6161        vote_state.last_timestamp = timestamp;
6162        let versioned = VoteStateVersions::new_v4(vote_state);
6163        vote_account.set_state(&versioned).unwrap();
6164        bank.store_account(vote_pubkey, &vote_account);
6165    }
6166
6167    pub fn deposit(
6168        bank: &Bank,
6169        pubkey: &Pubkey,
6170        lamports: u64,
6171    ) -> std::result::Result<u64, LamportsError> {
6172        // This doesn't collect rents intentionally.
6173        // Rents should only be applied to actual TXes
6174        let mut account = bank
6175            .get_account_with_fixed_root_no_cache(pubkey)
6176            .unwrap_or_default();
6177        account.checked_add_lamports(lamports)?;
6178        bank.store_account(pubkey, &account);
6179        Ok(account.lamports())
6180    }
6181}