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        ThreadPool, 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        let thread_pool = ThreadPoolBuilder::new()
1890            .thread_name(|i| format!("solBnkNewFlds{i:02}"))
1891            .build()
1892            .expect("new rayon threadpool");
1893        bank.initialize_after_snapshot_restore(
1894            genesis_config,
1895            additional_builtins,
1896            debug_do_not_add_builtins,
1897            || &thread_pool,
1898        );
1899
1900        let mut calculate_accounts_lt_hash_duration = None;
1901        if let Some(accounts_lt_hash) = fields.accounts_lt_hash {
1902            *bank.accounts_lt_hash.get_mut().unwrap() = accounts_lt_hash;
1903        } else {
1904            // Use the accounts lt hash from the snapshot, if present, otherwise calculate it.
1905            // When the feature gate is enabled, the snapshot *must* contain an accounts lt hash.
1906            assert!(
1907                !bank
1908                    .feature_set
1909                    .is_active(&feature_set::accounts_lt_hash::id()),
1910                "snapshot must have an accounts lt hash if the feature is enabled",
1911            );
1912            if bank.is_accounts_lt_hash_enabled() {
1913                info!(
1914                    "Calculating the accounts lt hash for slot {}...",
1915                    bank.slot(),
1916                );
1917                let (ancestors, slot) = if bank.is_frozen() {
1918                    // Loading from a snapshot necessarily means this slot was rooted, and thus
1919                    // the bank has been frozen.  So when calculating the accounts lt hash,
1920                    // do it based on *this slot*, not our parent, since
1921                    // update_accounts_lt_hash() will not be called on us again.
1922                    (bank.ancestors.clone(), bank.slot())
1923                } else {
1924                    // If the bank is not frozen (e.g. if called from tests), then when this bank
1925                    // is frozen later it will call `update_accounts_lt_hash()`.  Therefore, we
1926                    // must calculate the accounts lt hash *here* based on *our parent*, so that
1927                    // the accounts lt hash is correct after freezing.
1928                    let parent_ancestors = {
1929                        let mut ancestors = bank.ancestors.clone();
1930                        ancestors.remove(&bank.slot());
1931                        ancestors
1932                    };
1933                    (parent_ancestors, bank.parent_slot)
1934                };
1935                let (accounts_lt_hash, duration) = meas_dur!({
1936                    thread_pool.install(|| {
1937                        bank.rc
1938                            .accounts
1939                            .accounts_db
1940                            .calculate_accounts_lt_hash_at_startup_from_index(&ancestors, slot)
1941                    })
1942                });
1943                calculate_accounts_lt_hash_duration = Some(duration);
1944                *bank.accounts_lt_hash.get_mut().unwrap() = accounts_lt_hash;
1945                info!(
1946                    "Calculating the accounts lt hash for slot {}... \
1947                     Done in {duration:?}, accounts_lt_hash checksum: {}",
1948                    bank.slot(),
1949                    bank.accounts_lt_hash.get_mut().unwrap().0.checksum(),
1950                );
1951            }
1952        }
1953
1954        // Sanity assertions between bank snapshot and genesis config
1955        // Consider removing from serializable bank state
1956        // (BankFieldsToSerialize/BankFieldsToDeserialize) and initializing
1957        // from the passed in genesis_config instead (as new()/new_with_paths() already do)
1958        assert_eq!(
1959            bank.genesis_creation_time, genesis_config.creation_time,
1960            "Bank snapshot genesis creation time does not match genesis.bin creation time. \
1961             The snapshot and genesis.bin might pertain to different clusters"
1962        );
1963        assert_eq!(bank.ticks_per_slot, genesis_config.ticks_per_slot);
1964        assert_eq!(
1965            bank.ns_per_slot,
1966            genesis_config.poh_config.target_tick_duration.as_nanos()
1967                * genesis_config.ticks_per_slot as u128
1968        );
1969        assert_eq!(bank.max_tick_height, (bank.slot + 1) * bank.ticks_per_slot);
1970        assert_eq!(
1971            bank.slots_per_year,
1972            years_as_slots(
1973                1.0,
1974                &genesis_config.poh_config.target_tick_duration,
1975                bank.ticks_per_slot,
1976            )
1977        );
1978        assert_eq!(bank.epoch_schedule, genesis_config.epoch_schedule);
1979        assert_eq!(bank.epoch, bank.epoch_schedule.get_epoch(bank.slot));
1980
1981        datapoint_info!(
1982            "bank-new-from-fields",
1983            (
1984                "accounts_data_len-from-snapshot",
1985                fields.accounts_data_len as i64,
1986                i64
1987            ),
1988            (
1989                "accounts_data_len-from-generate_index",
1990                accounts_data_size_initial as i64,
1991                i64
1992            ),
1993            (
1994                "stakes_accounts_load_duration_us",
1995                stakes_accounts_load_duration.as_micros(),
1996                i64
1997            ),
1998            (
1999                "calculate_accounts_lt_hash_us",
2000                calculate_accounts_lt_hash_duration.as_ref().map(Duration::as_micros),
2001                Option<i64>
2002            ),
2003        );
2004        bank
2005    }
2006
2007    /// Return subset of bank fields representing serializable state
2008    pub(crate) fn get_fields_to_serialize(&self) -> BankFieldsToSerialize {
2009        let (epoch_stakes, versioned_epoch_stakes) = split_epoch_stakes(self.epoch_stakes.clone());
2010        BankFieldsToSerialize {
2011            blockhash_queue: self.blockhash_queue.read().unwrap().clone(),
2012            ancestors: AncestorsForSerialization::from(&self.ancestors),
2013            hash: *self.hash.read().unwrap(),
2014            parent_hash: self.parent_hash,
2015            parent_slot: self.parent_slot,
2016            hard_forks: self.hard_forks.read().unwrap().clone(),
2017            transaction_count: self.transaction_count.load(Relaxed),
2018            tick_height: self.tick_height.load(Relaxed),
2019            signature_count: self.signature_count.load(Relaxed),
2020            capitalization: self.capitalization.load(Relaxed),
2021            max_tick_height: self.max_tick_height,
2022            hashes_per_tick: self.hashes_per_tick,
2023            ticks_per_slot: self.ticks_per_slot,
2024            ns_per_slot: self.ns_per_slot,
2025            genesis_creation_time: self.genesis_creation_time,
2026            slots_per_year: self.slots_per_year,
2027            slot: self.slot,
2028            epoch: self.epoch,
2029            block_height: self.block_height,
2030            collector_id: self.collector_id,
2031            collector_fees: self.collector_fees.load(Relaxed),
2032            fee_rate_governor: self.fee_rate_governor.clone(),
2033            collected_rent: self.collected_rent.load(Relaxed),
2034            rent_collector: self.rent_collector.clone(),
2035            epoch_schedule: self.epoch_schedule.clone(),
2036            inflation: *self.inflation.read().unwrap(),
2037            stakes: StakesEnum::from(self.stakes_cache.stakes().clone()),
2038            epoch_stakes,
2039            is_delta: self.is_delta.load(Relaxed),
2040            accounts_data_len: self.load_accounts_data_size(),
2041            versioned_epoch_stakes,
2042            accounts_lt_hash: self
2043                .is_accounts_lt_hash_enabled()
2044                .then(|| self.accounts_lt_hash.lock().unwrap().clone()),
2045        }
2046    }
2047
2048    pub fn collector_id(&self) -> &Pubkey {
2049        &self.collector_id
2050    }
2051
2052    pub fn genesis_creation_time(&self) -> UnixTimestamp {
2053        self.genesis_creation_time
2054    }
2055
2056    pub fn slot(&self) -> Slot {
2057        self.slot
2058    }
2059
2060    pub fn bank_id(&self) -> BankId {
2061        self.bank_id
2062    }
2063
2064    pub fn epoch(&self) -> Epoch {
2065        self.epoch
2066    }
2067
2068    pub fn first_normal_epoch(&self) -> Epoch {
2069        self.epoch_schedule().first_normal_epoch
2070    }
2071
2072    pub fn freeze_lock(&self) -> RwLockReadGuard<Hash> {
2073        self.hash.read().unwrap()
2074    }
2075
2076    pub fn hash(&self) -> Hash {
2077        *self.hash.read().unwrap()
2078    }
2079
2080    pub fn is_frozen(&self) -> bool {
2081        *self.hash.read().unwrap() != Hash::default()
2082    }
2083
2084    pub fn freeze_started(&self) -> bool {
2085        self.freeze_started.load(Relaxed)
2086    }
2087
2088    pub fn status_cache_ancestors(&self) -> Vec<u64> {
2089        let mut roots = self.status_cache.read().unwrap().roots().clone();
2090        let min = roots.iter().min().cloned().unwrap_or(0);
2091        for ancestor in self.ancestors.keys() {
2092            if ancestor >= min {
2093                roots.insert(ancestor);
2094            }
2095        }
2096
2097        let mut ancestors: Vec<_> = roots.into_iter().collect();
2098        #[allow(clippy::stable_sort_primitive)]
2099        ancestors.sort();
2100        ancestors
2101    }
2102
2103    /// computed unix_timestamp at this slot height
2104    pub fn unix_timestamp_from_genesis(&self) -> i64 {
2105        self.genesis_creation_time.saturating_add(
2106            (self.slot as u128)
2107                .saturating_mul(self.ns_per_slot)
2108                .saturating_div(1_000_000_000) as i64,
2109        )
2110    }
2111
2112    fn update_sysvar_account<F>(&self, pubkey: &Pubkey, updater: F)
2113    where
2114        F: Fn(&Option<AccountSharedData>) -> AccountSharedData,
2115    {
2116        let old_account = self.get_account_with_fixed_root(pubkey);
2117        let mut new_account = updater(&old_account);
2118
2119        // When new sysvar comes into existence (with RENT_UNADJUSTED_INITIAL_BALANCE lamports),
2120        // this code ensures that the sysvar's balance is adjusted to be rent-exempt.
2121        //
2122        // More generally, this code always re-calculates for possible sysvar data size change,
2123        // although there is no such sysvars currently.
2124        self.adjust_sysvar_balance_for_rent(&mut new_account);
2125        self.store_account_and_update_capitalization(pubkey, &new_account);
2126    }
2127
2128    fn inherit_specially_retained_account_fields(
2129        &self,
2130        old_account: &Option<AccountSharedData>,
2131    ) -> InheritableAccountFields {
2132        const RENT_UNADJUSTED_INITIAL_BALANCE: u64 = 1;
2133
2134        (
2135            old_account
2136                .as_ref()
2137                .map(|a| a.lamports())
2138                .unwrap_or(RENT_UNADJUSTED_INITIAL_BALANCE),
2139            old_account
2140                .as_ref()
2141                .map(|a| a.rent_epoch())
2142                .unwrap_or(INITIAL_RENT_EPOCH),
2143        )
2144    }
2145
2146    pub fn clock(&self) -> sysvar::clock::Clock {
2147        from_account(&self.get_account(&sysvar::clock::id()).unwrap_or_default())
2148            .unwrap_or_default()
2149    }
2150
2151    fn update_clock(&self, parent_epoch: Option<Epoch>) {
2152        let mut unix_timestamp = self.clock().unix_timestamp;
2153        // set epoch_start_timestamp to None to warp timestamp
2154        let epoch_start_timestamp = {
2155            let epoch = if let Some(epoch) = parent_epoch {
2156                epoch
2157            } else {
2158                self.epoch()
2159            };
2160            let first_slot_in_epoch = self.epoch_schedule().get_first_slot_in_epoch(epoch);
2161            Some((first_slot_in_epoch, self.clock().epoch_start_timestamp))
2162        };
2163        let max_allowable_drift = MaxAllowableDrift {
2164            fast: MAX_ALLOWABLE_DRIFT_PERCENTAGE_FAST,
2165            slow: MAX_ALLOWABLE_DRIFT_PERCENTAGE_SLOW_V2,
2166        };
2167
2168        let ancestor_timestamp = self.clock().unix_timestamp;
2169        if let Some(timestamp_estimate) =
2170            self.get_timestamp_estimate(max_allowable_drift, epoch_start_timestamp)
2171        {
2172            unix_timestamp = timestamp_estimate;
2173            if timestamp_estimate < ancestor_timestamp {
2174                unix_timestamp = ancestor_timestamp;
2175            }
2176        }
2177        datapoint_info!(
2178            "bank-timestamp-correction",
2179            ("slot", self.slot(), i64),
2180            ("from_genesis", self.unix_timestamp_from_genesis(), i64),
2181            ("corrected", unix_timestamp, i64),
2182            ("ancestor_timestamp", ancestor_timestamp, i64),
2183        );
2184        let mut epoch_start_timestamp =
2185            // On epoch boundaries, update epoch_start_timestamp
2186            if parent_epoch.is_some() && parent_epoch.unwrap() != self.epoch() {
2187                unix_timestamp
2188            } else {
2189                self.clock().epoch_start_timestamp
2190            };
2191        if self.slot == 0 {
2192            unix_timestamp = self.unix_timestamp_from_genesis();
2193            epoch_start_timestamp = self.unix_timestamp_from_genesis();
2194        }
2195        let clock = sysvar::clock::Clock {
2196            slot: self.slot,
2197            epoch_start_timestamp,
2198            epoch: self.epoch_schedule().get_epoch(self.slot),
2199            leader_schedule_epoch: self.epoch_schedule().get_leader_schedule_epoch(self.slot),
2200            unix_timestamp,
2201        };
2202        self.update_sysvar_account(&sysvar::clock::id(), |account| {
2203            create_account(
2204                &clock,
2205                self.inherit_specially_retained_account_fields(account),
2206            )
2207        });
2208    }
2209
2210    pub fn update_last_restart_slot(&self) {
2211        let feature_flag = self
2212            .feature_set
2213            .is_active(&feature_set::last_restart_slot_sysvar::id());
2214
2215        if feature_flag {
2216            // First, see what the currently stored last restart slot is. This
2217            // account may not exist yet if the feature was just activated.
2218            let current_last_restart_slot = self
2219                .get_account(&sysvar::last_restart_slot::id())
2220                .and_then(|account| {
2221                    let lrs: Option<LastRestartSlot> = from_account(&account);
2222                    lrs
2223                })
2224                .map(|account| account.last_restart_slot);
2225
2226            let last_restart_slot = {
2227                let slot = self.slot;
2228                let hard_forks_r = self.hard_forks.read().unwrap();
2229
2230                // Only consider hard forks <= this bank's slot to avoid prematurely applying
2231                // a hard fork that is set to occur in the future.
2232                hard_forks_r
2233                    .iter()
2234                    .rev()
2235                    .find(|(hard_fork, _)| *hard_fork <= slot)
2236                    .map(|(slot, _)| *slot)
2237                    .unwrap_or(0)
2238            };
2239
2240            // Only need to write if the last restart has changed
2241            if current_last_restart_slot != Some(last_restart_slot) {
2242                self.update_sysvar_account(&sysvar::last_restart_slot::id(), |account| {
2243                    create_account(
2244                        &LastRestartSlot { last_restart_slot },
2245                        self.inherit_specially_retained_account_fields(account),
2246                    )
2247                });
2248            }
2249        }
2250    }
2251
2252    pub fn set_sysvar_for_tests<T>(&self, sysvar: &T)
2253    where
2254        T: Sysvar + SysvarId,
2255    {
2256        self.update_sysvar_account(&T::id(), |account| {
2257            create_account(
2258                sysvar,
2259                self.inherit_specially_retained_account_fields(account),
2260            )
2261        });
2262        // Simply force fill sysvar cache rather than checking which sysvar was
2263        // actually updated since tests don't need to be optimized for performance.
2264        self.transaction_processor.reset_sysvar_cache();
2265        self.transaction_processor
2266            .fill_missing_sysvar_cache_entries(self);
2267    }
2268
2269    fn update_slot_history(&self) {
2270        self.update_sysvar_account(&sysvar::slot_history::id(), |account| {
2271            let mut slot_history = account
2272                .as_ref()
2273                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
2274                .unwrap_or_default();
2275            slot_history.add(self.slot());
2276            create_account(
2277                &slot_history,
2278                self.inherit_specially_retained_account_fields(account),
2279            )
2280        });
2281    }
2282
2283    fn update_slot_hashes(&self) {
2284        self.update_sysvar_account(&sysvar::slot_hashes::id(), |account| {
2285            let mut slot_hashes = account
2286                .as_ref()
2287                .map(|account| from_account::<SlotHashes, _>(account).unwrap())
2288                .unwrap_or_default();
2289            slot_hashes.add(self.parent_slot, self.parent_hash);
2290            create_account(
2291                &slot_hashes,
2292                self.inherit_specially_retained_account_fields(account),
2293            )
2294        });
2295    }
2296
2297    pub fn get_slot_history(&self) -> SlotHistory {
2298        from_account(&self.get_account(&sysvar::slot_history::id()).unwrap()).unwrap()
2299    }
2300
2301    fn update_epoch_stakes(&mut self, leader_schedule_epoch: Epoch) {
2302        // update epoch_stakes cache
2303        //  if my parent didn't populate for this staker's epoch, we've
2304        //  crossed a boundary
2305        if !self.epoch_stakes.contains_key(&leader_schedule_epoch) {
2306            self.epoch_stakes.retain(|&epoch, _| {
2307                epoch >= leader_schedule_epoch.saturating_sub(MAX_LEADER_SCHEDULE_STAKES)
2308            });
2309            let stakes = self.stakes_cache.stakes().clone();
2310            let stakes = Arc::new(StakesEnum::from(stakes));
2311            let new_epoch_stakes = EpochStakes::new(stakes, leader_schedule_epoch);
2312            info!(
2313                "new epoch stakes, epoch: {}, total_stake: {}",
2314                leader_schedule_epoch,
2315                new_epoch_stakes.total_stake(),
2316            );
2317
2318            // It is expensive to log the details of epoch stakes. Only log them at "trace"
2319            // level for debugging purpose.
2320            if log::log_enabled!(log::Level::Trace) {
2321                let vote_stakes: HashMap<_, _> = self
2322                    .stakes_cache
2323                    .stakes()
2324                    .vote_accounts()
2325                    .delegated_stakes()
2326                    .map(|(pubkey, stake)| (*pubkey, stake))
2327                    .collect();
2328                trace!("new epoch stakes, stakes: {vote_stakes:#?}");
2329            }
2330            self.epoch_stakes
2331                .insert(leader_schedule_epoch, new_epoch_stakes);
2332        }
2333    }
2334
2335    #[cfg(feature = "dev-context-only-utils")]
2336    pub fn set_epoch_stakes_for_test(&mut self, epoch: Epoch, stakes: EpochStakes) {
2337        self.epoch_stakes.insert(epoch, stakes);
2338    }
2339
2340    fn update_rent(&self) {
2341        self.update_sysvar_account(&sysvar::rent::id(), |account| {
2342            create_account(
2343                &self.rent_collector.rent,
2344                self.inherit_specially_retained_account_fields(account),
2345            )
2346        });
2347    }
2348
2349    fn update_epoch_schedule(&self) {
2350        self.update_sysvar_account(&sysvar::epoch_schedule::id(), |account| {
2351            create_account(
2352                self.epoch_schedule(),
2353                self.inherit_specially_retained_account_fields(account),
2354            )
2355        });
2356    }
2357
2358    fn update_stake_history(&self, epoch: Option<Epoch>) {
2359        if epoch == Some(self.epoch()) {
2360            return;
2361        }
2362        // if I'm the first Bank in an epoch, ensure stake_history is updated
2363        self.update_sysvar_account(&sysvar::stake_history::id(), |account| {
2364            create_account::<sysvar::stake_history::StakeHistory>(
2365                self.stakes_cache.stakes().history(),
2366                self.inherit_specially_retained_account_fields(account),
2367            )
2368        });
2369    }
2370
2371    pub fn epoch_duration_in_years(&self, prev_epoch: Epoch) -> f64 {
2372        // period: time that has passed as a fraction of a year, basically the length of
2373        //  an epoch as a fraction of a year
2374        //  calculated as: slots_elapsed / (slots / year)
2375        self.epoch_schedule().get_slots_in_epoch(prev_epoch) as f64 / self.slots_per_year
2376    }
2377
2378    // Calculates the starting-slot for inflation from the activation slot.
2379    // This method assumes that `pico_inflation` will be enabled before `full_inflation`, giving
2380    // precedence to the latter. However, since `pico_inflation` is fixed-rate Inflation, should
2381    // `pico_inflation` be enabled 2nd, the incorrect start slot provided here should have no
2382    // effect on the inflation calculation.
2383    fn get_inflation_start_slot(&self) -> Slot {
2384        let mut slots = self
2385            .feature_set
2386            .full_inflation_features_enabled()
2387            .iter()
2388            .filter_map(|id| self.feature_set.activated_slot(id))
2389            .collect::<Vec<_>>();
2390        slots.sort_unstable();
2391        slots.first().cloned().unwrap_or_else(|| {
2392            self.feature_set
2393                .activated_slot(&feature_set::pico_inflation::id())
2394                .unwrap_or(0)
2395        })
2396    }
2397
2398    fn get_inflation_num_slots(&self) -> u64 {
2399        let inflation_activation_slot = self.get_inflation_start_slot();
2400        // Normalize inflation_start to align with the start of rewards accrual.
2401        let inflation_start_slot = self.epoch_schedule().get_first_slot_in_epoch(
2402            self.epoch_schedule()
2403                .get_epoch(inflation_activation_slot)
2404                .saturating_sub(1),
2405        );
2406        self.epoch_schedule().get_first_slot_in_epoch(self.epoch()) - inflation_start_slot
2407    }
2408
2409    pub fn slot_in_year_for_inflation(&self) -> f64 {
2410        let num_slots = self.get_inflation_num_slots();
2411
2412        // calculated as: num_slots / (slots / year)
2413        num_slots as f64 / self.slots_per_year
2414    }
2415
2416    fn calculate_previous_epoch_inflation_rewards(
2417        &self,
2418        prev_epoch_capitalization: u64,
2419        prev_epoch: Epoch,
2420    ) -> PrevEpochInflationRewards {
2421        let slot_in_year = self.slot_in_year_for_inflation();
2422        let (validator_rate, foundation_rate) = {
2423            let inflation = self.inflation.read().unwrap();
2424            (
2425                (*inflation).validator(slot_in_year),
2426                (*inflation).foundation(slot_in_year),
2427            )
2428        };
2429
2430        let prev_epoch_duration_in_years = self.epoch_duration_in_years(prev_epoch);
2431        let validator_rewards = (validator_rate
2432            * prev_epoch_capitalization as f64
2433            * prev_epoch_duration_in_years) as u64;
2434
2435        PrevEpochInflationRewards {
2436            validator_rewards,
2437            prev_epoch_duration_in_years,
2438            validator_rate,
2439            foundation_rate,
2440        }
2441    }
2442
2443    fn filter_stake_delegations<'a>(
2444        &self,
2445        stakes: &'a Stakes<StakeAccount<Delegation>>,
2446    ) -> Vec<(&'a Pubkey, &'a StakeAccount<Delegation>)> {
2447        if self
2448            .feature_set
2449            .is_active(&feature_set::stake_minimum_delegation_for_rewards::id())
2450        {
2451            let num_stake_delegations = stakes.stake_delegations().len();
2452            let min_stake_delegation = solana_stake_program::get_minimum_delegation(
2453                self.feature_set
2454                    .is_active(&agave_feature_set::stake_raise_minimum_delegation_to_1_sol::id()),
2455            )
2456            .max(LAMPORTS_PER_SOL);
2457
2458            let (stake_delegations, filter_time_us) = measure_us!(stakes
2459                .stake_delegations()
2460                .iter()
2461                .filter(|(_stake_pubkey, cached_stake_account)| {
2462                    cached_stake_account.delegation().stake >= min_stake_delegation
2463                })
2464                .collect::<Vec<_>>());
2465
2466            datapoint_info!(
2467                "stake_account_filter_time",
2468                ("filter_time_us", filter_time_us, i64),
2469                ("num_stake_delegations_before", num_stake_delegations, i64),
2470                ("num_stake_delegations_after", stake_delegations.len(), i64)
2471            );
2472            stake_delegations
2473        } else {
2474            stakes.stake_delegations().iter().collect()
2475        }
2476    }
2477
2478    /// return reward info for each vote account
2479    /// return account data for each vote account that needs to be stored
2480    /// 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.
2481    /// This function is copied from the existing code path's `store_vote_accounts`.
2482    /// The primary differences:
2483    /// - we want this fn to have no side effects (such as actually storing vote accounts) so that we
2484    ///   can compare the expected results with the current code path
2485    /// - we want to be able to batch store the vote accounts later for improved performance/cache updating
2486    fn calc_vote_accounts_to_store(vote_account_rewards: VoteRewards) -> VoteRewardsAccounts {
2487        let len = vote_account_rewards.len();
2488        let mut result = VoteRewardsAccounts {
2489            rewards: Vec::with_capacity(len),
2490            accounts_to_store: Vec::with_capacity(len),
2491            total_vote_rewards_lamports: 0,
2492        };
2493        vote_account_rewards.into_iter().for_each(
2494            |(
2495                vote_pubkey,
2496                VoteReward {
2497                    mut vote_account,
2498                    commission,
2499                    vote_rewards,
2500                },
2501            )| {
2502                if let Err(err) = vote_account.checked_add_lamports(vote_rewards) {
2503                    debug!("reward redemption failed for {}: {:?}", vote_pubkey, err);
2504                    return;
2505                }
2506
2507                result.rewards.push((
2508                    vote_pubkey,
2509                    RewardInfo {
2510                        reward_type: RewardType::Voting,
2511                        lamports: vote_rewards as i64,
2512                        post_balance: vote_account.lamports(),
2513                        commission: Some(commission),
2514                    },
2515                ));
2516                result.accounts_to_store.push((vote_pubkey, vote_account));
2517                result.total_vote_rewards_lamports += vote_rewards;
2518            },
2519        );
2520        result
2521    }
2522
2523    fn update_reward_history(
2524        &self,
2525        stake_rewards: StakeRewards,
2526        vote_rewards: &[(Pubkey, RewardInfo)],
2527    ) {
2528        let additional_reserve = stake_rewards.len() + vote_rewards.len();
2529        let mut rewards = self.rewards.write().unwrap();
2530        rewards.reserve(additional_reserve);
2531        vote_rewards.iter().for_each(|(vote_pubkey, vote_reward)| {
2532            rewards.push((*vote_pubkey, *vote_reward));
2533        });
2534        stake_rewards
2535            .into_iter()
2536            .filter(|x| x.get_stake_reward() > 0)
2537            .for_each(|x| rewards.push((x.stake_pubkey, x.stake_reward_info)));
2538    }
2539
2540    fn update_recent_blockhashes_locked(&self, locked_blockhash_queue: &BlockhashQueue) {
2541        #[allow(deprecated)]
2542        self.update_sysvar_account(&sysvar::recent_blockhashes::id(), |account| {
2543            let recent_blockhash_iter = locked_blockhash_queue.get_recent_blockhashes();
2544            recent_blockhashes_account::create_account_with_data_and_fields(
2545                recent_blockhash_iter,
2546                self.inherit_specially_retained_account_fields(account),
2547            )
2548        });
2549    }
2550
2551    pub fn update_recent_blockhashes(&self) {
2552        let blockhash_queue = self.blockhash_queue.read().unwrap();
2553        self.update_recent_blockhashes_locked(&blockhash_queue);
2554    }
2555
2556    fn get_timestamp_estimate(
2557        &self,
2558        max_allowable_drift: MaxAllowableDrift,
2559        epoch_start_timestamp: Option<(Slot, UnixTimestamp)>,
2560    ) -> Option<UnixTimestamp> {
2561        let mut get_timestamp_estimate_time = Measure::start("get_timestamp_estimate");
2562        let slots_per_epoch = self.epoch_schedule().slots_per_epoch;
2563        let vote_accounts = self.vote_accounts();
2564        let recent_timestamps = vote_accounts.iter().filter_map(|(pubkey, (_, account))| {
2565            let vote_state = account.vote_state_view();
2566            let last_timestamp = vote_state.last_timestamp();
2567            let slot_delta = self.slot().checked_sub(last_timestamp.slot)?;
2568            (slot_delta <= slots_per_epoch)
2569                .then_some((*pubkey, (last_timestamp.slot, last_timestamp.timestamp)))
2570        });
2571        let slot_duration = Duration::from_nanos(self.ns_per_slot as u64);
2572        let epoch = self.epoch_schedule().get_epoch(self.slot());
2573        let stakes = self.epoch_vote_accounts(epoch)?;
2574        let stake_weighted_timestamp = calculate_stake_weighted_timestamp(
2575            recent_timestamps,
2576            stakes,
2577            self.slot(),
2578            slot_duration,
2579            epoch_start_timestamp,
2580            max_allowable_drift,
2581            self.feature_set
2582                .is_active(&feature_set::warp_timestamp_again::id()),
2583        );
2584        get_timestamp_estimate_time.stop();
2585        datapoint_info!(
2586            "bank-timestamp",
2587            (
2588                "get_timestamp_estimate_us",
2589                get_timestamp_estimate_time.as_us(),
2590                i64
2591            ),
2592        );
2593        stake_weighted_timestamp
2594    }
2595
2596    /// Recalculates the bank hash
2597    ///
2598    /// This is used by ledger-tool when creating a snapshot, which
2599    /// recalcuates the bank hash.
2600    ///
2601    /// Note that the account state is *not* allowed to change by rehashing.
2602    /// If modifying accounts in ledger-tool is needed, create a new bank.
2603    pub fn rehash(&self) {
2604        let get_delta_hash = || {
2605            (!self
2606                .feature_set
2607                .is_active(&feature_set::remove_accounts_delta_hash::id()))
2608            .then(|| {
2609                self.rc
2610                    .accounts
2611                    .accounts_db
2612                    .get_accounts_delta_hash(self.slot())
2613            })
2614            .flatten()
2615        };
2616
2617        let mut hash = self.hash.write().unwrap();
2618        let curr_accounts_delta_hash = get_delta_hash();
2619        let new = self.hash_internal_state();
2620        if let Some(curr_accounts_delta_hash) = curr_accounts_delta_hash {
2621            let new_accounts_delta_hash = get_delta_hash().unwrap();
2622            assert_eq!(
2623                new_accounts_delta_hash, curr_accounts_delta_hash,
2624                "rehashing is not allowed to change the account state",
2625            );
2626        }
2627        if new != *hash {
2628            warn!("Updating bank hash to {new}");
2629            *hash = new;
2630        }
2631    }
2632
2633    pub fn freeze(&self) {
2634        // This lock prevents any new commits from BankingStage
2635        // `Consumer::execute_and_commit_transactions_locked()` from
2636        // coming in after the last tick is observed. This is because in
2637        // BankingStage, any transaction successfully recorded in
2638        // `record_transactions()` is recorded after this `hash` lock
2639        // is grabbed. At the time of the successful record,
2640        // this means the PoH has not yet reached the last tick,
2641        // so this means freeze() hasn't been called yet. And because
2642        // BankingStage doesn't release this hash lock until both
2643        // record and commit are finished, those transactions will be
2644        // committed before this write lock can be obtained here.
2645        let mut hash = self.hash.write().unwrap();
2646        if *hash == Hash::default() {
2647            // finish up any deferred changes to account state
2648            self.collect_rent_eagerly();
2649            self.distribute_transaction_fee_details();
2650            self.distribute_rent_fees();
2651            self.update_slot_history();
2652            self.run_incinerator();
2653
2654            // freeze is a one-way trip, idempotent
2655            self.freeze_started.store(true, Relaxed);
2656            if self.is_accounts_lt_hash_enabled() {
2657                // updating the accounts lt hash must happen *outside* of hash_internal_state() so
2658                // that rehash() can be called and *not* modify self.accounts_lt_hash.
2659                self.update_accounts_lt_hash();
2660
2661                // For lattice-hash R&D, we have a CLI arg to do extra verfication.  If set, we'll
2662                // re-calculate the accounts lt hash every slot and compare it against the value
2663                // already stored in the bank.
2664                if self
2665                    .rc
2666                    .accounts
2667                    .accounts_db
2668                    .verify_experimental_accumulator_hash
2669                {
2670                    let slot = self.slot();
2671                    info!("Verifying the accounts lt hash for slot {slot}...");
2672                    let (calculated_accounts_lt_hash, duration) = meas_dur!({
2673                        self.rc
2674                            .accounts
2675                            .accounts_db
2676                            .calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, slot)
2677                    });
2678                    let actual_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap();
2679                    assert_eq!(
2680                        calculated_accounts_lt_hash,
2681                        *actual_accounts_lt_hash,
2682                        "Verifying the accounts lt hash for slot {slot} failed! calculated checksum: {}, actual checksum: {}",
2683                        calculated_accounts_lt_hash.0.checksum(),
2684                        actual_accounts_lt_hash.0.checksum(),
2685                    );
2686                    info!("Verifying the accounts lt hash for slot {slot}... Done successfully in {duration:?}");
2687                }
2688            }
2689            *hash = self.hash_internal_state();
2690            self.rc.accounts.accounts_db.mark_slot_frozen(self.slot());
2691        }
2692    }
2693
2694    // dangerous; don't use this; this is only needed for ledger-tool's special command
2695    #[cfg(feature = "dev-context-only-utils")]
2696    pub fn unfreeze_for_ledger_tool(&self) {
2697        self.freeze_started.store(false, Relaxed);
2698    }
2699
2700    pub fn epoch_schedule(&self) -> &EpochSchedule {
2701        &self.epoch_schedule
2702    }
2703
2704    /// squash the parent's state up into this Bank,
2705    ///   this Bank becomes a root
2706    /// Note that this function is not thread-safe. If it is called concurrently on the same bank
2707    /// by multiple threads, the end result could be inconsistent.
2708    /// Calling code does not currently call this concurrently.
2709    pub fn squash(&self) -> SquashTiming {
2710        self.freeze();
2711
2712        //this bank and all its parents are now on the rooted path
2713        let mut roots = vec![self.slot()];
2714        roots.append(&mut self.parents().iter().map(|p| p.slot()).collect());
2715
2716        let mut total_index_us = 0;
2717        let mut total_cache_us = 0;
2718        let mut total_store_us = 0;
2719
2720        let mut squash_accounts_time = Measure::start("squash_accounts_time");
2721        for slot in roots.iter().rev() {
2722            // root forks cannot be purged
2723            let add_root_timing = self.rc.accounts.add_root(*slot);
2724            total_index_us += add_root_timing.index_us;
2725            total_cache_us += add_root_timing.cache_us;
2726            total_store_us += add_root_timing.store_us;
2727        }
2728        squash_accounts_time.stop();
2729
2730        *self.rc.parent.write().unwrap() = None;
2731
2732        let mut squash_cache_time = Measure::start("squash_cache_time");
2733        roots
2734            .iter()
2735            .for_each(|slot| self.status_cache.write().unwrap().add_root(*slot));
2736        squash_cache_time.stop();
2737
2738        SquashTiming {
2739            squash_accounts_ms: squash_accounts_time.as_ms(),
2740            squash_accounts_index_ms: total_index_us / 1000,
2741            squash_accounts_cache_ms: total_cache_us / 1000,
2742            squash_accounts_store_ms: total_store_us / 1000,
2743
2744            squash_cache_ms: squash_cache_time.as_ms(),
2745        }
2746    }
2747
2748    /// Return the more recent checkpoint of this bank instance.
2749    pub fn parent(&self) -> Option<Arc<Bank>> {
2750        self.rc.parent.read().unwrap().clone()
2751    }
2752
2753    pub fn parent_slot(&self) -> Slot {
2754        self.parent_slot
2755    }
2756
2757    pub fn parent_hash(&self) -> Hash {
2758        self.parent_hash
2759    }
2760
2761    fn process_genesis_config(
2762        &mut self,
2763        genesis_config: &GenesisConfig,
2764        #[cfg(feature = "dev-context-only-utils")] collector_id_for_tests: Option<Pubkey>,
2765        #[cfg(feature = "dev-context-only-utils")] genesis_hash: Option<Hash>,
2766    ) {
2767        // Bootstrap validator collects fees until `new_from_parent` is called.
2768        self.fee_rate_governor = genesis_config.fee_rate_governor.clone();
2769
2770        for (pubkey, account) in genesis_config.accounts.iter() {
2771            assert!(
2772                self.get_account(pubkey).is_none(),
2773                "{pubkey} repeated in genesis config"
2774            );
2775            self.store_account(pubkey, &account.to_account_shared_data());
2776            self.capitalization.fetch_add(account.lamports(), Relaxed);
2777            self.accounts_data_size_initial += account.data().len() as u64;
2778        }
2779
2780        for (pubkey, account) in genesis_config.rewards_pools.iter() {
2781            assert!(
2782                self.get_account(pubkey).is_none(),
2783                "{pubkey} repeated in genesis config"
2784            );
2785            self.store_account(pubkey, &account.to_account_shared_data());
2786            self.accounts_data_size_initial += account.data().len() as u64;
2787        }
2788
2789        // After storing genesis accounts, the bank stakes cache will be warmed
2790        // up and can be used to set the collector id to the highest staked
2791        // node. If no staked nodes exist, allow fallback to an unstaked test
2792        // collector id during tests.
2793        let collector_id = self.stakes_cache.stakes().highest_staked_node().copied();
2794        #[cfg(feature = "dev-context-only-utils")]
2795        let collector_id = collector_id.or(collector_id_for_tests);
2796        self.collector_id =
2797            collector_id.expect("genesis processing failed because no staked nodes exist");
2798
2799        #[cfg(not(feature = "dev-context-only-utils"))]
2800        let genesis_hash = genesis_config.hash();
2801        #[cfg(feature = "dev-context-only-utils")]
2802        let genesis_hash = genesis_hash.unwrap_or(genesis_config.hash());
2803
2804        self.blockhash_queue.write().unwrap().genesis_hash(
2805            &genesis_hash,
2806            genesis_config.fee_rate_governor.lamports_per_signature,
2807        );
2808
2809        self.hashes_per_tick = genesis_config.hashes_per_tick();
2810        self.ticks_per_slot = genesis_config.ticks_per_slot();
2811        self.ns_per_slot = genesis_config.ns_per_slot();
2812        self.genesis_creation_time = genesis_config.creation_time;
2813        self.max_tick_height = (self.slot + 1) * self.ticks_per_slot;
2814        self.slots_per_year = genesis_config.slots_per_year();
2815
2816        self.epoch_schedule = genesis_config.epoch_schedule.clone();
2817
2818        self.inflation = Arc::new(RwLock::new(genesis_config.inflation));
2819
2820        self.rent_collector = RentCollector::new(
2821            self.epoch,
2822            self.epoch_schedule().clone(),
2823            self.slots_per_year,
2824            genesis_config.rent.clone(),
2825        );
2826
2827        // Add additional builtin programs specified in the genesis config
2828        for (name, program_id) in &genesis_config.native_instruction_processors {
2829            self.add_builtin_account(name, program_id);
2830        }
2831    }
2832
2833    fn burn_and_purge_account(&self, program_id: &Pubkey, mut account: AccountSharedData) {
2834        let old_data_size = account.data().len();
2835        self.capitalization.fetch_sub(account.lamports(), Relaxed);
2836        // Both resetting account balance to 0 and zeroing the account data
2837        // is needed to really purge from AccountsDb and flush the Stakes cache
2838        account.set_lamports(0);
2839        account.data_as_mut_slice().fill(0);
2840        self.store_account(program_id, &account);
2841        self.calculate_and_update_accounts_data_size_delta_off_chain(old_data_size, 0);
2842    }
2843
2844    /// Add a precompiled program account
2845    pub fn add_precompiled_account(&self, program_id: &Pubkey) {
2846        self.add_precompiled_account_with_owner(program_id, native_loader::id())
2847    }
2848
2849    // Used by tests to simulate clusters with precompiles that aren't owned by the native loader
2850    fn add_precompiled_account_with_owner(&self, program_id: &Pubkey, owner: Pubkey) {
2851        if let Some(account) = self.get_account_with_fixed_root(program_id) {
2852            if account.executable() {
2853                return;
2854            } else {
2855                // malicious account is pre-occupying at program_id
2856                self.burn_and_purge_account(program_id, account);
2857            }
2858        };
2859
2860        assert!(
2861            !self.freeze_started(),
2862            "Can't change frozen bank by adding not-existing new precompiled program ({program_id}). \
2863                Maybe, inconsistent program activation is detected on snapshot restore?"
2864        );
2865
2866        // Add a bogus executable account, which will be loaded and ignored.
2867        let (lamports, rent_epoch) = self.inherit_specially_retained_account_fields(&None);
2868
2869        let account = AccountSharedData::from(Account {
2870            lamports,
2871            owner,
2872            data: vec![],
2873            executable: true,
2874            rent_epoch,
2875        });
2876        self.store_account_and_update_capitalization(program_id, &account);
2877    }
2878
2879    pub fn set_rent_burn_percentage(&mut self, burn_percent: u8) {
2880        self.rent_collector.rent.burn_percent = burn_percent;
2881    }
2882
2883    pub fn set_hashes_per_tick(&mut self, hashes_per_tick: Option<u64>) {
2884        self.hashes_per_tick = hashes_per_tick;
2885    }
2886
2887    /// Return the last block hash registered.
2888    pub fn last_blockhash(&self) -> Hash {
2889        self.blockhash_queue.read().unwrap().last_hash()
2890    }
2891
2892    pub fn last_blockhash_and_lamports_per_signature(&self) -> (Hash, u64) {
2893        let blockhash_queue = self.blockhash_queue.read().unwrap();
2894        let last_hash = blockhash_queue.last_hash();
2895        let last_lamports_per_signature = blockhash_queue
2896            .get_lamports_per_signature(&last_hash)
2897            .unwrap(); // safe so long as the BlockhashQueue is consistent
2898        (last_hash, last_lamports_per_signature)
2899    }
2900
2901    pub fn is_blockhash_valid(&self, hash: &Hash) -> bool {
2902        let blockhash_queue = self.blockhash_queue.read().unwrap();
2903        blockhash_queue.is_hash_valid_for_age(hash, MAX_PROCESSING_AGE)
2904    }
2905
2906    pub fn get_minimum_balance_for_rent_exemption(&self, data_len: usize) -> u64 {
2907        self.rent_collector.rent.minimum_balance(data_len).max(1)
2908    }
2909
2910    pub fn get_lamports_per_signature(&self) -> u64 {
2911        self.fee_rate_governor.lamports_per_signature
2912    }
2913
2914    pub fn get_lamports_per_signature_for_blockhash(&self, hash: &Hash) -> Option<u64> {
2915        let blockhash_queue = self.blockhash_queue.read().unwrap();
2916        blockhash_queue.get_lamports_per_signature(hash)
2917    }
2918
2919    pub fn get_fee_for_message(&self, message: &SanitizedMessage) -> Option<u64> {
2920        let lamports_per_signature = {
2921            let blockhash_queue = self.blockhash_queue.read().unwrap();
2922            blockhash_queue.get_lamports_per_signature(message.recent_blockhash())
2923        }
2924        .or_else(|| {
2925            self.load_message_nonce_account(message).map(
2926                |(_nonce_address, _nonce_account, nonce_data)| {
2927                    nonce_data.get_lamports_per_signature()
2928                },
2929            )
2930        })?;
2931        Some(self.get_fee_for_message_with_lamports_per_signature(message, lamports_per_signature))
2932    }
2933
2934    /// Returns true when startup accounts hash verification has completed or never had to run in background.
2935    pub fn get_startup_verification_complete(&self) -> &Arc<AtomicBool> {
2936        &self
2937            .rc
2938            .accounts
2939            .accounts_db
2940            .verify_accounts_hash_in_bg
2941            .verified
2942    }
2943
2944    /// return true if bg hash verification is complete
2945    /// return false if bg hash verification has not completed yet
2946    /// if hash verification failed, a panic will occur
2947    pub fn is_startup_verification_complete(&self) -> bool {
2948        self.has_initial_accounts_hash_verification_completed()
2949    }
2950
2951    /// This can occur because it completed in the background
2952    /// or if the verification was run in the foreground.
2953    pub fn set_startup_verification_complete(&self) {
2954        self.set_initial_accounts_hash_verification_completed();
2955    }
2956
2957    pub fn get_fee_for_message_with_lamports_per_signature(
2958        &self,
2959        message: &impl SVMMessage,
2960        lamports_per_signature: u64,
2961    ) -> u64 {
2962        let fee_budget_limits = FeeBudgetLimits::from(
2963            process_compute_budget_instructions(
2964                message.program_instructions_iter(),
2965                &self.feature_set,
2966            )
2967            .unwrap_or_default(),
2968        );
2969        solana_fee::calculate_fee(
2970            message,
2971            lamports_per_signature == 0,
2972            self.fee_structure().lamports_per_signature,
2973            fee_budget_limits.prioritization_fee,
2974            FeeFeatures::from(self.feature_set.as_ref()),
2975        )
2976    }
2977
2978    pub fn get_blockhash_last_valid_block_height(&self, blockhash: &Hash) -> Option<Slot> {
2979        let blockhash_queue = self.blockhash_queue.read().unwrap();
2980        // This calculation will need to be updated to consider epoch boundaries if BlockhashQueue
2981        // length is made variable by epoch
2982        blockhash_queue
2983            .get_hash_age(blockhash)
2984            .map(|age| self.block_height + MAX_PROCESSING_AGE as u64 - age)
2985    }
2986
2987    pub fn confirmed_last_blockhash(&self) -> Hash {
2988        const NUM_BLOCKHASH_CONFIRMATIONS: usize = 3;
2989
2990        let parents = self.parents();
2991        if parents.is_empty() {
2992            self.last_blockhash()
2993        } else {
2994            let index = NUM_BLOCKHASH_CONFIRMATIONS.min(parents.len() - 1);
2995            parents[index].last_blockhash()
2996        }
2997    }
2998
2999    /// Forget all signatures. Useful for benchmarking.
3000    pub fn clear_signatures(&self) {
3001        self.status_cache.write().unwrap().clear();
3002    }
3003
3004    pub fn clear_slot_signatures(&self, slot: Slot) {
3005        self.status_cache.write().unwrap().clear_slot_entries(slot);
3006    }
3007
3008    fn update_transaction_statuses(
3009        &self,
3010        sanitized_txs: &[impl TransactionWithMeta],
3011        processing_results: &[TransactionProcessingResult],
3012    ) {
3013        let mut status_cache = self.status_cache.write().unwrap();
3014        assert_eq!(sanitized_txs.len(), processing_results.len());
3015        for (tx, processing_result) in sanitized_txs.iter().zip(processing_results) {
3016            if let Ok(processed_tx) = &processing_result {
3017                // Add the message hash to the status cache to ensure that this message
3018                // won't be processed again with a different signature.
3019                status_cache.insert(
3020                    tx.recent_blockhash(),
3021                    tx.message_hash(),
3022                    self.slot(),
3023                    processed_tx.status(),
3024                );
3025                // Add the transaction signature to the status cache so that transaction status
3026                // can be queried by transaction signature over RPC. In the future, this should
3027                // only be added for API nodes because voting validators don't need to do this.
3028                status_cache.insert(
3029                    tx.recent_blockhash(),
3030                    tx.signature(),
3031                    self.slot(),
3032                    processed_tx.status(),
3033                );
3034            }
3035        }
3036    }
3037
3038    /// Register a new recent blockhash in the bank's recent blockhash queue. Called when a bank
3039    /// reaches its max tick height. Can be called by tests to get new blockhashes for transaction
3040    /// processing without advancing to a new bank slot.
3041    fn register_recent_blockhash(&self, blockhash: &Hash, scheduler: &InstalledSchedulerRwLock) {
3042        // This is needed because recent_blockhash updates necessitate synchronizations for
3043        // consistent tx check_age handling.
3044        BankWithScheduler::wait_for_paused_scheduler(self, scheduler);
3045
3046        // Only acquire the write lock for the blockhash queue on block boundaries because
3047        // readers can starve this write lock acquisition and ticks would be slowed down too
3048        // much if the write lock is acquired for each tick.
3049        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
3050
3051        #[cfg(feature = "dev-context-only-utils")]
3052        let blockhash_override = self
3053            .hash_overrides
3054            .lock()
3055            .unwrap()
3056            .get_blockhash_override(self.slot())
3057            .copied()
3058            .inspect(|blockhash_override| {
3059                if blockhash_override != blockhash {
3060                    info!(
3061                        "bank: slot: {}: overrode blockhash: {} with {}",
3062                        self.slot(),
3063                        blockhash,
3064                        blockhash_override
3065                    );
3066                }
3067            });
3068        #[cfg(feature = "dev-context-only-utils")]
3069        let blockhash = blockhash_override.as_ref().unwrap_or(blockhash);
3070
3071        w_blockhash_queue.register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
3072        self.update_recent_blockhashes_locked(&w_blockhash_queue);
3073    }
3074
3075    // gating this under #[cfg(feature = "dev-context-only-utils")] isn't easy due to
3076    // solana-program-test's usage...
3077    pub fn register_unique_recent_blockhash_for_test(&self) {
3078        self.register_recent_blockhash(
3079            &Hash::new_unique(),
3080            &BankWithScheduler::no_scheduler_available(),
3081        )
3082    }
3083
3084    #[cfg(feature = "dev-context-only-utils")]
3085    pub fn register_recent_blockhash_for_test(
3086        &self,
3087        blockhash: &Hash,
3088        lamports_per_signature: Option<u64>,
3089    ) {
3090        // Only acquire the write lock for the blockhash queue on block boundaries because
3091        // readers can starve this write lock acquisition and ticks would be slowed down too
3092        // much if the write lock is acquired for each tick.
3093        let mut w_blockhash_queue = self.blockhash_queue.write().unwrap();
3094        if let Some(lamports_per_signature) = lamports_per_signature {
3095            w_blockhash_queue.register_hash(blockhash, lamports_per_signature);
3096        } else {
3097            w_blockhash_queue
3098                .register_hash(blockhash, self.fee_rate_governor.lamports_per_signature);
3099        }
3100    }
3101
3102    /// Tell the bank which Entry IDs exist on the ledger. This function assumes subsequent calls
3103    /// correspond to later entries, and will boot the oldest ones once its internal cache is full.
3104    /// Once boot, the bank will reject transactions using that `hash`.
3105    ///
3106    /// This is NOT thread safe because if tick height is updated by two different threads, the
3107    /// block boundary condition could be missed.
3108    pub fn register_tick(&self, hash: &Hash, scheduler: &InstalledSchedulerRwLock) {
3109        assert!(
3110            !self.freeze_started(),
3111            "register_tick() working on a bank that is already frozen or is undergoing freezing!"
3112        );
3113
3114        if self.is_block_boundary(self.tick_height.load(Relaxed) + 1) {
3115            self.register_recent_blockhash(hash, scheduler);
3116        }
3117
3118        // ReplayStage will start computing the accounts delta hash when it
3119        // detects the tick height has reached the boundary, so the system
3120        // needs to guarantee all account updates for the slot have been
3121        // committed before this tick height is incremented (like the blockhash
3122        // sysvar above)
3123        self.tick_height.fetch_add(1, Relaxed);
3124    }
3125
3126    #[cfg(feature = "dev-context-only-utils")]
3127    pub fn register_tick_for_test(&self, hash: &Hash) {
3128        self.register_tick(hash, &BankWithScheduler::no_scheduler_available())
3129    }
3130
3131    #[cfg(feature = "dev-context-only-utils")]
3132    pub fn register_default_tick_for_test(&self) {
3133        self.register_tick_for_test(&Hash::default())
3134    }
3135
3136    pub fn is_complete(&self) -> bool {
3137        self.tick_height() == self.max_tick_height()
3138    }
3139
3140    pub fn is_block_boundary(&self, tick_height: u64) -> bool {
3141        tick_height == self.max_tick_height
3142    }
3143
3144    /// Get the max number of accounts that a transaction may lock in this block
3145    pub fn get_transaction_account_lock_limit(&self) -> usize {
3146        if let Some(transaction_account_lock_limit) = self.transaction_account_lock_limit {
3147            transaction_account_lock_limit
3148        } else if self
3149            .feature_set
3150            .is_active(&feature_set::increase_tx_account_lock_limit::id())
3151        {
3152            MAX_TX_ACCOUNT_LOCKS
3153        } else {
3154            64
3155        }
3156    }
3157
3158    /// Prepare a transaction batch from a list of versioned transactions from
3159    /// an entry. Used for tests only.
3160    pub fn prepare_entry_batch(
3161        &self,
3162        txs: Vec<VersionedTransaction>,
3163    ) -> Result<TransactionBatch<RuntimeTransaction<SanitizedTransaction>>> {
3164        let sanitized_txs = txs
3165            .into_iter()
3166            .map(|tx| {
3167                RuntimeTransaction::try_create(
3168                    tx,
3169                    MessageHash::Compute,
3170                    None,
3171                    self,
3172                    self.get_reserved_account_keys(),
3173                )
3174            })
3175            .collect::<Result<Vec<_>>>()?;
3176        Ok(TransactionBatch::new(
3177            self.try_lock_accounts(&sanitized_txs),
3178            self,
3179            OwnedOrBorrowed::Owned(sanitized_txs),
3180        ))
3181    }
3182
3183    /// Attempt to take locks on the accounts in a transaction batch
3184    pub fn try_lock_accounts(&self, txs: &[impl TransactionWithMeta]) -> Vec<Result<()>> {
3185        self.try_lock_accounts_with_results(txs, txs.iter().map(|_| Ok(())))
3186    }
3187
3188    /// Attempt to take locks on the accounts in a transaction batch, and their cost
3189    /// limited packing status and duplicate transaction conflict status
3190    pub fn try_lock_accounts_with_results(
3191        &self,
3192        txs: &[impl TransactionWithMeta],
3193        tx_results: impl Iterator<Item = Result<()>>,
3194    ) -> Vec<Result<()>> {
3195        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3196        let relax_intrabatch_account_locks = self
3197            .feature_set
3198            .is_active(&feature_set::relax_intrabatch_account_locks::id());
3199
3200        // with simd83 enabled, we must fail transactions that duplicate a prior message hash
3201        // previously, conflicting account locks would fail such transactions as a side effect
3202        let mut batch_message_hashes = AHashSet::with_capacity(txs.len());
3203        let tx_results = tx_results
3204            .enumerate()
3205            .map(|(i, tx_result)| match tx_result {
3206                Ok(()) if relax_intrabatch_account_locks => {
3207                    // `HashSet::insert()` returns `true` when the value does *not* already exist
3208                    if batch_message_hashes.insert(txs[i].message_hash()) {
3209                        Ok(())
3210                    } else {
3211                        Err(TransactionError::AlreadyProcessed)
3212                    }
3213                }
3214                Ok(()) => Ok(()),
3215                Err(e) => Err(e),
3216            });
3217
3218        self.rc.accounts.lock_accounts(
3219            txs.iter(),
3220            tx_results,
3221            tx_account_lock_limit,
3222            relax_intrabatch_account_locks,
3223        )
3224    }
3225
3226    /// Prepare a locked transaction batch from a list of sanitized transactions.
3227    pub fn prepare_sanitized_batch<'a, 'b, Tx: TransactionWithMeta>(
3228        &'a self,
3229        txs: &'b [Tx],
3230    ) -> TransactionBatch<'a, 'b, Tx> {
3231        self.prepare_sanitized_batch_with_results(txs, txs.iter().map(|_| Ok(())))
3232    }
3233
3234    /// Prepare a locked transaction batch from a list of sanitized transactions, and their cost
3235    /// limited packing status
3236    pub fn prepare_sanitized_batch_with_results<'a, 'b, Tx: TransactionWithMeta>(
3237        &'a self,
3238        transactions: &'b [Tx],
3239        transaction_results: impl Iterator<Item = Result<()>>,
3240    ) -> TransactionBatch<'a, 'b, Tx> {
3241        // this lock_results could be: Ok, AccountInUse, WouldExceedBlockMaxLimit or WouldExceedAccountMaxLimit
3242        TransactionBatch::new(
3243            self.try_lock_accounts_with_results(transactions, transaction_results),
3244            self,
3245            OwnedOrBorrowed::Borrowed(transactions),
3246        )
3247    }
3248
3249    /// Prepare a transaction batch from a single transaction without locking accounts
3250    pub fn prepare_unlocked_batch_from_single_tx<'a, Tx: SVMMessage>(
3251        &'a self,
3252        transaction: &'a Tx,
3253    ) -> TransactionBatch<'a, 'a, Tx> {
3254        let tx_account_lock_limit = self.get_transaction_account_lock_limit();
3255        let lock_result = validate_account_locks(transaction.account_keys(), tx_account_lock_limit);
3256        let mut batch = TransactionBatch::new(
3257            vec![lock_result],
3258            self,
3259            OwnedOrBorrowed::Borrowed(slice::from_ref(transaction)),
3260        );
3261        batch.set_needs_unlock(false);
3262        batch
3263    }
3264
3265    /// Prepare a transaction batch from a single transaction after locking accounts
3266    pub fn prepare_locked_batch_from_single_tx<'a, Tx: TransactionWithMeta>(
3267        &'a self,
3268        transaction: &'a Tx,
3269    ) -> TransactionBatch<'a, 'a, Tx> {
3270        self.prepare_sanitized_batch(slice::from_ref(transaction))
3271    }
3272
3273    /// Run transactions against a frozen bank without committing the results
3274    pub fn simulate_transaction(
3275        &self,
3276        transaction: &impl TransactionWithMeta,
3277        enable_cpi_recording: bool,
3278    ) -> TransactionSimulationResult {
3279        assert!(self.is_frozen(), "simulation bank must be frozen");
3280
3281        self.simulate_transaction_unchecked(transaction, enable_cpi_recording)
3282    }
3283
3284    /// Run transactions against a bank without committing the results; does not check if the bank
3285    /// is frozen, enabling use in single-Bank test frameworks
3286    pub fn simulate_transaction_unchecked(
3287        &self,
3288        transaction: &impl TransactionWithMeta,
3289        enable_cpi_recording: bool,
3290    ) -> TransactionSimulationResult {
3291        let account_keys = transaction.account_keys();
3292        let number_of_accounts = account_keys.len();
3293        let account_overrides = self.get_account_overrides_for_simulation(&account_keys);
3294        let batch = self.prepare_unlocked_batch_from_single_tx(transaction);
3295        let mut timings = ExecuteTimings::default();
3296
3297        let LoadAndExecuteTransactionsOutput {
3298            mut processing_results,
3299            ..
3300        } = self.load_and_execute_transactions(
3301            &batch,
3302            // After simulation, transactions will need to be forwarded to the leader
3303            // for processing. During forwarding, the transaction could expire if the
3304            // delay is not accounted for.
3305            MAX_PROCESSING_AGE - MAX_TRANSACTION_FORWARDING_DELAY,
3306            &mut timings,
3307            &mut TransactionErrorMetrics::default(),
3308            TransactionProcessingConfig {
3309                account_overrides: Some(&account_overrides),
3310                check_program_modification_slot: self.check_program_modification_slot,
3311                log_messages_bytes_limit: None,
3312                limit_to_load_programs: true,
3313                recording_config: ExecutionRecordingConfig {
3314                    enable_cpi_recording,
3315                    enable_log_recording: true,
3316                    enable_return_data_recording: true,
3317                    enable_transaction_balance_recording: false,
3318                },
3319            },
3320        );
3321
3322        debug!("simulate_transaction: {:?}", timings);
3323
3324        let processing_result = processing_results
3325            .pop()
3326            .unwrap_or(Err(TransactionError::InvalidProgramForExecution));
3327        let (
3328            post_simulation_accounts,
3329            result,
3330            logs,
3331            return_data,
3332            inner_instructions,
3333            units_consumed,
3334            loaded_accounts_data_size,
3335        ) = match processing_result {
3336            Ok(processed_tx) => match processed_tx {
3337                ProcessedTransaction::Executed(executed_tx) => {
3338                    let details = executed_tx.execution_details;
3339                    let post_simulation_accounts = executed_tx
3340                        .loaded_transaction
3341                        .accounts
3342                        .into_iter()
3343                        .take(number_of_accounts)
3344                        .collect::<Vec<_>>();
3345                    (
3346                        post_simulation_accounts,
3347                        details.status,
3348                        details.log_messages,
3349                        details.return_data,
3350                        details.inner_instructions,
3351                        details.executed_units,
3352                        executed_tx.loaded_transaction.loaded_accounts_data_size,
3353                    )
3354                }
3355                ProcessedTransaction::FeesOnly(fees_only_tx) => (
3356                    vec![],
3357                    Err(fees_only_tx.load_error),
3358                    None,
3359                    None,
3360                    None,
3361                    0,
3362                    fees_only_tx.rollback_accounts.data_size() as u32,
3363                ),
3364            },
3365            Err(error) => (vec![], Err(error), None, None, None, 0, 0),
3366        };
3367        let logs = logs.unwrap_or_default();
3368
3369        TransactionSimulationResult {
3370            result,
3371            logs,
3372            post_simulation_accounts,
3373            units_consumed,
3374            loaded_accounts_data_size,
3375            return_data,
3376            inner_instructions,
3377        }
3378    }
3379
3380    fn get_account_overrides_for_simulation(&self, account_keys: &AccountKeys) -> AccountOverrides {
3381        let mut account_overrides = AccountOverrides::default();
3382        let slot_history_id = sysvar::slot_history::id();
3383        if account_keys.iter().any(|pubkey| *pubkey == slot_history_id) {
3384            let current_account = self.get_account_with_fixed_root(&slot_history_id);
3385            let slot_history = current_account
3386                .as_ref()
3387                .map(|account| from_account::<SlotHistory, _>(account).unwrap())
3388                .unwrap_or_default();
3389            if slot_history.check(self.slot()) == Check::Found {
3390                let ancestors = Ancestors::from(self.proper_ancestors().collect::<Vec<_>>());
3391                if let Some((account, _)) =
3392                    self.load_slow_with_fixed_root(&ancestors, &slot_history_id)
3393                {
3394                    account_overrides.set_slot_history(Some(account));
3395                }
3396            }
3397        }
3398        account_overrides
3399    }
3400
3401    pub fn unlock_accounts<'a, Tx: SVMMessage + 'a>(
3402        &self,
3403        txs_and_results: impl Iterator<Item = (&'a Tx, &'a Result<()>)> + Clone,
3404    ) {
3405        self.rc.accounts.unlock_accounts(txs_and_results)
3406    }
3407
3408    pub fn remove_unrooted_slots(&self, slots: &[(Slot, BankId)]) {
3409        self.rc.accounts.accounts_db.remove_unrooted_slots(slots)
3410    }
3411
3412    pub fn get_hash_age(&self, hash: &Hash) -> Option<u64> {
3413        self.blockhash_queue.read().unwrap().get_hash_age(hash)
3414    }
3415
3416    pub fn is_hash_valid_for_age(&self, hash: &Hash, max_age: usize) -> bool {
3417        self.blockhash_queue
3418            .read()
3419            .unwrap()
3420            .is_hash_valid_for_age(hash, max_age)
3421    }
3422
3423    pub fn collect_balances(
3424        &self,
3425        batch: &TransactionBatch<impl SVMMessage>,
3426    ) -> TransactionBalances {
3427        let mut balances: TransactionBalances = vec![];
3428        for transaction in batch.sanitized_transactions() {
3429            let mut transaction_balances: Vec<u64> = vec![];
3430            for account_key in transaction.account_keys().iter() {
3431                transaction_balances.push(self.get_balance(account_key));
3432            }
3433            balances.push(transaction_balances);
3434        }
3435        balances
3436    }
3437
3438    pub fn load_and_execute_transactions(
3439        &self,
3440        batch: &TransactionBatch<impl TransactionWithMeta>,
3441        max_age: usize,
3442        timings: &mut ExecuteTimings,
3443        error_counters: &mut TransactionErrorMetrics,
3444        processing_config: TransactionProcessingConfig,
3445    ) -> LoadAndExecuteTransactionsOutput {
3446        let sanitized_txs = batch.sanitized_transactions();
3447
3448        let (check_results, check_us) = measure_us!(self.check_transactions(
3449            sanitized_txs,
3450            batch.lock_results(),
3451            max_age,
3452            error_counters,
3453        ));
3454        timings.saturating_add_in_place(ExecuteTimingType::CheckUs, check_us);
3455
3456        let (blockhash, blockhash_lamports_per_signature) =
3457            self.last_blockhash_and_lamports_per_signature();
3458        let rent_collector_with_metrics =
3459            RentCollectorWithMetrics::new(self.rent_collector.clone());
3460        let processing_environment = TransactionProcessingEnvironment {
3461            blockhash,
3462            blockhash_lamports_per_signature,
3463            epoch_total_stake: self.get_current_epoch_total_stake(),
3464            feature_set: self.feature_set.runtime_features(),
3465            rent_collector: Some(&rent_collector_with_metrics),
3466        };
3467
3468        let sanitized_output = self
3469            .transaction_processor
3470            .load_and_execute_sanitized_transactions(
3471                self,
3472                sanitized_txs,
3473                check_results,
3474                &processing_environment,
3475                &processing_config,
3476            );
3477
3478        // Accumulate the errors returned by the batch processor.
3479        error_counters.accumulate(&sanitized_output.error_metrics);
3480
3481        // Accumulate the transaction batch execution timings.
3482        timings.accumulate(&sanitized_output.execute_timings);
3483
3484        let ((), collect_logs_us) =
3485            measure_us!(self.collect_logs(sanitized_txs, &sanitized_output.processing_results));
3486        timings.saturating_add_in_place(ExecuteTimingType::CollectLogsUs, collect_logs_us);
3487
3488        let mut processed_counts = ProcessedTransactionCounts::default();
3489        let err_count = &mut error_counters.total;
3490
3491        for (processing_result, tx) in sanitized_output
3492            .processing_results
3493            .iter()
3494            .zip(sanitized_txs)
3495        {
3496            if let Some(debug_keys) = &self.transaction_debug_keys {
3497                for key in tx.account_keys().iter() {
3498                    if debug_keys.contains(key) {
3499                        let result = processing_result.flattened_result();
3500                        info!("slot: {} result: {:?} tx: {:?}", self.slot, result, tx);
3501                        break;
3502                    }
3503                }
3504            }
3505
3506            if processing_result.was_processed() {
3507                // Signature count must be accumulated only if the transaction
3508                // is processed, otherwise a mismatched count between banking
3509                // and replay could occur
3510                processed_counts.signature_count +=
3511                    tx.signature_details().num_transaction_signatures();
3512                processed_counts.processed_transactions_count += 1;
3513
3514                if !tx.is_simple_vote_transaction() {
3515                    processed_counts.processed_non_vote_transactions_count += 1;
3516                }
3517            }
3518
3519            match processing_result.flattened_result() {
3520                Ok(()) => {
3521                    processed_counts.processed_with_successful_result_count += 1;
3522                }
3523                Err(err) => {
3524                    if err_count.0 == 0 {
3525                        debug!("tx error: {:?} {:?}", err, tx);
3526                    }
3527                    *err_count += 1;
3528                }
3529            }
3530        }
3531
3532        LoadAndExecuteTransactionsOutput {
3533            processing_results: sanitized_output.processing_results,
3534            processed_counts,
3535            balance_collector: sanitized_output.balance_collector,
3536        }
3537    }
3538
3539    fn collect_logs(
3540        &self,
3541        transactions: &[impl TransactionWithMeta],
3542        processing_results: &[TransactionProcessingResult],
3543    ) {
3544        let transaction_log_collector_config =
3545            self.transaction_log_collector_config.read().unwrap();
3546        if transaction_log_collector_config.filter == TransactionLogCollectorFilter::None {
3547            return;
3548        }
3549
3550        let collected_logs: Vec<_> = processing_results
3551            .iter()
3552            .zip(transactions)
3553            .filter_map(|(processing_result, transaction)| {
3554                // Skip log collection for unprocessed transactions
3555                let processed_tx = processing_result.processed_transaction()?;
3556                // Skip log collection for unexecuted transactions
3557                let execution_details = processed_tx.execution_details()?;
3558                Self::collect_transaction_logs(
3559                    &transaction_log_collector_config,
3560                    transaction,
3561                    execution_details,
3562                )
3563            })
3564            .collect();
3565
3566        if !collected_logs.is_empty() {
3567            let mut transaction_log_collector = self.transaction_log_collector.write().unwrap();
3568            for (log, filtered_mentioned_addresses) in collected_logs {
3569                let transaction_log_index = transaction_log_collector.logs.len();
3570                transaction_log_collector.logs.push(log);
3571                for key in filtered_mentioned_addresses.into_iter() {
3572                    transaction_log_collector
3573                        .mentioned_address_map
3574                        .entry(key)
3575                        .or_default()
3576                        .push(transaction_log_index);
3577                }
3578            }
3579        }
3580    }
3581
3582    fn collect_transaction_logs(
3583        transaction_log_collector_config: &TransactionLogCollectorConfig,
3584        transaction: &impl TransactionWithMeta,
3585        execution_details: &TransactionExecutionDetails,
3586    ) -> Option<(TransactionLogInfo, Vec<Pubkey>)> {
3587        // Skip log collection if no log messages were recorded
3588        let log_messages = execution_details.log_messages.as_ref()?;
3589
3590        let mut filtered_mentioned_addresses = Vec::new();
3591        if !transaction_log_collector_config
3592            .mentioned_addresses
3593            .is_empty()
3594        {
3595            for key in transaction.account_keys().iter() {
3596                if transaction_log_collector_config
3597                    .mentioned_addresses
3598                    .contains(key)
3599                {
3600                    filtered_mentioned_addresses.push(*key);
3601                }
3602            }
3603        }
3604
3605        let is_vote = transaction.is_simple_vote_transaction();
3606        let store = match transaction_log_collector_config.filter {
3607            TransactionLogCollectorFilter::All => {
3608                !is_vote || !filtered_mentioned_addresses.is_empty()
3609            }
3610            TransactionLogCollectorFilter::AllWithVotes => true,
3611            TransactionLogCollectorFilter::None => false,
3612            TransactionLogCollectorFilter::OnlyMentionedAddresses => {
3613                !filtered_mentioned_addresses.is_empty()
3614            }
3615        };
3616
3617        if store {
3618            Some((
3619                TransactionLogInfo {
3620                    signature: *transaction.signature(),
3621                    result: execution_details.status.clone(),
3622                    is_vote,
3623                    log_messages: log_messages.clone(),
3624                },
3625                filtered_mentioned_addresses,
3626            ))
3627        } else {
3628            None
3629        }
3630    }
3631
3632    /// Load the accounts data size, in bytes
3633    pub fn load_accounts_data_size(&self) -> u64 {
3634        self.accounts_data_size_initial
3635            .saturating_add_signed(self.load_accounts_data_size_delta())
3636    }
3637
3638    /// Load the change in accounts data size in this Bank, in bytes
3639    pub fn load_accounts_data_size_delta(&self) -> i64 {
3640        let delta_on_chain = self.load_accounts_data_size_delta_on_chain();
3641        let delta_off_chain = self.load_accounts_data_size_delta_off_chain();
3642        delta_on_chain.saturating_add(delta_off_chain)
3643    }
3644
3645    /// Load the change in accounts data size in this Bank, in bytes, from on-chain events
3646    /// i.e. transactions
3647    pub fn load_accounts_data_size_delta_on_chain(&self) -> i64 {
3648        self.accounts_data_size_delta_on_chain.load(Acquire)
3649    }
3650
3651    /// Load the change in accounts data size in this Bank, in bytes, from off-chain events
3652    /// i.e. rent collection
3653    pub fn load_accounts_data_size_delta_off_chain(&self) -> i64 {
3654        self.accounts_data_size_delta_off_chain.load(Acquire)
3655    }
3656
3657    /// Update the accounts data size delta from on-chain events by adding `amount`.
3658    /// The arithmetic saturates.
3659    fn update_accounts_data_size_delta_on_chain(&self, amount: i64) {
3660        if amount == 0 {
3661            return;
3662        }
3663
3664        self.accounts_data_size_delta_on_chain
3665            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_on_chain| {
3666                Some(accounts_data_size_delta_on_chain.saturating_add(amount))
3667            })
3668            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3669            .unwrap();
3670    }
3671
3672    /// Update the accounts data size delta from off-chain events by adding `amount`.
3673    /// The arithmetic saturates.
3674    fn update_accounts_data_size_delta_off_chain(&self, amount: i64) {
3675        if amount == 0 {
3676            return;
3677        }
3678
3679        self.accounts_data_size_delta_off_chain
3680            .fetch_update(AcqRel, Acquire, |accounts_data_size_delta_off_chain| {
3681                Some(accounts_data_size_delta_off_chain.saturating_add(amount))
3682            })
3683            // SAFETY: unwrap() is safe since our update fn always returns `Some`
3684            .unwrap();
3685    }
3686
3687    /// Calculate the data size delta and update the off-chain accounts data size delta
3688    fn calculate_and_update_accounts_data_size_delta_off_chain(
3689        &self,
3690        old_data_size: usize,
3691        new_data_size: usize,
3692    ) {
3693        let data_size_delta = calculate_data_size_delta(old_data_size, new_data_size);
3694        self.update_accounts_data_size_delta_off_chain(data_size_delta);
3695    }
3696
3697    fn filter_program_errors_and_collect_fee_details(
3698        &self,
3699        processing_results: &[TransactionProcessingResult],
3700    ) {
3701        let mut accumulated_fee_details = FeeDetails::default();
3702
3703        processing_results.iter().for_each(|processing_result| {
3704            if let Ok(processed_tx) = processing_result {
3705                accumulated_fee_details.accumulate(&processed_tx.fee_details());
3706            }
3707        });
3708
3709        self.collector_fee_details
3710            .write()
3711            .unwrap()
3712            .accumulate(&accumulated_fee_details);
3713    }
3714
3715    fn update_bank_hash_stats<'a>(&self, accounts: &impl StorableAccounts<'a>) {
3716        let mut stats = BankHashStats::default();
3717        (0..accounts.len()).for_each(|i| {
3718            accounts.account(i, |account| {
3719                stats.update(&account);
3720            })
3721        });
3722        self.bank_hash_stats.accumulate(&stats);
3723    }
3724
3725    pub fn commit_transactions(
3726        &self,
3727        sanitized_txs: &[impl TransactionWithMeta],
3728        processing_results: Vec<TransactionProcessingResult>,
3729        processed_counts: &ProcessedTransactionCounts,
3730        timings: &mut ExecuteTimings,
3731    ) -> Vec<TransactionCommitResult> {
3732        assert!(
3733            !self.freeze_started(),
3734            "commit_transactions() working on a bank that is already frozen or is undergoing freezing!"
3735        );
3736
3737        let ProcessedTransactionCounts {
3738            processed_transactions_count,
3739            processed_non_vote_transactions_count,
3740            processed_with_successful_result_count,
3741            signature_count,
3742        } = *processed_counts;
3743
3744        self.increment_transaction_count(processed_transactions_count);
3745        self.increment_non_vote_transaction_count_since_restart(
3746            processed_non_vote_transactions_count,
3747        );
3748        self.increment_signature_count(signature_count);
3749
3750        let processed_with_failure_result_count =
3751            processed_transactions_count.saturating_sub(processed_with_successful_result_count);
3752        self.transaction_error_count
3753            .fetch_add(processed_with_failure_result_count, Relaxed);
3754
3755        if processed_transactions_count > 0 {
3756            self.is_delta.store(true, Relaxed);
3757            self.transaction_entries_count.fetch_add(1, Relaxed);
3758            self.transactions_per_entry_max
3759                .fetch_max(processed_transactions_count, Relaxed);
3760        }
3761
3762        let ((), store_accounts_us) = measure_us!({
3763            // If geyser is present, we must collect `SanitizedTransaction`
3764            // references in order to comply with that interface - until it
3765            // is changed.
3766            let maybe_transaction_refs = self
3767                .accounts()
3768                .accounts_db
3769                .has_accounts_update_notifier()
3770                .then(|| {
3771                    sanitized_txs
3772                        .iter()
3773                        .map(|tx| tx.as_sanitized_transaction())
3774                        .collect::<Vec<_>>()
3775                });
3776
3777            let (accounts_to_store, transactions) = collect_accounts_to_store(
3778                sanitized_txs,
3779                &maybe_transaction_refs,
3780                &processing_results,
3781            );
3782
3783            let to_store = (self.slot(), accounts_to_store.as_slice());
3784            self.update_bank_hash_stats(&to_store);
3785            self.rc
3786                .accounts
3787                .store_cached(to_store, transactions.as_deref());
3788        });
3789
3790        self.collect_rent(&processing_results);
3791
3792        // Cached vote and stake accounts are synchronized with accounts-db
3793        // after each transaction.
3794        let ((), update_stakes_cache_us) =
3795            measure_us!(self.update_stakes_cache(sanitized_txs, &processing_results));
3796
3797        let ((), update_executors_us) = measure_us!({
3798            let mut cache = None;
3799            for processing_result in &processing_results {
3800                if let Some(ProcessedTransaction::Executed(executed_tx)) =
3801                    processing_result.processed_transaction()
3802                {
3803                    let programs_modified_by_tx = &executed_tx.programs_modified_by_tx;
3804                    if executed_tx.was_successful() && !programs_modified_by_tx.is_empty() {
3805                        cache
3806                            .get_or_insert_with(|| {
3807                                self.transaction_processor.program_cache.write().unwrap()
3808                            })
3809                            .merge(programs_modified_by_tx);
3810                    }
3811                }
3812            }
3813        });
3814
3815        let accounts_data_len_delta = processing_results
3816            .iter()
3817            .filter_map(|processing_result| processing_result.processed_transaction())
3818            .filter_map(|processed_tx| processed_tx.execution_details())
3819            .filter_map(|details| {
3820                details
3821                    .status
3822                    .is_ok()
3823                    .then_some(details.accounts_data_len_delta)
3824            })
3825            .sum();
3826        self.update_accounts_data_size_delta_on_chain(accounts_data_len_delta);
3827
3828        let ((), update_transaction_statuses_us) =
3829            measure_us!(self.update_transaction_statuses(sanitized_txs, &processing_results));
3830
3831        self.filter_program_errors_and_collect_fee_details(&processing_results);
3832
3833        timings.saturating_add_in_place(ExecuteTimingType::StoreUs, store_accounts_us);
3834        timings.saturating_add_in_place(
3835            ExecuteTimingType::UpdateStakesCacheUs,
3836            update_stakes_cache_us,
3837        );
3838        timings.saturating_add_in_place(ExecuteTimingType::UpdateExecutorsUs, update_executors_us);
3839        timings.saturating_add_in_place(
3840            ExecuteTimingType::UpdateTransactionStatuses,
3841            update_transaction_statuses_us,
3842        );
3843
3844        Self::create_commit_results(processing_results)
3845    }
3846
3847    fn create_commit_results(
3848        processing_results: Vec<TransactionProcessingResult>,
3849    ) -> Vec<TransactionCommitResult> {
3850        processing_results
3851            .into_iter()
3852            .map(|processing_result| {
3853                let processing_result = processing_result?;
3854                let executed_units = processing_result.executed_units();
3855                let loaded_accounts_data_size = processing_result.loaded_accounts_data_size();
3856
3857                match processing_result {
3858                    ProcessedTransaction::Executed(executed_tx) => {
3859                        let execution_details = executed_tx.execution_details;
3860                        let LoadedTransaction {
3861                            rent_debits,
3862                            accounts: loaded_accounts,
3863                            fee_details,
3864                            ..
3865                        } = executed_tx.loaded_transaction;
3866
3867                        // Rent is only collected for successfully executed transactions
3868                        let rent_debits = if execution_details.was_successful() {
3869                            rent_debits
3870                        } else {
3871                            RentDebits::default()
3872                        };
3873
3874                        Ok(CommittedTransaction {
3875                            status: execution_details.status,
3876                            log_messages: execution_details.log_messages,
3877                            inner_instructions: execution_details.inner_instructions,
3878                            return_data: execution_details.return_data,
3879                            executed_units,
3880                            fee_details,
3881                            rent_debits,
3882                            loaded_account_stats: TransactionLoadedAccountsStats {
3883                                loaded_accounts_count: loaded_accounts.len(),
3884                                loaded_accounts_data_size,
3885                            },
3886                        })
3887                    }
3888                    ProcessedTransaction::FeesOnly(fees_only_tx) => Ok(CommittedTransaction {
3889                        status: Err(fees_only_tx.load_error),
3890                        log_messages: None,
3891                        inner_instructions: None,
3892                        return_data: None,
3893                        executed_units,
3894                        rent_debits: RentDebits::default(),
3895                        fee_details: fees_only_tx.fee_details,
3896                        loaded_account_stats: TransactionLoadedAccountsStats {
3897                            loaded_accounts_count: fees_only_tx.rollback_accounts.count(),
3898                            loaded_accounts_data_size,
3899                        },
3900                    }),
3901                }
3902            })
3903            .collect()
3904    }
3905
3906    fn collect_rent(&self, processing_results: &[TransactionProcessingResult]) {
3907        let collected_rent = processing_results
3908            .iter()
3909            .filter_map(|processing_result| processing_result.processed_transaction())
3910            .filter_map(|processed_tx| processed_tx.executed_transaction())
3911            .filter(|executed_tx| executed_tx.was_successful())
3912            .map(|executed_tx| executed_tx.loaded_transaction.rent)
3913            .sum();
3914        self.collected_rent.fetch_add(collected_rent, Relaxed);
3915    }
3916
3917    fn run_incinerator(&self) {
3918        if let Some((account, _)) =
3919            self.get_account_modified_since_parent_with_fixed_root(&incinerator::id())
3920        {
3921            self.capitalization.fetch_sub(account.lamports(), Relaxed);
3922            self.store_account(&incinerator::id(), &AccountSharedData::default());
3923        }
3924    }
3925
3926    /// Get stake and stake node accounts
3927    pub(crate) fn get_stake_accounts(&self, minimized_account_set: &DashSet<Pubkey>) {
3928        self.stakes_cache
3929            .stakes()
3930            .stake_delegations()
3931            .iter()
3932            .for_each(|(pubkey, _)| {
3933                minimized_account_set.insert(*pubkey);
3934            });
3935
3936        self.stakes_cache
3937            .stakes()
3938            .staked_nodes()
3939            .par_iter()
3940            .for_each(|(pubkey, _)| {
3941                minimized_account_set.insert(*pubkey);
3942            });
3943    }
3944
3945    /// After deserialize, populate skipped rewrites with accounts that would normally
3946    /// have had their data rewritten in this slot due to rent collection (but didn't).
3947    ///
3948    /// This is required when starting up from a snapshot to verify the bank hash.
3949    ///
3950    /// A second usage is from the `bank_to_xxx_snapshot_archive()` functions.  These fns call
3951    /// `Bank::rehash()` to handle if the user manually modified any accounts and thus requires
3952    /// calculating the bank hash again.  Since calculating the bank hash *takes* the skipped
3953    /// rewrites, this second time will not have any skipped rewrites, and thus the hash would be
3954    /// updated to the wrong value.  So, rebuild the skipped rewrites before rehashing.
3955    fn rebuild_skipped_rewrites(&self) {
3956        // If the feature gate to *not* add rent collection rewrites to the bank hash is enabled,
3957        // then do *not* add anything to our skipped_rewrites.
3958        if self.bank_hash_skips_rent_rewrites() {
3959            return;
3960        }
3961
3962        let (skipped_rewrites, measure_skipped_rewrites) =
3963            measure_time!(self.calculate_skipped_rewrites());
3964        info!(
3965            "Rebuilding skipped rewrites of {} accounts{measure_skipped_rewrites}",
3966            skipped_rewrites.len()
3967        );
3968
3969        *self.skipped_rewrites.lock().unwrap() = skipped_rewrites;
3970    }
3971
3972    /// Calculates (and returns) skipped rewrites for this bank
3973    ///
3974    /// Refer to `rebuild_skipped_rewrites()` for more documentation.
3975    /// This implementation is purposely separate to facilitate testing.
3976    ///
3977    /// The key observation is that accounts in Bank::skipped_rewrites are only used IFF the
3978    /// specific account is *not* already in the accounts delta hash.  If an account is not in
3979    /// the accounts delta hash, then it means the account was not modified.  Since (basically)
3980    /// all accounts are rent exempt, this means (basically) all accounts are unmodified by rent
3981    /// collection.  So we just need to load the accounts that would've been checked for rent
3982    /// collection, hash them, and add them to Bank::skipped_rewrites.
3983    ///
3984    /// As of this writing, there are ~350 million acounts on mainnet-beta.
3985    /// Rent collection almost always collects a single slot at a time.
3986    /// So 1 slot of 432,000, of 350 million accounts, is ~800 accounts per slot.
3987    /// Since we haven't started processing anything yet, it should be fast enough to simply
3988    /// load the accounts directly.
3989    /// Empirically, this takes about 3-4 milliseconds.
3990    fn calculate_skipped_rewrites(&self) -> HashMap<Pubkey, AccountHash> {
3991        // The returned skipped rewrites may include accounts that were actually *not* skipped!
3992        // (This is safe, as per the fn's documentation above.)
3993        self.get_accounts_for_skipped_rewrites()
3994            .map(|(pubkey, account_hash, _account)| (pubkey, account_hash))
3995            .collect()
3996    }
3997
3998    /// Loads accounts that were selected for rent collection this slot.
3999    /// After loading the accounts, also calculate and return the account hashes.
4000    /// This is used when dealing with skipped rewrites.
4001    fn get_accounts_for_skipped_rewrites(
4002        &self,
4003    ) -> impl Iterator<Item = (Pubkey, AccountHash, AccountSharedData)> + '_ {
4004        self.rent_collection_partitions()
4005            .into_iter()
4006            .map(accounts_partition::pubkey_range_from_partition)
4007            .flat_map(|pubkey_range| {
4008                self.rc
4009                    .accounts
4010                    .load_to_collect_rent_eagerly(&self.ancestors, pubkey_range)
4011            })
4012            .map(|(pubkey, account, _slot)| {
4013                let account_hash = AccountsDb::hash_account(&account, &pubkey);
4014                (pubkey, account_hash, account)
4015            })
4016    }
4017
4018    /// Returns the accounts, sorted by pubkey, that were part of accounts delta hash calculation
4019    /// This is used when writing a bank hash details file.
4020    pub(crate) fn get_accounts_for_bank_hash_details(&self) -> Vec<PubkeyHashAccount> {
4021        let accounts_db = &self.rc.accounts.accounts_db;
4022
4023        let mut accounts_written_this_slot =
4024            accounts_db.get_pubkey_hash_account_for_slot(self.slot());
4025
4026        // If we are skipping rewrites but also include them in the accounts delta hash, then we
4027        // need to go load those accounts and add them to the list of accounts written this slot.
4028        if !self.bank_hash_skips_rent_rewrites()
4029            && accounts_db.test_skip_rewrites_but_include_in_bank_hash
4030        {
4031            let pubkeys_written_this_slot: HashSet<_> = accounts_written_this_slot
4032                .iter()
4033                .map(|pubkey_hash_account| pubkey_hash_account.pubkey)
4034                .collect();
4035
4036            let rent_collection_accounts = self.get_accounts_for_skipped_rewrites();
4037            for (pubkey, hash, account) in rent_collection_accounts {
4038                if !pubkeys_written_this_slot.contains(&pubkey) {
4039                    accounts_written_this_slot.push(PubkeyHashAccount {
4040                        pubkey,
4041                        hash,
4042                        account,
4043                    });
4044                }
4045            }
4046        }
4047
4048        // Sort the accounts by pubkey to match the order of the accounts delta hash.
4049        // This also makes comparison of files from different nodes deterministic.
4050        accounts_written_this_slot.sort_unstable_by_key(|account| account.pubkey);
4051        accounts_written_this_slot
4052    }
4053
4054    fn collect_rent_eagerly(&self) {
4055        if self.lazy_rent_collection.load(Relaxed) {
4056            return;
4057        }
4058
4059        if self
4060            .feature_set
4061            .is_active(&feature_set::disable_partitioned_rent_collection::id())
4062        {
4063            return;
4064        }
4065
4066        let mut measure = Measure::start("collect_rent_eagerly-ms");
4067        let partitions = self.rent_collection_partitions();
4068        let count = partitions.len();
4069        let rent_metrics = RentMetrics::default();
4070        // partitions will usually be 1, but could be more if we skip slots
4071        let mut parallel = count > 1;
4072        if parallel {
4073            let ranges = partitions
4074                .iter()
4075                .map(|partition| {
4076                    (
4077                        *partition,
4078                        accounts_partition::pubkey_range_from_partition(*partition),
4079                    )
4080                })
4081                .collect::<Vec<_>>();
4082            // test every range to make sure ranges are not overlapping
4083            // some tests collect rent from overlapping ranges
4084            // example: [(0, 31, 32), (0, 0, 128), (0, 27, 128)]
4085            // read-modify-write of an account for rent collection cannot be done in parallel
4086            'outer: for i in 0..ranges.len() {
4087                for j in 0..ranges.len() {
4088                    if i == j {
4089                        continue;
4090                    }
4091
4092                    let i = &ranges[i].1;
4093                    let j = &ranges[j].1;
4094                    // make sure i doesn't contain j
4095                    if i.contains(j.start()) || i.contains(j.end()) {
4096                        parallel = false;
4097                        break 'outer;
4098                    }
4099                }
4100            }
4101
4102            if parallel {
4103                let thread_pool = &self.rc.accounts.accounts_db.thread_pool;
4104                thread_pool.install(|| {
4105                    ranges.into_par_iter().for_each(|range| {
4106                        self.collect_rent_in_range(range.0, range.1, &rent_metrics)
4107                    });
4108                });
4109            }
4110        }
4111        if !parallel {
4112            // collect serially
4113            partitions
4114                .into_iter()
4115                .for_each(|partition| self.collect_rent_in_partition(partition, &rent_metrics));
4116        }
4117        measure.stop();
4118        datapoint_info!(
4119            "collect_rent_eagerly",
4120            ("accounts", rent_metrics.count.load(Relaxed), i64),
4121            ("partitions", count, i64),
4122            ("total_time_us", measure.as_us(), i64),
4123            (
4124                "hold_range_us",
4125                rent_metrics.hold_range_us.load(Relaxed),
4126                i64
4127            ),
4128            ("load_us", rent_metrics.load_us.load(Relaxed), i64),
4129            ("collect_us", rent_metrics.collect_us.load(Relaxed), i64),
4130            ("hash_us", rent_metrics.hash_us.load(Relaxed), i64),
4131            ("store_us", rent_metrics.store_us.load(Relaxed), i64),
4132        );
4133    }
4134
4135    fn rent_collection_partitions(&self) -> Vec<Partition> {
4136        if !self.use_fixed_collection_cycle() {
4137            // This mode is for production/development/testing.
4138            // In this mode, we iterate over the whole pubkey value range for each epochs
4139            // including warm-up epochs.
4140            // The only exception is the situation where normal epochs are relatively short
4141            // (currently less than 2 day). In that case, we arrange a single collection
4142            // cycle to be multiple of epochs so that a cycle could be greater than the 2 day.
4143            self.variable_cycle_partitions()
4144        } else {
4145            // This mode is mainly for benchmarking only.
4146            // In this mode, we always iterate over the whole pubkey value range with
4147            // <slot_count_in_two_day> slots as a collection cycle, regardless warm-up or
4148            // alignment between collection cycles and epochs.
4149            // Thus, we can simulate stable processing load of eager rent collection,
4150            // strictly proportional to the number of pubkeys since genesis.
4151            self.fixed_cycle_partitions()
4152        }
4153    }
4154
4155    /// true if rent collection does NOT rewrite accounts whose pubkey indicates
4156    ///  it is time for rent collection, but the account is rent exempt.
4157    /// false if rent collection DOES rewrite accounts if the account is rent exempt
4158    /// This is the default behavior historically.
4159    fn bank_hash_skips_rent_rewrites(&self) -> bool {
4160        self.feature_set
4161            .is_active(&feature_set::skip_rent_rewrites::id())
4162    }
4163
4164    /// true if rent fees should be collected (i.e. disable_rent_fees_collection is NOT enabled)
4165    fn should_collect_rent(&self) -> bool {
4166        !self
4167            .feature_set
4168            .is_active(&feature_set::disable_rent_fees_collection::id())
4169    }
4170
4171    /// Collect rent from `accounts`
4172    ///
4173    /// This fn is called inside a parallel loop from `collect_rent_in_partition()`.  Avoid adding
4174    /// any code that causes contention on shared memory/data (i.e. do not update atomic metrics).
4175    ///
4176    /// The return value is a struct of computed values that `collect_rent_in_partition()` will
4177    /// reduce at the end of its parallel loop.  If possible, place data/computation that cause
4178    /// contention/take locks in the return struct and process them in
4179    /// `collect_rent_from_partition()` after reducing the parallel loop.
4180    fn collect_rent_from_accounts(
4181        &self,
4182        mut accounts: Vec<(Pubkey, AccountSharedData, Slot)>,
4183        rent_paying_pubkeys: Option<&HashSet<Pubkey>>,
4184        partition_index: PartitionIndex,
4185    ) -> CollectRentFromAccountsInfo {
4186        let mut rent_debits = RentDebits::default();
4187        let mut total_rent_collected_info = CollectedInfo::default();
4188        let mut accounts_to_store =
4189            Vec::<(&Pubkey, &AccountSharedData)>::with_capacity(accounts.len());
4190        let mut time_collecting_rent_us = 0;
4191        let mut time_storing_accounts_us = 0;
4192        let can_skip_rewrites = self.bank_hash_skips_rent_rewrites();
4193        let test_skip_rewrites_but_include_in_bank_hash = self
4194            .rc
4195            .accounts
4196            .accounts_db
4197            .test_skip_rewrites_but_include_in_bank_hash;
4198        let mut skipped_rewrites = Vec::default();
4199        for (pubkey, account, _loaded_slot) in accounts.iter_mut() {
4200            let rent_epoch_pre = account.rent_epoch();
4201            let (rent_collected_info, collect_rent_us) = measure_us!(collect_rent_from_account(
4202                &self.feature_set.runtime_features(),
4203                &self.rent_collector,
4204                pubkey,
4205                account
4206            ));
4207            time_collecting_rent_us += collect_rent_us;
4208            let rent_epoch_post = account.rent_epoch();
4209
4210            // did the account change in any way due to rent collection?
4211            let rent_epoch_changed = rent_epoch_post != rent_epoch_pre;
4212            let account_changed = rent_collected_info.rent_amount != 0 || rent_epoch_changed;
4213
4214            // always store the account, regardless if it changed or not
4215            let always_store_accounts =
4216                !can_skip_rewrites && !test_skip_rewrites_but_include_in_bank_hash;
4217
4218            // only store accounts where we collected rent
4219            // but get the hash for all these accounts even if collected rent is 0 (= not updated).
4220            // Also, there's another subtle side-effect from rewrites: this
4221            // ensures we verify the whole on-chain state (= all accounts)
4222            // via the bank delta hash slowly once per an epoch.
4223            if account_changed || always_store_accounts {
4224                if rent_collected_info.rent_amount > 0 {
4225                    if let Some(rent_paying_pubkeys) = rent_paying_pubkeys {
4226                        if !rent_paying_pubkeys.contains(pubkey) {
4227                            let partition_from_pubkey = accounts_partition::partition_from_pubkey(
4228                                pubkey,
4229                                self.epoch_schedule.slots_per_epoch,
4230                            );
4231                            // Submit datapoint instead of assert while we verify this is correct
4232                            datapoint_warn!(
4233                                "bank-unexpected_rent_paying_pubkey",
4234                                ("slot", self.slot(), i64),
4235                                ("pubkey", pubkey.to_string(), String),
4236                                ("partition_index", partition_index, i64),
4237                                ("partition_from_pubkey", partition_from_pubkey, i64)
4238                            );
4239                            warn!(
4240                                "Collecting rent from unexpected pubkey: {}, slot: {}, parent_slot: {:?}, \
4241                                partition_index: {}, partition_from_pubkey: {}",
4242                                pubkey,
4243                                self.slot(),
4244                                self.parent().map(|bank| bank.slot()),
4245                                partition_index,
4246                                partition_from_pubkey,
4247                            );
4248                        }
4249                    }
4250                } else {
4251                    debug_assert_eq!(rent_collected_info.rent_amount, 0);
4252                    if rent_epoch_changed {
4253                        datapoint_info!(
4254                            "bank-rent_collection_updated_only_rent_epoch",
4255                            ("slot", self.slot(), i64),
4256                            ("pubkey", pubkey.to_string(), String),
4257                            ("rent_epoch_pre", rent_epoch_pre, i64),
4258                            ("rent_epoch_post", rent_epoch_post, i64),
4259                        );
4260                    }
4261                }
4262                total_rent_collected_info += rent_collected_info;
4263                accounts_to_store.push((pubkey, account));
4264            } else if !account_changed
4265                && !can_skip_rewrites
4266                && test_skip_rewrites_but_include_in_bank_hash
4267            {
4268                // include rewrites that we skipped in the accounts delta hash.
4269                // This is what consensus requires prior to activation of bank_hash_skips_rent_rewrites.
4270                // This code path exists to allow us to test the long term effects on validators when the skipped rewrites
4271                // feature is enabled.
4272                let hash = AccountsDb::hash_account(account, pubkey);
4273                skipped_rewrites.push((*pubkey, hash));
4274            }
4275            rent_debits.insert(pubkey, rent_collected_info.rent_amount, account.lamports());
4276        }
4277
4278        if !accounts_to_store.is_empty() {
4279            // TODO: Maybe do not call `store_accounts()` here.  Instead return `accounts_to_store`
4280            // and have `collect_rent_in_partition()` perform all the stores.
4281            let (_, store_accounts_us) =
4282                measure_us!(self.store_accounts((self.slot(), &accounts_to_store[..])));
4283            time_storing_accounts_us += store_accounts_us;
4284        }
4285
4286        CollectRentFromAccountsInfo {
4287            skipped_rewrites,
4288            rent_collected_info: total_rent_collected_info,
4289            rent_rewards: rent_debits.into_unordered_rewards_iter().collect(),
4290            time_collecting_rent_us,
4291            time_storing_accounts_us,
4292            num_accounts: accounts.len(),
4293        }
4294    }
4295
4296    /// convert 'partition' to a pubkey range and 'collect_rent_in_range'
4297    fn collect_rent_in_partition(&self, partition: Partition, metrics: &RentMetrics) {
4298        let subrange_full = accounts_partition::pubkey_range_from_partition(partition);
4299        self.collect_rent_in_range(partition, subrange_full, metrics)
4300    }
4301
4302    /// 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)
4303    fn get_rent_paying_pubkeys(&self, partition: &Partition) -> Option<HashSet<Pubkey>> {
4304        self.rc
4305            .accounts
4306            .accounts_db
4307            .accounts_index
4308            .rent_paying_accounts_by_partition
4309            .get()
4310            .and_then(|rent_paying_accounts| {
4311                rent_paying_accounts.is_initialized().then(|| {
4312                    accounts_partition::get_partition_end_indexes(partition)
4313                        .into_iter()
4314                        .flat_map(|end_index| {
4315                            rent_paying_accounts.get_pubkeys_in_partition_index(end_index)
4316                        })
4317                        .cloned()
4318                        .collect::<HashSet<_>>()
4319                })
4320            })
4321    }
4322
4323    /// load accounts with pubkeys in 'subrange_full'
4324    /// collect rent and update 'account.rent_epoch' as necessary
4325    /// store accounts, whether rent was collected or not (depending on whether we skipping rewrites is enabled)
4326    /// update bank's rewrites set for all rewrites that were skipped
4327    fn collect_rent_in_range(
4328        &self,
4329        partition: Partition,
4330        subrange_full: RangeInclusive<Pubkey>,
4331        metrics: &RentMetrics,
4332    ) {
4333        let mut hold_range = Measure::start("hold_range");
4334        let thread_pool = &self.rc.accounts.accounts_db.thread_pool;
4335        thread_pool.install(|| {
4336            self.rc
4337                .accounts
4338                .hold_range_in_memory(&subrange_full, true, thread_pool);
4339            hold_range.stop();
4340            metrics.hold_range_us.fetch_add(hold_range.as_us(), Relaxed);
4341
4342            let rent_paying_pubkeys_ = self.get_rent_paying_pubkeys(&partition);
4343            let rent_paying_pubkeys = rent_paying_pubkeys_.as_ref();
4344
4345            // divide the range into num_threads smaller ranges and process in parallel
4346            // Note that 'pubkey_range_from_partition' cannot easily be re-used here to break the range smaller.
4347            // It has special handling of 0..0 and partition_count changes affect all ranges unevenly.
4348            let num_threads = solana_accounts_db::accounts_db::quarter_thread_count() as u64;
4349            let sz = std::mem::size_of::<u64>();
4350            let start_prefix = accounts_partition::prefix_from_pubkey(subrange_full.start());
4351            let end_prefix_inclusive = accounts_partition::prefix_from_pubkey(subrange_full.end());
4352            let range = end_prefix_inclusive - start_prefix;
4353            let increment = range / num_threads;
4354            let mut results = (0..num_threads)
4355                .into_par_iter()
4356                .map(|chunk| {
4357                    let offset = |chunk| start_prefix + chunk * increment;
4358                    let start = offset(chunk);
4359                    let last = chunk == num_threads - 1;
4360                    let merge_prefix = |prefix: u64, mut bound: Pubkey| {
4361                        bound.as_mut()[0..sz].copy_from_slice(&prefix.to_be_bytes());
4362                        bound
4363                    };
4364                    let start = merge_prefix(start, *subrange_full.start());
4365                    let (accounts, measure_load_accounts) = measure_time!(if last {
4366                        let end = *subrange_full.end();
4367                        let subrange = start..=end; // IN-clusive
4368                        self.rc
4369                            .accounts
4370                            .load_to_collect_rent_eagerly(&self.ancestors, subrange)
4371                    } else {
4372                        let end = merge_prefix(offset(chunk + 1), *subrange_full.start());
4373                        let subrange = start..end; // EX-clusive, the next 'start' will be this same value
4374                        self.rc
4375                            .accounts
4376                            .load_to_collect_rent_eagerly(&self.ancestors, subrange)
4377                    });
4378                    CollectRentInPartitionInfo::new(
4379                        self.collect_rent_from_accounts(accounts, rent_paying_pubkeys, partition.1),
4380                        Duration::from_nanos(measure_load_accounts.as_ns()),
4381                    )
4382                })
4383                .reduce(
4384                    CollectRentInPartitionInfo::default,
4385                    CollectRentInPartitionInfo::reduce,
4386                );
4387
4388            self.skipped_rewrites
4389                .lock()
4390                .unwrap()
4391                .extend(results.skipped_rewrites);
4392
4393            // We cannot assert here that we collected from all expected keys.
4394            // Some accounts may have been topped off or may have had all funds removed and gone to 0 lamports.
4395
4396            self.rc
4397                .accounts
4398                .hold_range_in_memory(&subrange_full, false, thread_pool);
4399
4400            self.collected_rent
4401                .fetch_add(results.rent_collected, Relaxed);
4402            self.update_accounts_data_size_delta_off_chain(
4403                -(results.accounts_data_size_reclaimed as i64),
4404            );
4405            self.rewards
4406                .write()
4407                .unwrap()
4408                .append(&mut results.rent_rewards);
4409
4410            metrics
4411                .load_us
4412                .fetch_add(results.time_loading_accounts_us, Relaxed);
4413            metrics
4414                .collect_us
4415                .fetch_add(results.time_collecting_rent_us, Relaxed);
4416            metrics
4417                .store_us
4418                .fetch_add(results.time_storing_accounts_us, Relaxed);
4419            metrics.count.fetch_add(results.num_accounts, Relaxed);
4420        });
4421    }
4422
4423    pub(crate) fn fixed_cycle_partitions_between_slots(
4424        &self,
4425        starting_slot: Slot,
4426        ending_slot: Slot,
4427    ) -> Vec<Partition> {
4428        let slot_count_in_two_day = self.slot_count_in_two_day();
4429        accounts_partition::get_partitions(ending_slot, starting_slot, slot_count_in_two_day)
4430    }
4431
4432    fn fixed_cycle_partitions(&self) -> Vec<Partition> {
4433        self.fixed_cycle_partitions_between_slots(self.parent_slot(), self.slot())
4434    }
4435
4436    pub(crate) fn variable_cycle_partitions_between_slots(
4437        &self,
4438        starting_slot: Slot,
4439        ending_slot: Slot,
4440    ) -> Vec<Partition> {
4441        let (starting_epoch, mut starting_slot_index) =
4442            self.get_epoch_and_slot_index(starting_slot);
4443        let (ending_epoch, ending_slot_index) = self.get_epoch_and_slot_index(ending_slot);
4444
4445        let mut partitions = vec![];
4446        if starting_epoch < ending_epoch {
4447            let slot_skipped = (ending_slot - starting_slot) > 1;
4448            if slot_skipped {
4449                // Generate special partitions because there are skipped slots
4450                // exactly at the epoch transition.
4451
4452                let parent_last_slot_index = self.get_slots_in_epoch(starting_epoch) - 1;
4453
4454                // ... for parent epoch
4455                partitions.push(self.partition_from_slot_indexes_with_gapped_epochs(
4456                    starting_slot_index,
4457                    parent_last_slot_index,
4458                    starting_epoch,
4459                ));
4460
4461                if ending_slot_index > 0 {
4462                    // ... for current epoch
4463                    partitions.push(self.partition_from_slot_indexes_with_gapped_epochs(
4464                        0,
4465                        0,
4466                        ending_epoch,
4467                    ));
4468                }
4469            }
4470            starting_slot_index = 0;
4471        }
4472
4473        partitions.push(self.partition_from_normal_slot_indexes(
4474            starting_slot_index,
4475            ending_slot_index,
4476            ending_epoch,
4477        ));
4478
4479        partitions
4480    }
4481
4482    fn variable_cycle_partitions(&self) -> Vec<Partition> {
4483        self.variable_cycle_partitions_between_slots(self.parent_slot(), self.slot())
4484    }
4485
4486    fn do_partition_from_slot_indexes(
4487        &self,
4488        start_slot_index: SlotIndex,
4489        end_slot_index: SlotIndex,
4490        epoch: Epoch,
4491        generated_for_gapped_epochs: bool,
4492    ) -> Partition {
4493        let slot_count_per_epoch = self.get_slots_in_epoch(epoch);
4494
4495        let cycle_params = if !self.use_multi_epoch_collection_cycle(epoch) {
4496            // mnb should always go through this code path
4497            accounts_partition::rent_single_epoch_collection_cycle_params(
4498                epoch,
4499                slot_count_per_epoch,
4500            )
4501        } else {
4502            accounts_partition::rent_multi_epoch_collection_cycle_params(
4503                epoch,
4504                slot_count_per_epoch,
4505                self.first_normal_epoch(),
4506                self.slot_count_in_two_day() / slot_count_per_epoch,
4507            )
4508        };
4509        accounts_partition::get_partition_from_slot_indexes(
4510            cycle_params,
4511            start_slot_index,
4512            end_slot_index,
4513            generated_for_gapped_epochs,
4514        )
4515    }
4516
4517    fn partition_from_normal_slot_indexes(
4518        &self,
4519        start_slot_index: SlotIndex,
4520        end_slot_index: SlotIndex,
4521        epoch: Epoch,
4522    ) -> Partition {
4523        self.do_partition_from_slot_indexes(start_slot_index, end_slot_index, epoch, false)
4524    }
4525
4526    fn partition_from_slot_indexes_with_gapped_epochs(
4527        &self,
4528        start_slot_index: SlotIndex,
4529        end_slot_index: SlotIndex,
4530        epoch: Epoch,
4531    ) -> Partition {
4532        self.do_partition_from_slot_indexes(start_slot_index, end_slot_index, epoch, true)
4533    }
4534
4535    // Given short epochs, it's too costly to collect rent eagerly
4536    // within an epoch, so lower the frequency of it.
4537    // These logic isn't strictly eager anymore and should only be used
4538    // for development/performance purpose.
4539    // Absolutely not under ClusterType::MainnetBeta!!!!
4540    fn use_multi_epoch_collection_cycle(&self, epoch: Epoch) -> bool {
4541        // Force normal behavior, disabling multi epoch collection cycle for manual local testing
4542        #[cfg(not(test))]
4543        if self.slot_count_per_normal_epoch() == solana_epoch_schedule::MINIMUM_SLOTS_PER_EPOCH {
4544            return false;
4545        }
4546
4547        epoch >= self.first_normal_epoch()
4548            && self.slot_count_per_normal_epoch() < self.slot_count_in_two_day()
4549    }
4550
4551    pub(crate) fn use_fixed_collection_cycle(&self) -> bool {
4552        // Force normal behavior, disabling fixed collection cycle for manual local testing
4553        #[cfg(not(test))]
4554        if self.slot_count_per_normal_epoch() == solana_epoch_schedule::MINIMUM_SLOTS_PER_EPOCH {
4555            return false;
4556        }
4557
4558        self.cluster_type() != ClusterType::MainnetBeta
4559            && self.slot_count_per_normal_epoch() < self.slot_count_in_two_day()
4560    }
4561
4562    fn slot_count_in_two_day(&self) -> SlotCount {
4563        Self::slot_count_in_two_day_helper(self.ticks_per_slot)
4564    }
4565
4566    // This value is specially chosen to align with slots per epoch in mainnet-beta and testnet
4567    // Also, assume 500GB account data set as the extreme, then for 2 day (=48 hours) to collect
4568    // rent eagerly, we'll consume 5.7 MB/s IO bandwidth, bidirectionally.
4569    pub fn slot_count_in_two_day_helper(ticks_per_slot: SlotCount) -> SlotCount {
4570        2 * DEFAULT_TICKS_PER_SECOND * SECONDS_PER_DAY / ticks_per_slot
4571    }
4572
4573    fn slot_count_per_normal_epoch(&self) -> SlotCount {
4574        self.get_slots_in_epoch(self.first_normal_epoch())
4575    }
4576
4577    pub fn cluster_type(&self) -> ClusterType {
4578        // unwrap is safe; self.cluster_type is ensured to be Some() always...
4579        // we only using Option here for ABI compatibility...
4580        self.cluster_type.unwrap()
4581    }
4582
4583    /// Process a batch of transactions.
4584    #[must_use]
4585    pub fn load_execute_and_commit_transactions(
4586        &self,
4587        batch: &TransactionBatch<impl TransactionWithMeta>,
4588        max_age: usize,
4589        recording_config: ExecutionRecordingConfig,
4590        timings: &mut ExecuteTimings,
4591        log_messages_bytes_limit: Option<usize>,
4592    ) -> (Vec<TransactionCommitResult>, Option<BalanceCollector>) {
4593        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
4594            batch,
4595            max_age,
4596            recording_config,
4597            timings,
4598            log_messages_bytes_limit,
4599            None::<fn(&mut _, &_) -> _>,
4600        )
4601        .unwrap()
4602    }
4603
4604    pub fn load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
4605        &'a self,
4606        batch: &TransactionBatch<impl TransactionWithMeta>,
4607        max_age: usize,
4608        recording_config: ExecutionRecordingConfig,
4609        timings: &mut ExecuteTimings,
4610        log_messages_bytes_limit: Option<usize>,
4611        pre_commit_callback: impl FnOnce(
4612            &mut ExecuteTimings,
4613            &[TransactionProcessingResult],
4614        ) -> PreCommitResult<'a>,
4615    ) -> Result<(Vec<TransactionCommitResult>, Option<BalanceCollector>)> {
4616        self.do_load_execute_and_commit_transactions_with_pre_commit_callback(
4617            batch,
4618            max_age,
4619            recording_config,
4620            timings,
4621            log_messages_bytes_limit,
4622            Some(pre_commit_callback),
4623        )
4624    }
4625
4626    fn do_load_execute_and_commit_transactions_with_pre_commit_callback<'a>(
4627        &'a self,
4628        batch: &TransactionBatch<impl TransactionWithMeta>,
4629        max_age: usize,
4630        recording_config: ExecutionRecordingConfig,
4631        timings: &mut ExecuteTimings,
4632        log_messages_bytes_limit: Option<usize>,
4633        pre_commit_callback: Option<
4634            impl FnOnce(&mut ExecuteTimings, &[TransactionProcessingResult]) -> PreCommitResult<'a>,
4635        >,
4636    ) -> Result<(Vec<TransactionCommitResult>, Option<BalanceCollector>)> {
4637        let LoadAndExecuteTransactionsOutput {
4638            processing_results,
4639            processed_counts,
4640            balance_collector,
4641        } = self.load_and_execute_transactions(
4642            batch,
4643            max_age,
4644            timings,
4645            &mut TransactionErrorMetrics::default(),
4646            TransactionProcessingConfig {
4647                account_overrides: None,
4648                check_program_modification_slot: self.check_program_modification_slot,
4649                log_messages_bytes_limit,
4650                limit_to_load_programs: false,
4651                recording_config,
4652            },
4653        );
4654
4655        // pre_commit_callback could initiate an atomic operation (i.e. poh recording with block
4656        // producing unified scheduler). in that case, it returns Some(freeze_lock), which should
4657        // unlocked only after calling commit_transactions() immediately after calling the
4658        // callback.
4659        let freeze_lock = if let Some(pre_commit_callback) = pre_commit_callback {
4660            pre_commit_callback(timings, &processing_results)?
4661        } else {
4662            None
4663        };
4664        let commit_results = self.commit_transactions(
4665            batch.sanitized_transactions(),
4666            processing_results,
4667            &processed_counts,
4668            timings,
4669        );
4670        drop(freeze_lock);
4671        Ok((commit_results, balance_collector))
4672    }
4673
4674    /// Process a Transaction. This is used for unit tests and simply calls the vector
4675    /// Bank::process_transactions method.
4676    pub fn process_transaction(&self, tx: &Transaction) -> Result<()> {
4677        self.try_process_transactions(std::iter::once(tx))?[0].clone()?;
4678        tx.signatures
4679            .first()
4680            .map_or(Ok(()), |sig| self.get_signature_status(sig).unwrap())
4681    }
4682
4683    /// Process a Transaction and store metadata. This is used for tests and the banks services. It
4684    /// replicates the vector Bank::process_transaction method with metadata recording enabled.
4685    pub fn process_transaction_with_metadata(
4686        &self,
4687        tx: impl Into<VersionedTransaction>,
4688    ) -> Result<CommittedTransaction> {
4689        let txs = vec![tx.into()];
4690        let batch = self.prepare_entry_batch(txs)?;
4691
4692        let (mut commit_results, ..) = self.load_execute_and_commit_transactions(
4693            &batch,
4694            MAX_PROCESSING_AGE,
4695            ExecutionRecordingConfig {
4696                enable_cpi_recording: false,
4697                enable_log_recording: true,
4698                enable_return_data_recording: true,
4699                enable_transaction_balance_recording: false,
4700            },
4701            &mut ExecuteTimings::default(),
4702            Some(1000 * 1000),
4703        );
4704
4705        commit_results.remove(0)
4706    }
4707
4708    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
4709    /// Short circuits if any of the transactions do not pass sanitization checks.
4710    pub fn try_process_transactions<'a>(
4711        &self,
4712        txs: impl Iterator<Item = &'a Transaction>,
4713    ) -> Result<Vec<Result<()>>> {
4714        let txs = txs
4715            .map(|tx| VersionedTransaction::from(tx.clone()))
4716            .collect();
4717        self.try_process_entry_transactions(txs)
4718    }
4719
4720    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
4721    /// Short circuits if any of the transactions do not pass sanitization checks.
4722    pub fn try_process_entry_transactions(
4723        &self,
4724        txs: Vec<VersionedTransaction>,
4725    ) -> Result<Vec<Result<()>>> {
4726        let batch = self.prepare_entry_batch(txs)?;
4727        Ok(self.process_transaction_batch(&batch))
4728    }
4729
4730    #[must_use]
4731    fn process_transaction_batch(
4732        &self,
4733        batch: &TransactionBatch<impl TransactionWithMeta>,
4734    ) -> Vec<Result<()>> {
4735        self.load_execute_and_commit_transactions(
4736            batch,
4737            MAX_PROCESSING_AGE,
4738            ExecutionRecordingConfig::new_single_setting(false),
4739            &mut ExecuteTimings::default(),
4740            None,
4741        )
4742        .0
4743        .into_iter()
4744        .map(|commit_result| commit_result.map(|_| ()))
4745        .collect()
4746    }
4747
4748    /// Create, sign, and process a Transaction from `keypair` to `to` of
4749    /// `n` lamports where `blockhash` is the last Entry ID observed by the client.
4750    pub fn transfer(&self, n: u64, keypair: &Keypair, to: &Pubkey) -> Result<Signature> {
4751        let blockhash = self.last_blockhash();
4752        let tx = system_transaction::transfer(keypair, to, n, blockhash);
4753        let signature = tx.signatures[0];
4754        self.process_transaction(&tx).map(|_| signature)
4755    }
4756
4757    pub fn read_balance(account: &AccountSharedData) -> u64 {
4758        account.lamports()
4759    }
4760    /// Each program would need to be able to introspect its own state
4761    /// this is hard-coded to the Budget language
4762    pub fn get_balance(&self, pubkey: &Pubkey) -> u64 {
4763        self.get_account(pubkey)
4764            .map(|x| Self::read_balance(&x))
4765            .unwrap_or(0)
4766    }
4767
4768    /// Compute all the parents of the bank in order
4769    pub fn parents(&self) -> Vec<Arc<Bank>> {
4770        let mut parents = vec![];
4771        let mut bank = self.parent();
4772        while let Some(parent) = bank {
4773            parents.push(parent.clone());
4774            bank = parent.parent();
4775        }
4776        parents
4777    }
4778
4779    /// Compute all the parents of the bank including this bank itself
4780    pub fn parents_inclusive(self: Arc<Self>) -> Vec<Arc<Bank>> {
4781        let mut parents = self.parents();
4782        parents.insert(0, self);
4783        parents
4784    }
4785
4786    /// fn store the single `account` with `pubkey`.
4787    /// Uses `store_accounts`, which works on a vector of accounts.
4788    pub fn store_account(&self, pubkey: &Pubkey, account: &AccountSharedData) {
4789        self.store_accounts((self.slot(), &[(pubkey, account)][..]))
4790    }
4791
4792    pub fn store_accounts<'a>(&self, accounts: impl StorableAccounts<'a>) {
4793        assert!(!self.freeze_started());
4794        let mut m = Measure::start("stakes_cache.check_and_store");
4795        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
4796
4797        (0..accounts.len()).for_each(|i| {
4798            accounts.account(i, |account| {
4799                self.stakes_cache.check_and_store(
4800                    account.pubkey(),
4801                    &account,
4802                    new_warmup_cooldown_rate_epoch,
4803                )
4804            })
4805        });
4806        self.update_bank_hash_stats(&accounts);
4807        self.rc.accounts.store_accounts_cached(accounts);
4808        m.stop();
4809        self.rc
4810            .accounts
4811            .accounts_db
4812            .stats
4813            .stakes_cache_check_and_store_us
4814            .fetch_add(m.as_us(), Relaxed);
4815    }
4816
4817    pub fn force_flush_accounts_cache(&self) {
4818        self.rc
4819            .accounts
4820            .accounts_db
4821            .flush_accounts_cache(true, Some(self.slot()))
4822    }
4823
4824    pub fn flush_accounts_cache_if_needed(&self) {
4825        self.rc
4826            .accounts
4827            .accounts_db
4828            .flush_accounts_cache(false, Some(self.slot()))
4829    }
4830
4831    /// Technically this issues (or even burns!) new lamports,
4832    /// so be extra careful for its usage
4833    fn store_account_and_update_capitalization(
4834        &self,
4835        pubkey: &Pubkey,
4836        new_account: &AccountSharedData,
4837    ) {
4838        let old_account_data_size =
4839            if let Some(old_account) = self.get_account_with_fixed_root_no_cache(pubkey) {
4840                match new_account.lamports().cmp(&old_account.lamports()) {
4841                    std::cmp::Ordering::Greater => {
4842                        let increased = new_account.lamports() - old_account.lamports();
4843                        trace!(
4844                            "store_account_and_update_capitalization: increased: {} {}",
4845                            pubkey,
4846                            increased
4847                        );
4848                        self.capitalization.fetch_add(increased, Relaxed);
4849                    }
4850                    std::cmp::Ordering::Less => {
4851                        let decreased = old_account.lamports() - new_account.lamports();
4852                        trace!(
4853                            "store_account_and_update_capitalization: decreased: {} {}",
4854                            pubkey,
4855                            decreased
4856                        );
4857                        self.capitalization.fetch_sub(decreased, Relaxed);
4858                    }
4859                    std::cmp::Ordering::Equal => {}
4860                }
4861                old_account.data().len()
4862            } else {
4863                trace!(
4864                    "store_account_and_update_capitalization: created: {} {}",
4865                    pubkey,
4866                    new_account.lamports()
4867                );
4868                self.capitalization
4869                    .fetch_add(new_account.lamports(), Relaxed);
4870                0
4871            };
4872
4873        self.store_account(pubkey, new_account);
4874        self.calculate_and_update_accounts_data_size_delta_off_chain(
4875            old_account_data_size,
4876            new_account.data().len(),
4877        );
4878    }
4879
4880    pub fn accounts(&self) -> Arc<Accounts> {
4881        self.rc.accounts.clone()
4882    }
4883
4884    fn finish_init(
4885        &mut self,
4886        genesis_config: &GenesisConfig,
4887        additional_builtins: Option<&[BuiltinPrototype]>,
4888        debug_do_not_add_builtins: bool,
4889    ) {
4890        if let Some(compute_budget) = self.compute_budget {
4891            self.transaction_processor
4892                .set_execution_cost(compute_budget.to_cost());
4893        }
4894
4895        self.rewards_pool_pubkeys =
4896            Arc::new(genesis_config.rewards_pools.keys().cloned().collect());
4897
4898        self.apply_feature_activations(
4899            ApplyFeatureActivationsCaller::FinishInit,
4900            debug_do_not_add_builtins,
4901        );
4902
4903        // Cost-Tracker is not serialized in snapshot or any configs.
4904        // We must apply previously activated features related to limits here
4905        // so that the initial bank state is consistent with the feature set.
4906        // Cost-tracker limits are propagated through children banks.
4907        if self
4908            .feature_set
4909            .is_active(&feature_set::raise_block_limits_to_60m::id())
4910        {
4911            let (account_cost_limit, block_cost_limit, vote_cost_limit) = simd_0256_block_limits();
4912            self.write_cost_tracker().unwrap().set_limits(
4913                account_cost_limit,
4914                block_cost_limit,
4915                vote_cost_limit,
4916            );
4917        }
4918
4919        // If the accounts delta hash is still in use, start the background account hasher
4920        if !self
4921            .feature_set
4922            .is_active(&feature_set::remove_accounts_delta_hash::id())
4923        {
4924            self.rc.accounts.accounts_db.start_background_hasher();
4925        }
4926
4927        if !debug_do_not_add_builtins {
4928            for builtin in BUILTINS
4929                .iter()
4930                .chain(additional_builtins.unwrap_or(&[]).iter())
4931            {
4932                // The builtin should be added if it has no enable feature ID
4933                // and it has not been migrated to Core BPF.
4934                //
4935                // If a program was previously migrated to Core BPF, accountsDB
4936                // from snapshot should contain the BPF program accounts.
4937                let builtin_is_bpf = |program_id: &Pubkey| {
4938                    self.get_account(program_id)
4939                        .map(|a| a.owner() == &bpf_loader_upgradeable::id())
4940                        .unwrap_or(false)
4941                };
4942                if builtin.enable_feature_id.is_none() && !builtin_is_bpf(&builtin.program_id) {
4943                    self.transaction_processor.add_builtin(
4944                        self,
4945                        builtin.program_id,
4946                        builtin.name,
4947                        ProgramCacheEntry::new_builtin(0, builtin.name.len(), builtin.entrypoint),
4948                    );
4949                }
4950            }
4951            for precompile in get_precompiles() {
4952                if precompile.feature.is_none() {
4953                    self.add_precompile(&precompile.program_id);
4954                }
4955            }
4956        }
4957
4958        self.transaction_processor
4959            .configure_program_runtime_environments(
4960                Some(Arc::new(
4961                    create_program_runtime_environment_v1(
4962                        &self.feature_set.runtime_features(),
4963                        &self.compute_budget().unwrap_or_default().to_budget(),
4964                        false, /* deployment */
4965                        false, /* debugging_features */
4966                    )
4967                    .unwrap(),
4968                )),
4969                Some(Arc::new(create_program_runtime_environment_v2(
4970                    &self.compute_budget().unwrap_or_default().to_budget(),
4971                    false, /* debugging_features */
4972                ))),
4973            );
4974    }
4975
4976    pub fn set_inflation(&self, inflation: Inflation) {
4977        *self.inflation.write().unwrap() = inflation;
4978    }
4979
4980    /// Get a snapshot of the current set of hard forks
4981    pub fn hard_forks(&self) -> HardForks {
4982        self.hard_forks.read().unwrap().clone()
4983    }
4984
4985    pub fn register_hard_fork(&self, new_hard_fork_slot: Slot) {
4986        let bank_slot = self.slot();
4987
4988        let lock = self.freeze_lock();
4989        let bank_frozen = *lock != Hash::default();
4990        if new_hard_fork_slot < bank_slot {
4991            warn!(
4992                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is older \
4993                than the bank at slot {bank_slot} that attempted to register it."
4994            );
4995        } else if (new_hard_fork_slot == bank_slot) && bank_frozen {
4996            warn!(
4997                "Hard fork at slot {new_hard_fork_slot} ignored, the hard fork is the same \
4998                slot as the bank at slot {bank_slot} that attempted to register it, but that \
4999                bank is already frozen."
5000            );
5001        } else {
5002            self.hard_forks
5003                .write()
5004                .unwrap()
5005                .register(new_hard_fork_slot);
5006        }
5007    }
5008
5009    pub fn get_account_with_fixed_root_no_cache(
5010        &self,
5011        pubkey: &Pubkey,
5012    ) -> Option<AccountSharedData> {
5013        self.load_account_with(pubkey, |_| false)
5014            .map(|(acc, _slot)| acc)
5015    }
5016
5017    fn load_account_with(
5018        &self,
5019        pubkey: &Pubkey,
5020        callback: impl for<'local> Fn(&'local AccountSharedData) -> bool,
5021    ) -> Option<(AccountSharedData, Slot)> {
5022        self.rc
5023            .accounts
5024            .accounts_db
5025            .load_account_with(&self.ancestors, pubkey, callback)
5026    }
5027
5028    // Hi! leaky abstraction here....
5029    // try to use get_account_with_fixed_root() if it's called ONLY from on-chain runtime account
5030    // processing. That alternative fn provides more safety.
5031    pub fn get_account(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
5032        self.get_account_modified_slot(pubkey)
5033            .map(|(acc, _slot)| acc)
5034    }
5035
5036    // Hi! leaky abstraction here....
5037    // use this over get_account() if it's called ONLY from on-chain runtime account
5038    // processing (i.e. from in-band replay/banking stage; that ensures root is *fixed* while
5039    // running).
5040    // pro: safer assertion can be enabled inside AccountsDb
5041    // con: panics!() if called from off-chain processing
5042    pub fn get_account_with_fixed_root(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
5043        self.get_account_modified_slot_with_fixed_root(pubkey)
5044            .map(|(acc, _slot)| acc)
5045    }
5046
5047    // See note above get_account_with_fixed_root() about when to prefer this function
5048    pub fn get_account_modified_slot_with_fixed_root(
5049        &self,
5050        pubkey: &Pubkey,
5051    ) -> Option<(AccountSharedData, Slot)> {
5052        self.load_slow_with_fixed_root(&self.ancestors, pubkey)
5053    }
5054
5055    pub fn get_account_modified_slot(&self, pubkey: &Pubkey) -> Option<(AccountSharedData, Slot)> {
5056        self.load_slow(&self.ancestors, pubkey)
5057    }
5058
5059    fn load_slow(
5060        &self,
5061        ancestors: &Ancestors,
5062        pubkey: &Pubkey,
5063    ) -> Option<(AccountSharedData, Slot)> {
5064        // get_account (= primary this fn caller) may be called from on-chain Bank code even if we
5065        // try hard to use get_account_with_fixed_root for that purpose...
5066        // so pass safer LoadHint:Unspecified here as a fallback
5067        self.rc.accounts.load_without_fixed_root(ancestors, pubkey)
5068    }
5069
5070    fn load_slow_with_fixed_root(
5071        &self,
5072        ancestors: &Ancestors,
5073        pubkey: &Pubkey,
5074    ) -> Option<(AccountSharedData, Slot)> {
5075        self.rc.accounts.load_with_fixed_root(ancestors, pubkey)
5076    }
5077
5078    pub fn get_program_accounts(
5079        &self,
5080        program_id: &Pubkey,
5081        config: &ScanConfig,
5082    ) -> ScanResult<Vec<TransactionAccount>> {
5083        self.rc
5084            .accounts
5085            .load_by_program(&self.ancestors, self.bank_id, program_id, config)
5086    }
5087
5088    pub fn get_filtered_program_accounts<F: Fn(&AccountSharedData) -> bool>(
5089        &self,
5090        program_id: &Pubkey,
5091        filter: F,
5092        config: &ScanConfig,
5093    ) -> ScanResult<Vec<TransactionAccount>> {
5094        self.rc.accounts.load_by_program_with_filter(
5095            &self.ancestors,
5096            self.bank_id,
5097            program_id,
5098            filter,
5099            config,
5100        )
5101    }
5102
5103    pub fn get_filtered_indexed_accounts<F: Fn(&AccountSharedData) -> bool>(
5104        &self,
5105        index_key: &IndexKey,
5106        filter: F,
5107        config: &ScanConfig,
5108        byte_limit_for_scan: Option<usize>,
5109    ) -> ScanResult<Vec<TransactionAccount>> {
5110        self.rc.accounts.load_by_index_key_with_filter(
5111            &self.ancestors,
5112            self.bank_id,
5113            index_key,
5114            filter,
5115            config,
5116            byte_limit_for_scan,
5117        )
5118    }
5119
5120    pub fn account_indexes_include_key(&self, key: &Pubkey) -> bool {
5121        self.rc.accounts.account_indexes_include_key(key)
5122    }
5123
5124    /// Returns all the accounts this bank can load
5125    pub fn get_all_accounts(&self, sort_results: bool) -> ScanResult<Vec<PubkeyAccountSlot>> {
5126        self.rc
5127            .accounts
5128            .load_all(&self.ancestors, self.bank_id, sort_results)
5129    }
5130
5131    // Scans all the accounts this bank can load, applying `scan_func`
5132    pub fn scan_all_accounts<F>(&self, scan_func: F, sort_results: bool) -> ScanResult<()>
5133    where
5134        F: FnMut(Option<(&Pubkey, AccountSharedData, Slot)>),
5135    {
5136        self.rc
5137            .accounts
5138            .scan_all(&self.ancestors, self.bank_id, scan_func, sort_results)
5139    }
5140
5141    pub fn get_program_accounts_modified_since_parent(
5142        &self,
5143        program_id: &Pubkey,
5144    ) -> Vec<TransactionAccount> {
5145        self.rc
5146            .accounts
5147            .load_by_program_slot(self.slot(), Some(program_id))
5148    }
5149
5150    pub fn get_transaction_logs(
5151        &self,
5152        address: Option<&Pubkey>,
5153    ) -> Option<Vec<TransactionLogInfo>> {
5154        self.transaction_log_collector
5155            .read()
5156            .unwrap()
5157            .get_logs_for_address(address)
5158    }
5159
5160    /// Returns all the accounts stored in this slot
5161    pub fn get_all_accounts_modified_since_parent(&self) -> Vec<TransactionAccount> {
5162        self.rc.accounts.load_by_program_slot(self.slot(), None)
5163    }
5164
5165    // if you want get_account_modified_since_parent without fixed_root, please define so...
5166    fn get_account_modified_since_parent_with_fixed_root(
5167        &self,
5168        pubkey: &Pubkey,
5169    ) -> Option<(AccountSharedData, Slot)> {
5170        let just_self: Ancestors = Ancestors::from(vec![self.slot()]);
5171        if let Some((account, slot)) = self.load_slow_with_fixed_root(&just_self, pubkey) {
5172            if slot == self.slot() {
5173                return Some((account, slot));
5174            }
5175        }
5176        None
5177    }
5178
5179    pub fn get_largest_accounts(
5180        &self,
5181        num: usize,
5182        filter_by_address: &HashSet<Pubkey>,
5183        filter: AccountAddressFilter,
5184        sort_results: bool,
5185    ) -> ScanResult<Vec<(Pubkey, u64)>> {
5186        self.rc.accounts.load_largest_accounts(
5187            &self.ancestors,
5188            self.bank_id,
5189            num,
5190            filter_by_address,
5191            filter,
5192            sort_results,
5193        )
5194    }
5195
5196    /// Return the accumulated executed transaction count
5197    pub fn transaction_count(&self) -> u64 {
5198        self.transaction_count.load(Relaxed)
5199    }
5200
5201    /// Returns the number of non-vote transactions processed without error
5202    /// since the most recent boot from snapshot or genesis.
5203    /// This value is not shared though the network, nor retained
5204    /// within snapshots, but is preserved in `Bank::new_from_parent`.
5205    pub fn non_vote_transaction_count_since_restart(&self) -> u64 {
5206        self.non_vote_transaction_count_since_restart.load(Relaxed)
5207    }
5208
5209    /// Return the transaction count executed only in this bank
5210    pub fn executed_transaction_count(&self) -> u64 {
5211        self.transaction_count()
5212            .saturating_sub(self.parent().map_or(0, |parent| parent.transaction_count()))
5213    }
5214
5215    pub fn transaction_error_count(&self) -> u64 {
5216        self.transaction_error_count.load(Relaxed)
5217    }
5218
5219    pub fn transaction_entries_count(&self) -> u64 {
5220        self.transaction_entries_count.load(Relaxed)
5221    }
5222
5223    pub fn transactions_per_entry_max(&self) -> u64 {
5224        self.transactions_per_entry_max.load(Relaxed)
5225    }
5226
5227    fn increment_transaction_count(&self, tx_count: u64) {
5228        self.transaction_count.fetch_add(tx_count, Relaxed);
5229    }
5230
5231    fn increment_non_vote_transaction_count_since_restart(&self, tx_count: u64) {
5232        self.non_vote_transaction_count_since_restart
5233            .fetch_add(tx_count, Relaxed);
5234    }
5235
5236    pub fn signature_count(&self) -> u64 {
5237        self.signature_count.load(Relaxed)
5238    }
5239
5240    fn increment_signature_count(&self, signature_count: u64) {
5241        self.signature_count.fetch_add(signature_count, Relaxed);
5242    }
5243
5244    pub fn get_signature_status_processed_since_parent(
5245        &self,
5246        signature: &Signature,
5247    ) -> Option<Result<()>> {
5248        if let Some((slot, status)) = self.get_signature_status_slot(signature) {
5249            if slot <= self.slot() {
5250                return Some(status);
5251            }
5252        }
5253        None
5254    }
5255
5256    pub fn get_signature_status_with_blockhash(
5257        &self,
5258        signature: &Signature,
5259        blockhash: &Hash,
5260    ) -> Option<Result<()>> {
5261        let rcache = self.status_cache.read().unwrap();
5262        rcache
5263            .get_status(signature, blockhash, &self.ancestors)
5264            .map(|v| v.1)
5265    }
5266
5267    pub fn get_signature_status_slot(&self, signature: &Signature) -> Option<(Slot, Result<()>)> {
5268        let rcache = self.status_cache.read().unwrap();
5269        rcache.get_status_any_blockhash(signature, &self.ancestors)
5270    }
5271
5272    pub fn get_signature_status(&self, signature: &Signature) -> Option<Result<()>> {
5273        self.get_signature_status_slot(signature).map(|v| v.1)
5274    }
5275
5276    pub fn has_signature(&self, signature: &Signature) -> bool {
5277        self.get_signature_status_slot(signature).is_some()
5278    }
5279
5280    /// Hash the `accounts` HashMap. This represents a validator's interpretation
5281    ///  of the delta of the ledger since the last vote and up to now
5282    fn hash_internal_state(&self) -> Hash {
5283        let measure_total = Measure::start("");
5284        let slot = self.slot();
5285
5286        let delta_hash_info = (!self
5287            .feature_set
5288            .is_active(&feature_set::remove_accounts_delta_hash::id()))
5289        .then(|| {
5290            measure_us!({
5291                self.rc
5292                    .accounts
5293                    .accounts_db
5294                    .calculate_accounts_delta_hash_internal(
5295                        slot,
5296                        None,
5297                        self.skipped_rewrites.lock().unwrap().clone(),
5298                    )
5299            })
5300        });
5301
5302        let mut hash = if let Some((accounts_delta_hash, _measure)) = delta_hash_info.as_ref() {
5303            hashv(&[
5304                self.parent_hash.as_ref(),
5305                accounts_delta_hash.0.as_ref(),
5306                &self.signature_count().to_le_bytes(),
5307                self.last_blockhash().as_ref(),
5308            ])
5309        } else {
5310            hashv(&[
5311                self.parent_hash.as_ref(),
5312                &self.signature_count().to_le_bytes(),
5313                self.last_blockhash().as_ref(),
5314            ])
5315        };
5316
5317        let accounts_hash_info = if self
5318            .feature_set
5319            .is_active(&feature_set::accounts_lt_hash::id())
5320        {
5321            let accounts_lt_hash = &*self.accounts_lt_hash.lock().unwrap();
5322            let lt_hash_bytes = bytemuck::must_cast_slice(&accounts_lt_hash.0 .0);
5323            hash = hashv(&[hash.as_ref(), lt_hash_bytes]);
5324            let checksum = accounts_lt_hash.0.checksum();
5325            Some(format!(", accounts_lt_hash checksum: {checksum}"))
5326        } else {
5327            let epoch_accounts_hash = self.wait_get_epoch_accounts_hash();
5328            epoch_accounts_hash.map(|epoch_accounts_hash| {
5329                hash = hashv(&[hash.as_ref(), epoch_accounts_hash.as_ref().as_ref()]);
5330                format!(", epoch_accounts_hash: {:?}", epoch_accounts_hash.as_ref())
5331            })
5332        };
5333
5334        let buf = self
5335            .hard_forks
5336            .read()
5337            .unwrap()
5338            .get_hash_data(slot, self.parent_slot());
5339        if let Some(buf) = buf {
5340            let hard_forked_hash = extend_and_hash(&hash, &buf);
5341            warn!("hard fork at slot {slot} by hashing {buf:?}: {hash} => {hard_forked_hash}");
5342            hash = hard_forked_hash;
5343        }
5344
5345        #[cfg(feature = "dev-context-only-utils")]
5346        let hash_override = self
5347            .hash_overrides
5348            .lock()
5349            .unwrap()
5350            .get_bank_hash_override(slot)
5351            .copied()
5352            .inspect(|&hash_override| {
5353                if hash_override != hash {
5354                    info!(
5355                        "bank: slot: {}: overrode bank hash: {} with {}",
5356                        self.slot(),
5357                        hash,
5358                        hash_override
5359                    );
5360                }
5361            });
5362        // Avoid to optimize out `hash` along with the whole computation by super smart rustc.
5363        // hash_override is used by ledger-tool's simulate-block-production, which prefers
5364        // the actual bank freezing processing for accurate simulation.
5365        #[cfg(feature = "dev-context-only-utils")]
5366        let hash = hash_override.unwrap_or(std::hint::black_box(hash));
5367
5368        let bank_hash_stats = self.bank_hash_stats.load();
5369
5370        let total_us = measure_total.end_as_us();
5371
5372        let (accounts_delta_hash_us, accounts_delta_hash_log) = delta_hash_info
5373            .map(|(hash, us)| (us, format!(" accounts_delta: {}", hash.0)))
5374            .unzip();
5375        datapoint_info!(
5376            "bank-hash_internal_state",
5377            ("slot", slot, i64),
5378            ("total_us", total_us, i64),
5379            ("accounts_delta_hash_us", accounts_delta_hash_us, Option<i64>),
5380        );
5381        info!(
5382            "bank frozen: {slot} hash: {hash}{} signature_count: {} last_blockhash: {} capitalization: {}{}, stats: {bank_hash_stats:?}",
5383            accounts_delta_hash_log.unwrap_or_default(),
5384            self.signature_count(),
5385            self.last_blockhash(),
5386            self.capitalization(),
5387            accounts_hash_info.unwrap_or_default(),
5388        );
5389        hash
5390    }
5391
5392    pub fn collector_fees(&self) -> u64 {
5393        self.collector_fees.load(Relaxed)
5394    }
5395
5396    /// The epoch accounts hash is hashed into the bank's hash once per epoch at a predefined slot.
5397    /// Should it be included in *this* bank?
5398    fn should_include_epoch_accounts_hash(&self) -> bool {
5399        if !epoch_accounts_hash_utils::is_enabled_this_epoch(self) {
5400            return false;
5401        }
5402
5403        let stop_slot = epoch_accounts_hash_utils::calculation_stop(self);
5404        self.parent_slot() < stop_slot && self.slot() >= stop_slot
5405    }
5406
5407    /// If the epoch accounts hash should be included in this Bank, then fetch it. If the EAH
5408    /// calculation has not completed yet, this fn will block until it does complete.
5409    fn wait_get_epoch_accounts_hash(&self) -> Option<EpochAccountsHash> {
5410        if !self.should_include_epoch_accounts_hash() {
5411            return None;
5412        }
5413
5414        let (epoch_accounts_hash, waiting_time_us) = measure_us!(self
5415            .rc
5416            .accounts
5417            .accounts_db
5418            .epoch_accounts_hash_manager
5419            .wait_get_epoch_accounts_hash());
5420
5421        datapoint_info!(
5422            "bank-wait_get_epoch_accounts_hash",
5423            ("slot", self.slot(), i64),
5424            ("waiting-time-us", waiting_time_us, i64),
5425        );
5426        Some(epoch_accounts_hash)
5427    }
5428
5429    /// Used by ledger tool to run a final hash calculation once all ledger replay has completed.
5430    /// This should not be called by validator code.
5431    pub fn run_final_hash_calc(&self, on_halt_store_hash_raw_data_for_debug: bool) {
5432        self.force_flush_accounts_cache();
5433        // note that this slot may not be a root
5434        _ = self.verify_accounts_hash(
5435            None,
5436            VerifyAccountsHashConfig {
5437                test_hash_calculation: false,
5438                ignore_mismatch: true,
5439                require_rooted_bank: false,
5440                run_in_background: false,
5441                store_hash_raw_data_for_debug: on_halt_store_hash_raw_data_for_debug,
5442            },
5443            None,
5444        );
5445    }
5446
5447    /// Recalculate the accounts hash from the account stores. Used to verify a snapshot.
5448    /// return true if all is good
5449    /// Only called from startup or test code.
5450    #[must_use]
5451    fn verify_accounts_hash(
5452        &self,
5453        base: Option<(Slot, /*capitalization*/ u64)>,
5454        mut config: VerifyAccountsHashConfig,
5455        duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
5456    ) -> bool {
5457        #[derive(Debug, Eq, PartialEq)]
5458        enum VerifyKind {
5459            Merkle,
5460            Lattice,
5461        }
5462
5463        let accounts = &self.rc.accounts;
5464        // Wait until initial hash calc is complete before starting a new hash calc.
5465        // This should only occur when we halt at a slot in ledger-tool.
5466        accounts
5467            .accounts_db
5468            .verify_accounts_hash_in_bg
5469            .join_background_thread();
5470
5471        let slot = self.slot();
5472
5473        let verify_kind = match (
5474            duplicates_lt_hash.is_some(),
5475            self.rc
5476                .accounts
5477                .accounts_db
5478                .is_experimental_accumulator_hash_enabled(),
5479        ) {
5480            (true, _) => VerifyKind::Lattice,
5481            (false, false) => VerifyKind::Merkle,
5482            (false, true) => {
5483                // Calculating the accounts lt hash from storages *requires* a duplicates_lt_hash.
5484                // If it is None here, then we must use the index instead, which also means we
5485                // cannot run in the background.
5486                config.run_in_background = false;
5487                VerifyKind::Lattice
5488            }
5489        };
5490
5491        if config.require_rooted_bank && !accounts.accounts_db.accounts_index.is_alive_root(slot) {
5492            if let Some(parent) = self.parent() {
5493                info!(
5494                    "slot {slot} is not a root, so verify accounts hash on parent bank at slot {}",
5495                    parent.slot(),
5496                );
5497                if verify_kind == VerifyKind::Lattice {
5498                    // The duplicates_lt_hash is only valid for the current slot, so we must fall
5499                    // back to verifying the accounts lt hash with the index (which also means we
5500                    // cannot run in the background).
5501                    config.run_in_background = false;
5502                }
5503                return parent.verify_accounts_hash(base, config, None);
5504            } else {
5505                // this will result in mismatch errors
5506                // accounts hash calc doesn't include unrooted slots
5507                panic!("cannot verify accounts hash because slot {slot} is not a root");
5508            }
5509        }
5510
5511        // The snapshot storages must be captured *before* starting the background verification.
5512        // Otherwise, it is possible that a delayed call to `get_snapshot_storages()` will *not*
5513        // get the correct storages required to calculate and verify the accounts hashes.
5514        let snapshot_storages = self.rc.accounts.accounts_db.get_storages(RangeFull);
5515        let capitalization = self.capitalization();
5516        let verify_config = VerifyAccountsHashAndLamportsConfig {
5517            ancestors: &self.ancestors,
5518            epoch_schedule: self.epoch_schedule(),
5519            rent_collector: self.rent_collector(),
5520            test_hash_calculation: config.test_hash_calculation,
5521            ignore_mismatch: config.ignore_mismatch,
5522            store_detailed_debug_info: config.store_hash_raw_data_for_debug,
5523            use_bg_thread_pool: config.run_in_background,
5524        };
5525
5526        info!(
5527            "Verifying accounts, in background? {}, verify kind: {verify_kind:?}",
5528            config.run_in_background,
5529        );
5530        if config.run_in_background {
5531            let accounts = Arc::clone(accounts);
5532            let accounts_ = Arc::clone(&accounts);
5533            let ancestors = self.ancestors.clone();
5534            let epoch_schedule = self.epoch_schedule().clone();
5535            let rent_collector = self.rent_collector().clone();
5536            let expected_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap().clone();
5537            accounts.accounts_db.verify_accounts_hash_in_bg.start(|| {
5538                Builder::new()
5539                    .name("solBgHashVerify".into())
5540                    .spawn(move || {
5541                        info!("Initial background accounts hash verification has started");
5542                        let start = Instant::now();
5543                        let mut lattice_verify_time = None;
5544                        let mut merkle_verify_time = None;
5545                        let is_ok = match verify_kind {
5546                            VerifyKind::Lattice => {
5547                                // accounts lt hash is *enabled* so use lattice-based verification
5548                                let accounts_db = &accounts_.accounts_db;
5549                                let (calculated_accounts_lt_hash, duration) =
5550                                    meas_dur!(accounts_db.thread_pool_hash.install(|| {
5551                                        accounts_db
5552                                            .calculate_accounts_lt_hash_at_startup_from_storages(
5553                                                snapshot_storages.0.as_slice(),
5554                                                &duplicates_lt_hash.unwrap(),
5555                                            )
5556                                    }));
5557                                let is_ok =
5558                                    calculated_accounts_lt_hash == expected_accounts_lt_hash;
5559                                if !is_ok {
5560                                    let expected = expected_accounts_lt_hash.0.checksum();
5561                                    let calculated = calculated_accounts_lt_hash.0.checksum();
5562                                    error!(
5563                                        "Verifying accounts failed: accounts lattice hashes do not \
5564                                         match, expected: {expected}, calculated: {calculated}",
5565                                    );
5566                                }
5567                                lattice_verify_time = Some(duration);
5568                                is_ok
5569                            }
5570                            VerifyKind::Merkle => {
5571                                // accounts lt hash is *disabled* so use merkle-based verification
5572                                let snapshot_storages_and_slots = (
5573                                    snapshot_storages.0.as_slice(),
5574                                    snapshot_storages.1.as_slice(),
5575                                );
5576                                let (is_ok, duration) = meas_dur!(accounts_
5577                                    .verify_accounts_hash_and_lamports(
5578                                        snapshot_storages_and_slots,
5579                                        slot,
5580                                        capitalization,
5581                                        base,
5582                                        VerifyAccountsHashAndLamportsConfig {
5583                                            ancestors: &ancestors,
5584                                            epoch_schedule: &epoch_schedule,
5585                                            rent_collector: &rent_collector,
5586                                            ..verify_config
5587                                        },
5588                                    ));
5589                                merkle_verify_time = Some(duration);
5590                                is_ok
5591                            }
5592                        };
5593                        accounts_
5594                            .accounts_db
5595                            .verify_accounts_hash_in_bg
5596                            .background_finished();
5597                        let total_time = start.elapsed();
5598                        datapoint_info!(
5599                            "startup_verify_accounts",
5600                            ("total_us", total_time.as_micros(), i64),
5601                            (
5602                                "verify_accounts_lt_hash_us",
5603                                lattice_verify_time.as_ref().map(Duration::as_micros),
5604                                Option<i64>
5605                            ),
5606                            ("verify_accounts_hash_us",
5607                                merkle_verify_time.as_ref().map(Duration::as_micros),
5608                                Option<i64>
5609                            ),
5610                        );
5611                        info!("Initial background accounts hash verification has stopped");
5612                        is_ok
5613                    })
5614                    .unwrap()
5615            });
5616            true // initial result is true. We haven't failed yet. If verification fails, we'll panic from bg thread.
5617        } else {
5618            match verify_kind {
5619                VerifyKind::Lattice => {
5620                    let expected_accounts_lt_hash = self.accounts_lt_hash.lock().unwrap().clone();
5621                    let calculated_accounts_lt_hash = if let Some(duplicates_lt_hash) =
5622                        duplicates_lt_hash
5623                    {
5624                        accounts
5625                            .accounts_db
5626                            .calculate_accounts_lt_hash_at_startup_from_storages(
5627                                snapshot_storages.0.as_slice(),
5628                                &duplicates_lt_hash,
5629                            )
5630                    } else {
5631                        accounts
5632                            .accounts_db
5633                            .calculate_accounts_lt_hash_at_startup_from_index(&self.ancestors, slot)
5634                    };
5635                    let is_ok = calculated_accounts_lt_hash == expected_accounts_lt_hash;
5636                    if !is_ok {
5637                        let expected = expected_accounts_lt_hash.0.checksum();
5638                        let calculated = calculated_accounts_lt_hash.0.checksum();
5639                        error!(
5640                            "Verifying accounts failed: accounts lattice hashes do not \
5641                             match, expected: {expected}, calculated: {calculated}",
5642                        );
5643                    }
5644                    is_ok
5645                }
5646                VerifyKind::Merkle => {
5647                    let snapshot_storages_and_slots = (
5648                        snapshot_storages.0.as_slice(),
5649                        snapshot_storages.1.as_slice(),
5650                    );
5651                    let is_ok = accounts.verify_accounts_hash_and_lamports(
5652                        snapshot_storages_and_slots,
5653                        slot,
5654                        capitalization,
5655                        base,
5656                        verify_config,
5657                    );
5658                    self.set_initial_accounts_hash_verification_completed();
5659                    is_ok
5660                }
5661            }
5662        }
5663    }
5664
5665    /// Specify that initial verification has completed.
5666    /// Called internally when verification runs in the foreground thread.
5667    /// Also has to be called by some tests which don't do verification on startup.
5668    pub fn set_initial_accounts_hash_verification_completed(&self) {
5669        self.rc
5670            .accounts
5671            .accounts_db
5672            .verify_accounts_hash_in_bg
5673            .verification_complete();
5674    }
5675
5676    /// return true if bg hash verification is complete
5677    /// return false if bg hash verification has not completed yet
5678    /// if hash verification failed, a panic will occur
5679    pub fn has_initial_accounts_hash_verification_completed(&self) -> bool {
5680        self.rc
5681            .accounts
5682            .accounts_db
5683            .verify_accounts_hash_in_bg
5684            .check_complete()
5685    }
5686
5687    /// Get this bank's storages to use for snapshots.
5688    ///
5689    /// If a base slot is provided, return only the storages that are *higher* than this slot.
5690    pub fn get_snapshot_storages(&self, base_slot: Option<Slot>) -> Vec<Arc<AccountStorageEntry>> {
5691        // if a base slot is provided, request storages starting at the slot *after*
5692        let start_slot = base_slot.map_or(0, |slot| slot.saturating_add(1));
5693        // we want to *include* the storage at our slot
5694        let requested_slots = start_slot..=self.slot();
5695
5696        self.rc.accounts.accounts_db.get_storages(requested_slots).0
5697    }
5698
5699    #[must_use]
5700    fn verify_hash(&self) -> bool {
5701        assert!(self.is_frozen());
5702        let calculated_hash = self.hash_internal_state();
5703        let expected_hash = self.hash();
5704
5705        if calculated_hash == expected_hash {
5706            true
5707        } else {
5708            warn!(
5709                "verify failed: slot: {}, {} (calculated) != {} (expected)",
5710                self.slot(),
5711                calculated_hash,
5712                expected_hash
5713            );
5714            false
5715        }
5716    }
5717
5718    pub fn verify_transaction(
5719        &self,
5720        tx: VersionedTransaction,
5721        verification_mode: TransactionVerificationMode,
5722    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
5723        let sanitized_tx = {
5724            let size =
5725                bincode::serialized_size(&tx).map_err(|_| TransactionError::SanitizeFailure)?;
5726            if size > PACKET_DATA_SIZE as u64 {
5727                return Err(TransactionError::SanitizeFailure);
5728            }
5729            let message_hash = if verification_mode == TransactionVerificationMode::FullVerification
5730            {
5731                tx.verify_and_hash_message()?
5732            } else {
5733                tx.message.hash()
5734            };
5735
5736            RuntimeTransaction::try_create(
5737                tx,
5738                MessageHash::Precomputed(message_hash),
5739                None,
5740                self,
5741                self.get_reserved_account_keys(),
5742            )
5743        }?;
5744
5745        Ok(sanitized_tx)
5746    }
5747
5748    pub fn fully_verify_transaction(
5749        &self,
5750        tx: VersionedTransaction,
5751    ) -> Result<RuntimeTransaction<SanitizedTransaction>> {
5752        self.verify_transaction(tx, TransactionVerificationMode::FullVerification)
5753    }
5754
5755    /// Checks if the transaction violates the bank's reserved keys.
5756    /// This needs to be checked upon epoch boundary crosses because the
5757    /// reserved key set may have changed since the initial sanitization.
5758    pub fn check_reserved_keys(&self, tx: &impl SVMMessage) -> Result<()> {
5759        // Check keys against the reserved set - these failures simply require us
5760        // to re-sanitize the transaction. We do not need to drop the transaction.
5761        let reserved_keys = self.get_reserved_account_keys();
5762        for (index, key) in tx.account_keys().iter().enumerate() {
5763            if tx.is_writable(index) && reserved_keys.contains(key) {
5764                return Err(TransactionError::ResanitizationNeeded);
5765            }
5766        }
5767
5768        Ok(())
5769    }
5770
5771    /// only called from ledger-tool or tests
5772    fn calculate_capitalization(&self, debug_verify: bool) -> u64 {
5773        let is_startup = true;
5774        self.rc
5775            .accounts
5776            .accounts_db
5777            .verify_accounts_hash_in_bg
5778            .join_background_thread();
5779        self.rc
5780            .accounts
5781            .accounts_db
5782            .update_accounts_hash_with_verify_from(
5783                // we have to use the index since the slot could be in the write cache still
5784                CalcAccountsHashDataSource::IndexForTests,
5785                debug_verify,
5786                self.slot(),
5787                &self.ancestors,
5788                None,
5789                self.epoch_schedule(),
5790                &self.rent_collector,
5791                is_startup,
5792            )
5793            .1
5794    }
5795
5796    /// only called from tests or ledger tool
5797    pub fn calculate_and_verify_capitalization(&self, debug_verify: bool) -> bool {
5798        let calculated = self.calculate_capitalization(debug_verify);
5799        let expected = self.capitalization();
5800        if calculated == expected {
5801            true
5802        } else {
5803            warn!(
5804                "Capitalization mismatch: calculated: {} != expected: {}",
5805                calculated, expected
5806            );
5807            false
5808        }
5809    }
5810
5811    /// Forcibly overwrites current capitalization by actually recalculating accounts' balances.
5812    /// This should only be used for developing purposes.
5813    pub fn set_capitalization(&self) -> u64 {
5814        let old = self.capitalization();
5815        // We cannot debug verify the hash calculation here because calculate_capitalization will use the index calculation due to callers using the write cache.
5816        // 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
5817        // for example.
5818        let debug_verify = false;
5819        self.capitalization
5820            .store(self.calculate_capitalization(debug_verify), Relaxed);
5821        old
5822    }
5823
5824    /// Returns the `AccountsHash` that was calculated for this bank's slot
5825    ///
5826    /// This fn is used when creating a snapshot with ledger-tool, or when
5827    /// packaging a snapshot into an archive (used to get the `SnapshotHash`).
5828    pub fn get_accounts_hash(&self) -> Option<AccountsHash> {
5829        self.rc
5830            .accounts
5831            .accounts_db
5832            .get_accounts_hash(self.slot())
5833            .map(|(accounts_hash, _)| accounts_hash)
5834    }
5835
5836    /// Returns the `IncrementalAccountsHash` that was calculated for this bank's slot
5837    ///
5838    /// This fn is used when creating an incremental snapshot with ledger-tool, or when
5839    /// packaging a snapshot into an archive (used to get the `SnapshotHash`).
5840    pub fn get_incremental_accounts_hash(&self) -> Option<IncrementalAccountsHash> {
5841        self.rc
5842            .accounts
5843            .accounts_db
5844            .get_incremental_accounts_hash(self.slot())
5845            .map(|(incremental_accounts_hash, _)| incremental_accounts_hash)
5846    }
5847
5848    /// Returns the `SnapshotHash` for this bank's slot
5849    ///
5850    /// This fn is used at startup to verify the bank was rebuilt correctly.
5851    ///
5852    /// # Panics
5853    ///
5854    /// If the snapshots lt hash feature is not enabled, panics if there is both-or-neither of an
5855    /// `AccountsHash` and an `IncrementalAccountsHash` for this bank's slot.  There may only be
5856    /// one or the other.
5857    pub fn get_snapshot_hash(&self) -> SnapshotHash {
5858        if self.is_snapshots_lt_hash_enabled() {
5859            self.get_lattice_snapshot_hash()
5860        } else {
5861            self.get_merkle_snapshot_hash()
5862        }
5863    }
5864
5865    /// Returns the merkle-based `SnapshotHash` for this bank's slot
5866    ///
5867    /// This fn is used at startup to verify the bank was rebuilt correctly.
5868    ///
5869    /// # Panics
5870    ///
5871    /// If the snapshots lt hash feature is not enabled, panics if there is both-or-neither of an
5872    /// `AccountsHash` and an `IncrementalAccountsHash` for this bank's slot.  There may only be
5873    /// one or the other.
5874    pub fn get_merkle_snapshot_hash(&self) -> SnapshotHash {
5875        let accounts_hash = self.get_accounts_hash();
5876        let incremental_accounts_hash = self.get_incremental_accounts_hash();
5877        let accounts_hash_kind = match (accounts_hash, incremental_accounts_hash) {
5878            (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()),
5879            (Some(accounts_hash), None) => accounts_hash.into(),
5880            (None, Some(incremental_accounts_hash)) => incremental_accounts_hash.into(),
5881            (None, None) => panic!("accounts hash is required to get snapshot hash"),
5882        };
5883        let epoch_accounts_hash = self.get_epoch_accounts_hash_to_serialize();
5884        SnapshotHash::new(
5885            &MerkleOrLatticeAccountsHash::Merkle(accounts_hash_kind),
5886            epoch_accounts_hash.as_ref(),
5887            None,
5888        )
5889    }
5890
5891    /// Returns the lattice-based `SnapshotHash` for this bank's slot
5892    ///
5893    /// This fn is used at startup to verify the bank was rebuilt correctly.
5894    pub fn get_lattice_snapshot_hash(&self) -> SnapshotHash {
5895        SnapshotHash::new(
5896            &MerkleOrLatticeAccountsHash::Lattice,
5897            None,
5898            Some(self.accounts_lt_hash.lock().unwrap().0.checksum()),
5899        )
5900    }
5901
5902    pub fn load_account_into_read_cache(&self, key: &Pubkey) {
5903        self.rc
5904            .accounts
5905            .accounts_db
5906            .load_account_into_read_cache(&self.ancestors, key);
5907    }
5908
5909    pub fn update_accounts_hash(
5910        &self,
5911        data_source: CalcAccountsHashDataSource,
5912        is_startup: bool,
5913    ) -> AccountsHash {
5914        let (accounts_hash, total_lamports) = self
5915            .rc
5916            .accounts
5917            .accounts_db
5918            .update_accounts_hash_with_verify_from(
5919                data_source,
5920                false, // debug_verify
5921                self.slot(),
5922                &self.ancestors,
5923                Some(self.capitalization()),
5924                self.epoch_schedule(),
5925                &self.rent_collector,
5926                is_startup,
5927            );
5928        if total_lamports != self.capitalization() {
5929            datapoint_info!(
5930                "capitalization_mismatch",
5931                ("slot", self.slot(), i64),
5932                ("calculated_lamports", total_lamports, i64),
5933                ("capitalization", self.capitalization(), i64),
5934            );
5935
5936            // cap mismatch detected. It has been logged to metrics above.
5937            // Run both versions of the calculation to attempt to get more info.
5938            let debug_verify = true;
5939            self.rc
5940                .accounts
5941                .accounts_db
5942                .update_accounts_hash_with_verify_from(
5943                    data_source,
5944                    debug_verify,
5945                    self.slot(),
5946                    &self.ancestors,
5947                    Some(self.capitalization()),
5948                    self.epoch_schedule(),
5949                    &self.rent_collector,
5950                    is_startup,
5951                );
5952
5953            panic!(
5954                "capitalization_mismatch. slot: {}, calculated_lamports: {}, capitalization: {}",
5955                self.slot(),
5956                total_lamports,
5957                self.capitalization()
5958            );
5959        }
5960        accounts_hash
5961    }
5962
5963    /// Calculate the incremental accounts hash from `base_slot` to `self`
5964    pub fn update_incremental_accounts_hash(&self, base_slot: Slot) -> IncrementalAccountsHash {
5965        let config = CalcAccountsHashConfig {
5966            use_bg_thread_pool: true,
5967            ancestors: None, // does not matter, will not be used
5968            epoch_schedule: &self.epoch_schedule,
5969            rent_collector: &self.rent_collector,
5970            store_detailed_debug_info_on_failure: false,
5971        };
5972        let storages = self.get_snapshot_storages(Some(base_slot));
5973        let sorted_storages = SortedStorages::new(&storages);
5974        self.rc
5975            .accounts
5976            .accounts_db
5977            .update_incremental_accounts_hash(
5978                &config,
5979                &sorted_storages,
5980                self.slot(),
5981                HashStats::default(),
5982            )
5983            .0
5984    }
5985
5986    /// A snapshot bank should be purged of 0 lamport accounts which are not part of the hash
5987    /// calculation and could shield other real accounts.
5988    pub fn verify_snapshot_bank(
5989        &self,
5990        test_hash_calculation: bool,
5991        skip_shrink: bool,
5992        force_clean: bool,
5993        latest_full_snapshot_slot: Slot,
5994        base: Option<(Slot, /*capitalization*/ u64)>,
5995        duplicates_lt_hash: Option<Box<DuplicatesLtHash>>,
5996    ) -> bool {
5997        // If we verify the accounts using the lattice-based hash *and* with storages (as opposed
5998        // to the index), then we rely on the DuplicatesLtHash as given by generate_index().  Since
5999        // the duplicates are based on a specific set of storages, we must use the exact same
6000        // storages to do the lattice-based accounts verification.  This means we must wait to
6001        // clean/shrink until *after* we've gotten Arcs to the storages (this prevents their
6002        // untimely removal).  Simply, we call `verify_accounts_hash()` before we call `clean` or
6003        // `shrink`.
6004        let (verified_accounts, verify_accounts_time_us) = measure_us!({
6005            let should_verify_accounts = !self.rc.accounts.accounts_db.skip_initial_hash_calc;
6006            if should_verify_accounts {
6007                info!("Verifying accounts...");
6008                let verified = self.verify_accounts_hash(
6009                    base,
6010                    VerifyAccountsHashConfig {
6011                        test_hash_calculation,
6012                        ignore_mismatch: false,
6013                        require_rooted_bank: false,
6014                        run_in_background: true,
6015                        store_hash_raw_data_for_debug: false,
6016                    },
6017                    duplicates_lt_hash,
6018                );
6019                info!("Verifying accounts... In background.");
6020                verified
6021            } else {
6022                info!("Verifying accounts... Skipped.");
6023                self.rc
6024                    .accounts
6025                    .accounts_db
6026                    .verify_accounts_hash_in_bg
6027                    .verification_complete();
6028                true
6029            }
6030        });
6031
6032        let (_, clean_time_us) = measure_us!({
6033            let should_clean = force_clean || (!skip_shrink && self.slot() > 0);
6034            if should_clean {
6035                info!("Cleaning...");
6036                // We cannot clean past the latest full snapshot's slot because we are about to
6037                // perform an accounts hash calculation *up to that slot*.  If we cleaned *past*
6038                // that slot, then accounts could be removed from older storages, which would
6039                // change the accounts hash.
6040                self.rc.accounts.accounts_db.clean_accounts(
6041                    Some(latest_full_snapshot_slot),
6042                    true,
6043                    self.epoch_schedule(),
6044                    self.clean_accounts_old_storages_policy(),
6045                );
6046                info!("Cleaning... Done.");
6047            } else {
6048                info!("Cleaning... Skipped.");
6049            }
6050        });
6051
6052        let (_, shrink_time_us) = measure_us!({
6053            let should_shrink = !skip_shrink && self.slot() > 0;
6054            if should_shrink {
6055                info!("Shrinking...");
6056                self.rc.accounts.accounts_db.shrink_all_slots(
6057                    true,
6058                    self.epoch_schedule(),
6059                    // we cannot allow the snapshot slot to be shrunk
6060                    Some(self.slot()),
6061                );
6062                info!("Shrinking... Done.");
6063            } else {
6064                info!("Shrinking... Skipped.");
6065            }
6066        });
6067
6068        info!("Verifying bank...");
6069        let (verified_bank, verify_bank_time_us) = measure_us!(self.verify_hash());
6070        info!("Verifying bank... Done.");
6071
6072        datapoint_info!(
6073            "verify_snapshot_bank",
6074            ("clean_us", clean_time_us, i64),
6075            ("shrink_us", shrink_time_us, i64),
6076            ("verify_accounts_us", verify_accounts_time_us, i64),
6077            ("verify_bank_us", verify_bank_time_us, i64),
6078        );
6079
6080        verified_accounts && verified_bank
6081    }
6082
6083    /// Return the number of hashes per tick
6084    pub fn hashes_per_tick(&self) -> &Option<u64> {
6085        &self.hashes_per_tick
6086    }
6087
6088    /// Return the number of ticks per slot
6089    pub fn ticks_per_slot(&self) -> u64 {
6090        self.ticks_per_slot
6091    }
6092
6093    /// Return the number of slots per year
6094    pub fn slots_per_year(&self) -> f64 {
6095        self.slots_per_year
6096    }
6097
6098    /// Return the number of ticks since genesis.
6099    pub fn tick_height(&self) -> u64 {
6100        self.tick_height.load(Relaxed)
6101    }
6102
6103    /// Return the inflation parameters of the Bank
6104    pub fn inflation(&self) -> Inflation {
6105        *self.inflation.read().unwrap()
6106    }
6107
6108    /// Return the rent collector for this Bank
6109    pub fn rent_collector(&self) -> &RentCollector {
6110        &self.rent_collector
6111    }
6112
6113    /// Return the total capitalization of the Bank
6114    pub fn capitalization(&self) -> u64 {
6115        self.capitalization.load(Relaxed)
6116    }
6117
6118    /// Return this bank's max_tick_height
6119    pub fn max_tick_height(&self) -> u64 {
6120        self.max_tick_height
6121    }
6122
6123    /// Return the block_height of this bank
6124    pub fn block_height(&self) -> u64 {
6125        self.block_height
6126    }
6127
6128    /// Return the number of slots per epoch for the given epoch
6129    pub fn get_slots_in_epoch(&self, epoch: Epoch) -> u64 {
6130        self.epoch_schedule().get_slots_in_epoch(epoch)
6131    }
6132
6133    /// returns the epoch for which this bank's leader_schedule_slot_offset and slot would
6134    ///  need to cache leader_schedule
6135    pub fn get_leader_schedule_epoch(&self, slot: Slot) -> Epoch {
6136        self.epoch_schedule().get_leader_schedule_epoch(slot)
6137    }
6138
6139    /// Returns whether the specified epoch should use the new vote account
6140    /// keyed leader schedule
6141    pub fn should_use_vote_keyed_leader_schedule(&self, epoch: Epoch) -> Option<bool> {
6142        let effective_epoch = self
6143            .feature_set
6144            .activated_slot(&agave_feature_set::enable_vote_address_leader_schedule::id())
6145            .map(|activation_slot| {
6146                // If the feature was activated at genesis, then the new leader
6147                // schedule should be effective immediately in the first epoch
6148                if activation_slot == 0 {
6149                    return 0;
6150                }
6151
6152                // Calculate the epoch that the feature became activated in
6153                let activation_epoch = self.epoch_schedule.get_epoch(activation_slot);
6154
6155                // The effective epoch is the epoch immediately after the
6156                // activation epoch
6157                activation_epoch.wrapping_add(1)
6158            });
6159
6160        // Starting from the effective epoch, always use the new leader schedule
6161        if let Some(effective_epoch) = effective_epoch {
6162            return Some(epoch >= effective_epoch);
6163        }
6164
6165        // Calculate the max epoch we can cache a leader schedule for
6166        let max_cached_leader_schedule = self.get_leader_schedule_epoch(self.slot());
6167        if epoch <= max_cached_leader_schedule {
6168            // The feature cannot be effective by the specified epoch
6169            Some(false)
6170        } else {
6171            // Cannot determine if an epoch should use the new leader schedule if the
6172            // the epoch is too far in the future because we won't know if the feature
6173            // will have been activated by then or not.
6174            None
6175        }
6176    }
6177
6178    /// a bank-level cache of vote accounts and stake delegation info
6179    fn update_stakes_cache(
6180        &self,
6181        txs: &[impl SVMMessage],
6182        processing_results: &[TransactionProcessingResult],
6183    ) {
6184        debug_assert_eq!(txs.len(), processing_results.len());
6185        let new_warmup_cooldown_rate_epoch = self.new_warmup_cooldown_rate_epoch();
6186        txs.iter()
6187            .zip(processing_results)
6188            .filter_map(|(tx, processing_result)| {
6189                processing_result
6190                    .processed_transaction()
6191                    .map(|processed_tx| (tx, processed_tx))
6192            })
6193            .filter_map(|(tx, processed_tx)| {
6194                processed_tx
6195                    .executed_transaction()
6196                    .map(|executed_tx| (tx, executed_tx))
6197            })
6198            .filter(|(_, executed_tx)| executed_tx.was_successful())
6199            .flat_map(|(tx, executed_tx)| {
6200                let num_account_keys = tx.account_keys().len();
6201                let loaded_tx = &executed_tx.loaded_transaction;
6202                loaded_tx.accounts.iter().take(num_account_keys)
6203            })
6204            .for_each(|(pubkey, account)| {
6205                // note that this could get timed to: self.rc.accounts.accounts_db.stats.stakes_cache_check_and_store_us,
6206                //  but this code path is captured separately in ExecuteTimingType::UpdateStakesCacheUs
6207                self.stakes_cache
6208                    .check_and_store(pubkey, account, new_warmup_cooldown_rate_epoch);
6209            });
6210    }
6211
6212    /// current vote accounts for this bank along with the stake
6213    ///   attributed to each account
6214    pub fn vote_accounts(&self) -> Arc<VoteAccountsHashMap> {
6215        let stakes = self.stakes_cache.stakes();
6216        Arc::from(stakes.vote_accounts())
6217    }
6218
6219    /// Vote account for the given vote account pubkey.
6220    pub fn get_vote_account(&self, vote_account: &Pubkey) -> Option<VoteAccount> {
6221        let stakes = self.stakes_cache.stakes();
6222        let vote_account = stakes.vote_accounts().get(vote_account)?;
6223        Some(vote_account.clone())
6224    }
6225
6226    /// Get the EpochStakes for the current Bank::epoch
6227    pub fn current_epoch_stakes(&self) -> &EpochStakes {
6228        // The stakes for a given epoch (E) in self.epoch_stakes are keyed by leader schedule epoch
6229        // (E + 1) so the stakes for the current epoch are stored at self.epoch_stakes[E + 1]
6230        self.epoch_stakes
6231            .get(&self.epoch.saturating_add(1))
6232            .expect("Current epoch stakes must exist")
6233    }
6234
6235    /// Get the EpochStakes for a given epoch
6236    pub fn epoch_stakes(&self, epoch: Epoch) -> Option<&EpochStakes> {
6237        self.epoch_stakes.get(&epoch)
6238    }
6239
6240    pub fn epoch_stakes_map(&self) -> &HashMap<Epoch, EpochStakes> {
6241        &self.epoch_stakes
6242    }
6243
6244    /// Get the staked nodes map for the current Bank::epoch
6245    pub fn current_epoch_staked_nodes(&self) -> Arc<HashMap<Pubkey, u64>> {
6246        self.current_epoch_stakes().stakes().staked_nodes()
6247    }
6248
6249    pub fn epoch_staked_nodes(&self, epoch: Epoch) -> Option<Arc<HashMap<Pubkey, u64>>> {
6250        Some(self.epoch_stakes.get(&epoch)?.stakes().staked_nodes())
6251    }
6252
6253    /// Get the total epoch stake for the given epoch.
6254    pub fn epoch_total_stake(&self, epoch: Epoch) -> Option<u64> {
6255        self.epoch_stakes
6256            .get(&epoch)
6257            .map(|epoch_stakes| epoch_stakes.total_stake())
6258    }
6259
6260    /// Get the total epoch stake for the current Bank::epoch
6261    pub fn get_current_epoch_total_stake(&self) -> u64 {
6262        self.current_epoch_stakes().total_stake()
6263    }
6264
6265    /// vote accounts for the specific epoch along with the stake
6266    ///   attributed to each account
6267    pub fn epoch_vote_accounts(&self, epoch: Epoch) -> Option<&VoteAccountsHashMap> {
6268        let epoch_stakes = self.epoch_stakes.get(&epoch)?.stakes();
6269        Some(epoch_stakes.vote_accounts().as_ref())
6270    }
6271
6272    /// Get the vote accounts along with the stake attributed to each account
6273    /// for the current Bank::epoch
6274    pub fn get_current_epoch_vote_accounts(&self) -> &VoteAccountsHashMap {
6275        self.current_epoch_stakes()
6276            .stakes()
6277            .vote_accounts()
6278            .as_ref()
6279    }
6280
6281    /// Get the fixed authorized voter for the given vote account for the
6282    /// current epoch
6283    pub fn epoch_authorized_voter(&self, vote_account: &Pubkey) -> Option<&Pubkey> {
6284        self.epoch_stakes
6285            .get(&self.epoch)
6286            .expect("Epoch stakes for bank's own epoch must exist")
6287            .epoch_authorized_voters()
6288            .get(vote_account)
6289    }
6290
6291    /// Get the fixed set of vote accounts for the given node id for the
6292    /// current epoch
6293    pub fn epoch_vote_accounts_for_node_id(&self, node_id: &Pubkey) -> Option<&NodeVoteAccounts> {
6294        self.epoch_stakes
6295            .get(&self.epoch)
6296            .expect("Epoch stakes for bank's own epoch must exist")
6297            .node_id_to_vote_accounts()
6298            .get(node_id)
6299    }
6300
6301    /// Get the total stake belonging to vote accounts associated with the given node id for the
6302    /// given epoch.
6303    pub fn epoch_node_id_to_stake(&self, epoch: Epoch, node_id: &Pubkey) -> Option<u64> {
6304        self.epoch_stakes(epoch)
6305            .and_then(|epoch_stakes| epoch_stakes.node_id_to_stake(node_id))
6306    }
6307
6308    /// Get the fixed total stake of all vote accounts for current epoch
6309    pub fn total_epoch_stake(&self) -> u64 {
6310        self.epoch_stakes
6311            .get(&self.epoch)
6312            .expect("Epoch stakes for bank's own epoch must exist")
6313            .total_stake()
6314    }
6315
6316    /// Get the fixed stake of the given vote account for the current epoch
6317    pub fn epoch_vote_account_stake(&self, vote_account: &Pubkey) -> u64 {
6318        *self
6319            .epoch_vote_accounts(self.epoch())
6320            .expect("Bank epoch vote accounts must contain entry for the bank's own epoch")
6321            .get(vote_account)
6322            .map(|(stake, _)| stake)
6323            .unwrap_or(&0)
6324    }
6325
6326    /// given a slot, return the epoch and offset into the epoch this slot falls
6327    /// e.g. with a fixed number for slots_per_epoch, the calculation is simply:
6328    ///
6329    ///  ( slot/slots_per_epoch, slot % slots_per_epoch )
6330    ///
6331    pub fn get_epoch_and_slot_index(&self, slot: Slot) -> (Epoch, SlotIndex) {
6332        self.epoch_schedule().get_epoch_and_slot_index(slot)
6333    }
6334
6335    pub fn get_epoch_info(&self) -> EpochInfo {
6336        let absolute_slot = self.slot();
6337        let block_height = self.block_height();
6338        let (epoch, slot_index) = self.get_epoch_and_slot_index(absolute_slot);
6339        let slots_in_epoch = self.get_slots_in_epoch(epoch);
6340        let transaction_count = Some(self.transaction_count());
6341        EpochInfo {
6342            epoch,
6343            slot_index,
6344            slots_in_epoch,
6345            absolute_slot,
6346            block_height,
6347            transaction_count,
6348        }
6349    }
6350
6351    pub fn is_empty(&self) -> bool {
6352        !self.is_delta.load(Relaxed)
6353    }
6354
6355    pub fn add_mockup_builtin(
6356        &mut self,
6357        program_id: Pubkey,
6358        builtin_function: BuiltinFunctionWithContext,
6359    ) {
6360        self.transaction_processor.add_builtin(
6361            self,
6362            program_id,
6363            "mockup",
6364            ProgramCacheEntry::new_builtin(self.slot, 0, builtin_function),
6365        );
6366    }
6367
6368    pub fn add_precompile(&mut self, program_id: &Pubkey) {
6369        debug!("Adding precompiled program {}", program_id);
6370        self.add_precompiled_account(program_id);
6371        debug!("Added precompiled program {:?}", program_id);
6372    }
6373
6374    // Call AccountsDb::clean_accounts()
6375    //
6376    // This fn is meant to be called by the snapshot handler in Accounts Background Service.  If
6377    // calling from elsewhere, ensure the same invariants hold/expectations are met.
6378    pub(crate) fn clean_accounts(&self) {
6379        // Don't clean the slot we're snapshotting because it may have zero-lamport
6380        // accounts that were included in the bank delta hash when the bank was frozen,
6381        // and if we clean them here, any newly created snapshot's hash for this bank
6382        // may not match the frozen hash.
6383        //
6384        // So when we're snapshotting, the highest slot to clean is lowered by one.
6385        let highest_slot_to_clean = self.slot().saturating_sub(1);
6386
6387        self.rc.accounts.accounts_db.clean_accounts(
6388            Some(highest_slot_to_clean),
6389            false,
6390            self.epoch_schedule(),
6391            self.clean_accounts_old_storages_policy(),
6392        );
6393    }
6394
6395    pub fn print_accounts_stats(&self) {
6396        self.rc.accounts.accounts_db.print_accounts_stats("");
6397    }
6398
6399    pub fn shrink_candidate_slots(&self) -> usize {
6400        self.rc
6401            .accounts
6402            .accounts_db
6403            .shrink_candidate_slots(self.epoch_schedule())
6404    }
6405
6406    pub(crate) fn shrink_ancient_slots(&self) {
6407        // Invoke ancient slot shrinking only when the validator is
6408        // explicitly configured to do so. This condition may be
6409        // removed when the skip rewrites feature is enabled.
6410        if self.are_ancient_storages_enabled() {
6411            self.rc
6412                .accounts
6413                .accounts_db
6414                .shrink_ancient_slots(self.epoch_schedule())
6415        }
6416    }
6417
6418    /// Returns if ancient storages are enabled or not
6419    pub fn are_ancient_storages_enabled(&self) -> bool {
6420        let can_skip_rewrites = self.bank_hash_skips_rent_rewrites();
6421        let test_skip_rewrites_but_include_in_bank_hash = self
6422            .rc
6423            .accounts
6424            .accounts_db
6425            .test_skip_rewrites_but_include_in_bank_hash;
6426        can_skip_rewrites || test_skip_rewrites_but_include_in_bank_hash
6427    }
6428
6429    /// Returns how clean_accounts() should handle old storages
6430    pub fn clean_accounts_old_storages_policy(&self) -> OldStoragesPolicy {
6431        if self.are_ancient_storages_enabled() {
6432            OldStoragesPolicy::Leave
6433        } else {
6434            OldStoragesPolicy::Clean
6435        }
6436    }
6437
6438    pub fn read_cost_tracker(&self) -> LockResult<RwLockReadGuard<CostTracker>> {
6439        self.cost_tracker.read()
6440    }
6441
6442    pub fn write_cost_tracker(&self) -> LockResult<RwLockWriteGuard<CostTracker>> {
6443        self.cost_tracker.write()
6444    }
6445
6446    // Check if the wallclock time from bank creation to now has exceeded the allotted
6447    // time for transaction processing
6448    pub fn should_bank_still_be_processing_txs(
6449        bank_creation_time: &Instant,
6450        max_tx_ingestion_nanos: u128,
6451    ) -> bool {
6452        // Do this check outside of the PoH lock, hence not a method on PohRecorder
6453        bank_creation_time.elapsed().as_nanos() <= max_tx_ingestion_nanos
6454    }
6455
6456    pub fn deactivate_feature(&mut self, id: &Pubkey) {
6457        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
6458        feature_set.active_mut().remove(id);
6459        feature_set.inactive_mut().insert(*id);
6460        self.feature_set = Arc::new(feature_set);
6461    }
6462
6463    pub fn activate_feature(&mut self, id: &Pubkey) {
6464        let mut feature_set = Arc::make_mut(&mut self.feature_set).clone();
6465        feature_set.inactive_mut().remove(id);
6466        feature_set.active_mut().insert(*id, 0);
6467        self.feature_set = Arc::new(feature_set);
6468    }
6469
6470    pub fn fill_bank_with_ticks_for_tests(&self) {
6471        self.do_fill_bank_with_ticks_for_tests(&BankWithScheduler::no_scheduler_available())
6472    }
6473
6474    pub(crate) fn do_fill_bank_with_ticks_for_tests(&self, scheduler: &InstalledSchedulerRwLock) {
6475        if self.tick_height.load(Relaxed) < self.max_tick_height {
6476            let last_blockhash = self.last_blockhash();
6477            while self.last_blockhash() == last_blockhash {
6478                self.register_tick(&Hash::new_unique(), scheduler)
6479            }
6480        } else {
6481            warn!("Bank already reached max tick height, cannot fill it with more ticks");
6482        }
6483    }
6484
6485    /// Get a set of all actively reserved account keys that are not allowed to
6486    /// be write-locked during transaction processing.
6487    pub fn get_reserved_account_keys(&self) -> &HashSet<Pubkey> {
6488        &self.reserved_account_keys.active
6489    }
6490
6491    /// Compute and apply all activated features, initialize the transaction
6492    /// processor, and recalculate partitioned rewards if needed
6493    fn initialize_after_snapshot_restore<F, TP>(
6494        &mut self,
6495        genesis_config: &GenesisConfig,
6496        additional_builtins: Option<&[BuiltinPrototype]>,
6497        debug_do_not_add_builtins: bool,
6498        thread_pool_builder: F,
6499    ) where
6500        F: FnOnce() -> TP,
6501        TP: std::borrow::Borrow<ThreadPool>,
6502    {
6503        self.transaction_processor =
6504            TransactionBatchProcessor::new_uninitialized(self.slot, self.epoch);
6505
6506        self.finish_init(
6507            genesis_config,
6508            additional_builtins,
6509            debug_do_not_add_builtins,
6510        );
6511
6512        let thread_pool = thread_pool_builder();
6513        self.recalculate_partitioned_rewards(null_tracer(), thread_pool.borrow());
6514
6515        self.transaction_processor
6516            .fill_missing_sysvar_cache_entries(self);
6517
6518        self.rebuild_skipped_rewrites();
6519    }
6520
6521    // This is called from snapshot restore AND for each epoch boundary
6522    // The entire code path herein must be idempotent
6523    fn apply_feature_activations(
6524        &mut self,
6525        caller: ApplyFeatureActivationsCaller,
6526        debug_do_not_add_builtins: bool,
6527    ) {
6528        use ApplyFeatureActivationsCaller as Caller;
6529        let allow_new_activations = match caller {
6530            Caller::FinishInit => false,
6531            Caller::NewFromParent => true,
6532            Caller::WarpFromParent => false,
6533        };
6534        let (feature_set, new_feature_activations) =
6535            self.compute_active_feature_set(allow_new_activations);
6536        self.feature_set = Arc::new(feature_set);
6537
6538        // Update activation slot of features in `new_feature_activations`
6539        for feature_id in new_feature_activations.iter() {
6540            if let Some(mut account) = self.get_account_with_fixed_root(feature_id) {
6541                if let Some(mut feature) = feature::from_account(&account) {
6542                    feature.activated_at = Some(self.slot());
6543                    if feature::to_account(&feature, &mut account).is_some() {
6544                        self.store_account(feature_id, &account);
6545                    }
6546                    info!("Feature {} activated at slot {}", feature_id, self.slot());
6547                }
6548            }
6549        }
6550
6551        // Update active set of reserved account keys which are not allowed to be write locked
6552        self.reserved_account_keys = {
6553            let mut reserved_keys = ReservedAccountKeys::clone(&self.reserved_account_keys);
6554            reserved_keys.update_active_set(&self.feature_set);
6555            Arc::new(reserved_keys)
6556        };
6557
6558        if new_feature_activations.contains(&feature_set::pico_inflation::id()) {
6559            *self.inflation.write().unwrap() = Inflation::pico();
6560            self.fee_rate_governor.burn_percent = solana_fee_calculator::DEFAULT_BURN_PERCENT; // 50% fee burn
6561            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
6562        }
6563
6564        if !new_feature_activations.is_disjoint(&self.feature_set.full_inflation_features_enabled())
6565        {
6566            *self.inflation.write().unwrap() = Inflation::full();
6567            self.fee_rate_governor.burn_percent = solana_fee_calculator::DEFAULT_BURN_PERCENT; // 50% fee burn
6568            self.rent_collector.rent.burn_percent = 50; // 50% rent burn
6569        }
6570
6571        if !debug_do_not_add_builtins {
6572            self.apply_builtin_program_feature_transitions(
6573                allow_new_activations,
6574                &new_feature_activations,
6575            );
6576        }
6577
6578        if new_feature_activations.contains(&feature_set::accounts_lt_hash::id()) {
6579            // Activating the accounts lt hash feature means we need to have an accounts lt hash
6580            // value at the end of this if-block.  If the cli arg has been used, that means we
6581            // already have an accounts lt hash and do not need to recalculate it.
6582            if self
6583                .rc
6584                .accounts
6585                .accounts_db
6586                .is_experimental_accumulator_hash_enabled()
6587            {
6588                // We already have an accounts lt hash value, so no need to recalculate it.
6589                // Nothing else to do here.
6590            } else {
6591                let parent_slot = self.parent_slot;
6592                info!(
6593                    "Calculating the accounts lt hash for slot {parent_slot} \
6594                     as part of feature activation; this may take some time...",
6595                );
6596                // We must calculate the accounts lt hash now as part of feature activation.
6597                // Note, this bank is *not* frozen yet, which means it will later call
6598                // `update_accounts_lt_hash()`.  Therefore, we calculate the accounts lt hash based
6599                // on *our parent*, not us!
6600                let parent_ancestors = {
6601                    let mut ancestors = self.ancestors.clone();
6602                    ancestors.remove(&self.slot());
6603                    ancestors
6604                };
6605                let (parent_accounts_lt_hash, duration) = meas_dur!({
6606                    self.rc
6607                        .accounts
6608                        .accounts_db
6609                        .calculate_accounts_lt_hash_at_startup_from_index(
6610                            &parent_ancestors,
6611                            parent_slot,
6612                        )
6613                });
6614                *self.accounts_lt_hash.get_mut().unwrap() = parent_accounts_lt_hash;
6615                info!(
6616                    "Calculating the accounts lt hash for slot {parent_slot} \
6617                     completed in {duration:?}, accounts_lt_hash checksum: {}",
6618                    self.accounts_lt_hash.get_mut().unwrap().0.checksum(),
6619                );
6620            }
6621        }
6622
6623        if new_feature_activations.contains(&feature_set::raise_block_limits_to_60m::id()) {
6624            let (account_cost_limit, block_cost_limit, vote_cost_limit) = simd_0256_block_limits();
6625            self.write_cost_tracker().unwrap().set_limits(
6626                account_cost_limit,
6627                block_cost_limit,
6628                vote_cost_limit,
6629            );
6630        }
6631
6632        if new_feature_activations.contains(&feature_set::remove_accounts_delta_hash::id()) {
6633            // If the accounts delta hash has been removed, then we no longer need to compute the
6634            // AccountHash for modified accounts, and can stop the background account hasher.
6635            self.rc.accounts.accounts_db.stop_background_hasher();
6636        }
6637    }
6638
6639    fn adjust_sysvar_balance_for_rent(&self, account: &mut AccountSharedData) {
6640        account.set_lamports(
6641            self.get_minimum_balance_for_rent_exemption(account.data().len())
6642                .max(account.lamports()),
6643        );
6644    }
6645
6646    /// Compute the active feature set based on the current bank state,
6647    /// and return it together with the set of newly activated features.
6648    fn compute_active_feature_set(&self, include_pending: bool) -> (FeatureSet, AHashSet<Pubkey>) {
6649        let mut active = self.feature_set.active().clone();
6650        let mut inactive = AHashSet::new();
6651        let mut pending = AHashSet::new();
6652        let slot = self.slot();
6653
6654        for feature_id in self.feature_set.inactive() {
6655            let mut activated = None;
6656            if let Some(account) = self.get_account_with_fixed_root(feature_id) {
6657                if let Some(feature) = feature::from_account(&account) {
6658                    match feature.activated_at {
6659                        None if include_pending => {
6660                            // Feature activation is pending
6661                            pending.insert(*feature_id);
6662                            activated = Some(slot);
6663                        }
6664                        Some(activation_slot) if slot >= activation_slot => {
6665                            // Feature has been activated already
6666                            activated = Some(activation_slot);
6667                        }
6668                        _ => {}
6669                    }
6670                }
6671            }
6672            if let Some(slot) = activated {
6673                active.insert(*feature_id, slot);
6674            } else {
6675                inactive.insert(*feature_id);
6676            }
6677        }
6678
6679        (FeatureSet::new(active, inactive), pending)
6680    }
6681
6682    fn apply_builtin_program_feature_transitions(
6683        &mut self,
6684        only_apply_transitions_for_new_features: bool,
6685        new_feature_activations: &AHashSet<Pubkey>,
6686    ) {
6687        for builtin in BUILTINS.iter() {
6688            // The `builtin_is_bpf` flag is used to handle the case where a
6689            // builtin is scheduled to be enabled by one feature gate and
6690            // later migrated to Core BPF by another.
6691            //
6692            // There should never be a case where a builtin is set to be
6693            // migrated to Core BPF and is also set to be enabled on feature
6694            // activation on the same feature gate. However, the
6695            // `builtin_is_bpf` flag will handle this case as well, electing
6696            // to first attempt the migration to Core BPF.
6697            //
6698            // The migration to Core BPF will fail gracefully because the
6699            // program account will not exist. The builtin will subsequently
6700            // be enabled, but it will never be migrated to Core BPF.
6701            //
6702            // Using the same feature gate for both enabling and migrating a
6703            // builtin to Core BPF should be strictly avoided.
6704            let mut builtin_is_bpf = false;
6705            if let Some(core_bpf_migration_config) = &builtin.core_bpf_migration_config {
6706                // If the builtin is set to be migrated to Core BPF on feature
6707                // activation, perform the migration and do not add the program
6708                // to the bank's builtins. The migration will remove it from
6709                // the builtins list and the cache.
6710                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
6711                    if let Err(e) = self
6712                        .migrate_builtin_to_core_bpf(&builtin.program_id, core_bpf_migration_config)
6713                    {
6714                        warn!(
6715                            "Failed to migrate builtin {} to Core BPF: {}",
6716                            builtin.name, e
6717                        );
6718                    } else {
6719                        builtin_is_bpf = true;
6720                    }
6721                } else {
6722                    // If the builtin has already been migrated to Core BPF, do not
6723                    // add it to the bank's builtins.
6724                    builtin_is_bpf = self
6725                        .get_account(&builtin.program_id)
6726                        .map(|a| a.owner() == &bpf_loader_upgradeable::id())
6727                        .unwrap_or(false);
6728                }
6729            };
6730
6731            if let Some(feature_id) = builtin.enable_feature_id {
6732                let should_enable_builtin_on_feature_transition = !builtin_is_bpf
6733                    && if only_apply_transitions_for_new_features {
6734                        new_feature_activations.contains(&feature_id)
6735                    } else {
6736                        self.feature_set.is_active(&feature_id)
6737                    };
6738
6739                if should_enable_builtin_on_feature_transition {
6740                    self.transaction_processor.add_builtin(
6741                        self,
6742                        builtin.program_id,
6743                        builtin.name,
6744                        ProgramCacheEntry::new_builtin(
6745                            self.feature_set.activated_slot(&feature_id).unwrap_or(0),
6746                            builtin.name.len(),
6747                            builtin.entrypoint,
6748                        ),
6749                    );
6750                }
6751            }
6752        }
6753
6754        // Migrate any necessary stateless builtins to core BPF.
6755        // Stateless builtins do not have an `enable_feature_id` since they
6756        // do not exist on-chain.
6757        for stateless_builtin in STATELESS_BUILTINS.iter() {
6758            if let Some(core_bpf_migration_config) = &stateless_builtin.core_bpf_migration_config {
6759                if new_feature_activations.contains(&core_bpf_migration_config.feature_id) {
6760                    if let Err(e) = self.migrate_builtin_to_core_bpf(
6761                        &stateless_builtin.program_id,
6762                        core_bpf_migration_config,
6763                    ) {
6764                        warn!(
6765                            "Failed to migrate stateless builtin {} to Core BPF: {}",
6766                            stateless_builtin.name, e
6767                        );
6768                    }
6769                }
6770            }
6771        }
6772
6773        for precompile in get_precompiles() {
6774            let should_add_precompile = precompile
6775                .feature
6776                .as_ref()
6777                .map(|feature_id| self.feature_set.is_active(feature_id))
6778                .unwrap_or(false);
6779            if should_add_precompile {
6780                self.add_precompile(&precompile.program_id);
6781            }
6782        }
6783    }
6784
6785    /// Use to replace programs by feature activation
6786    #[allow(dead_code)]
6787    fn replace_program_account(
6788        &mut self,
6789        old_address: &Pubkey,
6790        new_address: &Pubkey,
6791        datapoint_name: &'static str,
6792    ) {
6793        if let Some(old_account) = self.get_account_with_fixed_root(old_address) {
6794            if let Some(new_account) = self.get_account_with_fixed_root(new_address) {
6795                datapoint_info!(datapoint_name, ("slot", self.slot, i64));
6796
6797                // Burn lamports in the old account
6798                self.capitalization
6799                    .fetch_sub(old_account.lamports(), Relaxed);
6800
6801                // Transfer new account to old account
6802                self.store_account(old_address, &new_account);
6803
6804                // Clear new account
6805                self.store_account(new_address, &AccountSharedData::default());
6806
6807                // Unload a program from the bank's cache
6808                self.transaction_processor
6809                    .program_cache
6810                    .write()
6811                    .unwrap()
6812                    .remove_programs([*old_address].into_iter());
6813
6814                self.calculate_and_update_accounts_data_size_delta_off_chain(
6815                    old_account.data().len(),
6816                    new_account.data().len(),
6817                );
6818            }
6819        }
6820    }
6821
6822    /// Get all the accounts for this bank and calculate stats
6823    pub fn get_total_accounts_stats(&self) -> ScanResult<TotalAccountsStats> {
6824        let accounts = self.get_all_accounts(false)?;
6825        Ok(self.calculate_total_accounts_stats(
6826            accounts
6827                .iter()
6828                .map(|(pubkey, account, _slot)| (pubkey, account)),
6829        ))
6830    }
6831
6832    /// Given all the accounts for a bank, calculate stats
6833    pub fn calculate_total_accounts_stats<'a>(
6834        &self,
6835        accounts: impl Iterator<Item = (&'a Pubkey, &'a AccountSharedData)>,
6836    ) -> TotalAccountsStats {
6837        let rent_collector = self.rent_collector();
6838        let mut total_accounts_stats = TotalAccountsStats::default();
6839        accounts.for_each(|(pubkey, account)| {
6840            total_accounts_stats.accumulate_account(pubkey, account, rent_collector);
6841        });
6842
6843        total_accounts_stats
6844    }
6845
6846    /// Must a snapshot of this bank include the EAH?
6847    pub fn must_include_epoch_accounts_hash_in_snapshot(&self) -> bool {
6848        epoch_accounts_hash_utils::is_enabled_this_epoch(self)
6849            && epoch_accounts_hash_utils::is_in_calculation_window(self)
6850    }
6851
6852    /// Get the EAH that will be used by snapshots
6853    ///
6854    /// Since snapshots are taken on roots, if the bank is in the EAH calculation window then an
6855    /// EAH *must* be included.  This means if an EAH calculation is currently in-flight we will
6856    /// wait for it to complete.
6857    pub fn get_epoch_accounts_hash_to_serialize(&self) -> Option<EpochAccountsHash> {
6858        if !self.must_include_epoch_accounts_hash_in_snapshot() {
6859            return None;
6860        }
6861
6862        let (epoch_accounts_hash, waiting_time_us) = measure_us!(self
6863            .rc
6864            .accounts
6865            .accounts_db
6866            .epoch_accounts_hash_manager
6867            .wait_get_epoch_accounts_hash());
6868
6869        datapoint_info!(
6870            "bank-get_epoch_accounts_hash_to_serialize",
6871            ("slot", self.slot(), i64),
6872            ("waiting-time-us", waiting_time_us, i64),
6873        );
6874        Some(epoch_accounts_hash)
6875    }
6876
6877    /// Convenience fn to get the Epoch Accounts Hash
6878    pub fn epoch_accounts_hash(&self) -> Option<EpochAccountsHash> {
6879        self.rc
6880            .accounts
6881            .accounts_db
6882            .epoch_accounts_hash_manager
6883            .try_get_epoch_accounts_hash()
6884    }
6885
6886    pub fn is_in_slot_hashes_history(&self, slot: &Slot) -> bool {
6887        if slot < &self.slot {
6888            if let Ok(slot_hashes) = self.transaction_processor.sysvar_cache().get_slot_hashes() {
6889                return slot_hashes.get(slot).is_some();
6890            }
6891        }
6892        false
6893    }
6894
6895    pub fn check_program_modification_slot(&self) -> bool {
6896        self.check_program_modification_slot
6897    }
6898
6899    pub fn set_check_program_modification_slot(&mut self, check: bool) {
6900        self.check_program_modification_slot = check;
6901    }
6902
6903    pub fn fee_structure(&self) -> &FeeStructure {
6904        &self.fee_structure
6905    }
6906
6907    pub fn block_id(&self) -> Option<Hash> {
6908        *self.block_id.read().unwrap()
6909    }
6910
6911    pub fn set_block_id(&self, block_id: Option<Hash>) {
6912        *self.block_id.write().unwrap() = block_id;
6913    }
6914
6915    pub fn compute_budget(&self) -> Option<ComputeBudget> {
6916        self.compute_budget
6917    }
6918
6919    pub fn add_builtin(&self, program_id: Pubkey, name: &str, builtin: ProgramCacheEntry) {
6920        self.transaction_processor
6921            .add_builtin(self, program_id, name, builtin)
6922    }
6923
6924    pub fn get_bank_hash_stats(&self) -> BankHashStats {
6925        self.bank_hash_stats.load()
6926    }
6927
6928    pub fn clear_epoch_rewards_cache(&self) {
6929        self.epoch_rewards_calculation_cache.lock().unwrap().clear();
6930    }
6931
6932    /// Sets the accounts lt hash, only to be used by SnapshotMinimizer
6933    pub fn set_accounts_lt_hash_for_snapshot_minimizer(&self, accounts_lt_hash: AccountsLtHash) {
6934        *self.accounts_lt_hash.lock().unwrap() = accounts_lt_hash;
6935    }
6936}
6937
6938impl InvokeContextCallback for Bank {
6939    fn get_epoch_stake(&self) -> u64 {
6940        self.get_current_epoch_total_stake()
6941    }
6942
6943    fn get_epoch_stake_for_vote_account(&self, vote_address: &Pubkey) -> u64 {
6944        self.get_current_epoch_vote_accounts()
6945            .get(vote_address)
6946            .map(|(stake, _)| (*stake))
6947            .unwrap_or(0)
6948    }
6949
6950    fn is_precompile(&self, program_id: &Pubkey) -> bool {
6951        is_precompile(program_id, |feature_id: &Pubkey| {
6952            self.feature_set.is_active(feature_id)
6953        })
6954    }
6955
6956    fn process_precompile(
6957        &self,
6958        program_id: &Pubkey,
6959        data: &[u8],
6960        instruction_datas: Vec<&[u8]>,
6961    ) -> std::result::Result<(), PrecompileError> {
6962        if let Some(precompile) = get_precompile(program_id, |feature_id: &Pubkey| {
6963            self.feature_set.is_active(feature_id)
6964        }) {
6965            precompile.verify(data, &instruction_datas, &self.feature_set)
6966        } else {
6967            Err(PrecompileError::InvalidPublicKey)
6968        }
6969    }
6970}
6971
6972impl TransactionProcessingCallback for Bank {
6973    fn account_matches_owners(&self, account: &Pubkey, owners: &[Pubkey]) -> Option<usize> {
6974        self.rc
6975            .accounts
6976            .accounts_db
6977            .account_matches_owners(&self.ancestors, account, owners)
6978            .ok()
6979    }
6980
6981    fn get_account_shared_data(&self, pubkey: &Pubkey) -> Option<AccountSharedData> {
6982        self.rc
6983            .accounts
6984            .accounts_db
6985            .load_with_fixed_root(&self.ancestors, pubkey)
6986            .map(|(acc, _)| acc)
6987    }
6988
6989    // NOTE: must hold idempotent for the same set of arguments
6990    /// Add a builtin program account
6991    fn add_builtin_account(&self, name: &str, program_id: &Pubkey) {
6992        let existing_genuine_program =
6993            self.get_account_with_fixed_root(program_id)
6994                .and_then(|account| {
6995                    // it's very unlikely to be squatted at program_id as non-system account because of burden to
6996                    // find victim's pubkey/hash. So, when account.owner is indeed native_loader's, it's
6997                    // safe to assume it's a genuine program.
6998                    if native_loader::check_id(account.owner()) {
6999                        Some(account)
7000                    } else {
7001                        // malicious account is pre-occupying at program_id
7002                        self.burn_and_purge_account(program_id, account);
7003                        None
7004                    }
7005                });
7006
7007        // introducing builtin program
7008        if existing_genuine_program.is_some() {
7009            // The existing account is sufficient
7010            return;
7011        }
7012
7013        assert!(
7014            !self.freeze_started(),
7015            "Can't change frozen bank by adding not-existing new builtin program ({name}, {program_id}). \
7016            Maybe, inconsistent program activation is detected on snapshot restore?"
7017        );
7018
7019        // Add a bogus executable builtin account, which will be loaded and ignored.
7020        let (lamports, rent_epoch) =
7021            self.inherit_specially_retained_account_fields(&existing_genuine_program);
7022        let account: AccountSharedData = AccountSharedData::from(Account {
7023            lamports,
7024            data: name.as_bytes().to_vec(),
7025            owner: solana_sdk_ids::native_loader::id(),
7026            executable: true,
7027            rent_epoch,
7028        });
7029        self.store_account_and_update_capitalization(program_id, &account);
7030    }
7031
7032    fn inspect_account(&self, address: &Pubkey, account_state: AccountState, is_writable: bool) {
7033        if self.is_accounts_lt_hash_enabled() {
7034            self.inspect_account_for_accounts_lt_hash(address, &account_state, is_writable);
7035        }
7036    }
7037}
7038
7039impl fmt::Debug for Bank {
7040    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
7041        f.debug_struct("Bank")
7042            .field("slot", &self.slot)
7043            .field("bank_id", &self.bank_id)
7044            .field("block_height", &self.block_height)
7045            .field("parent_slot", &self.parent_slot)
7046            .field("capitalization", &self.capitalization())
7047            .finish_non_exhaustive()
7048    }
7049}
7050
7051#[cfg(feature = "dev-context-only-utils")]
7052impl Bank {
7053    pub fn wrap_with_bank_forks_for_tests(self) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
7054        let bank_forks = BankForks::new_rw_arc(self);
7055        let bank = bank_forks.read().unwrap().root_bank();
7056        (bank, bank_forks)
7057    }
7058
7059    pub fn default_for_tests() -> Self {
7060        let accounts_db = AccountsDb::default_for_tests();
7061        let accounts = Accounts::new(Arc::new(accounts_db));
7062        Self::default_with_accounts(accounts)
7063    }
7064
7065    pub fn new_with_bank_forks_for_tests(
7066        genesis_config: &GenesisConfig,
7067    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
7068        let bank = Self::new_for_tests(genesis_config);
7069        bank.wrap_with_bank_forks_for_tests()
7070    }
7071
7072    pub fn new_for_tests(genesis_config: &GenesisConfig) -> Self {
7073        Self::new_with_config_for_tests(genesis_config, BankTestConfig::default())
7074    }
7075
7076    pub fn new_with_mockup_builtin_for_tests(
7077        genesis_config: &GenesisConfig,
7078        program_id: Pubkey,
7079        builtin_function: BuiltinFunctionWithContext,
7080    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
7081        let mut bank = Self::new_for_tests(genesis_config);
7082        bank.add_mockup_builtin(program_id, builtin_function);
7083        bank.wrap_with_bank_forks_for_tests()
7084    }
7085
7086    pub fn new_no_wallclock_throttle_for_tests(
7087        genesis_config: &GenesisConfig,
7088    ) -> (Arc<Self>, Arc<RwLock<BankForks>>) {
7089        let mut bank = Self::new_for_tests(genesis_config);
7090
7091        bank.ns_per_slot = u128::MAX;
7092        bank.wrap_with_bank_forks_for_tests()
7093    }
7094
7095    pub fn new_with_config_for_tests(
7096        genesis_config: &GenesisConfig,
7097        test_config: BankTestConfig,
7098    ) -> Self {
7099        Self::new_with_paths_for_tests(
7100            genesis_config,
7101            Arc::new(RuntimeConfig::default()),
7102            test_config,
7103            Vec::new(),
7104        )
7105    }
7106
7107    pub fn new_with_paths_for_tests(
7108        genesis_config: &GenesisConfig,
7109        runtime_config: Arc<RuntimeConfig>,
7110        test_config: BankTestConfig,
7111        paths: Vec<PathBuf>,
7112    ) -> Self {
7113        Self::new_with_paths(
7114            genesis_config,
7115            runtime_config,
7116            paths,
7117            None,
7118            None,
7119            false,
7120            Some(test_config.accounts_db_config),
7121            None,
7122            Some(Pubkey::new_unique()),
7123            Arc::default(),
7124            None,
7125            None,
7126        )
7127    }
7128
7129    pub fn new_for_benches(genesis_config: &GenesisConfig) -> Self {
7130        Self::new_with_paths_for_benches(genesis_config, Vec::new())
7131    }
7132
7133    /// Intended for use by benches only.
7134    /// create new bank with the given config and paths.
7135    pub fn new_with_paths_for_benches(genesis_config: &GenesisConfig, paths: Vec<PathBuf>) -> Self {
7136        Self::new_with_paths(
7137            genesis_config,
7138            Arc::<RuntimeConfig>::default(),
7139            paths,
7140            None,
7141            None,
7142            false,
7143            Some(ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS),
7144            None,
7145            Some(Pubkey::new_unique()),
7146            Arc::default(),
7147            None,
7148            None,
7149        )
7150    }
7151
7152    /// Prepare a transaction batch from a list of legacy transactions. Used for tests only.
7153    #[cfg(feature = "dev-context-only-utils")]
7154    pub fn prepare_batch_for_tests(
7155        &self,
7156        txs: Vec<Transaction>,
7157    ) -> TransactionBatch<RuntimeTransaction<SanitizedTransaction>> {
7158        let sanitized_txs = txs
7159            .into_iter()
7160            .map(RuntimeTransaction::from_transaction_for_tests)
7161            .collect::<Vec<_>>();
7162        TransactionBatch::new(
7163            self.try_lock_accounts(&sanitized_txs),
7164            self,
7165            OwnedOrBorrowed::Owned(sanitized_txs),
7166        )
7167    }
7168
7169    /// Set the initial accounts data size
7170    /// NOTE: This fn is *ONLY FOR TESTS*
7171    pub fn set_accounts_data_size_initial_for_tests(&mut self, amount: u64) {
7172        self.accounts_data_size_initial = amount;
7173    }
7174
7175    /// Update the accounts data size off-chain delta
7176    /// NOTE: This fn is *ONLY FOR TESTS*
7177    pub fn update_accounts_data_size_delta_off_chain_for_tests(&self, amount: i64) {
7178        self.update_accounts_data_size_delta_off_chain(amount)
7179    }
7180
7181    #[cfg(test)]
7182    fn restore_old_behavior_for_fragile_tests(&self) {
7183        self.lazy_rent_collection.store(true, Relaxed);
7184    }
7185
7186    /// Process multiple transaction in a single batch. This is used for benches and unit tests.
7187    ///
7188    /// # Panics
7189    ///
7190    /// Panics if any of the transactions do not pass sanitization checks.
7191    #[must_use]
7192    pub fn process_transactions<'a>(
7193        &self,
7194        txs: impl Iterator<Item = &'a Transaction>,
7195    ) -> Vec<Result<()>> {
7196        self.try_process_transactions(txs).unwrap()
7197    }
7198
7199    /// Process entry transactions in a single batch. This is used for benches and unit tests.
7200    ///
7201    /// # Panics
7202    ///
7203    /// Panics if any of the transactions do not pass sanitization checks.
7204    #[must_use]
7205    pub fn process_entry_transactions(&self, txs: Vec<VersionedTransaction>) -> Vec<Result<()>> {
7206        self.try_process_entry_transactions(txs).unwrap()
7207    }
7208
7209    #[cfg(test)]
7210    pub fn flush_accounts_cache_slot_for_tests(&self) {
7211        self.rc
7212            .accounts
7213            .accounts_db
7214            .flush_accounts_cache_slot_for_tests(self.slot())
7215    }
7216
7217    /// This is only valid to call from tests.
7218    /// block until initial accounts hash verification has completed
7219    pub fn wait_for_initial_accounts_hash_verification_completed_for_tests(&self) {
7220        self.rc
7221            .accounts
7222            .accounts_db
7223            .verify_accounts_hash_in_bg
7224            .join_background_thread()
7225    }
7226
7227    pub fn get_sysvar_cache_for_tests(&self) -> SysvarCache {
7228        self.transaction_processor.get_sysvar_cache_for_tests()
7229    }
7230
7231    pub fn update_accounts_hash_for_tests(&self) -> AccountsHash {
7232        self.update_accounts_hash(CalcAccountsHashDataSource::IndexForTests, false)
7233    }
7234
7235    pub fn new_program_cache_for_tx_batch_for_slot(&self, slot: Slot) -> ProgramCacheForTxBatch {
7236        ProgramCacheForTxBatch::new_from_cache(
7237            slot,
7238            self.epoch_schedule.get_epoch(slot),
7239            &self.transaction_processor.program_cache.read().unwrap(),
7240        )
7241    }
7242
7243    pub fn get_transaction_processor(&self) -> &TransactionBatchProcessor<BankForks> {
7244        &self.transaction_processor
7245    }
7246
7247    pub fn set_fee_structure(&mut self, fee_structure: &FeeStructure) {
7248        self.fee_structure = fee_structure.clone();
7249    }
7250
7251    pub fn load_program(
7252        &self,
7253        pubkey: &Pubkey,
7254        reload: bool,
7255        effective_epoch: Epoch,
7256    ) -> Option<Arc<ProgramCacheEntry>> {
7257        let environments = self
7258            .transaction_processor
7259            .get_environments_for_epoch(effective_epoch)?;
7260        load_program_with_pubkey(
7261            self,
7262            &environments,
7263            pubkey,
7264            self.slot(),
7265            &mut ExecuteTimings::default(), // Called by ledger-tool, metrics not accumulated.
7266            reload,
7267        )
7268    }
7269
7270    pub fn withdraw(&self, pubkey: &Pubkey, lamports: u64) -> Result<()> {
7271        match self.get_account_with_fixed_root(pubkey) {
7272            Some(mut account) => {
7273                let min_balance = match get_system_account_kind(&account) {
7274                    Some(SystemAccountKind::Nonce) => self
7275                        .rent_collector
7276                        .rent
7277                        .minimum_balance(nonce::state::State::size()),
7278                    _ => 0,
7279                };
7280
7281                lamports
7282                    .checked_add(min_balance)
7283                    .filter(|required_balance| *required_balance <= account.lamports())
7284                    .ok_or(TransactionError::InsufficientFundsForFee)?;
7285                account
7286                    .checked_sub_lamports(lamports)
7287                    .map_err(|_| TransactionError::InsufficientFundsForFee)?;
7288                self.store_account(pubkey, &account);
7289
7290                Ok(())
7291            }
7292            None => Err(TransactionError::AccountNotFound),
7293        }
7294    }
7295
7296    pub fn set_hash_overrides(&self, hash_overrides: HashOverrides) {
7297        *self.hash_overrides.lock().unwrap() = hash_overrides;
7298    }
7299}
7300
7301/// Compute how much an account has changed size.  This function is useful when the data size delta
7302/// needs to be computed and passed to an `update_accounts_data_size_delta` function.
7303fn calculate_data_size_delta(old_data_size: usize, new_data_size: usize) -> i64 {
7304    assert!(old_data_size <= i64::MAX as usize);
7305    assert!(new_data_size <= i64::MAX as usize);
7306    let old_data_size = old_data_size as i64;
7307    let new_data_size = new_data_size as i64;
7308
7309    new_data_size.saturating_sub(old_data_size)
7310}
7311
7312/// Since `apply_feature_activations()` has different behavior depending on its caller, enumerate
7313/// those callers explicitly.
7314#[derive(Debug, Copy, Clone, Eq, PartialEq)]
7315enum ApplyFeatureActivationsCaller {
7316    FinishInit,
7317    NewFromParent,
7318    WarpFromParent,
7319}
7320
7321/// Return the computed values from `collect_rent_from_accounts()`
7322///
7323/// Since `collect_rent_from_accounts()` is running in parallel, instead of updating the
7324/// atomics/shared data inside this function, return those values in this struct for the caller to
7325/// process later.
7326#[derive(Debug, Default)]
7327struct CollectRentFromAccountsInfo {
7328    skipped_rewrites: Vec<(Pubkey, AccountHash)>,
7329    rent_collected_info: CollectedInfo,
7330    rent_rewards: Vec<(Pubkey, RewardInfo)>,
7331    time_collecting_rent_us: u64,
7332    time_storing_accounts_us: u64,
7333    num_accounts: usize,
7334}
7335
7336/// Return the computed values—of each iteration in the parallel loop inside
7337/// `collect_rent_in_partition()`—and then perform a reduce on all of them.
7338#[derive(Debug, Default)]
7339struct CollectRentInPartitionInfo {
7340    skipped_rewrites: Vec<(Pubkey, AccountHash)>,
7341    rent_collected: u64,
7342    accounts_data_size_reclaimed: u64,
7343    rent_rewards: Vec<(Pubkey, RewardInfo)>,
7344    time_loading_accounts_us: u64,
7345    time_collecting_rent_us: u64,
7346    time_storing_accounts_us: u64,
7347    num_accounts: usize,
7348}
7349
7350impl CollectRentInPartitionInfo {
7351    /// Create a new `CollectRentInPartitionInfo` from the results of loading accounts and
7352    /// collecting rent on them.
7353    #[must_use]
7354    fn new(info: CollectRentFromAccountsInfo, time_loading_accounts: Duration) -> Self {
7355        Self {
7356            skipped_rewrites: info.skipped_rewrites,
7357            rent_collected: info.rent_collected_info.rent_amount,
7358            accounts_data_size_reclaimed: info.rent_collected_info.account_data_len_reclaimed,
7359            rent_rewards: info.rent_rewards,
7360            time_loading_accounts_us: time_loading_accounts.as_micros() as u64,
7361            time_collecting_rent_us: info.time_collecting_rent_us,
7362            time_storing_accounts_us: info.time_storing_accounts_us,
7363            num_accounts: info.num_accounts,
7364        }
7365    }
7366
7367    /// Reduce (i.e. 'combine') two `CollectRentInPartitionInfo`s into one.
7368    ///
7369    /// This fn is used by `collect_rent_in_partition()` as the reduce step (of map-reduce) in its
7370    /// parallel loop of rent collection.
7371    #[must_use]
7372    fn reduce(lhs: Self, rhs: Self) -> Self {
7373        Self {
7374            skipped_rewrites: [lhs.skipped_rewrites, rhs.skipped_rewrites].concat(),
7375            rent_collected: lhs.rent_collected.saturating_add(rhs.rent_collected),
7376            accounts_data_size_reclaimed: lhs
7377                .accounts_data_size_reclaimed
7378                .saturating_add(rhs.accounts_data_size_reclaimed),
7379            rent_rewards: [lhs.rent_rewards, rhs.rent_rewards].concat(),
7380            time_loading_accounts_us: lhs
7381                .time_loading_accounts_us
7382                .saturating_add(rhs.time_loading_accounts_us),
7383            time_collecting_rent_us: lhs
7384                .time_collecting_rent_us
7385                .saturating_add(rhs.time_collecting_rent_us),
7386            time_storing_accounts_us: lhs
7387                .time_storing_accounts_us
7388                .saturating_add(rhs.time_storing_accounts_us),
7389            num_accounts: lhs.num_accounts.saturating_add(rhs.num_accounts),
7390        }
7391    }
7392}
7393
7394/// Struct to collect stats when scanning all accounts in `get_total_accounts_stats()`
7395#[derive(Debug, Default, Copy, Clone, Serialize)]
7396pub struct TotalAccountsStats {
7397    /// Total number of accounts
7398    pub num_accounts: usize,
7399    /// Total data size of all accounts
7400    pub data_len: usize,
7401
7402    /// Total number of executable accounts
7403    pub num_executable_accounts: usize,
7404    /// Total data size of executable accounts
7405    pub executable_data_len: usize,
7406
7407    /// Total number of rent exempt accounts
7408    pub num_rent_exempt_accounts: usize,
7409    /// Total number of rent paying accounts
7410    pub num_rent_paying_accounts: usize,
7411    /// Total number of rent paying accounts without data
7412    pub num_rent_paying_accounts_without_data: usize,
7413    /// Total amount of lamports in rent paying accounts
7414    pub lamports_in_rent_paying_accounts: u64,
7415}
7416
7417impl TotalAccountsStats {
7418    pub fn accumulate_account(
7419        &mut self,
7420        address: &Pubkey,
7421        account: &AccountSharedData,
7422        rent_collector: &RentCollector,
7423    ) {
7424        let data_len = account.data().len();
7425        self.num_accounts += 1;
7426        self.data_len += data_len;
7427
7428        if account.executable() {
7429            self.num_executable_accounts += 1;
7430            self.executable_data_len += data_len;
7431        }
7432
7433        if !rent_collector.should_collect_rent(address, account.executable())
7434            || rent_collector
7435                .get_rent_due(
7436                    account.lamports(),
7437                    account.data().len(),
7438                    account.rent_epoch(),
7439                )
7440                .is_exempt()
7441        {
7442            self.num_rent_exempt_accounts += 1;
7443        } else {
7444            self.num_rent_paying_accounts += 1;
7445            self.lamports_in_rent_paying_accounts += account.lamports();
7446            if data_len == 0 {
7447                self.num_rent_paying_accounts_without_data += 1;
7448            }
7449        }
7450    }
7451}
7452
7453impl Drop for Bank {
7454    fn drop(&mut self) {
7455        if let Some(drop_callback) = self.drop_callback.read().unwrap().0.as_ref() {
7456            drop_callback.callback(self);
7457        } else {
7458            // Default case for tests
7459            self.rc
7460                .accounts
7461                .accounts_db
7462                .purge_slot(self.slot(), self.bank_id(), false);
7463        }
7464    }
7465}
7466
7467/// utility function used for testing and benchmarking.
7468pub mod test_utils {
7469    use {
7470        super::Bank,
7471        crate::installed_scheduler_pool::BankWithScheduler,
7472        solana_account::{ReadableAccount, WritableAccount},
7473        solana_instruction::error::LamportsError,
7474        solana_pubkey::Pubkey,
7475        solana_sha256_hasher::hashv,
7476        solana_vote_program::vote_state::{self, BlockTimestamp, VoteStateVersions},
7477        std::sync::Arc,
7478    };
7479    pub fn goto_end_of_slot(bank: Arc<Bank>) {
7480        goto_end_of_slot_with_scheduler(&BankWithScheduler::new_without_scheduler(bank))
7481    }
7482
7483    pub fn goto_end_of_slot_with_scheduler(bank: &BankWithScheduler) {
7484        let mut tick_hash = bank.last_blockhash();
7485        loop {
7486            tick_hash = hashv(&[tick_hash.as_ref(), &[42]]);
7487            bank.register_tick(&tick_hash);
7488            if tick_hash == bank.last_blockhash() {
7489                bank.freeze();
7490                return;
7491            }
7492        }
7493    }
7494
7495    pub fn update_vote_account_timestamp(
7496        timestamp: BlockTimestamp,
7497        bank: &Bank,
7498        vote_pubkey: &Pubkey,
7499    ) {
7500        let mut vote_account = bank.get_account(vote_pubkey).unwrap_or_default();
7501        let mut vote_state = vote_state::from(&vote_account).unwrap_or_default();
7502        vote_state.last_timestamp = timestamp;
7503        let versioned = VoteStateVersions::new_current(vote_state);
7504        vote_state::to(&versioned, &mut vote_account).unwrap();
7505        bank.store_account(vote_pubkey, &vote_account);
7506    }
7507
7508    pub fn deposit(
7509        bank: &Bank,
7510        pubkey: &Pubkey,
7511        lamports: u64,
7512    ) -> std::result::Result<u64, LamportsError> {
7513        // This doesn't collect rents intentionally.
7514        // Rents should only be applied to actual TXes
7515        let mut account = bank
7516            .get_account_with_fixed_root_no_cache(pubkey)
7517            .unwrap_or_default();
7518        account.checked_add_lamports(lamports)?;
7519        bank.store_account(pubkey, &account);
7520        Ok(account.lamports())
7521    }
7522}