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