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        relax_programdata_account_check_migration, 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_data(message)
2739                .map(|(_nonce_address, nonce_data)| nonce_data.get_lamports_per_signature())
2740        })?;
2741        Some(self.get_fee_for_message_with_lamports_per_signature(message, lamports_per_signature))
2742    }
2743
2744    pub fn get_fee_for_message_with_lamports_per_signature(
2745        &self,
2746        message: &impl SVMMessage,
2747        lamports_per_signature: u64,
2748    ) -> u64 {
2749        let fee_budget_limits = FeeBudgetLimits::from(
2750            process_compute_budget_instructions(
2751                message.program_instructions_iter(),
2752                &self.feature_set,
2753            )
2754            .unwrap_or_default(),
2755        );
2756        solana_fee::calculate_fee(
2757            message,
2758            lamports_per_signature == 0,
2759            self.fee_structure().lamports_per_signature,
2760            fee_budget_limits.prioritization_fee,
2761            FeeFeatures::from(self.feature_set.as_ref()),
2762        )
2763    }
2764
2765    pub fn get_blockhash_last_valid_block_height(&self, blockhash: &Hash) -> Option<Slot> {
2766        let blockhash_queue = self.blockhash_queue.read().unwrap();
2767        // This calculation will need to be updated to consider epoch boundaries if BlockhashQueue
2768        // length is made variable by epoch
2769        blockhash_queue
2770            .get_hash_age(blockhash)
2771            .map(|age| self.block_height + MAX_PROCESSING_AGE as u64 - age)
2772    }
2773
2774    pub fn confirmed_last_blockhash(&self) -> Hash {
2775        const NUM_BLOCKHASH_CONFIRMATIONS: usize = 3;
2776
2777        let parents = self.parents();
2778        if parents.is_empty() {
2779            self.last_blockhash()
2780        } else {
2781            let index = NUM_BLOCKHASH_CONFIRMATIONS.min(parents.len() - 1);
2782            parents[index].last_blockhash()
2783        }
2784    }
2785
2786    /// Forget all signatures. Useful for benchmarking.
2787    #[cfg(feature = "dev-context-only-utils")]
2788    pub fn clear_signatures(&self) {
2789        self.status_cache.write().unwrap().clear();
2790    }
2791
2792    pub fn clear_slot_signatures(&self, slot: Slot) {
2793        self.status_cache.write().unwrap().clear_slot_entries(slot);
2794    }
2795
2796    fn update_transaction_statuses(
2797        &self,
2798        sanitized_txs: &[impl TransactionWithMeta],
2799        processing_results: &[TransactionProcessingResult],
2800    ) {
2801        let mut status_cache = self.status_cache.write().unwrap();
2802        assert_eq!(sanitized_txs.len(), processing_results.len());
2803        for (tx, processing_result) in sanitized_txs.iter().zip(processing_results) {
2804            if let Ok(processed_tx) = &processing_result {
2805                // Add the message hash to the status cache to ensure that this message
2806                // won't be processed again with a different signature.
2807                status_cache.insert(
2808                    tx.recent_blockhash(),
2809                    tx.message_hash(),
2810                    self.slot(),
2811                    processed_tx.status(),
2812                );
2813                // Add the transaction signature to the status cache so that transaction status
2814                // can be queried by transaction signature over RPC. In the future, this should
2815                // only be added for API nodes because voting validators don't need to do this.
2816                status_cache.insert(
2817                    tx.recent_blockhash(),
2818                    tx.signature(),
2819                    self.slot(),
2820                    processed_tx.status(),
2821                );
2822            }
2823        }
2824    }
2825
2826    /// Register a new recent blockhash in the bank's recent blockhash queue. Called when a bank
2827    /// reaches its max tick height. Can be called by tests to get new blockhashes for transaction
2828    /// processing without advancing to a new bank slot.
2829    fn register_recent_blockhash(&self, blockhash: &Hash, scheduler: &InstalledSchedulerRwLock) {
2830        // This is needed because recent_blockhash updates necessitate synchronizations for
2831        // consistent tx check_age handling.
2832        BankWithScheduler::wait_for_paused_scheduler(self, scheduler);
2833
2834        // Only acquire the write lock for the blockhash queue on block boundaries because
2835        // readers can starve this write lock acquisition and ticks would be slowed down too
2836        // much if the write lock is acquired for each tick.
2837        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
2838
2839        #[cfg(feature = "dev-context-only-utils")]
2840        let blockhash_override = self
2841            .hash_overrides
2842            .lock()
2843            .unwrap()
2844            .get_blockhash_override(self.slot())
2845            .copied()
2846            .inspect(|blockhash_override| {
2847                if blockhash_override != blockhash {
2848                    info!(
2849                        "bank: slot: {}: overrode blockhash: {} with {}",
2850                        self.slot(),
2851                        blockhash,
2852                        blockhash_override
2853                    );
2854                }
2855            });
2856        #[cfg(feature = "dev-context-only-utils")]
2857        let blockhash = blockhash_override.as_ref().unwrap_or(blockhash);
2858
2859        w_blockhash_queue.register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
2860        self.update_recent_blockhashes_locked(&w_blockhash_queue);
2861    }
2862
2863    // gating this under #[cfg(feature = "dev-context-only-utils")] isn't easy due to
2864    // solana-program-test's usage...
2865    pub fn register_unique_recent_blockhash_for_test(&self) {
2866        self.register_recent_blockhash(
2867            &Hash::new_unique(),
2868            &BankWithScheduler::no_scheduler_available(),
2869        )
2870    }
2871
2872    #[cfg(feature = "dev-context-only-utils")]
2873    pub fn register_recent_blockhash_for_test(
2874        &self,
2875        blockhash: &Hash,
2876        lamports_per_signature: Option<u64>,
2877    ) {
2878        // Only acquire the write lock for the blockhash queue on block boundaries because
2879        // readers can starve this write lock acquisition and ticks would be slowed down too
2880        // much if the write lock is acquired for each tick.
2881        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
2882        if let Some(lamports_per_signature) = lamports_per_signature {
2883            w_blockhash_queue.register_hash(blockhash, lamports_per_signature);
2884        } else {
2885            w_blockhash_queue
2886                .register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
2887        }
2888    }
2889
2890    /// Tell the bank which Entry IDs exist on the ledger. This function assumes subsequent calls
2891    /// correspond to later entries, and will boot the oldest ones once its internal cache is full.
2892    /// Once boot, the bank will reject transactions using that `hash`.
2893    ///
2894    /// This is NOT thread safe because if tick height is updated by two different threads, the
2895    /// block boundary condition could be missed.
2896    pub fn register_tick(&self, hash: &Hash, scheduler: &InstalledSchedulerRwLock) {
2897        assert!(
2898            !self.freeze_started(),
2899            "register_tick() working on a bank that is already frozen or is undergoing freezing!"
2900        );
2901
2902        if self.is_block_boundary(self.tick_height.load(Relaxed) + 1) {
2903            self.register_recent_blockhash(hash, scheduler);
2904        }
2905
2906        // ReplayStage will start computing the accounts delta hash when it
2907        // detects the tick height has reached the boundary, so the system
2908        // needs to guarantee all account updates for the slot have been
2909        // committed before this tick height is incremented (like the blockhash
2910        // sysvar above)
2911        self.tick_height.fetch_add(1, Relaxed);
2912    }
2913
2914    #[cfg(feature = "dev-context-only-utils")]
2915    pub fn register_tick_for_test(&self, hash: &Hash) {
2916        self.register_tick(hash, &BankWithScheduler::no_scheduler_available())
2917    }
2918
2919    #[cfg(feature = "dev-context-only-utils")]
2920    pub fn register_default_tick_for_test(&self) {
2921        self.register_tick_for_test(&Hash::default())
2922    }
2923
2924    pub fn is_complete(&self) -> bool {
2925        self.tick_height() == self.max_tick_height()
2926    }
2927
2928    pub fn is_block_boundary(&self, tick_height: u64) -> bool {
2929        tick_height == self.max_tick_height
2930    }
2931
2932    /// Get the max number of accounts that a transaction may lock in this block
2933    pub fn get_transaction_account_lock_limit(&self) -> usize {
2934        if let Some(transaction_account_lock_limit) = self.transaction_account_lock_limit {
2935            transaction_account_lock_limit
2936        } else if self
2937            .feature_set
2938            .is_active(&feature_set::increase_tx_account_lock_limit::id())
2939        {
2940            MAX_TX_ACCOUNT_LOCKS
2941        } else {
2942            64
2943        }
2944    }
2945
2946    /// Prepare a transaction batch from a list of versioned transactions from
2947    /// an entry. Used for tests only.
2948    pub fn prepare_entry_batch(
2949        &self,
2950        txs: Vec<VersionedTransaction>,
2951    ) -> Result<TransactionBatch<'_, '_, RuntimeTransaction<SanitizedTransaction>>> {
2952        let enable_static_instruction_limit = self
2953            .feature_set
2954            .is_active(&agave_feature_set::static_instruction_limit::id());
2955        let sanitized_txs = txs
2956            .into_iter()
2957            .map(|tx| {
2958                RuntimeTransaction::try_create(
2959                    tx,
2960                    MessageHash::Compute,
2961                    None,
2962                    self,
2963                    self.get_reserved_account_keys(),
2964                    enable_static_instruction_limit,
2965                )
2966            })
2967            .collect::<Result<Vec<_>>>()?;
2968        Ok(TransactionBatch::new(
2969            self.try_lock_accounts(&sanitized_txs),
2970            self,
2971            OwnedOrBorrowed::Owned(sanitized_txs),
2972        ))
2973    }
2974
2975    /// Attempt to take locks on the accounts in a transaction batch
2976    pub fn try_lock_accounts(&self, txs: &[impl TransactionWithMeta]) -> Vec<Result<()>> {
2977        self.try_lock_accounts_with_results(txs, txs.iter().map(|_| Ok(())))
2978    }
2979
2980    /// Attempt to take locks on the accounts in a transaction batch, and their cost
2981    /// limited packing status and duplicate transaction conflict status
2982    pub fn try_lock_accounts_with_results(
2983        &self,
2984        txs: &[impl TransactionWithMeta],
2985        tx_results: impl Iterator<Item = Result<()>>,
2986    ) -> Vec<Result<()>> {
2987        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
2988        let relax_intrabatch_account_locks = self
2989            .feature_set
2990            .is_active(&feature_set::relax_intrabatch_account_locks::id());
2991
2992        // with simd83 enabled, we must fail transactions that duplicate a prior message hash
2993        // previously, conflicting account locks would fail such transactions as a side effect
2994        let mut batch_message_hashes = AHashSet::with_capacity(txs.len());
2995        let tx_results = tx_results
2996            .enumerate()
2997            .map(|(i, tx_result)| match tx_result {
2998                Ok(()) if relax_intrabatch_account_locks => {
2999                    // `HashSet::insert()` returns `true` when the value does *not* already exist
3000                    if batch_message_hashes.insert(txs[i].message_hash()) {
3001                        Ok(())
3002                    } else {
3003                        Err(TransactionError::AlreadyProcessed)
3004                    }
3005                }
3006                Ok(()) => Ok(()),
3007                Err(e) => Err(e),
3008            });
3009
3010        self.rc.accounts.lock_accounts(
3011            txs.iter(),
3012            tx_results,
3013            tx_account_lock_limit,
3014            relax_intrabatch_account_locks,
3015        )
3016    }
3017
3018    /// Prepare a locked transaction batch from a list of sanitized transactions.
3019    pub fn prepare_sanitized_batch<'a, 'b, Tx: TransactionWithMeta>(
3020        &'a self,
3021        txs: &'b [Tx],
3022    ) -> TransactionBatch<'a, 'b, Tx> {
3023        self.prepare_sanitized_batch_with_results(txs, txs.iter().map(|_| Ok(())))
3024    }
3025
3026    /// Prepare a locked transaction batch from a list of sanitized transactions, and their cost
3027    /// limited packing status
3028    pub fn prepare_sanitized_batch_with_results<'a, 'b, Tx: TransactionWithMeta>(
3029        &'a self,
3030        transactions: &'b [Tx],
3031        transaction_results: impl Iterator<Item = Result<()>>,
3032    ) -> TransactionBatch<'a, 'b, Tx> {
3033        // this lock_results could be: Ok, AccountInUse, WouldExceedBlockMaxLimit or WouldExceedAccountMaxLimit
3034        TransactionBatch::new(
3035            self.try_lock_accounts_with_results(transactions, transaction_results),
3036            self,
3037            OwnedOrBorrowed::Borrowed(transactions),
3038        )
3039    }
3040
3041    /// Prepare a transaction batch from a single transaction without locking accounts
3042    pub fn prepare_unlocked_batch_from_single_tx<'a, Tx: SVMMessage>(
3043        &'a self,
3044        transaction: &'a Tx,
3045    ) -> TransactionBatch<'a, 'a, Tx> {
3046        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3047        let lock_result = validate_account_locks(transaction.account_keys(), tx_account_lock_limit);
3048        let mut batch = TransactionBatch::new(
3049            vec![lock_result],
3050            self,
3051            OwnedOrBorrowed::Borrowed(slice::from_ref(transaction)),
3052        );
3053        batch.set_needs_unlock(false);
3054        batch
3055    }
3056
3057    /// Prepare a transaction batch from a single transaction after locking accounts
3058    pub fn prepare_locked_batch_from_single_tx<'a, Tx: TransactionWithMeta>(
3059        &'a self,
3060        transaction: &'a Tx,
3061    ) -> TransactionBatch<'a, 'a, Tx> {
3062        self.prepare_sanitized_batch(slice::from_ref(transaction))
3063    }
3064
3065    /// Run transactions against a frozen bank without committing the results
3066    pub fn simulate_transaction(
3067        &self,
3068        transaction: &impl TransactionWithMeta,
3069        enable_cpi_recording: bool,
3070    ) -> TransactionSimulationResult {
3071        assert!(self.is_frozen(), "simulation bank must be frozen");
3072
3073        self.simulate_transaction_unchecked(transaction, enable_cpi_recording)
3074    }
3075
3076    /// Run transactions against a bank without committing the results; does not check if the bank
3077    /// is frozen, enabling use in single-Bank test frameworks
3078    pub fn simulate_transaction_unchecked(
3079        &self,
3080        transaction: &impl TransactionWithMeta,
3081        enable_cpi_recording: bool,
3082    ) -> TransactionSimulationResult {
3083        let account_keys = transaction.account_keys();
3084        let number_of_accounts = account_keys.len();
3085        let account_overrides = self.get_account_overrides_for_simulation(&account_keys);
3086        let batch = self.prepare_unlocked_batch_from_single_tx(transaction);
3087        let mut timings = ExecuteTimings::default();
3088
3089        let LoadAndExecuteTransactionsOutput {
3090            mut processing_results,
3091            balance_collector,
3092            ..
3093        } = self.load_and_execute_transactions(
3094            &batch,
3095            // After simulation, transactions will need to be forwarded to the leader
3096            // for processing. During forwarding, the transaction could expire if the
3097            // delay is not accounted for.
3098            MAX_PROCESSING_AGE - MAX_TRANSACTION_FORWARDING_DELAY,
3099            &mut timings,
3100            &mut TransactionErrorMetrics::default(),
3101            TransactionProcessingConfig {
3102                account_overrides: Some(&account_overrides),
3103                check_program_modification_slot: self.check_program_modification_slot,
3104                log_messages_bytes_limit: None,
3105                limit_to_load_programs: true,
3106                recording_config: ExecutionRecordingConfig {
3107                    enable_cpi_recording,
3108                    enable_log_recording: true,
3109                    enable_return_data_recording: true,
3110                    enable_transaction_balance_recording: true,
3111                },
3112            },
3113        );
3114
3115        debug!("simulate_transaction: {timings:?}");
3116
3117        let processing_result = processing_results
3118            .pop()
3119            .unwrap_or(Err(TransactionError::InvalidProgramForExecution));
3120        let (
3121            post_simulation_accounts,
3122            result,
3123            fee,
3124            logs,
3125            return_data,
3126            inner_instructions,
3127            units_consumed,
3128            loaded_accounts_data_size,
3129        ) = match processing_result {
3130            Ok(processed_tx) => {
3131                let executed_units = processed_tx.executed_units();
3132                let loaded_accounts_data_size = processed_tx.loaded_accounts_data_size();
3133
3134                match processed_tx {
3135                    ProcessedTransaction::Executed(executed_tx) => {
3136                        let details = executed_tx.execution_details;
3137                        let post_simulation_accounts = executed_tx
3138                            .loaded_transaction
3139                            .accounts
3140                            .into_iter()
3141                            .take(number_of_accounts)
3142                            .collect::<Vec<_>>();
3143                        (
3144                            post_simulation_accounts,
3145                            details.status,
3146                            Some(executed_tx.loaded_transaction.fee_details.total_fee()),
3147                            details.log_messages,
3148                            details.return_data,
3149                            details.inner_instructions,
3150                            executed_units,
3151                            loaded_accounts_data_size,
3152                        )
3153                    }
3154                    ProcessedTransaction::FeesOnly(fees_only_tx) => (
3155                        vec![],
3156                        Err(fees_only_tx.load_error),
3157                        Some(fees_only_tx.fee_details.total_fee()),
3158                        None,
3159                        None,
3160                        None,
3161                        executed_units,
3162                        loaded_accounts_data_size,
3163                    ),
3164                }
3165            }
3166            Err(error) => (vec![], Err(error), None, None, None, None, 0, 0),
3167        };
3168        let logs = logs.unwrap_or_default();
3169
3170        let (pre_balances, post_balances, pre_token_balances, post_token_balances) =
3171            match balance_collector {
3172                Some(balance_collector) => {
3173                    let (mut native_pre, mut native_post, mut token_pre, mut token_post) =
3174                        balance_collector.into_vecs();
3175
3176                    (
3177                        native_pre.pop(),
3178                        native_post.pop(),
3179                        token_pre.pop(),
3180                        token_post.pop(),
3181                    )
3182                }
3183                None => (None, None, None, None),
3184            };
3185
3186        TransactionSimulationResult {
3187            result,
3188            logs,
3189            post_simulation_accounts,
3190            units_consumed,
3191            loaded_accounts_data_size,
3192            return_data,
3193            inner_instructions,
3194            fee,
3195            pre_balances,
3196            post_balances,
3197            pre_token_balances,
3198            post_token_balances,
3199        }
3200    }
3201
3202    fn get_account_overrides_for_simulation(&self, account_keys: &AccountKeys) -> AccountOverrides {
3203        let mut account_overrides = AccountOverrides::default();
3204        let slot_history_id = sysvar::slot_history::id();
3205        if account_keys.iter().any(|pubkey| *pubkey == slot_history_id) {
3206            let current_account = self.get_account_with_fixed_root(&slot_history_id);
3207            let slot_history = current_account
3208                .as_ref()
3209                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
3210                .unwrap_or_default();
3211            if slot_history.check(self.slot()) == Check::Found {
3212                let ancestors = Ancestors::from(self.proper_ancestors().collect::<Vec<_>>());
3213                if let Some((account, _)) =
3214                    self.load_slow_with_fixed_root(&ancestors, &slot_history_id)
3215                {
3216                    account_overrides.set_slot_history(Some(account));
3217                }
3218            }
3219        }
3220        account_overrides
3221    }
3222
3223    pub fn unlock_accounts<'a, Tx: SVMMessage + 'a>(
3224        &self,
3225        txs_and_results: impl Iterator<Item = (&'a Tx, &'a Result<()>)> + Clone,
3226    ) {
3227        self.rc.accounts.unlock_accounts(txs_and_results)
3228    }
3229
3230    pub fn remove_unrooted_slots(&self, slots: &[(Slot, BankId)]) {
3231        self.rc.accounts.accounts_db.remove_unrooted_slots(slots)
3232    }
3233
3234    pub fn get_hash_age(&self, hash: &Hash) -> Option<u64> {
3235        self.blockhash_queue.read().unwrap().get_hash_age(hash)
3236    }
3237
3238    pub fn is_hash_valid_for_age(&self, hash: &Hash, max_age: usize) -> bool {
3239        self.blockhash_queue
3240            .read()
3241            .unwrap()
3242            .is_hash_valid_for_age(hash, max_age)
3243    }
3244
3245    pub fn collect_balances(
3246        &self,
3247        batch: &TransactionBatch<impl SVMMessage>,
3248    ) -> TransactionBalances {
3249        let mut balances: TransactionBalances = vec![];
3250        for transaction in batch.sanitized_transactions() {
3251            let mut transaction_balances: Vec<u64> = vec![];
3252            for account_key in transaction.account_keys().iter() {
3253                transaction_balances.push(self.get_balance(account_key));
3254            }
3255            balances.push(transaction_balances);
3256        }
3257        balances
3258    }
3259
3260    pub fn load_and_execute_transactions(
3261        &self,
3262        batch: &TransactionBatch<impl TransactionWithMeta>,
3263        max_age: usize,
3264        timings: &mut ExecuteTimings,
3265        error_counters: &mut TransactionErrorMetrics,
3266        processing_config: TransactionProcessingConfig,
3267    ) -> LoadAndExecuteTransactionsOutput {
3268        let sanitized_txs = batch.sanitized_transactions();
3269
3270        let (check_results, check_us) = measure_us!(self.check_transactions(
3271            sanitized_txs,
3272            batch.lock_results(),
3273            max_age,
3274            error_counters,
3275        ));
3276        timings.saturating_add_in_place(ExecuteTimingType::CheckUs, check_us);
3277
3278        let (blockhash, blockhash_lamports_per_signature) =
3279            self.last_blockhash_and_lamports_per_signature();
3280        let effective_epoch_of_deployments =
3281            self.epoch_schedule().get_epoch(self.slot.saturating_add(
3282                solana_program_runtime::loaded_programs::DELAY_VISIBILITY_SLOT_OFFSET,
3283            ));
3284        let processing_environment = TransactionProcessingEnvironment {
3285            blockhash,
3286            blockhash_lamports_per_signature,
3287            epoch_total_stake: self.get_current_epoch_total_stake(),
3288            feature_set: self.feature_set.runtime_features(),
3289            program_runtime_environments_for_execution: self
3290                .transaction_processor
3291                .environments
3292                .clone(),
3293            program_runtime_environments_for_deployment: self
3294                .transaction_processor
3295                .get_environments_for_epoch(effective_epoch_of_deployments),
3296            rent: self.rent_collector.rent.clone(),
3297        };
3298
3299        let sanitized_output = self
3300            .transaction_processor
3301            .load_and_execute_sanitized_transactions(
3302                self,
3303                sanitized_txs,
3304                check_results,
3305                &processing_environment,
3306                &processing_config,
3307            );
3308
3309        // Accumulate the errors returned by the batch processor.
3310        error_counters.accumulate(&sanitized_output.error_metrics);
3311
3312        // Accumulate the transaction batch execution timings.
3313        timings.accumulate(&sanitized_output.execute_timings);
3314
3315        let ((), collect_logs_us) =
3316            measure_us!(self.collect_logs(sanitized_txs, &sanitized_output.processing_results));
3317        timings.saturating_add_in_place(ExecuteTimingType::CollectLogsUs, collect_logs_us);
3318
3319        let mut processed_counts = ProcessedTransactionCounts::default();
3320        let err_count = &mut error_counters.total;
3321
3322        for (processing_result, tx) in sanitized_output
3323            .processing_results
3324            .iter()
3325            .zip(sanitized_txs)
3326        {
3327            if let Some(debug_keys) = &self.transaction_debug_keys {
3328                for key in tx.account_keys().iter() {
3329                    if debug_keys.contains(key) {
3330                        let result = processing_result.flattened_result();
3331                        info!("slot: {} result: {:?} tx: {:?}", self.slot, result, tx);
3332                        break;
3333                    }
3334                }
3335            }
3336
3337            if processing_result.was_processed() {
3338                // Signature count must be accumulated only if the transaction
3339                // is processed, otherwise a mismatched count between banking
3340                // and replay could occur
3341                processed_counts.signature_count +=
3342                    tx.signature_details().num_transaction_signatures();
3343                processed_counts.processed_transactions_count += 1;
3344
3345                if !tx.is_simple_vote_transaction() {
3346                    processed_counts.processed_non_vote_transactions_count += 1;
3347                }
3348            }
3349
3350            match processing_result.flattened_result() {
3351                Ok(()) => {
3352                    processed_counts.processed_with_successful_result_count += 1;
3353                }
3354                Err(err) => {
3355                    if err_count.0 == 0 {
3356                        debug!("tx error: {err:?} {tx:?}");
3357                    }
3358                    *err_count += 1;
3359                }
3360            }
3361        }
3362
3363        LoadAndExecuteTransactionsOutput {
3364            processing_results: sanitized_output.processing_results,
3365            processed_counts,
3366            balance_collector: sanitized_output.balance_collector,
3367        }
3368    }
3369
3370    fn collect_logs(
3371        &self,
3372        transactions: &[impl TransactionWithMeta],
3373        processing_results: &[TransactionProcessingResult],
3374    ) {
3375        let transaction_log_collector_config =
3376            self.transaction_log_collector_config.read().unwrap();
3377        if transaction_log_collector_config.filter == TransactionLogCollectorFilter::None {
3378            return;
3379        }
3380
3381        let collected_logs: Vec<_> = processing_results
3382            .iter()
3383            .zip(transactions)
3384            .filter_map(|(processing_result, transaction)| {
3385                // Skip log collection for unprocessed transactions
3386                let processed_tx = processing_result.processed_transaction()?;
3387                // Skip log collection for unexecuted transactions
3388                let execution_details = processed_tx.execution_details()?;
3389                Self::collect_transaction_logs(
3390                    &transaction_log_collector_config,
3391                    transaction,
3392                    execution_details,
3393                )
3394            })
3395            .collect();
3396
3397        if !collected_logs.is_empty() {
3398            let mut transaction_log_collector = self.transaction_log_collector.write().unwrap();
3399            for (log, filtered_mentioned_addresses) in collected_logs {
3400                let transaction_log_index = transaction_log_collector.logs.len();
3401                transaction_log_collector.logs.push(log);
3402                for key in filtered_mentioned_addresses.into_iter() {
3403                    transaction_log_collector
3404                        .mentioned_address_map
3405                        .entry(key)
3406                        .or_default()
3407                        .push(transaction_log_index);
3408                }
3409            }
3410        }
3411    }
3412
3413    fn collect_transaction_logs(
3414        transaction_log_collector_config: &TransactionLogCollectorConfig,
3415        transaction: &impl TransactionWithMeta,
3416        execution_details: &TransactionExecutionDetails,
3417    ) -> Option<(TransactionLogInfo, Vec<Pubkey>)> {
3418        // Skip log collection if no log messages were recorded
3419        let log_messages = execution_details.log_messages.as_ref()?;
3420
3421        let mut filtered_mentioned_addresses = Vec::new();
3422        if !transaction_log_collector_config
3423            .mentioned_addresses
3424            .is_empty()
3425        {
3426            for key in transaction.account_keys().iter() {
3427                if transaction_log_collector_config
3428                    .mentioned_addresses
3429                    .contains(key)
3430                {
3431                    filtered_mentioned_addresses.push(*key);
3432                }
3433            }
3434        }
3435
3436        let is_vote = transaction.is_simple_vote_transaction();
3437        let store = match transaction_log_collector_config.filter {
3438            TransactionLogCollectorFilter::All => {
3439                !is_vote || !filtered_mentioned_addresses.is_empty()
3440            }
3441            TransactionLogCollectorFilter::AllWithVotes => true,
3442            TransactionLogCollectorFilter::None => false,
3443            TransactionLogCollectorFilter::OnlyMentionedAddresses => {
3444                !filtered_mentioned_addresses.is_empty()
3445            }
3446        };
3447
3448        if store {
3449            Some((
3450                TransactionLogInfo {
3451                    signature: *transaction.signature(),
3452                    result: execution_details.status.clone(),
3453                    is_vote,
3454                    log_messages: log_messages.clone(),
3455                },
3456                filtered_mentioned_addresses,
3457            ))
3458        } else {
3459            None
3460        }
3461    }
3462
3463    /// Load the accounts data size, in bytes
3464    pub fn load_accounts_data_size(&self) -> u64 {
3465        self.accounts_data_size_initial
3466            .saturating_add_signed(self.load_accounts_data_size_delta())
3467    }
3468
3469    /// Load the change in accounts data size in this Bank, in bytes
3470    pub fn load_accounts_data_size_delta(&self) -> i64 {
3471        let delta_on_chain = self.load_accounts_data_size_delta_on_chain();
3472        let delta_off_chain = self.load_accounts_data_size_delta_off_chain();
3473        delta_on_chain.saturating_add(delta_off_chain)
3474    }
3475
3476    /// Load the change in accounts data size in this Bank, in bytes, from on-chain events
3477    /// i.e. transactions
3478    pub fn load_accounts_data_size_delta_on_chain(&self) -> i64 {
3479        self.accounts_data_size_delta_on_chain.load(Acquire)
3480    }
3481
3482    /// Load the change in accounts data size in this Bank, in bytes, from off-chain events
3483    /// i.e. rent collection
3484    pub fn load_accounts_data_size_delta_off_chain(&self) -> i64 {
3485        self.accounts_data_size_delta_off_chain.load(Acquire)
3486    }
3487
3488    /// Update the accounts data size delta from on-chain events by adding `amount`.
3489    /// The arithmetic saturates.
3490    fn update_accounts_data_size_delta_on_chain(&self, amount: i64) {
3491        if amount == 0 {
3492            return;
3493        }
3494
3495        self.accounts_data_size_delta_on_chain
3496            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_on_chain| {
3497                Some(accounts_data_size_delta_on_chain.saturating_add(amount))
3498            })
3499            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3500            .unwrap();
3501    }
3502
3503    /// Update the accounts data size delta from off-chain events by adding `amount`.
3504    /// The arithmetic saturates.
3505    fn update_accounts_data_size_delta_off_chain(&self, amount: i64) {
3506        if amount == 0 {
3507            return;
3508        }
3509
3510        self.accounts_data_size_delta_off_chain
3511            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_off_chain| {
3512                Some(accounts_data_size_delta_off_chain.saturating_add(amount))
3513            })
3514            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3515            .unwrap();
3516    }
3517
3518    /// Calculate the data size delta and update the off-chain accounts data size delta
3519    fn calculate_and_update_accounts_data_size_delta_off_chain(
3520        &self,
3521        old_data_size: usize,
3522        new_data_size: usize,
3523    ) {
3524        let data_size_delta = calculate_data_size_delta(old_data_size, new_data_size);
3525        self.update_accounts_data_size_delta_off_chain(data_size_delta);
3526    }
3527
3528    fn filter_program_errors_and_collect_fee_details(
3529        &self,
3530        processing_results: &[TransactionProcessingResult],
3531    ) {
3532        let mut accumulated_fee_details = FeeDetails::default();
3533
3534        processing_results.iter().for_each(|processing_result| {
3535            if let Ok(processed_tx) = processing_result {
3536                accumulated_fee_details.accumulate(&processed_tx.fee_details());
3537            }
3538        });
3539
3540        self.collector_fee_details
3541            .write()
3542            .unwrap()
3543            .accumulate(&accumulated_fee_details);
3544    }
3545
3546    fn update_bank_hash_stats<'a>(&self, accounts: &impl StorableAccounts<'a>) {
3547        let mut stats = BankHashStats::default();
3548        (0..accounts.len()).for_each(|i| {
3549            accounts.account(i, |account| {
3550                stats.update(&account);
3551            })
3552        });
3553        self.bank_hash_stats.accumulate(&stats);
3554    }
3555
3556    pub fn commit_transactions(
3557        &self,
3558        sanitized_txs: &[impl TransactionWithMeta],
3559        processing_results: Vec<TransactionProcessingResult>,
3560        processed_counts: &ProcessedTransactionCounts,
3561        timings: &mut ExecuteTimings,
3562    ) -> Vec<TransactionCommitResult> {
3563        assert!(
3564            !self.freeze_started(),
3565            "commit_transactions() working on a bank that is already frozen or is undergoing \
3566             freezing!"
3567        );
3568
3569        let ProcessedTransactionCounts {
3570            processed_transactions_count,
3571            processed_non_vote_transactions_count,
3572            processed_with_successful_result_count,
3573            signature_count,
3574        } = *processed_counts;
3575
3576        self.increment_transaction_count(processed_transactions_count);
3577        self.increment_non_vote_transaction_count_since_restart(
3578            processed_non_vote_transactions_count,
3579        );
3580        self.increment_signature_count(signature_count);
3581
3582        let processed_with_failure_result_count =
3583            processed_transactions_count.saturating_sub(processed_with_successful_result_count);
3584        self.transaction_error_count
3585            .fetch_add(processed_with_failure_result_count, Relaxed);
3586
3587        if processed_transactions_count > 0 {
3588            self.is_delta.store(true, Relaxed);
3589            self.transaction_entries_count.fetch_add(1, Relaxed);
3590            self.transactions_per_entry_max
3591                .fetch_max(processed_transactions_count, Relaxed);
3592        }
3593
3594        let ((), store_accounts_us) = measure_us!({
3595            // If geyser is present, we must collect `SanitizedTransaction`
3596            // references in order to comply with that interface - until it
3597            // is changed.
3598            let maybe_transaction_refs = self
3599                .accounts()
3600                .accounts_db
3601                .has_accounts_update_notifier()
3602                .then(|| {
3603                    sanitized_txs
3604                        .iter()
3605                        .map(|tx| tx.as_sanitized_transaction())
3606                        .collect::<Vec<_>>()
3607                });
3608
3609            let (accounts_to_store, transactions) = collect_accounts_to_store(
3610                sanitized_txs,
3611                &maybe_transaction_refs,
3612                &processing_results,
3613            );
3614
3615            let to_store = (self.slot(), accounts_to_store.as_slice());
3616            self.update_bank_hash_stats(&to_store);
3617            // See https://github.com/solana-labs/solana/pull/31455 for discussion
3618            // on *not* updating the index within a threadpool.
3619            self.rc
3620                .accounts
3621                .store_accounts_seq(to_store, transactions.as_deref());
3622        });
3623
3624        // Cached vote and stake accounts are synchronized with accounts-db
3625        // after each transaction.
3626        let ((), update_stakes_cache_us) =
3627            measure_us!(self.update_stakes_cache(sanitized_txs, &processing_results));
3628
3629        let ((), update_executors_us) = measure_us!({
3630            let mut cache = None;
3631            for processing_result in &processing_results {
3632                if let Some(ProcessedTransaction::Executed(executed_tx)) =
3633                    processing_result.processed_transaction()
3634                {
3635                    let programs_modified_by_tx = &executed_tx.programs_modified_by_tx;
3636                    if executed_tx.was_successful() && !programs_modified_by_tx.is_empty() {
3637                        cache
3638                            .get_or_insert_with(|| {
3639                                self.transaction_processor
3640                                    .global_program_cache
3641                                    .write()
3642                                    .unwrap()
3643                            })
3644                            .merge(
3645                                &self.transaction_processor.environments,
3646                                programs_modified_by_tx,
3647                            );
3648                    }
3649                }
3650            }
3651        });
3652
3653        let accounts_data_len_delta = processing_results
3654            .iter()
3655            .filter_map(|processing_result| processing_result.processed_transaction())
3656            .filter_map(|processed_tx| processed_tx.execution_details())
3657            .filter_map(|details| {
3658                details
3659                    .status
3660                    .is_ok()
3661                    .then_some(details.accounts_data_len_delta)
3662            })
3663            .sum();
3664        self.update_accounts_data_size_delta_on_chain(accounts_data_len_delta);
3665
3666        let ((), update_transaction_statuses_us) =
3667            measure_us!(self.update_transaction_statuses(sanitized_txs, &processing_results));
3668
3669        self.filter_program_errors_and_collect_fee_details(&processing_results);
3670
3671        timings.saturating_add_in_place(ExecuteTimingType::StoreUs, store_accounts_us);
3672        timings.saturating_add_in_place(
3673            ExecuteTimingType::UpdateStakesCacheUs,
3674            update_stakes_cache_us,
3675        );
3676        timings.saturating_add_in_place(ExecuteTimingType::UpdateExecutorsUs, update_executors_us);
3677        timings.saturating_add_in_place(
3678            ExecuteTimingType::UpdateTransactionStatuses,
3679            update_transaction_statuses_us,
3680        );
3681
3682        Self::create_commit_results(processing_results)
3683    }
3684
3685    fn create_commit_results(
3686        processing_results: Vec<TransactionProcessingResult>,
3687    ) -> Vec<TransactionCommitResult> {
3688        processing_results
3689            .into_iter()
3690            .map(|processing_result| {
3691                let processing_result = processing_result?;
3692                let executed_units = processing_result.executed_units();
3693                let loaded_accounts_data_size = processing_result.loaded_accounts_data_size();
3694
3695                match processing_result {
3696                    ProcessedTransaction::Executed(executed_tx) => {
3697                        let successful = executed_tx.was_successful();
3698                        let execution_details = executed_tx.execution_details;
3699                        let LoadedTransaction {
3700                            accounts: loaded_accounts,
3701                            fee_details,
3702                            rollback_accounts,
3703                            ..
3704                        } = executed_tx.loaded_transaction;
3705
3706                        // Rollback value is used for failure.
3707                        let fee_payer_post_balance = if successful {
3708                            loaded_accounts[0].1.lamports()
3709                        } else {
3710                            rollback_accounts.fee_payer().1.lamports()
3711                        };
3712
3713                        Ok(CommittedTransaction {
3714                            status: execution_details.status,
3715                            log_messages: execution_details.log_messages,
3716                            inner_instructions: execution_details.inner_instructions,
3717                            return_data: execution_details.return_data,
3718                            executed_units,
3719                            fee_details,
3720                            loaded_account_stats: TransactionLoadedAccountsStats {
3721                                loaded_accounts_count: loaded_accounts.len(),
3722                                loaded_accounts_data_size,
3723                            },
3724                            fee_payer_post_balance,
3725                        })
3726                    }
3727                    ProcessedTransaction::FeesOnly(fees_only_tx) => Ok(CommittedTransaction {
3728                        status: Err(fees_only_tx.load_error),
3729                        log_messages: None,
3730                        inner_instructions: None,
3731                        return_data: None,
3732                        executed_units,
3733                        fee_details: fees_only_tx.fee_details,
3734                        loaded_account_stats: TransactionLoadedAccountsStats {
3735                            loaded_accounts_count: fees_only_tx.rollback_accounts.count(),
3736                            loaded_accounts_data_size,
3737                        },
3738                        fee_payer_post_balance: fees_only_tx
3739                            .rollback_accounts
3740                            .fee_payer()
3741                            .1
3742                            .lamports(),
3743                    }),
3744                }
3745            })
3746            .collect()
3747    }
3748
3749    fn run_incinerator(&self) {
3750        if let Some((account, _)) =
3751            self.get_account_modified_since_parent_with_fixed_root(&incinerator::id())
3752        {
3753            self.capitalization.fetch_sub(account.lamports(), Relaxed);
3754            self.store_account(&incinerator::id(), &AccountSharedData::default());
3755        }
3756    }
3757
3758    /// Returns the accounts, sorted by pubkey, that were part of accounts lt hash calculation
3759    /// This is used when writing a bank hash details file.
3760    pub(crate) fn get_accounts_for_bank_hash_details(&self) -> Vec<(Pubkey, AccountSharedData)> {
3761        let mut accounts = self
3762            .rc
3763            .accounts
3764            .accounts_db
3765            .get_pubkey_account_for_slot(self.slot());
3766        // Sort the accounts by pubkey to make diff deterministic.
3767        accounts.sort_unstable_by(|a, b| a.0.cmp(&b.0));
3768        accounts
3769    }
3770
3771    pub fn cluster_type(&self) -> ClusterType {
3772        // unwrap is safe; self.cluster_type is ensured to be Some() always...
3773        // we only using Option here for ABI compatibility...
3774        self.cluster_type.unwrap()
3775    }
3776
3777    /// Process a batch of transactions.
3778    #[must_use]
3779    pub fn load_execute_and_commit_transactions(
3780        &self,
3781        batch: &TransactionBatch<impl TransactionWithMeta>,
3782        max_age: usize,
3783        recording_config: ExecutionRecordingConfig,
3784        timings: &mut ExecuteTimings,
3785        log_messages_bytes_limit: Option<usize>,
3786    ) -> (Vec<TransactionCommitResult>, Option<BalanceCollector>) {
3787        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
3788            batch,
3789            max_age,
3790            recording_config,
3791            timings,
3792            log_messages_bytes_limit,
3793            None::<fn(&mut _, &_) -> _>,
3794        )
3795        .unwrap()
3796    }
3797
3798    pub fn load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
3799        &'a self,
3800        batch: &TransactionBatch<impl TransactionWithMeta>,
3801        max_age: usize,
3802        recording_config: ExecutionRecordingConfig,
3803        timings: &mut ExecuteTimings,
3804        log_messages_bytes_limit: Option<usize>,
3805        pre_commit_callback: impl FnOnce(
3806            &mut ExecuteTimings,
3807            &[TransactionProcessingResult],
3808        ) -> PreCommitResult<'a>,
3809    ) -> Result<(Vec<TransactionCommitResult>, Option<BalanceCollector>)> {
3810        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
3811            batch,
3812            max_age,
3813            recording_config,
3814            timings,
3815            log_messages_bytes_limit,
3816            Some(pre_commit_callback),
3817        )
3818    }
3819
3820    fn do_load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
3821        &'a self,
3822        batch: &TransactionBatch<impl TransactionWithMeta>,
3823        max_age: usize,
3824        recording_config: ExecutionRecordingConfig,
3825        timings: &mut ExecuteTimings,
3826        log_messages_bytes_limit: Option<usize>,
3827        pre_commit_callback: Option<
3828            impl FnOnce(&mut ExecuteTimings, &[TransactionProcessingResult]) -> PreCommitResult<'a>,
3829        >,
3830    ) -> Result<(Vec<TransactionCommitResult>, Option<BalanceCollector>)> {
3831        let LoadAndExecuteTransactionsOutput {
3832            processing_results,
3833            processed_counts,
3834            balance_collector,
3835        } = self.load_and_execute_transactions(
3836            batch,
3837            max_age,
3838            timings,
3839            &mut TransactionErrorMetrics::default(),
3840            TransactionProcessingConfig {
3841                account_overrides: None,
3842                check_program_modification_slot: self.check_program_modification_slot,
3843                log_messages_bytes_limit,
3844                limit_to_load_programs: false,
3845                recording_config,
3846            },
3847        );
3848
3849        // pre_commit_callback could initiate an atomic operation (i.e. poh recording with block
3850        // producing unified scheduler). in that case, it returns Some(freeze_lock), which should
3851        // unlocked only after calling commit_transactions() immediately after calling the
3852        // callback.
3853        let freeze_lock = if let Some(pre_commit_callback) = pre_commit_callback {
3854            pre_commit_callback(timings, &processing_results)?
3855        } else {
3856            None
3857        };
3858        let commit_results = self.commit_transactions(
3859            batch.sanitized_transactions(),
3860            processing_results,
3861            &processed_counts,
3862            timings,
3863        );
3864        drop(freeze_lock);
3865        Ok((commit_results, balance_collector))
3866    }
3867
3868    /// Process a Transaction. This is used for unit tests and simply calls the vector
3869    /// Bank::process_transactions method.
3870    pub fn process_transaction(&self, tx: &Transaction) -> Result<()> {
3871        self.try_process_transactions(std::iter::once(tx))?[0].clone()
3872    }
3873
3874    /// Process a Transaction and store metadata. This is used for tests and the banks services. It
3875    /// replicates the vector Bank::process_transaction method with metadata recording enabled.
3876    pub fn process_transaction_with_metadata(
3877        &self,
3878        tx: impl Into<VersionedTransaction>,
3879    ) -> Result<CommittedTransaction> {
3880        let txs = vec![tx.into()];
3881        let batch = self.prepare_entry_batch(txs)?;
3882
3883        let (mut commit_results, ..) = self.load_execute_and_commit_transactions(
3884            &batch,
3885            MAX_PROCESSING_AGE,
3886            ExecutionRecordingConfig {
3887                enable_cpi_recording: false,
3888                enable_log_recording: true,
3889                enable_return_data_recording: true,
3890                enable_transaction_balance_recording: false,
3891            },
3892            &mut ExecuteTimings::default(),
3893            Some(1000 * 1000),
3894        );
3895
3896        commit_results.remove(0)
3897    }
3898
3899    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
3900    /// Short circuits if any of the transactions do not pass sanitization checks.
3901    pub fn try_process_transactions<'a>(
3902        &self,
3903        txs: impl Iterator<Item = &'a Transaction>,
3904    ) -> Result<Vec<Result<()>>> {
3905        let txs = txs
3906            .map(|tx| VersionedTransaction::from(tx.clone()))
3907            .collect();
3908        self.try_process_entry_transactions(txs)
3909    }
3910
3911    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
3912    /// Short circuits if any of the transactions do not pass sanitization checks.
3913    pub fn try_process_entry_transactions(
3914        &self,
3915        txs: Vec<VersionedTransaction>,
3916    ) -> Result<Vec<Result<()>>> {
3917        let batch = self.prepare_entry_batch(txs)?;
3918        Ok(self.process_transaction_batch(&batch))
3919    }
3920
3921    #[must_use]
3922    fn process_transaction_batch(
3923        &self,
3924        batch: &TransactionBatch<impl TransactionWithMeta>,
3925    ) -> Vec<Result<()>> {
3926        self.load_execute_and_commit_transactions(
3927            batch,
3928            MAX_PROCESSING_AGE,
3929            ExecutionRecordingConfig::new_single_setting(false),
3930            &mut ExecuteTimings::default(),
3931            None,
3932        )
3933        .0
3934        .into_iter()
3935        .map(|commit_result| commit_result.and_then(|committed_tx| committed_tx.status))
3936        .collect()
3937    }
3938
3939    /// Create, sign, and process a Transaction from `keypair` to `to` of
3940    /// `n` lamports where `blockhash` is the last Entry ID observed by the client.
3941    pub fn transfer(&self, n: u64, keypair: &Keypair, to: &Pubkey) -> Result<Signature> {
3942        let blockhash = self.last_blockhash();
3943        let tx = system_transaction::transfer(keypair, to, n, blockhash);
3944        let signature = tx.signatures[0];
3945        self.process_transaction(&tx).map(|_| signature)
3946    }
3947
3948    pub fn read_balance(account: &AccountSharedData) -> u64 {
3949        account.lamports()
3950    }
3951    /// Each program would need to be able to introspect its own state
3952    /// this is hard-coded to the Budget language
3953    pub fn get_balance(&self, pubkey: &Pubkey) -> u64 {
3954        self.get_account(pubkey)
3955            .map(|x| Self::read_balance(&x))
3956            .unwrap_or(0)
3957    }
3958
3959    /// Compute all the parents of the bank in order
3960    pub fn parents(&self) -> Vec<Arc<Bank>> {
3961        let mut parents = vec![];
3962        let mut bank = self.parent();
3963        while let Some(parent) = bank {
3964            parents.push(parent.clone());
3965            bank = parent.parent();
3966        }
3967        parents
3968    }
3969
3970    /// Compute all the parents of the bank including this bank itself
3971    pub fn parents_inclusive(self: Arc<Self>) -> Vec<Arc<Bank>> {
3972        let mut parents = self.parents();
3973        parents.insert(0, self);
3974        parents
3975    }
3976
3977    /// fn store the single `account` with `pubkey`.
3978    /// Uses `store_accounts`, which works on a vector of accounts.
3979    pub fn store_account(&self, pubkey: &Pubkey, account: &AccountSharedData) {
3980        self.store_accounts((self.slot(), &[(pubkey, account)][..]))
3981    }
3982
3983    pub fn store_accounts<'a>(&self, accounts: impl StorableAccounts<'a>) {
3984        assert!(!self.freeze_started());
3985        let mut m = Measure::start("stakes_cache.check_and_store");
3986        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
3987
3988        (0..accounts.len()).for_each(|i| {
3989            accounts.account(i, |account| {
3990                self.stakes_cache.check_and_store(
3991                    account.pubkey(),
3992                    &account,
3993                    new_warmup_cooldown_rate_epoch,
3994                )
3995            })
3996        });
3997        self.update_bank_hash_stats(&accounts);
3998        self.rc.accounts.store_accounts_par(accounts, None);
3999        m.stop();
4000        self.rc
4001            .accounts
4002            .accounts_db
4003            .stats
4004            .stakes_cache_check_and_store_us
4005            .fetch_add(m.as_us(), Relaxed);
4006    }
4007
4008    pub fn force_flush_accounts_cache(&self) {
4009        self.rc
4010            .accounts
4011            .accounts_db
4012            .flush_accounts_cache(true, Some(self.slot()))
4013    }
4014
4015    pub fn flush_accounts_cache_if_needed(&self) {
4016        self.rc
4017            .accounts
4018            .accounts_db
4019            .flush_accounts_cache(false, Some(self.slot()))
4020    }
4021
4022    /// Technically this issues (or even burns!) new lamports,
4023    /// so be extra careful for its usage
4024    fn store_account_and_update_capitalization(
4025        &self,
4026        pubkey: &Pubkey,
4027        new_account: &AccountSharedData,
4028    ) {
4029        let old_account_data_size = if let Some(old_account) =
4030            self.get_account_with_fixed_root_no_cache(pubkey)
4031        {
4032            match new_account.lamports().cmp(&old_account.lamports()) {
4033                std::cmp::Ordering::Greater => {
4034                    let diff = new_account.lamports() - old_account.lamports();
4035                    trace!("store_account_and_update_capitalization: increased: {pubkey} {diff}");
4036                    self.capitalization.fetch_add(diff, Relaxed);
4037                }
4038                std::cmp::Ordering::Less => {
4039                    let diff = old_account.lamports() - new_account.lamports();
4040                    trace!("store_account_and_update_capitalization: decreased: {pubkey} {diff}");
4041                    self.capitalization.fetch_sub(diff, Relaxed);
4042                }
4043                std::cmp::Ordering::Equal => {}
4044            }
4045            old_account.data().len()
4046        } else {
4047            trace!(
4048                "store_account_and_update_capitalization: created: {pubkey} {}",
4049                new_account.lamports()
4050            );
4051            self.capitalization
4052                .fetch_add(new_account.lamports(), Relaxed);
4053            0
4054        };
4055
4056        self.store_account(pubkey, new_account);
4057        self.calculate_and_update_accounts_data_size_delta_off_chain(
4058            old_account_data_size,
4059            new_account.data().len(),
4060        );
4061    }
4062
4063    pub fn accounts(&self) -> Arc<Accounts> {
4064        self.rc.accounts.clone()
4065    }
4066
4067    fn apply_simd_0306_cost_tracker_changes(&mut self) {
4068        let mut cost_tracker = self.write_cost_tracker().unwrap();
4069        let block_cost_limit = cost_tracker.get_block_limit();
4070        let vote_cost_limit = cost_tracker.get_vote_limit();
4071        // SIMD-0306 makes account cost limit 40% of the block cost limit.
4072        let account_cost_limit = block_cost_limit.saturating_mul(40).saturating_div(100);
4073        cost_tracker.set_limits(account_cost_limit, block_cost_limit, vote_cost_limit);
4074    }
4075
4076    fn apply_simd_0339_invoke_cost_changes(&mut self) {
4077        let simd_0268_active = self
4078            .feature_set
4079            .is_active(&raise_cpi_nesting_limit_to_8::id());
4080        let simd_0339_active = self
4081            .feature_set
4082            .is_active(&increase_cpi_account_info_limit::id());
4083        let compute_budget = self
4084            .compute_budget()
4085            .as_ref()
4086            .unwrap_or(&ComputeBudget::new_with_defaults(
4087                simd_0268_active,
4088                simd_0339_active,
4089            ))
4090            .to_cost();
4091
4092        self.transaction_processor
4093            .set_execution_cost(compute_budget);
4094    }
4095
4096    /// This is called from genesis and snapshot restore
4097    fn apply_activated_features(&mut self) {
4098        // Update active set of reserved account keys which are not allowed to be write locked
4099        self.reserved_account_keys = {
4100            let mut reserved_keys = ReservedAccountKeys::clone(&self.reserved_account_keys);
4101            reserved_keys.update_active_set(&self.feature_set);
4102            Arc::new(reserved_keys)
4103        };
4104
4105        // Update the transaction processor with all active built-in programs
4106        self.add_active_builtin_programs();
4107
4108        // Cost-Tracker is not serialized in snapshot or any configs.
4109        // We must apply previously activated features related to limits here
4110        // so that the initial bank state is consistent with the feature set.
4111        // Cost-tracker limits are propagated through children banks.
4112        if self
4113            .feature_set
4114            .is_active(&feature_set::raise_block_limits_to_100m::id())
4115        {
4116            let block_cost_limit = simd_0286_block_limits();
4117            let mut cost_tracker = self.write_cost_tracker().unwrap();
4118            let account_cost_limit = cost_tracker.get_account_limit();
4119            let vote_cost_limit = cost_tracker.get_vote_limit();
4120            cost_tracker.set_limits(account_cost_limit, block_cost_limit, vote_cost_limit);
4121        }
4122
4123        if self
4124            .feature_set
4125            .is_active(&feature_set::raise_account_cu_limit::id())
4126        {
4127            self.apply_simd_0306_cost_tracker_changes();
4128        }
4129
4130        if self
4131            .feature_set
4132            .is_active(&feature_set::increase_cpi_account_info_limit::id())
4133        {
4134            self.apply_simd_0339_invoke_cost_changes();
4135        }
4136
4137        let environments = self.create_program_runtime_environments(&self.feature_set);
4138        self.transaction_processor
4139            .global_program_cache
4140            .write()
4141            .unwrap()
4142            .latest_root_slot = self.slot;
4143        self.transaction_processor
4144            .epoch_boundary_preparation
4145            .write()
4146            .unwrap()
4147            .upcoming_epoch = self.epoch;
4148        self.transaction_processor.environments = environments;
4149    }
4150
4151    fn create_program_runtime_environments(
4152        &self,
4153        feature_set: &FeatureSet,
4154    ) -> ProgramRuntimeEnvironments {
4155        let simd_0268_active = feature_set.is_active(&raise_cpi_nesting_limit_to_8::id());
4156        let simd_0339_active = feature_set.is_active(&increase_cpi_account_info_limit::id());
4157        let compute_budget = self
4158            .compute_budget()
4159            .as_ref()
4160            .unwrap_or(&ComputeBudget::new_with_defaults(
4161                simd_0268_active,
4162                simd_0339_active,
4163            ))
4164            .to_budget();
4165        ProgramRuntimeEnvironments {
4166            program_runtime_v1: Arc::new(
4167                create_program_runtime_environment_v1(
4168                    &feature_set.runtime_features(),
4169                    &compute_budget,
4170                    false, /* deployment */
4171                    false, /* debugging_features */
4172                )
4173                .unwrap(),
4174            ),
4175            program_runtime_v2: Arc::new(create_program_runtime_environment_v2(
4176                &compute_budget,
4177                false, /* debugging_features */
4178            )),
4179        }
4180    }
4181
4182    pub fn set_tick_height(&self, tick_height: u64) {
4183        self.tick_height.store(tick_height, Relaxed)
4184    }
4185
4186    pub fn set_inflation(&self, inflation: Inflation) {
4187        *self.inflation.write().unwrap() = inflation;
4188    }
4189
4190    /// Get a snapshot of the current set of hard forks
4191    pub fn hard_forks(&self) -> HardForks {
4192        self.hard_forks.read().unwrap().clone()
4193    }
4194
4195    pub fn register_hard_fork(&self, new_hard_fork_slot: Slot) {
4196        let bank_slot = self.slot();
4197
4198        let lock = self.freeze_lock();
4199        let bank_frozen = *lock != Hash::default();
4200        if new_hard_fork_slot < bank_slot {
4201            warn!(
4202                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is older than the \
4203                 bank at slot {bank_slot} that attempted to register it."
4204            );
4205        } else if (new_hard_fork_slot == bank_slot) && bank_frozen {
4206            warn!(
4207                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is the same slot \
4208                 as the bank at slot {bank_slot} that attempted to register it, but that bank is \
4209                 already frozen."
4210            );
4211        } else {
4212            self.hard_forks
4213                .write()
4214                .unwrap()
4215                .register(new_hard_fork_slot);
4216        }
4217    }
4218
4219    pub fn get_account_with_fixed_root_no_cache(
4220        &self,
4221        pubkey: &Pubkey,
4222    ) -> Option<AccountSharedData> {
4223        self.load_account_with(pubkey, false)
4224            .map(|(acc, _slot)| acc)
4225    }
4226
4227    fn load_account_with(
4228        &self,
4229        pubkey: &Pubkey,
4230        should_put_in_read_cache: bool,
4231    ) -> Option<(AccountSharedData, Slot)> {
4232        self.rc.accounts.accounts_db.load_account_with(
4233            &self.ancestors,
4234            pubkey,
4235            should_put_in_read_cache,
4236        )
4237    }
4238
4239    // Hi! leaky abstraction here....
4240    // try to use get_account_with_fixed_root() if it's called ONLY from on-chain runtime account
4241    // processing. That alternative fn provides more safety.
4242    pub fn get_account(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
4243        self.get_account_modified_slot(pubkey)
4244            .map(|(acc, _slot)| acc)
4245    }
4246
4247    // Hi! leaky abstraction here....
4248    // use this over get_account() if it's called ONLY from on-chain runtime account
4249    // processing (i.e. from in-band replay/banking stage; that ensures root is *fixed* while
4250    // running).
4251    // pro: safer assertion can be enabled inside AccountsDb
4252    // con: panics!() if called from off-chain processing
4253    pub fn get_account_with_fixed_root(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
4254        self.get_account_modified_slot_with_fixed_root(pubkey)
4255            .map(|(acc, _slot)| acc)
4256    }
4257
4258    // See note above get_account_with_fixed_root() about when to prefer this function
4259    pub fn get_account_modified_slot_with_fixed_root(
4260        &self,
4261        pubkey: &Pubkey,
4262    ) -> Option<(AccountSharedData, Slot)> {
4263        self.load_slow_with_fixed_root(&self.ancestors, pubkey)
4264    }
4265
4266    pub fn get_account_modified_slot(&self, pubkey: &Pubkey) -> Option<(AccountSharedData, Slot)> {
4267        self.load_slow(&self.ancestors, pubkey)
4268    }
4269
4270    fn load_slow(
4271        &self,
4272        ancestors: &Ancestors,
4273        pubkey: &Pubkey,
4274    ) -> Option<(AccountSharedData, Slot)> {
4275        // get_account (= primary this fn caller) may be called from on-chain Bank code even if we
4276        // try hard to use get_account_with_fixed_root for that purpose...
4277        // so pass safer LoadHint:Unspecified here as a fallback
4278        self.rc.accounts.load_without_fixed_root(ancestors, pubkey)
4279    }
4280
4281    fn load_slow_with_fixed_root(
4282        &self,
4283        ancestors: &Ancestors,
4284        pubkey: &Pubkey,
4285    ) -> Option<(AccountSharedData, Slot)> {
4286        self.rc.accounts.load_with_fixed_root(ancestors, pubkey)
4287    }
4288
4289    pub fn get_program_accounts(
4290        &self,
4291        program_id: &Pubkey,
4292        config: &ScanConfig,
4293    ) -> ScanResult<Vec<KeyedAccountSharedData>> {
4294        self.rc
4295            .accounts
4296            .load_by_program(&self.ancestors, self.bank_id, program_id, config)
4297    }
4298
4299    pub fn get_filtered_program_accounts<F: Fn(&AccountSharedData) -> bool>(
4300        &self,
4301        program_id: &Pubkey,
4302        filter: F,
4303        config: &ScanConfig,
4304    ) -> ScanResult<Vec<KeyedAccountSharedData>> {
4305        self.rc.accounts.load_by_program_with_filter(
4306            &self.ancestors,
4307            self.bank_id,
4308            program_id,
4309            filter,
4310            config,
4311        )
4312    }
4313
4314    pub fn get_filtered_indexed_accounts<F: Fn(&AccountSharedData) -> bool>(
4315        &self,
4316        index_key: &IndexKey,
4317        filter: F,
4318        config: &ScanConfig,
4319        byte_limit_for_scan: Option<usize>,
4320    ) -> ScanResult<Vec<KeyedAccountSharedData>> {
4321        self.rc.accounts.load_by_index_key_with_filter(
4322            &self.ancestors,
4323            self.bank_id,
4324            index_key,
4325            filter,
4326            config,
4327            byte_limit_for_scan,
4328        )
4329    }
4330
4331    pub fn account_indexes_include_key(&self, key: &Pubkey) -> bool {
4332        self.rc.accounts.account_indexes_include_key(key)
4333    }
4334
4335    /// Returns all the accounts this bank can load
4336    pub fn get_all_accounts(&self, sort_results: bool) -> ScanResult<Vec<PubkeyAccountSlot>> {
4337        self.rc
4338            .accounts
4339            .load_all(&self.ancestors, self.bank_id, sort_results)
4340    }
4341
4342    // Scans all the accounts this bank can load, applying `scan_func`
4343    pub fn scan_all_accounts<F>(&self, scan_func: F, sort_results: bool) -> ScanResult<()>
4344    where
4345        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
4346    {
4347        self.rc
4348            .accounts
4349            .scan_all(&self.ancestors, self.bank_id, scan_func, sort_results)
4350    }
4351
4352    pub fn get_program_accounts_modified_since_parent(
4353        &self,
4354        program_id: &Pubkey,
4355    ) -> Vec<KeyedAccountSharedData> {
4356        self.rc
4357            .accounts
4358            .load_by_program_slot(self.slot(), Some(program_id))
4359    }
4360
4361    pub fn get_transaction_logs(
4362        &self,
4363        address: Option<&Pubkey>,
4364    ) -> Option<Vec<TransactionLogInfo>> {
4365        self.transaction_log_collector
4366            .read()
4367            .unwrap()
4368            .get_logs_for_address(address)
4369    }
4370
4371    /// Returns all the accounts stored in this slot
4372    pub fn get_all_accounts_modified_since_parent(&self) -> Vec<KeyedAccountSharedData> {
4373        self.rc.accounts.load_by_program_slot(self.slot(), None)
4374    }
4375
4376    // if you want get_account_modified_since_parent without fixed_root, please define so...
4377    fn get_account_modified_since_parent_with_fixed_root(
4378        &self,
4379        pubkey: &Pubkey,
4380    ) -> Option<(AccountSharedData, Slot)> {
4381        let just_self: Ancestors = Ancestors::from(vec![self.slot()]);
4382        if let Some((account, slot)) = self.load_slow_with_fixed_root(&just_self, pubkey) {
4383            if slot == self.slot() {
4384                return Some((account, slot));
4385            }
4386        }
4387        None
4388    }
4389
4390    pub fn get_largest_accounts(
4391        &self,
4392        num: usize,
4393        filter_by_address: &HashSet<Pubkey>,
4394        filter: AccountAddressFilter,
4395        sort_results: bool,
4396    ) -> ScanResult<Vec<(Pubkey, u64)>> {
4397        self.rc.accounts.load_largest_accounts(
4398            &self.ancestors,
4399            self.bank_id,
4400            num,
4401            filter_by_address,
4402            filter,
4403            sort_results,
4404        )
4405    }
4406
4407    /// Return the accumulated executed transaction count
4408    pub fn transaction_count(&self) -> u64 {
4409        self.transaction_count.load(Relaxed)
4410    }
4411
4412    /// Returns the number of non-vote transactions processed without error
4413    /// since the most recent boot from snapshot or genesis.
4414    /// This value is not shared though the network, nor retained
4415    /// within snapshots, but is preserved in `Bank::new_from_parent`.
4416    pub fn non_vote_transaction_count_since_restart(&self) -> u64 {
4417        self.non_vote_transaction_count_since_restart.load(Relaxed)
4418    }
4419
4420    /// Return the transaction count executed only in this bank
4421    pub fn executed_transaction_count(&self) -> u64 {
4422        self.transaction_count()
4423            .saturating_sub(self.parent().map_or(0, |parent| parent.transaction_count()))
4424    }
4425
4426    pub fn transaction_error_count(&self) -> u64 {
4427        self.transaction_error_count.load(Relaxed)
4428    }
4429
4430    pub fn transaction_entries_count(&self) -> u64 {
4431        self.transaction_entries_count.load(Relaxed)
4432    }
4433
4434    pub fn transactions_per_entry_max(&self) -> u64 {
4435        self.transactions_per_entry_max.load(Relaxed)
4436    }
4437
4438    fn increment_transaction_count(&self, tx_count: u64) {
4439        self.transaction_count.fetch_add(tx_count, Relaxed);
4440    }
4441
4442    fn increment_non_vote_transaction_count_since_restart(&self, tx_count: u64) {
4443        self.non_vote_transaction_count_since_restart
4444            .fetch_add(tx_count, Relaxed);
4445    }
4446
4447    pub fn signature_count(&self) -> u64 {
4448        self.signature_count.load(Relaxed)
4449    }
4450
4451    fn increment_signature_count(&self, signature_count: u64) {
4452        self.signature_count.fetch_add(signature_count, Relaxed);
4453    }
4454
4455    pub fn get_signature_status_processed_since_parent(
4456        &self,
4457        signature: &Signature,
4458    ) -> Option<Result<()>> {
4459        if let Some((slot, status)) = self.get_signature_status_slot(signature) {
4460            if slot <= self.slot() {
4461                return Some(status);
4462            }
4463        }
4464        None
4465    }
4466
4467    pub fn get_signature_status_with_blockhash(
4468        &self,
4469        signature: &Signature,
4470        blockhash: &Hash,
4471    ) -> Option<Result<()>> {
4472        let rcache = self.status_cache.read().unwrap();
4473        rcache
4474            .get_status(signature, blockhash, &self.ancestors)
4475            .map(|v| v.1)
4476    }
4477
4478    pub fn get_committed_transaction_status_and_slot(
4479        &self,
4480        message_hash: &Hash,
4481        transaction_blockhash: &Hash,
4482    ) -> Option<(Slot, bool)> {
4483        let rcache = self.status_cache.read().unwrap();
4484        rcache
4485            .get_status(message_hash, transaction_blockhash, &self.ancestors)
4486            .map(|(slot, status)| (slot, status.is_ok()))
4487    }
4488
4489    pub fn get_signature_status_slot(&self, signature: &Signature) -> Option<(Slot, Result<()>)> {
4490        let rcache = self.status_cache.read().unwrap();
4491        rcache.get_status_any_blockhash(signature, &self.ancestors)
4492    }
4493
4494    pub fn get_signature_status(&self, signature: &Signature) -> Option<Result<()>> {
4495        self.get_signature_status_slot(signature).map(|v| v.1)
4496    }
4497
4498    pub fn has_signature(&self, signature: &Signature) -> bool {
4499        self.get_signature_status_slot(signature).is_some()
4500    }
4501
4502    /// Hash the `accounts` HashMap. This represents a validator's interpretation
4503    ///  of the delta of the ledger since the last vote and up to now
4504    fn hash_internal_state(&self) -> Hash {
4505        let measure_total = Measure::start("");
4506        let slot = self.slot();
4507
4508        let mut hash = hashv(&[
4509            self.parent_hash.as_ref(),
4510            &self.signature_count().to_le_bytes(),
4511            self.last_blockhash().as_ref(),
4512        ]);
4513
4514        let accounts_lt_hash_checksum = {
4515            let accounts_lt_hash = &*self.accounts_lt_hash.lock().unwrap();
4516            let lt_hash_bytes = bytemuck::must_cast_slice(&accounts_lt_hash.0 .0);
4517            hash = hashv(&[hash.as_ref(), lt_hash_bytes]);
4518            accounts_lt_hash.0.checksum()
4519        };
4520
4521        let buf = self
4522            .hard_forks
4523            .read()
4524            .unwrap()
4525            .get_hash_data(slot, self.parent_slot());
4526        if let Some(buf) = buf {
4527            let hard_forked_hash = hashv(&[hash.as_ref(), &buf]);
4528            warn!("hard fork at slot {slot} by hashing {buf:?}: {hash} => {hard_forked_hash}");
4529            hash = hard_forked_hash;
4530        }
4531
4532        #[cfg(feature = "dev-context-only-utils")]
4533        let hash_override = self
4534            .hash_overrides
4535            .lock()
4536            .unwrap()
4537            .get_bank_hash_override(slot)
4538            .copied()
4539            .inspect(|&hash_override| {
4540                if hash_override != hash {
4541                    info!(
4542                        "bank: slot: {}: overrode bank hash: {} with {}",
4543                        self.slot(),
4544                        hash,
4545                        hash_override
4546                    );
4547                }
4548            });
4549        // Avoid to optimize out `hash` along with the whole computation by super smart rustc.
4550        // hash_override is used by ledger-tool's simulate-block-production, which prefers
4551        // the actual bank freezing processing for accurate simulation.
4552        #[cfg(feature = "dev-context-only-utils")]
4553        let hash = hash_override.unwrap_or(std::hint::black_box(hash));
4554
4555        let bank_hash_stats = self.bank_hash_stats.load();
4556
4557        let total_us = measure_total.end_as_us();
4558
4559        datapoint_info!(
4560            "bank-hash_internal_state",
4561            ("slot", slot, i64),
4562            ("total_us", total_us, i64),
4563        );
4564        info!(
4565            "bank frozen: {slot} hash: {hash} signature_count: {} last_blockhash: {} \
4566             capitalization: {}, accounts_lt_hash checksum: {accounts_lt_hash_checksum}, stats: \
4567             {bank_hash_stats:?}",
4568            self.signature_count(),
4569            self.last_blockhash(),
4570            self.capitalization(),
4571        );
4572        hash
4573    }
4574
4575    pub fn collector_fees(&self) -> u64 {
4576        self.collector_fees.load(Relaxed)
4577    }
4578
4579    /// Used by ledger tool to run a final hash calculation once all ledger replay has completed.
4580    /// This should not be called by validator code.
4581    pub fn run_final_hash_calc(&self) {
4582        self.force_flush_accounts_cache();
4583        // note that this slot may not be a root
4584        _ = self.verify_accounts(
4585            VerifyAccountsHashConfig {
4586                require_rooted_bank: false,
4587            },
4588            None,
4589        );
4590    }
4591
4592    /// Verify the account state as part of startup, typically from a snapshot.
4593    ///
4594    /// This fn compares the calculated accounts lt hash against the stored value in the bank.
4595    ///
4596    /// Normal validator operation will calculate the accounts lt hash during index generation.
4597    /// Tests/ledger-tool may not have the calculated value from index generation (or the bank
4598    /// being verified is different from the snapshot/startup bank), and thus will be calculated in
4599    /// this function, using the accounts index for input, running in the foreground.
4600    ///
4601    /// Returns true if all is good.
4602    ///
4603    /// Only intended to be called at startup, or from tests/ledger-tool.
4604    #[must_use]
4605    fn verify_accounts(
4606        &self,
4607        config: VerifyAccountsHashConfig,
4608        calculated_accounts_lt_hash: Option<&AccountsLtHash>,
4609    ) -> bool {
4610        let accounts_db = &self.rc.accounts.accounts_db;
4611
4612        let slot = self.slot();
4613
4614        if config.require_rooted_bank && !accounts_db.accounts_index.is_alive_root(slot) {
4615            if let Some(parent) = self.parent() {
4616                info!(
4617                    "slot {slot} is not a root, so verify accounts hash on parent bank at slot {}",
4618                    parent.slot(),
4619                );
4620                // The calculated_accounts_lt_hash parameter is only valid for the current slot, so
4621                // we must fall back to calculating the accounts lt hash with the index.
4622                return parent.verify_accounts(config, None);
4623            } else {
4624                // this will result in mismatch errors
4625                // accounts hash calc doesn't include unrooted slots
4626                panic!("cannot verify accounts hash because slot {slot} is not a root");
4627            }
4628        }
4629
4630        fn check_lt_hash(
4631            expected_accounts_lt_hash: &AccountsLtHash,
4632            calculated_accounts_lt_hash: &AccountsLtHash,
4633        ) -> bool {
4634            let is_ok = calculated_accounts_lt_hash == expected_accounts_lt_hash;
4635            if !is_ok {
4636                let expected = expected_accounts_lt_hash.0.checksum();
4637                let calculated = calculated_accounts_lt_hash.0.checksum();
4638                error!(
4639                    "Verifying accounts failed: accounts lattice hashes do not match, expected: \
4640                     {expected}, calculated: {calculated}",
4641                );
4642            }
4643            is_ok
4644        }
4645
4646        info!("Verifying accounts...");
4647        let start = Instant::now();
4648        let expected_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap().clone();
4649        let is_ok = if let Some(calculated_accounts_lt_hash) = calculated_accounts_lt_hash {
4650            check_lt_hash(&expected_accounts_lt_hash, calculated_accounts_lt_hash)
4651        } else {
4652            let calculated_accounts_lt_hash =
4653                accounts_db.calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, slot);
4654            check_lt_hash(&expected_accounts_lt_hash, &calculated_accounts_lt_hash)
4655        };
4656        info!("Verifying accounts... Done in {:?}", start.elapsed());
4657        is_ok
4658    }
4659
4660    /// Get this bank's storages to use for snapshots.
4661    ///
4662    /// If a base slot is provided, return only the storages that are *higher* than this slot.
4663    pub fn get_snapshot_storages(&self, base_slot: Option<Slot>) -> Vec<Arc<AccountStorageEntry>> {
4664        // if a base slot is provided, request storages starting at the slot *after*
4665        let start_slot = base_slot.map_or(0, |slot| slot.saturating_add(1));
4666        // we want to *include* the storage at our slot
4667        let requested_slots = start_slot..=self.slot();
4668
4669        self.rc.accounts.accounts_db.get_storages(requested_slots).0
4670    }
4671
4672    #[must_use]
4673    fn verify_hash(&self) -> bool {
4674        assert!(self.is_frozen());
4675        let calculated_hash = self.hash_internal_state();
4676        let expected_hash = self.hash();
4677
4678        if calculated_hash == expected_hash {
4679            true
4680        } else {
4681            warn!(
4682                "verify failed: slot: {}, {} (calculated) != {} (expected)",
4683                self.slot(),
4684                calculated_hash,
4685                expected_hash
4686            );
4687            false
4688        }
4689    }
4690
4691    pub fn verify_transaction(
4692        &self,
4693        tx: VersionedTransaction,
4694        verification_mode: TransactionVerificationMode,
4695    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
4696        let enable_static_instruction_limit = self
4697            .feature_set
4698            .is_active(&agave_feature_set::static_instruction_limit::id());
4699        let sanitized_tx = {
4700            let size =
4701                bincode::serialized_size(&tx).map_err(|_| TransactionError::SanitizeFailure)?;
4702            if size > PACKET_DATA_SIZE as u64 {
4703                return Err(TransactionError::SanitizeFailure);
4704            }
4705            let message_hash = if verification_mode == TransactionVerificationMode::FullVerification
4706            {
4707                // SIMD-0160, check instruction limit before signature verificaton
4708                if enable_static_instruction_limit
4709                    && tx.message.instructions().len()
4710                        > solana_transaction_context::MAX_INSTRUCTION_TRACE_LENGTH
4711                {
4712                    return Err(solana_transaction_error::TransactionError::SanitizeFailure);
4713                }
4714                tx.verify_and_hash_message()?
4715            } else {
4716                tx.message.hash()
4717            };
4718
4719            RuntimeTransaction::try_create(
4720                tx,
4721                MessageHash::Precomputed(message_hash),
4722                None,
4723                self,
4724                self.get_reserved_account_keys(),
4725                enable_static_instruction_limit,
4726            )
4727        }?;
4728
4729        Ok(sanitized_tx)
4730    }
4731
4732    pub fn fully_verify_transaction(
4733        &self,
4734        tx: VersionedTransaction,
4735    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
4736        self.verify_transaction(tx, TransactionVerificationMode::FullVerification)
4737    }
4738
4739    /// Checks if the transaction violates the bank's reserved keys.
4740    /// This needs to be checked upon epoch boundary crosses because the
4741    /// reserved key set may have changed since the initial sanitization.
4742    pub fn check_reserved_keys(&self, tx: &impl SVMMessage) -> Result<()> {
4743        // Check keys against the reserved set - these failures simply require us
4744        // to re-sanitize the transaction. We do not need to drop the transaction.
4745        let reserved_keys = self.get_reserved_account_keys();
4746        for (index, key) in tx.account_keys().iter().enumerate() {
4747            if tx.is_writable(index) && reserved_keys.contains(key) {
4748                return Err(TransactionError::ResanitizationNeeded);
4749            }
4750        }
4751
4752        Ok(())
4753    }
4754
4755    /// Calculates and returns the capitalization.
4756    ///
4757    /// Panics if capitalization overflows a u64.
4758    ///
4759    /// Note, this is *very* expensive!  It walks the whole accounts index,
4760    /// account-by-account, summing each account's balance.
4761    ///
4762    /// Only intended to be called at startup by ledger-tool or tests.
4763    /// (cannot be made DCOU due to solana-program-test)
4764    pub fn calculate_capitalization_for_tests(&self) -> u64 {
4765        self.rc
4766            .accounts
4767            .accounts_db
4768            .calculate_capitalization_at_startup_from_index(&self.ancestors, self.slot())
4769    }
4770
4771    /// Sets the capitalization.
4772    ///
4773    /// Only intended to be called by ledger-tool or tests.
4774    /// (cannot be made DCOU due to solana-program-test)
4775    pub fn set_capitalization_for_tests(&self, capitalization: u64) {
4776        self.capitalization.store(capitalization, Relaxed);
4777    }
4778
4779    /// Returns the `SnapshotHash` for this bank's slot
4780    ///
4781    /// This fn is used at startup to verify the bank was rebuilt correctly.
4782    pub fn get_snapshot_hash(&self) -> SnapshotHash {
4783        SnapshotHash::new(self.accounts_lt_hash.lock().unwrap().0.checksum())
4784    }
4785
4786    pub fn load_account_into_read_cache(&self, key: &Pubkey) {
4787        self.rc
4788            .accounts
4789            .accounts_db
4790            .load_account_into_read_cache(&self.ancestors, key);
4791    }
4792
4793    /// A snapshot bank should be purged of 0 lamport accounts which are not part of the hash
4794    /// calculation and could shield other real accounts.
4795    pub fn verify_snapshot_bank(
4796        &self,
4797        skip_shrink: bool,
4798        force_clean: bool,
4799        latest_full_snapshot_slot: Slot,
4800        calculated_accounts_lt_hash: Option<&AccountsLtHash>,
4801    ) -> bool {
4802        let (verified_accounts, verify_accounts_time_us) = measure_us!({
4803            let should_verify_accounts = !self.rc.accounts.accounts_db.skip_initial_hash_calc;
4804            if should_verify_accounts {
4805                self.verify_accounts(
4806                    VerifyAccountsHashConfig {
4807                        require_rooted_bank: false,
4808                    },
4809                    calculated_accounts_lt_hash,
4810                )
4811            } else {
4812                info!("Verifying accounts... Skipped.");
4813                true
4814            }
4815        });
4816
4817        let (_, clean_time_us) = measure_us!({
4818            let should_clean = force_clean || (!skip_shrink && self.slot() > 0);
4819            if should_clean {
4820                info!("Cleaning...");
4821                // We cannot clean past the latest full snapshot's slot because we are about to
4822                // perform an accounts hash calculation *up to that slot*.  If we cleaned *past*
4823                // that slot, then accounts could be removed from older storages, which would
4824                // change the accounts hash.
4825                self.rc.accounts.accounts_db.clean_accounts(
4826                    Some(latest_full_snapshot_slot),
4827                    true,
4828                    self.epoch_schedule(),
4829                );
4830                info!("Cleaning... Done.");
4831            } else {
4832                info!("Cleaning... Skipped.");
4833            }
4834        });
4835
4836        let (_, shrink_time_us) = measure_us!({
4837            let should_shrink = !skip_shrink && self.slot() > 0;
4838            if should_shrink {
4839                info!("Shrinking...");
4840                self.rc.accounts.accounts_db.shrink_all_slots(
4841                    true,
4842                    self.epoch_schedule(),
4843                    // we cannot allow the snapshot slot to be shrunk
4844                    Some(self.slot()),
4845                );
4846                info!("Shrinking... Done.");
4847            } else {
4848                info!("Shrinking... Skipped.");
4849            }
4850        });
4851
4852        info!("Verifying bank...");
4853        let (verified_bank, verify_bank_time_us) = measure_us!(self.verify_hash());
4854        info!("Verifying bank... Done.");
4855
4856        datapoint_info!(
4857            "verify_snapshot_bank",
4858            ("clean_us", clean_time_us, i64),
4859            ("shrink_us", shrink_time_us, i64),
4860            ("verify_accounts_us", verify_accounts_time_us, i64),
4861            ("verify_bank_us", verify_bank_time_us, i64),
4862        );
4863
4864        verified_accounts && verified_bank
4865    }
4866
4867    /// Return the number of hashes per tick
4868    pub fn hashes_per_tick(&self) -> &Option<u64> {
4869        &self.hashes_per_tick
4870    }
4871
4872    /// Return the number of ticks per slot
4873    pub fn ticks_per_slot(&self) -> u64 {
4874        self.ticks_per_slot
4875    }
4876
4877    /// Return the number of slots per year
4878    pub fn slots_per_year(&self) -> f64 {
4879        self.slots_per_year
4880    }
4881
4882    /// Return the number of ticks since genesis.
4883    pub fn tick_height(&self) -> u64 {
4884        self.tick_height.load(Relaxed)
4885    }
4886
4887    /// Return the inflation parameters of the Bank
4888    pub fn inflation(&self) -> Inflation {
4889        *self.inflation.read().unwrap()
4890    }
4891
4892    /// Return the rent collector for this Bank
4893    pub fn rent_collector(&self) -> &RentCollector {
4894        &self.rent_collector
4895    }
4896
4897    /// Return the total capitalization of the Bank
4898    pub fn capitalization(&self) -> u64 {
4899        self.capitalization.load(Relaxed)
4900    }
4901
4902    /// Return this bank's max_tick_height
4903    pub fn max_tick_height(&self) -> u64 {
4904        self.max_tick_height
4905    }
4906
4907    /// Return the block_height of this bank
4908    pub fn block_height(&self) -> u64 {
4909        self.block_height
4910    }
4911
4912    /// Return the number of slots per epoch for the given epoch
4913    pub fn get_slots_in_epoch(&self, epoch: Epoch) -> u64 {
4914        self.epoch_schedule().get_slots_in_epoch(epoch)
4915    }
4916
4917    /// returns the epoch for which this bank's leader_schedule_slot_offset and slot would
4918    ///  need to cache leader_schedule
4919    pub fn get_leader_schedule_epoch(&self, slot: Slot) -> Epoch {
4920        self.epoch_schedule().get_leader_schedule_epoch(slot)
4921    }
4922
4923    /// Returns whether the specified epoch should use the new vote account
4924    /// keyed leader schedule
4925    pub fn should_use_vote_keyed_leader_schedule(&self, epoch: Epoch) -> Option<bool> {
4926        let effective_epoch = self
4927            .feature_set
4928            .activated_slot(&agave_feature_set::enable_vote_address_leader_schedule::id())
4929            .map(|activation_slot| {
4930                // If the feature was activated at genesis, then the new leader
4931                // schedule should be effective immediately in the first epoch
4932                if activation_slot == 0 {
4933                    return 0;
4934                }
4935
4936                // Calculate the epoch that the feature became activated in
4937                let activation_epoch = self.epoch_schedule.get_epoch(activation_slot);
4938
4939                // The effective epoch is the epoch immediately after the
4940                // activation epoch
4941                activation_epoch.wrapping_add(1)
4942            });
4943
4944        // Starting from the effective epoch, always use the new leader schedule
4945        if let Some(effective_epoch) = effective_epoch {
4946            return Some(epoch >= effective_epoch);
4947        }
4948
4949        // Calculate the max epoch we can cache a leader schedule for
4950        let max_cached_leader_schedule = self.get_leader_schedule_epoch(self.slot());
4951        if epoch <= max_cached_leader_schedule {
4952            // The feature cannot be effective by the specified epoch
4953            Some(false)
4954        } else {
4955            // Cannot determine if an epoch should use the new leader schedule if the
4956            // the epoch is too far in the future because we won't know if the feature
4957            // will have been activated by then or not.
4958            None
4959        }
4960    }
4961
4962    /// a bank-level cache of vote accounts and stake delegation info
4963    fn update_stakes_cache(
4964        &self,
4965        txs: &[impl SVMMessage],
4966        processing_results: &[TransactionProcessingResult],
4967    ) {
4968        debug_assert_eq!(txs.len(), processing_results.len());
4969        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
4970        txs.iter()
4971            .zip(processing_results)
4972            .filter_map(|(tx, processing_result)| {
4973                processing_result
4974                    .processed_transaction()
4975                    .map(|processed_tx| (tx, processed_tx))
4976            })
4977            .filter_map(|(tx, processed_tx)| {
4978                processed_tx
4979                    .executed_transaction()
4980                    .map(|executed_tx| (tx, executed_tx))
4981            })
4982            .filter(|(_, executed_tx)| executed_tx.was_successful())
4983            .flat_map(|(tx, executed_tx)| {
4984                let num_account_keys = tx.account_keys().len();
4985                let loaded_tx = &executed_tx.loaded_transaction;
4986                loaded_tx.accounts.iter().take(num_account_keys)
4987            })
4988            .for_each(|(pubkey, account)| {
4989                // note that this could get timed to: self.rc.accounts.accounts_db.stats.stakes_cache_check_and_store_us,
4990                //  but this code path is captured separately in ExecuteTimingType::UpdateStakesCacheUs
4991                self.stakes_cache
4992                    .check_and_store(pubkey, account, new_warmup_cooldown_rate_epoch);
4993            });
4994    }
4995
4996    /// current vote accounts for this bank along with the stake
4997    ///   attributed to each account
4998    pub fn vote_accounts(&self) -> Arc<VoteAccountsHashMap> {
4999        let stakes = self.stakes_cache.stakes();
5000        Arc::from(stakes.vote_accounts())
5001    }
5002
5003    /// Vote account for the given vote account pubkey.
5004    pub fn get_vote_account(&self, vote_account: &Pubkey) -> Option<VoteAccount> {
5005        let stakes = self.stakes_cache.stakes();
5006        let vote_account = stakes.vote_accounts().get(vote_account)?;
5007        Some(vote_account.clone())
5008    }
5009
5010    /// Get the EpochStakes for the current Bank::epoch
5011    pub fn current_epoch_stakes(&self) -> &VersionedEpochStakes {
5012        // The stakes for a given epoch (E) in self.epoch_stakes are keyed by leader schedule epoch
5013        // (E + 1) so the stakes for the current epoch are stored at self.epoch_stakes[E + 1]
5014        self.epoch_stakes
5015            .get(&self.epoch.saturating_add(1))
5016            .expect("Current epoch stakes must exist")
5017    }
5018
5019    /// Get the EpochStakes for a given epoch
5020    pub fn epoch_stakes(&self, epoch: Epoch) -> Option<&VersionedEpochStakes> {
5021        self.epoch_stakes.get(&epoch)
5022    }
5023
5024    pub fn epoch_stakes_map(&self) -> &HashMap<Epoch, VersionedEpochStakes> {
5025        &self.epoch_stakes
5026    }
5027
5028    /// Get the staked nodes map for the current Bank::epoch
5029    pub fn current_epoch_staked_nodes(&self) -> Arc<HashMap<Pubkey, u64>> {
5030        self.current_epoch_stakes().stakes().staked_nodes()
5031    }
5032
5033    pub fn epoch_staked_nodes(&self, epoch: Epoch) -> Option<Arc<HashMap<Pubkey, u64>>> {
5034        Some(self.epoch_stakes.get(&epoch)?.stakes().staked_nodes())
5035    }
5036
5037    /// Get the total epoch stake for the given epoch.
5038    pub fn epoch_total_stake(&self, epoch: Epoch) -> Option<u64> {
5039        self.epoch_stakes
5040            .get(&epoch)
5041            .map(|epoch_stakes| epoch_stakes.total_stake())
5042    }
5043
5044    /// Get the total epoch stake for the current Bank::epoch
5045    pub fn get_current_epoch_total_stake(&self) -> u64 {
5046        self.current_epoch_stakes().total_stake()
5047    }
5048
5049    /// vote accounts for the specific epoch along with the stake
5050    ///   attributed to each account
5051    pub fn epoch_vote_accounts(&self, epoch: Epoch) -> Option<&VoteAccountsHashMap> {
5052        let epoch_stakes = self.epoch_stakes.get(&epoch)?.stakes();
5053        Some(epoch_stakes.vote_accounts().as_ref())
5054    }
5055
5056    /// Get the vote accounts along with the stake attributed to each account
5057    /// for the current Bank::epoch
5058    pub fn get_current_epoch_vote_accounts(&self) -> &VoteAccountsHashMap {
5059        self.current_epoch_stakes()
5060            .stakes()
5061            .vote_accounts()
5062            .as_ref()
5063    }
5064
5065    /// Get the fixed authorized voter for the given vote account for the
5066    /// current epoch
5067    pub fn epoch_authorized_voter(&self, vote_account: &Pubkey) -> Option<&Pubkey> {
5068        self.epoch_stakes
5069            .get(&self.epoch)
5070            .expect("Epoch stakes for bank's own epoch must exist")
5071            .epoch_authorized_voters()
5072            .get(vote_account)
5073    }
5074
5075    /// Get the fixed set of vote accounts for the given node id for the
5076    /// current epoch
5077    pub fn epoch_vote_accounts_for_node_id(&self, node_id: &Pubkey) -> Option<&NodeVoteAccounts> {
5078        self.epoch_stakes
5079            .get(&self.epoch)
5080            .expect("Epoch stakes for bank's own epoch must exist")
5081            .node_id_to_vote_accounts()
5082            .get(node_id)
5083    }
5084
5085    /// Get the total stake belonging to vote accounts associated with the given node id for the
5086    /// given epoch.
5087    pub fn epoch_node_id_to_stake(&self, epoch: Epoch, node_id: &Pubkey) -> Option<u64> {
5088        self.epoch_stakes(epoch)
5089            .and_then(|epoch_stakes| epoch_stakes.node_id_to_stake(node_id))
5090    }
5091
5092    /// Get the fixed total stake of all vote accounts for current epoch
5093    pub fn total_epoch_stake(&self) -> u64 {
5094        self.epoch_stakes
5095            .get(&self.epoch)
5096            .expect("Epoch stakes for bank's own epoch must exist")
5097            .total_stake()
5098    }
5099
5100    /// Get the fixed stake of the given vote account for the current epoch
5101    pub fn epoch_vote_account_stake(&self, vote_account: &Pubkey) -> u64 {
5102        *self
5103            .epoch_vote_accounts(self.epoch())
5104            .expect("Bank epoch vote accounts must contain entry for the bank's own epoch")
5105            .get(vote_account)
5106            .map(|(stake, _)| stake)
5107            .unwrap_or(&0)
5108    }
5109
5110    /// given a slot, return the epoch and offset into the epoch this slot falls
5111    /// e.g. with a fixed number for slots_per_epoch, the calculation is simply:
5112    ///
5113    ///  ( slot/slots_per_epoch, slot % slots_per_epoch )
5114    ///
5115    pub fn get_epoch_and_slot_index(&self, slot: Slot) -> (Epoch, SlotIndex) {
5116        self.epoch_schedule().get_epoch_and_slot_index(slot)
5117    }
5118
5119    pub fn get_epoch_info(&self) -> EpochInfo {
5120        let absolute_slot = self.slot();
5121        let block_height = self.block_height();
5122        let (epoch, slot_index) = self.get_epoch_and_slot_index(absolute_slot);
5123        let slots_in_epoch = self.get_slots_in_epoch(epoch);
5124        let transaction_count = Some(self.transaction_count());
5125        EpochInfo {
5126            epoch,
5127            slot_index,
5128            slots_in_epoch,
5129            absolute_slot,
5130            block_height,
5131            transaction_count,
5132        }
5133    }
5134
5135    pub fn is_empty(&self) -> bool {
5136        !self.is_delta.load(Relaxed)
5137    }
5138
5139    pub fn add_mockup_builtin(
5140        &mut self,
5141        program_id: Pubkey,
5142        builtin_function: BuiltinFunctionWithContext,
5143    ) {
5144        self.add_builtin(
5145            program_id,
5146            "mockup",
5147            ProgramCacheEntry::new_builtin(self.slot, 0, builtin_function),
5148        );
5149    }
5150
5151    pub fn add_precompile(&mut self, program_id: &Pubkey) {
5152        debug!("Adding precompiled program {program_id}");
5153        self.add_precompiled_account(program_id);
5154        debug!("Added precompiled program {program_id:?}");
5155    }
5156
5157    // Call AccountsDb::clean_accounts()
5158    //
5159    // This fn is meant to be called by the snapshot handler in Accounts Background Service.  If
5160    // calling from elsewhere, ensure the same invariants hold/expectations are met.
5161    pub(crate) fn clean_accounts(&self) {
5162        // Don't clean the slot we're snapshotting because it may have zero-lamport
5163        // accounts that were included in the bank delta hash when the bank was frozen,
5164        // and if we clean them here, any newly created snapshot's hash for this bank
5165        // may not match the frozen hash.
5166        //
5167        // So when we're snapshotting, the highest slot to clean is lowered by one.
5168        let highest_slot_to_clean = self.slot().saturating_sub(1);
5169
5170        self.rc.accounts.accounts_db.clean_accounts(
5171            Some(highest_slot_to_clean),
5172            false,
5173            self.epoch_schedule(),
5174        );
5175    }
5176
5177    pub fn print_accounts_stats(&self) {
5178        self.rc.accounts.accounts_db.print_accounts_stats("");
5179    }
5180
5181    pub fn shrink_candidate_slots(&self) -> usize {
5182        self.rc
5183            .accounts
5184            .accounts_db
5185            .shrink_candidate_slots(self.epoch_schedule())
5186    }
5187
5188    pub(crate) fn shrink_ancient_slots(&self) {
5189        self.rc
5190            .accounts
5191            .accounts_db
5192            .shrink_ancient_slots(self.epoch_schedule())
5193    }
5194
5195    pub fn read_cost_tracker(&self) -> LockResult<RwLockReadGuard<'_, CostTracker>> {
5196        self.cost_tracker.read()
5197    }
5198
5199    pub fn write_cost_tracker(&self) -> LockResult<RwLockWriteGuard<'_, CostTracker>> {
5200        self.cost_tracker.write()
5201    }
5202
5203    // Check if the wallclock time from bank creation to now has exceeded the allotted
5204    // time for transaction processing
5205    pub fn should_bank_still_be_processing_txs(
5206        bank_creation_time: &Instant,
5207        max_tx_ingestion_nanos: u128,
5208    ) -> bool {
5209        // Do this check outside of the PoH lock, hence not a method on PohRecorder
5210        bank_creation_time.elapsed().as_nanos() <= max_tx_ingestion_nanos
5211    }
5212
5213    pub fn deactivate_feature(&mut self, id: &Pubkey) {
5214        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
5215        feature_set.active_mut().remove(id);
5216        feature_set.inactive_mut().insert(*id);
5217        self.feature_set = Arc::new(feature_set);
5218    }
5219
5220    pub fn activate_feature(&mut self, id: &Pubkey) {
5221        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
5222        feature_set.inactive_mut().remove(id);
5223        feature_set.active_mut().insert(*id, 0);
5224        self.feature_set = Arc::new(feature_set);
5225    }
5226
5227    pub fn fill_bank_with_ticks_for_tests(&self) {
5228        self.do_fill_bank_with_ticks_for_tests(&BankWithScheduler::no_scheduler_available())
5229    }
5230
5231    pub(crate) fn do_fill_bank_with_ticks_for_tests(&self, scheduler: &InstalledSchedulerRwLock) {
5232        if self.tick_height.load(Relaxed) < self.max_tick_height {
5233            let last_blockhash = self.last_blockhash();
5234            while self.last_blockhash() == last_blockhash {
5235                self.register_tick(&Hash::new_unique(), scheduler)
5236            }
5237        } else {
5238            warn!("Bank already reached max tick height, cannot fill it with more ticks");
5239        }
5240    }
5241
5242    /// Get a set of all actively reserved account keys that are not allowed to
5243    /// be write-locked during transaction processing.
5244    pub fn get_reserved_account_keys(&self) -> &HashSet<Pubkey> {
5245        &self.reserved_account_keys.active
5246    }
5247
5248    /// Compute and apply all activated features, initialize the transaction
5249    /// processor, and recalculate partitioned rewards if needed
5250    fn initialize_after_snapshot_restore<F, TP>(&mut self, rewards_thread_pool_builder: F)
5251    where
5252        F: FnOnce() -> TP,
5253        TP: std::borrow::Borrow<ThreadPool>,
5254    {
5255        self.transaction_processor =
5256            TransactionBatchProcessor::new_uninitialized(self.slot, self.epoch);
5257        if let Some(compute_budget) = &self.compute_budget {
5258            self.transaction_processor
5259                .set_execution_cost(compute_budget.to_cost());
5260        }
5261
5262        self.compute_and_apply_features_after_snapshot_restore();
5263
5264        self.recalculate_partitioned_rewards_if_active(rewards_thread_pool_builder);
5265
5266        self.transaction_processor
5267            .fill_missing_sysvar_cache_entries(self);
5268    }
5269
5270    /// Compute and apply all activated features and also add accounts for builtins
5271    fn compute_and_apply_genesis_features(&mut self) {
5272        // Update the feature set to include all features active at this slot
5273        let feature_set = self.compute_active_feature_set(false).0;
5274        self.feature_set = Arc::new(feature_set);
5275
5276        // Apply rent deprecation feature if it's active at genesis
5277        // After feature cleanup, assert that rent exemption threshold is 1.0
5278        if self
5279            .feature_set
5280            .is_active(&feature_set::deprecate_rent_exemption_threshold::id())
5281        {
5282            self.rent_collector.deprecate_rent_exemption_threshold();
5283        }
5284
5285        // Add built-in program accounts to the bank if they don't already exist
5286        self.add_builtin_program_accounts();
5287
5288        self.apply_activated_features();
5289    }
5290
5291    /// Compute and apply all activated features but do not add built-in
5292    /// accounts because we shouldn't modify accounts db for a completed bank
5293    fn compute_and_apply_features_after_snapshot_restore(&mut self) {
5294        // Update the feature set to include all features active at this slot
5295        let feature_set = self.compute_active_feature_set(false).0;
5296        self.feature_set = Arc::new(feature_set);
5297
5298        self.apply_activated_features();
5299    }
5300
5301    /// This is called from each epoch boundary
5302    fn compute_and_apply_new_feature_activations(&mut self) {
5303        let include_pending = true;
5304        let (feature_set, new_feature_activations) =
5305            self.compute_active_feature_set(include_pending);
5306        self.feature_set = Arc::new(feature_set);
5307
5308        // Update activation slot of features in `new_feature_activations`
5309        for feature_id in new_feature_activations.iter() {
5310            if let Some(mut account) = self.get_account_with_fixed_root(feature_id) {
5311                if let Some(mut feature) = feature::state::from_account(&account) {
5312                    feature.activated_at = Some(self.slot());
5313                    if feature::state::to_account(&feature, &mut account).is_some() {
5314                        self.store_account(feature_id, &account);
5315                    }
5316                    info!("Feature {} activated at slot {}", feature_id, self.slot());
5317                }
5318            }
5319        }
5320
5321        // Update active set of reserved account keys which are not allowed to be write locked
5322        self.reserved_account_keys = {
5323            let mut reserved_keys = ReservedAccountKeys::clone(&self.reserved_account_keys);
5324            reserved_keys.update_active_set(&self.feature_set);
5325            Arc::new(reserved_keys)
5326        };
5327
5328        if new_feature_activations.contains(&feature_set::deprecate_rent_exemption_threshold::id())
5329        {
5330            self.rent_collector.deprecate_rent_exemption_threshold();
5331            self.update_rent();
5332        }
5333
5334        if new_feature_activations.contains(&feature_set::pico_inflation::id()) {
5335            *self.inflation.write().unwrap() = Inflation::pico();
5336            self.fee_rate_governor.burn_percent = solana_fee_calculator::DEFAULT_BURN_PERCENT; // 50% fee burn
5337            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
5338        }
5339
5340        if !new_feature_activations.is_disjoint(&self.feature_set.full_inflation_features_enabled())
5341        {
5342            *self.inflation.write().unwrap() = Inflation::full();
5343            self.fee_rate_governor.burn_percent = solana_fee_calculator::DEFAULT_BURN_PERCENT; // 50% fee burn
5344            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
5345        }
5346
5347        self.apply_new_builtin_program_feature_transitions(&new_feature_activations);
5348
5349        if new_feature_activations.contains(&feature_set::raise_block_limits_to_100m::id()) {
5350            let block_cost_limit = simd_0286_block_limits();
5351            let mut cost_tracker = self.write_cost_tracker().unwrap();
5352            let account_cost_limit = cost_tracker.get_account_limit();
5353            let vote_cost_limit = cost_tracker.get_vote_limit();
5354            cost_tracker.set_limits(account_cost_limit, block_cost_limit, vote_cost_limit);
5355            drop(cost_tracker);
5356
5357            if self
5358                .feature_set
5359                .is_active(&feature_set::raise_account_cu_limit::id())
5360            {
5361                self.apply_simd_0306_cost_tracker_changes();
5362            }
5363        }
5364
5365        if new_feature_activations.contains(&feature_set::raise_account_cu_limit::id()) {
5366            self.apply_simd_0306_cost_tracker_changes();
5367        }
5368
5369        if new_feature_activations.contains(&feature_set::vote_state_v4::id()) {
5370            if let Err(e) = self.upgrade_core_bpf_program(
5371                &solana_sdk_ids::stake::id(),
5372                &feature_set::vote_state_v4::stake_program_buffer::id(),
5373                "upgrade_stake_program_for_vote_state_v4",
5374            ) {
5375                error!("Failed to upgrade Core BPF Stake program: {e}");
5376            }
5377        }
5378        if new_feature_activations.contains(&feature_set::increase_cpi_account_info_limit::id()) {
5379            self.apply_simd_0339_invoke_cost_changes();
5380        }
5381
5382        if new_feature_activations.contains(&feature_set::replace_spl_token_with_p_token::id()) {
5383            if let Err(e) = self.upgrade_loader_v2_program_with_loader_v3_program(
5384                &feature_set::replace_spl_token_with_p_token::SPL_TOKEN_PROGRAM_ID,
5385                &feature_set::replace_spl_token_with_p_token::PTOKEN_PROGRAM_BUFFER,
5386                self.feature_set
5387                    .is_active(&relax_programdata_account_check_migration::id()),
5388                "replace_spl_token_with_p_token",
5389            ) {
5390                warn!(
5391                    "Failed to replace SPL Token with p-token buffer '{}': {e}",
5392                    feature_set::replace_spl_token_with_p_token::PTOKEN_PROGRAM_BUFFER,
5393                );
5394            }
5395        }
5396    }
5397
5398    fn apply_new_builtin_program_feature_transitions(
5399        &mut self,
5400        new_feature_activations: &AHashSet<Pubkey>,
5401    ) {
5402        for builtin in BUILTINS.iter() {
5403            if let Some(feature_id) = builtin.enable_feature_id {
5404                if new_feature_activations.contains(&feature_id) {
5405                    self.add_builtin(
5406                        builtin.program_id,
5407                        builtin.name,
5408                        ProgramCacheEntry::new_builtin(
5409                            self.feature_set.activated_slot(&feature_id).unwrap_or(0),
5410                            builtin.name.len(),
5411                            builtin.entrypoint,
5412                        ),
5413                    );
5414                }
5415            }
5416
5417            if let Some(core_bpf_migration_config) = &builtin.core_bpf_migration_config {
5418                // If the builtin is set to be migrated to Core BPF on feature
5419                // activation, perform the migration which will remove it from
5420                // the builtins list and the cache.
5421                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
5422                    if let Err(e) = self.migrate_builtin_to_core_bpf(
5423                        &builtin.program_id,
5424                        core_bpf_migration_config,
5425                        self.feature_set
5426                            .is_active(&relax_programdata_account_check_migration::id()),
5427                    ) {
5428                        warn!(
5429                            "Failed to migrate builtin {} to Core BPF: {}",
5430                            builtin.name, e
5431                        );
5432                    }
5433                }
5434            };
5435        }
5436
5437        // Migrate any necessary stateless builtins to core BPF.
5438        // Stateless builtins do not have an `enable_feature_id` since they
5439        // do not exist on-chain.
5440        for stateless_builtin in STATELESS_BUILTINS.iter() {
5441            if let Some(core_bpf_migration_config) = &stateless_builtin.core_bpf_migration_config {
5442                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
5443                    if let Err(e) = self.migrate_builtin_to_core_bpf(
5444                        &stateless_builtin.program_id,
5445                        core_bpf_migration_config,
5446                        self.feature_set
5447                            .is_active(&relax_programdata_account_check_migration::id()),
5448                    ) {
5449                        warn!(
5450                            "Failed to migrate stateless builtin {} to Core BPF: {}",
5451                            stateless_builtin.name, e
5452                        );
5453                    }
5454                }
5455            }
5456        }
5457
5458        for precompile in get_precompiles() {
5459            if let Some(feature_id) = &precompile.feature {
5460                if new_feature_activations.contains(feature_id) {
5461                    self.add_precompile(&precompile.program_id);
5462                }
5463            }
5464        }
5465    }
5466
5467    fn adjust_sysvar_balance_for_rent(&self, account: &mut AccountSharedData) {
5468        account.set_lamports(
5469            self.get_minimum_balance_for_rent_exemption(account.data().len())
5470                .max(account.lamports()),
5471        );
5472    }
5473
5474    /// Compute the active feature set based on the current bank state,
5475    /// and return it together with the set of newly activated features.
5476    fn compute_active_feature_set(&self, include_pending: bool) -> (FeatureSet, AHashSet<Pubkey>) {
5477        let mut active = self.feature_set.active().clone();
5478        let mut inactive = AHashSet::new();
5479        let mut pending = AHashSet::new();
5480        let slot = self.slot();
5481
5482        for feature_id in self.feature_set.inactive() {
5483            let mut activated = None;
5484            if let Some(account) = self.get_account_with_fixed_root(feature_id) {
5485                if let Some(feature) = feature::state::from_account(&account) {
5486                    match feature.activated_at {
5487                        None if include_pending => {
5488                            // Feature activation is pending
5489                            pending.insert(*feature_id);
5490                            activated = Some(slot);
5491                        }
5492                        Some(activation_slot) if slot >= activation_slot => {
5493                            // Feature has been activated already
5494                            activated = Some(activation_slot);
5495                        }
5496                        _ => {}
5497                    }
5498                }
5499            }
5500            if let Some(slot) = activated {
5501                active.insert(*feature_id, slot);
5502            } else {
5503                inactive.insert(*feature_id);
5504            }
5505        }
5506
5507        (FeatureSet::new(active, inactive), pending)
5508    }
5509
5510    /// If `feature_id` is pending to be activated at the next epoch boundary, return
5511    /// the first slot at which it will be active (the epoch boundary).
5512    pub fn compute_pending_activation_slot(&self, feature_id: &Pubkey) -> Option<Slot> {
5513        let account = self.get_account_with_fixed_root(feature_id)?;
5514        let feature = feature::from_account(&account)?;
5515        if feature.activated_at.is_some() {
5516            // Feature is already active
5517            return None;
5518        }
5519        // Feature will be active at the next epoch boundary
5520        let active_epoch = self.epoch + 1;
5521        Some(self.epoch_schedule.get_first_slot_in_epoch(active_epoch))
5522    }
5523
5524    fn add_active_builtin_programs(&mut self) {
5525        for builtin in BUILTINS.iter() {
5526            // The `builtin_is_bpf` flag is used to handle the case where a
5527            // builtin is scheduled to be enabled by one feature gate and
5528            // later migrated to Core BPF by another.
5529            //
5530            // There should never be a case where a builtin is set to be
5531            // migrated to Core BPF and is also set to be enabled on feature
5532            // activation on the same feature gate. However, the
5533            // `builtin_is_bpf` flag will handle this case as well, electing
5534            // to first attempt the migration to Core BPF.
5535            //
5536            // The migration to Core BPF will fail gracefully because the
5537            // program account will not exist. The builtin will subsequently
5538            // be enabled, but it will never be migrated to Core BPF.
5539            //
5540            // Using the same feature gate for both enabling and migrating a
5541            // builtin to Core BPF should be strictly avoided.
5542            let builtin_is_bpf = builtin.core_bpf_migration_config.is_some() && {
5543                self.get_account(&builtin.program_id)
5544                    .map(|a| a.owner() == &bpf_loader_upgradeable::id())
5545                    .unwrap_or(false)
5546            };
5547
5548            // If the builtin has already been migrated to Core BPF, do not
5549            // add it to the bank's builtins.
5550            if builtin_is_bpf {
5551                continue;
5552            }
5553
5554            let builtin_is_active = builtin
5555                .enable_feature_id
5556                .map(|feature_id| self.feature_set.is_active(&feature_id))
5557                .unwrap_or(true);
5558
5559            if builtin_is_active {
5560                let activation_slot = builtin
5561                    .enable_feature_id
5562                    .and_then(|feature_id| self.feature_set.activated_slot(&feature_id))
5563                    .unwrap_or(0);
5564                self.transaction_processor.add_builtin(
5565                    builtin.program_id,
5566                    ProgramCacheEntry::new_builtin(
5567                        activation_slot,
5568                        builtin.name.len(),
5569                        builtin.entrypoint,
5570                    ),
5571                );
5572            }
5573        }
5574    }
5575
5576    fn add_builtin_program_accounts(&mut self) {
5577        for builtin in BUILTINS.iter() {
5578            // The `builtin_is_bpf` flag is used to handle the case where a
5579            // builtin is scheduled to be enabled by one feature gate and
5580            // later migrated to Core BPF by another.
5581            //
5582            // There should never be a case where a builtin is set to be
5583            // migrated to Core BPF and is also set to be enabled on feature
5584            // activation on the same feature gate. However, the
5585            // `builtin_is_bpf` flag will handle this case as well, electing
5586            // to first attempt the migration to Core BPF.
5587            //
5588            // The migration to Core BPF will fail gracefully because the
5589            // program account will not exist. The builtin will subsequently
5590            // be enabled, but it will never be migrated to Core BPF.
5591            //
5592            // Using the same feature gate for both enabling and migrating a
5593            // builtin to Core BPF should be strictly avoided.
5594            let builtin_is_bpf = builtin.core_bpf_migration_config.is_some() && {
5595                self.get_account(&builtin.program_id)
5596                    .map(|a| a.owner() == &bpf_loader_upgradeable::id())
5597                    .unwrap_or(false)
5598            };
5599
5600            // If the builtin has already been migrated to Core BPF, do not
5601            // add it to the bank's builtins.
5602            if builtin_is_bpf {
5603                continue;
5604            }
5605
5606            let builtin_is_active = builtin
5607                .enable_feature_id
5608                .map(|feature_id| self.feature_set.is_active(&feature_id))
5609                .unwrap_or(true);
5610
5611            if builtin_is_active {
5612                self.add_builtin_account(builtin.name, &builtin.program_id);
5613            }
5614        }
5615
5616        for precompile in get_precompiles() {
5617            let precompile_is_active = precompile
5618                .feature
5619                .as_ref()
5620                .map(|feature_id| self.feature_set.is_active(feature_id))
5621                .unwrap_or(true);
5622
5623            if precompile_is_active {
5624                self.add_precompile(&precompile.program_id);
5625            }
5626        }
5627    }
5628
5629    /// Use to replace programs by feature activation
5630    #[allow(dead_code)]
5631    fn replace_program_account(
5632        &mut self,
5633        old_address: &Pubkey,
5634        new_address: &Pubkey,
5635        datapoint_name: &'static str,
5636    ) {
5637        if let Some(old_account) = self.get_account_with_fixed_root(old_address) {
5638            if let Some(new_account) = self.get_account_with_fixed_root(new_address) {
5639                datapoint_info!(datapoint_name, ("slot", self.slot, i64));
5640
5641                // Burn lamports in the old account
5642                self.capitalization
5643                    .fetch_sub(old_account.lamports(), Relaxed);
5644
5645                // Transfer new account to old account
5646                self.store_account(old_address, &new_account);
5647
5648                // Clear new account
5649                self.store_account(new_address, &AccountSharedData::default());
5650
5651                // Unload a program from the bank's cache
5652                self.transaction_processor
5653                    .global_program_cache
5654                    .write()
5655                    .unwrap()
5656                    .remove_programs([*old_address].into_iter());
5657
5658                self.calculate_and_update_accounts_data_size_delta_off_chain(
5659                    old_account.data().len(),
5660                    new_account.data().len(),
5661                );
5662            }
5663        }
5664    }
5665
5666    /// Calculates the accounts data size of all accounts
5667    ///
5668    /// Panics if total overflows a u64.
5669    ///
5670    /// Note, this may be *very* expensive, as *all* accounts are collected
5671    /// into a Vec before summing each account's data size.
5672    ///
5673    /// Only intended to be called by tests or when the number of accounts is small.
5674    pub fn calculate_accounts_data_size(&self) -> ScanResult<u64> {
5675        let accounts = self.get_all_accounts(false)?;
5676        let accounts_data_size = accounts
5677            .into_iter()
5678            .map(|(_pubkey, account, _slot)| account.data().len() as u64)
5679            .try_fold(0, u64::checked_add)
5680            .expect("accounts data size cannot overflow");
5681        Ok(accounts_data_size)
5682    }
5683
5684    pub fn is_in_slot_hashes_history(&self, slot: &Slot) -> bool {
5685        if slot < &self.slot {
5686            if let Ok(slot_hashes) = self.transaction_processor.sysvar_cache().get_slot_hashes() {
5687                return slot_hashes.get(slot).is_some();
5688            }
5689        }
5690        false
5691    }
5692
5693    pub fn check_program_modification_slot(&self) -> bool {
5694        self.check_program_modification_slot
5695    }
5696
5697    pub fn set_check_program_modification_slot(&mut self, check: bool) {
5698        self.check_program_modification_slot = check;
5699    }
5700
5701    pub fn fee_structure(&self) -> &FeeStructure {
5702        &self.fee_structure
5703    }
5704
5705    pub fn parent_block_id(&self) -> Option<Hash> {
5706        self.parent().and_then(|p| p.block_id())
5707    }
5708
5709    pub fn block_id(&self) -> Option<Hash> {
5710        *self.block_id.read().unwrap()
5711    }
5712
5713    pub fn set_block_id(&self, block_id: Option<Hash>) {
5714        *self.block_id.write().unwrap() = block_id;
5715    }
5716
5717    pub fn compute_budget(&self) -> Option<ComputeBudget> {
5718        self.compute_budget
5719    }
5720
5721    pub fn add_builtin(&self, program_id: Pubkey, name: &str, builtin: ProgramCacheEntry) {
5722        debug!("Adding program {name} under {program_id:?}");
5723        self.add_builtin_account(name, &program_id);
5724        self.transaction_processor.add_builtin(program_id, builtin);
5725        debug!("Added program {name} under {program_id:?}");
5726    }
5727
5728    // NOTE: must hold idempotent for the same set of arguments
5729    /// Add a builtin program account
5730    fn add_builtin_account(&self, name: &str, program_id: &Pubkey) {
5731        let existing_genuine_program =
5732            self.get_account_with_fixed_root(program_id)
5733                .and_then(|account| {
5734                    // it's very unlikely to be squatted at program_id as non-system account because of burden to
5735                    // find victim's pubkey/hash. So, when account.owner is indeed native_loader's, it's
5736                    // safe to assume it's a genuine program.
5737                    if native_loader::check_id(account.owner()) {
5738                        Some(account)
5739                    } else {
5740                        // malicious account is pre-occupying at program_id
5741                        self.burn_and_purge_account(program_id, account);
5742                        None
5743                    }
5744                });
5745
5746        // introducing builtin program
5747        if existing_genuine_program.is_some() {
5748            // The existing account is sufficient
5749            return;
5750        }
5751
5752        assert!(
5753            !self.freeze_started(),
5754            "Can't change frozen bank by adding not-existing new builtin program ({name}, \
5755             {program_id}). Maybe, inconsistent program activation is detected on snapshot \
5756             restore?"
5757        );
5758
5759        // Add a bogus executable builtin account, which will be loaded and ignored.
5760        let (lamports, rent_epoch) =
5761            self.inherit_specially_retained_account_fields(&existing_genuine_program);
5762        let account: AccountSharedData = AccountSharedData::from(Account {
5763            lamports,
5764            data: name.as_bytes().to_vec(),
5765            owner: solana_sdk_ids::native_loader::id(),
5766            executable: true,
5767            rent_epoch,
5768        });
5769        self.store_account_and_update_capitalization(program_id, &account);
5770    }
5771
5772    pub fn get_bank_hash_stats(&self) -> BankHashStats {
5773        self.bank_hash_stats.load()
5774    }
5775
5776    pub fn clear_epoch_rewards_cache(&self) {
5777        self.epoch_rewards_calculation_cache.lock().unwrap().clear();
5778    }
5779
5780    /// Sets the accounts lt hash, only to be used by SnapshotMinimizer
5781    pub fn set_accounts_lt_hash_for_snapshot_minimizer(&self, accounts_lt_hash: AccountsLtHash) {
5782        *self.accounts_lt_hash.lock().unwrap() = accounts_lt_hash;
5783    }
5784
5785    /// Return total transaction fee collected
5786    pub fn get_collector_fee_details(&self) -> CollectorFeeDetails {
5787        self.collector_fee_details.read().unwrap().clone()
5788    }
5789}
5790
5791impl InvokeContextCallback for Bank {
5792    fn get_epoch_stake(&self) -> u64 {
5793        self.get_current_epoch_total_stake()
5794    }
5795
5796    fn get_epoch_stake_for_vote_account(&self, vote_address: &Pubkey) -> u64 {
5797        self.get_current_epoch_vote_accounts()
5798            .get(vote_address)
5799            .map(|(stake, _)| *stake)
5800            .unwrap_or(0)
5801    }
5802
5803    fn is_precompile(&self, program_id: &Pubkey) -> bool {
5804        is_precompile(program_id, |feature_id: &Pubkey| {
5805            self.feature_set.is_active(feature_id)
5806        })
5807    }
5808
5809    fn process_precompile(
5810        &self,
5811        program_id: &Pubkey,
5812        data: &[u8],
5813        instruction_datas: Vec<&[u8]>,
5814    ) -> std::result::Result<(), PrecompileError> {
5815        if let Some(precompile) = get_precompile(program_id, |feature_id: &Pubkey| {
5816            self.feature_set.is_active(feature_id)
5817        }) {
5818            precompile.verify(data, &instruction_datas, &self.feature_set)
5819        } else {
5820            Err(PrecompileError::InvalidPublicKey)
5821        }
5822    }
5823}
5824
5825impl TransactionProcessingCallback for Bank {
5826    fn get_account_shared_data(&self, pubkey: &Pubkey) -> Option<(AccountSharedData, Slot)> {
5827        self.rc
5828            .accounts
5829            .accounts_db
5830            .load_with_fixed_root(&self.ancestors, pubkey)
5831    }
5832
5833    fn inspect_account(&self, address: &Pubkey, account_state: AccountState, is_writable: bool) {
5834        self.inspect_account_for_accounts_lt_hash(address, &account_state, is_writable);
5835    }
5836}
5837
5838impl fmt::Debug for Bank {
5839    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
5840        f.debug_struct("Bank")
5841            .field("slot", &self.slot)
5842            .field("bank_id", &self.bank_id)
5843            .field("block_height", &self.block_height)
5844            .field("parent_slot", &self.parent_slot)
5845            .field("capitalization", &self.capitalization())
5846            .finish_non_exhaustive()
5847    }
5848}
5849
5850#[cfg(feature = "dev-context-only-utils")]
5851impl Bank {
5852    pub fn wrap_with_bank_forks_for_tests(self) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5853        let bank_forks = BankForks::new_rw_arc(self);
5854        let bank = bank_forks.read().unwrap().root_bank();
5855        (bank, bank_forks)
5856    }
5857
5858    pub fn default_for_tests() -> Self {
5859        let accounts_db = AccountsDb::default_for_tests();
5860        let accounts = Accounts::new(Arc::new(accounts_db));
5861        Self::default_with_accounts(accounts)
5862    }
5863
5864    pub fn new_with_bank_forks_for_tests(
5865        genesis_config: &GenesisConfig,
5866    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5867        let bank = Self::new_for_tests(genesis_config);
5868        bank.wrap_with_bank_forks_for_tests()
5869    }
5870
5871    pub fn new_for_tests(genesis_config: &GenesisConfig) -> Self {
5872        Self::new_with_config_for_tests(genesis_config, BankTestConfig::default())
5873    }
5874
5875    pub fn new_with_mockup_builtin_for_tests(
5876        genesis_config: &GenesisConfig,
5877        program_id: Pubkey,
5878        builtin_function: BuiltinFunctionWithContext,
5879    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5880        let mut bank = Self::new_for_tests(genesis_config);
5881        bank.add_mockup_builtin(program_id, builtin_function);
5882        bank.wrap_with_bank_forks_for_tests()
5883    }
5884
5885    pub fn new_no_wallclock_throttle_for_tests(
5886        genesis_config: &GenesisConfig,
5887    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
5888        let mut bank = Self::new_for_tests(genesis_config);
5889
5890        bank.ns_per_slot = u128::MAX;
5891        bank.wrap_with_bank_forks_for_tests()
5892    }
5893
5894    pub fn new_with_config_for_tests(
5895        genesis_config: &GenesisConfig,
5896        test_config: BankTestConfig,
5897    ) -> Self {
5898        Self::new_with_paths_for_tests(
5899            genesis_config,
5900            Arc::new(RuntimeConfig::default()),
5901            test_config,
5902            Vec::new(),
5903        )
5904    }
5905
5906    pub fn new_with_paths_for_tests(
5907        genesis_config: &GenesisConfig,
5908        runtime_config: Arc<RuntimeConfig>,
5909        test_config: BankTestConfig,
5910        paths: Vec<PathBuf>,
5911    ) -> Self {
5912        Self::new_from_genesis(
5913            genesis_config,
5914            runtime_config,
5915            paths,
5916            None,
5917            test_config.accounts_db_config,
5918            None,
5919            Some(Pubkey::new_unique()),
5920            Arc::default(),
5921            None,
5922            None,
5923        )
5924    }
5925
5926    pub fn new_for_benches(genesis_config: &GenesisConfig) -> Self {
5927        Self::new_with_paths_for_benches(genesis_config, Vec::new())
5928    }
5929
5930    /// Intended for use by benches only.
5931    /// create new bank with the given config and paths.
5932    pub fn new_with_paths_for_benches(genesis_config: &GenesisConfig, paths: Vec<PathBuf>) -> Self {
5933        Self::new_from_genesis(
5934            genesis_config,
5935            Arc::<RuntimeConfig>::default(),
5936            paths,
5937            None,
5938            ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS,
5939            None,
5940            Some(Pubkey::new_unique()),
5941            Arc::default(),
5942            None,
5943            None,
5944        )
5945    }
5946
5947    pub fn new_from_parent_with_bank_forks(
5948        bank_forks: &RwLock<BankForks>,
5949        parent: Arc<Bank>,
5950        collector_id: &Pubkey,
5951        slot: Slot,
5952    ) -> Arc<Self> {
5953        let bank = Bank::new_from_parent(parent, collector_id, slot);
5954        bank_forks
5955            .write()
5956            .unwrap()
5957            .insert(bank)
5958            .clone_without_scheduler()
5959    }
5960
5961    /// Prepare a transaction batch from a list of legacy transactions. Used for tests only.
5962    pub fn prepare_batch_for_tests(
5963        &self,
5964        txs: Vec<Transaction>,
5965    ) -> TransactionBatch<'_, '_, RuntimeTransaction<SanitizedTransaction>> {
5966        let sanitized_txs = txs
5967            .into_iter()
5968            .map(RuntimeTransaction::from_transaction_for_tests)
5969            .collect::<Vec<_>>();
5970        TransactionBatch::new(
5971            self.try_lock_accounts(&sanitized_txs),
5972            self,
5973            OwnedOrBorrowed::Owned(sanitized_txs),
5974        )
5975    }
5976
5977    /// Set the initial accounts data size
5978    /// NOTE: This fn is *ONLY FOR TESTS*
5979    pub fn set_accounts_data_size_initial_for_tests(&mut self, amount: u64) {
5980        self.accounts_data_size_initial = amount;
5981    }
5982
5983    /// Update the accounts data size off-chain delta
5984    /// NOTE: This fn is *ONLY FOR TESTS*
5985    pub fn update_accounts_data_size_delta_off_chain_for_tests(&self, amount: i64) {
5986        self.update_accounts_data_size_delta_off_chain(amount)
5987    }
5988
5989    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
5990    ///
5991    /// # Panics
5992    ///
5993    /// Panics if any of the transactions do not pass sanitization checks.
5994    #[must_use]
5995    pub fn process_transactions<'a>(
5996        &self,
5997        txs: impl Iterator<Item = &'a Transaction>,
5998    ) -> Vec<Result<()>> {
5999        self.try_process_transactions(txs).unwrap()
6000    }
6001
6002    /// Process entry transactions in a single batch. This is used for benches and unit tests.
6003    ///
6004    /// # Panics
6005    ///
6006    /// Panics if any of the transactions do not pass sanitization checks.
6007    #[must_use]
6008    pub fn process_entry_transactions(&self, txs: Vec<VersionedTransaction>) -> Vec<Result<()>> {
6009        self.try_process_entry_transactions(txs).unwrap()
6010    }
6011
6012    #[cfg(test)]
6013    pub fn flush_accounts_cache_slot_for_tests(&self) {
6014        self.rc
6015            .accounts
6016            .accounts_db
6017            .flush_accounts_cache_slot_for_tests(self.slot())
6018    }
6019
6020    pub fn get_sysvar_cache_for_tests(&self) -> SysvarCache {
6021        self.transaction_processor.get_sysvar_cache_for_tests()
6022    }
6023
6024    pub fn calculate_accounts_lt_hash_for_tests(&self) -> AccountsLtHash {
6025        self.rc
6026            .accounts
6027            .accounts_db
6028            .calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, self.slot)
6029    }
6030
6031    pub fn get_transaction_processor(&self) -> &TransactionBatchProcessor<BankForks> {
6032        &self.transaction_processor
6033    }
6034
6035    pub fn set_fee_structure(&mut self, fee_structure: &FeeStructure) {
6036        self.fee_structure = fee_structure.clone();
6037    }
6038
6039    pub fn load_program(
6040        &self,
6041        pubkey: &Pubkey,
6042        reload: bool,
6043        effective_epoch: Epoch,
6044    ) -> Option<Arc<ProgramCacheEntry>> {
6045        let environments = self
6046            .transaction_processor
6047            .get_environments_for_epoch(effective_epoch);
6048        load_program_with_pubkey(
6049            self,
6050            &environments,
6051            pubkey,
6052            self.slot(),
6053            &mut ExecuteTimings::default(), // Called by ledger-tool, metrics not accumulated.
6054            reload,
6055        )
6056    }
6057
6058    pub fn withdraw(&self, pubkey: &Pubkey, lamports: u64) -> Result<()> {
6059        match self.get_account_with_fixed_root(pubkey) {
6060            Some(mut account) => {
6061                let min_balance = match get_system_account_kind(&account) {
6062                    Some(SystemAccountKind::Nonce) => self
6063                        .rent_collector
6064                        .rent
6065                        .minimum_balance(nonce::state::State::size()),
6066                    _ => 0,
6067                };
6068
6069                lamports
6070                    .checked_add(min_balance)
6071                    .filter(|required_balance| *required_balance <= account.lamports())
6072                    .ok_or(TransactionError::InsufficientFundsForFee)?;
6073                account
6074                    .checked_sub_lamports(lamports)
6075                    .map_err(|_| TransactionError::InsufficientFundsForFee)?;
6076                self.store_account(pubkey, &account);
6077
6078                Ok(())
6079            }
6080            None => Err(TransactionError::AccountNotFound),
6081        }
6082    }
6083
6084    pub fn set_hash_overrides(&self, hash_overrides: HashOverrides) {
6085        *self.hash_overrides.lock().unwrap() = hash_overrides;
6086    }
6087
6088    /// Get stake and stake node accounts
6089    pub(crate) fn get_stake_accounts(&self, minimized_account_set: &DashSet<Pubkey>) {
6090        self.stakes_cache
6091            .stakes()
6092            .stake_delegations()
6093            .iter()
6094            .for_each(|(pubkey, _)| {
6095                minimized_account_set.insert(*pubkey);
6096            });
6097
6098        self.stakes_cache
6099            .stakes()
6100            .staked_nodes()
6101            .par_iter()
6102            .for_each(|(pubkey, _)| {
6103                minimized_account_set.insert(*pubkey);
6104            });
6105    }
6106}
6107
6108/// Compute how much an account has changed size.  This function is useful when the data size delta
6109/// needs to be computed and passed to an `update_accounts_data_size_delta` function.
6110fn calculate_data_size_delta(old_data_size: usize, new_data_size: usize) -> i64 {
6111    assert!(old_data_size <= i64::MAX as usize);
6112    assert!(new_data_size <= i64::MAX as usize);
6113    let old_data_size = old_data_size as i64;
6114    let new_data_size = new_data_size as i64;
6115
6116    new_data_size.saturating_sub(old_data_size)
6117}
6118
6119impl Drop for Bank {
6120    fn drop(&mut self) {
6121        if let Some(drop_callback) = self.drop_callback.read().unwrap().0.as_ref() {
6122            drop_callback.callback(self);
6123        } else {
6124            // Default case for tests
6125            self.rc
6126                .accounts
6127                .accounts_db
6128                .purge_slot(self.slot(), self.bank_id(), false);
6129        }
6130    }
6131}
6132
6133/// utility function used for testing and benchmarking.
6134pub mod test_utils {
6135    use {
6136        super::Bank,
6137        crate::installed_scheduler_pool::BankWithScheduler,
6138        solana_account::{state_traits::StateMut, ReadableAccount, WritableAccount},
6139        solana_instruction::error::LamportsError,
6140        solana_pubkey::Pubkey,
6141        solana_sha256_hasher::hashv,
6142        solana_vote_interface::state::VoteStateV4,
6143        solana_vote_program::vote_state::{BlockTimestamp, VoteStateVersions},
6144        std::sync::Arc,
6145    };
6146    pub fn goto_end_of_slot(bank: Arc<Bank>) {
6147        goto_end_of_slot_with_scheduler(&BankWithScheduler::new_without_scheduler(bank))
6148    }
6149
6150    pub fn goto_end_of_slot_with_scheduler(bank: &BankWithScheduler) {
6151        let mut tick_hash = bank.last_blockhash();
6152        loop {
6153            tick_hash = hashv(&[tick_hash.as_ref(), &[42]]);
6154            bank.register_tick(&tick_hash);
6155            if tick_hash == bank.last_blockhash() {
6156                bank.freeze();
6157                return;
6158            }
6159        }
6160    }
6161
6162    pub fn update_vote_account_timestamp(
6163        timestamp: BlockTimestamp,
6164        bank: &Bank,
6165        vote_pubkey: &Pubkey,
6166    ) {
6167        let mut vote_account = bank.get_account(vote_pubkey).unwrap_or_default();
6168        let mut vote_state = VoteStateV4::deserialize(vote_account.data(), vote_pubkey)
6169            .ok()
6170            .unwrap_or_default();
6171        vote_state.last_timestamp = timestamp;
6172        let versioned = VoteStateVersions::new_v4(vote_state);
6173        vote_account.set_state(&versioned).unwrap();
6174        bank.store_account(vote_pubkey, &vote_account);
6175    }
6176
6177    pub fn deposit(
6178        bank: &Bank,
6179        pubkey: &Pubkey,
6180        lamports: u64,
6181    ) -> std::result::Result<u64, LamportsError> {
6182        // This doesn't collect rents intentionally.
6183        // Rents should only be applied to actual TXes
6184        let mut account = bank
6185            .get_account_with_fixed_root_no_cache(pubkey)
6186            .unwrap_or_default();
6187        account.checked_add_lamports(lamports)?;
6188        bank.store_account(pubkey, &account);
6189        Ok(account.lamports())
6190    }
6191}