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, StakeRewards, VoteRewardsAccounts},
42        },
43        bank_forks::BankForks,
44        epoch_stakes::{split_epoch_stakes, EpochStakes, NodeVoteAccounts, VersionedEpochStakes},
45        installed_scheduler_pool::{BankWithScheduler, InstalledSchedulerRwLock},
46        rent_collector::RentCollectorWithMetrics,
47        runtime_config::RuntimeConfig,
48        serde_snapshot::BankIncrementalSnapshotPersistence,
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::{Stakes, StakesCache, StakesEnum},
56        status_cache::{SlotDelta, StatusCache},
57        transaction_batch::{OwnedOrBorrowed, TransactionBatch},
58        verify_precompiles::verify_precompiles,
59    },
60    accounts_lt_hash::{CacheValue as AccountsLtHashCacheValue, Stats as AccountsLtHashStats},
61    agave_feature_set::{self as feature_set, reward_full_priority_fee, FeatureSet},
62    agave_precompiles::get_precompiles,
63    agave_reserved_account_keys::ReservedAccountKeys,
64    ahash::AHashSet,
65    dashmap::{DashMap, DashSet},
66    log::*,
67    rayon::{
68        iter::{IntoParallelIterator, IntoParallelRefIterator, ParallelIterator},
69        ThreadPoolBuilder,
70    },
71    serde::Serialize,
72    solana_accounts_db::{
73        account_locks::validate_account_locks,
74        accounts::{AccountAddressFilter, Accounts, PubkeyAccountSlot},
75        accounts_db::{
76            AccountStorageEntry, AccountsDb, AccountsDbConfig, CalcAccountsHashDataSource,
77            DuplicatesLtHash, OldStoragesPolicy, PubkeyHashAccount,
78            VerifyAccountsHashAndLamportsConfig,
79        },
80        accounts_hash::{
81            AccountHash, AccountsHash, AccountsLtHash, CalcAccountsHashConfig, HashStats,
82            IncrementalAccountsHash, MerkleOrLatticeAccountsHash,
83        },
84        accounts_index::{IndexKey, ScanConfig, ScanResult},
85        accounts_partition::{self, Partition, PartitionIndex},
86        accounts_update_notifier_interface::AccountsUpdateNotifier,
87        ancestors::{Ancestors, AncestorsForSerialization},
88        blockhash_queue::BlockhashQueue,
89        epoch_accounts_hash::EpochAccountsHash,
90        sorted_storages::SortedStorages,
91        storable_accounts::StorableAccounts,
92    },
93    solana_bpf_loader_program::syscalls::{
94        create_program_runtime_environment_v1, create_program_runtime_environment_v2,
95    },
96    solana_builtins::{prototype::BuiltinPrototype, BUILTINS, STATELESS_BUILTINS},
97    solana_compute_budget::compute_budget::ComputeBudget,
98    solana_compute_budget_instruction::instructions_processor::process_compute_budget_instructions,
99    solana_cost_model::{
100        block_cost_limits::{simd_0207_block_limits, simd_0256_block_limits},
101        cost_tracker::CostTracker,
102    },
103    solana_fee::FeeFeatures,
104    solana_lattice_hash::lt_hash::LtHash,
105    solana_measure::{meas_dur, measure::Measure, measure_time, measure_us},
106    solana_program_runtime::{
107        invoke_context::BuiltinFunctionWithContext, loaded_programs::ProgramCacheEntry,
108    },
109    solana_runtime_transaction::{
110        runtime_transaction::RuntimeTransaction, transaction_with_meta::TransactionWithMeta,
111    },
112    solana_sdk::{
113        account::{
114            create_account_shared_data_with_fields as create_account, from_account, Account,
115            AccountSharedData, InheritableAccountFields, ReadableAccount, WritableAccount,
116        },
117        bpf_loader_upgradeable,
118        clock::{
119            BankId, Epoch, Slot, SlotCount, SlotIndex, UnixTimestamp, DEFAULT_HASHES_PER_TICK,
120            DEFAULT_TICKS_PER_SECOND, INITIAL_RENT_EPOCH, MAX_PROCESSING_AGE,
121            MAX_TRANSACTION_FORWARDING_DELAY, SECONDS_PER_DAY, UPDATED_HASHES_PER_TICK2,
122            UPDATED_HASHES_PER_TICK3, UPDATED_HASHES_PER_TICK4, UPDATED_HASHES_PER_TICK5,
123            UPDATED_HASHES_PER_TICK6,
124        },
125        epoch_info::EpochInfo,
126        epoch_schedule::EpochSchedule,
127        feature,
128        fee::{FeeBudgetLimits, FeeDetails, FeeStructure},
129        fee_calculator::FeeRateGovernor,
130        genesis_config::{ClusterType, GenesisConfig},
131        hard_forks::HardForks,
132        hash::{extend_and_hash, hashv, Hash},
133        incinerator,
134        inflation::Inflation,
135        inner_instruction::InnerInstructions,
136        message::{AccountKeys, SanitizedMessage},
137        native_loader,
138        native_token::LAMPORTS_PER_SOL,
139        packet::PACKET_DATA_SIZE,
140        pubkey::Pubkey,
141        rent_collector::{CollectedInfo, RentCollector},
142        rent_debits::RentDebits,
143        reward_info::RewardInfo,
144        signature::{Keypair, Signature},
145        slot_hashes::SlotHashes,
146        slot_history::{Check, SlotHistory},
147        stake::state::Delegation,
148        system_transaction,
149        sysvar::{self, last_restart_slot::LastRestartSlot, Sysvar, SysvarId},
150        timing::years_as_slots,
151        transaction::{
152            MessageHash, Result, SanitizedTransaction, Transaction, TransactionError,
153            TransactionVerificationMode, VersionedTransaction, MAX_TX_ACCOUNT_LOCKS,
154        },
155    },
156    solana_stake_program::points::InflationPointCalculationEvent,
157    solana_svm::{
158        account_loader::{collect_rent_from_account, LoadedTransaction},
159        account_overrides::AccountOverrides,
160        transaction_commit_result::{CommittedTransaction, TransactionCommitResult},
161        transaction_error_metrics::TransactionErrorMetrics,
162        transaction_execution_result::{
163            TransactionExecutionDetails, TransactionLoadedAccountsStats,
164        },
165        transaction_processing_callback::{AccountState, TransactionProcessingCallback},
166        transaction_processing_result::{
167            ProcessedTransaction, TransactionProcessingResult,
168            TransactionProcessingResultExtensions,
169        },
170        transaction_processor::{
171            ExecutionRecordingConfig, TransactionBatchProcessor, TransactionLogMessages,
172            TransactionProcessingConfig, TransactionProcessingEnvironment,
173        },
174    },
175    solana_svm_transaction::svm_message::SVMMessage,
176    solana_timings::{ExecuteTimingType, ExecuteTimings},
177    solana_transaction_context::{TransactionAccount, TransactionReturnData},
178    solana_vote::vote_account::{VoteAccount, VoteAccountsHashMap},
179    std::{
180        collections::{HashMap, HashSet},
181        convert::TryFrom,
182        fmt,
183        ops::{AddAssign, RangeFull, RangeInclusive},
184        path::PathBuf,
185        slice,
186        sync::{
187            atomic::{
188                AtomicBool, AtomicI64, AtomicU64, AtomicUsize,
189                Ordering::{AcqRel, Acquire, Relaxed},
190            },
191            Arc, LockResult, Mutex, RwLock, RwLockReadGuard, RwLockWriteGuard, Weak,
192        },
193        thread::Builder,
194        time::{Duration, Instant},
195    },
196};
197pub use {
198    partitioned_epoch_rewards::KeyedRewardsAndNumPartitions, solana_sdk::reward_type::RewardType,
199};
200#[cfg(feature = "dev-context-only-utils")]
201use {
202    solana_accounts_db::accounts_db::{
203        ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS, ACCOUNTS_DB_CONFIG_FOR_TESTING,
204    },
205    solana_nonce_account::{get_system_account_kind, SystemAccountKind},
206    solana_program_runtime::{loaded_programs::ProgramCacheForTxBatch, sysvar_cache::SysvarCache},
207    solana_sdk::nonce,
208    solana_svm::program_loader::load_program_with_pubkey,
209};
210
211/// params to `verify_accounts_hash`
212struct VerifyAccountsHashConfig {
213    test_hash_calculation: bool,
214    ignore_mismatch: bool,
215    require_rooted_bank: bool,
216    run_in_background: bool,
217    store_hash_raw_data_for_debug: bool,
218}
219
220mod accounts_lt_hash;
221mod address_lookup_table;
222pub mod bank_hash_details;
223mod builtin_programs;
224pub mod builtins;
225mod check_transactions;
226pub mod epoch_accounts_hash_utils;
227mod fee_distribution;
228mod metrics;
229pub(crate) mod partitioned_epoch_rewards;
230mod recent_blockhashes_account;
231mod serde_snapshot;
232mod sysvar_cache;
233pub(crate) mod tests;
234
235pub const SECONDS_PER_YEAR: f64 = 365.25 * 24.0 * 60.0 * 60.0;
236
237pub const MAX_LEADER_SCHEDULE_STAKES: Epoch = 5;
238
239#[derive(Default)]
240struct RentMetrics {
241    hold_range_us: AtomicU64,
242    load_us: AtomicU64,
243    collect_us: AtomicU64,
244    hash_us: AtomicU64,
245    store_us: AtomicU64,
246    count: AtomicUsize,
247}
248
249pub type BankStatusCache = StatusCache<Result<()>>;
250#[cfg_attr(
251    feature = "frozen-abi",
252    frozen_abi(digest = "4e7a7AAsQrM5Lp5bhREdVZ5QGZfyETbBthhWjYMYb6zS")
253)]
254pub type BankSlotDelta = SlotDelta<Result<()>>;
255
256#[derive(Default, Copy, Clone, Debug, PartialEq, Eq)]
257pub struct SquashTiming {
258    pub squash_accounts_ms: u64,
259    pub squash_accounts_cache_ms: u64,
260    pub squash_accounts_index_ms: u64,
261    pub squash_accounts_store_ms: u64,
262
263    pub squash_cache_ms: u64,
264}
265
266impl AddAssign for SquashTiming {
267    fn add_assign(&mut self, rhs: Self) {
268        self.squash_accounts_ms += rhs.squash_accounts_ms;
269        self.squash_accounts_cache_ms += rhs.squash_accounts_cache_ms;
270        self.squash_accounts_index_ms += rhs.squash_accounts_index_ms;
271        self.squash_accounts_store_ms += rhs.squash_accounts_store_ms;
272        self.squash_cache_ms += rhs.squash_cache_ms;
273    }
274}
275
276#[derive(Debug, Default, PartialEq)]
277pub(crate) struct CollectorFeeDetails {
278    transaction_fee: u64,
279    priority_fee: u64,
280}
281
282impl CollectorFeeDetails {
283    pub(crate) fn accumulate(&mut self, fee_details: &FeeDetails) {
284        self.transaction_fee = self
285            .transaction_fee
286            .saturating_add(fee_details.transaction_fee());
287        self.priority_fee = self
288            .priority_fee
289            .saturating_add(fee_details.prioritization_fee());
290    }
291
292    pub(crate) fn total(&self) -> u64 {
293        self.transaction_fee.saturating_add(self.priority_fee)
294    }
295}
296
297impl From<FeeDetails> for CollectorFeeDetails {
298    fn from(fee_details: FeeDetails) -> Self {
299        CollectorFeeDetails {
300            transaction_fee: fee_details.transaction_fee(),
301            priority_fee: fee_details.prioritization_fee(),
302        }
303    }
304}
305
306#[derive(Debug)]
307pub struct BankRc {
308    /// where all the Accounts are stored
309    pub accounts: Arc<Accounts>,
310
311    /// Previous checkpoint of this bank
312    pub(crate) parent: RwLock<Option<Arc<Bank>>>,
313
314    pub(crate) bank_id_generator: Arc<AtomicU64>,
315}
316
317impl BankRc {
318    pub(crate) fn new(accounts: Accounts) -> Self {
319        Self {
320            accounts: Arc::new(accounts),
321            parent: RwLock::new(None),
322            bank_id_generator: Arc::new(AtomicU64::new(0)),
323        }
324    }
325}
326
327pub struct LoadAndExecuteTransactionsOutput {
328    // Vector of results indicating whether a transaction was processed or could not
329    // be processed. Note processed transactions can still have failed!
330    pub processing_results: Vec<TransactionProcessingResult>,
331    // Processed transaction counts used to update bank transaction counts and
332    // for metrics reporting.
333    pub processed_counts: ProcessedTransactionCounts,
334}
335
336#[derive(Debug, PartialEq)]
337pub struct TransactionSimulationResult {
338    pub result: Result<()>,
339    pub logs: TransactionLogMessages,
340    pub post_simulation_accounts: Vec<TransactionAccount>,
341    pub units_consumed: u64,
342    pub return_data: Option<TransactionReturnData>,
343    pub inner_instructions: Option<Vec<InnerInstructions>>,
344}
345
346#[derive(Clone, Debug)]
347pub struct TransactionBalancesSet {
348    pub pre_balances: TransactionBalances,
349    pub post_balances: TransactionBalances,
350}
351
352impl TransactionBalancesSet {
353    pub fn new(pre_balances: TransactionBalances, post_balances: TransactionBalances) -> Self {
354        assert_eq!(pre_balances.len(), post_balances.len());
355        Self {
356            pre_balances,
357            post_balances,
358        }
359    }
360}
361pub type TransactionBalances = Vec<Vec<u64>>;
362
363pub type PreCommitResult<'a> = Result<Option<RwLockReadGuard<'a, Hash>>>;
364
365#[derive(Serialize, Deserialize, Debug, PartialEq, Eq)]
366pub enum TransactionLogCollectorFilter {
367    All,
368    AllWithVotes,
369    None,
370    OnlyMentionedAddresses,
371}
372
373impl Default for TransactionLogCollectorFilter {
374    fn default() -> Self {
375        Self::None
376    }
377}
378
379#[derive(Debug, Default)]
380pub struct TransactionLogCollectorConfig {
381    pub mentioned_addresses: HashSet<Pubkey>,
382    pub filter: TransactionLogCollectorFilter,
383}
384
385#[derive(Clone, Debug, PartialEq, Eq)]
386pub struct TransactionLogInfo {
387    pub signature: Signature,
388    pub result: Result<()>,
389    pub is_vote: bool,
390    pub log_messages: TransactionLogMessages,
391}
392
393#[derive(Default, Debug)]
394pub struct TransactionLogCollector {
395    // All the logs collected for from this Bank.  Exact contents depend on the
396    // active `TransactionLogCollectorFilter`
397    pub logs: Vec<TransactionLogInfo>,
398
399    // For each `mentioned_addresses`, maintain a list of indices into `logs` to easily
400    // locate the logs from transactions that included the mentioned addresses.
401    pub mentioned_address_map: HashMap<Pubkey, Vec<usize>>,
402}
403
404impl TransactionLogCollector {
405    pub fn get_logs_for_address(
406        &self,
407        address: Option<&Pubkey>,
408    ) -> Option<Vec<TransactionLogInfo>> {
409        match address {
410            None => Some(self.logs.clone()),
411            Some(address) => self.mentioned_address_map.get(address).map(|log_indices| {
412                log_indices
413                    .iter()
414                    .filter_map(|i| self.logs.get(*i).cloned())
415                    .collect()
416            }),
417        }
418    }
419}
420
421/// Bank's common fields shared by all supported snapshot versions for deserialization.
422/// Sync fields with BankFieldsToSerialize! This is paired with it.
423/// All members are made public to remain Bank's members private and to make versioned deserializer workable on this
424/// Note that some fields are missing from the serializer struct. This is because of fields added later.
425/// Since it is difficult to insert fields to serialize/deserialize against existing code already deployed,
426/// new fields can be optionally serialized and optionally deserialized. At some point, the serialization and
427/// deserialization will use a new mechanism or otherwise be in sync more clearly.
428#[derive(Clone, Debug, Default)]
429#[cfg_attr(feature = "dev-context-only-utils", derive(PartialEq))]
430pub struct BankFieldsToDeserialize {
431    pub(crate) blockhash_queue: BlockhashQueue,
432    pub(crate) ancestors: AncestorsForSerialization,
433    pub(crate) hash: Hash,
434    pub(crate) parent_hash: Hash,
435    pub(crate) parent_slot: Slot,
436    pub(crate) hard_forks: HardForks,
437    pub(crate) transaction_count: u64,
438    pub(crate) tick_height: u64,
439    pub(crate) signature_count: u64,
440    pub(crate) capitalization: u64,
441    pub(crate) max_tick_height: u64,
442    pub(crate) hashes_per_tick: Option<u64>,
443    pub(crate) ticks_per_slot: u64,
444    pub(crate) ns_per_slot: u128,
445    pub(crate) genesis_creation_time: UnixTimestamp,
446    pub(crate) slots_per_year: f64,
447    pub(crate) slot: Slot,
448    pub(crate) epoch: Epoch,
449    pub(crate) block_height: u64,
450    pub(crate) collector_id: Pubkey,
451    pub(crate) collector_fees: u64,
452    pub(crate) fee_rate_governor: FeeRateGovernor,
453    pub(crate) collected_rent: u64,
454    pub(crate) rent_collector: RentCollector,
455    pub(crate) epoch_schedule: EpochSchedule,
456    pub(crate) inflation: Inflation,
457    pub(crate) stakes: Stakes<Delegation>,
458    pub(crate) epoch_stakes: HashMap<Epoch, EpochStakes>,
459    pub(crate) is_delta: bool,
460    pub(crate) accounts_data_len: u64,
461    pub(crate) incremental_snapshot_persistence: Option<BankIncrementalSnapshotPersistence>,
462    pub(crate) epoch_accounts_hash: Option<Hash>,
463    // When removing the accounts lt hash featurization code, also remove this Option wrapper
464    pub(crate) accounts_lt_hash: Option<AccountsLtHash>,
465    pub(crate) bank_hash_stats: BankHashStats,
466}
467
468/// Bank's common fields shared by all supported snapshot versions for serialization.
469/// This was separated from BankFieldsToDeserialize to avoid cloning by using refs.
470/// So, sync fields with BankFieldsToDeserialize!
471/// all members are made public to keep Bank private and to make versioned serializer workable on this.
472/// Note that some fields are missing from the serializer struct. This is because of fields added later.
473/// Since it is difficult to insert fields to serialize/deserialize against existing code already deployed,
474/// new fields can be optionally serialized and optionally deserialized. At some point, the serialization and
475/// deserialization will use a new mechanism or otherwise be in sync more clearly.
476#[derive(Debug)]
477pub struct BankFieldsToSerialize {
478    pub blockhash_queue: BlockhashQueue,
479    pub ancestors: AncestorsForSerialization,
480    pub hash: Hash,
481    pub parent_hash: Hash,
482    pub parent_slot: Slot,
483    pub hard_forks: HardForks,
484    pub transaction_count: u64,
485    pub tick_height: u64,
486    pub signature_count: u64,
487    pub capitalization: u64,
488    pub max_tick_height: u64,
489    pub hashes_per_tick: Option<u64>,
490    pub ticks_per_slot: u64,
491    pub ns_per_slot: u128,
492    pub genesis_creation_time: UnixTimestamp,
493    pub slots_per_year: f64,
494    pub slot: Slot,
495    pub epoch: Epoch,
496    pub block_height: u64,
497    pub collector_id: Pubkey,
498    pub collector_fees: u64,
499    pub fee_rate_governor: FeeRateGovernor,
500    pub collected_rent: u64,
501    pub rent_collector: RentCollector,
502    pub epoch_schedule: EpochSchedule,
503    pub inflation: Inflation,
504    pub stakes: StakesEnum,
505    pub epoch_stakes: HashMap<Epoch, EpochStakes>,
506    pub is_delta: bool,
507    pub accounts_data_len: u64,
508    pub versioned_epoch_stakes: HashMap<u64, VersionedEpochStakes>,
509    // When removing the accounts lt hash featurization code, also remove this Option wrapper
510    pub accounts_lt_hash: Option<AccountsLtHash>,
511}
512
513// Can't derive PartialEq because RwLock doesn't implement PartialEq
514#[cfg(feature = "dev-context-only-utils")]
515impl PartialEq for Bank {
516    fn eq(&self, other: &Self) -> bool {
517        if std::ptr::eq(self, other) {
518            return true;
519        }
520        // Suppress rustfmt until https://github.com/rust-lang/rustfmt/issues/5920 is fixed ...
521        #[rustfmt::skip]
522        let Self {
523            skipped_rewrites: _,
524            rc: _,
525            status_cache: _,
526            blockhash_queue,
527            ancestors,
528            hash,
529            parent_hash,
530            parent_slot,
531            hard_forks,
532            transaction_count,
533            non_vote_transaction_count_since_restart: _,
534            transaction_error_count: _,
535            transaction_entries_count: _,
536            transactions_per_entry_max: _,
537            tick_height,
538            signature_count,
539            capitalization,
540            max_tick_height,
541            hashes_per_tick,
542            ticks_per_slot,
543            ns_per_slot,
544            genesis_creation_time,
545            slots_per_year,
546            slot,
547            bank_id: _,
548            epoch,
549            block_height,
550            collector_id,
551            collector_fees,
552            fee_rate_governor,
553            collected_rent,
554            rent_collector,
555            epoch_schedule,
556            inflation,
557            stakes_cache,
558            epoch_stakes,
559            is_delta,
560            #[cfg(feature = "dev-context-only-utils")]
561            hash_overrides,
562            accounts_lt_hash,
563            // TODO: Confirm if all these fields are intentionally ignored!
564            rewards: _,
565            cluster_type: _,
566            lazy_rent_collection: _,
567            rewards_pool_pubkeys: _,
568            transaction_debug_keys: _,
569            transaction_log_collector_config: _,
570            transaction_log_collector: _,
571            feature_set: _,
572            reserved_account_keys: _,
573            drop_callback: _,
574            freeze_started: _,
575            vote_only_bank: _,
576            cost_tracker: _,
577            accounts_data_size_initial: _,
578            accounts_data_size_delta_on_chain: _,
579            accounts_data_size_delta_off_chain: _,
580            epoch_reward_status: _,
581            transaction_processor: _,
582            check_program_modification_slot: _,
583            collector_fee_details: _,
584            compute_budget: _,
585            transaction_account_lock_limit: _,
586            fee_structure: _,
587            cache_for_accounts_lt_hash: _,
588            stats_for_accounts_lt_hash: _,
589            block_id,
590            bank_hash_stats: _,
591            // Ignore new fields explicitly if they do not impact PartialEq.
592            // Adding ".." will remove compile-time checks that if a new field
593            // is added to the struct, this PartialEq is accordingly updated.
594        } = self;
595        *blockhash_queue.read().unwrap() == *other.blockhash_queue.read().unwrap()
596            && ancestors == &other.ancestors
597            && *hash.read().unwrap() == *other.hash.read().unwrap()
598            && parent_hash == &other.parent_hash
599            && parent_slot == &other.parent_slot
600            && *hard_forks.read().unwrap() == *other.hard_forks.read().unwrap()
601            && transaction_count.load(Relaxed) == other.transaction_count.load(Relaxed)
602            && tick_height.load(Relaxed) == other.tick_height.load(Relaxed)
603            && signature_count.load(Relaxed) == other.signature_count.load(Relaxed)
604            && capitalization.load(Relaxed) == other.capitalization.load(Relaxed)
605            && max_tick_height == &other.max_tick_height
606            && hashes_per_tick == &other.hashes_per_tick
607            && ticks_per_slot == &other.ticks_per_slot
608            && ns_per_slot == &other.ns_per_slot
609            && genesis_creation_time == &other.genesis_creation_time
610            && slots_per_year == &other.slots_per_year
611            && slot == &other.slot
612            && epoch == &other.epoch
613            && block_height == &other.block_height
614            && collector_id == &other.collector_id
615            && collector_fees.load(Relaxed) == other.collector_fees.load(Relaxed)
616            && fee_rate_governor == &other.fee_rate_governor
617            && collected_rent.load(Relaxed) == other.collected_rent.load(Relaxed)
618            && rent_collector == &other.rent_collector
619            && epoch_schedule == &other.epoch_schedule
620            && *inflation.read().unwrap() == *other.inflation.read().unwrap()
621            && *stakes_cache.stakes() == *other.stakes_cache.stakes()
622            && epoch_stakes == &other.epoch_stakes
623            && is_delta.load(Relaxed) == other.is_delta.load(Relaxed)
624            // No deadlock is possbile, when Arc::ptr_eq() returns false, because of being
625            // different Mutexes.
626            && (Arc::ptr_eq(hash_overrides, &other.hash_overrides) ||
627                *hash_overrides.lock().unwrap() == *other.hash_overrides.lock().unwrap())
628            && !(self.is_accounts_lt_hash_enabled() && other.is_accounts_lt_hash_enabled()
629                && *accounts_lt_hash.lock().unwrap() != *other.accounts_lt_hash.lock().unwrap())
630            && *block_id.read().unwrap() == *other.block_id.read().unwrap()
631    }
632}
633
634#[cfg(feature = "dev-context-only-utils")]
635impl BankFieldsToSerialize {
636    /// Create a new BankFieldsToSerialize where basically every field is defaulted.
637    /// Only use for tests; many of the fields are invalid!
638    pub fn default_for_tests() -> Self {
639        Self {
640            blockhash_queue: BlockhashQueue::default(),
641            ancestors: AncestorsForSerialization::default(),
642            hash: Hash::default(),
643            parent_hash: Hash::default(),
644            parent_slot: Slot::default(),
645            hard_forks: HardForks::default(),
646            transaction_count: u64::default(),
647            tick_height: u64::default(),
648            signature_count: u64::default(),
649            capitalization: u64::default(),
650            max_tick_height: u64::default(),
651            hashes_per_tick: Option::default(),
652            ticks_per_slot: u64::default(),
653            ns_per_slot: u128::default(),
654            genesis_creation_time: UnixTimestamp::default(),
655            slots_per_year: f64::default(),
656            slot: Slot::default(),
657            epoch: Epoch::default(),
658            block_height: u64::default(),
659            collector_id: Pubkey::default(),
660            collector_fees: u64::default(),
661            fee_rate_governor: FeeRateGovernor::default(),
662            collected_rent: u64::default(),
663            rent_collector: RentCollector::default(),
664            epoch_schedule: EpochSchedule::default(),
665            inflation: Inflation::default(),
666            stakes: Stakes::<Delegation>::default().into(),
667            epoch_stakes: HashMap::default(),
668            is_delta: bool::default(),
669            accounts_data_len: u64::default(),
670            versioned_epoch_stakes: HashMap::default(),
671            accounts_lt_hash: Some(AccountsLtHash(LtHash([0x7E57; LtHash::NUM_ELEMENTS]))),
672        }
673    }
674}
675
676#[derive(Debug)]
677pub enum RewardCalculationEvent<'a, 'b> {
678    Staking(&'a Pubkey, &'b InflationPointCalculationEvent),
679}
680
681/// type alias is not supported for trait in rust yet. As a workaround, we define the
682/// `RewardCalcTracer` trait explicitly and implement it on any type that implement
683/// `Fn(&RewardCalculationEvent) + Send + Sync`.
684pub trait RewardCalcTracer: Fn(&RewardCalculationEvent) + Send + Sync {}
685
686impl<T: Fn(&RewardCalculationEvent) + Send + Sync> RewardCalcTracer for T {}
687
688fn null_tracer() -> Option<impl RewardCalcTracer> {
689    None::<fn(&RewardCalculationEvent)>
690}
691
692pub trait DropCallback: fmt::Debug {
693    fn callback(&self, b: &Bank);
694    fn clone_box(&self) -> Box<dyn DropCallback + Send + Sync>;
695}
696
697#[derive(Debug, Default)]
698pub struct OptionalDropCallback(Option<Box<dyn DropCallback + Send + Sync>>);
699
700#[derive(Default, Debug, Clone, PartialEq)]
701#[cfg(feature = "dev-context-only-utils")]
702pub struct HashOverrides {
703    hashes: HashMap<Slot, HashOverride>,
704}
705
706#[cfg(feature = "dev-context-only-utils")]
707impl HashOverrides {
708    fn get_hash_override(&self, slot: Slot) -> Option<&HashOverride> {
709        self.hashes.get(&slot)
710    }
711
712    fn get_blockhash_override(&self, slot: Slot) -> Option<&Hash> {
713        self.get_hash_override(slot)
714            .map(|hash_override| &hash_override.blockhash)
715    }
716
717    fn get_bank_hash_override(&self, slot: Slot) -> Option<&Hash> {
718        self.get_hash_override(slot)
719            .map(|hash_override| &hash_override.bank_hash)
720    }
721
722    pub fn add_override(&mut self, slot: Slot, blockhash: Hash, bank_hash: Hash) {
723        let is_new = self
724            .hashes
725            .insert(
726                slot,
727                HashOverride {
728                    blockhash,
729                    bank_hash,
730                },
731            )
732            .is_none();
733        assert!(is_new);
734    }
735}
736
737#[derive(Debug, Clone, PartialEq)]
738#[cfg(feature = "dev-context-only-utils")]
739struct HashOverride {
740    blockhash: Hash,
741    bank_hash: Hash,
742}
743
744/// Manager for the state of all accounts and programs after processing its entries.
745#[derive(Debug)]
746pub struct Bank {
747    /// References to accounts, parent and signature status
748    pub rc: BankRc,
749
750    /// A cache of signature statuses
751    pub status_cache: Arc<RwLock<BankStatusCache>>,
752
753    /// FIFO queue of `recent_blockhash` items
754    blockhash_queue: RwLock<BlockhashQueue>,
755
756    /// The set of parents including this bank
757    pub ancestors: Ancestors,
758
759    /// Hash of this Bank's state. Only meaningful after freezing.
760    hash: RwLock<Hash>,
761
762    /// Hash of this Bank's parent's state
763    parent_hash: Hash,
764
765    /// parent's slot
766    parent_slot: Slot,
767
768    /// slots to hard fork at
769    hard_forks: Arc<RwLock<HardForks>>,
770
771    /// The number of committed transactions since genesis.
772    transaction_count: AtomicU64,
773
774    /// The number of non-vote transactions committed since the most
775    /// recent boot from snapshot or genesis. This value is only stored in
776    /// blockstore for the RPC method "getPerformanceSamples". It is not
777    /// retained within snapshots, but is preserved in `Bank::new_from_parent`.
778    non_vote_transaction_count_since_restart: AtomicU64,
779
780    /// The number of transaction errors in this slot
781    transaction_error_count: AtomicU64,
782
783    /// The number of transaction entries in this slot
784    transaction_entries_count: AtomicU64,
785
786    /// The max number of transaction in an entry in this slot
787    transactions_per_entry_max: AtomicU64,
788
789    /// Bank tick height
790    tick_height: AtomicU64,
791
792    /// The number of signatures from valid transactions in this slot
793    signature_count: AtomicU64,
794
795    /// Total capitalization, used to calculate inflation
796    capitalization: AtomicU64,
797
798    // Bank max_tick_height
799    max_tick_height: u64,
800
801    /// The number of hashes in each tick. None value means hashing is disabled.
802    hashes_per_tick: Option<u64>,
803
804    /// The number of ticks in each slot.
805    ticks_per_slot: u64,
806
807    /// length of a slot in ns
808    pub ns_per_slot: u128,
809
810    /// genesis time, used for computed clock
811    genesis_creation_time: UnixTimestamp,
812
813    /// The number of slots per year, used for inflation
814    slots_per_year: f64,
815
816    /// Bank slot (i.e. block)
817    slot: Slot,
818
819    bank_id: BankId,
820
821    /// Bank epoch
822    epoch: Epoch,
823
824    /// Bank block_height
825    block_height: u64,
826
827    /// The pubkey to send transactions fees to.
828    collector_id: Pubkey,
829
830    /// Fees that have been collected
831    collector_fees: AtomicU64,
832
833    /// Track cluster signature throughput and adjust fee rate
834    pub(crate) fee_rate_governor: FeeRateGovernor,
835
836    /// Rent that has been collected
837    collected_rent: AtomicU64,
838
839    /// latest rent collector, knows the epoch
840    rent_collector: RentCollector,
841
842    /// initialized from genesis
843    pub(crate) epoch_schedule: EpochSchedule,
844
845    /// inflation specs
846    inflation: Arc<RwLock<Inflation>>,
847
848    /// cache of vote_account and stake_account state for this fork
849    stakes_cache: StakesCache,
850
851    /// staked nodes on epoch boundaries, saved off when a bank.slot() is at
852    ///   a leader schedule calculation boundary
853    epoch_stakes: HashMap<Epoch, EpochStakes>,
854
855    /// A boolean reflecting whether any entries were recorded into the PoH
856    /// stream for the slot == self.slot
857    is_delta: AtomicBool,
858
859    /// Protocol-level rewards that were distributed by this bank
860    pub rewards: RwLock<Vec<(Pubkey, RewardInfo)>>,
861
862    pub cluster_type: Option<ClusterType>,
863
864    pub lazy_rent_collection: AtomicBool,
865
866    // this is temporary field only to remove rewards_pool entirely
867    pub rewards_pool_pubkeys: Arc<HashSet<Pubkey>>,
868
869    transaction_debug_keys: Option<Arc<HashSet<Pubkey>>>,
870
871    // Global configuration for how transaction logs should be collected across all banks
872    pub transaction_log_collector_config: Arc<RwLock<TransactionLogCollectorConfig>>,
873
874    // Logs from transactions that this Bank executed collected according to the criteria in
875    // `transaction_log_collector_config`
876    pub transaction_log_collector: Arc<RwLock<TransactionLogCollector>>,
877
878    pub feature_set: Arc<FeatureSet>,
879
880    /// Set of reserved account keys that cannot be write locked
881    reserved_account_keys: Arc<ReservedAccountKeys>,
882
883    /// callback function only to be called when dropping and should only be called once
884    pub drop_callback: RwLock<OptionalDropCallback>,
885
886    pub freeze_started: AtomicBool,
887
888    vote_only_bank: bool,
889
890    cost_tracker: RwLock<CostTracker>,
891
892    /// The initial accounts data size at the start of this Bank, before processing any transactions/etc
893    accounts_data_size_initial: u64,
894    /// The change to accounts data size in this Bank, due on-chain events (i.e. transactions)
895    accounts_data_size_delta_on_chain: AtomicI64,
896    /// The change to accounts data size in this Bank, due to off-chain events (i.e. rent collection)
897    accounts_data_size_delta_off_chain: AtomicI64,
898
899    /// until the skipped rewrites feature is activated, it is possible to skip rewrites and still include
900    /// the account hash of the accounts that would have been rewritten as bank hash expects.
901    skipped_rewrites: Mutex<HashMap<Pubkey, AccountHash>>,
902
903    epoch_reward_status: EpochRewardStatus,
904
905    transaction_processor: TransactionBatchProcessor<BankForks>,
906
907    check_program_modification_slot: bool,
908
909    /// Collected fee details
910    collector_fee_details: RwLock<CollectorFeeDetails>,
911
912    /// The compute budget to use for transaction execution.
913    compute_budget: Option<ComputeBudget>,
914
915    /// The max number of accounts that a transaction may lock.
916    transaction_account_lock_limit: Option<usize>,
917
918    /// Fee structure to use for assessing transaction fees.
919    fee_structure: FeeStructure,
920
921    /// blockhash and bank_hash overrides keyed by slot for simulated block production.
922    /// This _field_ was needed to be DCOU-ed to avoid 2 locks per bank freezing...
923    #[cfg(feature = "dev-context-only-utils")]
924    hash_overrides: Arc<Mutex<HashOverrides>>,
925
926    /// The lattice hash of all accounts
927    ///
928    /// The value is only meaningful after freezing.
929    accounts_lt_hash: Mutex<AccountsLtHash>,
930
931    /// A cache of *the initial state* of accounts modified in this slot
932    ///
933    /// The accounts lt hash needs both the initial and final state of each
934    /// account that was modified in this slot.  Cache the initial state here.
935    ///
936    /// Note: The initial state must be strictly from an ancestor,
937    /// and not an intermediate state within this slot.
938    cache_for_accounts_lt_hash: DashMap<Pubkey, AccountsLtHashCacheValue, ahash::RandomState>,
939
940    /// Stats related to the accounts lt hash
941    stats_for_accounts_lt_hash: AccountsLtHashStats,
942
943    /// The unique identifier for the corresponding block for this bank.
944    /// None for banks that have not yet completed replay or for leader banks as we cannot populate block_id
945    /// until bankless leader. Can be computed directly from shreds without needing to execute transactions.
946    block_id: RwLock<Option<Hash>>,
947
948    /// Accounts stats for computing the bank hash
949    bank_hash_stats: AtomicBankHashStats,
950}
951
952#[derive(Debug)]
953struct VoteReward {
954    vote_account: AccountSharedData,
955    commission: u8,
956    vote_rewards: u64,
957    vote_needs_store: bool,
958}
959
960type VoteRewards = DashMap<Pubkey, VoteReward>;
961
962#[derive(Debug, Default)]
963pub struct NewBankOptions {
964    pub vote_only_bank: bool,
965}
966
967#[cfg(feature = "dev-context-only-utils")]
968#[derive(Debug)]
969pub struct BankTestConfig {
970    pub accounts_db_config: AccountsDbConfig,
971}
972
973#[cfg(feature = "dev-context-only-utils")]
974impl Default for BankTestConfig {
975    fn default() -> Self {
976        Self {
977            accounts_db_config: ACCOUNTS_DB_CONFIG_FOR_TESTING,
978        }
979    }
980}
981
982#[derive(Debug)]
983struct PrevEpochInflationRewards {
984    validator_rewards: u64,
985    prev_epoch_duration_in_years: f64,
986    validator_rate: f64,
987    foundation_rate: f64,
988}
989
990#[derive(Debug, Default, PartialEq)]
991pub struct ProcessedTransactionCounts {
992    pub processed_transactions_count: u64,
993    pub processed_non_vote_transactions_count: u64,
994    pub processed_with_successful_result_count: u64,
995    pub signature_count: u64,
996}
997
998/// Account stats for computing the bank hash
999/// This struct is serialized and stored in the snapshot.
1000#[cfg_attr(feature = "frozen-abi", derive(AbiExample))]
1001#[derive(Clone, Default, Debug, Serialize, Deserialize, PartialEq, Eq)]
1002pub struct BankHashStats {
1003    pub num_updated_accounts: u64,
1004    pub num_removed_accounts: u64,
1005    pub num_lamports_stored: u64,
1006    pub total_data_len: u64,
1007    pub num_executable_accounts: u64,
1008}
1009
1010impl BankHashStats {
1011    pub fn update<T: ReadableAccount>(&mut self, account: &T) {
1012        if account.lamports() == 0 {
1013            self.num_removed_accounts += 1;
1014        } else {
1015            self.num_updated_accounts += 1;
1016        }
1017        self.total_data_len = self
1018            .total_data_len
1019            .wrapping_add(account.data().len() as u64);
1020        if account.executable() {
1021            self.num_executable_accounts += 1;
1022        }
1023        self.num_lamports_stored = self.num_lamports_stored.wrapping_add(account.lamports());
1024    }
1025    pub fn accumulate(&mut self, other: &BankHashStats) {
1026        self.num_updated_accounts += other.num_updated_accounts;
1027        self.num_removed_accounts += other.num_removed_accounts;
1028        self.total_data_len = self.total_data_len.wrapping_add(other.total_data_len);
1029        self.num_lamports_stored = self
1030            .num_lamports_stored
1031            .wrapping_add(other.num_lamports_stored);
1032        self.num_executable_accounts += other.num_executable_accounts;
1033    }
1034}
1035
1036#[derive(Debug, Default)]
1037pub struct AtomicBankHashStats {
1038    pub num_updated_accounts: AtomicU64,
1039    pub num_removed_accounts: AtomicU64,
1040    pub num_lamports_stored: AtomicU64,
1041    pub total_data_len: AtomicU64,
1042    pub num_executable_accounts: AtomicU64,
1043}
1044
1045impl AtomicBankHashStats {
1046    pub fn new(stat: &BankHashStats) -> Self {
1047        AtomicBankHashStats {
1048            num_updated_accounts: AtomicU64::new(stat.num_updated_accounts),
1049            num_removed_accounts: AtomicU64::new(stat.num_removed_accounts),
1050            num_lamports_stored: AtomicU64::new(stat.num_lamports_stored),
1051            total_data_len: AtomicU64::new(stat.total_data_len),
1052            num_executable_accounts: AtomicU64::new(stat.num_executable_accounts),
1053        }
1054    }
1055
1056    pub fn accumulate(&self, other: &BankHashStats) {
1057        self.num_updated_accounts
1058            .fetch_add(other.num_updated_accounts, Relaxed);
1059        self.num_removed_accounts
1060            .fetch_add(other.num_removed_accounts, Relaxed);
1061        self.total_data_len.fetch_add(other.total_data_len, Relaxed);
1062        self.num_lamports_stored
1063            .fetch_add(other.num_lamports_stored, Relaxed);
1064        self.num_executable_accounts
1065            .fetch_add(other.num_executable_accounts, Relaxed);
1066    }
1067
1068    pub fn load(&self) -> BankHashStats {
1069        BankHashStats {
1070            num_updated_accounts: self.num_updated_accounts.load(Relaxed),
1071            num_removed_accounts: self.num_removed_accounts.load(Relaxed),
1072            num_lamports_stored: self.num_lamports_stored.load(Relaxed),
1073            total_data_len: self.total_data_len.load(Relaxed),
1074            num_executable_accounts: self.num_executable_accounts.load(Relaxed),
1075        }
1076    }
1077}
1078
1079impl Bank {
1080    fn default_with_accounts(accounts: Accounts) -> Self {
1081        let mut bank = Self {
1082            skipped_rewrites: Mutex::default(),
1083            rc: BankRc::new(accounts),
1084            status_cache: Arc::<RwLock<BankStatusCache>>::default(),
1085            blockhash_queue: RwLock::<BlockhashQueue>::default(),
1086            ancestors: Ancestors::default(),
1087            hash: RwLock::<Hash>::default(),
1088            parent_hash: Hash::default(),
1089            parent_slot: Slot::default(),
1090            hard_forks: Arc::<RwLock<HardForks>>::default(),
1091            transaction_count: AtomicU64::default(),
1092            non_vote_transaction_count_since_restart: AtomicU64::default(),
1093            transaction_error_count: AtomicU64::default(),
1094            transaction_entries_count: AtomicU64::default(),
1095            transactions_per_entry_max: AtomicU64::default(),
1096            tick_height: AtomicU64::default(),
1097            signature_count: AtomicU64::default(),
1098            capitalization: AtomicU64::default(),
1099            max_tick_height: u64::default(),
1100            hashes_per_tick: Option::<u64>::default(),
1101            ticks_per_slot: u64::default(),
1102            ns_per_slot: u128::default(),
1103            genesis_creation_time: UnixTimestamp::default(),
1104            slots_per_year: f64::default(),
1105            slot: Slot::default(),
1106            bank_id: BankId::default(),
1107            epoch: Epoch::default(),
1108            block_height: u64::default(),
1109            collector_id: Pubkey::default(),
1110            collector_fees: AtomicU64::default(),
1111            fee_rate_governor: FeeRateGovernor::default(),
1112            collected_rent: AtomicU64::default(),
1113            rent_collector: RentCollector::default(),
1114            epoch_schedule: EpochSchedule::default(),
1115            inflation: Arc::<RwLock<Inflation>>::default(),
1116            stakes_cache: StakesCache::default(),
1117            epoch_stakes: HashMap::<Epoch, EpochStakes>::default(),
1118            is_delta: AtomicBool::default(),
1119            rewards: RwLock::<Vec<(Pubkey, RewardInfo)>>::default(),
1120            cluster_type: Option::<ClusterType>::default(),
1121            lazy_rent_collection: AtomicBool::default(),
1122            rewards_pool_pubkeys: Arc::<HashSet<Pubkey>>::default(),
1123            transaction_debug_keys: Option::<Arc<HashSet<Pubkey>>>::default(),
1124            transaction_log_collector_config: Arc::<RwLock<TransactionLogCollectorConfig>>::default(
1125            ),
1126            transaction_log_collector: Arc::<RwLock<TransactionLogCollector>>::default(),
1127            feature_set: Arc::<FeatureSet>::default(),
1128            reserved_account_keys: Arc::<ReservedAccountKeys>::default(),
1129            drop_callback: RwLock::new(OptionalDropCallback(None)),
1130            freeze_started: AtomicBool::default(),
1131            vote_only_bank: false,
1132            cost_tracker: RwLock::<CostTracker>::default(),
1133            accounts_data_size_initial: 0,
1134            accounts_data_size_delta_on_chain: AtomicI64::new(0),
1135            accounts_data_size_delta_off_chain: AtomicI64::new(0),
1136            epoch_reward_status: EpochRewardStatus::default(),
1137            transaction_processor: TransactionBatchProcessor::default(),
1138            check_program_modification_slot: false,
1139            collector_fee_details: RwLock::new(CollectorFeeDetails::default()),
1140            compute_budget: None,
1141            transaction_account_lock_limit: None,
1142            fee_structure: FeeStructure::default(),
1143            #[cfg(feature = "dev-context-only-utils")]
1144            hash_overrides: Arc::new(Mutex::new(HashOverrides::default())),
1145            accounts_lt_hash: Mutex::new(AccountsLtHash(LtHash::identity())),
1146            cache_for_accounts_lt_hash: DashMap::default(),
1147            stats_for_accounts_lt_hash: AccountsLtHashStats::default(),
1148            block_id: RwLock::new(None),
1149            bank_hash_stats: AtomicBankHashStats::default(),
1150        };
1151
1152        bank.transaction_processor =
1153            TransactionBatchProcessor::new_uninitialized(bank.slot, bank.epoch);
1154
1155        let accounts_data_size_initial = bank.get_total_accounts_stats().unwrap().data_len as u64;
1156        bank.accounts_data_size_initial = accounts_data_size_initial;
1157
1158        bank
1159    }
1160
1161    #[allow(clippy::too_many_arguments)]
1162    pub fn new_with_paths(
1163        genesis_config: &GenesisConfig,
1164        runtime_config: Arc<RuntimeConfig>,
1165        paths: Vec<PathBuf>,
1166        debug_keys: Option<Arc<HashSet<Pubkey>>>,
1167        additional_builtins: Option<&[BuiltinPrototype]>,
1168        debug_do_not_add_builtins: bool,
1169        accounts_db_config: Option<AccountsDbConfig>,
1170        accounts_update_notifier: Option<AccountsUpdateNotifier>,
1171        #[allow(unused)] collector_id_for_tests: Option<Pubkey>,
1172        exit: Arc<AtomicBool>,
1173        #[allow(unused)] genesis_hash: Option<Hash>,
1174        #[allow(unused)] feature_set: Option<FeatureSet>,
1175    ) -> Self {
1176        let accounts_db =
1177            AccountsDb::new_with_config(paths, accounts_db_config, accounts_update_notifier, exit);
1178        let accounts = Accounts::new(Arc::new(accounts_db));
1179        let mut bank = Self::default_with_accounts(accounts);
1180        bank.ancestors = Ancestors::from(vec![bank.slot()]);
1181        bank.compute_budget = runtime_config.compute_budget;
1182        bank.transaction_account_lock_limit = runtime_config.transaction_account_lock_limit;
1183        bank.transaction_debug_keys = debug_keys;
1184        bank.cluster_type = Some(genesis_config.cluster_type);
1185
1186        #[cfg(feature = "dev-context-only-utils")]
1187        {
1188            bank.feature_set = Arc::new(feature_set.unwrap_or_default());
1189        }
1190
1191        #[cfg(not(feature = "dev-context-only-utils"))]
1192        bank.process_genesis_config(genesis_config);
1193        #[cfg(feature = "dev-context-only-utils")]
1194        bank.process_genesis_config(genesis_config, collector_id_for_tests, genesis_hash);
1195
1196        bank.finish_init(
1197            genesis_config,
1198            additional_builtins,
1199            debug_do_not_add_builtins,
1200        );
1201
1202        // genesis needs stakes for all epochs up to the epoch implied by
1203        //  slot = 0 and genesis configuration
1204        {
1205            let stakes = bank.stakes_cache.stakes().clone();
1206            let stakes = Arc::new(StakesEnum::from(stakes));
1207            for epoch in 0..=bank.get_leader_schedule_epoch(bank.slot) {
1208                bank.epoch_stakes
1209                    .insert(epoch, EpochStakes::new(stakes.clone(), epoch));
1210            }
1211            bank.update_stake_history(None);
1212        }
1213        bank.update_clock(None);
1214        bank.update_rent();
1215        bank.update_epoch_schedule();
1216        bank.update_recent_blockhashes();
1217        bank.update_last_restart_slot();
1218        bank.transaction_processor
1219            .fill_missing_sysvar_cache_entries(&bank);
1220        bank
1221    }
1222
1223    /// Create a new bank that points to an immutable checkpoint of another bank.
1224    pub fn new_from_parent(parent: Arc<Bank>, collector_id: &Pubkey, slot: Slot) -> Self {
1225        Self::_new_from_parent(
1226            parent,
1227            collector_id,
1228            slot,
1229            null_tracer(),
1230            NewBankOptions::default(),
1231        )
1232    }
1233
1234    pub fn new_from_parent_with_options(
1235        parent: Arc<Bank>,
1236        collector_id: &Pubkey,
1237        slot: Slot,
1238        new_bank_options: NewBankOptions,
1239    ) -> Self {
1240        Self::_new_from_parent(parent, collector_id, slot, null_tracer(), new_bank_options)
1241    }
1242
1243    pub fn new_from_parent_with_tracer(
1244        parent: Arc<Bank>,
1245        collector_id: &Pubkey,
1246        slot: Slot,
1247        reward_calc_tracer: impl RewardCalcTracer,
1248    ) -> Self {
1249        Self::_new_from_parent(
1250            parent,
1251            collector_id,
1252            slot,
1253            Some(reward_calc_tracer),
1254            NewBankOptions::default(),
1255        )
1256    }
1257
1258    fn get_rent_collector_from(rent_collector: &RentCollector, epoch: Epoch) -> RentCollector {
1259        rent_collector.clone_with_epoch(epoch)
1260    }
1261
1262    fn _new_from_parent(
1263        parent: Arc<Bank>,
1264        collector_id: &Pubkey,
1265        slot: Slot,
1266        reward_calc_tracer: Option<impl RewardCalcTracer>,
1267        new_bank_options: NewBankOptions,
1268    ) -> Self {
1269        let mut time = Measure::start("bank::new_from_parent");
1270        let NewBankOptions { vote_only_bank } = new_bank_options;
1271
1272        parent.freeze();
1273        assert_ne!(slot, parent.slot());
1274
1275        let epoch_schedule = parent.epoch_schedule().clone();
1276        let epoch = epoch_schedule.get_epoch(slot);
1277
1278        let (rc, bank_rc_creation_time_us) = measure_us!({
1279            let accounts_db = Arc::clone(&parent.rc.accounts.accounts_db);
1280            BankRc {
1281                accounts: Arc::new(Accounts::new(accounts_db)),
1282                parent: RwLock::new(Some(Arc::clone(&parent))),
1283                bank_id_generator: Arc::clone(&parent.rc.bank_id_generator),
1284            }
1285        });
1286
1287        let (status_cache, status_cache_time_us) = measure_us!(Arc::clone(&parent.status_cache));
1288
1289        let (fee_rate_governor, fee_components_time_us) = measure_us!(
1290            FeeRateGovernor::new_derived(&parent.fee_rate_governor, parent.signature_count())
1291        );
1292
1293        let bank_id = rc.bank_id_generator.fetch_add(1, Relaxed) + 1;
1294        let (blockhash_queue, blockhash_queue_time_us) =
1295            measure_us!(RwLock::new(parent.blockhash_queue.read().unwrap().clone()));
1296
1297        let (stakes_cache, stakes_cache_time_us) =
1298            measure_us!(StakesCache::new(parent.stakes_cache.stakes().clone()));
1299
1300        let (epoch_stakes, epoch_stakes_time_us) = measure_us!(parent.epoch_stakes.clone());
1301
1302        let (transaction_processor, builtin_program_ids_time_us) = measure_us!(
1303            TransactionBatchProcessor::new_from(&parent.transaction_processor, slot, epoch)
1304        );
1305
1306        let (rewards_pool_pubkeys, rewards_pool_pubkeys_time_us) =
1307            measure_us!(parent.rewards_pool_pubkeys.clone());
1308
1309        let (transaction_debug_keys, transaction_debug_keys_time_us) =
1310            measure_us!(parent.transaction_debug_keys.clone());
1311
1312        let (transaction_log_collector_config, transaction_log_collector_config_time_us) =
1313            measure_us!(parent.transaction_log_collector_config.clone());
1314
1315        let (feature_set, feature_set_time_us) = measure_us!(parent.feature_set.clone());
1316
1317        let accounts_data_size_initial = parent.load_accounts_data_size();
1318        let mut new = Self {
1319            skipped_rewrites: Mutex::default(),
1320            rc,
1321            status_cache,
1322            slot,
1323            bank_id,
1324            epoch,
1325            blockhash_queue,
1326
1327            // TODO: clean this up, so much special-case copying...
1328            hashes_per_tick: parent.hashes_per_tick,
1329            ticks_per_slot: parent.ticks_per_slot,
1330            ns_per_slot: parent.ns_per_slot,
1331            genesis_creation_time: parent.genesis_creation_time,
1332            slots_per_year: parent.slots_per_year,
1333            epoch_schedule,
1334            collected_rent: AtomicU64::new(0),
1335            rent_collector: Self::get_rent_collector_from(&parent.rent_collector, epoch),
1336            max_tick_height: slot
1337                .checked_add(1)
1338                .expect("max tick height addition overflowed")
1339                .checked_mul(parent.ticks_per_slot)
1340                .expect("max tick height multiplication overflowed"),
1341            block_height: parent
1342                .block_height
1343                .checked_add(1)
1344                .expect("block height addition overflowed"),
1345            fee_rate_governor,
1346            capitalization: AtomicU64::new(parent.capitalization()),
1347            vote_only_bank,
1348            inflation: parent.inflation.clone(),
1349            transaction_count: AtomicU64::new(parent.transaction_count()),
1350            non_vote_transaction_count_since_restart: AtomicU64::new(
1351                parent.non_vote_transaction_count_since_restart(),
1352            ),
1353            transaction_error_count: AtomicU64::new(0),
1354            transaction_entries_count: AtomicU64::new(0),
1355            transactions_per_entry_max: AtomicU64::new(0),
1356            // we will .clone_with_epoch() this soon after stake data update; so just .clone() for now
1357            stakes_cache,
1358            epoch_stakes,
1359            parent_hash: parent.hash(),
1360            parent_slot: parent.slot(),
1361            collector_id: *collector_id,
1362            collector_fees: AtomicU64::new(0),
1363            ancestors: Ancestors::default(),
1364            hash: RwLock::new(Hash::default()),
1365            is_delta: AtomicBool::new(false),
1366            tick_height: AtomicU64::new(parent.tick_height.load(Relaxed)),
1367            signature_count: AtomicU64::new(0),
1368            hard_forks: parent.hard_forks.clone(),
1369            rewards: RwLock::new(vec![]),
1370            cluster_type: parent.cluster_type,
1371            lazy_rent_collection: AtomicBool::new(parent.lazy_rent_collection.load(Relaxed)),
1372            rewards_pool_pubkeys,
1373            transaction_debug_keys,
1374            transaction_log_collector_config,
1375            transaction_log_collector: Arc::new(RwLock::new(TransactionLogCollector::default())),
1376            feature_set: Arc::clone(&feature_set),
1377            reserved_account_keys: parent.reserved_account_keys.clone(),
1378            drop_callback: RwLock::new(OptionalDropCallback(
1379                parent
1380                    .drop_callback
1381                    .read()
1382                    .unwrap()
1383                    .0
1384                    .as_ref()
1385                    .map(|drop_callback| drop_callback.clone_box()),
1386            )),
1387            freeze_started: AtomicBool::new(false),
1388            cost_tracker: RwLock::new(parent.read_cost_tracker().unwrap().new_from_parent_limits()),
1389            accounts_data_size_initial,
1390            accounts_data_size_delta_on_chain: AtomicI64::new(0),
1391            accounts_data_size_delta_off_chain: AtomicI64::new(0),
1392            epoch_reward_status: parent.epoch_reward_status.clone(),
1393            transaction_processor,
1394            check_program_modification_slot: false,
1395            collector_fee_details: RwLock::new(CollectorFeeDetails::default()),
1396            compute_budget: parent.compute_budget,
1397            transaction_account_lock_limit: parent.transaction_account_lock_limit,
1398            fee_structure: parent.fee_structure.clone(),
1399            #[cfg(feature = "dev-context-only-utils")]
1400            hash_overrides: parent.hash_overrides.clone(),
1401            accounts_lt_hash: Mutex::new(parent.accounts_lt_hash.lock().unwrap().clone()),
1402            cache_for_accounts_lt_hash: DashMap::default(),
1403            stats_for_accounts_lt_hash: AccountsLtHashStats::default(),
1404            block_id: RwLock::new(None),
1405            bank_hash_stats: AtomicBankHashStats::default(),
1406        };
1407
1408        let (_, ancestors_time_us) = measure_us!({
1409            let mut ancestors = Vec::with_capacity(1 + new.parents().len());
1410            ancestors.push(new.slot());
1411            new.parents().iter().for_each(|p| {
1412                ancestors.push(p.slot());
1413            });
1414            new.ancestors = Ancestors::from(ancestors);
1415        });
1416
1417        // Following code may touch AccountsDb, requiring proper ancestors
1418        let (_, update_epoch_time_us) = measure_us!({
1419            if parent.epoch() < new.epoch() {
1420                new.process_new_epoch(
1421                    parent.epoch(),
1422                    parent.slot(),
1423                    parent.block_height(),
1424                    reward_calc_tracer,
1425                );
1426            } else {
1427                // Save a snapshot of stakes for use in consensus and stake weighted networking
1428                let leader_schedule_epoch = new.epoch_schedule().get_leader_schedule_epoch(slot);
1429                new.update_epoch_stakes(leader_schedule_epoch);
1430            }
1431            new.distribute_partitioned_epoch_rewards();
1432        });
1433
1434        let (_epoch, slot_index) = new.epoch_schedule.get_epoch_and_slot_index(new.slot);
1435        let slots_in_epoch = new.epoch_schedule.get_slots_in_epoch(new.epoch);
1436
1437        let (_, cache_preparation_time_us) = measure_us!(new
1438            .transaction_processor
1439            .prepare_program_cache_for_upcoming_feature_set(
1440                &new,
1441                &new.compute_active_feature_set(true).0,
1442                &new.compute_budget.unwrap_or_default(),
1443                slot_index,
1444                slots_in_epoch,
1445            ));
1446
1447        // Update sysvars before processing transactions
1448        let (_, update_sysvars_time_us) = measure_us!({
1449            new.update_slot_hashes();
1450            new.update_stake_history(Some(parent.epoch()));
1451            new.update_clock(Some(parent.epoch()));
1452            new.update_last_restart_slot()
1453        });
1454
1455        let (_, fill_sysvar_cache_time_us) = measure_us!(new
1456            .transaction_processor
1457            .fill_missing_sysvar_cache_entries(&new));
1458
1459        let (num_accounts_modified_this_slot, populate_cache_for_accounts_lt_hash_us) = new
1460            .is_accounts_lt_hash_enabled()
1461            .then(|| {
1462                measure_us!({
1463                    // The cache for accounts lt hash needs to be made aware of accounts modified
1464                    // before transaction processing begins.  Otherwise we may calculate the wrong
1465                    // accounts lt hash due to having the wrong initial state of the account.  The
1466                    // lt hash cache's initial state must always be from an ancestor, and cannot be
1467                    // an intermediate state within this Bank's slot.  If the lt hash cache has the
1468                    // wrong initial account state, we'll mix out the wrong lt hash value, and thus
1469                    // have the wrong overall accounts lt hash, and diverge.
1470                    let accounts_modified_this_slot =
1471                        new.rc.accounts.accounts_db.get_pubkeys_for_slot(slot);
1472                    let num_accounts_modified_this_slot = accounts_modified_this_slot.len();
1473                    for pubkey in accounts_modified_this_slot {
1474                        new.cache_for_accounts_lt_hash
1475                            .entry(pubkey)
1476                            .or_insert(AccountsLtHashCacheValue::BankNew);
1477                    }
1478                    num_accounts_modified_this_slot
1479                })
1480            })
1481            .unzip();
1482
1483        time.stop();
1484        report_new_bank_metrics(
1485            slot,
1486            parent.slot(),
1487            new.block_height,
1488            num_accounts_modified_this_slot,
1489            NewBankTimings {
1490                bank_rc_creation_time_us,
1491                total_elapsed_time_us: time.as_us(),
1492                status_cache_time_us,
1493                fee_components_time_us,
1494                blockhash_queue_time_us,
1495                stakes_cache_time_us,
1496                epoch_stakes_time_us,
1497                builtin_program_ids_time_us,
1498                rewards_pool_pubkeys_time_us,
1499                executor_cache_time_us: 0,
1500                transaction_debug_keys_time_us,
1501                transaction_log_collector_config_time_us,
1502                feature_set_time_us,
1503                ancestors_time_us,
1504                update_epoch_time_us,
1505                cache_preparation_time_us,
1506                update_sysvars_time_us,
1507                fill_sysvar_cache_time_us,
1508                populate_cache_for_accounts_lt_hash_us,
1509            },
1510        );
1511
1512        report_loaded_programs_stats(
1513            &parent
1514                .transaction_processor
1515                .program_cache
1516                .read()
1517                .unwrap()
1518                .stats,
1519            parent.slot(),
1520        );
1521
1522        new.transaction_processor
1523            .program_cache
1524            .write()
1525            .unwrap()
1526            .stats
1527            .reset();
1528
1529        new
1530    }
1531
1532    pub fn set_fork_graph_in_program_cache(&self, fork_graph: Weak<RwLock<BankForks>>) {
1533        self.transaction_processor
1534            .program_cache
1535            .write()
1536            .unwrap()
1537            .set_fork_graph(fork_graph);
1538    }
1539
1540    pub fn prune_program_cache(&self, new_root_slot: Slot, new_root_epoch: Epoch) {
1541        self.transaction_processor
1542            .program_cache
1543            .write()
1544            .unwrap()
1545            .prune(new_root_slot, new_root_epoch);
1546    }
1547
1548    pub fn prune_program_cache_by_deployment_slot(&self, deployment_slot: Slot) {
1549        self.transaction_processor
1550            .program_cache
1551            .write()
1552            .unwrap()
1553            .prune_by_deployment_slot(deployment_slot);
1554    }
1555
1556    /// Epoch in which the new cooldown warmup rate for stake was activated
1557    pub fn new_warmup_cooldown_rate_epoch(&self) -> Option<Epoch> {
1558        self.feature_set
1559            .new_warmup_cooldown_rate_epoch(&self.epoch_schedule)
1560    }
1561
1562    /// process for the start of a new epoch
1563    fn process_new_epoch(
1564        &mut self,
1565        parent_epoch: Epoch,
1566        parent_slot: Slot,
1567        parent_height: u64,
1568        reward_calc_tracer: Option<impl RewardCalcTracer>,
1569    ) {
1570        let epoch = self.epoch();
1571        let slot = self.slot();
1572        let (thread_pool, thread_pool_time_us) = measure_us!(ThreadPoolBuilder::new()
1573            .thread_name(|i| format!("solBnkNewEpch{i:02}"))
1574            .build()
1575            .expect("new rayon threadpool"));
1576
1577        let (_, apply_feature_activations_time_us) = measure_us!(thread_pool.install(|| {
1578            self.apply_feature_activations(ApplyFeatureActivationsCaller::NewFromParent, false)
1579        }));
1580
1581        // Add new entry to stakes.stake_history, set appropriate epoch and
1582        // update vote accounts with warmed up stakes before saving a
1583        // snapshot of stakes in epoch stakes
1584        let (_, activate_epoch_time_us) = measure_us!(self.stakes_cache.activate_epoch(
1585            epoch,
1586            &thread_pool,
1587            self.new_warmup_cooldown_rate_epoch()
1588        ));
1589
1590        // Save a snapshot of stakes for use in consensus and stake weighted networking
1591        let leader_schedule_epoch = self.epoch_schedule.get_leader_schedule_epoch(slot);
1592        let (_, update_epoch_stakes_time_us) =
1593            measure_us!(self.update_epoch_stakes(leader_schedule_epoch));
1594
1595        let mut rewards_metrics = RewardsMetrics::default();
1596        // After saving a snapshot of stakes, apply stake rewards and commission
1597        let (_, update_rewards_with_thread_pool_time_us) = measure_us!(self
1598            .begin_partitioned_rewards(
1599                reward_calc_tracer,
1600                &thread_pool,
1601                parent_epoch,
1602                parent_slot,
1603                parent_height,
1604                &mut rewards_metrics,
1605            ));
1606
1607        report_new_epoch_metrics(
1608            epoch,
1609            slot,
1610            parent_slot,
1611            NewEpochTimings {
1612                thread_pool_time_us,
1613                apply_feature_activations_time_us,
1614                activate_epoch_time_us,
1615                update_epoch_stakes_time_us,
1616                update_rewards_with_thread_pool_time_us,
1617            },
1618            rewards_metrics,
1619        );
1620    }
1621
1622    pub fn byte_limit_for_scans(&self) -> Option<usize> {
1623        self.rc
1624            .accounts
1625            .accounts_db
1626            .accounts_index
1627            .scan_results_limit_bytes
1628    }
1629
1630    pub fn proper_ancestors_set(&self) -> HashSet<Slot> {
1631        HashSet::from_iter(self.proper_ancestors())
1632    }
1633
1634    /// Returns all ancestors excluding self.slot.
1635    pub(crate) fn proper_ancestors(&self) -> impl Iterator<Item = Slot> + '_ {
1636        self.ancestors
1637            .keys()
1638            .into_iter()
1639            .filter(move |slot| *slot != self.slot)
1640    }
1641
1642    pub fn set_callback(&self, callback: Option<Box<dyn DropCallback + Send + Sync>>) {
1643        *self.drop_callback.write().unwrap() = OptionalDropCallback(callback);
1644    }
1645
1646    pub fn vote_only_bank(&self) -> bool {
1647        self.vote_only_bank
1648    }
1649
1650    /// Like `new_from_parent` but additionally:
1651    /// * Doesn't assume that the parent is anywhere near `slot`, parent could be millions of slots
1652    ///   in the past
1653    /// * Adjusts the new bank's tick height to avoid having to run PoH for millions of slots
1654    /// * Freezes the new bank, assuming that the user will `Bank::new_from_parent` from this bank
1655    /// * Calculates and sets the epoch accounts hash from the parent
1656    pub fn warp_from_parent(
1657        parent: Arc<Bank>,
1658        collector_id: &Pubkey,
1659        slot: Slot,
1660        data_source: CalcAccountsHashDataSource,
1661    ) -> Self {
1662        parent.freeze();
1663        parent
1664            .rc
1665            .accounts
1666            .accounts_db
1667            .epoch_accounts_hash_manager
1668            .set_in_flight(parent.slot());
1669        let accounts_hash = parent.update_accounts_hash(data_source, false, true);
1670        let epoch_accounts_hash = accounts_hash.into();
1671        parent
1672            .rc
1673            .accounts
1674            .accounts_db
1675            .epoch_accounts_hash_manager
1676            .set_valid(epoch_accounts_hash, parent.slot());
1677
1678        let parent_timestamp = parent.clock().unix_timestamp;
1679        let mut new = Bank::new_from_parent(parent, collector_id, slot);
1680        new.apply_feature_activations(ApplyFeatureActivationsCaller::WarpFromParent, false);
1681        new.update_epoch_stakes(new.epoch_schedule().get_epoch(slot));
1682        new.tick_height.store(new.max_tick_height(), Relaxed);
1683
1684        let mut clock = new.clock();
1685        clock.epoch_start_timestamp = parent_timestamp;
1686        clock.unix_timestamp = parent_timestamp;
1687        new.update_sysvar_account(&sysvar::clock::id(), |account| {
1688            create_account(
1689                &clock,
1690                new.inherit_specially_retained_account_fields(account),
1691            )
1692        });
1693        new.transaction_processor
1694            .fill_missing_sysvar_cache_entries(&new);
1695        new.freeze();
1696        new
1697    }
1698
1699    /// Create a bank from explicit arguments and deserialized fields from snapshot
1700    pub(crate) fn new_from_fields(
1701        bank_rc: BankRc,
1702        genesis_config: &GenesisConfig,
1703        runtime_config: Arc<RuntimeConfig>,
1704        fields: BankFieldsToDeserialize,
1705        debug_keys: Option<Arc<HashSet<Pubkey>>>,
1706        additional_builtins: Option<&[BuiltinPrototype]>,
1707        debug_do_not_add_builtins: bool,
1708        accounts_data_size_initial: u64,
1709    ) -> Self {
1710        let now = Instant::now();
1711        let ancestors = Ancestors::from(&fields.ancestors);
1712        // For backward compatibility, we can only serialize and deserialize
1713        // Stakes<Delegation> in BankFieldsTo{Serialize,Deserialize}. But Bank
1714        // caches Stakes<StakeAccount>. Below Stakes<StakeAccount> is obtained
1715        // from Stakes<Delegation> by reading the full account state from
1716        // accounts-db. Note that it is crucial that these accounts are loaded
1717        // at the right slot and match precisely with serialized Delegations.
1718        //
1719        // Note that we are disabling the read cache while we populate the stakes cache.
1720        // The stakes accounts will not be expected to be loaded again.
1721        // If we populate the read cache with these loads, then we'll just soon have to evict these.
1722        let (stakes, stakes_time) = measure_time!(Stakes::new(&fields.stakes, |pubkey| {
1723            let (account, _slot) = bank_rc
1724                .accounts
1725                .load_with_fixed_root_do_not_populate_read_cache(&ancestors, pubkey)?;
1726            Some(account)
1727        })
1728        .expect(
1729            "Stakes cache is inconsistent with accounts-db. This can indicate \
1730            a corrupted snapshot or bugs in cached accounts or accounts-db.",
1731        ));
1732        info!("Loading Stakes took: {stakes_time}");
1733        let stakes_accounts_load_duration = now.elapsed();
1734        let mut bank = Self {
1735            skipped_rewrites: Mutex::default(),
1736            rc: bank_rc,
1737            status_cache: Arc::<RwLock<BankStatusCache>>::default(),
1738            blockhash_queue: RwLock::new(fields.blockhash_queue),
1739            ancestors,
1740            hash: RwLock::new(fields.hash),
1741            parent_hash: fields.parent_hash,
1742            parent_slot: fields.parent_slot,
1743            hard_forks: Arc::new(RwLock::new(fields.hard_forks)),
1744            transaction_count: AtomicU64::new(fields.transaction_count),
1745            non_vote_transaction_count_since_restart: AtomicU64::default(),
1746            transaction_error_count: AtomicU64::default(),
1747            transaction_entries_count: AtomicU64::default(),
1748            transactions_per_entry_max: AtomicU64::default(),
1749            tick_height: AtomicU64::new(fields.tick_height),
1750            signature_count: AtomicU64::new(fields.signature_count),
1751            capitalization: AtomicU64::new(fields.capitalization),
1752            max_tick_height: fields.max_tick_height,
1753            hashes_per_tick: fields.hashes_per_tick,
1754            ticks_per_slot: fields.ticks_per_slot,
1755            ns_per_slot: fields.ns_per_slot,
1756            genesis_creation_time: fields.genesis_creation_time,
1757            slots_per_year: fields.slots_per_year,
1758            slot: fields.slot,
1759            bank_id: 0,
1760            epoch: fields.epoch,
1761            block_height: fields.block_height,
1762            collector_id: fields.collector_id,
1763            collector_fees: AtomicU64::new(fields.collector_fees),
1764            fee_rate_governor: fields.fee_rate_governor,
1765            collected_rent: AtomicU64::new(fields.collected_rent),
1766            // clone()-ing is needed to consider a gated behavior in rent_collector
1767            rent_collector: Self::get_rent_collector_from(&fields.rent_collector, fields.epoch),
1768            epoch_schedule: fields.epoch_schedule,
1769            inflation: Arc::new(RwLock::new(fields.inflation)),
1770            stakes_cache: StakesCache::new(stakes),
1771            epoch_stakes: fields.epoch_stakes,
1772            is_delta: AtomicBool::new(fields.is_delta),
1773            rewards: RwLock::new(vec![]),
1774            cluster_type: Some(genesis_config.cluster_type),
1775            lazy_rent_collection: AtomicBool::default(),
1776            rewards_pool_pubkeys: Arc::<HashSet<Pubkey>>::default(),
1777            transaction_debug_keys: debug_keys,
1778            transaction_log_collector_config: Arc::<RwLock<TransactionLogCollectorConfig>>::default(
1779            ),
1780            transaction_log_collector: Arc::<RwLock<TransactionLogCollector>>::default(),
1781            feature_set: Arc::<FeatureSet>::default(),
1782            reserved_account_keys: Arc::<ReservedAccountKeys>::default(),
1783            drop_callback: RwLock::new(OptionalDropCallback(None)),
1784            freeze_started: AtomicBool::new(fields.hash != Hash::default()),
1785            vote_only_bank: false,
1786            cost_tracker: RwLock::new(CostTracker::default()),
1787            accounts_data_size_initial,
1788            accounts_data_size_delta_on_chain: AtomicI64::new(0),
1789            accounts_data_size_delta_off_chain: AtomicI64::new(0),
1790            epoch_reward_status: EpochRewardStatus::default(),
1791            transaction_processor: TransactionBatchProcessor::default(),
1792            check_program_modification_slot: false,
1793            // collector_fee_details is not serialized to snapshot
1794            collector_fee_details: RwLock::new(CollectorFeeDetails::default()),
1795            compute_budget: runtime_config.compute_budget,
1796            transaction_account_lock_limit: runtime_config.transaction_account_lock_limit,
1797            fee_structure: FeeStructure::default(),
1798            #[cfg(feature = "dev-context-only-utils")]
1799            hash_overrides: Arc::new(Mutex::new(HashOverrides::default())),
1800            accounts_lt_hash: Mutex::new(AccountsLtHash(LtHash([0xBAD1; LtHash::NUM_ELEMENTS]))),
1801            cache_for_accounts_lt_hash: DashMap::default(),
1802            stats_for_accounts_lt_hash: AccountsLtHashStats::default(),
1803            block_id: RwLock::new(None),
1804            bank_hash_stats: AtomicBankHashStats::new(&fields.bank_hash_stats),
1805        };
1806
1807        bank.transaction_processor =
1808            TransactionBatchProcessor::new_uninitialized(bank.slot, bank.epoch);
1809
1810        let thread_pool = ThreadPoolBuilder::new()
1811            .thread_name(|i| format!("solBnkNewFlds{i:02}"))
1812            .build()
1813            .expect("new rayon threadpool");
1814        bank.recalculate_partitioned_rewards(null_tracer(), &thread_pool);
1815
1816        bank.finish_init(
1817            genesis_config,
1818            additional_builtins,
1819            debug_do_not_add_builtins,
1820        );
1821        bank.transaction_processor
1822            .fill_missing_sysvar_cache_entries(&bank);
1823        bank.rebuild_skipped_rewrites();
1824
1825        let mut calculate_accounts_lt_hash_duration = None;
1826        if let Some(accounts_lt_hash) = fields.accounts_lt_hash {
1827            *bank.accounts_lt_hash.get_mut().unwrap() = accounts_lt_hash;
1828        } else {
1829            // Use the accounts lt hash from the snapshot, if present, otherwise calculate it.
1830            // When the feature gate is enabled, the snapshot *must* contain an accounts lt hash.
1831            assert!(
1832                !bank
1833                    .feature_set
1834                    .is_active(&feature_set::accounts_lt_hash::id()),
1835                "snapshot must have an accounts lt hash if the feature is enabled",
1836            );
1837            if bank.is_accounts_lt_hash_enabled() {
1838                info!(
1839                    "Calculating the accounts lt hash for slot {}...",
1840                    bank.slot(),
1841                );
1842                let (ancestors, slot) = if bank.is_frozen() {
1843                    // Loading from a snapshot necessarily means this slot was rooted, and thus
1844                    // the bank has been frozen.  So when calculating the accounts lt hash,
1845                    // do it based on *this slot*, not our parent, since
1846                    // update_accounts_lt_hash() will not be called on us again.
1847                    (bank.ancestors.clone(), bank.slot())
1848                } else {
1849                    // If the bank is not frozen (e.g. if called from tests), then when this bank
1850                    // is frozen later it will call `update_accounts_lt_hash()`.  Therefore, we
1851                    // must calculate the accounts lt hash *here* based on *our parent*, so that
1852                    // the accounts lt hash is correct after freezing.
1853                    let parent_ancestors = {
1854                        let mut ancestors = bank.ancestors.clone();
1855                        ancestors.remove(&bank.slot());
1856                        ancestors
1857                    };
1858                    (parent_ancestors, bank.parent_slot)
1859                };
1860                let (accounts_lt_hash, duration) = meas_dur!({
1861                    thread_pool.install(|| {
1862                        bank.rc
1863                            .accounts
1864                            .accounts_db
1865                            .calculate_accounts_lt_hash_at_startup_from_index(&ancestors, slot)
1866                    })
1867                });
1868                calculate_accounts_lt_hash_duration = Some(duration);
1869                *bank.accounts_lt_hash.get_mut().unwrap() = accounts_lt_hash;
1870                info!(
1871                    "Calculating the accounts lt hash for slot {}... \
1872                     Done in {duration:?}, accounts_lt_hash checksum: {}",
1873                    bank.slot(),
1874                    bank.accounts_lt_hash.get_mut().unwrap().0.checksum(),
1875                );
1876            }
1877        }
1878
1879        // Sanity assertions between bank snapshot and genesis config
1880        // Consider removing from serializable bank state
1881        // (BankFieldsToSerialize/BankFieldsToDeserialize) and initializing
1882        // from the passed in genesis_config instead (as new()/new_with_paths() already do)
1883        assert_eq!(
1884            bank.genesis_creation_time, genesis_config.creation_time,
1885            "Bank snapshot genesis creation time does not match genesis.bin creation time. \
1886             The snapshot and genesis.bin might pertain to different clusters"
1887        );
1888        assert_eq!(bank.ticks_per_slot, genesis_config.ticks_per_slot);
1889        assert_eq!(
1890            bank.ns_per_slot,
1891            genesis_config.poh_config.target_tick_duration.as_nanos()
1892                * genesis_config.ticks_per_slot as u128
1893        );
1894        assert_eq!(bank.max_tick_height, (bank.slot + 1) * bank.ticks_per_slot);
1895        assert_eq!(
1896            bank.slots_per_year,
1897            years_as_slots(
1898                1.0,
1899                &genesis_config.poh_config.target_tick_duration,
1900                bank.ticks_per_slot,
1901            )
1902        );
1903        assert_eq!(bank.epoch_schedule, genesis_config.epoch_schedule);
1904        assert_eq!(bank.epoch, bank.epoch_schedule.get_epoch(bank.slot));
1905
1906        datapoint_info!(
1907            "bank-new-from-fields",
1908            (
1909                "accounts_data_len-from-snapshot",
1910                fields.accounts_data_len as i64,
1911                i64
1912            ),
1913            (
1914                "accounts_data_len-from-generate_index",
1915                accounts_data_size_initial as i64,
1916                i64
1917            ),
1918            (
1919                "stakes_accounts_load_duration_us",
1920                stakes_accounts_load_duration.as_micros(),
1921                i64
1922            ),
1923            (
1924                "calculate_accounts_lt_hash_us",
1925                calculate_accounts_lt_hash_duration.as_ref().map(Duration::as_micros),
1926                Option<i64>
1927            ),
1928        );
1929        bank
1930    }
1931
1932    /// Return subset of bank fields representing serializable state
1933    pub(crate) fn get_fields_to_serialize(&self) -> BankFieldsToSerialize {
1934        let (epoch_stakes, versioned_epoch_stakes) = split_epoch_stakes(self.epoch_stakes.clone());
1935        BankFieldsToSerialize {
1936            blockhash_queue: self.blockhash_queue.read().unwrap().clone(),
1937            ancestors: AncestorsForSerialization::from(&self.ancestors),
1938            hash: *self.hash.read().unwrap(),
1939            parent_hash: self.parent_hash,
1940            parent_slot: self.parent_slot,
1941            hard_forks: self.hard_forks.read().unwrap().clone(),
1942            transaction_count: self.transaction_count.load(Relaxed),
1943            tick_height: self.tick_height.load(Relaxed),
1944            signature_count: self.signature_count.load(Relaxed),
1945            capitalization: self.capitalization.load(Relaxed),
1946            max_tick_height: self.max_tick_height,
1947            hashes_per_tick: self.hashes_per_tick,
1948            ticks_per_slot: self.ticks_per_slot,
1949            ns_per_slot: self.ns_per_slot,
1950            genesis_creation_time: self.genesis_creation_time,
1951            slots_per_year: self.slots_per_year,
1952            slot: self.slot,
1953            epoch: self.epoch,
1954            block_height: self.block_height,
1955            collector_id: self.collector_id,
1956            collector_fees: self.collector_fees.load(Relaxed),
1957            fee_rate_governor: self.fee_rate_governor.clone(),
1958            collected_rent: self.collected_rent.load(Relaxed),
1959            rent_collector: self.rent_collector.clone(),
1960            epoch_schedule: self.epoch_schedule.clone(),
1961            inflation: *self.inflation.read().unwrap(),
1962            stakes: StakesEnum::from(self.stakes_cache.stakes().clone()),
1963            epoch_stakes,
1964            is_delta: self.is_delta.load(Relaxed),
1965            accounts_data_len: self.load_accounts_data_size(),
1966            versioned_epoch_stakes,
1967            accounts_lt_hash: self
1968                .is_accounts_lt_hash_enabled()
1969                .then(|| self.accounts_lt_hash.lock().unwrap().clone()),
1970        }
1971    }
1972
1973    pub fn collector_id(&self) -> &Pubkey {
1974        &self.collector_id
1975    }
1976
1977    pub fn genesis_creation_time(&self) -> UnixTimestamp {
1978        self.genesis_creation_time
1979    }
1980
1981    pub fn slot(&self) -> Slot {
1982        self.slot
1983    }
1984
1985    pub fn bank_id(&self) -> BankId {
1986        self.bank_id
1987    }
1988
1989    pub fn epoch(&self) -> Epoch {
1990        self.epoch
1991    }
1992
1993    pub fn first_normal_epoch(&self) -> Epoch {
1994        self.epoch_schedule().first_normal_epoch
1995    }
1996
1997    pub fn freeze_lock(&self) -> RwLockReadGuard<Hash> {
1998        self.hash.read().unwrap()
1999    }
2000
2001    pub fn hash(&self) -> Hash {
2002        *self.hash.read().unwrap()
2003    }
2004
2005    pub fn is_frozen(&self) -> bool {
2006        *self.hash.read().unwrap() != Hash::default()
2007    }
2008
2009    pub fn freeze_started(&self) -> bool {
2010        self.freeze_started.load(Relaxed)
2011    }
2012
2013    pub fn status_cache_ancestors(&self) -> Vec<u64> {
2014        let mut roots = self.status_cache.read().unwrap().roots().clone();
2015        let min = roots.iter().min().cloned().unwrap_or(0);
2016        for ancestor in self.ancestors.keys() {
2017            if ancestor >= min {
2018                roots.insert(ancestor);
2019            }
2020        }
2021
2022        let mut ancestors: Vec<_> = roots.into_iter().collect();
2023        #[allow(clippy::stable_sort_primitive)]
2024        ancestors.sort();
2025        ancestors
2026    }
2027
2028    /// computed unix_timestamp at this slot height
2029    pub fn unix_timestamp_from_genesis(&self) -> i64 {
2030        self.genesis_creation_time.saturating_add(
2031            (self.slot as u128)
2032                .saturating_mul(self.ns_per_slot)
2033                .saturating_div(1_000_000_000) as i64,
2034        )
2035    }
2036
2037    fn update_sysvar_account<F>(&self, pubkey: &Pubkey, updater: F)
2038    where
2039        F: Fn(&Option<AccountSharedData>) -> AccountSharedData,
2040    {
2041        let old_account = self.get_account_with_fixed_root(pubkey);
2042        let mut new_account = updater(&old_account);
2043
2044        // When new sysvar comes into existence (with RENT_UNADJUSTED_INITIAL_BALANCE lamports),
2045        // this code ensures that the sysvar's balance is adjusted to be rent-exempt.
2046        //
2047        // More generally, this code always re-calculates for possible sysvar data size change,
2048        // although there is no such sysvars currently.
2049        self.adjust_sysvar_balance_for_rent(&mut new_account);
2050        self.store_account_and_update_capitalization(pubkey, &new_account);
2051    }
2052
2053    fn inherit_specially_retained_account_fields(
2054        &self,
2055        old_account: &Option<AccountSharedData>,
2056    ) -> InheritableAccountFields {
2057        const RENT_UNADJUSTED_INITIAL_BALANCE: u64 = 1;
2058
2059        (
2060            old_account
2061                .as_ref()
2062                .map(|a| a.lamports())
2063                .unwrap_or(RENT_UNADJUSTED_INITIAL_BALANCE),
2064            old_account
2065                .as_ref()
2066                .map(|a| a.rent_epoch())
2067                .unwrap_or(INITIAL_RENT_EPOCH),
2068        )
2069    }
2070
2071    pub fn clock(&self) -> sysvar::clock::Clock {
2072        from_account(&self.get_account(&sysvar::clock::id()).unwrap_or_default())
2073            .unwrap_or_default()
2074    }
2075
2076    fn update_clock(&self, parent_epoch: Option<Epoch>) {
2077        let mut unix_timestamp = self.clock().unix_timestamp;
2078        // set epoch_start_timestamp to None to warp timestamp
2079        let epoch_start_timestamp = {
2080            let epoch = if let Some(epoch) = parent_epoch {
2081                epoch
2082            } else {
2083                self.epoch()
2084            };
2085            let first_slot_in_epoch = self.epoch_schedule().get_first_slot_in_epoch(epoch);
2086            Some((first_slot_in_epoch, self.clock().epoch_start_timestamp))
2087        };
2088        let max_allowable_drift = MaxAllowableDrift {
2089            fast: MAX_ALLOWABLE_DRIFT_PERCENTAGE_FAST,
2090            slow: MAX_ALLOWABLE_DRIFT_PERCENTAGE_SLOW_V2,
2091        };
2092
2093        let ancestor_timestamp = self.clock().unix_timestamp;
2094        if let Some(timestamp_estimate) =
2095            self.get_timestamp_estimate(max_allowable_drift, epoch_start_timestamp)
2096        {
2097            unix_timestamp = timestamp_estimate;
2098            if timestamp_estimate < ancestor_timestamp {
2099                unix_timestamp = ancestor_timestamp;
2100            }
2101        }
2102        datapoint_info!(
2103            "bank-timestamp-correction",
2104            ("slot", self.slot(), i64),
2105            ("from_genesis", self.unix_timestamp_from_genesis(), i64),
2106            ("corrected", unix_timestamp, i64),
2107            ("ancestor_timestamp", ancestor_timestamp, i64),
2108        );
2109        let mut epoch_start_timestamp =
2110            // On epoch boundaries, update epoch_start_timestamp
2111            if parent_epoch.is_some() && parent_epoch.unwrap() != self.epoch() {
2112                unix_timestamp
2113            } else {
2114                self.clock().epoch_start_timestamp
2115            };
2116        if self.slot == 0 {
2117            unix_timestamp = self.unix_timestamp_from_genesis();
2118            epoch_start_timestamp = self.unix_timestamp_from_genesis();
2119        }
2120        let clock = sysvar::clock::Clock {
2121            slot: self.slot,
2122            epoch_start_timestamp,
2123            epoch: self.epoch_schedule().get_epoch(self.slot),
2124            leader_schedule_epoch: self.epoch_schedule().get_leader_schedule_epoch(self.slot),
2125            unix_timestamp,
2126        };
2127        self.update_sysvar_account(&sysvar::clock::id(), |account| {
2128            create_account(
2129                &clock,
2130                self.inherit_specially_retained_account_fields(account),
2131            )
2132        });
2133    }
2134
2135    pub fn update_last_restart_slot(&self) {
2136        let feature_flag = self
2137            .feature_set
2138            .is_active(&feature_set::last_restart_slot_sysvar::id());
2139
2140        if feature_flag {
2141            // First, see what the currently stored last restart slot is. This
2142            // account may not exist yet if the feature was just activated.
2143            let current_last_restart_slot = self
2144                .get_account(&sysvar::last_restart_slot::id())
2145                .and_then(|account| {
2146                    let lrs: Option<LastRestartSlot> = from_account(&account);
2147                    lrs
2148                })
2149                .map(|account| account.last_restart_slot);
2150
2151            let last_restart_slot = {
2152                let slot = self.slot;
2153                let hard_forks_r = self.hard_forks.read().unwrap();
2154
2155                // Only consider hard forks <= this bank's slot to avoid prematurely applying
2156                // a hard fork that is set to occur in the future.
2157                hard_forks_r
2158                    .iter()
2159                    .rev()
2160                    .find(|(hard_fork, _)| *hard_fork <= slot)
2161                    .map(|(slot, _)| *slot)
2162                    .unwrap_or(0)
2163            };
2164
2165            // Only need to write if the last restart has changed
2166            if current_last_restart_slot != Some(last_restart_slot) {
2167                self.update_sysvar_account(&sysvar::last_restart_slot::id(), |account| {
2168                    create_account(
2169                        &LastRestartSlot { last_restart_slot },
2170                        self.inherit_specially_retained_account_fields(account),
2171                    )
2172                });
2173            }
2174        }
2175    }
2176
2177    pub fn set_sysvar_for_tests<T>(&self, sysvar: &T)
2178    where
2179        T: Sysvar + SysvarId,
2180    {
2181        self.update_sysvar_account(&T::id(), |account| {
2182            create_account(
2183                sysvar,
2184                self.inherit_specially_retained_account_fields(account),
2185            )
2186        });
2187        // Simply force fill sysvar cache rather than checking which sysvar was
2188        // actually updated since tests don't need to be optimized for performance.
2189        self.transaction_processor.reset_sysvar_cache();
2190        self.transaction_processor
2191            .fill_missing_sysvar_cache_entries(self);
2192    }
2193
2194    fn update_slot_history(&self) {
2195        self.update_sysvar_account(&sysvar::slot_history::id(), |account| {
2196            let mut slot_history = account
2197                .as_ref()
2198                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
2199                .unwrap_or_default();
2200            slot_history.add(self.slot());
2201            create_account(
2202                &slot_history,
2203                self.inherit_specially_retained_account_fields(account),
2204            )
2205        });
2206    }
2207
2208    fn update_slot_hashes(&self) {
2209        self.update_sysvar_account(&sysvar::slot_hashes::id(), |account| {
2210            let mut slot_hashes = account
2211                .as_ref()
2212                .map(|account| from_account::<SlotHashes, _>(account).unwrap())
2213                .unwrap_or_default();
2214            slot_hashes.add(self.parent_slot, self.parent_hash);
2215            create_account(
2216                &slot_hashes,
2217                self.inherit_specially_retained_account_fields(account),
2218            )
2219        });
2220    }
2221
2222    pub fn get_slot_history(&self) -> SlotHistory {
2223        from_account(&self.get_account(&sysvar::slot_history::id()).unwrap()).unwrap()
2224    }
2225
2226    fn update_epoch_stakes(&mut self, leader_schedule_epoch: Epoch) {
2227        // update epoch_stakes cache
2228        //  if my parent didn't populate for this staker's epoch, we've
2229        //  crossed a boundary
2230        if !self.epoch_stakes.contains_key(&leader_schedule_epoch) {
2231            self.epoch_stakes.retain(|&epoch, _| {
2232                epoch >= leader_schedule_epoch.saturating_sub(MAX_LEADER_SCHEDULE_STAKES)
2233            });
2234            let stakes = self.stakes_cache.stakes().clone();
2235            let stakes = Arc::new(StakesEnum::from(stakes));
2236            let new_epoch_stakes = EpochStakes::new(stakes, leader_schedule_epoch);
2237            info!(
2238                "new epoch stakes, epoch: {}, total_stake: {}",
2239                leader_schedule_epoch,
2240                new_epoch_stakes.total_stake(),
2241            );
2242
2243            // It is expensive to log the details of epoch stakes. Only log them at "trace"
2244            // level for debugging purpose.
2245            if log::log_enabled!(log::Level::Trace) {
2246                let vote_stakes: HashMap<_, _> = self
2247                    .stakes_cache
2248                    .stakes()
2249                    .vote_accounts()
2250                    .delegated_stakes()
2251                    .map(|(pubkey, stake)| (*pubkey, stake))
2252                    .collect();
2253                trace!("new epoch stakes, stakes: {vote_stakes:#?}");
2254            }
2255            self.epoch_stakes
2256                .insert(leader_schedule_epoch, new_epoch_stakes);
2257        }
2258    }
2259
2260    #[cfg(feature = "dev-context-only-utils")]
2261    pub fn set_epoch_stakes_for_test(&mut self, epoch: Epoch, stakes: EpochStakes) {
2262        self.epoch_stakes.insert(epoch, stakes);
2263    }
2264
2265    fn update_rent(&self) {
2266        self.update_sysvar_account(&sysvar::rent::id(), |account| {
2267            create_account(
2268                &self.rent_collector.rent,
2269                self.inherit_specially_retained_account_fields(account),
2270            )
2271        });
2272    }
2273
2274    fn update_epoch_schedule(&self) {
2275        self.update_sysvar_account(&sysvar::epoch_schedule::id(), |account| {
2276            create_account(
2277                self.epoch_schedule(),
2278                self.inherit_specially_retained_account_fields(account),
2279            )
2280        });
2281    }
2282
2283    fn update_stake_history(&self, epoch: Option<Epoch>) {
2284        if epoch == Some(self.epoch()) {
2285            return;
2286        }
2287        // if I'm the first Bank in an epoch, ensure stake_history is updated
2288        self.update_sysvar_account(&sysvar::stake_history::id(), |account| {
2289            create_account::<sysvar::stake_history::StakeHistory>(
2290                self.stakes_cache.stakes().history(),
2291                self.inherit_specially_retained_account_fields(account),
2292            )
2293        });
2294    }
2295
2296    pub fn epoch_duration_in_years(&self, prev_epoch: Epoch) -> f64 {
2297        // period: time that has passed as a fraction of a year, basically the length of
2298        //  an epoch as a fraction of a year
2299        //  calculated as: slots_elapsed / (slots / year)
2300        self.epoch_schedule().get_slots_in_epoch(prev_epoch) as f64 / self.slots_per_year
2301    }
2302
2303    // Calculates the starting-slot for inflation from the activation slot.
2304    // This method assumes that `pico_inflation` will be enabled before `full_inflation`, giving
2305    // precedence to the latter. However, since `pico_inflation` is fixed-rate Inflation, should
2306    // `pico_inflation` be enabled 2nd, the incorrect start slot provided here should have no
2307    // effect on the inflation calculation.
2308    fn get_inflation_start_slot(&self) -> Slot {
2309        let mut slots = self
2310            .feature_set
2311            .full_inflation_features_enabled()
2312            .iter()
2313            .filter_map(|id| self.feature_set.activated_slot(id))
2314            .collect::<Vec<_>>();
2315        slots.sort_unstable();
2316        slots.first().cloned().unwrap_or_else(|| {
2317            self.feature_set
2318                .activated_slot(&feature_set::pico_inflation::id())
2319                .unwrap_or(0)
2320        })
2321    }
2322
2323    fn get_inflation_num_slots(&self) -> u64 {
2324        let inflation_activation_slot = self.get_inflation_start_slot();
2325        // Normalize inflation_start to align with the start of rewards accrual.
2326        let inflation_start_slot = self.epoch_schedule().get_first_slot_in_epoch(
2327            self.epoch_schedule()
2328                .get_epoch(inflation_activation_slot)
2329                .saturating_sub(1),
2330        );
2331        self.epoch_schedule().get_first_slot_in_epoch(self.epoch()) - inflation_start_slot
2332    }
2333
2334    pub fn slot_in_year_for_inflation(&self) -> f64 {
2335        let num_slots = self.get_inflation_num_slots();
2336
2337        // calculated as: num_slots / (slots / year)
2338        num_slots as f64 / self.slots_per_year
2339    }
2340
2341    fn calculate_previous_epoch_inflation_rewards(
2342        &self,
2343        prev_epoch_capitalization: u64,
2344        prev_epoch: Epoch,
2345    ) -> PrevEpochInflationRewards {
2346        let slot_in_year = self.slot_in_year_for_inflation();
2347        let (validator_rate, foundation_rate) = {
2348            let inflation = self.inflation.read().unwrap();
2349            (
2350                (*inflation).validator(slot_in_year),
2351                (*inflation).foundation(slot_in_year),
2352            )
2353        };
2354
2355        let prev_epoch_duration_in_years = self.epoch_duration_in_years(prev_epoch);
2356        let validator_rewards = (validator_rate
2357            * prev_epoch_capitalization as f64
2358            * prev_epoch_duration_in_years) as u64;
2359
2360        PrevEpochInflationRewards {
2361            validator_rewards,
2362            prev_epoch_duration_in_years,
2363            validator_rate,
2364            foundation_rate,
2365        }
2366    }
2367
2368    fn assert_validator_rewards_paid(&self, validator_rewards_paid: u64) {
2369        assert_eq!(
2370            validator_rewards_paid,
2371            u64::try_from(
2372                self.rewards
2373                    .read()
2374                    .unwrap()
2375                    .par_iter()
2376                    .map(|(_address, reward_info)| {
2377                        match reward_info.reward_type {
2378                            RewardType::Voting | RewardType::Staking => reward_info.lamports,
2379                            _ => 0,
2380                        }
2381                    })
2382                    .sum::<i64>()
2383            )
2384            .unwrap()
2385        );
2386    }
2387
2388    fn filter_stake_delegations<'a>(
2389        &self,
2390        stakes: &'a Stakes<StakeAccount<Delegation>>,
2391    ) -> Vec<(&'a Pubkey, &'a StakeAccount<Delegation>)> {
2392        if self
2393            .feature_set
2394            .is_active(&feature_set::stake_minimum_delegation_for_rewards::id())
2395        {
2396            let num_stake_delegations = stakes.stake_delegations().len();
2397            let min_stake_delegation =
2398                solana_stake_program::get_minimum_delegation(&self.feature_set)
2399                    .max(LAMPORTS_PER_SOL);
2400
2401            let (stake_delegations, filter_time_us) = measure_us!(stakes
2402                .stake_delegations()
2403                .iter()
2404                .filter(|(_stake_pubkey, cached_stake_account)| {
2405                    cached_stake_account.delegation().stake >= min_stake_delegation
2406                })
2407                .collect::<Vec<_>>());
2408
2409            datapoint_info!(
2410                "stake_account_filter_time",
2411                ("filter_time_us", filter_time_us, i64),
2412                ("num_stake_delegations_before", num_stake_delegations, i64),
2413                ("num_stake_delegations_after", stake_delegations.len(), i64)
2414            );
2415            stake_delegations
2416        } else {
2417            stakes.stake_delegations().iter().collect()
2418        }
2419    }
2420
2421    /// return reward info for each vote account
2422    /// return account data for each vote account that needs to be stored
2423    /// This return value is a little awkward at the moment so that downstream existing code in the non-partitioned rewards code path can be re-used without duplication or modification.
2424    /// This function is copied from the existing code path's `store_vote_accounts`.
2425    /// The primary differences:
2426    /// - we want this fn to have no side effects (such as actually storing vote accounts) so that we
2427    ///   can compare the expected results with the current code path
2428    /// - we want to be able to batch store the vote accounts later for improved performance/cache updating
2429    fn calc_vote_accounts_to_store(
2430        vote_account_rewards: DashMap<Pubkey, VoteReward>,
2431    ) -> VoteRewardsAccounts {
2432        let len = vote_account_rewards.len();
2433        let mut result = VoteRewardsAccounts {
2434            rewards: Vec::with_capacity(len),
2435            accounts_to_store: Vec::with_capacity(len),
2436        };
2437        vote_account_rewards.into_iter().for_each(
2438            |(
2439                vote_pubkey,
2440                VoteReward {
2441                    mut vote_account,
2442                    commission,
2443                    vote_rewards,
2444                    vote_needs_store,
2445                },
2446            )| {
2447                if let Err(err) = vote_account.checked_add_lamports(vote_rewards) {
2448                    debug!("reward redemption failed for {}: {:?}", vote_pubkey, err);
2449                    return;
2450                }
2451
2452                result.rewards.push((
2453                    vote_pubkey,
2454                    RewardInfo {
2455                        reward_type: RewardType::Voting,
2456                        lamports: vote_rewards as i64,
2457                        post_balance: vote_account.lamports(),
2458                        commission: Some(commission),
2459                    },
2460                ));
2461                result
2462                    .accounts_to_store
2463                    .push(vote_needs_store.then_some(vote_account));
2464            },
2465        );
2466        result
2467    }
2468
2469    fn update_reward_history(
2470        &self,
2471        stake_rewards: StakeRewards,
2472        mut vote_rewards: Vec<(Pubkey, RewardInfo)>,
2473    ) {
2474        let additional_reserve = stake_rewards.len() + vote_rewards.len();
2475        let mut rewards = self.rewards.write().unwrap();
2476        rewards.reserve(additional_reserve);
2477        rewards.append(&mut vote_rewards);
2478        stake_rewards
2479            .into_iter()
2480            .filter(|x| x.get_stake_reward() > 0)
2481            .for_each(|x| rewards.push((x.stake_pubkey, x.stake_reward_info)));
2482    }
2483
2484    fn update_recent_blockhashes_locked(&self, locked_blockhash_queue: &BlockhashQueue) {
2485        #[allow(deprecated)]
2486        self.update_sysvar_account(&sysvar::recent_blockhashes::id(), |account| {
2487            let recent_blockhash_iter = locked_blockhash_queue.get_recent_blockhashes();
2488            recent_blockhashes_account::create_account_with_data_and_fields(
2489                recent_blockhash_iter,
2490                self.inherit_specially_retained_account_fields(account),
2491            )
2492        });
2493    }
2494
2495    pub fn update_recent_blockhashes(&self) {
2496        let blockhash_queue = self.blockhash_queue.read().unwrap();
2497        self.update_recent_blockhashes_locked(&blockhash_queue);
2498    }
2499
2500    fn get_timestamp_estimate(
2501        &self,
2502        max_allowable_drift: MaxAllowableDrift,
2503        epoch_start_timestamp: Option<(Slot, UnixTimestamp)>,
2504    ) -> Option<UnixTimestamp> {
2505        let mut get_timestamp_estimate_time = Measure::start("get_timestamp_estimate");
2506        let slots_per_epoch = self.epoch_schedule().slots_per_epoch;
2507        let vote_accounts = self.vote_accounts();
2508        let recent_timestamps = vote_accounts.iter().filter_map(|(pubkey, (_, account))| {
2509            let vote_state = account.vote_state();
2510            let slot_delta = self.slot().checked_sub(vote_state.last_timestamp.slot)?;
2511            (slot_delta <= slots_per_epoch).then_some({
2512                (
2513                    *pubkey,
2514                    (
2515                        vote_state.last_timestamp.slot,
2516                        vote_state.last_timestamp.timestamp,
2517                    ),
2518                )
2519            })
2520        });
2521        let slot_duration = Duration::from_nanos(self.ns_per_slot as u64);
2522        let epoch = self.epoch_schedule().get_epoch(self.slot());
2523        let stakes = self.epoch_vote_accounts(epoch)?;
2524        let stake_weighted_timestamp = calculate_stake_weighted_timestamp(
2525            recent_timestamps,
2526            stakes,
2527            self.slot(),
2528            slot_duration,
2529            epoch_start_timestamp,
2530            max_allowable_drift,
2531            self.feature_set
2532                .is_active(&feature_set::warp_timestamp_again::id()),
2533        );
2534        get_timestamp_estimate_time.stop();
2535        datapoint_info!(
2536            "bank-timestamp",
2537            (
2538                "get_timestamp_estimate_us",
2539                get_timestamp_estimate_time.as_us(),
2540                i64
2541            ),
2542        );
2543        stake_weighted_timestamp
2544    }
2545
2546    /// Recalculates the bank hash
2547    ///
2548    /// This is used by ledger-tool when creating a snapshot, which
2549    /// recalcuates the bank hash.
2550    ///
2551    /// Note that the account state is *not* allowed to change by rehashing.
2552    /// If modifying accounts in ledger-tool is needed, create a new bank.
2553    pub fn rehash(&self) {
2554        let get_delta_hash = || {
2555            (!self
2556                .feature_set
2557                .is_active(&feature_set::remove_accounts_delta_hash::id()))
2558            .then(|| {
2559                self.rc
2560                    .accounts
2561                    .accounts_db
2562                    .get_accounts_delta_hash(self.slot())
2563            })
2564            .flatten()
2565        };
2566
2567        let mut hash = self.hash.write().unwrap();
2568        let curr_accounts_delta_hash = get_delta_hash();
2569        let new = self.hash_internal_state();
2570        if let Some(curr_accounts_delta_hash) = curr_accounts_delta_hash {
2571            let new_accounts_delta_hash = get_delta_hash().unwrap();
2572            assert_eq!(
2573                new_accounts_delta_hash, curr_accounts_delta_hash,
2574                "rehashing is not allowed to change the account state",
2575            );
2576        }
2577        if new != *hash {
2578            warn!("Updating bank hash to {new}");
2579            *hash = new;
2580        }
2581    }
2582
2583    pub fn freeze(&self) {
2584        // This lock prevents any new commits from BankingStage
2585        // `Consumer::execute_and_commit_transactions_locked()` from
2586        // coming in after the last tick is observed. This is because in
2587        // BankingStage, any transaction successfully recorded in
2588        // `record_transactions()` is recorded after this `hash` lock
2589        // is grabbed. At the time of the successful record,
2590        // this means the PoH has not yet reached the last tick,
2591        // so this means freeze() hasn't been called yet. And because
2592        // BankingStage doesn't release this hash lock until both
2593        // record and commit are finished, those transactions will be
2594        // committed before this write lock can be obtained here.
2595        let mut hash = self.hash.write().unwrap();
2596        if *hash == Hash::default() {
2597            // finish up any deferred changes to account state
2598            self.collect_rent_eagerly();
2599            if self.feature_set.is_active(&reward_full_priority_fee::id()) {
2600                self.distribute_transaction_fee_details();
2601            } else {
2602                self.distribute_transaction_fees();
2603            }
2604            self.distribute_rent_fees();
2605            self.update_slot_history();
2606            self.run_incinerator();
2607
2608            // freeze is a one-way trip, idempotent
2609            self.freeze_started.store(true, Relaxed);
2610            if self.is_accounts_lt_hash_enabled() {
2611                // updating the accounts lt hash must happen *outside* of hash_internal_state() so
2612                // that rehash() can be called and *not* modify self.accounts_lt_hash.
2613                self.update_accounts_lt_hash();
2614
2615                // For lattice-hash R&D, we have a CLI arg to do extra verfication.  If set, we'll
2616                // re-calculate the accounts lt hash every slot and compare it against the value
2617                // already stored in the bank.
2618                if self
2619                    .rc
2620                    .accounts
2621                    .accounts_db
2622                    .verify_experimental_accumulator_hash
2623                {
2624                    let slot = self.slot();
2625                    info!("Verifying the accounts lt hash for slot {slot}...");
2626                    let (calculated_accounts_lt_hash, duration) = meas_dur!({
2627                        self.rc
2628                            .accounts
2629                            .accounts_db
2630                            .calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, slot)
2631                    });
2632                    let actual_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap();
2633                    assert_eq!(
2634                        calculated_accounts_lt_hash,
2635                        *actual_accounts_lt_hash,
2636                        "Verifying the accounts lt hash for slot {slot} failed! calculated checksum: {}, actual checksum: {}",
2637                        calculated_accounts_lt_hash.0.checksum(),
2638                        actual_accounts_lt_hash.0.checksum(),
2639                    );
2640                    info!("Verifying the accounts lt hash for slot {slot}... Done successfully in {duration:?}");
2641                }
2642            }
2643            *hash = self.hash_internal_state();
2644            self.rc.accounts.accounts_db.mark_slot_frozen(self.slot());
2645        }
2646    }
2647
2648    // dangerous; don't use this; this is only needed for ledger-tool's special command
2649    #[cfg(feature = "dev-context-only-utils")]
2650    pub fn unfreeze_for_ledger_tool(&self) {
2651        self.freeze_started.store(false, Relaxed);
2652    }
2653
2654    pub fn epoch_schedule(&self) -> &EpochSchedule {
2655        &self.epoch_schedule
2656    }
2657
2658    /// squash the parent's state up into this Bank,
2659    ///   this Bank becomes a root
2660    /// Note that this function is not thread-safe. If it is called concurrently on the same bank
2661    /// by multiple threads, the end result could be inconsistent.
2662    /// Calling code does not currently call this concurrently.
2663    pub fn squash(&self) -> SquashTiming {
2664        self.freeze();
2665
2666        //this bank and all its parents are now on the rooted path
2667        let mut roots = vec![self.slot()];
2668        roots.append(&mut self.parents().iter().map(|p| p.slot()).collect());
2669
2670        let mut total_index_us = 0;
2671        let mut total_cache_us = 0;
2672        let mut total_store_us = 0;
2673
2674        let mut squash_accounts_time = Measure::start("squash_accounts_time");
2675        for slot in roots.iter().rev() {
2676            // root forks cannot be purged
2677            let add_root_timing = self.rc.accounts.add_root(*slot);
2678            total_index_us += add_root_timing.index_us;
2679            total_cache_us += add_root_timing.cache_us;
2680            total_store_us += add_root_timing.store_us;
2681        }
2682        squash_accounts_time.stop();
2683
2684        *self.rc.parent.write().unwrap() = None;
2685
2686        let mut squash_cache_time = Measure::start("squash_cache_time");
2687        roots
2688            .iter()
2689            .for_each(|slot| self.status_cache.write().unwrap().add_root(*slot));
2690        squash_cache_time.stop();
2691
2692        SquashTiming {
2693            squash_accounts_ms: squash_accounts_time.as_ms(),
2694            squash_accounts_index_ms: total_index_us / 1000,
2695            squash_accounts_cache_ms: total_cache_us / 1000,
2696            squash_accounts_store_ms: total_store_us / 1000,
2697
2698            squash_cache_ms: squash_cache_time.as_ms(),
2699        }
2700    }
2701
2702    /// Return the more recent checkpoint of this bank instance.
2703    pub fn parent(&self) -> Option<Arc<Bank>> {
2704        self.rc.parent.read().unwrap().clone()
2705    }
2706
2707    pub fn parent_slot(&self) -> Slot {
2708        self.parent_slot
2709    }
2710
2711    pub fn parent_hash(&self) -> Hash {
2712        self.parent_hash
2713    }
2714
2715    fn process_genesis_config(
2716        &mut self,
2717        genesis_config: &GenesisConfig,
2718        #[cfg(feature = "dev-context-only-utils")] collector_id_for_tests: Option<Pubkey>,
2719        #[cfg(feature = "dev-context-only-utils")] genesis_hash: Option<Hash>,
2720    ) {
2721        // Bootstrap validator collects fees until `new_from_parent` is called.
2722        self.fee_rate_governor = genesis_config.fee_rate_governor.clone();
2723
2724        for (pubkey, account) in genesis_config.accounts.iter() {
2725            assert!(
2726                self.get_account(pubkey).is_none(),
2727                "{pubkey} repeated in genesis config"
2728            );
2729            self.store_account(pubkey, &account.to_account_shared_data());
2730            self.capitalization.fetch_add(account.lamports(), Relaxed);
2731            self.accounts_data_size_initial += account.data().len() as u64;
2732        }
2733
2734        for (pubkey, account) in genesis_config.rewards_pools.iter() {
2735            assert!(
2736                self.get_account(pubkey).is_none(),
2737                "{pubkey} repeated in genesis config"
2738            );
2739            self.store_account(pubkey, &account.to_account_shared_data());
2740            self.accounts_data_size_initial += account.data().len() as u64;
2741        }
2742
2743        // After storing genesis accounts, the bank stakes cache will be warmed
2744        // up and can be used to set the collector id to the highest staked
2745        // node. If no staked nodes exist, allow fallback to an unstaked test
2746        // collector id during tests.
2747        let collector_id = self.stakes_cache.stakes().highest_staked_node().copied();
2748        #[cfg(feature = "dev-context-only-utils")]
2749        let collector_id = collector_id.or(collector_id_for_tests);
2750        self.collector_id =
2751            collector_id.expect("genesis processing failed because no staked nodes exist");
2752
2753        #[cfg(not(feature = "dev-context-only-utils"))]
2754        let genesis_hash = genesis_config.hash();
2755        #[cfg(feature = "dev-context-only-utils")]
2756        let genesis_hash = genesis_hash.unwrap_or(genesis_config.hash());
2757
2758        self.blockhash_queue
2759            .write()
2760            .unwrap()
2761            .genesis_hash(&genesis_hash, self.fee_rate_governor.lamports_per_signature);
2762
2763        self.hashes_per_tick = genesis_config.hashes_per_tick();
2764        self.ticks_per_slot = genesis_config.ticks_per_slot();
2765        self.ns_per_slot = genesis_config.ns_per_slot();
2766        self.genesis_creation_time = genesis_config.creation_time;
2767        self.max_tick_height = (self.slot + 1) * self.ticks_per_slot;
2768        self.slots_per_year = genesis_config.slots_per_year();
2769
2770        self.epoch_schedule = genesis_config.epoch_schedule.clone();
2771
2772        self.inflation = Arc::new(RwLock::new(genesis_config.inflation));
2773
2774        self.rent_collector = RentCollector::new(
2775            self.epoch,
2776            self.epoch_schedule().clone(),
2777            self.slots_per_year,
2778            genesis_config.rent.clone(),
2779        );
2780
2781        // Add additional builtin programs specified in the genesis config
2782        for (name, program_id) in &genesis_config.native_instruction_processors {
2783            self.add_builtin_account(name, program_id);
2784        }
2785    }
2786
2787    fn burn_and_purge_account(&self, program_id: &Pubkey, mut account: AccountSharedData) {
2788        let old_data_size = account.data().len();
2789        self.capitalization.fetch_sub(account.lamports(), Relaxed);
2790        // Both resetting account balance to 0 and zeroing the account data
2791        // is needed to really purge from AccountsDb and flush the Stakes cache
2792        account.set_lamports(0);
2793        account.data_as_mut_slice().fill(0);
2794        self.store_account(program_id, &account);
2795        self.calculate_and_update_accounts_data_size_delta_off_chain(old_data_size, 0);
2796    }
2797
2798    /// Add a precompiled program account
2799    pub fn add_precompiled_account(&self, program_id: &Pubkey) {
2800        self.add_precompiled_account_with_owner(program_id, native_loader::id())
2801    }
2802
2803    // Used by tests to simulate clusters with precompiles that aren't owned by the native loader
2804    fn add_precompiled_account_with_owner(&self, program_id: &Pubkey, owner: Pubkey) {
2805        if let Some(account) = self.get_account_with_fixed_root(program_id) {
2806            if account.executable() {
2807                return;
2808            } else {
2809                // malicious account is pre-occupying at program_id
2810                self.burn_and_purge_account(program_id, account);
2811            }
2812        };
2813
2814        assert!(
2815            !self.freeze_started(),
2816            "Can't change frozen bank by adding not-existing new precompiled program ({program_id}). \
2817                Maybe, inconsistent program activation is detected on snapshot restore?"
2818        );
2819
2820        // Add a bogus executable account, which will be loaded and ignored.
2821        let (lamports, rent_epoch) = self.inherit_specially_retained_account_fields(&None);
2822
2823        let account = AccountSharedData::from(Account {
2824            lamports,
2825            owner,
2826            data: vec![],
2827            executable: true,
2828            rent_epoch,
2829        });
2830        self.store_account_and_update_capitalization(program_id, &account);
2831    }
2832
2833    pub fn set_rent_burn_percentage(&mut self, burn_percent: u8) {
2834        self.rent_collector.rent.burn_percent = burn_percent;
2835    }
2836
2837    pub fn set_hashes_per_tick(&mut self, hashes_per_tick: Option<u64>) {
2838        self.hashes_per_tick = hashes_per_tick;
2839    }
2840
2841    /// Return the last block hash registered.
2842    pub fn last_blockhash(&self) -> Hash {
2843        self.blockhash_queue.read().unwrap().last_hash()
2844    }
2845
2846    pub fn last_blockhash_and_lamports_per_signature(&self) -> (Hash, u64) {
2847        let blockhash_queue = self.blockhash_queue.read().unwrap();
2848        let last_hash = blockhash_queue.last_hash();
2849        let last_lamports_per_signature = blockhash_queue
2850            .get_lamports_per_signature(&last_hash)
2851            .unwrap(); // safe so long as the BlockhashQueue is consistent
2852        (last_hash, last_lamports_per_signature)
2853    }
2854
2855    pub fn is_blockhash_valid(&self, hash: &Hash) -> bool {
2856        let blockhash_queue = self.blockhash_queue.read().unwrap();
2857        blockhash_queue.is_hash_valid_for_age(hash, MAX_PROCESSING_AGE)
2858    }
2859
2860    pub fn get_minimum_balance_for_rent_exemption(&self, data_len: usize) -> u64 {
2861        self.rent_collector.rent.minimum_balance(data_len).max(1)
2862    }
2863
2864    pub fn get_lamports_per_signature(&self) -> u64 {
2865        self.fee_rate_governor.lamports_per_signature
2866    }
2867
2868    pub fn get_lamports_per_signature_for_blockhash(&self, hash: &Hash) -> Option<u64> {
2869        let blockhash_queue = self.blockhash_queue.read().unwrap();
2870        blockhash_queue.get_lamports_per_signature(hash)
2871    }
2872
2873    pub fn get_fee_for_message(&self, message: &SanitizedMessage) -> Option<u64> {
2874        let lamports_per_signature = {
2875            let blockhash_queue = self.blockhash_queue.read().unwrap();
2876            blockhash_queue.get_lamports_per_signature(message.recent_blockhash())
2877        }
2878        .or_else(|| {
2879            self.load_message_nonce_account(message).map(
2880                |(_nonce_address, _nonce_account, nonce_data)| {
2881                    nonce_data.get_lamports_per_signature()
2882                },
2883            )
2884        })?;
2885        Some(self.get_fee_for_message_with_lamports_per_signature(message, lamports_per_signature))
2886    }
2887
2888    /// Returns true when startup accounts hash verification has completed or never had to run in background.
2889    pub fn get_startup_verification_complete(&self) -> &Arc<AtomicBool> {
2890        &self
2891            .rc
2892            .accounts
2893            .accounts_db
2894            .verify_accounts_hash_in_bg
2895            .verified
2896    }
2897
2898    /// return true if bg hash verification is complete
2899    /// return false if bg hash verification has not completed yet
2900    /// if hash verification failed, a panic will occur
2901    pub fn is_startup_verification_complete(&self) -> bool {
2902        self.has_initial_accounts_hash_verification_completed()
2903    }
2904
2905    /// This can occur because it completed in the background
2906    /// or if the verification was run in the foreground.
2907    pub fn set_startup_verification_complete(&self) {
2908        self.set_initial_accounts_hash_verification_completed();
2909    }
2910
2911    pub fn get_fee_for_message_with_lamports_per_signature(
2912        &self,
2913        message: &impl SVMMessage,
2914        lamports_per_signature: u64,
2915    ) -> u64 {
2916        let fee_budget_limits = FeeBudgetLimits::from(
2917            process_compute_budget_instructions(
2918                message.program_instructions_iter(),
2919                &self.feature_set,
2920            )
2921            .unwrap_or_default(),
2922        );
2923        solana_fee::calculate_fee(
2924            message,
2925            lamports_per_signature == 0,
2926            self.fee_structure().lamports_per_signature,
2927            fee_budget_limits.prioritization_fee,
2928            FeeFeatures::from(self.feature_set.as_ref()),
2929        )
2930    }
2931
2932    pub fn get_blockhash_last_valid_block_height(&self, blockhash: &Hash) -> Option<Slot> {
2933        let blockhash_queue = self.blockhash_queue.read().unwrap();
2934        // This calculation will need to be updated to consider epoch boundaries if BlockhashQueue
2935        // length is made variable by epoch
2936        blockhash_queue
2937            .get_hash_age(blockhash)
2938            .map(|age| self.block_height + MAX_PROCESSING_AGE as u64 - age)
2939    }
2940
2941    pub fn confirmed_last_blockhash(&self) -> Hash {
2942        const NUM_BLOCKHASH_CONFIRMATIONS: usize = 3;
2943
2944        let parents = self.parents();
2945        if parents.is_empty() {
2946            self.last_blockhash()
2947        } else {
2948            let index = NUM_BLOCKHASH_CONFIRMATIONS.min(parents.len() - 1);
2949            parents[index].last_blockhash()
2950        }
2951    }
2952
2953    /// Forget all signatures. Useful for benchmarking.
2954    pub fn clear_signatures(&self) {
2955        self.status_cache.write().unwrap().clear();
2956    }
2957
2958    pub fn clear_slot_signatures(&self, slot: Slot) {
2959        self.status_cache.write().unwrap().clear_slot_entries(slot);
2960    }
2961
2962    fn update_transaction_statuses(
2963        &self,
2964        sanitized_txs: &[impl TransactionWithMeta],
2965        processing_results: &[TransactionProcessingResult],
2966    ) {
2967        let mut status_cache = self.status_cache.write().unwrap();
2968        assert_eq!(sanitized_txs.len(), processing_results.len());
2969        for (tx, processing_result) in sanitized_txs.iter().zip(processing_results) {
2970            if let Ok(processed_tx) = &processing_result {
2971                // Add the message hash to the status cache to ensure that this message
2972                // won't be processed again with a different signature.
2973                status_cache.insert(
2974                    tx.recent_blockhash(),
2975                    tx.message_hash(),
2976                    self.slot(),
2977                    processed_tx.status(),
2978                );
2979                // Add the transaction signature to the status cache so that transaction status
2980                // can be queried by transaction signature over RPC. In the future, this should
2981                // only be added for API nodes because voting validators don't need to do this.
2982                status_cache.insert(
2983                    tx.recent_blockhash(),
2984                    tx.signature(),
2985                    self.slot(),
2986                    processed_tx.status(),
2987                );
2988            }
2989        }
2990    }
2991
2992    /// Register a new recent blockhash in the bank's recent blockhash queue. Called when a bank
2993    /// reaches its max tick height. Can be called by tests to get new blockhashes for transaction
2994    /// processing without advancing to a new bank slot.
2995    fn register_recent_blockhash(&self, blockhash: &Hash, scheduler: &InstalledSchedulerRwLock) {
2996        // This is needed because recent_blockhash updates necessitate synchronizations for
2997        // consistent tx check_age handling.
2998        BankWithScheduler::wait_for_paused_scheduler(self, scheduler);
2999
3000        // Only acquire the write lock for the blockhash queue on block boundaries because
3001        // readers can starve this write lock acquisition and ticks would be slowed down too
3002        // much if the write lock is acquired for each tick.
3003        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
3004
3005        #[cfg(feature = "dev-context-only-utils")]
3006        let blockhash_override = self
3007            .hash_overrides
3008            .lock()
3009            .unwrap()
3010            .get_blockhash_override(self.slot())
3011            .copied()
3012            .inspect(|blockhash_override| {
3013                if blockhash_override != blockhash {
3014                    info!(
3015                        "bank: slot: {}: overrode blockhash: {} with {}",
3016                        self.slot(),
3017                        blockhash,
3018                        blockhash_override
3019                    );
3020                }
3021            });
3022        #[cfg(feature = "dev-context-only-utils")]
3023        let blockhash = blockhash_override.as_ref().unwrap_or(blockhash);
3024
3025        w_blockhash_queue.register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
3026        self.update_recent_blockhashes_locked(&w_blockhash_queue);
3027    }
3028
3029    // gating this under #[cfg(feature = "dev-context-only-utils")] isn't easy due to
3030    // solana-program-test's usage...
3031    pub fn register_unique_recent_blockhash_for_test(&self) {
3032        self.register_recent_blockhash(
3033            &Hash::new_unique(),
3034            &BankWithScheduler::no_scheduler_available(),
3035        )
3036    }
3037
3038    #[cfg(feature = "dev-context-only-utils")]
3039    pub fn register_recent_blockhash_for_test(
3040        &self,
3041        blockhash: &Hash,
3042        lamports_per_signature: Option<u64>,
3043    ) {
3044        // Only acquire the write lock for the blockhash queue on block boundaries because
3045        // readers can starve this write lock acquisition and ticks would be slowed down too
3046        // much if the write lock is acquired for each tick.
3047        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
3048        if let Some(lamports_per_signature) = lamports_per_signature {
3049            w_blockhash_queue.register_hash(blockhash, lamports_per_signature);
3050        } else {
3051            w_blockhash_queue
3052                .register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
3053        }
3054    }
3055
3056    /// Tell the bank which Entry IDs exist on the ledger. This function assumes subsequent calls
3057    /// correspond to later entries, and will boot the oldest ones once its internal cache is full.
3058    /// Once boot, the bank will reject transactions using that `hash`.
3059    ///
3060    /// This is NOT thread safe because if tick height is updated by two different threads, the
3061    /// block boundary condition could be missed.
3062    pub fn register_tick(&self, hash: &Hash, scheduler: &InstalledSchedulerRwLock) {
3063        assert!(
3064            !self.freeze_started(),
3065            "register_tick() working on a bank that is already frozen or is undergoing freezing!"
3066        );
3067
3068        if self.is_block_boundary(self.tick_height.load(Relaxed) + 1) {
3069            self.register_recent_blockhash(hash, scheduler);
3070        }
3071
3072        // ReplayStage will start computing the accounts delta hash when it
3073        // detects the tick height has reached the boundary, so the system
3074        // needs to guarantee all account updates for the slot have been
3075        // committed before this tick height is incremented (like the blockhash
3076        // sysvar above)
3077        self.tick_height.fetch_add(1, Relaxed);
3078    }
3079
3080    #[cfg(feature = "dev-context-only-utils")]
3081    pub fn register_tick_for_test(&self, hash: &Hash) {
3082        self.register_tick(hash, &BankWithScheduler::no_scheduler_available())
3083    }
3084
3085    #[cfg(feature = "dev-context-only-utils")]
3086    pub fn register_default_tick_for_test(&self) {
3087        self.register_tick_for_test(&Hash::default())
3088    }
3089
3090    #[cfg(feature = "dev-context-only-utils")]
3091    pub fn register_unique_tick(&self) {
3092        self.register_tick_for_test(&Hash::new_unique())
3093    }
3094
3095    pub fn is_complete(&self) -> bool {
3096        self.tick_height() == self.max_tick_height()
3097    }
3098
3099    pub fn is_block_boundary(&self, tick_height: u64) -> bool {
3100        tick_height == self.max_tick_height
3101    }
3102
3103    /// Get the max number of accounts that a transaction may lock in this block
3104    pub fn get_transaction_account_lock_limit(&self) -> usize {
3105        if let Some(transaction_account_lock_limit) = self.transaction_account_lock_limit {
3106            transaction_account_lock_limit
3107        } else if self
3108            .feature_set
3109            .is_active(&feature_set::increase_tx_account_lock_limit::id())
3110        {
3111            MAX_TX_ACCOUNT_LOCKS
3112        } else {
3113            64
3114        }
3115    }
3116
3117    /// Prepare a transaction batch from a list of versioned transactions from
3118    /// an entry. Used for tests only.
3119    pub fn prepare_entry_batch(
3120        &self,
3121        txs: Vec<VersionedTransaction>,
3122    ) -> Result<TransactionBatch<RuntimeTransaction<SanitizedTransaction>>> {
3123        let sanitized_txs = txs
3124            .into_iter()
3125            .map(|tx| {
3126                RuntimeTransaction::try_create(
3127                    tx,
3128                    MessageHash::Compute,
3129                    None,
3130                    self,
3131                    self.get_reserved_account_keys(),
3132                )
3133            })
3134            .collect::<Result<Vec<_>>>()?;
3135        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3136        let lock_results = self
3137            .rc
3138            .accounts
3139            .lock_accounts(sanitized_txs.iter(), tx_account_lock_limit);
3140        Ok(TransactionBatch::new(
3141            lock_results,
3142            self,
3143            OwnedOrBorrowed::Owned(sanitized_txs),
3144        ))
3145    }
3146
3147    /// Attempt to take locks on the accounts in a transaction batch
3148    pub fn try_lock_accounts(&self, txs: &[impl SVMMessage]) -> Vec<Result<()>> {
3149        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3150        self.rc
3151            .accounts
3152            .lock_accounts(txs.iter(), tx_account_lock_limit)
3153    }
3154
3155    /// Prepare a locked transaction batch from a list of sanitized transactions.
3156    pub fn prepare_sanitized_batch<'a, 'b, Tx: SVMMessage>(
3157        &'a self,
3158        txs: &'b [Tx],
3159    ) -> TransactionBatch<'a, 'b, Tx> {
3160        TransactionBatch::new(
3161            self.try_lock_accounts(txs),
3162            self,
3163            OwnedOrBorrowed::Borrowed(txs),
3164        )
3165    }
3166
3167    /// Prepare a locked transaction batch from a list of sanitized transactions, and their cost
3168    /// limited packing status
3169    pub fn prepare_sanitized_batch_with_results<'a, 'b, Tx: SVMMessage>(
3170        &'a self,
3171        transactions: &'b [Tx],
3172        transaction_results: impl Iterator<Item = Result<()>>,
3173    ) -> TransactionBatch<'a, 'b, Tx> {
3174        // this lock_results could be: Ok, AccountInUse, WouldExceedBlockMaxLimit or WouldExceedAccountMaxLimit
3175        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3176        let lock_results = self.rc.accounts.lock_accounts_with_results(
3177            transactions.iter(),
3178            transaction_results,
3179            tx_account_lock_limit,
3180        );
3181        TransactionBatch::new(lock_results, self, OwnedOrBorrowed::Borrowed(transactions))
3182    }
3183
3184    /// Prepare a transaction batch from a single transaction without locking accounts
3185    pub fn prepare_unlocked_batch_from_single_tx<'a, Tx: SVMMessage>(
3186        &'a self,
3187        transaction: &'a Tx,
3188    ) -> TransactionBatch<'a, 'a, Tx> {
3189        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3190        let lock_result = validate_account_locks(transaction.account_keys(), tx_account_lock_limit);
3191        let mut batch = TransactionBatch::new(
3192            vec![lock_result],
3193            self,
3194            OwnedOrBorrowed::Borrowed(slice::from_ref(transaction)),
3195        );
3196        batch.set_needs_unlock(false);
3197        batch
3198    }
3199
3200    /// Run transactions against a frozen bank without committing the results
3201    pub fn simulate_transaction(
3202        &self,
3203        transaction: &impl TransactionWithMeta,
3204        enable_cpi_recording: bool,
3205    ) -> TransactionSimulationResult {
3206        assert!(self.is_frozen(), "simulation bank must be frozen");
3207
3208        self.simulate_transaction_unchecked(transaction, enable_cpi_recording)
3209    }
3210
3211    /// Run transactions against a bank without committing the results; does not check if the bank
3212    /// is frozen, enabling use in single-Bank test frameworks
3213    pub fn simulate_transaction_unchecked(
3214        &self,
3215        transaction: &impl TransactionWithMeta,
3216        enable_cpi_recording: bool,
3217    ) -> TransactionSimulationResult {
3218        let account_keys = transaction.account_keys();
3219        let number_of_accounts = account_keys.len();
3220        let account_overrides = self.get_account_overrides_for_simulation(&account_keys);
3221        let batch = self.prepare_unlocked_batch_from_single_tx(transaction);
3222        let mut timings = ExecuteTimings::default();
3223
3224        let LoadAndExecuteTransactionsOutput {
3225            mut processing_results,
3226            ..
3227        } = self.load_and_execute_transactions(
3228            &batch,
3229            // After simulation, transactions will need to be forwarded to the leader
3230            // for processing. During forwarding, the transaction could expire if the
3231            // delay is not accounted for.
3232            MAX_PROCESSING_AGE - MAX_TRANSACTION_FORWARDING_DELAY,
3233            &mut timings,
3234            &mut TransactionErrorMetrics::default(),
3235            TransactionProcessingConfig {
3236                account_overrides: Some(&account_overrides),
3237                check_program_modification_slot: self.check_program_modification_slot,
3238                compute_budget: self.compute_budget(),
3239                log_messages_bytes_limit: None,
3240                limit_to_load_programs: true,
3241                recording_config: ExecutionRecordingConfig {
3242                    enable_cpi_recording,
3243                    enable_log_recording: true,
3244                    enable_return_data_recording: true,
3245                },
3246                transaction_account_lock_limit: Some(self.get_transaction_account_lock_limit()),
3247            },
3248        );
3249
3250        let units_consumed =
3251            timings
3252                .details
3253                .per_program_timings
3254                .iter()
3255                .fold(0, |acc: u64, (_, program_timing)| {
3256                    (std::num::Saturating(acc)
3257                        + program_timing.accumulated_units
3258                        + program_timing.total_errored_units)
3259                        .0
3260                });
3261
3262        debug!("simulate_transaction: {:?}", timings);
3263
3264        let processing_result = processing_results
3265            .pop()
3266            .unwrap_or(Err(TransactionError::InvalidProgramForExecution));
3267        let (post_simulation_accounts, result, logs, return_data, inner_instructions) =
3268            match processing_result {
3269                Ok(processed_tx) => match processed_tx {
3270                    ProcessedTransaction::Executed(executed_tx) => {
3271                        let details = executed_tx.execution_details;
3272                        let post_simulation_accounts = executed_tx
3273                            .loaded_transaction
3274                            .accounts
3275                            .into_iter()
3276                            .take(number_of_accounts)
3277                            .collect::<Vec<_>>();
3278                        (
3279                            post_simulation_accounts,
3280                            details.status,
3281                            details.log_messages,
3282                            details.return_data,
3283                            details.inner_instructions,
3284                        )
3285                    }
3286                    ProcessedTransaction::FeesOnly(fees_only_tx) => {
3287                        (vec![], Err(fees_only_tx.load_error), None, None, None)
3288                    }
3289                },
3290                Err(error) => (vec![], Err(error), None, None, None),
3291            };
3292        let logs = logs.unwrap_or_default();
3293
3294        TransactionSimulationResult {
3295            result,
3296            logs,
3297            post_simulation_accounts,
3298            units_consumed,
3299            return_data,
3300            inner_instructions,
3301        }
3302    }
3303
3304    fn get_account_overrides_for_simulation(&self, account_keys: &AccountKeys) -> AccountOverrides {
3305        let mut account_overrides = AccountOverrides::default();
3306        let slot_history_id = sysvar::slot_history::id();
3307        if account_keys.iter().any(|pubkey| *pubkey == slot_history_id) {
3308            let current_account = self.get_account_with_fixed_root(&slot_history_id);
3309            let slot_history = current_account
3310                .as_ref()
3311                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
3312                .unwrap_or_default();
3313            if slot_history.check(self.slot()) == Check::Found {
3314                let ancestors = Ancestors::from(self.proper_ancestors().collect::<Vec<_>>());
3315                if let Some((account, _)) =
3316                    self.load_slow_with_fixed_root(&ancestors, &slot_history_id)
3317                {
3318                    account_overrides.set_slot_history(Some(account));
3319                }
3320            }
3321        }
3322        account_overrides
3323    }
3324
3325    pub fn unlock_accounts<'a, Tx: SVMMessage + 'a>(
3326        &self,
3327        txs_and_results: impl Iterator<Item = (&'a Tx, &'a Result<()>)> + Clone,
3328    ) {
3329        self.rc.accounts.unlock_accounts(txs_and_results)
3330    }
3331
3332    pub fn remove_unrooted_slots(&self, slots: &[(Slot, BankId)]) {
3333        self.rc.accounts.accounts_db.remove_unrooted_slots(slots)
3334    }
3335
3336    pub fn get_hash_age(&self, hash: &Hash) -> Option<u64> {
3337        self.blockhash_queue.read().unwrap().get_hash_age(hash)
3338    }
3339
3340    pub fn is_hash_valid_for_age(&self, hash: &Hash, max_age: usize) -> bool {
3341        self.blockhash_queue
3342            .read()
3343            .unwrap()
3344            .is_hash_valid_for_age(hash, max_age)
3345    }
3346
3347    pub fn collect_balances(
3348        &self,
3349        batch: &TransactionBatch<impl SVMMessage>,
3350    ) -> TransactionBalances {
3351        let mut balances: TransactionBalances = vec![];
3352        for transaction in batch.sanitized_transactions() {
3353            let mut transaction_balances: Vec<u64> = vec![];
3354            for account_key in transaction.account_keys().iter() {
3355                transaction_balances.push(self.get_balance(account_key));
3356            }
3357            balances.push(transaction_balances);
3358        }
3359        balances
3360    }
3361
3362    pub fn load_and_execute_transactions(
3363        &self,
3364        batch: &TransactionBatch<impl TransactionWithMeta>,
3365        max_age: usize,
3366        timings: &mut ExecuteTimings,
3367        error_counters: &mut TransactionErrorMetrics,
3368        processing_config: TransactionProcessingConfig,
3369    ) -> LoadAndExecuteTransactionsOutput {
3370        let sanitized_txs = batch.sanitized_transactions();
3371
3372        let (check_results, check_us) = measure_us!(self.check_transactions(
3373            sanitized_txs,
3374            batch.lock_results(),
3375            max_age,
3376            error_counters,
3377        ));
3378        timings.saturating_add_in_place(ExecuteTimingType::CheckUs, check_us);
3379
3380        let (blockhash, blockhash_lamports_per_signature) =
3381            self.last_blockhash_and_lamports_per_signature();
3382        let rent_collector_with_metrics =
3383            RentCollectorWithMetrics::new(self.rent_collector.clone());
3384        let processing_environment = TransactionProcessingEnvironment {
3385            blockhash,
3386            blockhash_lamports_per_signature,
3387            epoch_total_stake: self.get_current_epoch_total_stake(),
3388            feature_set: Arc::clone(&self.feature_set),
3389            fee_lamports_per_signature: self.fee_structure.lamports_per_signature,
3390            rent_collector: Some(&rent_collector_with_metrics),
3391        };
3392
3393        let sanitized_output = self
3394            .transaction_processor
3395            .load_and_execute_sanitized_transactions(
3396                self,
3397                sanitized_txs,
3398                check_results,
3399                &processing_environment,
3400                &processing_config,
3401            );
3402
3403        // Accumulate the errors returned by the batch processor.
3404        error_counters.accumulate(&sanitized_output.error_metrics);
3405
3406        // Accumulate the transaction batch execution timings.
3407        timings.accumulate(&sanitized_output.execute_timings);
3408
3409        let ((), collect_logs_us) =
3410            measure_us!(self.collect_logs(sanitized_txs, &sanitized_output.processing_results));
3411        timings.saturating_add_in_place(ExecuteTimingType::CollectLogsUs, collect_logs_us);
3412
3413        let mut processed_counts = ProcessedTransactionCounts::default();
3414        let err_count = &mut error_counters.total;
3415
3416        for (processing_result, tx) in sanitized_output
3417            .processing_results
3418            .iter()
3419            .zip(sanitized_txs)
3420        {
3421            if let Some(debug_keys) = &self.transaction_debug_keys {
3422                for key in tx.account_keys().iter() {
3423                    if debug_keys.contains(key) {
3424                        let result = processing_result.flattened_result();
3425                        info!("slot: {} result: {:?} tx: {:?}", self.slot, result, tx);
3426                        break;
3427                    }
3428                }
3429            }
3430
3431            if processing_result.was_processed() {
3432                // Signature count must be accumulated only if the transaction
3433                // is processed, otherwise a mismatched count between banking
3434                // and replay could occur
3435                processed_counts.signature_count +=
3436                    tx.signature_details().num_transaction_signatures();
3437                processed_counts.processed_transactions_count += 1;
3438
3439                if !tx.is_simple_vote_transaction() {
3440                    processed_counts.processed_non_vote_transactions_count += 1;
3441                }
3442            }
3443
3444            match processing_result.flattened_result() {
3445                Ok(()) => {
3446                    processed_counts.processed_with_successful_result_count += 1;
3447                }
3448                Err(err) => {
3449                    if err_count.0 == 0 {
3450                        debug!("tx error: {:?} {:?}", err, tx);
3451                    }
3452                    *err_count += 1;
3453                }
3454            }
3455        }
3456
3457        LoadAndExecuteTransactionsOutput {
3458            processing_results: sanitized_output.processing_results,
3459            processed_counts,
3460        }
3461    }
3462
3463    fn collect_logs(
3464        &self,
3465        transactions: &[impl TransactionWithMeta],
3466        processing_results: &[TransactionProcessingResult],
3467    ) {
3468        let transaction_log_collector_config =
3469            self.transaction_log_collector_config.read().unwrap();
3470        if transaction_log_collector_config.filter == TransactionLogCollectorFilter::None {
3471            return;
3472        }
3473
3474        let collected_logs: Vec<_> = processing_results
3475            .iter()
3476            .zip(transactions)
3477            .filter_map(|(processing_result, transaction)| {
3478                // Skip log collection for unprocessed transactions
3479                let processed_tx = processing_result.processed_transaction()?;
3480                // Skip log collection for unexecuted transactions
3481                let execution_details = processed_tx.execution_details()?;
3482                Self::collect_transaction_logs(
3483                    &transaction_log_collector_config,
3484                    transaction,
3485                    execution_details,
3486                )
3487            })
3488            .collect();
3489
3490        if !collected_logs.is_empty() {
3491            let mut transaction_log_collector = self.transaction_log_collector.write().unwrap();
3492            for (log, filtered_mentioned_addresses) in collected_logs {
3493                let transaction_log_index = transaction_log_collector.logs.len();
3494                transaction_log_collector.logs.push(log);
3495                for key in filtered_mentioned_addresses.into_iter() {
3496                    transaction_log_collector
3497                        .mentioned_address_map
3498                        .entry(key)
3499                        .or_default()
3500                        .push(transaction_log_index);
3501                }
3502            }
3503        }
3504    }
3505
3506    fn collect_transaction_logs(
3507        transaction_log_collector_config: &TransactionLogCollectorConfig,
3508        transaction: &impl TransactionWithMeta,
3509        execution_details: &TransactionExecutionDetails,
3510    ) -> Option<(TransactionLogInfo, Vec<Pubkey>)> {
3511        // Skip log collection if no log messages were recorded
3512        let log_messages = execution_details.log_messages.as_ref()?;
3513
3514        let mut filtered_mentioned_addresses = Vec::new();
3515        if !transaction_log_collector_config
3516            .mentioned_addresses
3517            .is_empty()
3518        {
3519            for key in transaction.account_keys().iter() {
3520                if transaction_log_collector_config
3521                    .mentioned_addresses
3522                    .contains(key)
3523                {
3524                    filtered_mentioned_addresses.push(*key);
3525                }
3526            }
3527        }
3528
3529        let is_vote = transaction.is_simple_vote_transaction();
3530        let store = match transaction_log_collector_config.filter {
3531            TransactionLogCollectorFilter::All => {
3532                !is_vote || !filtered_mentioned_addresses.is_empty()
3533            }
3534            TransactionLogCollectorFilter::AllWithVotes => true,
3535            TransactionLogCollectorFilter::None => false,
3536            TransactionLogCollectorFilter::OnlyMentionedAddresses => {
3537                !filtered_mentioned_addresses.is_empty()
3538            }
3539        };
3540
3541        if store {
3542            Some((
3543                TransactionLogInfo {
3544                    signature: *transaction.signature(),
3545                    result: execution_details.status.clone(),
3546                    is_vote,
3547                    log_messages: log_messages.clone(),
3548                },
3549                filtered_mentioned_addresses,
3550            ))
3551        } else {
3552            None
3553        }
3554    }
3555
3556    /// Load the accounts data size, in bytes
3557    pub fn load_accounts_data_size(&self) -> u64 {
3558        self.accounts_data_size_initial
3559            .saturating_add_signed(self.load_accounts_data_size_delta())
3560    }
3561
3562    /// Load the change in accounts data size in this Bank, in bytes
3563    pub fn load_accounts_data_size_delta(&self) -> i64 {
3564        let delta_on_chain = self.load_accounts_data_size_delta_on_chain();
3565        let delta_off_chain = self.load_accounts_data_size_delta_off_chain();
3566        delta_on_chain.saturating_add(delta_off_chain)
3567    }
3568
3569    /// Load the change in accounts data size in this Bank, in bytes, from on-chain events
3570    /// i.e. transactions
3571    pub fn load_accounts_data_size_delta_on_chain(&self) -> i64 {
3572        self.accounts_data_size_delta_on_chain.load(Acquire)
3573    }
3574
3575    /// Load the change in accounts data size in this Bank, in bytes, from off-chain events
3576    /// i.e. rent collection
3577    pub fn load_accounts_data_size_delta_off_chain(&self) -> i64 {
3578        self.accounts_data_size_delta_off_chain.load(Acquire)
3579    }
3580
3581    /// Update the accounts data size delta from on-chain events by adding `amount`.
3582    /// The arithmetic saturates.
3583    fn update_accounts_data_size_delta_on_chain(&self, amount: i64) {
3584        if amount == 0 {
3585            return;
3586        }
3587
3588        self.accounts_data_size_delta_on_chain
3589            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_on_chain| {
3590                Some(accounts_data_size_delta_on_chain.saturating_add(amount))
3591            })
3592            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3593            .unwrap();
3594    }
3595
3596    /// Update the accounts data size delta from off-chain events by adding `amount`.
3597    /// The arithmetic saturates.
3598    fn update_accounts_data_size_delta_off_chain(&self, amount: i64) {
3599        if amount == 0 {
3600            return;
3601        }
3602
3603        self.accounts_data_size_delta_off_chain
3604            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_off_chain| {
3605                Some(accounts_data_size_delta_off_chain.saturating_add(amount))
3606            })
3607            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3608            .unwrap();
3609    }
3610
3611    /// Calculate the data size delta and update the off-chain accounts data size delta
3612    fn calculate_and_update_accounts_data_size_delta_off_chain(
3613        &self,
3614        old_data_size: usize,
3615        new_data_size: usize,
3616    ) {
3617        let data_size_delta = calculate_data_size_delta(old_data_size, new_data_size);
3618        self.update_accounts_data_size_delta_off_chain(data_size_delta);
3619    }
3620
3621    fn filter_program_errors_and_collect_fee(
3622        &self,
3623        processing_results: &[TransactionProcessingResult],
3624    ) {
3625        let mut fees = 0;
3626
3627        processing_results.iter().for_each(|processing_result| {
3628            if let Ok(processed_tx) = processing_result {
3629                fees += processed_tx.fee_details().total_fee();
3630            }
3631        });
3632
3633        self.collector_fees.fetch_add(fees, Relaxed);
3634    }
3635
3636    // Note: this function is not yet used; next PR will call it behind a feature gate
3637    fn filter_program_errors_and_collect_fee_details(
3638        &self,
3639        processing_results: &[TransactionProcessingResult],
3640    ) {
3641        let mut accumulated_fee_details = FeeDetails::default();
3642
3643        processing_results.iter().for_each(|processing_result| {
3644            if let Ok(processed_tx) = processing_result {
3645                accumulated_fee_details.accumulate(&processed_tx.fee_details());
3646            }
3647        });
3648
3649        self.collector_fee_details
3650            .write()
3651            .unwrap()
3652            .accumulate(&accumulated_fee_details);
3653    }
3654
3655    fn update_bank_hash_stats<'a>(&self, accounts: &impl StorableAccounts<'a>) {
3656        let mut stats = BankHashStats::default();
3657        (0..accounts.len()).for_each(|i| {
3658            accounts.account(i, |account| {
3659                stats.update(&account);
3660            })
3661        });
3662        self.bank_hash_stats.accumulate(&stats);
3663    }
3664
3665    pub fn commit_transactions(
3666        &self,
3667        sanitized_txs: &[impl TransactionWithMeta],
3668        processing_results: Vec<TransactionProcessingResult>,
3669        processed_counts: &ProcessedTransactionCounts,
3670        timings: &mut ExecuteTimings,
3671    ) -> Vec<TransactionCommitResult> {
3672        assert!(
3673            !self.freeze_started(),
3674            "commit_transactions() working on a bank that is already frozen or is undergoing freezing!"
3675        );
3676
3677        let ProcessedTransactionCounts {
3678            processed_transactions_count,
3679            processed_non_vote_transactions_count,
3680            processed_with_successful_result_count,
3681            signature_count,
3682        } = *processed_counts;
3683
3684        self.increment_transaction_count(processed_transactions_count);
3685        self.increment_non_vote_transaction_count_since_restart(
3686            processed_non_vote_transactions_count,
3687        );
3688        self.increment_signature_count(signature_count);
3689
3690        let processed_with_failure_result_count =
3691            processed_transactions_count.saturating_sub(processed_with_successful_result_count);
3692        self.transaction_error_count
3693            .fetch_add(processed_with_failure_result_count, Relaxed);
3694
3695        if processed_transactions_count > 0 {
3696            self.is_delta.store(true, Relaxed);
3697            self.transaction_entries_count.fetch_add(1, Relaxed);
3698            self.transactions_per_entry_max
3699                .fetch_max(processed_transactions_count, Relaxed);
3700        }
3701
3702        let ((), store_accounts_us) = measure_us!({
3703            // If geyser is present, we must collect `SanitizedTransaction`
3704            // references in order to comply with that interface - until it
3705            // is changed.
3706            let maybe_transaction_refs = self
3707                .accounts()
3708                .accounts_db
3709                .has_accounts_update_notifier()
3710                .then(|| {
3711                    sanitized_txs
3712                        .iter()
3713                        .map(|tx| tx.as_sanitized_transaction())
3714                        .collect::<Vec<_>>()
3715                });
3716
3717            let (accounts_to_store, transactions) = collect_accounts_to_store(
3718                sanitized_txs,
3719                &maybe_transaction_refs,
3720                &processing_results,
3721            );
3722
3723            let to_store = (self.slot(), accounts_to_store.as_slice());
3724            self.update_bank_hash_stats(&to_store);
3725            self.rc
3726                .accounts
3727                .store_cached(to_store, transactions.as_deref());
3728        });
3729
3730        self.collect_rent(&processing_results);
3731
3732        // Cached vote and stake accounts are synchronized with accounts-db
3733        // after each transaction.
3734        let ((), update_stakes_cache_us) =
3735            measure_us!(self.update_stakes_cache(sanitized_txs, &processing_results));
3736
3737        let ((), update_executors_us) = measure_us!({
3738            let mut cache = None;
3739            for processing_result in &processing_results {
3740                if let Some(ProcessedTransaction::Executed(executed_tx)) =
3741                    processing_result.processed_transaction()
3742                {
3743                    let programs_modified_by_tx = &executed_tx.programs_modified_by_tx;
3744                    if executed_tx.was_successful() && !programs_modified_by_tx.is_empty() {
3745                        cache
3746                            .get_or_insert_with(|| {
3747                                self.transaction_processor.program_cache.write().unwrap()
3748                            })
3749                            .merge(programs_modified_by_tx);
3750                    }
3751                }
3752            }
3753        });
3754
3755        let accounts_data_len_delta = processing_results
3756            .iter()
3757            .filter_map(|processing_result| processing_result.processed_transaction())
3758            .filter_map(|processed_tx| processed_tx.execution_details())
3759            .filter_map(|details| {
3760                details
3761                    .status
3762                    .is_ok()
3763                    .then_some(details.accounts_data_len_delta)
3764            })
3765            .sum();
3766        self.update_accounts_data_size_delta_on_chain(accounts_data_len_delta);
3767
3768        let ((), update_transaction_statuses_us) =
3769            measure_us!(self.update_transaction_statuses(sanitized_txs, &processing_results));
3770
3771        if self.feature_set.is_active(&reward_full_priority_fee::id()) {
3772            self.filter_program_errors_and_collect_fee_details(&processing_results)
3773        } else {
3774            self.filter_program_errors_and_collect_fee(&processing_results)
3775        };
3776
3777        timings.saturating_add_in_place(ExecuteTimingType::StoreUs, store_accounts_us);
3778        timings.saturating_add_in_place(
3779            ExecuteTimingType::UpdateStakesCacheUs,
3780            update_stakes_cache_us,
3781        );
3782        timings.saturating_add_in_place(ExecuteTimingType::UpdateExecutorsUs, update_executors_us);
3783        timings.saturating_add_in_place(
3784            ExecuteTimingType::UpdateTransactionStatuses,
3785            update_transaction_statuses_us,
3786        );
3787
3788        Self::create_commit_results(processing_results)
3789    }
3790
3791    fn create_commit_results(
3792        processing_results: Vec<TransactionProcessingResult>,
3793    ) -> Vec<TransactionCommitResult> {
3794        processing_results
3795            .into_iter()
3796            .map(|processing_result| {
3797                let processing_result = processing_result?;
3798                let executed_units = processing_result.executed_units();
3799                let loaded_accounts_data_size = processing_result.loaded_accounts_data_size();
3800
3801                match processing_result {
3802                    ProcessedTransaction::Executed(executed_tx) => {
3803                        let execution_details = executed_tx.execution_details;
3804                        let LoadedTransaction {
3805                            rent_debits,
3806                            accounts: loaded_accounts,
3807                            fee_details,
3808                            ..
3809                        } = executed_tx.loaded_transaction;
3810
3811                        // Rent is only collected for successfully executed transactions
3812                        let rent_debits = if execution_details.was_successful() {
3813                            rent_debits
3814                        } else {
3815                            RentDebits::default()
3816                        };
3817
3818                        Ok(CommittedTransaction {
3819                            status: execution_details.status,
3820                            log_messages: execution_details.log_messages,
3821                            inner_instructions: execution_details.inner_instructions,
3822                            return_data: execution_details.return_data,
3823                            executed_units,
3824                            fee_details,
3825                            rent_debits,
3826                            loaded_account_stats: TransactionLoadedAccountsStats {
3827                                loaded_accounts_count: loaded_accounts.len(),
3828                                loaded_accounts_data_size,
3829                            },
3830                        })
3831                    }
3832                    ProcessedTransaction::FeesOnly(fees_only_tx) => Ok(CommittedTransaction {
3833                        status: Err(fees_only_tx.load_error),
3834                        log_messages: None,
3835                        inner_instructions: None,
3836                        return_data: None,
3837                        executed_units,
3838                        rent_debits: RentDebits::default(),
3839                        fee_details: fees_only_tx.fee_details,
3840                        loaded_account_stats: TransactionLoadedAccountsStats {
3841                            loaded_accounts_count: fees_only_tx.rollback_accounts.count(),
3842                            loaded_accounts_data_size,
3843                        },
3844                    }),
3845                }
3846            })
3847            .collect()
3848    }
3849
3850    fn collect_rent(&self, processing_results: &[TransactionProcessingResult]) {
3851        let collected_rent = processing_results
3852            .iter()
3853            .filter_map(|processing_result| processing_result.processed_transaction())
3854            .filter_map(|processed_tx| processed_tx.executed_transaction())
3855            .filter(|executed_tx| executed_tx.was_successful())
3856            .map(|executed_tx| executed_tx.loaded_transaction.rent)
3857            .sum();
3858        self.collected_rent.fetch_add(collected_rent, Relaxed);
3859    }
3860
3861    fn run_incinerator(&self) {
3862        if let Some((account, _)) =
3863            self.get_account_modified_since_parent_with_fixed_root(&incinerator::id())
3864        {
3865            self.capitalization.fetch_sub(account.lamports(), Relaxed);
3866            self.store_account(&incinerator::id(), &AccountSharedData::default());
3867        }
3868    }
3869
3870    /// Get stake and stake node accounts
3871    pub(crate) fn get_stake_accounts(&self, minimized_account_set: &DashSet<Pubkey>) {
3872        self.stakes_cache
3873            .stakes()
3874            .stake_delegations()
3875            .iter()
3876            .for_each(|(pubkey, _)| {
3877                minimized_account_set.insert(*pubkey);
3878            });
3879
3880        self.stakes_cache
3881            .stakes()
3882            .staked_nodes()
3883            .par_iter()
3884            .for_each(|(pubkey, _)| {
3885                minimized_account_set.insert(*pubkey);
3886            });
3887    }
3888
3889    /// After deserialize, populate skipped rewrites with accounts that would normally
3890    /// have had their data rewritten in this slot due to rent collection (but didn't).
3891    ///
3892    /// This is required when starting up from a snapshot to verify the bank hash.
3893    ///
3894    /// A second usage is from the `bank_to_xxx_snapshot_archive()` functions.  These fns call
3895    /// `Bank::rehash()` to handle if the user manually modified any accounts and thus requires
3896    /// calculating the bank hash again.  Since calculating the bank hash *takes* the skipped
3897    /// rewrites, this second time will not have any skipped rewrites, and thus the hash would be
3898    /// updated to the wrong value.  So, rebuild the skipped rewrites before rehashing.
3899    fn rebuild_skipped_rewrites(&self) {
3900        // If the feature gate to *not* add rent collection rewrites to the bank hash is enabled,
3901        // then do *not* add anything to our skipped_rewrites.
3902        if self.bank_hash_skips_rent_rewrites() {
3903            return;
3904        }
3905
3906        let (skipped_rewrites, measure_skipped_rewrites) =
3907            measure_time!(self.calculate_skipped_rewrites());
3908        info!(
3909            "Rebuilding skipped rewrites of {} accounts{measure_skipped_rewrites}",
3910            skipped_rewrites.len()
3911        );
3912
3913        *self.skipped_rewrites.lock().unwrap() = skipped_rewrites;
3914    }
3915
3916    /// Calculates (and returns) skipped rewrites for this bank
3917    ///
3918    /// Refer to `rebuild_skipped_rewrites()` for more documentation.
3919    /// This implementation is purposely separate to facilitate testing.
3920    ///
3921    /// The key observation is that accounts in Bank::skipped_rewrites are only used IFF the
3922    /// specific account is *not* already in the accounts delta hash.  If an account is not in
3923    /// the accounts delta hash, then it means the account was not modified.  Since (basically)
3924    /// all accounts are rent exempt, this means (basically) all accounts are unmodified by rent
3925    /// collection.  So we just need to load the accounts that would've been checked for rent
3926    /// collection, hash them, and add them to Bank::skipped_rewrites.
3927    ///
3928    /// As of this writing, there are ~350 million acounts on mainnet-beta.
3929    /// Rent collection almost always collects a single slot at a time.
3930    /// So 1 slot of 432,000, of 350 million accounts, is ~800 accounts per slot.
3931    /// Since we haven't started processing anything yet, it should be fast enough to simply
3932    /// load the accounts directly.
3933    /// Empirically, this takes about 3-4 milliseconds.
3934    fn calculate_skipped_rewrites(&self) -> HashMap<Pubkey, AccountHash> {
3935        // The returned skipped rewrites may include accounts that were actually *not* skipped!
3936        // (This is safe, as per the fn's documentation above.)
3937        self.get_accounts_for_skipped_rewrites()
3938            .map(|(pubkey, account_hash, _account)| (pubkey, account_hash))
3939            .collect()
3940    }
3941
3942    /// Loads accounts that were selected for rent collection this slot.
3943    /// After loading the accounts, also calculate and return the account hashes.
3944    /// This is used when dealing with skipped rewrites.
3945    fn get_accounts_for_skipped_rewrites(
3946        &self,
3947    ) -> impl Iterator<Item = (Pubkey, AccountHash, AccountSharedData)> + '_ {
3948        self.rent_collection_partitions()
3949            .into_iter()
3950            .map(accounts_partition::pubkey_range_from_partition)
3951            .flat_map(|pubkey_range| {
3952                self.rc
3953                    .accounts
3954                    .load_to_collect_rent_eagerly(&self.ancestors, pubkey_range)
3955            })
3956            .map(|(pubkey, account, _slot)| {
3957                let account_hash = AccountsDb::hash_account(&account, &pubkey);
3958                (pubkey, account_hash, account)
3959            })
3960    }
3961
3962    /// Returns the accounts, sorted by pubkey, that were part of accounts delta hash calculation
3963    /// This is used when writing a bank hash details file.
3964    pub(crate) fn get_accounts_for_bank_hash_details(&self) -> Vec<PubkeyHashAccount> {
3965        let accounts_db = &self.rc.accounts.accounts_db;
3966
3967        let mut accounts_written_this_slot =
3968            accounts_db.get_pubkey_hash_account_for_slot(self.slot());
3969
3970        // If we are skipping rewrites but also include them in the accounts delta hash, then we
3971        // need to go load those accounts and add them to the list of accounts written this slot.
3972        if !self.bank_hash_skips_rent_rewrites()
3973            && accounts_db.test_skip_rewrites_but_include_in_bank_hash
3974        {
3975            let pubkeys_written_this_slot: HashSet<_> = accounts_written_this_slot
3976                .iter()
3977                .map(|pubkey_hash_account| pubkey_hash_account.pubkey)
3978                .collect();
3979
3980            let rent_collection_accounts = self.get_accounts_for_skipped_rewrites();
3981            for (pubkey, hash, account) in rent_collection_accounts {
3982                if !pubkeys_written_this_slot.contains(&pubkey) {
3983                    accounts_written_this_slot.push(PubkeyHashAccount {
3984                        pubkey,
3985                        hash,
3986                        account,
3987                    });
3988                }
3989            }
3990        }
3991
3992        // Sort the accounts by pubkey to match the order of the accounts delta hash.
3993        // This also makes comparison of files from different nodes deterministic.
3994        accounts_written_this_slot.sort_unstable_by_key(|account| account.pubkey);
3995        accounts_written_this_slot
3996    }
3997
3998    fn collect_rent_eagerly(&self) {
3999        if self.lazy_rent_collection.load(Relaxed) {
4000            return;
4001        }
4002
4003        if self
4004            .feature_set
4005            .is_active(&feature_set::disable_partitioned_rent_collection::id())
4006        {
4007            return;
4008        }
4009
4010        let mut measure = Measure::start("collect_rent_eagerly-ms");
4011        let partitions = self.rent_collection_partitions();
4012        let count = partitions.len();
4013        let rent_metrics = RentMetrics::default();
4014        // partitions will usually be 1, but could be more if we skip slots
4015        let mut parallel = count > 1;
4016        if parallel {
4017            let ranges = partitions
4018                .iter()
4019                .map(|partition| {
4020                    (
4021                        *partition,
4022                        accounts_partition::pubkey_range_from_partition(*partition),
4023                    )
4024                })
4025                .collect::<Vec<_>>();
4026            // test every range to make sure ranges are not overlapping
4027            // some tests collect rent from overlapping ranges
4028            // example: [(0, 31, 32), (0, 0, 128), (0, 27, 128)]
4029            // read-modify-write of an account for rent collection cannot be done in parallel
4030            'outer: for i in 0..ranges.len() {
4031                for j in 0..ranges.len() {
4032                    if i == j {
4033                        continue;
4034                    }
4035
4036                    let i = &ranges[i].1;
4037                    let j = &ranges[j].1;
4038                    // make sure i doesn't contain j
4039                    if i.contains(j.start()) || i.contains(j.end()) {
4040                        parallel = false;
4041                        break 'outer;
4042                    }
4043                }
4044            }
4045
4046            if parallel {
4047                let thread_pool = &self.rc.accounts.accounts_db.thread_pool;
4048                thread_pool.install(|| {
4049                    ranges.into_par_iter().for_each(|range| {
4050                        self.collect_rent_in_range(range.0, range.1, &rent_metrics)
4051                    });
4052                });
4053            }
4054        }
4055        if !parallel {
4056            // collect serially
4057            partitions
4058                .into_iter()
4059                .for_each(|partition| self.collect_rent_in_partition(partition, &rent_metrics));
4060        }
4061        measure.stop();
4062        datapoint_info!(
4063            "collect_rent_eagerly",
4064            ("accounts", rent_metrics.count.load(Relaxed), i64),
4065            ("partitions", count, i64),
4066            ("total_time_us", measure.as_us(), i64),
4067            (
4068                "hold_range_us",
4069                rent_metrics.hold_range_us.load(Relaxed),
4070                i64
4071            ),
4072            ("load_us", rent_metrics.load_us.load(Relaxed), i64),
4073            ("collect_us", rent_metrics.collect_us.load(Relaxed), i64),
4074            ("hash_us", rent_metrics.hash_us.load(Relaxed), i64),
4075            ("store_us", rent_metrics.store_us.load(Relaxed), i64),
4076        );
4077    }
4078
4079    fn rent_collection_partitions(&self) -> Vec<Partition> {
4080        if !self.use_fixed_collection_cycle() {
4081            // This mode is for production/development/testing.
4082            // In this mode, we iterate over the whole pubkey value range for each epochs
4083            // including warm-up epochs.
4084            // The only exception is the situation where normal epochs are relatively short
4085            // (currently less than 2 day). In that case, we arrange a single collection
4086            // cycle to be multiple of epochs so that a cycle could be greater than the 2 day.
4087            self.variable_cycle_partitions()
4088        } else {
4089            // This mode is mainly for benchmarking only.
4090            // In this mode, we always iterate over the whole pubkey value range with
4091            // <slot_count_in_two_day> slots as a collection cycle, regardless warm-up or
4092            // alignment between collection cycles and epochs.
4093            // Thus, we can simulate stable processing load of eager rent collection,
4094            // strictly proportional to the number of pubkeys since genesis.
4095            self.fixed_cycle_partitions()
4096        }
4097    }
4098
4099    /// true if rent collection does NOT rewrite accounts whose pubkey indicates
4100    ///  it is time for rent collection, but the account is rent exempt.
4101    /// false if rent collection DOES rewrite accounts if the account is rent exempt
4102    /// This is the default behavior historically.
4103    fn bank_hash_skips_rent_rewrites(&self) -> bool {
4104        self.feature_set
4105            .is_active(&feature_set::skip_rent_rewrites::id())
4106    }
4107
4108    /// true if rent fees should be collected (i.e. disable_rent_fees_collection is NOT enabled)
4109    fn should_collect_rent(&self) -> bool {
4110        !self
4111            .feature_set
4112            .is_active(&feature_set::disable_rent_fees_collection::id())
4113    }
4114
4115    /// Collect rent from `accounts`
4116    ///
4117    /// This fn is called inside a parallel loop from `collect_rent_in_partition()`.  Avoid adding
4118    /// any code that causes contention on shared memory/data (i.e. do not update atomic metrics).
4119    ///
4120    /// The return value is a struct of computed values that `collect_rent_in_partition()` will
4121    /// reduce at the end of its parallel loop.  If possible, place data/computation that cause
4122    /// contention/take locks in the return struct and process them in
4123    /// `collect_rent_from_partition()` after reducing the parallel loop.
4124    fn collect_rent_from_accounts(
4125        &self,
4126        mut accounts: Vec<(Pubkey, AccountSharedData, Slot)>,
4127        rent_paying_pubkeys: Option<&HashSet<Pubkey>>,
4128        partition_index: PartitionIndex,
4129    ) -> CollectRentFromAccountsInfo {
4130        let mut rent_debits = RentDebits::default();
4131        let mut total_rent_collected_info = CollectedInfo::default();
4132        let mut accounts_to_store =
4133            Vec::<(&Pubkey, &AccountSharedData)>::with_capacity(accounts.len());
4134        let mut time_collecting_rent_us = 0;
4135        let mut time_storing_accounts_us = 0;
4136        let can_skip_rewrites = self.bank_hash_skips_rent_rewrites();
4137        let test_skip_rewrites_but_include_in_bank_hash = self
4138            .rc
4139            .accounts
4140            .accounts_db
4141            .test_skip_rewrites_but_include_in_bank_hash;
4142        let mut skipped_rewrites = Vec::default();
4143        for (pubkey, account, _loaded_slot) in accounts.iter_mut() {
4144            let rent_epoch_pre = account.rent_epoch();
4145            let (rent_collected_info, collect_rent_us) = measure_us!(collect_rent_from_account(
4146                &self.feature_set,
4147                &self.rent_collector,
4148                pubkey,
4149                account
4150            ));
4151            time_collecting_rent_us += collect_rent_us;
4152            let rent_epoch_post = account.rent_epoch();
4153
4154            // did the account change in any way due to rent collection?
4155            let rent_epoch_changed = rent_epoch_post != rent_epoch_pre;
4156            let account_changed = rent_collected_info.rent_amount != 0 || rent_epoch_changed;
4157
4158            // always store the account, regardless if it changed or not
4159            let always_store_accounts =
4160                !can_skip_rewrites && !test_skip_rewrites_but_include_in_bank_hash;
4161
4162            // only store accounts where we collected rent
4163            // but get the hash for all these accounts even if collected rent is 0 (= not updated).
4164            // Also, there's another subtle side-effect from rewrites: this
4165            // ensures we verify the whole on-chain state (= all accounts)
4166            // via the bank delta hash slowly once per an epoch.
4167            if account_changed || always_store_accounts {
4168                if rent_collected_info.rent_amount > 0 {
4169                    if let Some(rent_paying_pubkeys) = rent_paying_pubkeys {
4170                        if !rent_paying_pubkeys.contains(pubkey) {
4171                            let partition_from_pubkey = accounts_partition::partition_from_pubkey(
4172                                pubkey,
4173                                self.epoch_schedule.slots_per_epoch,
4174                            );
4175                            // Submit datapoint instead of assert while we verify this is correct
4176                            datapoint_warn!(
4177                                "bank-unexpected_rent_paying_pubkey",
4178                                ("slot", self.slot(), i64),
4179                                ("pubkey", pubkey.to_string(), String),
4180                                ("partition_index", partition_index, i64),
4181                                ("partition_from_pubkey", partition_from_pubkey, i64)
4182                            );
4183                            warn!(
4184                                "Collecting rent from unexpected pubkey: {}, slot: {}, parent_slot: {:?}, \
4185                                partition_index: {}, partition_from_pubkey: {}",
4186                                pubkey,
4187                                self.slot(),
4188                                self.parent().map(|bank| bank.slot()),
4189                                partition_index,
4190                                partition_from_pubkey,
4191                            );
4192                        }
4193                    }
4194                } else {
4195                    debug_assert_eq!(rent_collected_info.rent_amount, 0);
4196                    if rent_epoch_changed {
4197                        datapoint_info!(
4198                            "bank-rent_collection_updated_only_rent_epoch",
4199                            ("slot", self.slot(), i64),
4200                            ("pubkey", pubkey.to_string(), String),
4201                            ("rent_epoch_pre", rent_epoch_pre, i64),
4202                            ("rent_epoch_post", rent_epoch_post, i64),
4203                        );
4204                    }
4205                }
4206                total_rent_collected_info += rent_collected_info;
4207                accounts_to_store.push((pubkey, account));
4208            } else if !account_changed
4209                && !can_skip_rewrites
4210                && test_skip_rewrites_but_include_in_bank_hash
4211            {
4212                // include rewrites that we skipped in the accounts delta hash.
4213                // This is what consensus requires prior to activation of bank_hash_skips_rent_rewrites.
4214                // This code path exists to allow us to test the long term effects on validators when the skipped rewrites
4215                // feature is enabled.
4216                let hash = AccountsDb::hash_account(account, pubkey);
4217                skipped_rewrites.push((*pubkey, hash));
4218            }
4219            rent_debits.insert(pubkey, rent_collected_info.rent_amount, account.lamports());
4220        }
4221
4222        if !accounts_to_store.is_empty() {
4223            // TODO: Maybe do not call `store_accounts()` here.  Instead return `accounts_to_store`
4224            // and have `collect_rent_in_partition()` perform all the stores.
4225            let (_, store_accounts_us) =
4226                measure_us!(self.store_accounts((self.slot(), &accounts_to_store[..])));
4227            time_storing_accounts_us += store_accounts_us;
4228        }
4229
4230        CollectRentFromAccountsInfo {
4231            skipped_rewrites,
4232            rent_collected_info: total_rent_collected_info,
4233            rent_rewards: rent_debits.into_unordered_rewards_iter().collect(),
4234            time_collecting_rent_us,
4235            time_storing_accounts_us,
4236            num_accounts: accounts.len(),
4237        }
4238    }
4239
4240    /// convert 'partition' to a pubkey range and 'collect_rent_in_range'
4241    fn collect_rent_in_partition(&self, partition: Partition, metrics: &RentMetrics) {
4242        let subrange_full = accounts_partition::pubkey_range_from_partition(partition);
4243        self.collect_rent_in_range(partition, subrange_full, metrics)
4244    }
4245
4246    /// get all pubkeys that we expect to be rent-paying or None, if this was not initialized at load time (that should only exist in test cases)
4247    fn get_rent_paying_pubkeys(&self, partition: &Partition) -> Option<HashSet<Pubkey>> {
4248        self.rc
4249            .accounts
4250            .accounts_db
4251            .accounts_index
4252            .rent_paying_accounts_by_partition
4253            .get()
4254            .and_then(|rent_paying_accounts| {
4255                rent_paying_accounts.is_initialized().then(|| {
4256                    accounts_partition::get_partition_end_indexes(partition)
4257                        .into_iter()
4258                        .flat_map(|end_index| {
4259                            rent_paying_accounts.get_pubkeys_in_partition_index(end_index)
4260                        })
4261                        .cloned()
4262                        .collect::<HashSet<_>>()
4263                })
4264            })
4265    }
4266
4267    /// load accounts with pubkeys in 'subrange_full'
4268    /// collect rent and update 'account.rent_epoch' as necessary
4269    /// store accounts, whether rent was collected or not (depending on whether we skipping rewrites is enabled)
4270    /// update bank's rewrites set for all rewrites that were skipped
4271    fn collect_rent_in_range(
4272        &self,
4273        partition: Partition,
4274        subrange_full: RangeInclusive<Pubkey>,
4275        metrics: &RentMetrics,
4276    ) {
4277        let mut hold_range = Measure::start("hold_range");
4278        let thread_pool = &self.rc.accounts.accounts_db.thread_pool;
4279        thread_pool.install(|| {
4280            self.rc
4281                .accounts
4282                .hold_range_in_memory(&subrange_full, true, thread_pool);
4283            hold_range.stop();
4284            metrics.hold_range_us.fetch_add(hold_range.as_us(), Relaxed);
4285
4286            let rent_paying_pubkeys_ = self.get_rent_paying_pubkeys(&partition);
4287            let rent_paying_pubkeys = rent_paying_pubkeys_.as_ref();
4288
4289            // divide the range into num_threads smaller ranges and process in parallel
4290            // Note that 'pubkey_range_from_partition' cannot easily be re-used here to break the range smaller.
4291            // It has special handling of 0..0 and partition_count changes affect all ranges unevenly.
4292            let num_threads = solana_accounts_db::accounts_db::quarter_thread_count() as u64;
4293            let sz = std::mem::size_of::<u64>();
4294            let start_prefix = accounts_partition::prefix_from_pubkey(subrange_full.start());
4295            let end_prefix_inclusive = accounts_partition::prefix_from_pubkey(subrange_full.end());
4296            let range = end_prefix_inclusive - start_prefix;
4297            let increment = range / num_threads;
4298            let mut results = (0..num_threads)
4299                .into_par_iter()
4300                .map(|chunk| {
4301                    let offset = |chunk| start_prefix + chunk * increment;
4302                    let start = offset(chunk);
4303                    let last = chunk == num_threads - 1;
4304                    let merge_prefix = |prefix: u64, mut bound: Pubkey| {
4305                        bound.as_mut()[0..sz].copy_from_slice(&prefix.to_be_bytes());
4306                        bound
4307                    };
4308                    let start = merge_prefix(start, *subrange_full.start());
4309                    let (accounts, measure_load_accounts) = measure_time!(if last {
4310                        let end = *subrange_full.end();
4311                        let subrange = start..=end; // IN-clusive
4312                        self.rc
4313                            .accounts
4314                            .load_to_collect_rent_eagerly(&self.ancestors, subrange)
4315                    } else {
4316                        let end = merge_prefix(offset(chunk + 1), *subrange_full.start());
4317                        let subrange = start..end; // EX-clusive, the next 'start' will be this same value
4318                        self.rc
4319                            .accounts
4320                            .load_to_collect_rent_eagerly(&self.ancestors, subrange)
4321                    });
4322                    CollectRentInPartitionInfo::new(
4323                        self.collect_rent_from_accounts(accounts, rent_paying_pubkeys, partition.1),
4324                        Duration::from_nanos(measure_load_accounts.as_ns()),
4325                    )
4326                })
4327                .reduce(
4328                    CollectRentInPartitionInfo::default,
4329                    CollectRentInPartitionInfo::reduce,
4330                );
4331
4332            self.skipped_rewrites
4333                .lock()
4334                .unwrap()
4335                .extend(results.skipped_rewrites);
4336
4337            // We cannot assert here that we collected from all expected keys.
4338            // Some accounts may have been topped off or may have had all funds removed and gone to 0 lamports.
4339
4340            self.rc
4341                .accounts
4342                .hold_range_in_memory(&subrange_full, false, thread_pool);
4343
4344            self.collected_rent
4345                .fetch_add(results.rent_collected, Relaxed);
4346            self.update_accounts_data_size_delta_off_chain(
4347                -(results.accounts_data_size_reclaimed as i64),
4348            );
4349            self.rewards
4350                .write()
4351                .unwrap()
4352                .append(&mut results.rent_rewards);
4353
4354            metrics
4355                .load_us
4356                .fetch_add(results.time_loading_accounts_us, Relaxed);
4357            metrics
4358                .collect_us
4359                .fetch_add(results.time_collecting_rent_us, Relaxed);
4360            metrics
4361                .store_us
4362                .fetch_add(results.time_storing_accounts_us, Relaxed);
4363            metrics.count.fetch_add(results.num_accounts, Relaxed);
4364        });
4365    }
4366
4367    pub(crate) fn fixed_cycle_partitions_between_slots(
4368        &self,
4369        starting_slot: Slot,
4370        ending_slot: Slot,
4371    ) -> Vec<Partition> {
4372        let slot_count_in_two_day = self.slot_count_in_two_day();
4373        accounts_partition::get_partitions(ending_slot, starting_slot, slot_count_in_two_day)
4374    }
4375
4376    fn fixed_cycle_partitions(&self) -> Vec<Partition> {
4377        self.fixed_cycle_partitions_between_slots(self.parent_slot(), self.slot())
4378    }
4379
4380    pub(crate) fn variable_cycle_partitions_between_slots(
4381        &self,
4382        starting_slot: Slot,
4383        ending_slot: Slot,
4384    ) -> Vec<Partition> {
4385        let (starting_epoch, mut starting_slot_index) =
4386            self.get_epoch_and_slot_index(starting_slot);
4387        let (ending_epoch, ending_slot_index) = self.get_epoch_and_slot_index(ending_slot);
4388
4389        let mut partitions = vec![];
4390        if starting_epoch < ending_epoch {
4391            let slot_skipped = (ending_slot - starting_slot) > 1;
4392            if slot_skipped {
4393                // Generate special partitions because there are skipped slots
4394                // exactly at the epoch transition.
4395
4396                let parent_last_slot_index = self.get_slots_in_epoch(starting_epoch) - 1;
4397
4398                // ... for parent epoch
4399                partitions.push(self.partition_from_slot_indexes_with_gapped_epochs(
4400                    starting_slot_index,
4401                    parent_last_slot_index,
4402                    starting_epoch,
4403                ));
4404
4405                if ending_slot_index > 0 {
4406                    // ... for current epoch
4407                    partitions.push(self.partition_from_slot_indexes_with_gapped_epochs(
4408                        0,
4409                        0,
4410                        ending_epoch,
4411                    ));
4412                }
4413            }
4414            starting_slot_index = 0;
4415        }
4416
4417        partitions.push(self.partition_from_normal_slot_indexes(
4418            starting_slot_index,
4419            ending_slot_index,
4420            ending_epoch,
4421        ));
4422
4423        partitions
4424    }
4425
4426    fn variable_cycle_partitions(&self) -> Vec<Partition> {
4427        self.variable_cycle_partitions_between_slots(self.parent_slot(), self.slot())
4428    }
4429
4430    fn do_partition_from_slot_indexes(
4431        &self,
4432        start_slot_index: SlotIndex,
4433        end_slot_index: SlotIndex,
4434        epoch: Epoch,
4435        generated_for_gapped_epochs: bool,
4436    ) -> Partition {
4437        let slot_count_per_epoch = self.get_slots_in_epoch(epoch);
4438
4439        let cycle_params = if !self.use_multi_epoch_collection_cycle(epoch) {
4440            // mnb should always go through this code path
4441            accounts_partition::rent_single_epoch_collection_cycle_params(
4442                epoch,
4443                slot_count_per_epoch,
4444            )
4445        } else {
4446            accounts_partition::rent_multi_epoch_collection_cycle_params(
4447                epoch,
4448                slot_count_per_epoch,
4449                self.first_normal_epoch(),
4450                self.slot_count_in_two_day() / slot_count_per_epoch,
4451            )
4452        };
4453        accounts_partition::get_partition_from_slot_indexes(
4454            cycle_params,
4455            start_slot_index,
4456            end_slot_index,
4457            generated_for_gapped_epochs,
4458        )
4459    }
4460
4461    fn partition_from_normal_slot_indexes(
4462        &self,
4463        start_slot_index: SlotIndex,
4464        end_slot_index: SlotIndex,
4465        epoch: Epoch,
4466    ) -> Partition {
4467        self.do_partition_from_slot_indexes(start_slot_index, end_slot_index, epoch, false)
4468    }
4469
4470    fn partition_from_slot_indexes_with_gapped_epochs(
4471        &self,
4472        start_slot_index: SlotIndex,
4473        end_slot_index: SlotIndex,
4474        epoch: Epoch,
4475    ) -> Partition {
4476        self.do_partition_from_slot_indexes(start_slot_index, end_slot_index, epoch, true)
4477    }
4478
4479    // Given short epochs, it's too costly to collect rent eagerly
4480    // within an epoch, so lower the frequency of it.
4481    // These logic isn't strictly eager anymore and should only be used
4482    // for development/performance purpose.
4483    // Absolutely not under ClusterType::MainnetBeta!!!!
4484    fn use_multi_epoch_collection_cycle(&self, epoch: Epoch) -> bool {
4485        // Force normal behavior, disabling multi epoch collection cycle for manual local testing
4486        #[cfg(not(test))]
4487        if self.slot_count_per_normal_epoch() == solana_sdk::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH
4488        {
4489            return false;
4490        }
4491
4492        epoch >= self.first_normal_epoch()
4493            && self.slot_count_per_normal_epoch() < self.slot_count_in_two_day()
4494    }
4495
4496    pub(crate) fn use_fixed_collection_cycle(&self) -> bool {
4497        // Force normal behavior, disabling fixed collection cycle for manual local testing
4498        #[cfg(not(test))]
4499        if self.slot_count_per_normal_epoch() == solana_sdk::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH
4500        {
4501            return false;
4502        }
4503
4504        self.cluster_type() != ClusterType::MainnetBeta
4505            && self.slot_count_per_normal_epoch() < self.slot_count_in_two_day()
4506    }
4507
4508    fn slot_count_in_two_day(&self) -> SlotCount {
4509        Self::slot_count_in_two_day_helper(self.ticks_per_slot)
4510    }
4511
4512    // This value is specially chosen to align with slots per epoch in mainnet-beta and testnet
4513    // Also, assume 500GB account data set as the extreme, then for 2 day (=48 hours) to collect
4514    // rent eagerly, we'll consume 5.7 MB/s IO bandwidth, bidirectionally.
4515    pub fn slot_count_in_two_day_helper(ticks_per_slot: SlotCount) -> SlotCount {
4516        2 * DEFAULT_TICKS_PER_SECOND * SECONDS_PER_DAY / ticks_per_slot
4517    }
4518
4519    fn slot_count_per_normal_epoch(&self) -> SlotCount {
4520        self.get_slots_in_epoch(self.first_normal_epoch())
4521    }
4522
4523    pub fn cluster_type(&self) -> ClusterType {
4524        // unwrap is safe; self.cluster_type is ensured to be Some() always...
4525        // we only using Option here for ABI compatibility...
4526        self.cluster_type.unwrap()
4527    }
4528
4529    /// Process a batch of transactions.
4530    #[must_use]
4531    pub fn load_execute_and_commit_transactions(
4532        &self,
4533        batch: &TransactionBatch<impl TransactionWithMeta>,
4534        max_age: usize,
4535        collect_balances: bool,
4536        recording_config: ExecutionRecordingConfig,
4537        timings: &mut ExecuteTimings,
4538        log_messages_bytes_limit: Option<usize>,
4539    ) -> (Vec<TransactionCommitResult>, TransactionBalancesSet) {
4540        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
4541            batch,
4542            max_age,
4543            collect_balances,
4544            recording_config,
4545            timings,
4546            log_messages_bytes_limit,
4547            None::<fn(&mut _, &_) -> _>,
4548        )
4549        .unwrap()
4550    }
4551
4552    pub fn load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
4553        &'a self,
4554        batch: &TransactionBatch<impl TransactionWithMeta>,
4555        max_age: usize,
4556        collect_balances: bool,
4557        recording_config: ExecutionRecordingConfig,
4558        timings: &mut ExecuteTimings,
4559        log_messages_bytes_limit: Option<usize>,
4560        pre_commit_callback: impl FnOnce(
4561            &mut ExecuteTimings,
4562            &[TransactionProcessingResult],
4563        ) -> PreCommitResult<'a>,
4564    ) -> Result<(Vec<TransactionCommitResult>, TransactionBalancesSet)> {
4565        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
4566            batch,
4567            max_age,
4568            collect_balances,
4569            recording_config,
4570            timings,
4571            log_messages_bytes_limit,
4572            Some(pre_commit_callback),
4573        )
4574    }
4575
4576    fn do_load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
4577        &'a self,
4578        batch: &TransactionBatch<impl TransactionWithMeta>,
4579        max_age: usize,
4580        collect_balances: bool,
4581        recording_config: ExecutionRecordingConfig,
4582        timings: &mut ExecuteTimings,
4583        log_messages_bytes_limit: Option<usize>,
4584        pre_commit_callback: Option<
4585            impl FnOnce(&mut ExecuteTimings, &[TransactionProcessingResult]) -> PreCommitResult<'a>,
4586        >,
4587    ) -> Result<(Vec<TransactionCommitResult>, TransactionBalancesSet)> {
4588        let pre_balances = if collect_balances {
4589            self.collect_balances(batch)
4590        } else {
4591            vec![]
4592        };
4593
4594        let LoadAndExecuteTransactionsOutput {
4595            processing_results,
4596            processed_counts,
4597        } = self.load_and_execute_transactions(
4598            batch,
4599            max_age,
4600            timings,
4601            &mut TransactionErrorMetrics::default(),
4602            TransactionProcessingConfig {
4603                account_overrides: None,
4604                check_program_modification_slot: self.check_program_modification_slot,
4605                compute_budget: self.compute_budget(),
4606                log_messages_bytes_limit,
4607                limit_to_load_programs: false,
4608                recording_config,
4609                transaction_account_lock_limit: Some(self.get_transaction_account_lock_limit()),
4610            },
4611        );
4612
4613        // pre_commit_callback could initiate an atomic operation (i.e. poh recording with block
4614        // producing unified scheduler). in that case, it returns Some(freeze_lock), which should
4615        // unlocked only after calling commit_transactions() immediately after calling the
4616        // callback.
4617        let freeze_lock = if let Some(pre_commit_callback) = pre_commit_callback {
4618            pre_commit_callback(timings, &processing_results)?
4619        } else {
4620            None
4621        };
4622        let commit_results = self.commit_transactions(
4623            batch.sanitized_transactions(),
4624            processing_results,
4625            &processed_counts,
4626            timings,
4627        );
4628        drop(freeze_lock);
4629        let post_balances = if collect_balances {
4630            self.collect_balances(batch)
4631        } else {
4632            vec![]
4633        };
4634        Ok((
4635            commit_results,
4636            TransactionBalancesSet::new(pre_balances, post_balances),
4637        ))
4638    }
4639
4640    /// Process a Transaction. This is used for unit tests and simply calls the vector
4641    /// Bank::process_transactions method.
4642    pub fn process_transaction(&self, tx: &Transaction) -> Result<()> {
4643        self.try_process_transactions(std::iter::once(tx))?[0].clone()?;
4644        tx.signatures
4645            .first()
4646            .map_or(Ok(()), |sig| self.get_signature_status(sig).unwrap())
4647    }
4648
4649    /// Process a Transaction and store metadata. This is used for tests and the banks services. It
4650    /// replicates the vector Bank::process_transaction method with metadata recording enabled.
4651    pub fn process_transaction_with_metadata(
4652        &self,
4653        tx: impl Into<VersionedTransaction>,
4654    ) -> Result<CommittedTransaction> {
4655        let txs = vec![tx.into()];
4656        let batch = self.prepare_entry_batch(txs)?;
4657
4658        let (mut commit_results, ..) = self.load_execute_and_commit_transactions(
4659            &batch,
4660            MAX_PROCESSING_AGE,
4661            false, // collect_balances
4662            ExecutionRecordingConfig {
4663                enable_cpi_recording: false,
4664                enable_log_recording: true,
4665                enable_return_data_recording: true,
4666            },
4667            &mut ExecuteTimings::default(),
4668            Some(1000 * 1000),
4669        );
4670
4671        commit_results.remove(0)
4672    }
4673
4674    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
4675    /// Short circuits if any of the transactions do not pass sanitization checks.
4676    pub fn try_process_transactions<'a>(
4677        &self,
4678        txs: impl Iterator<Item = &'a Transaction>,
4679    ) -> Result<Vec<Result<()>>> {
4680        let txs = txs
4681            .map(|tx| VersionedTransaction::from(tx.clone()))
4682            .collect();
4683        self.try_process_entry_transactions(txs)
4684    }
4685
4686    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
4687    /// Short circuits if any of the transactions do not pass sanitization checks.
4688    pub fn try_process_entry_transactions(
4689        &self,
4690        txs: Vec<VersionedTransaction>,
4691    ) -> Result<Vec<Result<()>>> {
4692        let batch = self.prepare_entry_batch(txs)?;
4693        Ok(self.process_transaction_batch(&batch))
4694    }
4695
4696    #[must_use]
4697    fn process_transaction_batch(
4698        &self,
4699        batch: &TransactionBatch<impl TransactionWithMeta>,
4700    ) -> Vec<Result<()>> {
4701        self.load_execute_and_commit_transactions(
4702            batch,
4703            MAX_PROCESSING_AGE,
4704            false,
4705            ExecutionRecordingConfig::new_single_setting(false),
4706            &mut ExecuteTimings::default(),
4707            None,
4708        )
4709        .0
4710        .into_iter()
4711        .map(|commit_result| commit_result.map(|_| ()))
4712        .collect()
4713    }
4714
4715    /// Create, sign, and process a Transaction from `keypair` to `to` of
4716    /// `n` lamports where `blockhash` is the last Entry ID observed by the client.
4717    pub fn transfer(&self, n: u64, keypair: &Keypair, to: &Pubkey) -> Result<Signature> {
4718        let blockhash = self.last_blockhash();
4719        let tx = system_transaction::transfer(keypair, to, n, blockhash);
4720        let signature = tx.signatures[0];
4721        self.process_transaction(&tx).map(|_| signature)
4722    }
4723
4724    pub fn read_balance(account: &AccountSharedData) -> u64 {
4725        account.lamports()
4726    }
4727    /// Each program would need to be able to introspect its own state
4728    /// this is hard-coded to the Budget language
4729    pub fn get_balance(&self, pubkey: &Pubkey) -> u64 {
4730        self.get_account(pubkey)
4731            .map(|x| Self::read_balance(&x))
4732            .unwrap_or(0)
4733    }
4734
4735    /// Compute all the parents of the bank in order
4736    pub fn parents(&self) -> Vec<Arc<Bank>> {
4737        let mut parents = vec![];
4738        let mut bank = self.parent();
4739        while let Some(parent) = bank {
4740            parents.push(parent.clone());
4741            bank = parent.parent();
4742        }
4743        parents
4744    }
4745
4746    /// Compute all the parents of the bank including this bank itself
4747    pub fn parents_inclusive(self: Arc<Self>) -> Vec<Arc<Bank>> {
4748        let mut parents = self.parents();
4749        parents.insert(0, self);
4750        parents
4751    }
4752
4753    /// fn store the single `account` with `pubkey`.
4754    /// Uses `store_accounts`, which works on a vector of accounts.
4755    pub fn store_account(&self, pubkey: &Pubkey, account: &AccountSharedData) {
4756        self.store_accounts((self.slot(), &[(pubkey, account)][..]))
4757    }
4758
4759    pub fn store_accounts<'a>(&self, accounts: impl StorableAccounts<'a>) {
4760        assert!(!self.freeze_started());
4761        let mut m = Measure::start("stakes_cache.check_and_store");
4762        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
4763
4764        (0..accounts.len()).for_each(|i| {
4765            accounts.account(i, |account| {
4766                self.stakes_cache.check_and_store(
4767                    account.pubkey(),
4768                    &account,
4769                    new_warmup_cooldown_rate_epoch,
4770                )
4771            })
4772        });
4773        self.update_bank_hash_stats(&accounts);
4774        self.rc.accounts.store_accounts_cached(accounts);
4775        m.stop();
4776        self.rc
4777            .accounts
4778            .accounts_db
4779            .stats
4780            .stakes_cache_check_and_store_us
4781            .fetch_add(m.as_us(), Relaxed);
4782    }
4783
4784    pub fn force_flush_accounts_cache(&self) {
4785        self.rc
4786            .accounts
4787            .accounts_db
4788            .flush_accounts_cache(true, Some(self.slot()))
4789    }
4790
4791    pub fn flush_accounts_cache_if_needed(&self) {
4792        self.rc
4793            .accounts
4794            .accounts_db
4795            .flush_accounts_cache(false, Some(self.slot()))
4796    }
4797
4798    /// Technically this issues (or even burns!) new lamports,
4799    /// so be extra careful for its usage
4800    fn store_account_and_update_capitalization(
4801        &self,
4802        pubkey: &Pubkey,
4803        new_account: &AccountSharedData,
4804    ) {
4805        let old_account_data_size =
4806            if let Some(old_account) = self.get_account_with_fixed_root_no_cache(pubkey) {
4807                match new_account.lamports().cmp(&old_account.lamports()) {
4808                    std::cmp::Ordering::Greater => {
4809                        let increased = new_account.lamports() - old_account.lamports();
4810                        trace!(
4811                            "store_account_and_update_capitalization: increased: {} {}",
4812                            pubkey,
4813                            increased
4814                        );
4815                        self.capitalization.fetch_add(increased, Relaxed);
4816                    }
4817                    std::cmp::Ordering::Less => {
4818                        let decreased = old_account.lamports() - new_account.lamports();
4819                        trace!(
4820                            "store_account_and_update_capitalization: decreased: {} {}",
4821                            pubkey,
4822                            decreased
4823                        );
4824                        self.capitalization.fetch_sub(decreased, Relaxed);
4825                    }
4826                    std::cmp::Ordering::Equal => {}
4827                }
4828                old_account.data().len()
4829            } else {
4830                trace!(
4831                    "store_account_and_update_capitalization: created: {} {}",
4832                    pubkey,
4833                    new_account.lamports()
4834                );
4835                self.capitalization
4836                    .fetch_add(new_account.lamports(), Relaxed);
4837                0
4838            };
4839
4840        self.store_account(pubkey, new_account);
4841        self.calculate_and_update_accounts_data_size_delta_off_chain(
4842            old_account_data_size,
4843            new_account.data().len(),
4844        );
4845    }
4846
4847    pub fn accounts(&self) -> Arc<Accounts> {
4848        self.rc.accounts.clone()
4849    }
4850
4851    fn finish_init(
4852        &mut self,
4853        genesis_config: &GenesisConfig,
4854        additional_builtins: Option<&[BuiltinPrototype]>,
4855        debug_do_not_add_builtins: bool,
4856    ) {
4857        self.rewards_pool_pubkeys =
4858            Arc::new(genesis_config.rewards_pools.keys().cloned().collect());
4859
4860        self.apply_feature_activations(
4861            ApplyFeatureActivationsCaller::FinishInit,
4862            debug_do_not_add_builtins,
4863        );
4864
4865        // Cost-Tracker is not serialized in snapshot or any configs.
4866        // We must apply previously activated features related to limits here
4867        // so that the initial bank state is consistent with the feature set.
4868        // Cost-tracker limits are propagated through children banks.
4869        if self
4870            .feature_set
4871            .is_active(&feature_set::raise_block_limits_to_50m::id())
4872        {
4873            let (account_cost_limit, block_cost_limit, vote_cost_limit) = simd_0207_block_limits();
4874            self.write_cost_tracker().unwrap().set_limits(
4875                account_cost_limit,
4876                block_cost_limit,
4877                vote_cost_limit,
4878            );
4879        }
4880
4881        if self
4882            .feature_set
4883            .is_active(&feature_set::raise_block_limits_to_60m::id())
4884        {
4885            let (account_cost_limit, block_cost_limit, vote_cost_limit) = simd_0256_block_limits();
4886            self.write_cost_tracker().unwrap().set_limits(
4887                account_cost_limit,
4888                block_cost_limit,
4889                vote_cost_limit,
4890            );
4891        }
4892
4893        // If the accounts delta hash is still in use, start the background account hasher
4894        if !self
4895            .feature_set
4896            .is_active(&feature_set::remove_accounts_delta_hash::id())
4897        {
4898            self.rc.accounts.accounts_db.start_background_hasher();
4899        }
4900
4901        if !debug_do_not_add_builtins {
4902            for builtin in BUILTINS
4903                .iter()
4904                .chain(additional_builtins.unwrap_or(&[]).iter())
4905            {
4906                // The builtin should be added if it has no enable feature ID
4907                // and it has not been migrated to Core BPF.
4908                //
4909                // If a program was previously migrated to Core BPF, accountsDB
4910                // from snapshot should contain the BPF program accounts.
4911                let builtin_is_bpf = |program_id: &Pubkey| {
4912                    self.get_account(program_id)
4913                        .map(|a| a.owner() == &bpf_loader_upgradeable::id())
4914                        .unwrap_or(false)
4915                };
4916                if builtin.enable_feature_id.is_none() && !builtin_is_bpf(&builtin.program_id) {
4917                    self.transaction_processor.add_builtin(
4918                        self,
4919                        builtin.program_id,
4920                        builtin.name,
4921                        ProgramCacheEntry::new_builtin(0, builtin.name.len(), builtin.entrypoint),
4922                    );
4923                }
4924            }
4925            for precompile in get_precompiles() {
4926                if precompile.feature.is_none() {
4927                    self.add_precompile(&precompile.program_id);
4928                }
4929            }
4930        }
4931
4932        self.transaction_processor
4933            .configure_program_runtime_environments(
4934                Some(Arc::new(
4935                    create_program_runtime_environment_v1(
4936                        &self.feature_set,
4937                        &self.compute_budget().unwrap_or_default(),
4938                        false, /* deployment */
4939                        false, /* debugging_features */
4940                    )
4941                    .unwrap(),
4942                )),
4943                Some(Arc::new(create_program_runtime_environment_v2(
4944                    &self.compute_budget().unwrap_or_default(),
4945                    false, /* debugging_features */
4946                ))),
4947            );
4948    }
4949
4950    pub fn set_inflation(&self, inflation: Inflation) {
4951        *self.inflation.write().unwrap() = inflation;
4952    }
4953
4954    /// Get a snapshot of the current set of hard forks
4955    pub fn hard_forks(&self) -> HardForks {
4956        self.hard_forks.read().unwrap().clone()
4957    }
4958
4959    pub fn register_hard_fork(&self, new_hard_fork_slot: Slot) {
4960        let bank_slot = self.slot();
4961
4962        let lock = self.freeze_lock();
4963        let bank_frozen = *lock != Hash::default();
4964        if new_hard_fork_slot < bank_slot {
4965            warn!(
4966                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is older \
4967                than the bank at slot {bank_slot} that attempted to register it."
4968            );
4969        } else if (new_hard_fork_slot == bank_slot) && bank_frozen {
4970            warn!(
4971                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is the same \
4972                slot as the bank at slot {bank_slot} that attempted to register it, but that \
4973                bank is already frozen."
4974            );
4975        } else {
4976            self.hard_forks
4977                .write()
4978                .unwrap()
4979                .register(new_hard_fork_slot);
4980        }
4981    }
4982
4983    pub fn get_account_with_fixed_root_no_cache(
4984        &self,
4985        pubkey: &Pubkey,
4986    ) -> Option<AccountSharedData> {
4987        self.load_account_with(pubkey, |_| false)
4988            .map(|(acc, _slot)| acc)
4989    }
4990
4991    fn load_account_with(
4992        &self,
4993        pubkey: &Pubkey,
4994        callback: impl for<'local> Fn(&'local AccountSharedData) -> bool,
4995    ) -> Option<(AccountSharedData, Slot)> {
4996        self.rc
4997            .accounts
4998            .accounts_db
4999            .load_account_with(&self.ancestors, pubkey, callback)
5000    }
5001
5002    // Hi! leaky abstraction here....
5003    // try to use get_account_with_fixed_root() if it's called ONLY from on-chain runtime account
5004    // processing. That alternative fn provides more safety.
5005    pub fn get_account(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
5006        self.get_account_modified_slot(pubkey)
5007            .map(|(acc, _slot)| acc)
5008    }
5009
5010    // Hi! leaky abstraction here....
5011    // use this over get_account() if it's called ONLY from on-chain runtime account
5012    // processing (i.e. from in-band replay/banking stage; that ensures root is *fixed* while
5013    // running).
5014    // pro: safer assertion can be enabled inside AccountsDb
5015    // con: panics!() if called from off-chain processing
5016    pub fn get_account_with_fixed_root(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
5017        self.get_account_modified_slot_with_fixed_root(pubkey)
5018            .map(|(acc, _slot)| acc)
5019    }
5020
5021    // See note above get_account_with_fixed_root() about when to prefer this function
5022    pub fn get_account_modified_slot_with_fixed_root(
5023        &self,
5024        pubkey: &Pubkey,
5025    ) -> Option<(AccountSharedData, Slot)> {
5026        self.load_slow_with_fixed_root(&self.ancestors, pubkey)
5027    }
5028
5029    pub fn get_account_modified_slot(&self, pubkey: &Pubkey) -> Option<(AccountSharedData, Slot)> {
5030        self.load_slow(&self.ancestors, pubkey)
5031    }
5032
5033    fn load_slow(
5034        &self,
5035        ancestors: &Ancestors,
5036        pubkey: &Pubkey,
5037    ) -> Option<(AccountSharedData, Slot)> {
5038        // get_account (= primary this fn caller) may be called from on-chain Bank code even if we
5039        // try hard to use get_account_with_fixed_root for that purpose...
5040        // so pass safer LoadHint:Unspecified here as a fallback
5041        self.rc.accounts.load_without_fixed_root(ancestors, pubkey)
5042    }
5043
5044    fn load_slow_with_fixed_root(
5045        &self,
5046        ancestors: &Ancestors,
5047        pubkey: &Pubkey,
5048    ) -> Option<(AccountSharedData, Slot)> {
5049        self.rc.accounts.load_with_fixed_root(ancestors, pubkey)
5050    }
5051
5052    pub fn get_program_accounts(
5053        &self,
5054        program_id: &Pubkey,
5055        config: &ScanConfig,
5056    ) -> ScanResult<Vec<TransactionAccount>> {
5057        self.rc
5058            .accounts
5059            .load_by_program(&self.ancestors, self.bank_id, program_id, config)
5060    }
5061
5062    pub fn get_filtered_program_accounts<F: Fn(&AccountSharedData) -> bool>(
5063        &self,
5064        program_id: &Pubkey,
5065        filter: F,
5066        config: &ScanConfig,
5067    ) -> ScanResult<Vec<TransactionAccount>> {
5068        self.rc.accounts.load_by_program_with_filter(
5069            &self.ancestors,
5070            self.bank_id,
5071            program_id,
5072            filter,
5073            config,
5074        )
5075    }
5076
5077    pub fn get_filtered_indexed_accounts<F: Fn(&AccountSharedData) -> bool>(
5078        &self,
5079        index_key: &IndexKey,
5080        filter: F,
5081        config: &ScanConfig,
5082        byte_limit_for_scan: Option<usize>,
5083    ) -> ScanResult<Vec<TransactionAccount>> {
5084        self.rc.accounts.load_by_index_key_with_filter(
5085            &self.ancestors,
5086            self.bank_id,
5087            index_key,
5088            filter,
5089            config,
5090            byte_limit_for_scan,
5091        )
5092    }
5093
5094    pub fn account_indexes_include_key(&self, key: &Pubkey) -> bool {
5095        self.rc.accounts.account_indexes_include_key(key)
5096    }
5097
5098    /// Returns all the accounts this bank can load
5099    pub fn get_all_accounts(&self, sort_results: bool) -> ScanResult<Vec<PubkeyAccountSlot>> {
5100        self.rc
5101            .accounts
5102            .load_all(&self.ancestors, self.bank_id, sort_results)
5103    }
5104
5105    // Scans all the accounts this bank can load, applying `scan_func`
5106    pub fn scan_all_accounts<F>(&self, scan_func: F, sort_results: bool) -> ScanResult<()>
5107    where
5108        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
5109    {
5110        self.rc
5111            .accounts
5112            .scan_all(&self.ancestors, self.bank_id, scan_func, sort_results)
5113    }
5114
5115    pub fn get_program_accounts_modified_since_parent(
5116        &self,
5117        program_id: &Pubkey,
5118    ) -> Vec<TransactionAccount> {
5119        self.rc
5120            .accounts
5121            .load_by_program_slot(self.slot(), Some(program_id))
5122    }
5123
5124    pub fn get_transaction_logs(
5125        &self,
5126        address: Option<&Pubkey>,
5127    ) -> Option<Vec<TransactionLogInfo>> {
5128        self.transaction_log_collector
5129            .read()
5130            .unwrap()
5131            .get_logs_for_address(address)
5132    }
5133
5134    /// Returns all the accounts stored in this slot
5135    pub fn get_all_accounts_modified_since_parent(&self) -> Vec<TransactionAccount> {
5136        self.rc.accounts.load_by_program_slot(self.slot(), None)
5137    }
5138
5139    // if you want get_account_modified_since_parent without fixed_root, please define so...
5140    fn get_account_modified_since_parent_with_fixed_root(
5141        &self,
5142        pubkey: &Pubkey,
5143    ) -> Option<(AccountSharedData, Slot)> {
5144        let just_self: Ancestors = Ancestors::from(vec![self.slot()]);
5145        if let Some((account, slot)) = self.load_slow_with_fixed_root(&just_self, pubkey) {
5146            if slot == self.slot() {
5147                return Some((account, slot));
5148            }
5149        }
5150        None
5151    }
5152
5153    pub fn get_largest_accounts(
5154        &self,
5155        num: usize,
5156        filter_by_address: &HashSet<Pubkey>,
5157        filter: AccountAddressFilter,
5158        sort_results: bool,
5159    ) -> ScanResult<Vec<(Pubkey, u64)>> {
5160        self.rc.accounts.load_largest_accounts(
5161            &self.ancestors,
5162            self.bank_id,
5163            num,
5164            filter_by_address,
5165            filter,
5166            sort_results,
5167        )
5168    }
5169
5170    /// Return the accumulated executed transaction count
5171    pub fn transaction_count(&self) -> u64 {
5172        self.transaction_count.load(Relaxed)
5173    }
5174
5175    /// Returns the number of non-vote transactions processed without error
5176    /// since the most recent boot from snapshot or genesis.
5177    /// This value is not shared though the network, nor retained
5178    /// within snapshots, but is preserved in `Bank::new_from_parent`.
5179    pub fn non_vote_transaction_count_since_restart(&self) -> u64 {
5180        self.non_vote_transaction_count_since_restart.load(Relaxed)
5181    }
5182
5183    /// Return the transaction count executed only in this bank
5184    pub fn executed_transaction_count(&self) -> u64 {
5185        self.transaction_count()
5186            .saturating_sub(self.parent().map_or(0, |parent| parent.transaction_count()))
5187    }
5188
5189    pub fn transaction_error_count(&self) -> u64 {
5190        self.transaction_error_count.load(Relaxed)
5191    }
5192
5193    pub fn transaction_entries_count(&self) -> u64 {
5194        self.transaction_entries_count.load(Relaxed)
5195    }
5196
5197    pub fn transactions_per_entry_max(&self) -> u64 {
5198        self.transactions_per_entry_max.load(Relaxed)
5199    }
5200
5201    fn increment_transaction_count(&self, tx_count: u64) {
5202        self.transaction_count.fetch_add(tx_count, Relaxed);
5203    }
5204
5205    fn increment_non_vote_transaction_count_since_restart(&self, tx_count: u64) {
5206        self.non_vote_transaction_count_since_restart
5207            .fetch_add(tx_count, Relaxed);
5208    }
5209
5210    pub fn signature_count(&self) -> u64 {
5211        self.signature_count.load(Relaxed)
5212    }
5213
5214    fn increment_signature_count(&self, signature_count: u64) {
5215        self.signature_count.fetch_add(signature_count, Relaxed);
5216    }
5217
5218    pub fn get_signature_status_processed_since_parent(
5219        &self,
5220        signature: &Signature,
5221    ) -> Option<Result<()>> {
5222        if let Some((slot, status)) = self.get_signature_status_slot(signature) {
5223            if slot <= self.slot() {
5224                return Some(status);
5225            }
5226        }
5227        None
5228    }
5229
5230    pub fn get_signature_status_with_blockhash(
5231        &self,
5232        signature: &Signature,
5233        blockhash: &Hash,
5234    ) -> Option<Result<()>> {
5235        let rcache = self.status_cache.read().unwrap();
5236        rcache
5237            .get_status(signature, blockhash, &self.ancestors)
5238            .map(|v| v.1)
5239    }
5240
5241    pub fn get_signature_status_slot(&self, signature: &Signature) -> Option<(Slot, Result<()>)> {
5242        let rcache = self.status_cache.read().unwrap();
5243        rcache.get_status_any_blockhash(signature, &self.ancestors)
5244    }
5245
5246    pub fn get_signature_status(&self, signature: &Signature) -> Option<Result<()>> {
5247        self.get_signature_status_slot(signature).map(|v| v.1)
5248    }
5249
5250    pub fn has_signature(&self, signature: &Signature) -> bool {
5251        self.get_signature_status_slot(signature).is_some()
5252    }
5253
5254    /// Hash the `accounts` HashMap. This represents a validator's interpretation
5255    ///  of the delta of the ledger since the last vote and up to now
5256    fn hash_internal_state(&self) -> Hash {
5257        let measure_total = Measure::start("");
5258        let slot = self.slot();
5259
5260        let delta_hash_info = (!self
5261            .feature_set
5262            .is_active(&feature_set::remove_accounts_delta_hash::id()))
5263        .then(|| {
5264            measure_us!({
5265                self.rc
5266                    .accounts
5267                    .accounts_db
5268                    .calculate_accounts_delta_hash_internal(
5269                        slot,
5270                        None,
5271                        self.skipped_rewrites.lock().unwrap().clone(),
5272                    )
5273            })
5274        });
5275
5276        let mut hash = if let Some((accounts_delta_hash, _measure)) = delta_hash_info.as_ref() {
5277            hashv(&[
5278                self.parent_hash.as_ref(),
5279                accounts_delta_hash.0.as_ref(),
5280                &self.signature_count().to_le_bytes(),
5281                self.last_blockhash().as_ref(),
5282            ])
5283        } else {
5284            hashv(&[
5285                self.parent_hash.as_ref(),
5286                &self.signature_count().to_le_bytes(),
5287                self.last_blockhash().as_ref(),
5288            ])
5289        };
5290
5291        let accounts_hash_info = if self
5292            .feature_set
5293            .is_active(&feature_set::accounts_lt_hash::id())
5294        {
5295            let accounts_lt_hash = &*self.accounts_lt_hash.lock().unwrap();
5296            let lt_hash_bytes = bytemuck::must_cast_slice(&accounts_lt_hash.0 .0);
5297            hash = hashv(&[hash.as_ref(), lt_hash_bytes]);
5298            let checksum = accounts_lt_hash.0.checksum();
5299            Some(format!(", accounts_lt_hash checksum: {checksum}"))
5300        } else {
5301            let epoch_accounts_hash = self.wait_get_epoch_accounts_hash();
5302            epoch_accounts_hash.map(|epoch_accounts_hash| {
5303                hash = hashv(&[hash.as_ref(), epoch_accounts_hash.as_ref().as_ref()]);
5304                format!(", epoch_accounts_hash: {:?}", epoch_accounts_hash.as_ref())
5305            })
5306        };
5307
5308        let buf = self
5309            .hard_forks
5310            .read()
5311            .unwrap()
5312            .get_hash_data(slot, self.parent_slot());
5313        if let Some(buf) = buf {
5314            let hard_forked_hash = extend_and_hash(&hash, &buf);
5315            warn!("hard fork at slot {slot} by hashing {buf:?}: {hash} => {hard_forked_hash}");
5316            hash = hard_forked_hash;
5317        }
5318
5319        #[cfg(feature = "dev-context-only-utils")]
5320        let hash_override = self
5321            .hash_overrides
5322            .lock()
5323            .unwrap()
5324            .get_bank_hash_override(slot)
5325            .copied()
5326            .inspect(|&hash_override| {
5327                if hash_override != hash {
5328                    info!(
5329                        "bank: slot: {}: overrode bank hash: {} with {}",
5330                        self.slot(),
5331                        hash,
5332                        hash_override
5333                    );
5334                }
5335            });
5336        // Avoid to optimize out `hash` along with the whole computation by super smart rustc.
5337        // hash_override is used by ledger-tool's simulate-block-production, which prefers
5338        // the actual bank freezing processing for accurate simulation.
5339        #[cfg(feature = "dev-context-only-utils")]
5340        let hash = hash_override.unwrap_or(std::hint::black_box(hash));
5341
5342        let bank_hash_stats = self.bank_hash_stats.load();
5343
5344        let total_us = measure_total.end_as_us();
5345
5346        let (accounts_delta_hash_us, accounts_delta_hash_log) = delta_hash_info
5347            .map(|(hash, us)| (us, format!(" accounts_delta: {}", hash.0)))
5348            .unzip();
5349        datapoint_info!(
5350            "bank-hash_internal_state",
5351            ("slot", slot, i64),
5352            ("total_us", total_us, i64),
5353            ("accounts_delta_hash_us", accounts_delta_hash_us, Option<i64>),
5354        );
5355        info!(
5356            "bank frozen: {slot} hash: {hash}{} signature_count: {} last_blockhash: {} capitalization: {}{}, stats: {bank_hash_stats:?}",
5357            accounts_delta_hash_log.unwrap_or_default(),
5358            self.signature_count(),
5359            self.last_blockhash(),
5360            self.capitalization(),
5361            accounts_hash_info.unwrap_or_default(),
5362        );
5363        hash
5364    }
5365
5366    pub fn collector_fees(&self) -> u64 {
5367        self.collector_fees.load(Relaxed)
5368    }
5369
5370    /// The epoch accounts hash is hashed into the bank's hash once per epoch at a predefined slot.
5371    /// Should it be included in *this* bank?
5372    fn should_include_epoch_accounts_hash(&self) -> bool {
5373        if !epoch_accounts_hash_utils::is_enabled_this_epoch(self) {
5374            return false;
5375        }
5376
5377        let stop_slot = epoch_accounts_hash_utils::calculation_stop(self);
5378        self.parent_slot() < stop_slot && self.slot() >= stop_slot
5379    }
5380
5381    /// If the epoch accounts hash should be included in this Bank, then fetch it. If the EAH
5382    /// calculation has not completed yet, this fn will block until it does complete.
5383    fn wait_get_epoch_accounts_hash(&self) -> Option<EpochAccountsHash> {
5384        if !self.should_include_epoch_accounts_hash() {
5385            return None;
5386        }
5387
5388        let (epoch_accounts_hash, waiting_time_us) = measure_us!(self
5389            .rc
5390            .accounts
5391            .accounts_db
5392            .epoch_accounts_hash_manager
5393            .wait_get_epoch_accounts_hash());
5394
5395        datapoint_info!(
5396            "bank-wait_get_epoch_accounts_hash",
5397            ("slot", self.slot(), i64),
5398            ("waiting-time-us", waiting_time_us, i64),
5399        );
5400        Some(epoch_accounts_hash)
5401    }
5402
5403    /// Used by ledger tool to run a final hash calculation once all ledger replay has completed.
5404    /// This should not be called by validator code.
5405    pub fn run_final_hash_calc(&self, on_halt_store_hash_raw_data_for_debug: bool) {
5406        self.force_flush_accounts_cache();
5407        // note that this slot may not be a root
5408        _ = self.verify_accounts_hash(
5409            None,
5410            VerifyAccountsHashConfig {
5411                test_hash_calculation: false,
5412                ignore_mismatch: true,
5413                require_rooted_bank: false,
5414                run_in_background: false,
5415                store_hash_raw_data_for_debug: on_halt_store_hash_raw_data_for_debug,
5416            },
5417            None,
5418        );
5419    }
5420
5421    /// Recalculate the accounts hash from the account stores. Used to verify a snapshot.
5422    /// return true if all is good
5423    /// Only called from startup or test code.
5424    #[must_use]
5425    fn verify_accounts_hash(
5426        &self,
5427        base: Option<(Slot, /*capitalization*/ u64)>,
5428        mut config: VerifyAccountsHashConfig,
5429        duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
5430    ) -> bool {
5431        #[derive(Debug, Eq, PartialEq)]
5432        enum VerifyKind {
5433            Merkle,
5434            Lattice,
5435        }
5436
5437        let accounts = &self.rc.accounts;
5438        // Wait until initial hash calc is complete before starting a new hash calc.
5439        // This should only occur when we halt at a slot in ledger-tool.
5440        accounts
5441            .accounts_db
5442            .verify_accounts_hash_in_bg
5443            .join_background_thread();
5444
5445        let slot = self.slot();
5446
5447        let verify_kind = match (
5448            duplicates_lt_hash.is_some(),
5449            self.rc
5450                .accounts
5451                .accounts_db
5452                .is_experimental_accumulator_hash_enabled(),
5453        ) {
5454            (true, _) => VerifyKind::Lattice,
5455            (false, false) => VerifyKind::Merkle,
5456            (false, true) => {
5457                // Calculating the accounts lt hash from storages *requires* a duplicates_lt_hash.
5458                // If it is None here, then we must use the index instead, which also means we
5459                // cannot run in the background.
5460                config.run_in_background = false;
5461                VerifyKind::Lattice
5462            }
5463        };
5464
5465        if config.require_rooted_bank && !accounts.accounts_db.accounts_index.is_alive_root(slot) {
5466            if let Some(parent) = self.parent() {
5467                info!(
5468                    "slot {slot} is not a root, so verify accounts hash on parent bank at slot {}",
5469                    parent.slot(),
5470                );
5471                if verify_kind == VerifyKind::Lattice {
5472                    // The duplicates_lt_hash is only valid for the current slot, so we must fall
5473                    // back to verifying the accounts lt hash with the index (which also means we
5474                    // cannot run in the background).
5475                    config.run_in_background = false;
5476                }
5477                return parent.verify_accounts_hash(base, config, None);
5478            } else {
5479                // this will result in mismatch errors
5480                // accounts hash calc doesn't include unrooted slots
5481                panic!("cannot verify accounts hash because slot {slot} is not a root");
5482            }
5483        }
5484
5485        // The snapshot storages must be captured *before* starting the background verification.
5486        // Otherwise, it is possible that a delayed call to `get_snapshot_storages()` will *not*
5487        // get the correct storages required to calculate and verify the accounts hashes.
5488        let snapshot_storages = self.rc.accounts.accounts_db.get_storages(RangeFull);
5489        let capitalization = self.capitalization();
5490        let verify_config = VerifyAccountsHashAndLamportsConfig {
5491            ancestors: &self.ancestors,
5492            epoch_schedule: self.epoch_schedule(),
5493            rent_collector: self.rent_collector(),
5494            test_hash_calculation: config.test_hash_calculation,
5495            ignore_mismatch: config.ignore_mismatch,
5496            store_detailed_debug_info: config.store_hash_raw_data_for_debug,
5497            use_bg_thread_pool: config.run_in_background,
5498        };
5499
5500        info!(
5501            "Verifying accounts, in background? {}, verify kind: {verify_kind:?}",
5502            config.run_in_background,
5503        );
5504        if config.run_in_background {
5505            let accounts = Arc::clone(accounts);
5506            let accounts_ = Arc::clone(&accounts);
5507            let ancestors = self.ancestors.clone();
5508            let epoch_schedule = self.epoch_schedule().clone();
5509            let rent_collector = self.rent_collector().clone();
5510            let expected_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap().clone();
5511            accounts.accounts_db.verify_accounts_hash_in_bg.start(|| {
5512                Builder::new()
5513                    .name("solBgHashVerify".into())
5514                    .spawn(move || {
5515                        info!("Initial background accounts hash verification has started");
5516                        let start = Instant::now();
5517                        let mut lattice_verify_time = None;
5518                        let mut merkle_verify_time = None;
5519                        let is_ok = match verify_kind {
5520                            VerifyKind::Lattice => {
5521                                // accounts lt hash is *enabled* so use lattice-based verification
5522                                let accounts_db = &accounts_.accounts_db;
5523                                let (calculated_accounts_lt_hash, duration) =
5524                                    meas_dur!(accounts_db.thread_pool_hash.install(|| {
5525                                        accounts_db
5526                                            .calculate_accounts_lt_hash_at_startup_from_storages(
5527                                                snapshot_storages.0.as_slice(),
5528                                                &duplicates_lt_hash.unwrap(),
5529                                            )
5530                                    }));
5531                                let is_ok =
5532                                    calculated_accounts_lt_hash == expected_accounts_lt_hash;
5533                                if !is_ok {
5534                                    let expected = expected_accounts_lt_hash.0.checksum();
5535                                    let calculated = calculated_accounts_lt_hash.0.checksum();
5536                                    error!(
5537                                        "Verifying accounts failed: accounts lattice hashes do not \
5538                                         match, expected: {expected}, calculated: {calculated}",
5539                                    );
5540                                }
5541                                lattice_verify_time = Some(duration);
5542                                is_ok
5543                            }
5544                            VerifyKind::Merkle => {
5545                                // accounts lt hash is *disabled* so use merkle-based verification
5546                                let snapshot_storages_and_slots = (
5547                                    snapshot_storages.0.as_slice(),
5548                                    snapshot_storages.1.as_slice(),
5549                                );
5550                                let (is_ok, duration) = meas_dur!(accounts_
5551                                    .verify_accounts_hash_and_lamports(
5552                                        snapshot_storages_and_slots,
5553                                        slot,
5554                                        capitalization,
5555                                        base,
5556                                        VerifyAccountsHashAndLamportsConfig {
5557                                            ancestors: &ancestors,
5558                                            epoch_schedule: &epoch_schedule,
5559                                            rent_collector: &rent_collector,
5560                                            ..verify_config
5561                                        },
5562                                    ));
5563                                merkle_verify_time = Some(duration);
5564                                is_ok
5565                            }
5566                        };
5567                        accounts_
5568                            .accounts_db
5569                            .verify_accounts_hash_in_bg
5570                            .background_finished();
5571                        let total_time = start.elapsed();
5572                        datapoint_info!(
5573                            "startup_verify_accounts",
5574                            ("total_us", total_time.as_micros(), i64),
5575                            (
5576                                "verify_accounts_lt_hash_us",
5577                                lattice_verify_time.as_ref().map(Duration::as_micros),
5578                                Option<i64>
5579                            ),
5580                            ("verify_accounts_hash_us",
5581                                merkle_verify_time.as_ref().map(Duration::as_micros),
5582                                Option<i64>
5583                            ),
5584                        );
5585                        info!("Initial background accounts hash verification has stopped");
5586                        is_ok
5587                    })
5588                    .unwrap()
5589            });
5590            true // initial result is true. We haven't failed yet. If verification fails, we'll panic from bg thread.
5591        } else {
5592            match verify_kind {
5593                VerifyKind::Lattice => {
5594                    let expected_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap().clone();
5595                    let calculated_accounts_lt_hash = if let Some(duplicates_lt_hash) =
5596                        duplicates_lt_hash
5597                    {
5598                        accounts
5599                            .accounts_db
5600                            .calculate_accounts_lt_hash_at_startup_from_storages(
5601                                snapshot_storages.0.as_slice(),
5602                                &duplicates_lt_hash,
5603                            )
5604                    } else {
5605                        accounts
5606                            .accounts_db
5607                            .calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, slot)
5608                    };
5609                    let is_ok = calculated_accounts_lt_hash == expected_accounts_lt_hash;
5610                    if !is_ok {
5611                        let expected = expected_accounts_lt_hash.0.checksum();
5612                        let calculated = calculated_accounts_lt_hash.0.checksum();
5613                        error!(
5614                            "Verifying accounts failed: accounts lattice hashes do not \
5615                             match, expected: {expected}, calculated: {calculated}",
5616                        );
5617                    }
5618                    is_ok
5619                }
5620                VerifyKind::Merkle => {
5621                    let snapshot_storages_and_slots = (
5622                        snapshot_storages.0.as_slice(),
5623                        snapshot_storages.1.as_slice(),
5624                    );
5625                    let is_ok = accounts.verify_accounts_hash_and_lamports(
5626                        snapshot_storages_and_slots,
5627                        slot,
5628                        capitalization,
5629                        base,
5630                        verify_config,
5631                    );
5632                    self.set_initial_accounts_hash_verification_completed();
5633                    is_ok
5634                }
5635            }
5636        }
5637    }
5638
5639    /// Specify that initial verification has completed.
5640    /// Called internally when verification runs in the foreground thread.
5641    /// Also has to be called by some tests which don't do verification on startup.
5642    pub fn set_initial_accounts_hash_verification_completed(&self) {
5643        self.rc
5644            .accounts
5645            .accounts_db
5646            .verify_accounts_hash_in_bg
5647            .verification_complete();
5648    }
5649
5650    /// return true if bg hash verification is complete
5651    /// return false if bg hash verification has not completed yet
5652    /// if hash verification failed, a panic will occur
5653    pub fn has_initial_accounts_hash_verification_completed(&self) -> bool {
5654        self.rc
5655            .accounts
5656            .accounts_db
5657            .verify_accounts_hash_in_bg
5658            .check_complete()
5659    }
5660
5661    /// Get this bank's storages to use for snapshots.
5662    ///
5663    /// If a base slot is provided, return only the storages that are *higher* than this slot.
5664    pub fn get_snapshot_storages(&self, base_slot: Option<Slot>) -> Vec<Arc<AccountStorageEntry>> {
5665        // if a base slot is provided, request storages starting at the slot *after*
5666        let start_slot = base_slot.map_or(0, |slot| slot.saturating_add(1));
5667        // we want to *include* the storage at our slot
5668        let requested_slots = start_slot..=self.slot();
5669
5670        self.rc.accounts.accounts_db.get_storages(requested_slots).0
5671    }
5672
5673    #[must_use]
5674    fn verify_hash(&self) -> bool {
5675        assert!(self.is_frozen());
5676        let calculated_hash = self.hash_internal_state();
5677        let expected_hash = self.hash();
5678
5679        if calculated_hash == expected_hash {
5680            true
5681        } else {
5682            warn!(
5683                "verify failed: slot: {}, {} (calculated) != {} (expected)",
5684                self.slot(),
5685                calculated_hash,
5686                expected_hash
5687            );
5688            false
5689        }
5690    }
5691
5692    pub fn verify_transaction(
5693        &self,
5694        tx: VersionedTransaction,
5695        verification_mode: TransactionVerificationMode,
5696    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
5697        let sanitized_tx = {
5698            let size =
5699                bincode::serialized_size(&tx).map_err(|_| TransactionError::SanitizeFailure)?;
5700            if size > PACKET_DATA_SIZE as u64 {
5701                return Err(TransactionError::SanitizeFailure);
5702            }
5703            let message_hash = if verification_mode == TransactionVerificationMode::FullVerification
5704            {
5705                tx.verify_and_hash_message()?
5706            } else {
5707                tx.message.hash()
5708            };
5709
5710            RuntimeTransaction::try_create(
5711                tx,
5712                MessageHash::Precomputed(message_hash),
5713                None,
5714                self,
5715                self.get_reserved_account_keys(),
5716            )
5717        }?;
5718
5719        let move_precompile_verification_to_svm = self
5720            .feature_set
5721            .is_active(&feature_set::move_precompile_verification_to_svm::id());
5722        if !move_precompile_verification_to_svm && {
5723            verification_mode == TransactionVerificationMode::HashAndVerifyPrecompiles
5724                || verification_mode == TransactionVerificationMode::FullVerification
5725        } {
5726            verify_precompiles(&sanitized_tx, &self.feature_set)?;
5727        }
5728
5729        Ok(sanitized_tx)
5730    }
5731
5732    pub fn fully_verify_transaction(
5733        &self,
5734        tx: VersionedTransaction,
5735    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
5736        self.verify_transaction(tx, TransactionVerificationMode::FullVerification)
5737    }
5738
5739    /// Checks if the transaction violates the bank's reserved keys.
5740    /// This needs to be checked upon epoch boundary crosses because the
5741    /// reserved key set may have changed since the initial sanitization.
5742    pub fn check_reserved_keys(&self, tx: &impl SVMMessage) -> Result<()> {
5743        // Check keys against the reserved set - these failures simply require us
5744        // to re-sanitize the transaction. We do not need to drop the transaction.
5745        let reserved_keys = self.get_reserved_account_keys();
5746        for (index, key) in tx.account_keys().iter().enumerate() {
5747            if tx.is_writable(index) && reserved_keys.contains(key) {
5748                return Err(TransactionError::ResanitizationNeeded);
5749            }
5750        }
5751
5752        Ok(())
5753    }
5754
5755    /// only called from ledger-tool or tests
5756    fn calculate_capitalization(&self, debug_verify: bool) -> u64 {
5757        let is_startup = true;
5758        self.rc
5759            .accounts
5760            .accounts_db
5761            .verify_accounts_hash_in_bg
5762            .join_background_thread();
5763        self.rc
5764            .accounts
5765            .accounts_db
5766            .update_accounts_hash_with_verify_from(
5767                // we have to use the index since the slot could be in the write cache still
5768                CalcAccountsHashDataSource::IndexForTests,
5769                debug_verify,
5770                self.slot(),
5771                &self.ancestors,
5772                None,
5773                self.epoch_schedule(),
5774                &self.rent_collector,
5775                is_startup,
5776            )
5777            .1
5778    }
5779
5780    /// only called from tests or ledger tool
5781    pub fn calculate_and_verify_capitalization(&self, debug_verify: bool) -> bool {
5782        let calculated = self.calculate_capitalization(debug_verify);
5783        let expected = self.capitalization();
5784        if calculated == expected {
5785            true
5786        } else {
5787            warn!(
5788                "Capitalization mismatch: calculated: {} != expected: {}",
5789                calculated, expected
5790            );
5791            false
5792        }
5793    }
5794
5795    /// Forcibly overwrites current capitalization by actually recalculating accounts' balances.
5796    /// This should only be used for developing purposes.
5797    pub fn set_capitalization(&self) -> u64 {
5798        let old = self.capitalization();
5799        // We cannot debug verify the hash calculation here because calculate_capitalization will use the index calculation due to callers using the write cache.
5800        // debug_verify only exists as an extra debugging step under the assumption that this code path is only used for tests. But, this is used by ledger-tool create-snapshot
5801        // for example.
5802        let debug_verify = false;
5803        self.capitalization
5804            .store(self.calculate_capitalization(debug_verify), Relaxed);
5805        old
5806    }
5807
5808    /// Returns the `AccountsHash` that was calculated for this bank's slot
5809    ///
5810    /// This fn is used when creating a snapshot with ledger-tool, or when
5811    /// packaging a snapshot into an archive (used to get the `SnapshotHash`).
5812    pub fn get_accounts_hash(&self) -> Option<AccountsHash> {
5813        self.rc
5814            .accounts
5815            .accounts_db
5816            .get_accounts_hash(self.slot())
5817            .map(|(accounts_hash, _)| accounts_hash)
5818    }
5819
5820    /// Returns the `IncrementalAccountsHash` that was calculated for this bank's slot
5821    ///
5822    /// This fn is used when creating an incremental snapshot with ledger-tool, or when
5823    /// packaging a snapshot into an archive (used to get the `SnapshotHash`).
5824    pub fn get_incremental_accounts_hash(&self) -> Option<IncrementalAccountsHash> {
5825        self.rc
5826            .accounts
5827            .accounts_db
5828            .get_incremental_accounts_hash(self.slot())
5829            .map(|(incremental_accounts_hash, _)| incremental_accounts_hash)
5830    }
5831
5832    /// Returns the `SnapshotHash` for this bank's slot
5833    ///
5834    /// This fn is used at startup to verify the bank was rebuilt correctly.
5835    ///
5836    /// # Panics
5837    ///
5838    /// If the snapshots lt hash feature is not enabled, panics if there is both-or-neither of an
5839    /// `AccountsHash` and an `IncrementalAccountsHash` for this bank's slot.  There may only be
5840    /// one or the other.
5841    pub fn get_snapshot_hash(&self) -> SnapshotHash {
5842        if self.is_snapshots_lt_hash_enabled() {
5843            self.get_lattice_snapshot_hash()
5844        } else {
5845            self.get_merkle_snapshot_hash()
5846        }
5847    }
5848
5849    /// Returns the merkle-based `SnapshotHash` for this bank's slot
5850    ///
5851    /// This fn is used at startup to verify the bank was rebuilt correctly.
5852    ///
5853    /// # Panics
5854    ///
5855    /// If the snapshots lt hash feature is not enabled, panics if there is both-or-neither of an
5856    /// `AccountsHash` and an `IncrementalAccountsHash` for this bank's slot.  There may only be
5857    /// one or the other.
5858    pub fn get_merkle_snapshot_hash(&self) -> SnapshotHash {
5859        let accounts_hash = self.get_accounts_hash();
5860        let incremental_accounts_hash = self.get_incremental_accounts_hash();
5861        let accounts_hash_kind = match (accounts_hash, incremental_accounts_hash) {
5862            (Some(_), Some(_)) => panic!("Both full and incremental accounts hashes are present for slot {}; it is ambiguous which one to use for the snapshot hash!", self.slot()),
5863            (Some(accounts_hash), None) => accounts_hash.into(),
5864            (None, Some(incremental_accounts_hash)) => incremental_accounts_hash.into(),
5865            (None, None) => panic!("accounts hash is required to get snapshot hash"),
5866        };
5867        let epoch_accounts_hash = self.get_epoch_accounts_hash_to_serialize();
5868        SnapshotHash::new(
5869            &MerkleOrLatticeAccountsHash::Merkle(accounts_hash_kind),
5870            epoch_accounts_hash.as_ref(),
5871            None,
5872        )
5873    }
5874
5875    /// Returns the lattice-based `SnapshotHash` for this bank's slot
5876    ///
5877    /// This fn is used at startup to verify the bank was rebuilt correctly.
5878    pub fn get_lattice_snapshot_hash(&self) -> SnapshotHash {
5879        SnapshotHash::new(
5880            &MerkleOrLatticeAccountsHash::Lattice,
5881            None,
5882            Some(self.accounts_lt_hash.lock().unwrap().0.checksum()),
5883        )
5884    }
5885
5886    pub fn load_account_into_read_cache(&self, key: &Pubkey) {
5887        self.rc
5888            .accounts
5889            .accounts_db
5890            .load_account_into_read_cache(&self.ancestors, key);
5891    }
5892
5893    pub fn update_accounts_hash(
5894        &self,
5895        data_source: CalcAccountsHashDataSource,
5896        mut debug_verify: bool,
5897        is_startup: bool,
5898    ) -> AccountsHash {
5899        let (accounts_hash, total_lamports) = self
5900            .rc
5901            .accounts
5902            .accounts_db
5903            .update_accounts_hash_with_verify_from(
5904                data_source,
5905                debug_verify,
5906                self.slot(),
5907                &self.ancestors,
5908                Some(self.capitalization()),
5909                self.epoch_schedule(),
5910                &self.rent_collector,
5911                is_startup,
5912            );
5913        if total_lamports != self.capitalization() {
5914            datapoint_info!(
5915                "capitalization_mismatch",
5916                ("slot", self.slot(), i64),
5917                ("calculated_lamports", total_lamports, i64),
5918                ("capitalization", self.capitalization(), i64),
5919            );
5920
5921            if !debug_verify {
5922                // cap mismatch detected. It has been logged to metrics above.
5923                // Run both versions of the calculation to attempt to get more info.
5924                debug_verify = true;
5925                self.rc
5926                    .accounts
5927                    .accounts_db
5928                    .update_accounts_hash_with_verify_from(
5929                        data_source,
5930                        debug_verify,
5931                        self.slot(),
5932                        &self.ancestors,
5933                        Some(self.capitalization()),
5934                        self.epoch_schedule(),
5935                        &self.rent_collector,
5936                        is_startup,
5937                    );
5938            }
5939
5940            panic!(
5941                "capitalization_mismatch. slot: {}, calculated_lamports: {}, capitalization: {}",
5942                self.slot(),
5943                total_lamports,
5944                self.capitalization()
5945            );
5946        }
5947        accounts_hash
5948    }
5949
5950    /// Calculate the incremental accounts hash from `base_slot` to `self`
5951    pub fn update_incremental_accounts_hash(&self, base_slot: Slot) -> IncrementalAccountsHash {
5952        let config = CalcAccountsHashConfig {
5953            use_bg_thread_pool: true,
5954            ancestors: None, // does not matter, will not be used
5955            epoch_schedule: &self.epoch_schedule,
5956            rent_collector: &self.rent_collector,
5957            store_detailed_debug_info_on_failure: false,
5958        };
5959        let storages = self.get_snapshot_storages(Some(base_slot));
5960        let sorted_storages = SortedStorages::new(&storages);
5961        self.rc
5962            .accounts
5963            .accounts_db
5964            .update_incremental_accounts_hash(
5965                &config,
5966                &sorted_storages,
5967                self.slot(),
5968                HashStats::default(),
5969            )
5970            .0
5971    }
5972
5973    /// A snapshot bank should be purged of 0 lamport accounts which are not part of the hash
5974    /// calculation and could shield other real accounts.
5975    pub fn verify_snapshot_bank(
5976        &self,
5977        test_hash_calculation: bool,
5978        skip_shrink: bool,
5979        force_clean: bool,
5980        latest_full_snapshot_slot: Slot,
5981        base: Option<(Slot, /*capitalization*/ u64)>,
5982        duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
5983    ) -> bool {
5984        // If we verify the accounts using the lattice-based hash *and* with storages (as opposed
5985        // to the index), then we rely on the DuplicatesLtHash as given by generate_index().  Since
5986        // the duplicates are based on a specific set of storages, we must use the exact same
5987        // storages to do the lattice-based accounts verification.  This means we must wait to
5988        // clean/shrink until *after* we've gotten Arcs to the storages (this prevents their
5989        // untimely removal).  Simply, we call `verify_accounts_hash()` before we call `clean` or
5990        // `shrink`.
5991        let (verified_accounts, verify_accounts_time_us) = measure_us!({
5992            let should_verify_accounts = !self.rc.accounts.accounts_db.skip_initial_hash_calc;
5993            if should_verify_accounts {
5994                info!("Verifying accounts...");
5995                let verified = self.verify_accounts_hash(
5996                    base,
5997                    VerifyAccountsHashConfig {
5998                        test_hash_calculation,
5999                        ignore_mismatch: false,
6000                        require_rooted_bank: false,
6001                        run_in_background: true,
6002                        store_hash_raw_data_for_debug: false,
6003                    },
6004                    duplicates_lt_hash,
6005                );
6006                info!("Verifying accounts... In background.");
6007                verified
6008            } else {
6009                info!("Verifying accounts... Skipped.");
6010                self.rc
6011                    .accounts
6012                    .accounts_db
6013                    .verify_accounts_hash_in_bg
6014                    .verification_complete();
6015                true
6016            }
6017        });
6018
6019        let (_, clean_time_us) = measure_us!({
6020            let should_clean = force_clean || (!skip_shrink && self.slot() > 0);
6021            if should_clean {
6022                info!("Cleaning...");
6023                // We cannot clean past the latest full snapshot's slot because we are about to
6024                // perform an accounts hash calculation *up to that slot*.  If we cleaned *past*
6025                // that slot, then accounts could be removed from older storages, which would
6026                // change the accounts hash.
6027                self.rc.accounts.accounts_db.clean_accounts(
6028                    Some(latest_full_snapshot_slot),
6029                    true,
6030                    self.epoch_schedule(),
6031                    self.clean_accounts_old_storages_policy(),
6032                );
6033                info!("Cleaning... Done.");
6034            } else {
6035                info!("Cleaning... Skipped.");
6036            }
6037        });
6038
6039        let (_, shrink_time_us) = measure_us!({
6040            let should_shrink = !skip_shrink && self.slot() > 0;
6041            if should_shrink {
6042                info!("Shrinking...");
6043                self.rc.accounts.accounts_db.shrink_all_slots(
6044                    true,
6045                    self.epoch_schedule(),
6046                    // we cannot allow the snapshot slot to be shrunk
6047                    Some(self.slot()),
6048                );
6049                info!("Shrinking... Done.");
6050            } else {
6051                info!("Shrinking... Skipped.");
6052            }
6053        });
6054
6055        info!("Verifying bank...");
6056        let (verified_bank, verify_bank_time_us) = measure_us!(self.verify_hash());
6057        info!("Verifying bank... Done.");
6058
6059        datapoint_info!(
6060            "verify_snapshot_bank",
6061            ("clean_us", clean_time_us, i64),
6062            ("shrink_us", shrink_time_us, i64),
6063            ("verify_accounts_us", verify_accounts_time_us, i64),
6064            ("verify_bank_us", verify_bank_time_us, i64),
6065        );
6066
6067        verified_accounts && verified_bank
6068    }
6069
6070    /// Return the number of hashes per tick
6071    pub fn hashes_per_tick(&self) -> &Option<u64> {
6072        &self.hashes_per_tick
6073    }
6074
6075    /// Return the number of ticks per slot
6076    pub fn ticks_per_slot(&self) -> u64 {
6077        self.ticks_per_slot
6078    }
6079
6080    /// Return the number of slots per year
6081    pub fn slots_per_year(&self) -> f64 {
6082        self.slots_per_year
6083    }
6084
6085    /// Return the number of ticks since genesis.
6086    pub fn tick_height(&self) -> u64 {
6087        self.tick_height.load(Relaxed)
6088    }
6089
6090    /// Return the inflation parameters of the Bank
6091    pub fn inflation(&self) -> Inflation {
6092        *self.inflation.read().unwrap()
6093    }
6094
6095    /// Return the rent collector for this Bank
6096    pub fn rent_collector(&self) -> &RentCollector {
6097        &self.rent_collector
6098    }
6099
6100    /// Return the total capitalization of the Bank
6101    pub fn capitalization(&self) -> u64 {
6102        self.capitalization.load(Relaxed)
6103    }
6104
6105    /// Return this bank's max_tick_height
6106    pub fn max_tick_height(&self) -> u64 {
6107        self.max_tick_height
6108    }
6109
6110    /// Return the block_height of this bank
6111    pub fn block_height(&self) -> u64 {
6112        self.block_height
6113    }
6114
6115    /// Return the number of slots per epoch for the given epoch
6116    pub fn get_slots_in_epoch(&self, epoch: Epoch) -> u64 {
6117        self.epoch_schedule().get_slots_in_epoch(epoch)
6118    }
6119
6120    /// returns the epoch for which this bank's leader_schedule_slot_offset and slot would
6121    ///  need to cache leader_schedule
6122    pub fn get_leader_schedule_epoch(&self, slot: Slot) -> Epoch {
6123        self.epoch_schedule().get_leader_schedule_epoch(slot)
6124    }
6125
6126    /// a bank-level cache of vote accounts and stake delegation info
6127    fn update_stakes_cache(
6128        &self,
6129        txs: &[impl SVMMessage],
6130        processing_results: &[TransactionProcessingResult],
6131    ) {
6132        debug_assert_eq!(txs.len(), processing_results.len());
6133        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
6134        txs.iter()
6135            .zip(processing_results)
6136            .filter_map(|(tx, processing_result)| {
6137                processing_result
6138                    .processed_transaction()
6139                    .map(|processed_tx| (tx, processed_tx))
6140            })
6141            .filter_map(|(tx, processed_tx)| {
6142                processed_tx
6143                    .executed_transaction()
6144                    .map(|executed_tx| (tx, executed_tx))
6145            })
6146            .filter(|(_, executed_tx)| executed_tx.was_successful())
6147            .flat_map(|(tx, executed_tx)| {
6148                let num_account_keys = tx.account_keys().len();
6149                let loaded_tx = &executed_tx.loaded_transaction;
6150                loaded_tx.accounts.iter().take(num_account_keys)
6151            })
6152            .for_each(|(pubkey, account)| {
6153                // note that this could get timed to: self.rc.accounts.accounts_db.stats.stakes_cache_check_and_store_us,
6154                //  but this code path is captured separately in ExecuteTimingType::UpdateStakesCacheUs
6155                self.stakes_cache
6156                    .check_and_store(pubkey, account, new_warmup_cooldown_rate_epoch);
6157            });
6158    }
6159
6160    /// current vote accounts for this bank along with the stake
6161    ///   attributed to each account
6162    pub fn vote_accounts(&self) -> Arc<VoteAccountsHashMap> {
6163        let stakes = self.stakes_cache.stakes();
6164        Arc::from(stakes.vote_accounts())
6165    }
6166
6167    /// Vote account for the given vote account pubkey.
6168    pub fn get_vote_account(&self, vote_account: &Pubkey) -> Option<VoteAccount> {
6169        let stakes = self.stakes_cache.stakes();
6170        let vote_account = stakes.vote_accounts().get(vote_account)?;
6171        Some(vote_account.clone())
6172    }
6173
6174    /// Get the EpochStakes for the current Bank::epoch
6175    pub fn current_epoch_stakes(&self) -> &EpochStakes {
6176        // The stakes for a given epoch (E) in self.epoch_stakes are keyed by leader schedule epoch
6177        // (E + 1) so the stakes for the current epoch are stored at self.epoch_stakes[E + 1]
6178        self.epoch_stakes
6179            .get(&self.epoch.saturating_add(1))
6180            .expect("Current epoch stakes must exist")
6181    }
6182
6183    /// Get the EpochStakes for a given epoch
6184    pub fn epoch_stakes(&self, epoch: Epoch) -> Option<&EpochStakes> {
6185        self.epoch_stakes.get(&epoch)
6186    }
6187
6188    pub fn epoch_stakes_map(&self) -> &HashMap<Epoch, EpochStakes> {
6189        &self.epoch_stakes
6190    }
6191
6192    /// Get the staked nodes map for the current Bank::epoch
6193    pub fn current_epoch_staked_nodes(&self) -> Arc<HashMap<Pubkey, u64>> {
6194        self.current_epoch_stakes().stakes().staked_nodes()
6195    }
6196
6197    pub fn epoch_staked_nodes(&self, epoch: Epoch) -> Option<Arc<HashMap<Pubkey, u64>>> {
6198        Some(self.epoch_stakes.get(&epoch)?.stakes().staked_nodes())
6199    }
6200
6201    /// Get the total epoch stake for the given epoch.
6202    pub fn epoch_total_stake(&self, epoch: Epoch) -> Option<u64> {
6203        self.epoch_stakes
6204            .get(&epoch)
6205            .map(|epoch_stakes| epoch_stakes.total_stake())
6206    }
6207
6208    /// Get the total epoch stake for the current Bank::epoch
6209    pub fn get_current_epoch_total_stake(&self) -> u64 {
6210        self.current_epoch_stakes().total_stake()
6211    }
6212
6213    /// vote accounts for the specific epoch along with the stake
6214    ///   attributed to each account
6215    pub fn epoch_vote_accounts(&self, epoch: Epoch) -> Option<&VoteAccountsHashMap> {
6216        let epoch_stakes = self.epoch_stakes.get(&epoch)?.stakes();
6217        Some(epoch_stakes.vote_accounts().as_ref())
6218    }
6219
6220    /// Get the vote accounts along with the stake attributed to each account
6221    /// for the current Bank::epoch
6222    pub fn get_current_epoch_vote_accounts(&self) -> &VoteAccountsHashMap {
6223        self.current_epoch_stakes()
6224            .stakes()
6225            .vote_accounts()
6226            .as_ref()
6227    }
6228
6229    /// Get the fixed authorized voter for the given vote account for the
6230    /// current epoch
6231    pub fn epoch_authorized_voter(&self, vote_account: &Pubkey) -> Option<&Pubkey> {
6232        self.epoch_stakes
6233            .get(&self.epoch)
6234            .expect("Epoch stakes for bank's own epoch must exist")
6235            .epoch_authorized_voters()
6236            .get(vote_account)
6237    }
6238
6239    /// Get the fixed set of vote accounts for the given node id for the
6240    /// current epoch
6241    pub fn epoch_vote_accounts_for_node_id(&self, node_id: &Pubkey) -> Option<&NodeVoteAccounts> {
6242        self.epoch_stakes
6243            .get(&self.epoch)
6244            .expect("Epoch stakes for bank's own epoch must exist")
6245            .node_id_to_vote_accounts()
6246            .get(node_id)
6247    }
6248
6249    /// Get the total stake belonging to vote accounts associated with the given node id for the
6250    /// given epoch.
6251    pub fn epoch_node_id_to_stake(&self, epoch: Epoch, node_id: &Pubkey) -> Option<u64> {
6252        self.epoch_stakes(epoch)
6253            .and_then(|epoch_stakes| epoch_stakes.node_id_to_stake(node_id))
6254    }
6255
6256    /// Get the fixed total stake of all vote accounts for current epoch
6257    pub fn total_epoch_stake(&self) -> u64 {
6258        self.epoch_stakes
6259            .get(&self.epoch)
6260            .expect("Epoch stakes for bank's own epoch must exist")
6261            .total_stake()
6262    }
6263
6264    /// Get the fixed stake of the given vote account for the current epoch
6265    pub fn epoch_vote_account_stake(&self, vote_account: &Pubkey) -> u64 {
6266        *self
6267            .epoch_vote_accounts(self.epoch())
6268            .expect("Bank epoch vote accounts must contain entry for the bank's own epoch")
6269            .get(vote_account)
6270            .map(|(stake, _)| stake)
6271            .unwrap_or(&0)
6272    }
6273
6274    /// given a slot, return the epoch and offset into the epoch this slot falls
6275    /// e.g. with a fixed number for slots_per_epoch, the calculation is simply:
6276    ///
6277    ///  ( slot/slots_per_epoch, slot % slots_per_epoch )
6278    ///
6279    pub fn get_epoch_and_slot_index(&self, slot: Slot) -> (Epoch, SlotIndex) {
6280        self.epoch_schedule().get_epoch_and_slot_index(slot)
6281    }
6282
6283    pub fn get_epoch_info(&self) -> EpochInfo {
6284        let absolute_slot = self.slot();
6285        let block_height = self.block_height();
6286        let (epoch, slot_index) = self.get_epoch_and_slot_index(absolute_slot);
6287        let slots_in_epoch = self.get_slots_in_epoch(epoch);
6288        let transaction_count = Some(self.transaction_count());
6289        EpochInfo {
6290            epoch,
6291            slot_index,
6292            slots_in_epoch,
6293            absolute_slot,
6294            block_height,
6295            transaction_count,
6296        }
6297    }
6298
6299    pub fn is_empty(&self) -> bool {
6300        !self.is_delta.load(Relaxed)
6301    }
6302
6303    pub fn add_mockup_builtin(
6304        &mut self,
6305        program_id: Pubkey,
6306        builtin_function: BuiltinFunctionWithContext,
6307    ) {
6308        self.transaction_processor.add_builtin(
6309            self,
6310            program_id,
6311            "mockup",
6312            ProgramCacheEntry::new_builtin(self.slot, 0, builtin_function),
6313        );
6314    }
6315
6316    pub fn add_precompile(&mut self, program_id: &Pubkey) {
6317        debug!("Adding precompiled program {}", program_id);
6318        self.add_precompiled_account(program_id);
6319        debug!("Added precompiled program {:?}", program_id);
6320    }
6321
6322    // Call AccountsDb::clean_accounts()
6323    //
6324    // This fn is meant to be called by the snapshot handler in Accounts Background Service.  If
6325    // calling from elsewhere, ensure the same invariants hold/expectations are met.
6326    pub(crate) fn clean_accounts(&self) {
6327        // Don't clean the slot we're snapshotting because it may have zero-lamport
6328        // accounts that were included in the bank delta hash when the bank was frozen,
6329        // and if we clean them here, any newly created snapshot's hash for this bank
6330        // may not match the frozen hash.
6331        //
6332        // So when we're snapshotting, the highest slot to clean is lowered by one.
6333        let highest_slot_to_clean = self.slot().saturating_sub(1);
6334
6335        self.rc.accounts.accounts_db.clean_accounts(
6336            Some(highest_slot_to_clean),
6337            false,
6338            self.epoch_schedule(),
6339            self.clean_accounts_old_storages_policy(),
6340        );
6341    }
6342
6343    pub fn print_accounts_stats(&self) {
6344        self.rc.accounts.accounts_db.print_accounts_stats("");
6345    }
6346
6347    pub fn shrink_candidate_slots(&self) -> usize {
6348        self.rc
6349            .accounts
6350            .accounts_db
6351            .shrink_candidate_slots(self.epoch_schedule())
6352    }
6353
6354    pub(crate) fn shrink_ancient_slots(&self) {
6355        // Invoke ancient slot shrinking only when the validator is
6356        // explicitly configured to do so. This condition may be
6357        // removed when the skip rewrites feature is enabled.
6358        if self.are_ancient_storages_enabled() {
6359            self.rc
6360                .accounts
6361                .accounts_db
6362                .shrink_ancient_slots(self.epoch_schedule())
6363        }
6364    }
6365
6366    /// Returns if ancient storages are enabled or not
6367    pub fn are_ancient_storages_enabled(&self) -> bool {
6368        let can_skip_rewrites = self.bank_hash_skips_rent_rewrites();
6369        let test_skip_rewrites_but_include_in_bank_hash = self
6370            .rc
6371            .accounts
6372            .accounts_db
6373            .test_skip_rewrites_but_include_in_bank_hash;
6374        can_skip_rewrites || test_skip_rewrites_but_include_in_bank_hash
6375    }
6376
6377    /// Returns how clean_accounts() should handle old storages
6378    fn clean_accounts_old_storages_policy(&self) -> OldStoragesPolicy {
6379        if self.are_ancient_storages_enabled() {
6380            OldStoragesPolicy::Leave
6381        } else {
6382            OldStoragesPolicy::Clean
6383        }
6384    }
6385
6386    pub fn read_cost_tracker(&self) -> LockResult<RwLockReadGuard<CostTracker>> {
6387        self.cost_tracker.read()
6388    }
6389
6390    pub fn write_cost_tracker(&self) -> LockResult<RwLockWriteGuard<CostTracker>> {
6391        self.cost_tracker.write()
6392    }
6393
6394    // Check if the wallclock time from bank creation to now has exceeded the allotted
6395    // time for transaction processing
6396    pub fn should_bank_still_be_processing_txs(
6397        bank_creation_time: &Instant,
6398        max_tx_ingestion_nanos: u128,
6399    ) -> bool {
6400        // Do this check outside of the PoH lock, hence not a method on PohRecorder
6401        bank_creation_time.elapsed().as_nanos() <= max_tx_ingestion_nanos
6402    }
6403
6404    pub fn deactivate_feature(&mut self, id: &Pubkey) {
6405        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
6406        feature_set.active_mut().remove(id);
6407        feature_set.inactive_mut().insert(*id);
6408        self.feature_set = Arc::new(feature_set);
6409    }
6410
6411    pub fn activate_feature(&mut self, id: &Pubkey) {
6412        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
6413        feature_set.inactive_mut().remove(id);
6414        feature_set.active_mut().insert(*id, 0);
6415        self.feature_set = Arc::new(feature_set);
6416    }
6417
6418    pub fn fill_bank_with_ticks_for_tests(&self) {
6419        self.do_fill_bank_with_ticks_for_tests(&BankWithScheduler::no_scheduler_available())
6420    }
6421
6422    pub(crate) fn do_fill_bank_with_ticks_for_tests(&self, scheduler: &InstalledSchedulerRwLock) {
6423        if self.tick_height.load(Relaxed) < self.max_tick_height {
6424            let last_blockhash = self.last_blockhash();
6425            while self.last_blockhash() == last_blockhash {
6426                self.register_tick(&Hash::new_unique(), scheduler)
6427            }
6428        } else {
6429            warn!("Bank already reached max tick height, cannot fill it with more ticks");
6430        }
6431    }
6432
6433    /// Get a set of all actively reserved account keys that are not allowed to
6434    /// be write-locked during transaction processing.
6435    pub fn get_reserved_account_keys(&self) -> &HashSet<Pubkey> {
6436        &self.reserved_account_keys.active
6437    }
6438
6439    // This is called from snapshot restore AND for each epoch boundary
6440    // The entire code path herein must be idempotent
6441    fn apply_feature_activations(
6442        &mut self,
6443        caller: ApplyFeatureActivationsCaller,
6444        debug_do_not_add_builtins: bool,
6445    ) {
6446        use ApplyFeatureActivationsCaller as Caller;
6447        let allow_new_activations = match caller {
6448            Caller::FinishInit => false,
6449            Caller::NewFromParent => true,
6450            Caller::WarpFromParent => false,
6451        };
6452        let (feature_set, new_feature_activations) =
6453            self.compute_active_feature_set(allow_new_activations);
6454        self.feature_set = Arc::new(feature_set);
6455
6456        // Update activation slot of features in `new_feature_activations`
6457        for feature_id in new_feature_activations.iter() {
6458            if let Some(mut account) = self.get_account_with_fixed_root(feature_id) {
6459                if let Some(mut feature) = feature::from_account(&account) {
6460                    feature.activated_at = Some(self.slot());
6461                    if feature::to_account(&feature, &mut account).is_some() {
6462                        self.store_account(feature_id, &account);
6463                    }
6464                    info!("Feature {} activated at slot {}", feature_id, self.slot());
6465                }
6466            }
6467        }
6468
6469        // Update active set of reserved account keys which are not allowed to be write locked
6470        self.reserved_account_keys = {
6471            let mut reserved_keys = ReservedAccountKeys::clone(&self.reserved_account_keys);
6472            reserved_keys.update_active_set(&self.feature_set);
6473            Arc::new(reserved_keys)
6474        };
6475
6476        if new_feature_activations.contains(&feature_set::pico_inflation::id()) {
6477            *self.inflation.write().unwrap() = Inflation::pico();
6478            self.fee_rate_governor.burn_percent = 50; // 50% fee burn
6479            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
6480        }
6481
6482        if !new_feature_activations.is_disjoint(&self.feature_set.full_inflation_features_enabled())
6483        {
6484            *self.inflation.write().unwrap() = Inflation::full();
6485            self.fee_rate_governor.burn_percent = 50; // 50% fee burn
6486            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
6487        }
6488
6489        if !debug_do_not_add_builtins {
6490            self.apply_builtin_program_feature_transitions(
6491                allow_new_activations,
6492                &new_feature_activations,
6493            );
6494        }
6495
6496        if new_feature_activations.contains(&feature_set::update_hashes_per_tick::id()) {
6497            self.apply_updated_hashes_per_tick(DEFAULT_HASHES_PER_TICK);
6498        }
6499
6500        if new_feature_activations.contains(&feature_set::update_hashes_per_tick2::id()) {
6501            self.apply_updated_hashes_per_tick(UPDATED_HASHES_PER_TICK2);
6502        }
6503
6504        if new_feature_activations.contains(&feature_set::update_hashes_per_tick3::id()) {
6505            self.apply_updated_hashes_per_tick(UPDATED_HASHES_PER_TICK3);
6506        }
6507
6508        if new_feature_activations.contains(&feature_set::update_hashes_per_tick4::id()) {
6509            self.apply_updated_hashes_per_tick(UPDATED_HASHES_PER_TICK4);
6510        }
6511
6512        if new_feature_activations.contains(&feature_set::update_hashes_per_tick5::id()) {
6513            self.apply_updated_hashes_per_tick(UPDATED_HASHES_PER_TICK5);
6514        }
6515
6516        if new_feature_activations.contains(&feature_set::update_hashes_per_tick6::id()) {
6517            self.apply_updated_hashes_per_tick(UPDATED_HASHES_PER_TICK6);
6518        }
6519
6520        if new_feature_activations.contains(&feature_set::accounts_lt_hash::id()) {
6521            // Activating the accounts lt hash feature means we need to have an accounts lt hash
6522            // value at the end of this if-block.  If the cli arg has been used, that means we
6523            // already have an accounts lt hash and do not need to recalculate it.
6524            if self
6525                .rc
6526                .accounts
6527                .accounts_db
6528                .is_experimental_accumulator_hash_enabled()
6529            {
6530                // We already have an accounts lt hash value, so no need to recalculate it.
6531                // Nothing else to do here.
6532            } else {
6533                let parent_slot = self.parent_slot;
6534                info!(
6535                    "Calculating the accounts lt hash for slot {parent_slot} \
6536                     as part of feature activation; this may take some time...",
6537                );
6538                // We must calculate the accounts lt hash now as part of feature activation.
6539                // Note, this bank is *not* frozen yet, which means it will later call
6540                // `update_accounts_lt_hash()`.  Therefore, we calculate the accounts lt hash based
6541                // on *our parent*, not us!
6542                let parent_ancestors = {
6543                    let mut ancestors = self.ancestors.clone();
6544                    ancestors.remove(&self.slot());
6545                    ancestors
6546                };
6547                let (parent_accounts_lt_hash, duration) = meas_dur!({
6548                    self.rc
6549                        .accounts
6550                        .accounts_db
6551                        .calculate_accounts_lt_hash_at_startup_from_index(
6552                            &parent_ancestors,
6553                            parent_slot,
6554                        )
6555                });
6556                *self.accounts_lt_hash.get_mut().unwrap() = parent_accounts_lt_hash;
6557                info!(
6558                    "Calculating the accounts lt hash for slot {parent_slot} \
6559                     completed in {duration:?}, accounts_lt_hash checksum: {}",
6560                    self.accounts_lt_hash.get_mut().unwrap().0.checksum(),
6561                );
6562            }
6563        }
6564
6565        if new_feature_activations.contains(&feature_set::raise_block_limits_to_50m::id())
6566            && !self
6567                .feature_set
6568                .is_active(&feature_set::raise_block_limits_to_60m::id())
6569        {
6570            let (account_cost_limit, block_cost_limit, vote_cost_limit) = simd_0207_block_limits();
6571            self.write_cost_tracker().unwrap().set_limits(
6572                account_cost_limit,
6573                block_cost_limit,
6574                vote_cost_limit,
6575            );
6576        }
6577
6578        if new_feature_activations.contains(&feature_set::raise_block_limits_to_60m::id()) {
6579            let (account_cost_limit, block_cost_limit, vote_cost_limit) = simd_0256_block_limits();
6580            self.write_cost_tracker().unwrap().set_limits(
6581                account_cost_limit,
6582                block_cost_limit,
6583                vote_cost_limit,
6584            );
6585        }
6586
6587        if new_feature_activations.contains(&feature_set::remove_accounts_delta_hash::id()) {
6588            // If the accounts delta hash has been removed, then we no longer need to compute the
6589            // AccountHash for modified accounts, and can stop the background account hasher.
6590            self.rc.accounts.accounts_db.stop_background_hasher();
6591        }
6592    }
6593
6594    fn apply_updated_hashes_per_tick(&mut self, hashes_per_tick: u64) {
6595        info!(
6596            "Activating update_hashes_per_tick {} at slot {}",
6597            hashes_per_tick,
6598            self.slot(),
6599        );
6600        self.hashes_per_tick = Some(hashes_per_tick);
6601    }
6602
6603    fn adjust_sysvar_balance_for_rent(&self, account: &mut AccountSharedData) {
6604        account.set_lamports(
6605            self.get_minimum_balance_for_rent_exemption(account.data().len())
6606                .max(account.lamports()),
6607        );
6608    }
6609
6610    /// Compute the active feature set based on the current bank state,
6611    /// and return it together with the set of newly activated features.
6612    fn compute_active_feature_set(&self, include_pending: bool) -> (FeatureSet, AHashSet<Pubkey>) {
6613        let mut active = self.feature_set.active().clone();
6614        let mut inactive = AHashSet::new();
6615        let mut pending = AHashSet::new();
6616        let slot = self.slot();
6617
6618        for feature_id in self.feature_set.inactive() {
6619            let mut activated = None;
6620            if let Some(account) = self.get_account_with_fixed_root(feature_id) {
6621                if let Some(feature) = feature::from_account(&account) {
6622                    match feature.activated_at {
6623                        None if include_pending => {
6624                            // Feature activation is pending
6625                            pending.insert(*feature_id);
6626                            activated = Some(slot);
6627                        }
6628                        Some(activation_slot) if slot >= activation_slot => {
6629                            // Feature has been activated already
6630                            activated = Some(activation_slot);
6631                        }
6632                        _ => {}
6633                    }
6634                }
6635            }
6636            if let Some(slot) = activated {
6637                active.insert(*feature_id, slot);
6638            } else {
6639                inactive.insert(*feature_id);
6640            }
6641        }
6642
6643        (FeatureSet::new(active, inactive), pending)
6644    }
6645
6646    fn apply_builtin_program_feature_transitions(
6647        &mut self,
6648        only_apply_transitions_for_new_features: bool,
6649        new_feature_activations: &AHashSet<Pubkey>,
6650    ) {
6651        for builtin in BUILTINS.iter() {
6652            // The `builtin_is_bpf` flag is used to handle the case where a
6653            // builtin is scheduled to be enabled by one feature gate and
6654            // later migrated to Core BPF by another.
6655            //
6656            // There should never be a case where a builtin is set to be
6657            // migrated to Core BPF and is also set to be enabled on feature
6658            // activation on the same feature gate. However, the
6659            // `builtin_is_bpf` flag will handle this case as well, electing
6660            // to first attempt the migration to Core BPF.
6661            //
6662            // The migration to Core BPF will fail gracefully because the
6663            // program account will not exist. The builtin will subsequently
6664            // be enabled, but it will never be migrated to Core BPF.
6665            //
6666            // Using the same feature gate for both enabling and migrating a
6667            // builtin to Core BPF should be strictly avoided.
6668            let mut builtin_is_bpf = false;
6669            if let Some(core_bpf_migration_config) = &builtin.core_bpf_migration_config {
6670                // If the builtin is set to be migrated to Core BPF on feature
6671                // activation, perform the migration and do not add the program
6672                // to the bank's builtins. The migration will remove it from
6673                // the builtins list and the cache.
6674                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
6675                    if let Err(e) = self
6676                        .migrate_builtin_to_core_bpf(&builtin.program_id, core_bpf_migration_config)
6677                    {
6678                        warn!(
6679                            "Failed to migrate builtin {} to Core BPF: {}",
6680                            builtin.name, e
6681                        );
6682                    } else {
6683                        builtin_is_bpf = true;
6684                    }
6685                } else {
6686                    // If the builtin has already been migrated to Core BPF, do not
6687                    // add it to the bank's builtins.
6688                    builtin_is_bpf = self
6689                        .get_account(&builtin.program_id)
6690                        .map(|a| a.owner() == &bpf_loader_upgradeable::id())
6691                        .unwrap_or(false);
6692                }
6693            };
6694
6695            if let Some(feature_id) = builtin.enable_feature_id {
6696                let should_enable_builtin_on_feature_transition = !builtin_is_bpf
6697                    && if only_apply_transitions_for_new_features {
6698                        new_feature_activations.contains(&feature_id)
6699                    } else {
6700                        self.feature_set.is_active(&feature_id)
6701                    };
6702
6703                if should_enable_builtin_on_feature_transition {
6704                    self.transaction_processor.add_builtin(
6705                        self,
6706                        builtin.program_id,
6707                        builtin.name,
6708                        ProgramCacheEntry::new_builtin(
6709                            self.feature_set.activated_slot(&feature_id).unwrap_or(0),
6710                            builtin.name.len(),
6711                            builtin.entrypoint,
6712                        ),
6713                    );
6714                }
6715            }
6716        }
6717
6718        // Migrate any necessary stateless builtins to core BPF.
6719        // Stateless builtins do not have an `enable_feature_id` since they
6720        // do not exist on-chain.
6721        for stateless_builtin in STATELESS_BUILTINS.iter() {
6722            if let Some(core_bpf_migration_config) = &stateless_builtin.core_bpf_migration_config {
6723                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
6724                    if let Err(e) = self.migrate_builtin_to_core_bpf(
6725                        &stateless_builtin.program_id,
6726                        core_bpf_migration_config,
6727                    ) {
6728                        warn!(
6729                            "Failed to migrate stateless builtin {} to Core BPF: {}",
6730                            stateless_builtin.name, e
6731                        );
6732                    }
6733                }
6734            }
6735        }
6736
6737        for precompile in get_precompiles() {
6738            let should_add_precompile = precompile
6739                .feature
6740                .as_ref()
6741                .map(|feature_id| self.feature_set.is_active(feature_id))
6742                .unwrap_or(false);
6743            if should_add_precompile {
6744                self.add_precompile(&precompile.program_id);
6745            }
6746        }
6747    }
6748
6749    /// Use to replace programs by feature activation
6750    #[allow(dead_code)]
6751    fn replace_program_account(
6752        &mut self,
6753        old_address: &Pubkey,
6754        new_address: &Pubkey,
6755        datapoint_name: &'static str,
6756    ) {
6757        if let Some(old_account) = self.get_account_with_fixed_root(old_address) {
6758            if let Some(new_account) = self.get_account_with_fixed_root(new_address) {
6759                datapoint_info!(datapoint_name, ("slot", self.slot, i64));
6760
6761                // Burn lamports in the old account
6762                self.capitalization
6763                    .fetch_sub(old_account.lamports(), Relaxed);
6764
6765                // Transfer new account to old account
6766                self.store_account(old_address, &new_account);
6767
6768                // Clear new account
6769                self.store_account(new_address, &AccountSharedData::default());
6770
6771                // Unload a program from the bank's cache
6772                self.transaction_processor
6773                    .program_cache
6774                    .write()
6775                    .unwrap()
6776                    .remove_programs([*old_address].into_iter());
6777
6778                self.calculate_and_update_accounts_data_size_delta_off_chain(
6779                    old_account.data().len(),
6780                    new_account.data().len(),
6781                );
6782            }
6783        }
6784    }
6785
6786    /// Get all the accounts for this bank and calculate stats
6787    pub fn get_total_accounts_stats(&self) -> ScanResult<TotalAccountsStats> {
6788        let accounts = self.get_all_accounts(false)?;
6789        Ok(self.calculate_total_accounts_stats(
6790            accounts
6791                .iter()
6792                .map(|(pubkey, account, _slot)| (pubkey, account)),
6793        ))
6794    }
6795
6796    /// Given all the accounts for a bank, calculate stats
6797    pub fn calculate_total_accounts_stats<'a>(
6798        &self,
6799        accounts: impl Iterator<Item = (&'a Pubkey, &'a AccountSharedData)>,
6800    ) -> TotalAccountsStats {
6801        let rent_collector = self.rent_collector();
6802        let mut total_accounts_stats = TotalAccountsStats::default();
6803        accounts.for_each(|(pubkey, account)| {
6804            total_accounts_stats.accumulate_account(pubkey, account, rent_collector);
6805        });
6806
6807        total_accounts_stats
6808    }
6809
6810    /// Must a snapshot of this bank include the EAH?
6811    pub fn must_include_epoch_accounts_hash_in_snapshot(&self) -> bool {
6812        epoch_accounts_hash_utils::is_enabled_this_epoch(self)
6813            && epoch_accounts_hash_utils::is_in_calculation_window(self)
6814    }
6815
6816    /// Get the EAH that will be used by snapshots
6817    ///
6818    /// Since snapshots are taken on roots, if the bank is in the EAH calculation window then an
6819    /// EAH *must* be included.  This means if an EAH calculation is currently in-flight we will
6820    /// wait for it to complete.
6821    pub fn get_epoch_accounts_hash_to_serialize(&self) -> Option<EpochAccountsHash> {
6822        if !self.must_include_epoch_accounts_hash_in_snapshot() {
6823            return None;
6824        }
6825
6826        let (epoch_accounts_hash, waiting_time_us) = measure_us!(self
6827            .rc
6828            .accounts
6829            .accounts_db
6830            .epoch_accounts_hash_manager
6831            .wait_get_epoch_accounts_hash());
6832
6833        datapoint_info!(
6834            "bank-get_epoch_accounts_hash_to_serialize",
6835            ("slot", self.slot(), i64),
6836            ("waiting-time-us", waiting_time_us, i64),
6837        );
6838        Some(epoch_accounts_hash)
6839    }
6840
6841    /// Convenience fn to get the Epoch Accounts Hash
6842    pub fn epoch_accounts_hash(&self) -> Option<EpochAccountsHash> {
6843        self.rc
6844            .accounts
6845            .accounts_db
6846            .epoch_accounts_hash_manager
6847            .try_get_epoch_accounts_hash()
6848    }
6849
6850    pub fn is_in_slot_hashes_history(&self, slot: &Slot) -> bool {
6851        if slot < &self.slot {
6852            if let Ok(slot_hashes) = self.transaction_processor.sysvar_cache().get_slot_hashes() {
6853                return slot_hashes.get(slot).is_some();
6854            }
6855        }
6856        false
6857    }
6858
6859    pub fn check_program_modification_slot(&self) -> bool {
6860        self.check_program_modification_slot
6861    }
6862
6863    pub fn set_check_program_modification_slot(&mut self, check: bool) {
6864        self.check_program_modification_slot = check;
6865    }
6866
6867    pub fn fee_structure(&self) -> &FeeStructure {
6868        &self.fee_structure
6869    }
6870
6871    pub fn block_id(&self) -> Option<Hash> {
6872        *self.block_id.read().unwrap()
6873    }
6874
6875    pub fn set_block_id(&self, block_id: Option<Hash>) {
6876        *self.block_id.write().unwrap() = block_id;
6877    }
6878
6879    pub fn compute_budget(&self) -> Option<ComputeBudget> {
6880        self.compute_budget
6881    }
6882
6883    pub fn add_builtin(&self, program_id: Pubkey, name: &str, builtin: ProgramCacheEntry) {
6884        self.transaction_processor
6885            .add_builtin(self, program_id, name, builtin)
6886    }
6887
6888    pub fn get_bank_hash_stats(&self) -> BankHashStats {
6889        self.bank_hash_stats.load()
6890    }
6891}
6892
6893impl TransactionProcessingCallback for Bank {
6894    fn account_matches_owners(&self, account: &Pubkey, owners: &[Pubkey]) -> Option<usize> {
6895        self.rc
6896            .accounts
6897            .accounts_db
6898            .account_matches_owners(&self.ancestors, account, owners)
6899            .ok()
6900    }
6901
6902    fn get_account_shared_data(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
6903        self.rc
6904            .accounts
6905            .accounts_db
6906            .load_with_fixed_root(&self.ancestors, pubkey)
6907            .map(|(acc, _)| acc)
6908    }
6909
6910    // NOTE: must hold idempotent for the same set of arguments
6911    /// Add a builtin program account
6912    fn add_builtin_account(&self, name: &str, program_id: &Pubkey) {
6913        let existing_genuine_program =
6914            self.get_account_with_fixed_root(program_id)
6915                .and_then(|account| {
6916                    // it's very unlikely to be squatted at program_id as non-system account because of burden to
6917                    // find victim's pubkey/hash. So, when account.owner is indeed native_loader's, it's
6918                    // safe to assume it's a genuine program.
6919                    if native_loader::check_id(account.owner()) {
6920                        Some(account)
6921                    } else {
6922                        // malicious account is pre-occupying at program_id
6923                        self.burn_and_purge_account(program_id, account);
6924                        None
6925                    }
6926                });
6927
6928        // introducing builtin program
6929        if existing_genuine_program.is_some() {
6930            // The existing account is sufficient
6931            return;
6932        }
6933
6934        assert!(
6935            !self.freeze_started(),
6936            "Can't change frozen bank by adding not-existing new builtin program ({name}, {program_id}). \
6937            Maybe, inconsistent program activation is detected on snapshot restore?"
6938        );
6939
6940        // Add a bogus executable builtin account, which will be loaded and ignored.
6941        let account = native_loader::create_loadable_account_with_fields(
6942            name,
6943            self.inherit_specially_retained_account_fields(&existing_genuine_program),
6944        );
6945        self.store_account_and_update_capitalization(program_id, &account);
6946    }
6947
6948    fn inspect_account(&self, address: &Pubkey, account_state: AccountState, is_writable: bool) {
6949        if self.is_accounts_lt_hash_enabled() {
6950            self.inspect_account_for_accounts_lt_hash(address, &account_state, is_writable);
6951        }
6952    }
6953
6954    fn get_current_epoch_vote_account_stake(&self, vote_address: &Pubkey) -> u64 {
6955        self.get_current_epoch_vote_accounts()
6956            .get(vote_address)
6957            .map(|(stake, _)| (*stake))
6958            .unwrap_or(0)
6959    }
6960
6961    fn calculate_fee(
6962        &self,
6963        message: &impl SVMMessage,
6964        lamports_per_signature: u64,
6965        prioritization_fee: u64,
6966        feature_set: &FeatureSet,
6967    ) -> FeeDetails {
6968        solana_fee::calculate_fee_details(
6969            message,
6970            false, /* zero_fees_for_test */
6971            lamports_per_signature,
6972            prioritization_fee,
6973            FeeFeatures::from(feature_set),
6974        )
6975    }
6976}
6977
6978#[cfg(feature = "dev-context-only-utils")]
6979impl Bank {
6980    pub fn wrap_with_bank_forks_for_tests(self) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
6981        let bank_forks = BankForks::new_rw_arc(self);
6982        let bank = bank_forks.read().unwrap().root_bank();
6983        (bank, bank_forks)
6984    }
6985
6986    pub fn default_for_tests() -> Self {
6987        let accounts_db = AccountsDb::default_for_tests();
6988        let accounts = Accounts::new(Arc::new(accounts_db));
6989        Self::default_with_accounts(accounts)
6990    }
6991
6992    pub fn new_with_bank_forks_for_tests(
6993        genesis_config: &GenesisConfig,
6994    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
6995        let bank = Self::new_for_tests(genesis_config);
6996        bank.wrap_with_bank_forks_for_tests()
6997    }
6998
6999    pub fn new_for_tests(genesis_config: &GenesisConfig) -> Self {
7000        Self::new_with_config_for_tests(genesis_config, BankTestConfig::default())
7001    }
7002
7003    pub fn new_with_mockup_builtin_for_tests(
7004        genesis_config: &GenesisConfig,
7005        program_id: Pubkey,
7006        builtin_function: BuiltinFunctionWithContext,
7007    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
7008        let mut bank = Self::new_for_tests(genesis_config);
7009        bank.add_mockup_builtin(program_id, builtin_function);
7010        bank.wrap_with_bank_forks_for_tests()
7011    }
7012
7013    pub fn new_no_wallclock_throttle_for_tests(
7014        genesis_config: &GenesisConfig,
7015    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
7016        let mut bank = Self::new_for_tests(genesis_config);
7017
7018        bank.ns_per_slot = u128::MAX;
7019        bank.wrap_with_bank_forks_for_tests()
7020    }
7021
7022    pub fn new_with_config_for_tests(
7023        genesis_config: &GenesisConfig,
7024        test_config: BankTestConfig,
7025    ) -> Self {
7026        Self::new_with_paths_for_tests(
7027            genesis_config,
7028            Arc::new(RuntimeConfig::default()),
7029            test_config,
7030            Vec::new(),
7031        )
7032    }
7033
7034    pub fn new_with_paths_for_tests(
7035        genesis_config: &GenesisConfig,
7036        runtime_config: Arc<RuntimeConfig>,
7037        test_config: BankTestConfig,
7038        paths: Vec<PathBuf>,
7039    ) -> Self {
7040        Self::new_with_paths(
7041            genesis_config,
7042            runtime_config,
7043            paths,
7044            None,
7045            None,
7046            false,
7047            Some(test_config.accounts_db_config),
7048            None,
7049            Some(Pubkey::new_unique()),
7050            Arc::default(),
7051            None,
7052            None,
7053        )
7054    }
7055
7056    pub fn new_for_benches(genesis_config: &GenesisConfig) -> Self {
7057        Self::new_with_paths_for_benches(genesis_config, Vec::new())
7058    }
7059
7060    /// Intended for use by benches only.
7061    /// create new bank with the given config and paths.
7062    pub fn new_with_paths_for_benches(genesis_config: &GenesisConfig, paths: Vec<PathBuf>) -> Self {
7063        Self::new_with_paths(
7064            genesis_config,
7065            Arc::<RuntimeConfig>::default(),
7066            paths,
7067            None,
7068            None,
7069            false,
7070            Some(ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS),
7071            None,
7072            Some(Pubkey::new_unique()),
7073            Arc::default(),
7074            None,
7075            None,
7076        )
7077    }
7078
7079    /// Prepare a transaction batch from a list of legacy transactions. Used for tests only.
7080    #[cfg(feature = "dev-context-only-utils")]
7081    pub fn prepare_batch_for_tests(
7082        &self,
7083        txs: Vec<Transaction>,
7084    ) -> TransactionBatch<RuntimeTransaction<SanitizedTransaction>> {
7085        let transaction_account_lock_limit = self.get_transaction_account_lock_limit();
7086        let sanitized_txs = txs
7087            .into_iter()
7088            .map(RuntimeTransaction::from_transaction_for_tests)
7089            .collect::<Vec<_>>();
7090        let lock_results = self
7091            .rc
7092            .accounts
7093            .lock_accounts(sanitized_txs.iter(), transaction_account_lock_limit);
7094        TransactionBatch::new(lock_results, self, OwnedOrBorrowed::Owned(sanitized_txs))
7095    }
7096
7097    /// Set the initial accounts data size
7098    /// NOTE: This fn is *ONLY FOR TESTS*
7099    pub fn set_accounts_data_size_initial_for_tests(&mut self, amount: u64) {
7100        self.accounts_data_size_initial = amount;
7101    }
7102
7103    /// Update the accounts data size off-chain delta
7104    /// NOTE: This fn is *ONLY FOR TESTS*
7105    pub fn update_accounts_data_size_delta_off_chain_for_tests(&self, amount: i64) {
7106        self.update_accounts_data_size_delta_off_chain(amount)
7107    }
7108
7109    #[cfg(test)]
7110    fn restore_old_behavior_for_fragile_tests(&self) {
7111        self.lazy_rent_collection.store(true, Relaxed);
7112    }
7113
7114    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
7115    ///
7116    /// # Panics
7117    ///
7118    /// Panics if any of the transactions do not pass sanitization checks.
7119    #[must_use]
7120    pub fn process_transactions<'a>(
7121        &self,
7122        txs: impl Iterator<Item = &'a Transaction>,
7123    ) -> Vec<Result<()>> {
7124        self.try_process_transactions(txs).unwrap()
7125    }
7126
7127    /// Process entry transactions in a single batch. This is used for benches and unit tests.
7128    ///
7129    /// # Panics
7130    ///
7131    /// Panics if any of the transactions do not pass sanitization checks.
7132    #[must_use]
7133    pub fn process_entry_transactions(&self, txs: Vec<VersionedTransaction>) -> Vec<Result<()>> {
7134        self.try_process_entry_transactions(txs).unwrap()
7135    }
7136
7137    #[cfg(test)]
7138    pub fn flush_accounts_cache_slot_for_tests(&self) {
7139        self.rc
7140            .accounts
7141            .accounts_db
7142            .flush_accounts_cache_slot_for_tests(self.slot())
7143    }
7144
7145    /// This is only valid to call from tests.
7146    /// block until initial accounts hash verification has completed
7147    pub fn wait_for_initial_accounts_hash_verification_completed_for_tests(&self) {
7148        self.rc
7149            .accounts
7150            .accounts_db
7151            .verify_accounts_hash_in_bg
7152            .join_background_thread()
7153    }
7154
7155    pub fn get_sysvar_cache_for_tests(&self) -> SysvarCache {
7156        self.transaction_processor.get_sysvar_cache_for_tests()
7157    }
7158
7159    pub fn update_accounts_hash_for_tests(&self) -> AccountsHash {
7160        self.update_accounts_hash(CalcAccountsHashDataSource::IndexForTests, false, false)
7161    }
7162
7163    pub fn new_program_cache_for_tx_batch_for_slot(&self, slot: Slot) -> ProgramCacheForTxBatch {
7164        ProgramCacheForTxBatch::new_from_cache(
7165            slot,
7166            self.epoch_schedule.get_epoch(slot),
7167            &self.transaction_processor.program_cache.read().unwrap(),
7168        )
7169    }
7170
7171    pub fn get_transaction_processor(&self) -> &TransactionBatchProcessor<BankForks> {
7172        &self.transaction_processor
7173    }
7174
7175    pub fn set_fee_structure(&mut self, fee_structure: &FeeStructure) {
7176        self.fee_structure = fee_structure.clone();
7177    }
7178
7179    pub fn load_program(
7180        &self,
7181        pubkey: &Pubkey,
7182        reload: bool,
7183        effective_epoch: Epoch,
7184    ) -> Option<Arc<ProgramCacheEntry>> {
7185        let environments = self
7186            .transaction_processor
7187            .get_environments_for_epoch(effective_epoch)?;
7188        load_program_with_pubkey(
7189            self,
7190            &environments,
7191            pubkey,
7192            self.slot(),
7193            &mut ExecuteTimings::default(), // Called by ledger-tool, metrics not accumulated.
7194            reload,
7195        )
7196    }
7197
7198    pub fn withdraw(&self, pubkey: &Pubkey, lamports: u64) -> Result<()> {
7199        match self.get_account_with_fixed_root(pubkey) {
7200            Some(mut account) => {
7201                let min_balance = match get_system_account_kind(&account) {
7202                    Some(SystemAccountKind::Nonce) => self
7203                        .rent_collector
7204                        .rent
7205                        .minimum_balance(nonce::State::size()),
7206                    _ => 0,
7207                };
7208
7209                lamports
7210                    .checked_add(min_balance)
7211                    .filter(|required_balance| *required_balance <= account.lamports())
7212                    .ok_or(TransactionError::InsufficientFundsForFee)?;
7213                account
7214                    .checked_sub_lamports(lamports)
7215                    .map_err(|_| TransactionError::InsufficientFundsForFee)?;
7216                self.store_account(pubkey, &account);
7217
7218                Ok(())
7219            }
7220            None => Err(TransactionError::AccountNotFound),
7221        }
7222    }
7223
7224    pub fn set_hash_overrides(&self, hash_overrides: HashOverrides) {
7225        *self.hash_overrides.lock().unwrap() = hash_overrides;
7226    }
7227}
7228
7229/// Compute how much an account has changed size.  This function is useful when the data size delta
7230/// needs to be computed and passed to an `update_accounts_data_size_delta` function.
7231fn calculate_data_size_delta(old_data_size: usize, new_data_size: usize) -> i64 {
7232    assert!(old_data_size <= i64::MAX as usize);
7233    assert!(new_data_size <= i64::MAX as usize);
7234    let old_data_size = old_data_size as i64;
7235    let new_data_size = new_data_size as i64;
7236
7237    new_data_size.saturating_sub(old_data_size)
7238}
7239
7240/// Since `apply_feature_activations()` has different behavior depending on its caller, enumerate
7241/// those callers explicitly.
7242#[derive(Debug, Copy, Clone, Eq, PartialEq)]
7243enum ApplyFeatureActivationsCaller {
7244    FinishInit,
7245    NewFromParent,
7246    WarpFromParent,
7247}
7248
7249/// Return the computed values from `collect_rent_from_accounts()`
7250///
7251/// Since `collect_rent_from_accounts()` is running in parallel, instead of updating the
7252/// atomics/shared data inside this function, return those values in this struct for the caller to
7253/// process later.
7254#[derive(Debug, Default)]
7255struct CollectRentFromAccountsInfo {
7256    skipped_rewrites: Vec<(Pubkey, AccountHash)>,
7257    rent_collected_info: CollectedInfo,
7258    rent_rewards: Vec<(Pubkey, RewardInfo)>,
7259    time_collecting_rent_us: u64,
7260    time_storing_accounts_us: u64,
7261    num_accounts: usize,
7262}
7263
7264/// Return the computed values—of each iteration in the parallel loop inside
7265/// `collect_rent_in_partition()`—and then perform a reduce on all of them.
7266#[derive(Debug, Default)]
7267struct CollectRentInPartitionInfo {
7268    skipped_rewrites: Vec<(Pubkey, AccountHash)>,
7269    rent_collected: u64,
7270    accounts_data_size_reclaimed: u64,
7271    rent_rewards: Vec<(Pubkey, RewardInfo)>,
7272    time_loading_accounts_us: u64,
7273    time_collecting_rent_us: u64,
7274    time_storing_accounts_us: u64,
7275    num_accounts: usize,
7276}
7277
7278impl CollectRentInPartitionInfo {
7279    /// Create a new `CollectRentInPartitionInfo` from the results of loading accounts and
7280    /// collecting rent on them.
7281    #[must_use]
7282    fn new(info: CollectRentFromAccountsInfo, time_loading_accounts: Duration) -> Self {
7283        Self {
7284            skipped_rewrites: info.skipped_rewrites,
7285            rent_collected: info.rent_collected_info.rent_amount,
7286            accounts_data_size_reclaimed: info.rent_collected_info.account_data_len_reclaimed,
7287            rent_rewards: info.rent_rewards,
7288            time_loading_accounts_us: time_loading_accounts.as_micros() as u64,
7289            time_collecting_rent_us: info.time_collecting_rent_us,
7290            time_storing_accounts_us: info.time_storing_accounts_us,
7291            num_accounts: info.num_accounts,
7292        }
7293    }
7294
7295    /// Reduce (i.e. 'combine') two `CollectRentInPartitionInfo`s into one.
7296    ///
7297    /// This fn is used by `collect_rent_in_partition()` as the reduce step (of map-reduce) in its
7298    /// parallel loop of rent collection.
7299    #[must_use]
7300    fn reduce(lhs: Self, rhs: Self) -> Self {
7301        Self {
7302            skipped_rewrites: [lhs.skipped_rewrites, rhs.skipped_rewrites].concat(),
7303            rent_collected: lhs.rent_collected.saturating_add(rhs.rent_collected),
7304            accounts_data_size_reclaimed: lhs
7305                .accounts_data_size_reclaimed
7306                .saturating_add(rhs.accounts_data_size_reclaimed),
7307            rent_rewards: [lhs.rent_rewards, rhs.rent_rewards].concat(),
7308            time_loading_accounts_us: lhs
7309                .time_loading_accounts_us
7310                .saturating_add(rhs.time_loading_accounts_us),
7311            time_collecting_rent_us: lhs
7312                .time_collecting_rent_us
7313                .saturating_add(rhs.time_collecting_rent_us),
7314            time_storing_accounts_us: lhs
7315                .time_storing_accounts_us
7316                .saturating_add(rhs.time_storing_accounts_us),
7317            num_accounts: lhs.num_accounts.saturating_add(rhs.num_accounts),
7318        }
7319    }
7320}
7321
7322/// Struct to collect stats when scanning all accounts in `get_total_accounts_stats()`
7323#[derive(Debug, Default, Copy, Clone, Serialize)]
7324pub struct TotalAccountsStats {
7325    /// Total number of accounts
7326    pub num_accounts: usize,
7327    /// Total data size of all accounts
7328    pub data_len: usize,
7329
7330    /// Total number of executable accounts
7331    pub num_executable_accounts: usize,
7332    /// Total data size of executable accounts
7333    pub executable_data_len: usize,
7334
7335    /// Total number of rent exempt accounts
7336    pub num_rent_exempt_accounts: usize,
7337    /// Total number of rent paying accounts
7338    pub num_rent_paying_accounts: usize,
7339    /// Total number of rent paying accounts without data
7340    pub num_rent_paying_accounts_without_data: usize,
7341    /// Total amount of lamports in rent paying accounts
7342    pub lamports_in_rent_paying_accounts: u64,
7343}
7344
7345impl TotalAccountsStats {
7346    pub fn accumulate_account(
7347        &mut self,
7348        address: &Pubkey,
7349        account: &AccountSharedData,
7350        rent_collector: &RentCollector,
7351    ) {
7352        let data_len = account.data().len();
7353        self.num_accounts += 1;
7354        self.data_len += data_len;
7355
7356        if account.executable() {
7357            self.num_executable_accounts += 1;
7358            self.executable_data_len += data_len;
7359        }
7360
7361        if !rent_collector.should_collect_rent(address, account.executable())
7362            || rent_collector
7363                .get_rent_due(
7364                    account.lamports(),
7365                    account.data().len(),
7366                    account.rent_epoch(),
7367                )
7368                .is_exempt()
7369        {
7370            self.num_rent_exempt_accounts += 1;
7371        } else {
7372            self.num_rent_paying_accounts += 1;
7373            self.lamports_in_rent_paying_accounts += account.lamports();
7374            if data_len == 0 {
7375                self.num_rent_paying_accounts_without_data += 1;
7376            }
7377        }
7378    }
7379}
7380
7381impl Drop for Bank {
7382    fn drop(&mut self) {
7383        if let Some(drop_callback) = self.drop_callback.read().unwrap().0.as_ref() {
7384            drop_callback.callback(self);
7385        } else {
7386            // Default case for tests
7387            self.rc
7388                .accounts
7389                .accounts_db
7390                .purge_slot(self.slot(), self.bank_id(), false);
7391        }
7392    }
7393}
7394
7395/// utility function used for testing and benchmarking.
7396pub mod test_utils {
7397    use {
7398        super::Bank,
7399        crate::installed_scheduler_pool::BankWithScheduler,
7400        solana_sdk::{
7401            account::{ReadableAccount, WritableAccount},
7402            hash::hashv,
7403            lamports::LamportsError,
7404            pubkey::Pubkey,
7405        },
7406        solana_vote_program::vote_state::{self, BlockTimestamp, VoteStateVersions},
7407        std::sync::Arc,
7408    };
7409    pub fn goto_end_of_slot(bank: Arc<Bank>) {
7410        goto_end_of_slot_with_scheduler(&BankWithScheduler::new_without_scheduler(bank))
7411    }
7412
7413    pub fn goto_end_of_slot_with_scheduler(bank: &BankWithScheduler) {
7414        let mut tick_hash = bank.last_blockhash();
7415        loop {
7416            tick_hash = hashv(&[tick_hash.as_ref(), &[42]]);
7417            bank.register_tick(&tick_hash);
7418            if tick_hash == bank.last_blockhash() {
7419                bank.freeze();
7420                return;
7421            }
7422        }
7423    }
7424
7425    pub fn update_vote_account_timestamp(
7426        timestamp: BlockTimestamp,
7427        bank: &Bank,
7428        vote_pubkey: &Pubkey,
7429    ) {
7430        let mut vote_account = bank.get_account(vote_pubkey).unwrap_or_default();
7431        let mut vote_state = vote_state::from(&vote_account).unwrap_or_default();
7432        vote_state.last_timestamp = timestamp;
7433        let versioned = VoteStateVersions::new_current(vote_state);
7434        vote_state::to(&versioned, &mut vote_account).unwrap();
7435        bank.store_account(vote_pubkey, &vote_account);
7436    }
7437
7438    pub fn deposit(
7439        bank: &Bank,
7440        pubkey: &Pubkey,
7441        lamports: u64,
7442    ) -> std::result::Result<u64, LamportsError> {
7443        // This doesn't collect rents intentionally.
7444        // Rents should only be applied to actual TXes
7445        let mut account = bank
7446            .get_account_with_fixed_root_no_cache(pubkey)
7447            .unwrap_or_default();
7448        account.checked_add_lamports(lamports)?;
7449        bank.store_account(pubkey, &account);
7450        Ok(account.lamports())
7451    }
7452}