refactor(trie): move state root database operations into extension trait (#9635)

Co-authored-by: Roman Krasiuk <rokrassyuk@gmail.com>
This commit is contained in:
Roman Hodulák
2024-07-22 12:59:49 +02:00
committed by GitHub
parent a82b3c5195
commit 5ec0c26980
32 changed files with 1504 additions and 1321 deletions

42
Cargo.lock generated
View File

@ -6274,6 +6274,7 @@ dependencies = [
"reth-tracing",
"reth-transaction-pool",
"reth-trie",
"reth-trie-db",
"serde",
"serde_json",
"similar-asserts",
@ -6453,6 +6454,7 @@ dependencies = [
"reth-storage-errors",
"reth-testing-utils",
"reth-trie",
"reth-trie-db",
"reth-trie-parallel",
"tokio",
"tracing",
@ -6549,6 +6551,7 @@ dependencies = [
"reth-static-file",
"reth-static-file-types",
"reth-trie",
"reth-trie-db",
"serde",
"serde_json",
"tokio",
@ -6758,6 +6761,7 @@ dependencies = [
"reth-provider",
"reth-stages-types",
"reth-trie",
"reth-trie-db",
"serde",
"serde_json",
"thiserror",
@ -8103,6 +8107,7 @@ dependencies = [
"reth-storage-errors",
"reth-testing-utils",
"reth-trie",
"reth-trie-db",
"revm",
"strum",
"tempfile",
@ -8519,6 +8524,7 @@ dependencies = [
"reth-storage-errors",
"reth-testing-utils",
"reth-trie",
"reth-trie-db",
"serde_json",
"tempfile",
"thiserror",
@ -8785,6 +8791,41 @@ dependencies = [
"toml",
]
[[package]]
name = "reth-trie-db"
version = "1.0.3"
dependencies = [
"alloy-rlp",
"auto_impl",
"criterion",
"derive_more",
"itertools 0.13.0",
"metrics",
"once_cell",
"proptest",
"proptest-arbitrary-interop",
"rayon",
"reth-chainspec",
"reth-db",
"reth-db-api",
"reth-execution-errors",
"reth-metrics",
"reth-primitives",
"reth-provider",
"reth-stages-types",
"reth-storage-errors",
"reth-trie",
"reth-trie-common",
"revm",
"serde",
"serde_json",
"similar-asserts",
"tokio",
"tokio-stream",
"tracing",
"triehash",
]
[[package]]
name = "reth-trie-parallel"
version = "1.0.3"
@ -8806,6 +8847,7 @@ dependencies = [
"reth-provider",
"reth-tasks",
"reth-trie",
"reth-trie-db",
"thiserror",
"tokio",
"tracing",

View File

@ -114,6 +114,7 @@ members = [
"crates/tracing/",
"crates/transaction-pool/",
"crates/trie/common",
"crates/trie/db",
"crates/trie/parallel/",
"crates/trie/trie",
"examples/beacon-api-sidecar-fetcher/",

View File

@ -54,6 +54,7 @@ reth-basic-payload-builder.workspace = true
reth-static-file.workspace = true
reth-static-file-types = { workspace = true, features = ["clap"] }
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-db = { workspace = true, features = ["metrics"] }
reth-node-api.workspace = true
reth-node-optimism = { workspace = true, optional = true, features = [
"optimism",

View File

@ -37,6 +37,8 @@ use reth_transaction_pool::{
blobstore::InMemoryBlobStore, BlobStore, EthPooledTransaction, PoolConfig, TransactionOrigin,
TransactionPool, TransactionValidationTaskExecutor,
};
use reth_trie::StateRoot;
use reth_trie_db::DatabaseStateRoot;
use std::{path::PathBuf, str::FromStr, sync::Arc};
use tracing::*;
@ -283,8 +285,10 @@ impl Command {
debug!(target: "reth::cli", ?execution_outcome, "Executed block");
let hashed_post_state = execution_outcome.hash_state_slow();
let (state_root, trie_updates) = hashed_post_state
.state_root_with_updates(provider_factory.provider()?.tx_ref())?;
let (state_root, trie_updates) = StateRoot::overlay_root_with_updates(
provider_factory.provider()?.tx_ref(),
hashed_post_state.clone(),
)?;
if state_root != block_with_senders.state_root {
eyre::bail!(

View File

@ -27,6 +27,7 @@ use reth_revm::database::StateProviderDatabase;
use reth_stages::StageId;
use reth_tasks::TaskExecutor;
use reth_trie::StateRoot;
use reth_trie_db::DatabaseStateRoot;
use std::{path::PathBuf, sync::Arc};
use tracing::*;
@ -148,8 +149,10 @@ impl Command {
ExecutionOutcome::new(state, receipts.into(), block.number, vec![requests.into()]);
// Unpacked `BundleState::state_root_slow` function
let (in_memory_state_root, in_memory_updates) =
execution_outcome.hash_state_slow().state_root_with_updates(provider.tx_ref())?;
let (in_memory_state_root, in_memory_updates) = StateRoot::overlay_root_with_updates(
provider.tx_ref(),
execution_outcome.hash_state_slow(),
)?;
if in_memory_state_root == block.state_root {
info!(target: "reth::cli", state_root = ?in_memory_state_root, "Computed in-memory state root matches");

View File

@ -25,6 +25,7 @@ reth-execution-types.workspace = true
reth-prune-types.workspace = true
reth-stages-api.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-db = { workspace = true, features = ["metrics"] }
reth-trie-parallel = { workspace = true, features = ["parallel"] }
reth-network.workspace = true
reth-consensus.workspace = true

View File

@ -27,6 +27,7 @@ use reth_prune_types::PruneModes;
use reth_stages_api::{MetricEvent, MetricEventsSender};
use reth_storage_errors::provider::{ProviderResult, RootMismatch};
use reth_trie::{hashed_cursor::HashedPostStateCursorFactory, StateRoot};
use reth_trie_db::DatabaseStateRoot;
use std::{
collections::{btree_map::Entry, BTreeMap, HashSet},
sync::Arc,

View File

@ -35,6 +35,7 @@ reth-stages.workspace = true
reth-static-file-types.workspace = true
reth-static-file.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-db = { workspace = true, features = ["metrics"] }
itertools.workspace = true
futures.workspace = true

View File

@ -8,6 +8,7 @@ use reth_db_api::{
};
use reth_provider::{BlockNumReader, HeaderProvider, ProviderError};
use reth_trie::StateRoot;
use reth_trie_db::DatabaseStateRoot;
use tracing::*;
/// `reth recover storage-tries` command

View File

@ -33,6 +33,7 @@ reth-storage-errors.workspace = true
reth-revm.workspace = true
reth-stages-api.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-db = { workspace = true, features = ["metrics"] }
reth-testing-utils = { workspace = true, optional = true }

View File

@ -26,6 +26,7 @@ mod constants;
mod account_hashing;
pub use account_hashing::*;
use reth_stages_api::{ExecInput, Stage, UnwindInput};
use reth_trie_db::DatabaseStateRoot;
pub(crate) type StageRange = (ExecInput, UnwindInput);

View File

@ -15,6 +15,7 @@ use reth_stages_api::{
StageCheckpoint, StageError, StageId, UnwindInput, UnwindOutput,
};
use reth_trie::{IntermediateStateRootState, StateRoot, StateRootProgress, StoredSubNode};
use reth_trie_db::DatabaseStateRoot;
use std::fmt::Debug;
use tracing::*;

View File

@ -16,6 +16,7 @@ reth-db-api.workspace = true
reth-provider.workspace = true
reth-config.workspace = true
reth-trie.workspace = true
reth-trie-db.workspace = true
reth-etl.workspace = true
reth-codecs.workspace = true
reth-stages-types.workspace = true

View File

@ -20,6 +20,7 @@ use reth_provider::{
};
use reth_stages_types::{StageCheckpoint, StageId};
use reth_trie::{IntermediateStateRootState, StateRoot as StateRootComputer, StateRootProgress};
use reth_trie_db::DatabaseStateRoot;
use serde::{Deserialize, Serialize};
use std::{
collections::{BTreeMap, HashMap},

View File

@ -27,6 +27,7 @@ reth-db-api.workspace = true
reth-prune-types.workspace = true
reth-stages-types.workspace = true
reth-trie = { workspace = true, features = ["metrics"] }
reth-trie-db = { workspace = true, features = ["metrics"] }
reth-nippy-jar.workspace = true
reth-codecs.workspace = true
reth-evm.workspace = true

View File

@ -45,6 +45,7 @@ mod tests {
keccak256, Account, Address, Receipt, Receipts, StorageEntry, B256, U256,
};
use reth_trie::{test_utils::state_root, StateRoot};
use reth_trie_db::DatabaseStateRoot;
use revm::{
db::{
states::{
@ -863,6 +864,8 @@ mod tests {
let assert_state_root = |state: &State<EmptyDB>, expected: &PreState, msg| {
assert_eq!(
StateRoot::overlay_root(
&tx,
ExecutionOutcome::new(
state.bundle_state.clone(),
Receipts::default(),
@ -870,7 +873,7 @@ mod tests {
Vec::new()
)
.hash_state_slow()
.state_root(&tx)
)
.unwrap(),
state_root(expected.clone().into_iter().map(|(address, (account, storage))| (
address,

View File

@ -47,6 +47,7 @@ use reth_trie::{
updates::TrieUpdates,
HashedPostStateSorted, Nibbles, StateRoot,
};
use reth_trie_db::DatabaseStateRoot;
use revm::primitives::{BlockEnv, CfgEnvWithHandlerCfg};
use std::{
cmp::Ordering,

View File

@ -15,7 +15,8 @@ use reth_primitives::{
};
use reth_storage_api::StateProofProvider;
use reth_storage_errors::provider::ProviderResult;
use reth_trie::{updates::TrieUpdates, AccountProof, HashedPostState};
use reth_trie::{updates::TrieUpdates, AccountProof, HashedPostState, StateRoot};
use reth_trie_db::DatabaseStateRoot;
use std::fmt::Debug;
/// State provider for a given block number which takes a tx reference.
@ -259,7 +260,8 @@ impl<'b, TX: DbTx> StateRootProvider for HistoricalStateProviderRef<'b, TX> {
fn hashed_state_root(&self, hashed_state: &HashedPostState) -> ProviderResult<B256> {
let mut revert_state = self.revert_state()?;
revert_state.extend(hashed_state.clone());
revert_state.state_root(self.tx).map_err(|err| ProviderError::Database(err.into()))
StateRoot::overlay_root(self.tx, revert_state)
.map_err(|err| ProviderError::Database(err.into()))
}
fn hashed_state_root_with_updates(
@ -268,8 +270,7 @@ impl<'b, TX: DbTx> StateRootProvider for HistoricalStateProviderRef<'b, TX> {
) -> ProviderResult<(B256, TrieUpdates)> {
let mut revert_state = self.revert_state()?;
revert_state.extend(hashed_state.clone());
revert_state
.state_root_with_updates(self.tx)
StateRoot::overlay_root_with_updates(self.tx, revert_state)
.map_err(|err| ProviderError::Database(err.into()))
}
}

View File

@ -12,7 +12,8 @@ use reth_primitives::{
};
use reth_storage_api::StateProofProvider;
use reth_storage_errors::provider::{ProviderError, ProviderResult};
use reth_trie::{updates::TrieUpdates, AccountProof, HashedPostState};
use reth_trie::{updates::TrieUpdates, AccountProof, HashedPostState, StateRoot};
use reth_trie_db::DatabaseStateRoot;
/// State provider over latest state that takes tx reference.
#[derive(Debug)]
@ -75,15 +76,15 @@ impl<'b, TX: DbTx> BlockHashReader for LatestStateProviderRef<'b, TX> {
impl<'b, TX: DbTx> StateRootProvider for LatestStateProviderRef<'b, TX> {
fn hashed_state_root(&self, hashed_state: &HashedPostState) -> ProviderResult<B256> {
hashed_state.state_root(self.tx).map_err(|err| ProviderError::Database(err.into()))
StateRoot::overlay_root(self.tx, hashed_state.clone())
.map_err(|err| ProviderError::Database(err.into()))
}
fn hashed_state_root_with_updates(
&self,
hashed_state: &HashedPostState,
) -> ProviderResult<(B256, TrieUpdates)> {
hashed_state
.state_root_with_updates(self.tx)
StateRoot::overlay_root_with_updates(self.tx, hashed_state.clone())
.map_err(|err| ProviderError::Database(err.into()))
}
}

View File

@ -10,3 +10,69 @@ description = "Database integration with merkle trie implementation"
[lints]
workspace = true
[dependencies]
# reth
reth-primitives.workspace = true
reth-execution-errors.workspace = true
reth-db.workspace = true
reth-db-api.workspace = true
reth-stages-types.workspace = true
reth-trie-common.workspace = true
reth-trie.workspace = true
revm.workspace = true
# alloy
alloy-rlp.workspace = true
# tracing
tracing.workspace = true
# misc
rayon.workspace = true
derive_more.workspace = true
auto_impl.workspace = true
itertools.workspace = true
# `metrics` feature
reth-metrics = { workspace = true, optional = true }
metrics = { workspace = true, optional = true }
# `test-utils` feature
triehash = { version = "0.8", optional = true }
# `serde` feature
serde = { workspace = true, optional = true }
[dev-dependencies]
# reth
reth-chainspec.workspace = true
reth-primitives = { workspace = true, features = ["test-utils", "arbitrary"] }
reth-db = { workspace = true, features = ["test-utils"] }
reth-provider = { workspace = true, features = ["test-utils"] }
reth-storage-errors.workspace = true
reth-trie-common = { workspace = true, features = ["test-utils", "arbitrary"] }
reth-trie = { workspace = true, features = ["test-utils"] }
# trie
triehash = "0.8"
# misc
proptest.workspace = true
proptest-arbitrary-interop.workspace = true
tokio = { workspace = true, default-features = false, features = [
"sync",
"rt",
"macros",
] }
tokio-stream.workspace = true
once_cell.workspace = true
serde_json.workspace = true
similar-asserts.workspace = true
criterion.workspace = true
[features]
metrics = ["reth-metrics", "dep:metrics"]
serde = ["dep:serde"]
test-utils = ["triehash", "reth-trie-common/test-utils"]

View File

@ -1 +1,4 @@
//! An integration of [`reth-trie`] with [`reth-db`].
mod state;
pub use state::DatabaseStateRoot;

200
crates/trie/db/src/state.rs Normal file
View File

@ -0,0 +1,200 @@
use reth_db_api::transaction::DbTx;
use reth_execution_errors::StateRootError;
use reth_primitives::{BlockNumber, B256};
use reth_trie::{
hashed_cursor::HashedPostStateCursorFactory, prefix_set::PrefixSetLoader, updates::TrieUpdates,
HashedPostState, StateRoot, StateRootProgress,
};
use std::ops::RangeInclusive;
use tracing::debug;
/// Extends [`StateRoot`] with operations specific for working with a database transaction.
pub trait DatabaseStateRoot<'a, TX>: Sized {
/// Create a new [`StateRoot`] instance.
fn from_tx(tx: &'a TX) -> Self;
/// Given a block number range, identifies all the accounts and storage keys that
/// have changed.
///
/// # Returns
///
/// An instance of state root calculator with account and storage prefixes loaded.
fn incremental_root_calculator(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<Self, StateRootError>;
/// Computes the state root of the trie with the changed account and storage prefixes and
/// existing trie nodes.
///
/// # Returns
///
/// The updated state root.
fn incremental_root(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<B256, StateRootError>;
/// Computes the state root of the trie with the changed account and storage prefixes and
/// existing trie nodes collecting updates in the process.
///
/// Ignores the threshold.
///
/// # Returns
///
/// The updated state root and the trie updates.
fn incremental_root_with_updates(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<(B256, TrieUpdates), StateRootError>;
/// Computes the state root of the trie with the changed account and storage prefixes and
/// existing trie nodes collecting updates in the process.
///
/// # Returns
///
/// The intermediate progress of state root computation.
fn incremental_root_with_progress(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<StateRootProgress, StateRootError>;
/// Calculate the state root for this [`HashedPostState`].
/// Internally, this method retrieves prefixsets and uses them
/// to calculate incremental state root.
///
/// # Example
///
/// ```
/// use reth_db::test_utils::create_test_rw_db;
/// use reth_db_api::database::Database;
/// use reth_primitives::{Account, U256};
/// use reth_trie::{HashedPostState, StateRoot};
/// use reth_trie_db::DatabaseStateRoot;
///
/// // Initialize the database
/// let db = create_test_rw_db();
///
/// // Initialize hashed post state
/// let mut hashed_state = HashedPostState::default();
/// hashed_state.accounts.insert(
/// [0x11; 32].into(),
/// Some(Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }),
/// );
///
/// // Calculate the state root
/// let tx = db.tx().expect("failed to create transaction");
/// let state_root = StateRoot::overlay_root(&tx, hashed_state);
/// ```
///
/// # Returns
///
/// The state root for this [`HashedPostState`].
fn overlay_root(tx: &'a TX, post_state: HashedPostState) -> Result<B256, StateRootError>;
/// Calculates the state root for this [`HashedPostState`] and returns it alongside trie
/// updates. See [`Self::overlay_root`] for more info.
fn overlay_root_with_updates(
tx: &'a TX,
post_state: HashedPostState,
) -> Result<(B256, TrieUpdates), StateRootError>;
}
impl<'a, TX: DbTx> DatabaseStateRoot<'a, TX> for StateRoot<&'a TX, &'a TX> {
fn from_tx(tx: &'a TX) -> Self {
Self::new(tx, tx)
}
fn incremental_root_calculator(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<Self, StateRootError> {
let loaded_prefix_sets = PrefixSetLoader::new(tx).load(range)?;
Ok(Self::from_tx(tx).with_prefix_sets(loaded_prefix_sets))
}
fn incremental_root(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<B256, StateRootError> {
debug!(target: "trie::loader", ?range, "incremental state root");
Self::incremental_root_calculator(tx, range)?.root()
}
fn incremental_root_with_updates(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<(B256, TrieUpdates), StateRootError> {
debug!(target: "trie::loader", ?range, "incremental state root");
Self::incremental_root_calculator(tx, range)?.root_with_updates()
}
fn incremental_root_with_progress(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<StateRootProgress, StateRootError> {
debug!(target: "trie::loader", ?range, "incremental state root with progress");
Self::incremental_root_calculator(tx, range)?.root_with_progress()
}
fn overlay_root(tx: &'a TX, post_state: HashedPostState) -> Result<B256, StateRootError> {
let prefix_sets = post_state.construct_prefix_sets().freeze();
let sorted = post_state.into_sorted();
Self::from_tx(tx)
.with_hashed_cursor_factory(HashedPostStateCursorFactory::new(tx, &sorted))
.with_prefix_sets(prefix_sets)
.root()
}
fn overlay_root_with_updates(
tx: &'a TX,
post_state: HashedPostState,
) -> Result<(B256, TrieUpdates), StateRootError> {
let prefix_sets = post_state.construct_prefix_sets().freeze();
let sorted = post_state.into_sorted();
Self::from_tx(tx)
.with_hashed_cursor_factory(HashedPostStateCursorFactory::new(tx, &sorted))
.with_prefix_sets(prefix_sets)
.root_with_updates()
}
}
#[cfg(test)]
mod tests {
use super::*;
use reth_db::test_utils::create_test_rw_db;
use reth_db_api::database::Database;
use reth_primitives::{hex, revm_primitives::AccountInfo, Address, U256};
use revm::db::BundleState;
use std::collections::HashMap;
#[test]
fn from_bundle_state_with_rayon() {
let address1 = Address::with_last_byte(1);
let address2 = Address::with_last_byte(2);
let slot1 = U256::from(1015);
let slot2 = U256::from(2015);
let account1 = AccountInfo { nonce: 1, ..Default::default() };
let account2 = AccountInfo { nonce: 2, ..Default::default() };
let bundle_state = BundleState::builder(2..=2)
.state_present_account_info(address1, account1)
.state_present_account_info(address2, account2)
.state_storage(address1, HashMap::from([(slot1, (U256::ZERO, U256::from(10)))]))
.state_storage(address2, HashMap::from([(slot2, (U256::ZERO, U256::from(20)))]))
.build();
assert_eq!(bundle_state.reverts.len(), 1);
let post_state = HashedPostState::from_bundle_state(&bundle_state.state);
assert_eq!(post_state.accounts.len(), 2);
assert_eq!(post_state.storages.len(), 2);
let db = create_test_rw_db();
let tx = db.tx().expect("failed to create transaction");
assert_eq!(
StateRoot::overlay_root(&tx, post_state).unwrap(),
hex!("b464525710cafcf5d4044ac85b72c08b1e76231b8d91f288fe438cc41d8eaafd")
);
}
}

View File

@ -0,0 +1,59 @@
use proptest::prelude::*;
use reth_db::{cursor::DbCursorRW, tables, transaction::DbTxMut};
use reth_primitives::{Account, B256, U256};
use reth_provider::test_utils::create_test_provider_factory;
use reth_trie::{
prefix_set::{PrefixSetMut, TriePrefixSets},
test_utils::state_root_prehashed,
trie_cursor::InMemoryTrieCursorFactory,
StateRoot,
};
use reth_trie_common::Nibbles;
use reth_trie_db::DatabaseStateRoot;
use std::collections::BTreeMap;
proptest! {
#![proptest_config(ProptestConfig {
cases: 128, ..ProptestConfig::default()
})]
#[test]
fn fuzz_in_memory_nodes(mut init_state: BTreeMap<B256, U256>, mut updated_state: BTreeMap<B256, U256>) {
let factory = create_test_provider_factory();
let provider = factory.provider_rw().unwrap();
let mut hashed_account_cursor = provider.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
// Insert init state into database
for (hashed_address, balance) in init_state.clone() {
hashed_account_cursor.upsert(hashed_address, Account { balance, ..Default::default() }).unwrap();
}
// Compute initial root and updates
let (_, trie_updates) = StateRoot::from_tx(provider.tx_ref())
.root_with_updates()
.unwrap();
// Insert state updates into database
let mut changes = PrefixSetMut::default();
for (hashed_address, balance) in updated_state.clone() {
hashed_account_cursor.upsert(hashed_address, Account { balance, ..Default::default() }).unwrap();
changes.insert(Nibbles::unpack(hashed_address));
}
// Compute root with in-memory trie nodes overlay
let (state_root, _) = StateRoot::from_tx(provider.tx_ref())
.with_prefix_sets(TriePrefixSets { account_prefix_set: changes.freeze(), ..Default::default() })
.with_trie_cursor_factory(InMemoryTrieCursorFactory::new(provider.tx_ref(), &trie_updates.into_sorted()))
.root_with_updates()
.unwrap();
// Verify the result
let mut state = BTreeMap::default();
state.append(&mut init_state);
state.append(&mut updated_state);
let expected_root = state_root_prehashed(
state.iter().map(|(&key, &balance)| (key, (Account { balance, ..Default::default() }, std::iter::empty())))
);
assert_eq!(expected_root, state_root);
}
}

View File

@ -0,0 +1,286 @@
use once_cell::sync::Lazy;
use reth_chainspec::{Chain, ChainSpec, HOLESKY, MAINNET};
use reth_db_api::database::Database;
use reth_primitives::{
constants::EMPTY_ROOT_HASH, keccak256, Account, Address, Bytes, StorageEntry, B256, U256,
};
use reth_provider::{test_utils::create_test_provider_factory, HashingWriter, ProviderFactory};
use reth_storage_errors::provider::ProviderResult;
use reth_trie::{proof::Proof, Nibbles, StateRoot};
use reth_trie_common::{AccountProof, StorageProof};
use reth_trie_db::DatabaseStateRoot;
use std::{str::FromStr, sync::Arc};
/*
World State (sampled from <https://ethereum.stackexchange.com/questions/268/ethereum-block-architecture/6413#6413>)
| address | prefix | hash | balance
|--------------------------------------------|-----------|--------------------------------------------------------------------|--------
| 0x2031f89b3ea8014eb51a78c316e42af3e0d7695f | 0xa711355 | 0xa711355ec1c8f7e26bb3ccbcb0b75d870d15846c0b98e5cc452db46c37faea40 | 45 eth
| 0x33f0fc440b8477fcfbe9d0bf8649e7dea9baedb2 | 0xa77d337 | 0xa77d337781e762f3577784bab7491fcc43e291ce5a356b9bc517ac52eed3a37a | 1 wei
| 0x62b0dd4aab2b1a0a04e279e2b828791a10755528 | 0xa7f9365 | 0xa7f936599f93b769acf90c7178fd2ddcac1b5b4bc9949ee5a04b7e0823c2446e | 1.1 eth
| 0x1ed9b1dd266b607ee278726d324b855a093394a6 | 0xa77d397 | 0xa77d397a32b8ab5eb4b043c65b1f00c93f517bc8883c5cd31baf8e8a279475e3 | .12 eth
All expected testspec results were obtained from querying proof RPC on the running geth instance `geth init crates/trie/testdata/proof-genesis.json && geth --http`.
*/
static TEST_SPEC: Lazy<Arc<ChainSpec>> = Lazy::new(|| {
ChainSpec {
chain: Chain::from_id(12345),
genesis: serde_json::from_str(include_str!("../../trie/testdata/proof-genesis.json"))
.expect("Can't deserialize test genesis json"),
..Default::default()
}
.into()
});
fn convert_to_proof<'a>(path: impl IntoIterator<Item = &'a str>) -> Vec<Bytes> {
path.into_iter().map(Bytes::from_str).collect::<Result<Vec<_>, _>>().unwrap()
}
fn insert_genesis<DB: Database>(
provider_factory: &ProviderFactory<DB>,
chain_spec: Arc<ChainSpec>,
) -> ProviderResult<B256> {
let mut provider = provider_factory.provider_rw()?;
// Hash accounts and insert them into hashing table.
let genesis = chain_spec.genesis();
let alloc_accounts = genesis
.alloc
.iter()
.map(|(addr, account)| (*addr, Some(Account::from_genesis_account(account))));
provider.insert_account_for_hashing(alloc_accounts).unwrap();
let alloc_storage = genesis.alloc.clone().into_iter().filter_map(|(addr, account)| {
// Only return `Some` if there is storage.
account.storage.map(|storage| {
(
addr,
storage.into_iter().map(|(key, value)| StorageEntry { key, value: value.into() }),
)
})
});
provider.insert_storage_for_hashing(alloc_storage)?;
let (root, updates) = StateRoot::from_tx(provider.tx_ref())
.root_with_updates()
.map_err(Into::<reth_db::DatabaseError>::into)?;
updates.write_to_database(provider.tx_mut())?;
provider.commit()?;
Ok(root)
}
#[test]
fn testspec_proofs() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, TEST_SPEC.clone()).unwrap();
let data = Vec::from([
(
"0x2031f89b3ea8014eb51a78c316e42af3e0d7695f",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xf8719f31355ec1c8f7e26bb3ccbcb0b75d870d15846c0b98e5cc452db46c37faea40b84ff84d80890270801d946c940000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
(
"0x33f0fc440b8477fcfbe9d0bf8649e7dea9baedb2",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xe48200d3a0ef957210bca5b9b402d614eb8408c88cfbf4913eb6ab83ca233c8b8f0e626b54",
"0xf851808080a02743a5addaf4cf9b8c0c073e1eaa555deaaf8c41cb2b41958e88624fa45c2d908080808080a0bfbf6937911dfb88113fecdaa6bde822e4e99dae62489fcf61a91cb2f36793d680808080808080",
"0xf8679e207781e762f3577784bab7491fcc43e291ce5a356b9bc517ac52eed3a37ab846f8448001a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
(
"0x62b0dd4aab2b1a0a04e279e2b828791a10755528",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xf8709f3936599f93b769acf90c7178fd2ddcac1b5b4bc9949ee5a04b7e0823c2446eb84ef84c80880f43fc2c04ee0000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
(
"0x1ed9b1dd266b607ee278726d324b855a093394a6",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xe48200d3a0ef957210bca5b9b402d614eb8408c88cfbf4913eb6ab83ca233c8b8f0e626b54",
"0xf851808080a02743a5addaf4cf9b8c0c073e1eaa555deaaf8c41cb2b41958e88624fa45c2d908080808080a0bfbf6937911dfb88113fecdaa6bde822e4e99dae62489fcf61a91cb2f36793d680808080808080",
"0xf86f9e207a32b8ab5eb4b043c65b1f00c93f517bc8883c5cd31baf8e8a279475e3b84ef84c808801aa535d3d0c0000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
]);
let provider = factory.provider().unwrap();
for (target, expected_proof) in data {
let target = Address::from_str(target).unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &[]).unwrap();
similar_asserts::assert_eq!(
account_proof.proof,
expected_proof,
"proof for {target:?} does not match"
);
assert_eq!(account_proof.verify(root), Ok(()));
}
}
#[test]
fn testspec_empty_storage_proof() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, TEST_SPEC.clone()).unwrap();
let target = Address::from_str("0x1ed9b1dd266b607ee278726d324b855a093394a6").unwrap();
let slots = Vec::from([B256::with_last_byte(1), B256::with_last_byte(3)]);
let provider = factory.provider().unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &slots).unwrap();
assert_eq!(account_proof.storage_root, EMPTY_ROOT_HASH, "expected empty storage root");
assert_eq!(slots.len(), account_proof.storage_proofs.len());
for (idx, slot) in slots.into_iter().enumerate() {
let proof = account_proof.storage_proofs.get(idx).unwrap();
assert_eq!(proof, &StorageProof::new(slot));
assert_eq!(proof.verify(account_proof.storage_root), Ok(()));
}
assert_eq!(account_proof.verify(root), Ok(()));
}
#[test]
fn mainnet_genesis_account_proof() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, MAINNET.clone()).unwrap();
// Address from mainnet genesis allocation.
// keccak256 - `0xcf67b71c90b0d523dd5004cf206f325748da347685071b34812e21801f5270c4`
let target = Address::from_str("0x000d836201318ec6899a67540690382780743280").unwrap();
// `cast proof 0x000d836201318ec6899a67540690382780743280 --block 0`
let expected_account_proof = convert_to_proof([
"0xf90211a090dcaf88c40c7bbc95a912cbdde67c175767b31173df9ee4b0d733bfdd511c43a0babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bda0473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021a0bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0a04e44caecff45c9891f74f6a2156735886eedf6f1a733628ebc802ec79d844648a0a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7a0e823850f50bf72baae9d1733a36a444ab65d0a6faaba404f0583ce0ca4dad92da0f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721a07117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681a069eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472a0203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8ea09287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208a06fc2d754e304c48ce6a517753c62b1a9c1d5925b89707486d7fc08919e0a94eca07b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475a051f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0a089d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb80",
"0xf90211a0dae48f5b47930c28bb116fbd55e52cd47242c71bf55373b55eb2805ee2e4a929a00f1f37f337ec800e2e5974e2e7355f10f1a4832b39b846d916c3597a460e0676a0da8f627bb8fbeead17b318e0a8e4f528db310f591bb6ab2deda4a9f7ca902ab5a0971c662648d58295d0d0aa4b8055588da0037619951217c22052802549d94a2fa0ccc701efe4b3413fd6a61a6c9f40e955af774649a8d9fd212d046a5a39ddbb67a0d607cdb32e2bd635ee7f2f9e07bc94ddbd09b10ec0901b66628e15667aec570ba05b89203dc940e6fa70ec19ad4e01d01849d3a5baa0a8f9c0525256ed490b159fa0b84227d48df68aecc772939a59afa9e1a4ab578f7b698bdb1289e29b6044668ea0fd1c992070b94ace57e48cbf6511a16aa770c645f9f5efba87bbe59d0a042913a0e16a7ccea6748ae90de92f8aef3b3dc248a557b9ac4e296934313f24f7fced5fa042373cf4a00630d94de90d0a23b8f38ced6b0f7cb818b8925fee8f0c2a28a25aa05f89d2161c1741ff428864f7889866484cef622de5023a46e795dfdec336319fa07597a017664526c8c795ce1da27b8b72455c49657113e0455552dbc068c5ba31a0d5be9089012fda2c585a1b961e988ea5efcd3a06988e150a8682091f694b37c5a0f7b0352e38c315b2d9a14d51baea4ddee1770974c806e209355233c3c89dce6ea049bf6e8df0acafd0eff86defeeb305568e44d52d2235cf340ae15c6034e2b24180",
"0xf901f1a0cf67e0f5d5f8d70e53a6278056a14ddca46846f5ef69c7bde6810d058d4a9eda80a06732ada65afd192197fe7ce57792a7f25d26978e64e954b7b84a1f7857ac279da05439f8d011683a6fc07efb90afca198fd7270c795c835c7c85d91402cda992eaa0449b93033b6152d289045fdb0bf3f44926f831566faa0e616b7be1abaad2cb2da031be6c3752bcd7afb99b1bb102baf200f8567c394d464315323a363697646616a0a40e3ed11d906749aa501279392ffde868bd35102db41364d9c601fd651f974aa0044bfa4fe8dd1a58e6c7144da79326e94d1331c0b00373f6ae7f3662f45534b7a098005e3e48db68cb1dc9b9f034ff74d2392028ddf718b0f2084133017da2c2e7a02a62bc40414ee95b02e202a9e89babbabd24bef0abc3fc6dcd3e9144ceb0b725a0239facd895bbf092830390a8676f34b35b29792ae561f196f86614e0448a5792a0a4080f88925daff6b4ce26d188428841bd65655d8e93509f2106020e76d41eefa04918987904be42a6894256ca60203283d1b89139cf21f09f5719c44b8cdbb8f7a06201fc3ef0827e594d953b5e3165520af4fceb719e11cc95fd8d3481519bfd8ca05d0e353d596bd725b09de49c01ede0f29023f0153d7b6d401556aeb525b2959ba0cd367d0679950e9c5f2aa4298fd4b081ade2ea429d71ff390c50f8520e16e30880",
"0xf87180808080808080a0dbee8b33c73b86df839f309f7ac92eee19836e08b39302ffa33921b3c6a09f66a06068b283d51aeeee682b8fb5458354315d0b91737441ede5e137c18b4775174a8080808080a0fe7779c7d58c2fda43eba0a6644043c86ebb9ceb4836f89e30831f23eb059ece8080",
"0xf8719f20b71c90b0d523dd5004cf206f325748da347685071b34812e21801f5270c4b84ff84d80890ad78ebc5ac6200000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
]);
let provider = factory.provider().unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &[]).unwrap();
similar_asserts::assert_eq!(account_proof.proof, expected_account_proof);
assert_eq!(account_proof.verify(root), Ok(()));
}
#[test]
fn mainnet_genesis_account_proof_nonexistent() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, MAINNET.clone()).unwrap();
// Address that does not exist in mainnet genesis allocation.
// keccak256 - `0x18f415ffd7f66bb1924d90f0e82fb79ca8c6d8a3473cd9a95446a443b9db1761`
let target = Address::from_str("0x000d836201318ec6899a67540690382780743281").unwrap();
// `cast proof 0x000d836201318ec6899a67540690382780743281 --block 0`
let expected_account_proof = convert_to_proof([
"0xf90211a090dcaf88c40c7bbc95a912cbdde67c175767b31173df9ee4b0d733bfdd511c43a0babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bda0473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021a0bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0a04e44caecff45c9891f74f6a2156735886eedf6f1a733628ebc802ec79d844648a0a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7a0e823850f50bf72baae9d1733a36a444ab65d0a6faaba404f0583ce0ca4dad92da0f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721a07117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681a069eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472a0203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8ea09287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208a06fc2d754e304c48ce6a517753c62b1a9c1d5925b89707486d7fc08919e0a94eca07b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475a051f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0a089d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb80",
"0xf90211a0586b1ddec8db4824154209d355a1989b6c43aa69aba36e9d70c9faa53e7452baa0f86db47d628c73764d74b9ccaed73b8486d97a7731d57008fc9efaf417411860a0d9faed7b9ea107b5d98524246c977e782377f976e34f70717e8b1207f2f9b981a00218f59ccedf797c95e27c56405b9bf16845050fb43e773b66b26bc6992744f5a0dbf396f480c4e024156644adea7c331688d03742369e9d87ab8913bc439ff975a0aced524f39b22c62a5be512ddbca89f0b89b47c311065ccf423dee7013c7ea83a0c06b05f80b237b403adc019c0bc95b5de935021b14a75cbc18509eec60dfd83aa085339d45c4a52b7d523c301701f1ab339964e9c907440cff0a871c98dcf8811ea03ae9f6b8e227ec9be9461f0947b01696f78524c4519a6dee9fba14d209952cf9a0af17f551f9fa1ba4be41d0b342b160e2e8468d7e98a65a2dbf9d5fe5d6928024a0b850ac3bc03e9a309cc59ce5f1ab8db264870a7a22786081753d1db91897b8e6a09e796a4904bd78cb2655b5f346c94350e2d5f0dbf2bc00ac00871cd7ba46b241a0f6f0377427b900529caf32abf32ba1eb93f5f70153aa50b90bf55319a434c252a0725eaf27c8ee07e9b2511a6d6a0d71c649d855e8a9ed26e667903e2e94ae47cba0e4139fb48aa1a524d47f6e0df80314b88b52202d7e853da33c276aa8572283a8a05e9003d54a45935fdebae3513dc7cd16626dc05e1d903ae7f47f1a35aa6e234580",
"0xf901d1a0b7c55b381eb205712a2f5d1b7d6309ac725da79ab159cb77dc2783af36e6596da0b3b48aa390e0f3718b486ccc32b01682f92819e652315c1629058cd4d9bb1545a0e3c0cc68af371009f14416c27e17f05f4f696566d2ba45362ce5711d4a01d0e4a0bad1e085e431b510508e2a9e3712633a414b3fe6fd358635ab206021254c1e10a0f8407fe8d5f557b9e012d52e688139bd932fec40d48630d7ff4204d27f8cc68da08c6ca46eff14ad4950e65469c394ca9d6b8690513b1c1a6f91523af00082474c80a0630c034178cb1290d4d906edf28688804d79d5e37a3122c909adab19ac7dc8c5a059f6d047c5d1cc75228c4517a537763cb410c38554f273e5448a53bc3c7166e7a0d842f53ce70c3aad1e616fa6485d3880d15c936fcc306ec14ae35236e5a60549a0218ee2ee673c69b4e1b953194b2568157a69085b86e4f01644fa06ab472c6cf9a016a35a660ea496df7c0da646378bfaa9562f401e42a5c2fe770b7bbe22433585a0dd0fbbe227a4d50868cdbb3107573910fd97131ea8d835bef81d91a2fc30b175a06aafa3d78cf179bf055bd5ec629be0ff8352ce0aec9125a4d75be3ee7eb71f10a01d6817ef9f64fcbb776ff6df0c83138dcd2001bd752727af3e60f4afc123d8d58080"
]);
let provider = factory.provider().unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &[]).unwrap();
similar_asserts::assert_eq!(account_proof.proof, expected_account_proof);
assert_eq!(account_proof.verify(root), Ok(()));
}
#[test]
fn holesky_deposit_contract_proof() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, HOLESKY.clone()).unwrap();
let target = Address::from_str("0x4242424242424242424242424242424242424242").unwrap();
// existent
let slot_22 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000022")
.unwrap();
let slot_23 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000023")
.unwrap();
let slot_24 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000024")
.unwrap();
// non-existent
let slot_100 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000100")
.unwrap();
let slots = Vec::from([slot_22, slot_23, slot_24, slot_100]);
// `cast proof 0x4242424242424242424242424242424242424242 0x22 0x23 0x24 0x100 --block 0`
let expected = AccountProof {
address: target,
info: Some(Account {
balance: U256::ZERO,
nonce: 0,
bytecode_hash: Some(B256::from_str("0x2034f79e0e33b0ae6bef948532021baceb116adf2616478703bec6b17329f1cc").unwrap())
}),
storage_root: B256::from_str("0x556a482068355939c95a3412bdb21213a301483edb1b64402fb66ac9f3583599").unwrap(),
proof: convert_to_proof([
"0xf90211a0ea92fb71507739d5afe328d607b2c5e98322b7aa7cdfeccf817543058b54af70a0bd0c2525b5bee47abf7120c9e01ec3249699d687f80ebb96ed9ad9de913dbab0a0ab4b14b89416eb23c6b64204fa45cfcb39d4220016a9cd0815ebb751fe45eb71a0986ae29c2148b9e61f9a7543f44a1f8d029f1c5095b359652e9ec94e64b5d393a0555d54aa23ed990b0488153418637df7b2c878b604eb761aa2673b609937b0eba0140afb6a3909cc6047b3d44af13fc83f161a7e4c4ddba430a2841862912eb222a031b1185c1f455022d9e42ce04a71f174eb9441b1ada67449510500f4d85b3b22a051ecd01e18113b23cc65e62f67d69b33ee15d20bf81a6b524f7df90ded00ca15a0703769d6a7befad000bc2b4faae3e41b809b1b1241fe2964262554e7e3603488a0e5de7f600e4e6c3c3e5630e0c66f50506a17c9715642fccb63667e81397bbf93a095f783cd1d464a60e3c8adcadc28c6eb9fec7306664df39553be41dccc909606a04225fda3b89f0c59bf40129d1d5e5c3bf67a2129f0c55e53ffdd2cebf185d644a078e0f7fd3ae5a9bc90f66169614211b48fe235eb64818b3935d3e69c53523b9aa0a870e00e53ebaa1e9ec16e5f36606fd7d21d3a3c96894c0a2a23550949d4fdf7a0809226b69cee1f4f22ced1974e7805230da1909036a49a7652428999431afac2a0f11593b2407e86e11997325d8df2d22d937bbe0aef8302ba40c6be0601b04fc380",
"0xf901f1a09da7d9755fe0c558b3c3de9fdcdf9f28ae641f38c9787b05b73ab22ae53af3e2a0d9990bf0b810d1145ecb2b011fd68c63cc85564e6724166fd4a9520180706e5fa05f5f09855df46330aa310e8d6be5fb82d1a4b975782d9b29acf06ac8d3e72b1ca0ca976997ddaf06f18992f6207e4f6a05979d07acead96568058789017cc6d06ba04d78166b48044fdc28ed22d2fd39c8df6f8aaa04cb71d3a17286856f6893ff83a004f8c7cc4f1335182a1709fb28fc67d52e59878480210abcba864d5d1fd4a066a0fc3b71c33e2e6b77c5e494c1db7fdbb447473f003daf378c7a63ba9bf3f0049d80a07b8e7a21c1178d28074f157b50fca85ee25c12568ff8e9706dcbcdacb77bf854a0973274526811393ea0bf4811ca9077531db00d06b86237a2ecd683f55ba4bcb0a03a93d726d7487874e51b52d8d534c63aa2a689df18e3b307c0d6cb0a388b00f3a06aa67101d011d1c22fe739ef83b04b5214a3e2f8e1a2625d8bfdb116b447e86fa02dd545b33c62d33a183e127a08a4767fba891d9f3b94fc20a2ca02600d6d1fffa0f3b039a4f32349e85c782d1164c1890e5bf16badc9ee4cf827db6afd2229dde6a0d9240a9d2d5851d05a97ff3305334dfdb0101e1e321fc279d2bb3cad6afa8fc8a01b69c6ab5173de8a8ec53a6ebba965713a4cc7feb86cb3e230def37c230ca2b280",
"0xf869a0202a47fc6863b89a6b51890ef3c1550d560886c027141d2058ba1e2d4c66d99ab846f8448080a0556a482068355939c95a3412bdb21213a301483edb1b64402fb66ac9f3583599a02034f79e0e33b0ae6bef948532021baceb116adf2616478703bec6b17329f1cc"
]),
storage_proofs: Vec::from([
StorageProof {
key: slot_22,
nibbles: Nibbles::unpack(keccak256(slot_22)),
value: U256::from_str("0xf5a5fd42d16a20302798ef6ed309979b43003d2320d9f0e8ea9831a92759fb4b").unwrap(),
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf85180a0776aa456ba9c5008e03b82b841a9cf2fc1e8578cfacd5c9015804eae315f17fb80808080808080808080808080a072e3e284d47badbb0a5ca1421e1179d3ea90cc10785b26b74fb8a81f0f9e841880",
"0xf843a020035b26e3e9eee00e0d72fd1ee8ddca6894550dca6916ea2ac6baa90d11e510a1a0f5a5fd42d16a20302798ef6ed309979b43003d2320d9f0e8ea9831a92759fb4b"
])
},
StorageProof {
key: slot_23,
nibbles: Nibbles::unpack(keccak256(slot_23)),
value: U256::from_str("0xdb56114e00fdd4c1f85c892bf35ac9a89289aaecb1ebd0a96cde606a748b5d71").unwrap(),
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf8518080808080a0d546c4ca227a267d29796643032422374624ed109b3d94848c5dc06baceaee76808080808080a027c48e210ccc6e01686be2d4a199d35f0e1e8df624a8d3a17c163be8861acd6680808080",
"0xf843a0207b2b5166478fd4318d2acc6cc2c704584312bdd8781b32d5d06abda57f4230a1a0db56114e00fdd4c1f85c892bf35ac9a89289aaecb1ebd0a96cde606a748b5d71"
])
},
StorageProof {
key: slot_24,
nibbles: Nibbles::unpack(keccak256(slot_24)),
value: U256::from_str("0xc78009fdf07fc56a11f122370658a353aaa542ed63e44c4bc15ff4cd105ab33c").unwrap(),
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf85180808080a030263404acfee103d0b1019053ff3240fce433c69b709831673285fa5887ce4c80808080808080a0f8f1fbb1f7b482d9860480feebb83ff54a8b6ec1ead61cc7d2f25d7c01659f9c80808080",
"0xf843a020d332d19b93bcabe3cce7ca0c18a052f57e5fd03b4758a09f30f5ddc4b22ec4a1a0c78009fdf07fc56a11f122370658a353aaa542ed63e44c4bc15ff4cd105ab33c"
])
},
StorageProof {
key: slot_100,
nibbles: Nibbles::unpack(keccak256(slot_100)),
value: U256::ZERO,
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf891a090bacef44b189ddffdc5f22edc70fe298c58e5e523e6e1dfdf7dbc6d657f7d1b80a026eed68746028bc369eb456b7d3ee475aa16f34e5eaa0c98fdedb9c59ebc53b0808080a09ce86197173e14e0633db84ce8eea32c5454eebe954779255644b45b717e8841808080a0328c7afb2c58ef3f8c4117a8ebd336f1a61d24591067ed9c5aae94796cac987d808080808080"
])
},
])
};
let provider = factory.provider().unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &slots).unwrap();
similar_asserts::assert_eq!(account_proof, expected);
assert_eq!(account_proof.verify(root), Ok(()));
}

View File

@ -0,0 +1,771 @@
use proptest::{prelude::ProptestConfig, proptest};
use proptest_arbitrary_interop::arb;
use reth_db::{tables, test_utils::TempDatabase, DatabaseEnv};
use reth_db_api::{
cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO},
transaction::DbTxMut,
};
use reth_primitives::{hex_literal::hex, Account, StorageEntry, U256};
use reth_provider::{test_utils::create_test_provider_factory, DatabaseProviderRW};
use reth_trie::{
prefix_set::PrefixSetMut,
test_utils::{state_root, state_root_prehashed, storage_root, storage_root_prehashed},
BranchNodeCompact, StateRoot, StorageRoot, TrieMask,
};
use reth_trie_common::triehash::KeccakHasher;
use reth_trie_db::DatabaseStateRoot;
use std::{
collections::{BTreeMap, HashMap},
ops::Mul,
str::FromStr,
sync::Arc,
};
use alloy_rlp::Encodable;
use reth_db_api::transaction::DbTx;
use reth_primitives::{constants::EMPTY_ROOT_HASH, keccak256, Address, B256};
use reth_trie::{
prefix_set::TriePrefixSets, updates::StorageTrieUpdates, HashBuilder,
IntermediateStateRootState, Nibbles, StateRootProgress, TrieAccount,
};
fn insert_account(
tx: &impl DbTxMut,
address: Address,
account: Account,
storage: &BTreeMap<B256, U256>,
) {
let hashed_address = keccak256(address);
tx.put::<tables::HashedAccounts>(hashed_address, account).unwrap();
insert_storage(tx, hashed_address, storage);
}
fn insert_storage(tx: &impl DbTxMut, hashed_address: B256, storage: &BTreeMap<B256, U256>) {
for (k, v) in storage {
tx.put::<tables::HashedStorages>(
hashed_address,
StorageEntry { key: keccak256(k), value: *v },
)
.unwrap();
}
}
fn incremental_vs_full_root(inputs: &[&str], modified: &str) {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let hashed_address = B256::with_last_byte(1);
let mut hashed_storage_cursor =
tx.tx_ref().cursor_dup_write::<tables::HashedStorages>().unwrap();
let data = inputs.iter().map(|x| B256::from_str(x).unwrap());
let value = U256::from(0);
for key in data {
hashed_storage_cursor.upsert(hashed_address, StorageEntry { key, value }).unwrap();
}
// Generate the intermediate nodes on the receiving end of the channel
let (_, _, trie_updates) =
StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address).root_with_updates().unwrap();
// 1. Some state transition happens, update the hashed storage to the new value
let modified_key = B256::from_str(modified).unwrap();
let value = U256::from(1);
if hashed_storage_cursor.seek_by_key_subkey(hashed_address, modified_key).unwrap().is_some() {
hashed_storage_cursor.delete_current().unwrap();
}
hashed_storage_cursor
.upsert(hashed_address, StorageEntry { key: modified_key, value })
.unwrap();
// 2. Calculate full merkle root
let loader = StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address);
let modified_root = loader.root().unwrap();
// Update the intermediate roots table so that we can run the incremental verification
trie_updates.write_to_database(tx.tx_ref(), hashed_address).unwrap();
// 3. Calculate the incremental root
let mut storage_changes = PrefixSetMut::default();
storage_changes.insert(Nibbles::unpack(modified_key));
let loader = StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address)
.with_prefix_set(storage_changes.freeze());
let incremental_root = loader.root().unwrap();
assert_eq!(modified_root, incremental_root);
}
#[test]
fn branch_node_child_changes() {
incremental_vs_full_root(
&[
"1000000000000000000000000000000000000000000000000000000000000000",
"1100000000000000000000000000000000000000000000000000000000000000",
"1110000000000000000000000000000000000000000000000000000000000000",
"1200000000000000000000000000000000000000000000000000000000000000",
"1220000000000000000000000000000000000000000000000000000000000000",
"1320000000000000000000000000000000000000000000000000000000000000",
],
"1200000000000000000000000000000000000000000000000000000000000000",
);
}
#[test]
fn arbitrary_storage_root() {
proptest!(ProptestConfig::with_cases(10), |(item in arb::<(Address, std::collections::BTreeMap<B256, U256>)>())| {
let (address, storage) = item;
let hashed_address = keccak256(address);
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
for (key, value) in &storage {
tx.tx_ref().put::<tables::HashedStorages>(
hashed_address,
StorageEntry { key: keccak256(key), value: *value },
)
.unwrap();
}
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let got = StorageRoot::from_tx(tx.tx_ref(), address).root().unwrap();
let expected = storage_root(storage.into_iter());
assert_eq!(expected, got);
});
}
#[test]
// This ensures we dont add empty accounts to the trie
fn test_empty_account() {
let state: State = BTreeMap::from([
(
Address::random(),
(
Account { nonce: 0, balance: U256::from(0), bytecode_hash: None },
BTreeMap::from([(B256::with_last_byte(0x4), U256::from(12))]),
),
),
(
Address::random(),
(
Account { nonce: 0, balance: U256::from(0), bytecode_hash: None },
BTreeMap::default(),
),
),
(
Address::random(),
(
Account {
nonce: 155,
balance: U256::from(414241124u32),
bytecode_hash: Some(keccak256("test")),
},
BTreeMap::from([
(B256::ZERO, U256::from(3)),
(B256::with_last_byte(2), U256::from(1)),
]),
),
),
]);
test_state_root_with_state(state);
}
#[test]
// This ensures we return an empty root when there are no storage entries
fn test_empty_storage_root() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let address = Address::random();
let code = "el buen fla";
let account = Account {
nonce: 155,
balance: U256::from(414241124u32),
bytecode_hash: Some(keccak256(code)),
};
insert_account(tx.tx_ref(), address, account, &Default::default());
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let got = StorageRoot::from_tx(tx.tx_ref(), address).root().unwrap();
assert_eq!(got, EMPTY_ROOT_HASH);
}
#[test]
// This ensures that the walker goes over all the storage slots
fn test_storage_root() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let address = Address::random();
let storage =
BTreeMap::from([(B256::ZERO, U256::from(3)), (B256::with_last_byte(2), U256::from(1))]);
let code = "el buen fla";
let account = Account {
nonce: 155,
balance: U256::from(414241124u32),
bytecode_hash: Some(keccak256(code)),
};
insert_account(tx.tx_ref(), address, account, &storage);
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let got = StorageRoot::from_tx(tx.tx_ref(), address).root().unwrap();
assert_eq!(storage_root(storage.into_iter()), got);
}
type State = BTreeMap<Address, (Account, BTreeMap<B256, U256>)>;
#[test]
fn arbitrary_state_root() {
proptest!(
ProptestConfig::with_cases(10), | (state in arb::<State>()) | {
test_state_root_with_state(state);
}
);
}
#[test]
fn arbitrary_state_root_with_progress() {
proptest!(
ProptestConfig::with_cases(10), | (state in arb::<State>()) | {
let hashed_entries_total = state.len() +
state.values().map(|(_, slots)| slots.len()).sum::<usize>();
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
for (address, (account, storage)) in &state {
insert_account(tx.tx_ref(), *address, *account, storage)
}
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let expected = state_root(state);
let threshold = 10;
let mut got = None;
let mut hashed_entries_walked = 0;
let mut intermediate_state: Option<Box<IntermediateStateRootState>> = None;
while got.is_none() {
let calculator = StateRoot::from_tx(tx.tx_ref())
.with_threshold(threshold)
.with_intermediate_state(intermediate_state.take().map(|state| *state));
match calculator.root_with_progress().unwrap() {
StateRootProgress::Progress(state, walked, _) => {
intermediate_state = Some(state);
hashed_entries_walked += walked;
},
StateRootProgress::Complete(root, walked, _) => {
got = Some(root);
hashed_entries_walked += walked;
},
};
}
assert_eq!(expected, got.unwrap());
assert_eq!(hashed_entries_total, hashed_entries_walked)
}
);
}
fn test_state_root_with_state(state: State) {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
for (address, (account, storage)) in &state {
insert_account(tx.tx_ref(), *address, *account, storage)
}
tx.commit().unwrap();
let expected = state_root(state);
let tx = factory.provider_rw().unwrap();
let got = StateRoot::from_tx(tx.tx_ref()).root().unwrap();
assert_eq!(expected, got);
}
fn encode_account(account: Account, storage_root: Option<B256>) -> Vec<u8> {
let account = TrieAccount::from((account, storage_root.unwrap_or(EMPTY_ROOT_HASH)));
let mut account_rlp = Vec::with_capacity(account.length());
account.encode(&mut account_rlp);
account_rlp
}
#[test]
fn storage_root_regression() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
// Some address whose hash starts with 0xB041
let address3 = Address::from_str("16b07afd1c635f77172e842a000ead9a2a222459").unwrap();
let key3 = keccak256(address3);
assert_eq!(key3[0], 0xB0);
assert_eq!(key3[1], 0x41);
let storage = BTreeMap::from(
[
("1200000000000000000000000000000000000000000000000000000000000000", 0x42),
("1400000000000000000000000000000000000000000000000000000000000000", 0x01),
("3000000000000000000000000000000000000000000000000000000000E00000", 0x127a89),
("3000000000000000000000000000000000000000000000000000000000E00001", 0x05),
]
.map(|(slot, val)| (B256::from_str(slot).unwrap(), U256::from(val))),
);
let mut hashed_storage_cursor =
tx.tx_ref().cursor_dup_write::<tables::HashedStorages>().unwrap();
for (hashed_slot, value) in storage.clone() {
hashed_storage_cursor.upsert(key3, StorageEntry { key: hashed_slot, value }).unwrap();
}
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let account3_storage_root = StorageRoot::from_tx(tx.tx_ref(), address3).root().unwrap();
let expected_root = storage_root_prehashed(storage);
assert_eq!(expected_root, account3_storage_root);
}
#[test]
fn account_and_storage_trie() {
let ether = U256::from(1e18);
let storage = BTreeMap::from(
[
("1200000000000000000000000000000000000000000000000000000000000000", 0x42),
("1400000000000000000000000000000000000000000000000000000000000000", 0x01),
("3000000000000000000000000000000000000000000000000000000000E00000", 0x127a89),
("3000000000000000000000000000000000000000000000000000000000E00001", 0x05),
]
.map(|(slot, val)| (B256::from_str(slot).unwrap(), U256::from(val))),
);
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor = tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let mut hashed_storage_cursor =
tx.tx_ref().cursor_dup_write::<tables::HashedStorages>().unwrap();
let mut hash_builder = HashBuilder::default();
// Insert first account
let key1 =
B256::from_str("b000000000000000000000000000000000000000000000000000000000000000").unwrap();
let account1 = Account { nonce: 0, balance: U256::from(3).mul(ether), bytecode_hash: None };
hashed_account_cursor.upsert(key1, account1).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key1), &encode_account(account1, None));
// Some address whose hash starts with 0xB040
let address2 = Address::from_str("7db3e81b72d2695e19764583f6d219dbee0f35ca").unwrap();
let key2 = keccak256(address2);
assert_eq!(key2[0], 0xB0);
assert_eq!(key2[1], 0x40);
let account2 = Account { nonce: 0, balance: ether, ..Default::default() };
hashed_account_cursor.upsert(key2, account2).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key2), &encode_account(account2, None));
// Some address whose hash starts with 0xB041
let address3 = Address::from_str("16b07afd1c635f77172e842a000ead9a2a222459").unwrap();
let key3 = keccak256(address3);
assert_eq!(key3[0], 0xB0);
assert_eq!(key3[1], 0x41);
let code_hash =
B256::from_str("5be74cad16203c4905c068b012a2e9fb6d19d036c410f16fd177f337541440dd").unwrap();
let account3 =
Account { nonce: 0, balance: U256::from(2).mul(ether), bytecode_hash: Some(code_hash) };
hashed_account_cursor.upsert(key3, account3).unwrap();
for (hashed_slot, value) in storage {
if hashed_storage_cursor
.seek_by_key_subkey(key3, hashed_slot)
.unwrap()
.filter(|e| e.key == hashed_slot)
.is_some()
{
hashed_storage_cursor.delete_current().unwrap();
}
hashed_storage_cursor.upsert(key3, StorageEntry { key: hashed_slot, value }).unwrap();
}
let account3_storage_root = StorageRoot::from_tx(tx.tx_ref(), address3).root().unwrap();
hash_builder
.add_leaf(Nibbles::unpack(key3), &encode_account(account3, Some(account3_storage_root)));
let key4a =
B256::from_str("B1A0000000000000000000000000000000000000000000000000000000000000").unwrap();
let account4a = Account { nonce: 0, balance: U256::from(4).mul(ether), ..Default::default() };
hashed_account_cursor.upsert(key4a, account4a).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key4a), &encode_account(account4a, None));
let key5 =
B256::from_str("B310000000000000000000000000000000000000000000000000000000000000").unwrap();
let account5 = Account { nonce: 0, balance: U256::from(8).mul(ether), ..Default::default() };
hashed_account_cursor.upsert(key5, account5).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key5), &encode_account(account5, None));
let key6 =
B256::from_str("B340000000000000000000000000000000000000000000000000000000000000").unwrap();
let account6 = Account { nonce: 0, balance: U256::from(1).mul(ether), ..Default::default() };
hashed_account_cursor.upsert(key6, account6).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key6), &encode_account(account6, None));
// Populate account & storage trie DB tables
let expected_root =
B256::from_str("72861041bc90cd2f93777956f058a545412b56de79af5eb6b8075fe2eabbe015").unwrap();
let computed_expected_root: B256 = triehash::trie_root::<KeccakHasher, _, _, _>([
(key1, encode_account(account1, None)),
(key2, encode_account(account2, None)),
(key3, encode_account(account3, Some(account3_storage_root))),
(key4a, encode_account(account4a, None)),
(key5, encode_account(account5, None)),
(key6, encode_account(account6, None)),
]);
// Check computed trie root to ensure correctness
assert_eq!(computed_expected_root, expected_root);
// Check hash builder root
assert_eq!(hash_builder.root(), computed_expected_root);
// Check state root calculation from scratch
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(root, computed_expected_root);
// Check account trie
let account_updates = trie_updates.clone().into_sorted();
let account_updates = account_updates.account_nodes_ref();
assert_eq!(account_updates.len(), 2);
let (nibbles1a, node1a) = account_updates.first().unwrap();
assert_eq!(nibbles1a[..], [0xB]);
assert_eq!(node1a.state_mask, TrieMask::new(0b1011));
assert_eq!(node1a.tree_mask, TrieMask::new(0b0001));
assert_eq!(node1a.hash_mask, TrieMask::new(0b1001));
assert_eq!(node1a.root_hash, None);
assert_eq!(node1a.hashes.len(), 2);
let (nibbles2a, node2a) = account_updates.last().unwrap();
assert_eq!(nibbles2a[..], [0xB, 0x0]);
assert_eq!(node2a.state_mask, TrieMask::new(0b10001));
assert_eq!(node2a.tree_mask, TrieMask::new(0b00000));
assert_eq!(node2a.hash_mask, TrieMask::new(0b10000));
assert_eq!(node2a.root_hash, None);
assert_eq!(node2a.hashes.len(), 1);
// Check storage trie
let mut updated_storage_trie =
trie_updates.storage_tries_ref().iter().filter(|(_, u)| !u.storage_nodes_ref().is_empty());
assert_eq!(updated_storage_trie.clone().count(), 1);
let (_, storage_trie_updates) = updated_storage_trie.next().unwrap();
assert_eq!(storage_trie_updates.storage_nodes_ref().len(), 1);
let (nibbles3, node3) = storage_trie_updates.storage_nodes_ref().iter().next().unwrap();
assert!(nibbles3.is_empty());
assert_eq!(node3.state_mask, TrieMask::new(0b1010));
assert_eq!(node3.tree_mask, TrieMask::new(0b0000));
assert_eq!(node3.hash_mask, TrieMask::new(0b0010));
assert_eq!(node3.hashes.len(), 1);
assert_eq!(node3.root_hash, Some(account3_storage_root));
// Add an account
// Some address whose hash starts with 0xB1
let address4b = Address::from_str("4f61f2d5ebd991b85aa1677db97307caf5215c91").unwrap();
let key4b = keccak256(address4b);
assert_eq!(key4b.0[0], key4a.0[0]);
let account4b = Account { nonce: 0, balance: U256::from(5).mul(ether), bytecode_hash: None };
hashed_account_cursor.upsert(key4b, account4b).unwrap();
let mut prefix_set = PrefixSetMut::default();
prefix_set.insert(Nibbles::unpack(key4b));
let expected_state_root =
B256::from_str("8e263cd4eefb0c3cbbb14e5541a66a755cad25bcfab1e10dd9d706263e811b28").unwrap();
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets {
account_prefix_set: prefix_set.freeze(),
..Default::default()
})
.root_with_updates()
.unwrap();
assert_eq!(root, expected_state_root);
let account_updates = trie_updates.into_sorted();
let account_updates = account_updates.account_nodes_ref();
assert_eq!(account_updates.len(), 2);
let (nibbles1b, node1b) = account_updates.first().unwrap();
assert_eq!(nibbles1b[..], [0xB]);
assert_eq!(node1b.state_mask, TrieMask::new(0b1011));
assert_eq!(node1b.tree_mask, TrieMask::new(0b0001));
assert_eq!(node1b.hash_mask, TrieMask::new(0b1011));
assert_eq!(node1b.root_hash, None);
assert_eq!(node1b.hashes.len(), 3);
assert_eq!(node1a.hashes[0], node1b.hashes[0]);
assert_eq!(node1a.hashes[1], node1b.hashes[2]);
let (nibbles2b, node2b) = account_updates.last().unwrap();
assert_eq!(nibbles2b[..], [0xB, 0x0]);
assert_eq!(node2a, node2b);
tx.commit().unwrap();
{
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor =
tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let account = hashed_account_cursor.seek_exact(key2).unwrap().unwrap();
hashed_account_cursor.delete_current().unwrap();
let mut account_prefix_set = PrefixSetMut::default();
account_prefix_set.insert(Nibbles::unpack(account.0));
let computed_expected_root: B256 = triehash::trie_root::<KeccakHasher, _, _, _>([
(key1, encode_account(account1, None)),
// DELETED: (key2, encode_account(account2, None)),
(key3, encode_account(account3, Some(account3_storage_root))),
(key4a, encode_account(account4a, None)),
(key4b, encode_account(account4b, None)),
(key5, encode_account(account5, None)),
(key6, encode_account(account6, None)),
]);
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets {
account_prefix_set: account_prefix_set.freeze(),
..Default::default()
})
.root_with_updates()
.unwrap();
assert_eq!(root, computed_expected_root);
assert_eq!(
trie_updates.account_nodes_ref().len() + trie_updates.removed_nodes_ref().len(),
1
);
assert_eq!(trie_updates.account_nodes_ref().len(), 1);
let (nibbles1c, node1c) = trie_updates.account_nodes_ref().iter().next().unwrap();
assert_eq!(nibbles1c[..], [0xB]);
assert_eq!(node1c.state_mask, TrieMask::new(0b1011));
assert_eq!(node1c.tree_mask, TrieMask::new(0b0000));
assert_eq!(node1c.hash_mask, TrieMask::new(0b1011));
assert_eq!(node1c.root_hash, None);
assert_eq!(node1c.hashes.len(), 3);
assert_ne!(node1c.hashes[0], node1b.hashes[0]);
assert_eq!(node1c.hashes[1], node1b.hashes[1]);
assert_eq!(node1c.hashes[2], node1b.hashes[2]);
}
{
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor =
tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let account2 = hashed_account_cursor.seek_exact(key2).unwrap().unwrap();
hashed_account_cursor.delete_current().unwrap();
let account3 = hashed_account_cursor.seek_exact(key3).unwrap().unwrap();
hashed_account_cursor.delete_current().unwrap();
let mut account_prefix_set = PrefixSetMut::default();
account_prefix_set.insert(Nibbles::unpack(account2.0));
account_prefix_set.insert(Nibbles::unpack(account3.0));
let computed_expected_root: B256 = triehash::trie_root::<KeccakHasher, _, _, _>([
(key1, encode_account(account1, None)),
// DELETED: (key2, encode_account(account2, None)),
// DELETED: (key3, encode_account(account3, Some(account3_storage_root))),
(key4a, encode_account(account4a, None)),
(key4b, encode_account(account4b, None)),
(key5, encode_account(account5, None)),
(key6, encode_account(account6, None)),
]);
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets {
account_prefix_set: account_prefix_set.freeze(),
..Default::default()
})
.root_with_updates()
.unwrap();
assert_eq!(root, computed_expected_root);
assert_eq!(
trie_updates.account_nodes_ref().len() + trie_updates.removed_nodes_ref().len(),
1
);
assert!(!trie_updates
.storage_tries_ref()
.iter()
.any(|(_, u)| !u.storage_nodes_ref().is_empty() || !u.removed_nodes_ref().is_empty())); // no storage root update
assert_eq!(trie_updates.account_nodes_ref().len(), 1);
let (nibbles1d, node1d) = trie_updates.account_nodes_ref().iter().next().unwrap();
assert_eq!(nibbles1d[..], [0xB]);
assert_eq!(node1d.state_mask, TrieMask::new(0b1011));
assert_eq!(node1d.tree_mask, TrieMask::new(0b0000));
assert_eq!(node1d.hash_mask, TrieMask::new(0b1010));
assert_eq!(node1d.root_hash, None);
assert_eq!(node1d.hashes.len(), 2);
assert_eq!(node1d.hashes[0], node1b.hashes[1]);
assert_eq!(node1d.hashes[1], node1b.hashes[2]);
}
}
#[test]
fn account_trie_around_extension_node() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let expected = extension_node_trie(&tx);
let (got, updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(expected, got);
assert_trie_updates(updates.account_nodes_ref());
}
#[test]
fn account_trie_around_extension_node_with_dbtrie() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let expected = extension_node_trie(&tx);
let (got, updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(expected, got);
updates.write_to_database(tx.tx_ref()).unwrap();
// read the account updates from the db
let mut accounts_trie = tx.tx_ref().cursor_read::<tables::AccountsTrie>().unwrap();
let walker = accounts_trie.walk(None).unwrap();
let account_updates = walker
.into_iter()
.map(|item| {
let (key, node) = item.unwrap();
(key.0, node)
})
.collect();
assert_trie_updates(&account_updates);
}
proptest! {
#![proptest_config(ProptestConfig {
cases: 128, ..ProptestConfig::default()
})]
#[test]
fn fuzz_state_root_incremental(account_changes: [BTreeMap<B256, U256>; 5]) {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor = tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let mut state = BTreeMap::default();
for accounts in account_changes {
let should_generate_changeset = !state.is_empty();
let mut changes = PrefixSetMut::default();
for (hashed_address, balance) in accounts.clone() {
hashed_account_cursor.upsert(hashed_address, Account { balance, ..Default::default() }).unwrap();
if should_generate_changeset {
changes.insert(Nibbles::unpack(hashed_address));
}
}
let (state_root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets { account_prefix_set: changes.freeze(), ..Default::default() })
.root_with_updates()
.unwrap();
state.append(&mut accounts.clone());
let expected_root = state_root_prehashed(
state.iter().map(|(&key, &balance)| (key, (Account { balance, ..Default::default() }, std::iter::empty())))
);
assert_eq!(expected_root, state_root);
trie_updates.write_to_database(tx.tx_ref()).unwrap();
}
}
}
#[test]
fn storage_trie_around_extension_node() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let hashed_address = B256::random();
let (expected_root, expected_updates) = extension_node_storage_trie(&tx, hashed_address);
let (got, _, updates) =
StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address).root_with_updates().unwrap();
assert_eq!(expected_root, got);
assert_eq!(expected_updates, updates);
assert_trie_updates(updates.storage_nodes_ref());
}
fn extension_node_storage_trie(
tx: &DatabaseProviderRW<Arc<TempDatabase<DatabaseEnv>>>,
hashed_address: B256,
) -> (B256, StorageTrieUpdates) {
let value = U256::from(1);
let mut hashed_storage = tx.tx_ref().cursor_write::<tables::HashedStorages>().unwrap();
let mut hb = HashBuilder::default().with_updates(true);
for key in [
hex!("30af561000000000000000000000000000000000000000000000000000000000"),
hex!("30af569000000000000000000000000000000000000000000000000000000000"),
hex!("30af650000000000000000000000000000000000000000000000000000000000"),
hex!("30af6f0000000000000000000000000000000000000000000000000000000000"),
hex!("30af8f0000000000000000000000000000000000000000000000000000000000"),
hex!("3100000000000000000000000000000000000000000000000000000000000000"),
] {
hashed_storage.upsert(hashed_address, StorageEntry { key: B256::new(key), value }).unwrap();
hb.add_leaf(Nibbles::unpack(key), &alloy_rlp::encode_fixed_size(&value));
}
let root = hb.root();
let (_, updates) = hb.split();
let trie_updates = StorageTrieUpdates::new(updates);
(root, trie_updates)
}
fn extension_node_trie(tx: &DatabaseProviderRW<Arc<TempDatabase<DatabaseEnv>>>) -> B256 {
let a = Account { nonce: 0, balance: U256::from(1u64), bytecode_hash: Some(B256::random()) };
let val = encode_account(a, None);
let mut hashed_accounts = tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let mut hb = HashBuilder::default();
for key in [
hex!("30af561000000000000000000000000000000000000000000000000000000000"),
hex!("30af569000000000000000000000000000000000000000000000000000000000"),
hex!("30af650000000000000000000000000000000000000000000000000000000000"),
hex!("30af6f0000000000000000000000000000000000000000000000000000000000"),
hex!("30af8f0000000000000000000000000000000000000000000000000000000000"),
hex!("3100000000000000000000000000000000000000000000000000000000000000"),
] {
hashed_accounts.upsert(B256::new(key), a).unwrap();
hb.add_leaf(Nibbles::unpack(key), &val);
}
hb.root()
}
fn assert_trie_updates(account_updates: &HashMap<Nibbles, BranchNodeCompact>) {
assert_eq!(account_updates.len(), 2);
let node = account_updates.get(&[0x3][..]).unwrap();
let expected = BranchNodeCompact::new(0b0011, 0b0001, 0b0000, vec![], None);
assert_eq!(node, &expected);
let node = account_updates.get(&[0x3, 0x0, 0xA, 0xF][..]).unwrap();
assert_eq!(node.state_mask, TrieMask::new(0b101100000));
assert_eq!(node.tree_mask, TrieMask::new(0b000000000));
assert_eq!(node.hash_mask, TrieMask::new(0b001000000));
assert_eq!(node.root_hash, None);
assert_eq!(node.hashes.len(), 1);
}

View File

@ -17,6 +17,7 @@ reth-primitives.workspace = true
reth-db.workspace = true
reth-db-api.workspace = true
reth-trie.workspace = true
reth-trie-db.workspace = true
reth-execution-errors.workspace = true
reth-provider.workspace = true

View File

@ -11,6 +11,7 @@ use reth_tasks::pool::BlockingTaskPool;
use reth_trie::{
hashed_cursor::HashedPostStateCursorFactory, HashedPostState, HashedStorage, StateRoot,
};
use reth_trie_db::DatabaseStateRoot;
use reth_trie_parallel::{async_root::AsyncStateRoot, parallel_root::ParallelStateRoot};
use std::collections::HashMap;

View File

@ -181,295 +181,3 @@ where
Ok((root, proofs))
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::StateRoot;
use once_cell::sync::Lazy;
use reth_chainspec::{Chain, ChainSpec, HOLESKY, MAINNET};
use reth_db_api::database::Database;
use reth_primitives::{Account, Bytes, StorageEntry, U256};
use reth_provider::{test_utils::create_test_provider_factory, HashingWriter, ProviderFactory};
use reth_storage_errors::provider::ProviderResult;
use std::{str::FromStr, sync::Arc};
/*
World State (sampled from <https://ethereum.stackexchange.com/questions/268/ethereum-block-architecture/6413#6413>)
| address | prefix | hash | balance
|--------------------------------------------|-----------|--------------------------------------------------------------------|--------
| 0x2031f89b3ea8014eb51a78c316e42af3e0d7695f | 0xa711355 | 0xa711355ec1c8f7e26bb3ccbcb0b75d870d15846c0b98e5cc452db46c37faea40 | 45 eth
| 0x33f0fc440b8477fcfbe9d0bf8649e7dea9baedb2 | 0xa77d337 | 0xa77d337781e762f3577784bab7491fcc43e291ce5a356b9bc517ac52eed3a37a | 1 wei
| 0x62b0dd4aab2b1a0a04e279e2b828791a10755528 | 0xa7f9365 | 0xa7f936599f93b769acf90c7178fd2ddcac1b5b4bc9949ee5a04b7e0823c2446e | 1.1 eth
| 0x1ed9b1dd266b607ee278726d324b855a093394a6 | 0xa77d397 | 0xa77d397a32b8ab5eb4b043c65b1f00c93f517bc8883c5cd31baf8e8a279475e3 | .12 eth
All expected testspec results were obtained from querying proof RPC on the running geth instance `geth init crates/trie/testdata/proof-genesis.json && geth --http`.
*/
static TEST_SPEC: Lazy<Arc<ChainSpec>> = Lazy::new(|| {
ChainSpec {
chain: Chain::from_id(12345),
genesis: serde_json::from_str(include_str!("../testdata/proof-genesis.json"))
.expect("Can't deserialize test genesis json"),
..Default::default()
}
.into()
});
fn convert_to_proof<'a>(path: impl IntoIterator<Item = &'a str>) -> Vec<Bytes> {
path.into_iter().map(Bytes::from_str).collect::<Result<Vec<_>, _>>().unwrap()
}
fn insert_genesis<DB: Database>(
provider_factory: &ProviderFactory<DB>,
chain_spec: Arc<ChainSpec>,
) -> ProviderResult<B256> {
let mut provider = provider_factory.provider_rw()?;
// Hash accounts and insert them into hashing table.
let genesis = chain_spec.genesis();
let alloc_accounts = genesis
.alloc
.iter()
.map(|(addr, account)| (*addr, Some(Account::from_genesis_account(account))));
provider.insert_account_for_hashing(alloc_accounts).unwrap();
let alloc_storage = genesis.alloc.clone().into_iter().filter_map(|(addr, account)| {
// Only return `Some` if there is storage.
account.storage.map(|storage| {
(
addr,
storage
.into_iter()
.map(|(key, value)| StorageEntry { key, value: value.into() }),
)
})
});
provider.insert_storage_for_hashing(alloc_storage)?;
let (root, updates) = StateRoot::from_tx(provider.tx_ref())
.root_with_updates()
.map_err(Into::<reth_db::DatabaseError>::into)?;
updates.write_to_database(provider.tx_mut())?;
provider.commit()?;
Ok(root)
}
#[test]
fn testspec_proofs() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, TEST_SPEC.clone()).unwrap();
let data = Vec::from([
(
"0x2031f89b3ea8014eb51a78c316e42af3e0d7695f",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xf8719f31355ec1c8f7e26bb3ccbcb0b75d870d15846c0b98e5cc452db46c37faea40b84ff84d80890270801d946c940000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
(
"0x33f0fc440b8477fcfbe9d0bf8649e7dea9baedb2",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xe48200d3a0ef957210bca5b9b402d614eb8408c88cfbf4913eb6ab83ca233c8b8f0e626b54",
"0xf851808080a02743a5addaf4cf9b8c0c073e1eaa555deaaf8c41cb2b41958e88624fa45c2d908080808080a0bfbf6937911dfb88113fecdaa6bde822e4e99dae62489fcf61a91cb2f36793d680808080808080",
"0xf8679e207781e762f3577784bab7491fcc43e291ce5a356b9bc517ac52eed3a37ab846f8448001a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
(
"0x62b0dd4aab2b1a0a04e279e2b828791a10755528",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xf8709f3936599f93b769acf90c7178fd2ddcac1b5b4bc9949ee5a04b7e0823c2446eb84ef84c80880f43fc2c04ee0000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
(
"0x1ed9b1dd266b607ee278726d324b855a093394a6",
convert_to_proof([
"0xe48200a7a040f916999be583c572cc4dd369ec53b0a99f7de95f13880cf203d98f935ed1b3",
"0xf87180a04fb9bab4bb88c062f32452b7c94c8f64d07b5851d44a39f1e32ba4b1829fdbfb8080808080a0b61eeb2eb82808b73c4ad14140a2836689f4ab8445d69dd40554eaf1fce34bc080808080808080a0dea230ff2026e65de419288183a340125b04b8405cc61627b3b4137e2260a1e880",
"0xe48200d3a0ef957210bca5b9b402d614eb8408c88cfbf4913eb6ab83ca233c8b8f0e626b54",
"0xf851808080a02743a5addaf4cf9b8c0c073e1eaa555deaaf8c41cb2b41958e88624fa45c2d908080808080a0bfbf6937911dfb88113fecdaa6bde822e4e99dae62489fcf61a91cb2f36793d680808080808080",
"0xf86f9e207a32b8ab5eb4b043c65b1f00c93f517bc8883c5cd31baf8e8a279475e3b84ef84c808801aa535d3d0c0000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
])
),
]);
let provider = factory.provider().unwrap();
for (target, expected_proof) in data {
let target = Address::from_str(target).unwrap();
let account_proof =
Proof::from_tx(provider.tx_ref()).account_proof(target, &[]).unwrap();
similar_asserts::assert_eq!(
account_proof.proof,
expected_proof,
"proof for {target:?} does not match"
);
assert_eq!(account_proof.verify(root), Ok(()));
}
}
#[test]
fn testspec_empty_storage_proof() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, TEST_SPEC.clone()).unwrap();
let target = Address::from_str("0x1ed9b1dd266b607ee278726d324b855a093394a6").unwrap();
let slots = Vec::from([B256::with_last_byte(1), B256::with_last_byte(3)]);
let provider = factory.provider().unwrap();
let account_proof =
Proof::from_tx(provider.tx_ref()).account_proof(target, &slots).unwrap();
assert_eq!(account_proof.storage_root, EMPTY_ROOT_HASH, "expected empty storage root");
assert_eq!(slots.len(), account_proof.storage_proofs.len());
for (idx, slot) in slots.into_iter().enumerate() {
let proof = account_proof.storage_proofs.get(idx).unwrap();
assert_eq!(proof, &StorageProof::new(slot));
assert_eq!(proof.verify(account_proof.storage_root), Ok(()));
}
assert_eq!(account_proof.verify(root), Ok(()));
}
#[test]
fn mainnet_genesis_account_proof() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, MAINNET.clone()).unwrap();
// Address from mainnet genesis allocation.
// keccak256 - `0xcf67b71c90b0d523dd5004cf206f325748da347685071b34812e21801f5270c4`
let target = Address::from_str("0x000d836201318ec6899a67540690382780743280").unwrap();
// `cast proof 0x000d836201318ec6899a67540690382780743280 --block 0`
let expected_account_proof = convert_to_proof([
"0xf90211a090dcaf88c40c7bbc95a912cbdde67c175767b31173df9ee4b0d733bfdd511c43a0babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bda0473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021a0bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0a04e44caecff45c9891f74f6a2156735886eedf6f1a733628ebc802ec79d844648a0a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7a0e823850f50bf72baae9d1733a36a444ab65d0a6faaba404f0583ce0ca4dad92da0f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721a07117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681a069eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472a0203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8ea09287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208a06fc2d754e304c48ce6a517753c62b1a9c1d5925b89707486d7fc08919e0a94eca07b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475a051f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0a089d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb80",
"0xf90211a0dae48f5b47930c28bb116fbd55e52cd47242c71bf55373b55eb2805ee2e4a929a00f1f37f337ec800e2e5974e2e7355f10f1a4832b39b846d916c3597a460e0676a0da8f627bb8fbeead17b318e0a8e4f528db310f591bb6ab2deda4a9f7ca902ab5a0971c662648d58295d0d0aa4b8055588da0037619951217c22052802549d94a2fa0ccc701efe4b3413fd6a61a6c9f40e955af774649a8d9fd212d046a5a39ddbb67a0d607cdb32e2bd635ee7f2f9e07bc94ddbd09b10ec0901b66628e15667aec570ba05b89203dc940e6fa70ec19ad4e01d01849d3a5baa0a8f9c0525256ed490b159fa0b84227d48df68aecc772939a59afa9e1a4ab578f7b698bdb1289e29b6044668ea0fd1c992070b94ace57e48cbf6511a16aa770c645f9f5efba87bbe59d0a042913a0e16a7ccea6748ae90de92f8aef3b3dc248a557b9ac4e296934313f24f7fced5fa042373cf4a00630d94de90d0a23b8f38ced6b0f7cb818b8925fee8f0c2a28a25aa05f89d2161c1741ff428864f7889866484cef622de5023a46e795dfdec336319fa07597a017664526c8c795ce1da27b8b72455c49657113e0455552dbc068c5ba31a0d5be9089012fda2c585a1b961e988ea5efcd3a06988e150a8682091f694b37c5a0f7b0352e38c315b2d9a14d51baea4ddee1770974c806e209355233c3c89dce6ea049bf6e8df0acafd0eff86defeeb305568e44d52d2235cf340ae15c6034e2b24180",
"0xf901f1a0cf67e0f5d5f8d70e53a6278056a14ddca46846f5ef69c7bde6810d058d4a9eda80a06732ada65afd192197fe7ce57792a7f25d26978e64e954b7b84a1f7857ac279da05439f8d011683a6fc07efb90afca198fd7270c795c835c7c85d91402cda992eaa0449b93033b6152d289045fdb0bf3f44926f831566faa0e616b7be1abaad2cb2da031be6c3752bcd7afb99b1bb102baf200f8567c394d464315323a363697646616a0a40e3ed11d906749aa501279392ffde868bd35102db41364d9c601fd651f974aa0044bfa4fe8dd1a58e6c7144da79326e94d1331c0b00373f6ae7f3662f45534b7a098005e3e48db68cb1dc9b9f034ff74d2392028ddf718b0f2084133017da2c2e7a02a62bc40414ee95b02e202a9e89babbabd24bef0abc3fc6dcd3e9144ceb0b725a0239facd895bbf092830390a8676f34b35b29792ae561f196f86614e0448a5792a0a4080f88925daff6b4ce26d188428841bd65655d8e93509f2106020e76d41eefa04918987904be42a6894256ca60203283d1b89139cf21f09f5719c44b8cdbb8f7a06201fc3ef0827e594d953b5e3165520af4fceb719e11cc95fd8d3481519bfd8ca05d0e353d596bd725b09de49c01ede0f29023f0153d7b6d401556aeb525b2959ba0cd367d0679950e9c5f2aa4298fd4b081ade2ea429d71ff390c50f8520e16e30880",
"0xf87180808080808080a0dbee8b33c73b86df839f309f7ac92eee19836e08b39302ffa33921b3c6a09f66a06068b283d51aeeee682b8fb5458354315d0b91737441ede5e137c18b4775174a8080808080a0fe7779c7d58c2fda43eba0a6644043c86ebb9ceb4836f89e30831f23eb059ece8080",
"0xf8719f20b71c90b0d523dd5004cf206f325748da347685071b34812e21801f5270c4b84ff84d80890ad78ebc5ac6200000a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
]);
let provider = factory.provider().unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &[]).unwrap();
similar_asserts::assert_eq!(account_proof.proof, expected_account_proof);
assert_eq!(account_proof.verify(root), Ok(()));
}
#[test]
fn mainnet_genesis_account_proof_nonexistent() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, MAINNET.clone()).unwrap();
// Address that does not exist in mainnet genesis allocation.
// keccak256 - `0x18f415ffd7f66bb1924d90f0e82fb79ca8c6d8a3473cd9a95446a443b9db1761`
let target = Address::from_str("0x000d836201318ec6899a67540690382780743281").unwrap();
// `cast proof 0x000d836201318ec6899a67540690382780743281 --block 0`
let expected_account_proof = convert_to_proof([
"0xf90211a090dcaf88c40c7bbc95a912cbdde67c175767b31173df9ee4b0d733bfdd511c43a0babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bda0473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021a0bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0a04e44caecff45c9891f74f6a2156735886eedf6f1a733628ebc802ec79d844648a0a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7a0e823850f50bf72baae9d1733a36a444ab65d0a6faaba404f0583ce0ca4dad92da0f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721a07117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681a069eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472a0203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8ea09287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208a06fc2d754e304c48ce6a517753c62b1a9c1d5925b89707486d7fc08919e0a94eca07b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475a051f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0a089d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb80",
"0xf90211a0586b1ddec8db4824154209d355a1989b6c43aa69aba36e9d70c9faa53e7452baa0f86db47d628c73764d74b9ccaed73b8486d97a7731d57008fc9efaf417411860a0d9faed7b9ea107b5d98524246c977e782377f976e34f70717e8b1207f2f9b981a00218f59ccedf797c95e27c56405b9bf16845050fb43e773b66b26bc6992744f5a0dbf396f480c4e024156644adea7c331688d03742369e9d87ab8913bc439ff975a0aced524f39b22c62a5be512ddbca89f0b89b47c311065ccf423dee7013c7ea83a0c06b05f80b237b403adc019c0bc95b5de935021b14a75cbc18509eec60dfd83aa085339d45c4a52b7d523c301701f1ab339964e9c907440cff0a871c98dcf8811ea03ae9f6b8e227ec9be9461f0947b01696f78524c4519a6dee9fba14d209952cf9a0af17f551f9fa1ba4be41d0b342b160e2e8468d7e98a65a2dbf9d5fe5d6928024a0b850ac3bc03e9a309cc59ce5f1ab8db264870a7a22786081753d1db91897b8e6a09e796a4904bd78cb2655b5f346c94350e2d5f0dbf2bc00ac00871cd7ba46b241a0f6f0377427b900529caf32abf32ba1eb93f5f70153aa50b90bf55319a434c252a0725eaf27c8ee07e9b2511a6d6a0d71c649d855e8a9ed26e667903e2e94ae47cba0e4139fb48aa1a524d47f6e0df80314b88b52202d7e853da33c276aa8572283a8a05e9003d54a45935fdebae3513dc7cd16626dc05e1d903ae7f47f1a35aa6e234580",
"0xf901d1a0b7c55b381eb205712a2f5d1b7d6309ac725da79ab159cb77dc2783af36e6596da0b3b48aa390e0f3718b486ccc32b01682f92819e652315c1629058cd4d9bb1545a0e3c0cc68af371009f14416c27e17f05f4f696566d2ba45362ce5711d4a01d0e4a0bad1e085e431b510508e2a9e3712633a414b3fe6fd358635ab206021254c1e10a0f8407fe8d5f557b9e012d52e688139bd932fec40d48630d7ff4204d27f8cc68da08c6ca46eff14ad4950e65469c394ca9d6b8690513b1c1a6f91523af00082474c80a0630c034178cb1290d4d906edf28688804d79d5e37a3122c909adab19ac7dc8c5a059f6d047c5d1cc75228c4517a537763cb410c38554f273e5448a53bc3c7166e7a0d842f53ce70c3aad1e616fa6485d3880d15c936fcc306ec14ae35236e5a60549a0218ee2ee673c69b4e1b953194b2568157a69085b86e4f01644fa06ab472c6cf9a016a35a660ea496df7c0da646378bfaa9562f401e42a5c2fe770b7bbe22433585a0dd0fbbe227a4d50868cdbb3107573910fd97131ea8d835bef81d91a2fc30b175a06aafa3d78cf179bf055bd5ec629be0ff8352ce0aec9125a4d75be3ee7eb71f10a01d6817ef9f64fcbb776ff6df0c83138dcd2001bd752727af3e60f4afc123d8d58080"
]);
let provider = factory.provider().unwrap();
let account_proof = Proof::from_tx(provider.tx_ref()).account_proof(target, &[]).unwrap();
similar_asserts::assert_eq!(account_proof.proof, expected_account_proof);
assert_eq!(account_proof.verify(root), Ok(()));
}
#[test]
fn holesky_deposit_contract_proof() {
// Create test database and insert genesis accounts.
let factory = create_test_provider_factory();
let root = insert_genesis(&factory, HOLESKY.clone()).unwrap();
let target = Address::from_str("0x4242424242424242424242424242424242424242").unwrap();
// existent
let slot_22 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000022")
.unwrap();
let slot_23 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000023")
.unwrap();
let slot_24 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000024")
.unwrap();
// non-existent
let slot_100 =
B256::from_str("0x0000000000000000000000000000000000000000000000000000000000000100")
.unwrap();
let slots = Vec::from([slot_22, slot_23, slot_24, slot_100]);
// `cast proof 0x4242424242424242424242424242424242424242 0x22 0x23 0x24 0x100 --block 0`
let expected = AccountProof {
address: target,
info: Some(Account {
balance: U256::ZERO,
nonce: 0,
bytecode_hash: Some(B256::from_str("0x2034f79e0e33b0ae6bef948532021baceb116adf2616478703bec6b17329f1cc").unwrap())
}),
storage_root: B256::from_str("0x556a482068355939c95a3412bdb21213a301483edb1b64402fb66ac9f3583599").unwrap(),
proof: convert_to_proof([
"0xf90211a0ea92fb71507739d5afe328d607b2c5e98322b7aa7cdfeccf817543058b54af70a0bd0c2525b5bee47abf7120c9e01ec3249699d687f80ebb96ed9ad9de913dbab0a0ab4b14b89416eb23c6b64204fa45cfcb39d4220016a9cd0815ebb751fe45eb71a0986ae29c2148b9e61f9a7543f44a1f8d029f1c5095b359652e9ec94e64b5d393a0555d54aa23ed990b0488153418637df7b2c878b604eb761aa2673b609937b0eba0140afb6a3909cc6047b3d44af13fc83f161a7e4c4ddba430a2841862912eb222a031b1185c1f455022d9e42ce04a71f174eb9441b1ada67449510500f4d85b3b22a051ecd01e18113b23cc65e62f67d69b33ee15d20bf81a6b524f7df90ded00ca15a0703769d6a7befad000bc2b4faae3e41b809b1b1241fe2964262554e7e3603488a0e5de7f600e4e6c3c3e5630e0c66f50506a17c9715642fccb63667e81397bbf93a095f783cd1d464a60e3c8adcadc28c6eb9fec7306664df39553be41dccc909606a04225fda3b89f0c59bf40129d1d5e5c3bf67a2129f0c55e53ffdd2cebf185d644a078e0f7fd3ae5a9bc90f66169614211b48fe235eb64818b3935d3e69c53523b9aa0a870e00e53ebaa1e9ec16e5f36606fd7d21d3a3c96894c0a2a23550949d4fdf7a0809226b69cee1f4f22ced1974e7805230da1909036a49a7652428999431afac2a0f11593b2407e86e11997325d8df2d22d937bbe0aef8302ba40c6be0601b04fc380",
"0xf901f1a09da7d9755fe0c558b3c3de9fdcdf9f28ae641f38c9787b05b73ab22ae53af3e2a0d9990bf0b810d1145ecb2b011fd68c63cc85564e6724166fd4a9520180706e5fa05f5f09855df46330aa310e8d6be5fb82d1a4b975782d9b29acf06ac8d3e72b1ca0ca976997ddaf06f18992f6207e4f6a05979d07acead96568058789017cc6d06ba04d78166b48044fdc28ed22d2fd39c8df6f8aaa04cb71d3a17286856f6893ff83a004f8c7cc4f1335182a1709fb28fc67d52e59878480210abcba864d5d1fd4a066a0fc3b71c33e2e6b77c5e494c1db7fdbb447473f003daf378c7a63ba9bf3f0049d80a07b8e7a21c1178d28074f157b50fca85ee25c12568ff8e9706dcbcdacb77bf854a0973274526811393ea0bf4811ca9077531db00d06b86237a2ecd683f55ba4bcb0a03a93d726d7487874e51b52d8d534c63aa2a689df18e3b307c0d6cb0a388b00f3a06aa67101d011d1c22fe739ef83b04b5214a3e2f8e1a2625d8bfdb116b447e86fa02dd545b33c62d33a183e127a08a4767fba891d9f3b94fc20a2ca02600d6d1fffa0f3b039a4f32349e85c782d1164c1890e5bf16badc9ee4cf827db6afd2229dde6a0d9240a9d2d5851d05a97ff3305334dfdb0101e1e321fc279d2bb3cad6afa8fc8a01b69c6ab5173de8a8ec53a6ebba965713a4cc7feb86cb3e230def37c230ca2b280",
"0xf869a0202a47fc6863b89a6b51890ef3c1550d560886c027141d2058ba1e2d4c66d99ab846f8448080a0556a482068355939c95a3412bdb21213a301483edb1b64402fb66ac9f3583599a02034f79e0e33b0ae6bef948532021baceb116adf2616478703bec6b17329f1cc"
]),
storage_proofs: Vec::from([
StorageProof {
key: slot_22,
nibbles: Nibbles::unpack(keccak256(slot_22)),
value: U256::from_str("0xf5a5fd42d16a20302798ef6ed309979b43003d2320d9f0e8ea9831a92759fb4b").unwrap(),
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf85180a0776aa456ba9c5008e03b82b841a9cf2fc1e8578cfacd5c9015804eae315f17fb80808080808080808080808080a072e3e284d47badbb0a5ca1421e1179d3ea90cc10785b26b74fb8a81f0f9e841880",
"0xf843a020035b26e3e9eee00e0d72fd1ee8ddca6894550dca6916ea2ac6baa90d11e510a1a0f5a5fd42d16a20302798ef6ed309979b43003d2320d9f0e8ea9831a92759fb4b"
])
},
StorageProof {
key: slot_23,
nibbles: Nibbles::unpack(keccak256(slot_23)),
value: U256::from_str("0xdb56114e00fdd4c1f85c892bf35ac9a89289aaecb1ebd0a96cde606a748b5d71").unwrap(),
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf8518080808080a0d546c4ca227a267d29796643032422374624ed109b3d94848c5dc06baceaee76808080808080a027c48e210ccc6e01686be2d4a199d35f0e1e8df624a8d3a17c163be8861acd6680808080",
"0xf843a0207b2b5166478fd4318d2acc6cc2c704584312bdd8781b32d5d06abda57f4230a1a0db56114e00fdd4c1f85c892bf35ac9a89289aaecb1ebd0a96cde606a748b5d71"
])
},
StorageProof {
key: slot_24,
nibbles: Nibbles::unpack(keccak256(slot_24)),
value: U256::from_str("0xc78009fdf07fc56a11f122370658a353aaa542ed63e44c4bc15ff4cd105ab33c").unwrap(),
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf85180808080a030263404acfee103d0b1019053ff3240fce433c69b709831673285fa5887ce4c80808080808080a0f8f1fbb1f7b482d9860480feebb83ff54a8b6ec1ead61cc7d2f25d7c01659f9c80808080",
"0xf843a020d332d19b93bcabe3cce7ca0c18a052f57e5fd03b4758a09f30f5ddc4b22ec4a1a0c78009fdf07fc56a11f122370658a353aaa542ed63e44c4bc15ff4cd105ab33c"
])
},
StorageProof {
key: slot_100,
nibbles: Nibbles::unpack(keccak256(slot_100)),
value: U256::ZERO,
proof: convert_to_proof([
"0xf9019180a0aafd5b14a6edacd149e110ba6776a654f2dbffca340902be933d011113f2750380a0a502c93b1918c4c6534d4593ae03a5a23fa10ebc30ffb7080b297bff2446e42da02eb2bf45fd443bd1df8b6f9c09726a4c6252a0f7896a131a081e39a7f644b38980a0a9cf7f673a0bce76fd40332afe8601542910b48dea44e93933a3e5e930da5d19a0ddf79db0a36d0c8134ba143bcb541cd4795a9a2bae8aca0ba24b8d8963c2a77da0b973ec0f48f710bf79f63688485755cbe87f9d4c68326bb83c26af620802a80ea0f0855349af6bf84afc8bca2eda31c8ef8c5139be1929eeb3da4ba6b68a818cb0a0c271e189aeeb1db5d59d7fe87d7d6327bbe7cfa389619016459196497de3ccdea0e7503ba5799e77aa31bbe1310c312ca17b2c5bcc8fa38f266675e8f154c2516ba09278b846696d37213ab9d20a5eb42b03db3173ce490a2ef3b2f3b3600579fc63a0e9041059114f9c910adeca12dbba1fef79b2e2c8899f2d7213cd22dfe4310561a047c59da56bb2bf348c9dd2a2e8f5538a92b904b661cfe54a4298b85868bbe4858080",
"0xf891a090bacef44b189ddffdc5f22edc70fe298c58e5e523e6e1dfdf7dbc6d657f7d1b80a026eed68746028bc369eb456b7d3ee475aa16f34e5eaa0c98fdedb9c59ebc53b0808080a09ce86197173e14e0633db84ce8eea32c5454eebe954779255644b45b717e8841808080a0328c7afb2c58ef3f8c4117a8ebd336f1a61d24591067ed9c5aae94796cac987d808080808080"
])
},
])
};
let provider = factory.provider().unwrap();
let account_proof =
Proof::from_tx(provider.tx_ref()).account_proof(target, &slots).unwrap();
similar_asserts::assert_eq!(account_proof, expected);
assert_eq!(account_proof.verify(root), Ok(()));
}
}

View File

@ -2,8 +2,7 @@ use crate::{
hashed_cursor::HashedPostStateCursorFactory,
prefix_set::{PrefixSetMut, TriePrefixSetsMut},
proof::Proof,
updates::TrieUpdates,
Nibbles, StateRoot,
Nibbles,
};
use itertools::Itertools;
use rayon::prelude::{IntoParallelIterator, ParallelIterator};
@ -195,59 +194,6 @@ impl HashedPostState {
TriePrefixSetsMut { account_prefix_set, storage_prefix_sets, destroyed_accounts }
}
/// Calculate the state root for this [`HashedPostState`].
/// Internally, this method retrieves prefixsets and uses them
/// to calculate incremental state root.
///
/// # Example
///
/// ```
/// use reth_db::test_utils::create_test_rw_db;
/// use reth_db_api::database::Database;
/// use reth_primitives::{Account, U256};
/// use reth_trie::HashedPostState;
///
/// // Initialize the database
/// let db = create_test_rw_db();
///
/// // Initialize hashed post state
/// let mut hashed_state = HashedPostState::default();
/// hashed_state.accounts.insert(
/// [0x11; 32].into(),
/// Some(Account { nonce: 1, balance: U256::from(10), bytecode_hash: None }),
/// );
///
/// // Calculate the state root
/// let tx = db.tx().expect("failed to create transaction");
/// let state_root = hashed_state.state_root(&tx);
/// ```
///
/// # Returns
///
/// The state root for this [`HashedPostState`].
pub fn state_root<TX: DbTx>(&self, tx: &TX) -> Result<B256, StateRootError> {
let sorted = self.clone().into_sorted();
let prefix_sets = self.construct_prefix_sets().freeze();
StateRoot::from_tx(tx)
.with_hashed_cursor_factory(HashedPostStateCursorFactory::new(tx, &sorted))
.with_prefix_sets(prefix_sets)
.root()
}
/// Calculates the state root for this [`HashedPostState`] and returns it alongside trie
/// updates. See [`Self::state_root`] for more info.
pub fn state_root_with_updates<TX: DbTx>(
&self,
tx: &TX,
) -> Result<(B256, TrieUpdates), StateRootError> {
let sorted = self.clone().into_sorted();
let prefix_sets = self.construct_prefix_sets().freeze();
StateRoot::from_tx(tx)
.with_hashed_cursor_factory(HashedPostStateCursorFactory::new(tx, &sorted))
.with_prefix_sets(prefix_sets)
.root_with_updates()
}
/// Generates the state proof for target account and slots on top of this [`HashedPostState`].
pub fn account_proof<TX: DbTx>(
&self,
@ -384,13 +330,6 @@ impl HashedStorageSorted {
#[cfg(test)]
mod tests {
use super::*;
use reth_db::test_utils::create_test_rw_db;
use reth_db_api::database::Database;
use reth_primitives::hex;
use revm::{
db::states::BundleState,
primitives::{AccountInfo, HashMap},
};
#[test]
fn hashed_state_wiped_extension() {
@ -465,34 +404,4 @@ mod tests {
);
assert_eq!(account_storage.map(|st| st.wiped), Some(true));
}
#[test]
fn from_bundle_state_with_rayon() {
let address1 = Address::with_last_byte(1);
let address2 = Address::with_last_byte(2);
let slot1 = U256::from(1015);
let slot2 = U256::from(2015);
let account1 = AccountInfo { nonce: 1, ..Default::default() };
let account2 = AccountInfo { nonce: 2, ..Default::default() };
let bundle_state = BundleState::builder(2..=2)
.state_present_account_info(address1, account1)
.state_present_account_info(address2, account2)
.state_storage(address1, HashMap::from([(slot1, (U256::ZERO, U256::from(10)))]))
.state_storage(address2, HashMap::from([(slot2, (U256::ZERO, U256::from(20)))]))
.build();
assert_eq!(bundle_state.reverts.len(), 1);
let post_state = HashedPostState::from_bundle_state(&bundle_state.state);
assert_eq!(post_state.accounts.len(), 2);
assert_eq!(post_state.storages.len(), 2);
let db = create_test_rw_db();
let tx = db.tx().expect("failed to create transaction");
assert_eq!(
post_state.state_root(&tx).unwrap(),
hex!("b464525710cafcf5d4044ac85b72c08b1e76231b8d91f288fe438cc41d8eaafd")
);
}
}

View File

@ -1,7 +1,7 @@
use crate::{
hashed_cursor::{HashedCursorFactory, HashedStorageCursor},
node_iter::{TrieElement, TrieNodeIter},
prefix_set::{PrefixSet, PrefixSetLoader, TriePrefixSets},
prefix_set::{PrefixSet, TriePrefixSets},
progress::{IntermediateStateRootState, StateRootProgress},
stats::TrieTracker,
trie_cursor::TrieCursorFactory,
@ -12,9 +12,8 @@ use crate::{
use alloy_rlp::{BufMut, Encodable};
use reth_db_api::transaction::DbTx;
use reth_execution_errors::{StateRootError, StorageRootError};
use reth_primitives::{constants::EMPTY_ROOT_HASH, keccak256, Address, BlockNumber, B256};
use std::ops::RangeInclusive;
use tracing::{debug, trace};
use reth_primitives::{constants::EMPTY_ROOT_HASH, keccak256, Address, B256};
use tracing::trace;
#[cfg(feature = "metrics")]
use crate::metrics::{StateRootMetrics, TrieRootMetrics, TrieType};
@ -38,6 +37,23 @@ pub struct StateRoot<T, H> {
}
impl<T, H> StateRoot<T, H> {
/// Creates [`StateRoot`] with `trie_cursor_factory` and `hashed_cursor_factory`. All other
/// parameters are set to reasonable defaults.
///
/// The cursors created by given factories are then used to walk through the accounts and
/// calculate the state root value with.
pub fn new(trie_cursor_factory: T, hashed_cursor_factory: H) -> Self {
Self {
trie_cursor_factory,
hashed_cursor_factory,
prefix_sets: TriePrefixSets::default(),
previous_state: None,
threshold: 100_000,
#[cfg(feature = "metrics")]
metrics: StateRootMetrics::default(),
}
}
/// Set the prefix sets.
pub fn with_prefix_sets(mut self, prefix_sets: TriePrefixSets) -> Self {
self.prefix_sets = prefix_sets;
@ -89,79 +105,6 @@ impl<T, H> StateRoot<T, H> {
}
}
impl<'a, TX: DbTx> StateRoot<&'a TX, &'a TX> {
/// Create a new [`StateRoot`] instance.
pub fn from_tx(tx: &'a TX) -> Self {
Self {
trie_cursor_factory: tx,
hashed_cursor_factory: tx,
prefix_sets: TriePrefixSets::default(),
previous_state: None,
threshold: 100_000,
#[cfg(feature = "metrics")]
metrics: StateRootMetrics::default(),
}
}
/// Given a block number range, identifies all the accounts and storage keys that
/// have changed.
///
/// # Returns
///
/// An instance of state root calculator with account and storage prefixes loaded.
pub fn incremental_root_calculator(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<Self, StateRootError> {
let loaded_prefix_sets = PrefixSetLoader::new(tx).load(range)?;
Ok(Self::from_tx(tx).with_prefix_sets(loaded_prefix_sets))
}
/// Computes the state root of the trie with the changed account and storage prefixes and
/// existing trie nodes.
///
/// # Returns
///
/// The updated state root.
pub fn incremental_root(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<B256, StateRootError> {
debug!(target: "trie::loader", ?range, "incremental state root");
Self::incremental_root_calculator(tx, range)?.root()
}
/// Computes the state root of the trie with the changed account and storage prefixes and
/// existing trie nodes collecting updates in the process.
///
/// Ignores the threshold.
///
/// # Returns
///
/// The updated state root and the trie updates.
pub fn incremental_root_with_updates(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<(B256, TrieUpdates), StateRootError> {
debug!(target: "trie::loader", ?range, "incremental state root");
Self::incremental_root_calculator(tx, range)?.root_with_updates()
}
/// Computes the state root of the trie with the changed account and storage prefixes and
/// existing trie nodes collecting updates in the process.
///
/// # Returns
///
/// The intermediate progress of state root computation.
pub fn incremental_root_with_progress(
tx: &'a TX,
range: RangeInclusive<BlockNumber>,
) -> Result<StateRootProgress, StateRootError> {
debug!(target: "trie::loader", ?range, "incremental state root with progress");
Self::incremental_root_calculator(tx, range)?.root_with_progress()
}
}
impl<T, H> StateRoot<T, H>
where
T: TrieCursorFactory + Clone,
@ -536,780 +479,3 @@ where
Ok((root, storage_slots_walked, trie_updates))
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::{
prefix_set::PrefixSetMut,
test_utils::{state_root, state_root_prehashed, storage_root, storage_root_prehashed},
BranchNodeCompact, TrieMask,
};
use proptest::{prelude::ProptestConfig, proptest};
use proptest_arbitrary_interop::arb;
use reth_db::{tables, test_utils::TempDatabase, DatabaseEnv};
use reth_db_api::{
cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO},
transaction::DbTxMut,
};
use reth_primitives::{hex_literal::hex, Account, StorageEntry, U256};
use reth_provider::{test_utils::create_test_provider_factory, DatabaseProviderRW};
use reth_trie_common::triehash::KeccakHasher;
use std::{
collections::{BTreeMap, HashMap},
ops::Mul,
str::FromStr,
sync::Arc,
};
fn insert_account(
tx: &impl DbTxMut,
address: Address,
account: Account,
storage: &BTreeMap<B256, U256>,
) {
let hashed_address = keccak256(address);
tx.put::<tables::HashedAccounts>(hashed_address, account).unwrap();
insert_storage(tx, hashed_address, storage);
}
fn insert_storage(tx: &impl DbTxMut, hashed_address: B256, storage: &BTreeMap<B256, U256>) {
for (k, v) in storage {
tx.put::<tables::HashedStorages>(
hashed_address,
StorageEntry { key: keccak256(k), value: *v },
)
.unwrap();
}
}
fn incremental_vs_full_root(inputs: &[&str], modified: &str) {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let hashed_address = B256::with_last_byte(1);
let mut hashed_storage_cursor =
tx.tx_ref().cursor_dup_write::<tables::HashedStorages>().unwrap();
let data = inputs.iter().map(|x| B256::from_str(x).unwrap());
let value = U256::from(0);
for key in data {
hashed_storage_cursor.upsert(hashed_address, StorageEntry { key, value }).unwrap();
}
// Generate the intermediate nodes on the receiving end of the channel
let (_, _, trie_updates) =
StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address).root_with_updates().unwrap();
// 1. Some state transition happens, update the hashed storage to the new value
let modified_key = B256::from_str(modified).unwrap();
let value = U256::from(1);
if hashed_storage_cursor.seek_by_key_subkey(hashed_address, modified_key).unwrap().is_some()
{
hashed_storage_cursor.delete_current().unwrap();
}
hashed_storage_cursor
.upsert(hashed_address, StorageEntry { key: modified_key, value })
.unwrap();
// 2. Calculate full merkle root
let loader = StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address);
let modified_root = loader.root().unwrap();
// Update the intermediate roots table so that we can run the incremental verification
trie_updates.write_to_database(tx.tx_ref(), hashed_address).unwrap();
// 3. Calculate the incremental root
let mut storage_changes = PrefixSetMut::default();
storage_changes.insert(Nibbles::unpack(modified_key));
let loader = StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address)
.with_prefix_set(storage_changes.freeze());
let incremental_root = loader.root().unwrap();
assert_eq!(modified_root, incremental_root);
}
#[test]
fn branch_node_child_changes() {
incremental_vs_full_root(
&[
"1000000000000000000000000000000000000000000000000000000000000000",
"1100000000000000000000000000000000000000000000000000000000000000",
"1110000000000000000000000000000000000000000000000000000000000000",
"1200000000000000000000000000000000000000000000000000000000000000",
"1220000000000000000000000000000000000000000000000000000000000000",
"1320000000000000000000000000000000000000000000000000000000000000",
],
"1200000000000000000000000000000000000000000000000000000000000000",
);
}
#[test]
fn arbitrary_storage_root() {
proptest!(ProptestConfig::with_cases(10), |(item in arb::<(Address, std::collections::BTreeMap<B256, U256>)>())| {
let (address, storage) = item;
let hashed_address = keccak256(address);
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
for (key, value) in &storage {
tx.tx_ref().put::<tables::HashedStorages>(
hashed_address,
StorageEntry { key: keccak256(key), value: *value },
)
.unwrap();
}
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let got = StorageRoot::from_tx(tx.tx_ref(), address).root().unwrap();
let expected = storage_root(storage.into_iter());
assert_eq!(expected, got);
});
}
#[test]
// This ensures we dont add empty accounts to the trie
fn test_empty_account() {
let state: State = BTreeMap::from([
(
Address::random(),
(
Account { nonce: 0, balance: U256::from(0), bytecode_hash: None },
BTreeMap::from([(B256::with_last_byte(0x4), U256::from(12))]),
),
),
(
Address::random(),
(
Account { nonce: 0, balance: U256::from(0), bytecode_hash: None },
BTreeMap::default(),
),
),
(
Address::random(),
(
Account {
nonce: 155,
balance: U256::from(414241124u32),
bytecode_hash: Some(keccak256("test")),
},
BTreeMap::from([
(B256::ZERO, U256::from(3)),
(B256::with_last_byte(2), U256::from(1)),
]),
),
),
]);
test_state_root_with_state(state);
}
#[test]
// This ensures we return an empty root when there are no storage entries
fn test_empty_storage_root() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let address = Address::random();
let code = "el buen fla";
let account = Account {
nonce: 155,
balance: U256::from(414241124u32),
bytecode_hash: Some(keccak256(code)),
};
insert_account(tx.tx_ref(), address, account, &Default::default());
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let got = StorageRoot::from_tx(tx.tx_ref(), address).root().unwrap();
assert_eq!(got, EMPTY_ROOT_HASH);
}
#[test]
// This ensures that the walker goes over all the storage slots
fn test_storage_root() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let address = Address::random();
let storage =
BTreeMap::from([(B256::ZERO, U256::from(3)), (B256::with_last_byte(2), U256::from(1))]);
let code = "el buen fla";
let account = Account {
nonce: 155,
balance: U256::from(414241124u32),
bytecode_hash: Some(keccak256(code)),
};
insert_account(tx.tx_ref(), address, account, &storage);
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let got = StorageRoot::from_tx(tx.tx_ref(), address).root().unwrap();
assert_eq!(storage_root(storage.into_iter()), got);
}
type State = BTreeMap<Address, (Account, BTreeMap<B256, U256>)>;
#[test]
fn arbitrary_state_root() {
proptest!(
ProptestConfig::with_cases(10), | (state in arb::<State>()) | {
test_state_root_with_state(state);
}
);
}
#[test]
fn arbitrary_state_root_with_progress() {
proptest!(
ProptestConfig::with_cases(10), | (state in arb::<State>()) | {
let hashed_entries_total = state.len() +
state.values().map(|(_, slots)| slots.len()).sum::<usize>();
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
for (address, (account, storage)) in &state {
insert_account(tx.tx_ref(), *address, *account, storage)
}
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let expected = state_root(state);
let threshold = 10;
let mut got = None;
let mut hashed_entries_walked = 0;
let mut intermediate_state: Option<Box<IntermediateStateRootState>> = None;
while got.is_none() {
let calculator = StateRoot::from_tx(tx.tx_ref())
.with_threshold(threshold)
.with_intermediate_state(intermediate_state.take().map(|state| *state));
match calculator.root_with_progress().unwrap() {
StateRootProgress::Progress(state, walked, _) => {
intermediate_state = Some(state);
hashed_entries_walked += walked;
},
StateRootProgress::Complete(root, walked, _) => {
got = Some(root);
hashed_entries_walked += walked;
},
};
}
assert_eq!(expected, got.unwrap());
assert_eq!(hashed_entries_total, hashed_entries_walked)
}
);
}
fn test_state_root_with_state(state: State) {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
for (address, (account, storage)) in &state {
insert_account(tx.tx_ref(), *address, *account, storage)
}
tx.commit().unwrap();
let expected = state_root(state);
let tx = factory.provider_rw().unwrap();
let got = StateRoot::from_tx(tx.tx_ref()).root().unwrap();
assert_eq!(expected, got);
}
fn encode_account(account: Account, storage_root: Option<B256>) -> Vec<u8> {
let account = TrieAccount::from((account, storage_root.unwrap_or(EMPTY_ROOT_HASH)));
let mut account_rlp = Vec::with_capacity(account.length());
account.encode(&mut account_rlp);
account_rlp
}
#[test]
fn storage_root_regression() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
// Some address whose hash starts with 0xB041
let address3 = Address::from_str("16b07afd1c635f77172e842a000ead9a2a222459").unwrap();
let key3 = keccak256(address3);
assert_eq!(key3[0], 0xB0);
assert_eq!(key3[1], 0x41);
let storage = BTreeMap::from(
[
("1200000000000000000000000000000000000000000000000000000000000000", 0x42),
("1400000000000000000000000000000000000000000000000000000000000000", 0x01),
("3000000000000000000000000000000000000000000000000000000000E00000", 0x127a89),
("3000000000000000000000000000000000000000000000000000000000E00001", 0x05),
]
.map(|(slot, val)| (B256::from_str(slot).unwrap(), U256::from(val))),
);
let mut hashed_storage_cursor =
tx.tx_ref().cursor_dup_write::<tables::HashedStorages>().unwrap();
for (hashed_slot, value) in storage.clone() {
hashed_storage_cursor.upsert(key3, StorageEntry { key: hashed_slot, value }).unwrap();
}
tx.commit().unwrap();
let tx = factory.provider_rw().unwrap();
let account3_storage_root = StorageRoot::from_tx(tx.tx_ref(), address3).root().unwrap();
let expected_root = storage_root_prehashed(storage);
assert_eq!(expected_root, account3_storage_root);
}
#[test]
fn account_and_storage_trie() {
let ether = U256::from(1e18);
let storage = BTreeMap::from(
[
("1200000000000000000000000000000000000000000000000000000000000000", 0x42),
("1400000000000000000000000000000000000000000000000000000000000000", 0x01),
("3000000000000000000000000000000000000000000000000000000000E00000", 0x127a89),
("3000000000000000000000000000000000000000000000000000000000E00001", 0x05),
]
.map(|(slot, val)| (B256::from_str(slot).unwrap(), U256::from(val))),
);
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor =
tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let mut hashed_storage_cursor =
tx.tx_ref().cursor_dup_write::<tables::HashedStorages>().unwrap();
let mut hash_builder = HashBuilder::default();
// Insert first account
let key1 =
B256::from_str("b000000000000000000000000000000000000000000000000000000000000000")
.unwrap();
let account1 = Account { nonce: 0, balance: U256::from(3).mul(ether), bytecode_hash: None };
hashed_account_cursor.upsert(key1, account1).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key1), &encode_account(account1, None));
// Some address whose hash starts with 0xB040
let address2 = Address::from_str("7db3e81b72d2695e19764583f6d219dbee0f35ca").unwrap();
let key2 = keccak256(address2);
assert_eq!(key2[0], 0xB0);
assert_eq!(key2[1], 0x40);
let account2 = Account { nonce: 0, balance: ether, ..Default::default() };
hashed_account_cursor.upsert(key2, account2).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key2), &encode_account(account2, None));
// Some address whose hash starts with 0xB041
let address3 = Address::from_str("16b07afd1c635f77172e842a000ead9a2a222459").unwrap();
let key3 = keccak256(address3);
assert_eq!(key3[0], 0xB0);
assert_eq!(key3[1], 0x41);
let code_hash =
B256::from_str("5be74cad16203c4905c068b012a2e9fb6d19d036c410f16fd177f337541440dd")
.unwrap();
let account3 =
Account { nonce: 0, balance: U256::from(2).mul(ether), bytecode_hash: Some(code_hash) };
hashed_account_cursor.upsert(key3, account3).unwrap();
for (hashed_slot, value) in storage {
if hashed_storage_cursor
.seek_by_key_subkey(key3, hashed_slot)
.unwrap()
.filter(|e| e.key == hashed_slot)
.is_some()
{
hashed_storage_cursor.delete_current().unwrap();
}
hashed_storage_cursor.upsert(key3, StorageEntry { key: hashed_slot, value }).unwrap();
}
let account3_storage_root = StorageRoot::from_tx(tx.tx_ref(), address3).root().unwrap();
hash_builder.add_leaf(
Nibbles::unpack(key3),
&encode_account(account3, Some(account3_storage_root)),
);
let key4a =
B256::from_str("B1A0000000000000000000000000000000000000000000000000000000000000")
.unwrap();
let account4a =
Account { nonce: 0, balance: U256::from(4).mul(ether), ..Default::default() };
hashed_account_cursor.upsert(key4a, account4a).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key4a), &encode_account(account4a, None));
let key5 =
B256::from_str("B310000000000000000000000000000000000000000000000000000000000000")
.unwrap();
let account5 =
Account { nonce: 0, balance: U256::from(8).mul(ether), ..Default::default() };
hashed_account_cursor.upsert(key5, account5).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key5), &encode_account(account5, None));
let key6 =
B256::from_str("B340000000000000000000000000000000000000000000000000000000000000")
.unwrap();
let account6 =
Account { nonce: 0, balance: U256::from(1).mul(ether), ..Default::default() };
hashed_account_cursor.upsert(key6, account6).unwrap();
hash_builder.add_leaf(Nibbles::unpack(key6), &encode_account(account6, None));
// Populate account & storage trie DB tables
let expected_root =
B256::from_str("72861041bc90cd2f93777956f058a545412b56de79af5eb6b8075fe2eabbe015")
.unwrap();
let computed_expected_root: B256 = triehash::trie_root::<KeccakHasher, _, _, _>([
(key1, encode_account(account1, None)),
(key2, encode_account(account2, None)),
(key3, encode_account(account3, Some(account3_storage_root))),
(key4a, encode_account(account4a, None)),
(key5, encode_account(account5, None)),
(key6, encode_account(account6, None)),
]);
// Check computed trie root to ensure correctness
assert_eq!(computed_expected_root, expected_root);
// Check hash builder root
assert_eq!(hash_builder.root(), computed_expected_root);
// Check state root calculation from scratch
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(root, computed_expected_root);
// Check account trie
let account_updates = trie_updates.clone().into_sorted().account_nodes;
assert_eq!(account_updates.len(), 2);
let (nibbles1a, node1a) = account_updates.first().unwrap();
assert_eq!(nibbles1a[..], [0xB]);
assert_eq!(node1a.state_mask, TrieMask::new(0b1011));
assert_eq!(node1a.tree_mask, TrieMask::new(0b0001));
assert_eq!(node1a.hash_mask, TrieMask::new(0b1001));
assert_eq!(node1a.root_hash, None);
assert_eq!(node1a.hashes.len(), 2);
let (nibbles2a, node2a) = account_updates.last().unwrap();
assert_eq!(nibbles2a[..], [0xB, 0x0]);
assert_eq!(node2a.state_mask, TrieMask::new(0b10001));
assert_eq!(node2a.tree_mask, TrieMask::new(0b00000));
assert_eq!(node2a.hash_mask, TrieMask::new(0b10000));
assert_eq!(node2a.root_hash, None);
assert_eq!(node2a.hashes.len(), 1);
// Check storage trie
let mut updated_storage_trie =
trie_updates.storage_tries.iter().filter(|(_, u)| !u.storage_nodes.is_empty());
assert_eq!(updated_storage_trie.clone().count(), 1);
let (_, storage_trie_updates) = updated_storage_trie.next().unwrap();
assert_eq!(storage_trie_updates.storage_nodes.len(), 1);
let (nibbles3, node3) = storage_trie_updates.storage_nodes.iter().next().unwrap();
assert!(nibbles3.is_empty());
assert_eq!(node3.state_mask, TrieMask::new(0b1010));
assert_eq!(node3.tree_mask, TrieMask::new(0b0000));
assert_eq!(node3.hash_mask, TrieMask::new(0b0010));
assert_eq!(node3.hashes.len(), 1);
assert_eq!(node3.root_hash, Some(account3_storage_root));
// Add an account
// Some address whose hash starts with 0xB1
let address4b = Address::from_str("4f61f2d5ebd991b85aa1677db97307caf5215c91").unwrap();
let key4b = keccak256(address4b);
assert_eq!(key4b.0[0], key4a.0[0]);
let account4b =
Account { nonce: 0, balance: U256::from(5).mul(ether), bytecode_hash: None };
hashed_account_cursor.upsert(key4b, account4b).unwrap();
let mut prefix_set = PrefixSetMut::default();
prefix_set.insert(Nibbles::unpack(key4b));
let expected_state_root =
B256::from_str("8e263cd4eefb0c3cbbb14e5541a66a755cad25bcfab1e10dd9d706263e811b28")
.unwrap();
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets {
account_prefix_set: prefix_set.freeze(),
..Default::default()
})
.root_with_updates()
.unwrap();
assert_eq!(root, expected_state_root);
let account_updates = trie_updates.into_sorted().account_nodes;
assert_eq!(account_updates.len(), 2);
let (nibbles1b, node1b) = account_updates.first().unwrap();
assert_eq!(nibbles1b[..], [0xB]);
assert_eq!(node1b.state_mask, TrieMask::new(0b1011));
assert_eq!(node1b.tree_mask, TrieMask::new(0b0001));
assert_eq!(node1b.hash_mask, TrieMask::new(0b1011));
assert_eq!(node1b.root_hash, None);
assert_eq!(node1b.hashes.len(), 3);
assert_eq!(node1a.hashes[0], node1b.hashes[0]);
assert_eq!(node1a.hashes[1], node1b.hashes[2]);
let (nibbles2b, node2b) = account_updates.last().unwrap();
assert_eq!(nibbles2b[..], [0xB, 0x0]);
assert_eq!(node2a, node2b);
tx.commit().unwrap();
{
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor =
tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let account = hashed_account_cursor.seek_exact(key2).unwrap().unwrap();
hashed_account_cursor.delete_current().unwrap();
let mut account_prefix_set = PrefixSetMut::default();
account_prefix_set.insert(Nibbles::unpack(account.0));
let computed_expected_root: B256 = triehash::trie_root::<KeccakHasher, _, _, _>([
(key1, encode_account(account1, None)),
// DELETED: (key2, encode_account(account2, None)),
(key3, encode_account(account3, Some(account3_storage_root))),
(key4a, encode_account(account4a, None)),
(key4b, encode_account(account4b, None)),
(key5, encode_account(account5, None)),
(key6, encode_account(account6, None)),
]);
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets {
account_prefix_set: account_prefix_set.freeze(),
..Default::default()
})
.root_with_updates()
.unwrap();
assert_eq!(root, computed_expected_root);
assert_eq!(trie_updates.account_nodes.len() + trie_updates.removed_nodes.len(), 1);
assert_eq!(trie_updates.account_nodes.len(), 1);
let (nibbles1c, node1c) = trie_updates.account_nodes.iter().next().unwrap();
assert_eq!(nibbles1c[..], [0xB]);
assert_eq!(node1c.state_mask, TrieMask::new(0b1011));
assert_eq!(node1c.tree_mask, TrieMask::new(0b0000));
assert_eq!(node1c.hash_mask, TrieMask::new(0b1011));
assert_eq!(node1c.root_hash, None);
assert_eq!(node1c.hashes.len(), 3);
assert_ne!(node1c.hashes[0], node1b.hashes[0]);
assert_eq!(node1c.hashes[1], node1b.hashes[1]);
assert_eq!(node1c.hashes[2], node1b.hashes[2]);
}
{
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor =
tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let account2 = hashed_account_cursor.seek_exact(key2).unwrap().unwrap();
hashed_account_cursor.delete_current().unwrap();
let account3 = hashed_account_cursor.seek_exact(key3).unwrap().unwrap();
hashed_account_cursor.delete_current().unwrap();
let mut account_prefix_set = PrefixSetMut::default();
account_prefix_set.insert(Nibbles::unpack(account2.0));
account_prefix_set.insert(Nibbles::unpack(account3.0));
let computed_expected_root: B256 = triehash::trie_root::<KeccakHasher, _, _, _>([
(key1, encode_account(account1, None)),
// DELETED: (key2, encode_account(account2, None)),
// DELETED: (key3, encode_account(account3, Some(account3_storage_root))),
(key4a, encode_account(account4a, None)),
(key4b, encode_account(account4b, None)),
(key5, encode_account(account5, None)),
(key6, encode_account(account6, None)),
]);
let (root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets {
account_prefix_set: account_prefix_set.freeze(),
..Default::default()
})
.root_with_updates()
.unwrap();
assert_eq!(root, computed_expected_root);
assert_eq!(trie_updates.account_nodes.len() + trie_updates.removed_nodes.len(), 1);
assert!(!trie_updates
.storage_tries
.iter()
.any(|(_, u)| !u.storage_nodes.is_empty() || !u.removed_nodes.is_empty())); // no storage root update
assert_eq!(trie_updates.account_nodes.len(), 1);
let (nibbles1d, node1d) = trie_updates.account_nodes.iter().next().unwrap();
assert_eq!(nibbles1d[..], [0xB]);
assert_eq!(node1d.state_mask, TrieMask::new(0b1011));
assert_eq!(node1d.tree_mask, TrieMask::new(0b0000));
assert_eq!(node1d.hash_mask, TrieMask::new(0b1010));
assert_eq!(node1d.root_hash, None);
assert_eq!(node1d.hashes.len(), 2);
assert_eq!(node1d.hashes[0], node1b.hashes[1]);
assert_eq!(node1d.hashes[1], node1b.hashes[2]);
}
}
#[test]
fn account_trie_around_extension_node() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let expected = extension_node_trie(&tx);
let (got, updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(expected, got);
assert_trie_updates(&updates.account_nodes);
}
#[test]
fn account_trie_around_extension_node_with_dbtrie() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let expected = extension_node_trie(&tx);
let (got, updates) = StateRoot::from_tx(tx.tx_ref()).root_with_updates().unwrap();
assert_eq!(expected, got);
updates.write_to_database(tx.tx_ref()).unwrap();
// read the account updates from the db
let mut accounts_trie = tx.tx_ref().cursor_read::<tables::AccountsTrie>().unwrap();
let walker = accounts_trie.walk(None).unwrap();
let account_updates = walker
.into_iter()
.map(|item| {
let (key, node) = item.unwrap();
(key.0, node)
})
.collect();
assert_trie_updates(&account_updates);
}
proptest! {
#![proptest_config(ProptestConfig {
cases: 128, ..ProptestConfig::default()
})]
#[test]
fn fuzz_state_root_incremental(account_changes: [BTreeMap<B256, U256>; 5]) {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let mut hashed_account_cursor = tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let mut state = BTreeMap::default();
for accounts in account_changes {
let should_generate_changeset = !state.is_empty();
let mut changes = PrefixSetMut::default();
for (hashed_address, balance) in accounts.clone() {
hashed_account_cursor.upsert(hashed_address, Account { balance, ..Default::default() }).unwrap();
if should_generate_changeset {
changes.insert(Nibbles::unpack(hashed_address));
}
}
let (state_root, trie_updates) = StateRoot::from_tx(tx.tx_ref())
.with_prefix_sets(TriePrefixSets { account_prefix_set: changes.freeze(), ..Default::default() })
.root_with_updates()
.unwrap();
state.append(&mut accounts.clone());
let expected_root = state_root_prehashed(
state.iter().map(|(&key, &balance)| (key, (Account { balance, ..Default::default() }, std::iter::empty())))
);
assert_eq!(expected_root, state_root);
trie_updates.write_to_database(tx.tx_ref()).unwrap();
}
}
}
#[test]
fn storage_trie_around_extension_node() {
let factory = create_test_provider_factory();
let tx = factory.provider_rw().unwrap();
let hashed_address = B256::random();
let (expected_root, expected_updates) = extension_node_storage_trie(&tx, hashed_address);
let (got, _, updates) =
StorageRoot::from_tx_hashed(tx.tx_ref(), hashed_address).root_with_updates().unwrap();
assert_eq!(expected_root, got);
assert_eq!(expected_updates, updates);
assert_trie_updates(&updates.storage_nodes);
}
fn extension_node_storage_trie(
tx: &DatabaseProviderRW<Arc<TempDatabase<DatabaseEnv>>>,
hashed_address: B256,
) -> (B256, StorageTrieUpdates) {
let value = U256::from(1);
let mut hashed_storage = tx.tx_ref().cursor_write::<tables::HashedStorages>().unwrap();
let mut hb = HashBuilder::default().with_updates(true);
for key in [
hex!("30af561000000000000000000000000000000000000000000000000000000000"),
hex!("30af569000000000000000000000000000000000000000000000000000000000"),
hex!("30af650000000000000000000000000000000000000000000000000000000000"),
hex!("30af6f0000000000000000000000000000000000000000000000000000000000"),
hex!("30af8f0000000000000000000000000000000000000000000000000000000000"),
hex!("3100000000000000000000000000000000000000000000000000000000000000"),
] {
hashed_storage
.upsert(hashed_address, StorageEntry { key: B256::new(key), value })
.unwrap();
hb.add_leaf(Nibbles::unpack(key), &alloy_rlp::encode_fixed_size(&value));
}
let root = hb.root();
let (_, updates) = hb.split();
let trie_updates = StorageTrieUpdates { storage_nodes: updates, ..Default::default() };
(root, trie_updates)
}
fn extension_node_trie(tx: &DatabaseProviderRW<Arc<TempDatabase<DatabaseEnv>>>) -> B256 {
let a =
Account { nonce: 0, balance: U256::from(1u64), bytecode_hash: Some(B256::random()) };
let val = encode_account(a, None);
let mut hashed_accounts = tx.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
let mut hb = HashBuilder::default();
for key in [
hex!("30af561000000000000000000000000000000000000000000000000000000000"),
hex!("30af569000000000000000000000000000000000000000000000000000000000"),
hex!("30af650000000000000000000000000000000000000000000000000000000000"),
hex!("30af6f0000000000000000000000000000000000000000000000000000000000"),
hex!("30af8f0000000000000000000000000000000000000000000000000000000000"),
hex!("3100000000000000000000000000000000000000000000000000000000000000"),
] {
hashed_accounts.upsert(B256::new(key), a).unwrap();
hb.add_leaf(Nibbles::unpack(key), &val);
}
hb.root()
}
fn assert_trie_updates(account_updates: &HashMap<Nibbles, BranchNodeCompact>) {
assert_eq!(account_updates.len(), 2);
let node = account_updates.get(&[0x3][..]).unwrap();
let expected = BranchNodeCompact::new(0b0011, 0b0001, 0b0000, vec![], None);
assert_eq!(node, &expected);
let node = account_updates.get(&[0x3, 0x0, 0xA, 0xF][..]).unwrap();
assert_eq!(node.state_mask, TrieMask::new(0b101100000));
assert_eq!(node.tree_mask, TrieMask::new(0b000000000));
assert_eq!(node.hash_mask, TrieMask::new(0b001000000));
assert_eq!(node.root_hash, None);
assert_eq!(node.hashes.len(), 1);
}
}

View File

@ -296,65 +296,3 @@ fn compare_trie_node_entries(
db_item.or(in_memory_item)
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::{
prefix_set::{PrefixSetMut, TriePrefixSets},
test_utils::state_root_prehashed,
StateRoot,
};
use proptest::prelude::*;
use reth_db::{cursor::DbCursorRW, tables, transaction::DbTxMut};
use reth_primitives::{Account, U256};
use reth_provider::test_utils::create_test_provider_factory;
use std::collections::BTreeMap;
proptest! {
#![proptest_config(ProptestConfig {
cases: 128, ..ProptestConfig::default()
})]
#[test]
fn fuzz_in_memory_nodes(mut init_state: BTreeMap<B256, U256>, mut updated_state: BTreeMap<B256, U256>) {
let factory = create_test_provider_factory();
let provider = factory.provider_rw().unwrap();
let mut hashed_account_cursor = provider.tx_ref().cursor_write::<tables::HashedAccounts>().unwrap();
// Insert init state into database
for (hashed_address, balance) in init_state.clone() {
hashed_account_cursor.upsert(hashed_address, Account { balance, ..Default::default() }).unwrap();
}
// Compute initial root and updates
let (_, trie_updates) = StateRoot::from_tx(provider.tx_ref())
.root_with_updates()
.unwrap();
// Insert state updates into database
let mut changes = PrefixSetMut::default();
for (hashed_address, balance) in updated_state.clone() {
hashed_account_cursor.upsert(hashed_address, Account { balance, ..Default::default() }).unwrap();
changes.insert(Nibbles::unpack(hashed_address));
}
// Compute root with in-memory trie nodes overlay
let (state_root, _) = StateRoot::from_tx(provider.tx_ref())
.with_prefix_sets(TriePrefixSets { account_prefix_set: changes.freeze(), ..Default::default() })
.with_trie_cursor_factory(InMemoryTrieCursorFactory::new(provider.tx_ref(), &trie_updates.into_sorted()))
.root_with_updates()
.unwrap();
// Verify the result
let mut state = BTreeMap::default();
state.append(&mut init_state);
state.append(&mut updated_state);
let expected_root = state_root_prehashed(
state.iter().map(|(&key, &balance)| (key, (Account { balance, ..Default::default() }, std::iter::empty())))
);
assert_eq!(expected_root, state_root);
}
}
}

View File

@ -156,6 +156,14 @@ pub struct StorageTrieUpdates {
pub(crate) removed_nodes: HashSet<Nibbles>,
}
#[cfg(feature = "test-utils")]
impl StorageTrieUpdates {
/// Creates a new storage trie updates that are not marked as deleted.
pub fn new(updates: HashMap<Nibbles, BranchNodeCompact>) -> Self {
Self { storage_nodes: updates, ..Default::default() }
}
}
impl StorageTrieUpdates {
/// Returns empty storage trie updates with `deleted` set to `true`.
pub fn deleted() -> Self {