Bladeren bron

Adds experimental support for accounts lt hash (#3060)

Brooks 1 jaar geleden
bovenliggende
commit
f65aebb10c

+ 1 - 0
Cargo.lock

@@ -7632,6 +7632,7 @@ name = "solana-runtime"
 version = "2.1.0"
 dependencies = [
  "agave-transaction-view",
+ "ahash 0.8.10",
  "aquamarine",
  "arrayref",
  "assert_matches",

+ 100 - 2
accounts-db/src/accounts_db.rs

@@ -36,8 +36,8 @@ use {
         },
         accounts_hash::{
             AccountHash, AccountLtHash, AccountsDeltaHash, AccountsHash, AccountsHashKind,
-            AccountsHasher, CalcAccountsHashConfig, CalculateHashIntermediate, HashStats,
-            IncrementalAccountsHash, SerdeAccountsDeltaHash, SerdeAccountsHash,
+            AccountsHasher, AccountsLtHash, CalcAccountsHashConfig, CalculateHashIntermediate,
+            HashStats, IncrementalAccountsHash, SerdeAccountsDeltaHash, SerdeAccountsHash,
             SerdeIncrementalAccountsHash, ZeroLamportAccounts, ZERO_LAMPORT_ACCOUNT_HASH,
             ZERO_LAMPORT_ACCOUNT_LT_HASH,
         },
@@ -501,6 +501,7 @@ pub const ACCOUNTS_DB_CONFIG_FOR_TESTING: AccountsDbConfig = AccountsDbConfig {
     test_skip_rewrites_but_include_in_bank_hash: false,
     storage_access: StorageAccess::Mmap,
     scan_filter_for_shrinking: ScanFilter::OnlyAbnormalWithVerify,
+    enable_experimental_accumulator_hash: false,
 };
 pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig {
     index: Some(ACCOUNTS_INDEX_CONFIG_FOR_BENCHMARKS),
@@ -517,6 +518,7 @@ pub const ACCOUNTS_DB_CONFIG_FOR_BENCHMARKS: AccountsDbConfig = AccountsDbConfig
     test_skip_rewrites_but_include_in_bank_hash: false,
     storage_access: StorageAccess::Mmap,
     scan_filter_for_shrinking: ScanFilter::OnlyAbnormalWithVerify,
+    enable_experimental_accumulator_hash: false,
 };
 
 pub type BinnedHashData = Vec<Vec<CalculateHashIntermediate>>;
@@ -612,6 +614,7 @@ pub struct AccountsDbConfig {
     pub test_partitioned_epoch_rewards: TestPartitionedEpochRewards,
     pub storage_access: StorageAccess,
     pub scan_filter_for_shrinking: ScanFilter,
+    pub enable_experimental_accumulator_hash: bool,
 }
 
 #[cfg(not(test))]
@@ -1508,6 +1511,10 @@ pub struct AccountsDb {
 
     /// The latest full snapshot slot dictates how to handle zero lamport accounts
     latest_full_snapshot_slot: SeqLock<Option<Slot>>,
+
+    /// Flag to indicate if the experimental accounts lattice hash is enabled.
+    /// (For R&D only; a feature-gate also exists to turn this on and make it a part of consensus.)
+    pub is_experimental_accumulator_hash_enabled: AtomicBool,
 }
 
 #[derive(Debug, Default)]
@@ -2477,6 +2484,8 @@ impl AccountsDb {
         // rayon needs a lot of stack
         const ACCOUNTS_STACK_SIZE: usize = 8 * 1024 * 1024;
 
+        let default_accounts_db_config = AccountsDbConfig::default();
+
         AccountsDb {
             create_ancient_storage: CreateAncientStorage::default(),
             verify_accounts_hash_in_bg: VerifyAccountsHashInBackground::default(),
@@ -2541,6 +2550,9 @@ impl AccountsDb {
             epoch_accounts_hash_manager: EpochAccountsHashManager::new_invalid(),
             test_skip_rewrites_but_include_in_bank_hash: false,
             latest_full_snapshot_slot: SeqLock::new(None),
+            is_experimental_accumulator_hash_enabled: default_accounts_db_config
+                .enable_experimental_accumulator_hash
+                .into(),
         }
     }
 
@@ -2587,6 +2599,8 @@ impl AccountsDb {
         accounts_update_notifier: Option<AccountsUpdateNotifier>,
         exit: Arc<AtomicBool>,
     ) -> Self {
+        let default_accounts_db_config = AccountsDbConfig::default();
+
         let accounts_index = AccountsIndex::new(
             accounts_db_config.as_mut().and_then(|x| x.index.take()),
             exit,
@@ -2648,6 +2662,12 @@ impl AccountsDb {
             .map(|config| config.scan_filter_for_shrinking)
             .unwrap_or_default();
 
+        let enable_experimental_accumulator_hash = accounts_db_config
+            .as_ref()
+            .map(|config| config.enable_experimental_accumulator_hash)
+            .unwrap_or(default_accounts_db_config.enable_experimental_accumulator_hash)
+            .into();
+
         let paths_is_empty = paths.is_empty();
         let mut new = Self {
             paths,
@@ -2671,6 +2691,7 @@ impl AccountsDb {
             test_skip_rewrites_but_include_in_bank_hash,
             storage_access,
             scan_filter_for_shrinking,
+            is_experimental_accumulator_hash_enabled: enable_experimental_accumulator_hash,
             ..Self::default_with_accounts_index(
                 accounts_index,
                 base_working_path,
@@ -2737,6 +2758,18 @@ impl AccountsDb {
             .expect("Cluster type must be set at initialization")
     }
 
+    /// Returns if the experimental accounts lattice hash is enabled
+    pub fn is_experimental_accumulator_hash_enabled(&self) -> bool {
+        self.is_experimental_accumulator_hash_enabled
+            .load(Ordering::Acquire)
+    }
+
+    /// Sets if the experimental accounts lattice hash is enabled
+    pub fn set_is_experimental_accumulator_hash_enabled(&self, is_enabled: bool) {
+        self.is_experimental_accumulator_hash_enabled
+            .store(is_enabled, Ordering::Release);
+    }
+
     /// While scanning cleaning candidates obtain slots that can be
     /// reclaimed for each pubkey. In addition, if the pubkey is
     /// removed from the index, insert in pubkeys_removed_from_accounts_index.
@@ -7147,6 +7180,71 @@ impl AccountsDb {
         (accounts_hash, total_lamports)
     }
 
+    /// Calculates the accounts lt hash
+    ///
+    /// Only intended to be called at startup (or by tests).
+    /// Only intended to be used while testing the experimental accumulator hash.
+    pub fn calculate_accounts_lt_hash_at_startup(
+        &self,
+        ancestors: &Ancestors,
+        startup_slot: Slot,
+    ) -> AccountsLtHash {
+        debug_assert!(self.is_experimental_accumulator_hash_enabled());
+
+        // This impl iterates over all the index bins in parallel, and computes the lt hash
+        // sequentially per bin.  Then afterwards reduces to a single lt hash.
+        // This implementation is quite fast.  Runtime is about 150 seconds on mnb as of 10/2/2024.
+        // The sequential implementation took about 6,275 seconds!
+        // A different parallel implementation that iterated over the bins *sequentially* and then
+        // hashed the accounts *within* a bin in parallel took about 600 seconds.  That impl uses
+        // less memory, as only a single index bin is loaded into mem at a time.
+        let lt_hash = self
+            .accounts_index
+            .account_maps
+            .par_iter()
+            .fold(
+                LtHash::identity,
+                |mut accumulator_lt_hash, accounts_index_bin| {
+                    for pubkey in accounts_index_bin.keys() {
+                        let account_lt_hash = self
+                            .accounts_index
+                            .get_with_and_then(
+                                &pubkey,
+                                Some(ancestors),
+                                Some(startup_slot),
+                                false,
+                                |(slot, account_info)| {
+                                    (!account_info.is_zero_lamport()).then(|| {
+                                        self.get_account_accessor(
+                                            slot,
+                                            &pubkey,
+                                            &account_info.storage_location(),
+                                        )
+                                        .get_loaded_account(|loaded_account| {
+                                            Self::lt_hash_account(&loaded_account, &pubkey)
+                                        })
+                                        // SAFETY: The index said this pubkey exists, so
+                                        // there must be an account to load.
+                                        .unwrap()
+                                    })
+                                },
+                            )
+                            .flatten();
+                        if let Some(account_lt_hash) = account_lt_hash {
+                            accumulator_lt_hash.mix_in(&account_lt_hash.0);
+                        }
+                    }
+                    accumulator_lt_hash
+                },
+            )
+            .reduce(LtHash::identity, |mut accum, elem| {
+                accum.mix_in(&elem);
+                accum
+            });
+
+        AccountsLtHash(lt_hash)
+    }
+
     /// This is only valid to call from tests.
     /// run the accounts hash calculation and store the results
     pub fn update_accounts_hash_for_tests(

+ 1 - 2
accounts-db/src/accounts_hash.rs

@@ -1262,8 +1262,7 @@ pub const ZERO_LAMPORT_ACCOUNT_HASH: AccountHash =
 pub struct AccountLtHash(pub LtHash);
 
 /// The AccountLtHash for a zero-lamport account
-pub const ZERO_LAMPORT_ACCOUNT_LT_HASH: AccountLtHash =
-    AccountLtHash(LtHash([0; LtHash::NUM_ELEMENTS]));
+pub const ZERO_LAMPORT_ACCOUNT_LT_HASH: AccountLtHash = AccountLtHash(LtHash::identity());
 
 /// Lattice hash of all accounts
 #[derive(Debug, Clone, Eq, PartialEq)]

+ 4 - 0
ledger-tool/src/args.rs

@@ -127,6 +127,10 @@ pub fn accounts_db_args<'a, 'b>() -> Box<[Arg<'a, 'b>]> {
             .possible_values(&["mmap", "file"])
             .help("Access account storage using this method")
             .hidden(hidden_unless_forced()),
+        Arg::with_name("accounts_db_experimental_accumulator_hash")
+            .long("accounts-db-experimental-accumulator-hash")
+            .help("Enables the experimental accumulator hash")
+            .hidden(hidden_unless_forced()),
     ]
     .into_boxed_slice()
 }

+ 1 - 0
programs/sbf/Cargo.lock

@@ -5972,6 +5972,7 @@ dependencies = [
 name = "solana-runtime"
 version = "2.1.0"
 dependencies = [
+ "ahash 0.8.10",
  "aquamarine",
  "arrayref",
  "base64 0.22.1",

+ 1 - 0
runtime/Cargo.toml

@@ -10,6 +10,7 @@ license = { workspace = true }
 edition = { workspace = true }
 
 [dependencies]
+ahash = { workspace = true }
 aquamarine = { workspace = true }
 arrayref = { workspace = true }
 base64 = { workspace = true }

+ 72 - 5
runtime/src/bank.rs

@@ -59,6 +59,8 @@ use {
         transaction_batch::{OwnedOrBorrowed, TransactionBatch},
         verify_precompiles::verify_precompiles,
     },
+    accounts_lt_hash::InitialStateOfAccount,
+    ahash::AHashMap,
     byteorder::{ByteOrder, LittleEndian},
     dashmap::{DashMap, DashSet},
     log::*,
@@ -76,7 +78,8 @@ use {
             CalcAccountsHashDataSource, PubkeyHashAccount, VerifyAccountsHashAndLamportsConfig,
         },
         accounts_hash::{
-            AccountHash, AccountsHash, CalcAccountsHashConfig, HashStats, IncrementalAccountsHash,
+            AccountHash, AccountsHash, AccountsLtHash, CalcAccountsHashConfig, HashStats,
+            IncrementalAccountsHash,
         },
         accounts_index::{AccountSecondaryIndexes, IndexKey, ScanConfig, ScanResult},
         accounts_partition::{self, Partition, PartitionIndex},
@@ -97,7 +100,8 @@ use {
         self as feature_set, remove_rounding_in_fee_calculation, reward_full_priority_fee,
         FeatureSet,
     },
-    solana_measure::{measure::Measure, measure_time, measure_us},
+    solana_lattice_hash::lt_hash::LtHash,
+    solana_measure::{meas_dur, measure::Measure, measure_time, measure_us},
     solana_program_runtime::{
         invoke_context::BuiltinFunctionWithContext, loaded_programs::ProgramCacheEntry,
     },
@@ -161,7 +165,7 @@ use {
         transaction_execution_result::{
             TransactionExecutionDetails, TransactionLoadedAccountsStats,
         },
-        transaction_processing_callback::TransactionProcessingCallback,
+        transaction_processing_callback::{AccountState, TransactionProcessingCallback},
         transaction_processing_result::{
             ProcessedTransaction, TransactionProcessingResult,
             TransactionProcessingResultExtensions,
@@ -216,6 +220,7 @@ struct VerifyAccountsHashConfig {
     store_hash_raw_data_for_debug: bool,
 }
 
+mod accounts_lt_hash;
 mod address_lookup_table;
 pub mod bank_hash_details;
 mod builtin_programs;
@@ -572,6 +577,8 @@ impl PartialEq for Bank {
             compute_budget: _,
             transaction_account_lock_limit: _,
             fee_structure: _,
+            accounts_lt_hash: _,
+            cache_for_accounts_lt_hash: _,
             // Ignore new fields explicitly if they do not impact PartialEq.
             // Adding ".." will remove compile-time checks that if a new field
             // is added to the struct, this PartialEq is accordingly updated.
@@ -902,6 +909,17 @@ pub struct Bank {
     /// This _field_ was needed to be DCOU-ed to avoid 2 locks per bank freezing...
     #[cfg(feature = "dev-context-only-utils")]
     hash_overrides: Arc<Mutex<HashOverrides>>,
+
+    /// The lattice hash of all accounts
+    ///
+    /// The value is only meaningful after freezing.
+    accounts_lt_hash: Mutex<AccountsLtHash>,
+
+    /// A cache of *the initial state* of accounts modified in this slot
+    ///
+    /// The accounts lt hash needs both the initial and final state of each
+    /// account that was modified in this slot.  Cache the initial state here.
+    cache_for_accounts_lt_hash: RwLock<AHashMap<Pubkey, InitialStateOfAccount>>,
 }
 
 struct VoteWithStakeDelegations {
@@ -1022,6 +1040,8 @@ impl Bank {
             fee_structure: FeeStructure::default(),
             #[cfg(feature = "dev-context-only-utils")]
             hash_overrides: Arc::new(Mutex::new(HashOverrides::default())),
+            accounts_lt_hash: Mutex::new(AccountsLtHash(LtHash([0xBAD1; LtHash::NUM_ELEMENTS]))),
+            cache_for_accounts_lt_hash: RwLock::new(AHashMap::new()),
         };
 
         bank.transaction_processor =
@@ -1030,6 +1050,17 @@ impl Bank {
         let accounts_data_size_initial = bank.get_total_accounts_stats().unwrap().data_len as u64;
         bank.accounts_data_size_initial = accounts_data_size_initial;
 
+        let accounts_lt_hash = {
+            let mut accounts_lt_hash = AccountsLtHash(LtHash::identity());
+            let accounts = bank.get_all_accounts(false).unwrap();
+            for account in accounts {
+                let account_lt_hash = AccountsDb::lt_hash_account(&account.1, &account.0);
+                accounts_lt_hash.0.mix_in(&account_lt_hash.0);
+            }
+            accounts_lt_hash
+        };
+        *bank.accounts_lt_hash.get_mut().unwrap() = accounts_lt_hash;
+
         bank
     }
 
@@ -1283,6 +1314,8 @@ impl Bank {
             fee_structure: parent.fee_structure.clone(),
             #[cfg(feature = "dev-context-only-utils")]
             hash_overrides: parent.hash_overrides.clone(),
+            accounts_lt_hash: Mutex::new(parent.accounts_lt_hash.lock().unwrap().clone()),
+            cache_for_accounts_lt_hash: RwLock::new(AHashMap::new()),
         };
 
         let (_, ancestors_time_us) = measure_us!({
@@ -1661,6 +1694,8 @@ impl Bank {
             fee_structure: FeeStructure::default(),
             #[cfg(feature = "dev-context-only-utils")]
             hash_overrides: Arc::new(Mutex::new(HashOverrides::default())),
+            accounts_lt_hash: Mutex::new(AccountsLtHash(LtHash([0xBAD2; LtHash::NUM_ELEMENTS]))),
+            cache_for_accounts_lt_hash: RwLock::new(AHashMap::new()),
         };
 
         bank.transaction_processor =
@@ -1681,6 +1716,17 @@ impl Bank {
             .fill_missing_sysvar_cache_entries(&bank);
         bank.rebuild_skipped_rewrites();
 
+        let calculate_accounts_lt_hash_duration = bank.is_accounts_lt_hash_enabled().then(|| {
+            let (_, duration) = meas_dur!({
+                *bank.accounts_lt_hash.get_mut().unwrap() = bank
+                    .rc
+                    .accounts
+                    .accounts_db
+                    .calculate_accounts_lt_hash_at_startup(&bank.ancestors, bank.slot());
+            });
+            duration
+        });
+
         // Sanity assertions between bank snapshot and genesis config
         // Consider removing from serializable bank state
         // (BankFieldsToSerialize/BankFieldsToDeserialize) and initializing
@@ -1725,6 +1771,11 @@ impl Bank {
                 stakes_accounts_load_duration.as_micros(),
                 i64
             ),
+            (
+                "calculate_accounts_lt_hash_us",
+                calculate_accounts_lt_hash_duration.as_ref().map(Duration::as_micros),
+                Option<i64>
+            ),
         );
         bank
     }
@@ -5373,6 +5424,10 @@ impl Bank {
             hash = hashv(&[hash.as_ref(), epoch_accounts_hash.as_ref().as_ref()]);
         };
 
+        let accounts_lt_hash_checksum = self
+            .is_accounts_lt_hash_enabled()
+            .then(|| self.update_accounts_lt_hash());
+
         let buf = self
             .hard_forks
             .read()
@@ -5413,8 +5468,9 @@ impl Bank {
             .accounts_db
             .get_bank_hash_stats(slot)
             .expect("No bank hash stats were found for this bank, that should not be possible");
+
         info!(
-            "bank frozen: {slot} hash: {hash} accounts_delta: {} signature_count: {} last_blockhash: {} capitalization: {}{}, stats: {bank_hash_stats:?}",
+            "bank frozen: {slot} hash: {hash} accounts_delta: {} signature_count: {} last_blockhash: {} capitalization: {}{}, stats: {bank_hash_stats:?}{}",
             accounts_delta_hash.0,
             self.signature_count(),
             self.last_blockhash(),
@@ -5423,7 +5479,12 @@ impl Bank {
                 format!(", epoch_accounts_hash: {:?}", epoch_accounts_hash.as_ref())
             } else {
                 "".to_string()
-            }
+            },
+            if let Some(accounts_lt_hash_checksum) = accounts_lt_hash_checksum {
+                format!(", accounts_lt_hash checksum: {accounts_lt_hash_checksum}")
+            } else {
+                String::new()
+            },
         );
         hash
     }
@@ -6710,6 +6771,12 @@ impl TransactionProcessingCallback for Bank {
         );
         self.store_account_and_update_capitalization(program_id, &account);
     }
+
+    fn inspect_account(&self, address: &Pubkey, account_state: AccountState, is_writable: bool) {
+        if self.is_accounts_lt_hash_enabled() {
+            self.inspect_account_for_accounts_lt_hash(address, &account_state, is_writable);
+        }
+    }
 }
 
 #[cfg(feature = "dev-context-only-utils")]

+ 633 - 0
runtime/src/bank/accounts_lt_hash.rs

@@ -0,0 +1,633 @@
+use {
+    super::Bank,
+    rayon::prelude::*,
+    solana_accounts_db::accounts_db::AccountsDb,
+    solana_lattice_hash::lt_hash::{Checksum as LtChecksum, LtHash},
+    solana_measure::{meas_dur, measure::Measure},
+    solana_sdk::{
+        account::{accounts_equal, AccountSharedData},
+        pubkey::Pubkey,
+    },
+    solana_svm::transaction_processing_callback::AccountState,
+    std::{ops::AddAssign, time::Duration},
+};
+
+impl Bank {
+    /// Returns if the accounts lt hash is enabled
+    pub fn is_accounts_lt_hash_enabled(&self) -> bool {
+        self.rc
+            .accounts
+            .accounts_db
+            .is_experimental_accumulator_hash_enabled()
+    }
+
+    /// Updates the accounts lt hash
+    ///
+    /// When freezing a bank, we compute and update the accounts lt hash.
+    /// For each account modified in this bank, we:
+    /// - mix out its previous state, and
+    /// - mix in its current state
+    ///
+    /// Since this function is non-idempotent, it should only be called once per bank.
+    pub fn update_accounts_lt_hash(&self) -> LtChecksum {
+        debug_assert!(self.is_accounts_lt_hash_enabled());
+        let delta_lt_hash = self.calculate_delta_lt_hash();
+        let mut accounts_lt_hash = self.accounts_lt_hash.lock().unwrap();
+        accounts_lt_hash.0.mix_in(&delta_lt_hash);
+        accounts_lt_hash.0.checksum()
+    }
+
+    /// Calculates the lt hash *of only this slot*
+    ///
+    /// This can be thought of as akin to the accounts delta hash.
+    ///
+    /// For each account modified in this bank, we:
+    /// - mix out its previous state, and
+    /// - mix in its current state
+    ///
+    /// This function is idempotent, and may be called more than once.
+    fn calculate_delta_lt_hash(&self) -> LtHash {
+        debug_assert!(self.is_accounts_lt_hash_enabled());
+        let measure_total = Measure::start("");
+        let slot = self.slot();
+
+        // If we don't find the account in the cache, we need to go load it.
+        // We want the version of the account *before* it was written in this slot.
+        // Bank::ancestors *includes* this slot, so we need to remove it before loading.
+        let strictly_ancestors = {
+            let mut ancestors = self.ancestors.clone();
+            ancestors.remove(&self.slot());
+            ancestors
+        };
+
+        // Get all the accounts stored in this slot.
+        // Since this bank is in the middle of being frozen, it hasn't been rooted.
+        // That means the accounts should all be in the write cache, and loading will be fast.
+        let (accounts_curr, time_loading_accounts_curr) = meas_dur!({
+            self.rc
+                .accounts
+                .accounts_db
+                .get_pubkey_hash_account_for_slot(slot)
+        });
+        let num_accounts_total = accounts_curr.len();
+
+        #[derive(Debug, Default)]
+        struct Stats {
+            num_cache_misses: usize,
+            num_accounts_unmodified: usize,
+            time_loading_accounts_prev: Duration,
+            time_comparing_accounts: Duration,
+            time_computing_hashes: Duration,
+            time_mixing_hashes: Duration,
+        }
+        impl AddAssign for Stats {
+            fn add_assign(&mut self, other: Self) {
+                self.num_cache_misses += other.num_cache_misses;
+                self.num_accounts_unmodified += other.num_accounts_unmodified;
+                self.time_loading_accounts_prev += other.time_loading_accounts_prev;
+                self.time_comparing_accounts += other.time_comparing_accounts;
+                self.time_computing_hashes += other.time_computing_hashes;
+                self.time_mixing_hashes += other.time_mixing_hashes;
+            }
+        }
+
+        let do_calculate_delta_lt_hash = || {
+            // Work on chunks of 128 pubkeys, which is 4 KiB.
+            // And 4 KiB is likely the smallest a real page size will be.
+            // And a single page is likely the smallest size a disk read will actually read.
+            // This can be tuned larger, but likely not smaller.
+            const CHUNK_SIZE: usize = 128;
+            let cache_for_accounts_lt_hash = self.cache_for_accounts_lt_hash.read().unwrap();
+            accounts_curr
+                .par_iter()
+                .fold_chunks(
+                    CHUNK_SIZE,
+                    || (LtHash::identity(), Stats::default()),
+                    |mut accum, elem| {
+                        let pubkey = &elem.pubkey;
+                        let curr_account = &elem.account;
+
+                        // load the initial state of the account
+                        let (initial_state_of_account, measure_load) = meas_dur!({
+                            match cache_for_accounts_lt_hash.get(pubkey) {
+                                Some(initial_state_of_account) => initial_state_of_account.clone(),
+                                None => {
+                                    accum.1.num_cache_misses += 1;
+                                    // If the initial state of the account is not in the accounts
+                                    // lt hash cache, it is likely this account was stored
+                                    // *outside* of transaction processing (e.g. as part of rent
+                                    // collection).  Do not populate the read cache, as this
+                                    // account likely will not be accessed again soon.
+                                    let account_slot = self
+                                        .rc
+                                        .accounts
+                                        .load_with_fixed_root_do_not_populate_read_cache(
+                                            &strictly_ancestors,
+                                            pubkey,
+                                        );
+                                    match account_slot {
+                                        Some((account, _slot)) => {
+                                            InitialStateOfAccount::Alive(account)
+                                        }
+                                        None => InitialStateOfAccount::Dead,
+                                    }
+                                }
+                            }
+                        });
+                        accum.1.time_loading_accounts_prev += measure_load;
+
+                        // mix out the previous version of the account
+                        match initial_state_of_account {
+                            InitialStateOfAccount::Dead => {
+                                // nothing to do here
+                            }
+                            InitialStateOfAccount::Alive(prev_account) => {
+                                let (are_accounts_equal, measure_is_equal) =
+                                    meas_dur!(accounts_equal(curr_account, &prev_account));
+                                accum.1.time_comparing_accounts += measure_is_equal;
+                                if are_accounts_equal {
+                                    // this account didn't actually change, so skip it for lt hashing
+                                    accum.1.num_accounts_unmodified += 1;
+                                    return accum;
+                                }
+                                let (prev_lt_hash, measure_hashing) =
+                                    meas_dur!(AccountsDb::lt_hash_account(&prev_account, pubkey));
+                                let (_, measure_mixing) =
+                                    meas_dur!(accum.0.mix_out(&prev_lt_hash.0));
+                                accum.1.time_computing_hashes += measure_hashing;
+                                accum.1.time_mixing_hashes += measure_mixing;
+                            }
+                        }
+
+                        // mix in the new version of the account
+                        let (curr_lt_hash, measure_hashing) =
+                            meas_dur!(AccountsDb::lt_hash_account(curr_account, pubkey));
+                        let (_, measure_mixing) = meas_dur!(accum.0.mix_in(&curr_lt_hash.0));
+                        accum.1.time_computing_hashes += measure_hashing;
+                        accum.1.time_mixing_hashes += measure_mixing;
+
+                        accum
+                    },
+                )
+                .reduce(
+                    || (LtHash::identity(), Stats::default()),
+                    |mut accum, elem| {
+                        accum.0.mix_in(&elem.0);
+                        accum.1 += elem.1;
+                        accum
+                    },
+                )
+        };
+        let (delta_lt_hash, stats) = self
+            .rc
+            .accounts
+            .accounts_db
+            .thread_pool
+            .install(do_calculate_delta_lt_hash);
+
+        let total_time = measure_total.end_as_duration();
+        let num_accounts_modified =
+            num_accounts_total.saturating_sub(stats.num_accounts_unmodified);
+        datapoint_info!(
+            "bank-accounts_lt_hash",
+            ("slot", slot, i64),
+            ("num_accounts_total", num_accounts_total, i64),
+            ("num_accounts_modified", num_accounts_modified, i64),
+            (
+                "num_accounts_unmodified",
+                stats.num_accounts_unmodified,
+                i64
+            ),
+            ("num_cache_misses", stats.num_cache_misses, i64),
+            ("total_us", total_time.as_micros(), i64),
+            (
+                "loading_accounts_curr_us",
+                time_loading_accounts_curr.as_micros(),
+                i64
+            ),
+            (
+                "par_loading_accounts_prev_us",
+                stats.time_loading_accounts_prev.as_micros(),
+                i64
+            ),
+            (
+                "par_comparing_accounts_us",
+                stats.time_comparing_accounts.as_micros(),
+                i64
+            ),
+            (
+                "par_computing_hashes_us",
+                stats.time_computing_hashes.as_micros(),
+                i64
+            ),
+            (
+                "par_mixing_hashes_us",
+                stats.time_mixing_hashes.as_micros(),
+                i64
+            ),
+        );
+
+        delta_lt_hash
+    }
+
+    /// Caches initial state of writeable accounts
+    ///
+    /// If a transaction account is writeable, cache its initial account state.
+    /// The initial state is needed when computing the accounts lt hash for the slot, and caching
+    /// the initial state saves us from having to look it up on disk later.
+    pub fn inspect_account_for_accounts_lt_hash(
+        &self,
+        address: &Pubkey,
+        account_state: &AccountState,
+        is_writable: bool,
+    ) {
+        debug_assert!(self.is_accounts_lt_hash_enabled());
+        if !is_writable {
+            // if the account is not writable, then it cannot be modified; nothing to do here
+            return;
+        }
+
+        // Only insert the account the *first* time we see it.
+        // We want to capture the value of the account *before* any modifications during this slot.
+        let is_in_cache = self
+            .cache_for_accounts_lt_hash
+            .read()
+            .unwrap()
+            .contains_key(address);
+        if !is_in_cache {
+            self.cache_for_accounts_lt_hash
+                .write()
+                .unwrap()
+                .entry(*address)
+                .or_insert_with(|| match account_state {
+                    AccountState::Dead => InitialStateOfAccount::Dead,
+                    AccountState::Alive(account) => {
+                        InitialStateOfAccount::Alive((*account).clone())
+                    }
+                });
+        }
+    }
+}
+
+/// The initial state of an account prior to being modified in this slot/transaction
+#[derive(Debug, Clone)]
+pub enum InitialStateOfAccount {
+    /// The account was initiall dead
+    Dead,
+    /// The account was initially alive
+    Alive(AccountSharedData),
+}
+
+#[cfg(test)]
+mod tests {
+    use {
+        super::*,
+        crate::bank::tests::new_bank_from_parent_with_bank_forks,
+        solana_accounts_db::accounts::Accounts,
+        solana_sdk::{
+            account::{ReadableAccount as _, WritableAccount as _},
+            fee_calculator::FeeRateGovernor,
+            genesis_config::create_genesis_config,
+            native_token::LAMPORTS_PER_SOL,
+            pubkey::{self, Pubkey},
+            signature::Signer as _,
+            signer::keypair::Keypair,
+        },
+        std::{cmp, str::FromStr as _, sync::Arc},
+    };
+
+    #[test]
+    fn test_update_accounts_lt_hash() {
+        // Write to address 1, 2, and 5 in first bank, so that in second bank we have
+        // updates to these three accounts.  Make address 2 go to zero (dead).  Make address 1 and 3 stay
+        // alive.  Make address 5 unchanged.  Ensure the updates are expected.
+        //
+        // 1: alive -> alive
+        // 2: alive -> dead
+        // 3: dead -> alive
+        // 4. dead -> dead
+        // 5. alive -> alive *unchanged*
+
+        let keypair1 = Keypair::new();
+        let keypair2 = Keypair::new();
+        let keypair3 = Keypair::new();
+        let keypair4 = Keypair::new();
+        let keypair5 = Keypair::new();
+
+        let (mut genesis_config, mint_keypair) =
+            create_genesis_config(123_456_789 * LAMPORTS_PER_SOL);
+        genesis_config.fee_rate_governor = FeeRateGovernor::new(0, 0);
+        let (bank, bank_forks) = Bank::new_with_bank_forks_for_tests(&genesis_config);
+        bank.rc
+            .accounts
+            .accounts_db
+            .set_is_experimental_accumulator_hash_enabled(true);
+
+        // ensure the accounts lt hash is enabled, otherwise this test doesn't actually do anything...
+        assert!(bank.is_accounts_lt_hash_enabled());
+
+        let amount = cmp::max(
+            bank.get_minimum_balance_for_rent_exemption(0),
+            LAMPORTS_PER_SOL,
+        );
+
+        // send lamports to accounts 1, 2, and 5 so they are alive,
+        // and so we'll have a delta in the next bank
+        bank.register_unique_recent_blockhash_for_test();
+        bank.transfer(amount, &mint_keypair, &keypair1.pubkey())
+            .unwrap();
+        bank.transfer(amount, &mint_keypair, &keypair2.pubkey())
+            .unwrap();
+        bank.transfer(amount, &mint_keypair, &keypair5.pubkey())
+            .unwrap();
+
+        // manually freeze the bank to trigger update_accounts_lt_hash() to run
+        bank.freeze();
+        let prev_accounts_lt_hash = bank.accounts_lt_hash.lock().unwrap().clone();
+
+        // save the initial values of the accounts to use for asserts later
+        let prev_mint = bank.get_account_with_fixed_root(&mint_keypair.pubkey());
+        let prev_account1 = bank.get_account_with_fixed_root(&keypair1.pubkey());
+        let prev_account2 = bank.get_account_with_fixed_root(&keypair2.pubkey());
+        let prev_account3 = bank.get_account_with_fixed_root(&keypair3.pubkey());
+        let prev_account4 = bank.get_account_with_fixed_root(&keypair4.pubkey());
+        let prev_account5 = bank.get_account_with_fixed_root(&keypair5.pubkey());
+
+        assert!(prev_mint.is_some());
+        assert!(prev_account1.is_some());
+        assert!(prev_account2.is_some());
+        assert!(prev_account3.is_none());
+        assert!(prev_account4.is_none());
+        assert!(prev_account5.is_some());
+
+        // These sysvars are also updated, but outside of transaction processing.  This means they
+        // will not be in the accounts lt hash cache, but *will* be in the list of modified
+        // accounts.  They must be included in the accounts lt hash.
+        let sysvars = [
+            Pubkey::from_str("SysvarS1otHashes111111111111111111111111111").unwrap(),
+            Pubkey::from_str("SysvarC1ock11111111111111111111111111111111").unwrap(),
+            Pubkey::from_str("SysvarRecentB1ockHashes11111111111111111111").unwrap(),
+            Pubkey::from_str("SysvarS1otHistory11111111111111111111111111").unwrap(),
+        ];
+        let prev_sysvar_accounts: Vec<_> = sysvars
+            .iter()
+            .map(|address| bank.get_account_with_fixed_root(address))
+            .collect();
+
+        let bank = {
+            let slot = bank.slot() + 1;
+            new_bank_from_parent_with_bank_forks(&bank_forks, bank, &Pubkey::default(), slot)
+        };
+
+        // send from account 2 to account 1; account 1 stays alive, account 2 ends up dead
+        bank.register_unique_recent_blockhash_for_test();
+        bank.transfer(amount, &keypair2, &keypair1.pubkey())
+            .unwrap();
+
+        // send lamports to account 4, then turn around and send them to account 3
+        // account 3 will be alive, and account 4 will end dead
+        bank.register_unique_recent_blockhash_for_test();
+        bank.transfer(amount, &mint_keypair, &keypair4.pubkey())
+            .unwrap();
+        bank.register_unique_recent_blockhash_for_test();
+        bank.transfer(amount, &keypair4, &keypair3.pubkey())
+            .unwrap();
+
+        // store account 5 into this new bank, unchanged
+        bank.rc.accounts.store_cached(
+            (
+                bank.slot(),
+                [(&keypair5.pubkey(), &prev_account5.clone().unwrap())].as_slice(),
+            ),
+            None,
+        );
+
+        // freeze the bank to trigger update_accounts_lt_hash() to run
+        bank.freeze();
+
+        let actual_delta_lt_hash = bank.calculate_delta_lt_hash();
+        let post_accounts_lt_hash = bank.accounts_lt_hash.lock().unwrap().clone();
+        let post_mint = bank.get_account_with_fixed_root(&mint_keypair.pubkey());
+        let post_account1 = bank.get_account_with_fixed_root(&keypair1.pubkey());
+        let post_account2 = bank.get_account_with_fixed_root(&keypair2.pubkey());
+        let post_account3 = bank.get_account_with_fixed_root(&keypair3.pubkey());
+        let post_account4 = bank.get_account_with_fixed_root(&keypair4.pubkey());
+        let post_account5 = bank.get_account_with_fixed_root(&keypair5.pubkey());
+
+        assert!(post_mint.is_some());
+        assert!(post_account1.is_some());
+        assert!(post_account2.is_none());
+        assert!(post_account3.is_some());
+        assert!(post_account4.is_none());
+        assert!(post_account5.is_some());
+
+        let post_sysvar_accounts: Vec<_> = sysvars
+            .iter()
+            .map(|address| bank.get_account_with_fixed_root(address))
+            .collect();
+
+        let mut expected_delta_lt_hash = LtHash::identity();
+        let mut expected_accounts_lt_hash = prev_accounts_lt_hash.clone();
+        let mut updater =
+            |address: &Pubkey, prev: Option<AccountSharedData>, post: Option<AccountSharedData>| {
+                // if there was an alive account, mix out
+                if let Some(prev) = prev {
+                    let prev_lt_hash = AccountsDb::lt_hash_account(&prev, address);
+                    expected_delta_lt_hash.mix_out(&prev_lt_hash.0);
+                    expected_accounts_lt_hash.0.mix_out(&prev_lt_hash.0);
+                }
+
+                // mix in the new one
+                let post = post.unwrap_or_default();
+                let post_lt_hash = AccountsDb::lt_hash_account(&post, address);
+                expected_delta_lt_hash.mix_in(&post_lt_hash.0);
+                expected_accounts_lt_hash.0.mix_in(&post_lt_hash.0);
+            };
+        updater(&mint_keypair.pubkey(), prev_mint, post_mint);
+        updater(&keypair1.pubkey(), prev_account1, post_account1);
+        updater(&keypair2.pubkey(), prev_account2, post_account2);
+        updater(&keypair3.pubkey(), prev_account3, post_account3);
+        updater(&keypair4.pubkey(), prev_account4, post_account4);
+        updater(&keypair5.pubkey(), prev_account5, post_account5);
+        for (i, sysvar) in sysvars.iter().enumerate() {
+            updater(
+                sysvar,
+                prev_sysvar_accounts[i].clone(),
+                post_sysvar_accounts[i].clone(),
+            );
+        }
+
+        // now make sure the delta lt hashes match
+        let expected = expected_delta_lt_hash.checksum();
+        let actual = actual_delta_lt_hash.checksum();
+        assert_eq!(
+            expected, actual,
+            "delta_lt_hash, expected: {expected}, actual: {actual}",
+        );
+
+        // ...and the accounts lt hashes match too
+        let expected = expected_accounts_lt_hash.0.checksum();
+        let actual = post_accounts_lt_hash.0.checksum();
+        assert_eq!(
+            expected, actual,
+            "accounts_lt_hash, expected: {expected}, actual: {actual}",
+        );
+    }
+
+    #[test]
+    fn test_inspect_account_for_accounts_lt_hash() {
+        let accounts_db = AccountsDb::default_for_tests();
+        accounts_db.set_is_experimental_accumulator_hash_enabled(true);
+        let accounts = Accounts::new(Arc::new(accounts_db));
+        let bank = Bank::default_with_accounts(accounts);
+
+        // ensure the accounts lt hash is enabled, otherwise this test doesn't actually do anything...
+        assert!(bank.is_accounts_lt_hash_enabled());
+
+        // the cache should start off empty
+        assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 0);
+
+        // ensure non-writable accounts are *not* added to the cache
+        bank.inspect_account_for_accounts_lt_hash(
+            &Pubkey::new_unique(),
+            &AccountState::Dead,
+            false,
+        );
+        bank.inspect_account_for_accounts_lt_hash(
+            &Pubkey::new_unique(),
+            &AccountState::Alive(&AccountSharedData::default()),
+            false,
+        );
+        assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 0);
+
+        // ensure *new* accounts are added to the cache
+        let address = Pubkey::new_unique();
+        bank.inspect_account_for_accounts_lt_hash(&address, &AccountState::Dead, true);
+        assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 1);
+        assert!(bank
+            .cache_for_accounts_lt_hash
+            .read()
+            .unwrap()
+            .contains_key(&address));
+
+        // ensure *existing* accounts are added to the cache
+        let address = Pubkey::new_unique();
+        let initial_lamports = 123;
+        let mut account = AccountSharedData::new(initial_lamports, 0, &Pubkey::default());
+        bank.inspect_account_for_accounts_lt_hash(&address, &AccountState::Alive(&account), true);
+        assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 2);
+        if let InitialStateOfAccount::Alive(cached_account) = bank
+            .cache_for_accounts_lt_hash
+            .read()
+            .unwrap()
+            .get(&address)
+            .unwrap()
+        {
+            assert_eq!(*cached_account, account);
+        } else {
+            panic!("wrong initial state for account");
+        };
+
+        // ensure if an account is modified multiple times that we only cache the *first* one
+        let updated_lamports = account.lamports() + 1;
+        account.set_lamports(updated_lamports);
+        bank.inspect_account_for_accounts_lt_hash(&address, &AccountState::Alive(&account), true);
+        assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 2);
+        if let InitialStateOfAccount::Alive(cached_account) = bank
+            .cache_for_accounts_lt_hash
+            .read()
+            .unwrap()
+            .get(&address)
+            .unwrap()
+        {
+            assert_eq!(cached_account.lamports(), initial_lamports);
+        } else {
+            panic!("wrong initial state for account");
+        };
+
+        // and ensure multiple updates are handled correctly when the account is initially dead
+        {
+            let address = Pubkey::new_unique();
+            bank.inspect_account_for_accounts_lt_hash(&address, &AccountState::Dead, true);
+            assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 3);
+            match bank
+                .cache_for_accounts_lt_hash
+                .read()
+                .unwrap()
+                .get(&address)
+                .unwrap()
+            {
+                InitialStateOfAccount::Dead => { /* this is expected, nothing to do here*/ }
+                _ => panic!("wrong initial state for account"),
+            };
+
+            bank.inspect_account_for_accounts_lt_hash(
+                &address,
+                &AccountState::Alive(&AccountSharedData::default()),
+                true,
+            );
+            assert_eq!(bank.cache_for_accounts_lt_hash.read().unwrap().len(), 3);
+            match bank
+                .cache_for_accounts_lt_hash
+                .read()
+                .unwrap()
+                .get(&address)
+                .unwrap()
+            {
+                InitialStateOfAccount::Dead => { /* this is expected, nothing to do here*/ }
+                _ => panic!("wrong initial state for account"),
+            };
+        }
+    }
+
+    #[test]
+    fn test_calculate_accounts_lt_hash_at_startup() {
+        let (genesis_config, mint_keypair) = create_genesis_config(123_456_789 * LAMPORTS_PER_SOL);
+        let (mut bank, bank_forks) = Bank::new_with_bank_forks_for_tests(&genesis_config);
+        bank.rc
+            .accounts
+            .accounts_db
+            .set_is_experimental_accumulator_hash_enabled(true);
+
+        // ensure the accounts lt hash is enabled, otherwise this test doesn't actually do anything...
+        assert!(bank.is_accounts_lt_hash_enabled());
+
+        let amount = cmp::max(
+            bank.get_minimum_balance_for_rent_exemption(0),
+            LAMPORTS_PER_SOL,
+        );
+
+        // create some banks with some modified accounts so that there are stored accounts
+        // (note: the number of banks and transfers are arbitrary)
+        for _ in 0..7 {
+            let slot = bank.slot() + 1;
+            bank =
+                new_bank_from_parent_with_bank_forks(&bank_forks, bank, &Pubkey::default(), slot);
+            for _ in 0..13 {
+                bank.register_unique_recent_blockhash_for_test();
+                // note: use a random pubkey here to ensure accounts
+                // are spread across all the index bins
+                bank.transfer(amount, &mint_keypair, &pubkey::new_rand())
+                    .unwrap();
+            }
+            bank.freeze();
+        }
+        let expected_accounts_lt_hash = bank.accounts_lt_hash.lock().unwrap().clone();
+
+        // root the bank and flush the accounts write cache to disk
+        // (this more accurately simulates startup, where accounts are in storages on disk)
+        bank.squash();
+        bank.force_flush_accounts_cache();
+
+        // call the fn that calculates the accounts lt hash at startup, then ensure it matches
+        let calculated_accounts_lt_hash = bank
+            .rc
+            .accounts
+            .accounts_db
+            .calculate_accounts_lt_hash_at_startup(&bank.ancestors, bank.slot());
+
+        let expected = expected_accounts_lt_hash.0.checksum();
+        let actual = calculated_accounts_lt_hash.0.checksum();
+        assert_eq!(expected, actual, "expected: {expected}, actual: {actual}");
+    }
+}

+ 6 - 0
validator/src/cli.rs

@@ -1405,6 +1405,12 @@ pub fn app<'a>(version: &'a str, default_args: &'a DefaultArgs) -> App<'a, 'a> {
                 )
                 .hidden(hidden_unless_forced()),
         )
+        .arg(
+            Arg::with_name("accounts_db_experimental_accumulator_hash")
+                .long("accounts-db-experimental-accumulator-hash")
+                .help("Enables the experimental accumulator hash")
+                .hidden(hidden_unless_forced()),
+        )
         .arg(
             Arg::with_name("accounts_index_scan_results_limit_mb")
                 .long("accounts-index-scan-results-limit-mb")

+ 2 - 0
validator/src/main.rs

@@ -1301,6 +1301,8 @@ pub fn main() {
             .is_present("accounts_db_test_skip_rewrites"),
         storage_access,
         scan_filter_for_shrinking,
+        enable_experimental_accumulator_hash: matches
+            .is_present("accounts_db_experimental_accumulator_hash"),
         ..AccountsDbConfig::default()
     };