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