feat(trie): read-only root calculation (#2233)

This commit is contained in:
Roman Krasiuk
2023-04-21 12:36:48 +03:00
committed by GitHub
parent 9452b3658b
commit ec418d924d
25 changed files with 1279 additions and 494 deletions

View File

@@ -98,6 +98,7 @@ impl Command {
tx.clear::<tables::StoragesTrie>()?;
tx.put::<tables::SyncStage>(MERKLE_EXECUTION.0.to_string(), 0)?;
tx.put::<tables::SyncStage>(MERKLE_UNWIND.0.to_string(), 0)?;
tx.delete::<tables::SyncStageProgress>(MERKLE_EXECUTION.0.into(), None)?;
Ok::<_, eyre::Error>(())
})??;
}

View File

@@ -1,18 +1,70 @@
use crate::{Address, H256};
use crate::{
trie::{HashBuilderState, StoredSubNode},
Address, H256,
};
use bytes::Buf;
use reth_codecs::{main_codec, Compact};
/// Saves the progress of MerkleStage
#[main_codec]
#[derive(Default, Debug, Copy, Clone, PartialEq)]
pub struct ProofCheckpoint {
/// The next hashed account to insert into the trie.
pub hashed_address: Option<H256>,
/// The next storage entry to insert into the trie.
pub storage_key: Option<H256>,
/// Current intermediate root for `AccountsTrie`.
pub account_root: Option<H256>,
/// Current intermediate storage root from an account.
pub storage_root: Option<H256>,
/// Saves the progress of Merkle stage.
#[derive(Default, Debug, Clone, PartialEq)]
pub struct MerkleCheckpoint {
// TODO: target block?
/// The last hashed account key processed.
pub last_account_key: H256,
/// The last walker key processed.
pub last_walker_key: Vec<u8>,
/// Previously recorded walker stack.
pub walker_stack: Vec<StoredSubNode>,
/// The hash builder state.
pub state: HashBuilderState,
}
impl Compact for MerkleCheckpoint {
fn to_compact<B>(self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let mut len = 0;
buf.put_slice(self.last_account_key.as_slice());
len += self.last_account_key.len();
buf.put_u16(self.last_walker_key.len() as u16);
buf.put_slice(&self.last_walker_key[..]);
len += 2 + self.last_walker_key.len();
buf.put_u16(self.walker_stack.len() as u16);
len += 2;
for item in self.walker_stack.into_iter() {
len += item.to_compact(buf);
}
len += self.state.to_compact(buf);
len
}
fn from_compact(mut buf: &[u8], _len: usize) -> (Self, &[u8])
where
Self: Sized,
{
let last_account_key = H256::from_slice(&buf[..32]);
buf.advance(32);
let last_walker_key_len = buf.get_u16() as usize;
let last_walker_key = Vec::from(&buf[..last_walker_key_len]);
buf.advance(last_walker_key_len);
let walker_stack_len = buf.get_u16() as usize;
let mut walker_stack = Vec::with_capacity(walker_stack_len);
for _ in 0..walker_stack_len {
let (item, rest) = StoredSubNode::from_compact(buf, 0);
walker_stack.push(item);
buf = rest;
}
let (state, buf) = HashBuilderState::from_compact(buf, 0);
(MerkleCheckpoint { last_account_key, last_walker_key, walker_stack, state }, buf)
}
}
/// Saves the progress of AccountHashing

View File

@@ -48,7 +48,7 @@ pub use chain::{
AllGenesisFormats, Chain, ChainInfo, ChainSpec, ChainSpecBuilder, ForkCondition, GOERLI,
MAINNET, SEPOLIA,
};
pub use checkpoints::{AccountHashingCheckpoint, ProofCheckpoint, StorageHashingCheckpoint};
pub use checkpoints::{AccountHashingCheckpoint, MerkleCheckpoint, StorageHashingCheckpoint};
pub use constants::{
EMPTY_OMMER_ROOT, GOERLI_GENESIS, KECCAK_EMPTY, MAINNET_GENESIS, SEPOLIA_GENESIS,
};

View File

@@ -1,5 +1,6 @@
use super::TrieMask;
use crate::H256;
use bytes::Buf;
use reth_codecs::Compact;
use serde::{Deserialize, Serialize};
@@ -88,7 +89,7 @@ impl Compact for BranchNodeCompact {
buf_size
}
fn from_compact(buf: &[u8], len: usize) -> (Self, &[u8])
fn from_compact(buf: &[u8], _len: usize) -> (Self, &[u8])
where
Self: Sized,
{
@@ -98,9 +99,9 @@ impl Compact for BranchNodeCompact {
assert_eq!(buf.len() % hash_len, 6);
// Consume the masks.
let (state_mask, buf) = TrieMask::from_compact(buf, len);
let (tree_mask, buf) = TrieMask::from_compact(buf, len);
let (hash_mask, buf) = TrieMask::from_compact(buf, len);
let (state_mask, buf) = TrieMask::from_compact(buf, 0);
let (tree_mask, buf) = TrieMask::from_compact(buf, 0);
let (hash_mask, buf) = TrieMask::from_compact(buf, 0);
let mut buf = buf;
let mut num_hashes = buf.len() / hash_len;
@@ -108,18 +109,16 @@ impl Compact for BranchNodeCompact {
// Check if the root hash is present
if hash_mask.count_ones() as usize + 1 == num_hashes {
let (hash, remaining) = H256::from_compact(buf, hash_len);
root_hash = Some(hash);
buf = remaining;
root_hash = Some(H256::from_slice(&buf[..hash_len]));
buf.advance(hash_len);
num_hashes -= 1;
}
// Consume all remaining hashes.
let mut hashes = Vec::<H256>::with_capacity(num_hashes);
for _ in 0..num_hashes {
let (hash, remaining) = H256::from_compact(buf, hash_len);
hashes.push(hash);
buf = remaining;
hashes.push(H256::from_slice(&buf[..hash_len]));
buf.advance(hash_len);
}
(Self::new(state_mask, tree_mask, hash_mask, hashes, root_hash), buf)

View File

@@ -0,0 +1,219 @@
use super::TrieMask;
use crate::H256;
use bytes::Buf;
use reth_codecs::{derive_arbitrary, Compact};
use serde::{Deserialize, Serialize};
/// The hash builder state for storing in the database.
/// Check the `reth-trie` crate for more info on hash builder.
#[derive_arbitrary(compact)]
#[derive(Debug, Clone, PartialEq, Default, Serialize, Deserialize)]
pub struct HashBuilderState {
/// The current key.
pub key: Vec<u8>,
/// The builder stack.
pub stack: Vec<Vec<u8>>,
/// The current node value.
pub value: HashBuilderValue,
/// Group masks.
pub groups: Vec<TrieMask>,
/// Tree masks.
pub tree_masks: Vec<TrieMask>,
/// Hash masks.
pub hash_masks: Vec<TrieMask>,
/// Flag indicating if the current node is stored in the database.
pub stored_in_database: bool,
}
impl Compact for HashBuilderState {
fn to_compact<B>(self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let mut len = 0;
len += self.key.to_compact(buf);
buf.put_u16(self.stack.len() as u16);
len += 2;
for item in self.stack.iter() {
buf.put_u16(item.len() as u16);
buf.put_slice(&item[..]);
len += 2 + item.len();
}
len += self.value.to_compact(buf);
buf.put_u16(self.groups.len() as u16);
len += 2;
for item in self.groups.iter() {
len += item.to_compact(buf);
}
buf.put_u16(self.tree_masks.len() as u16);
len += 2;
for item in self.tree_masks.iter() {
len += item.to_compact(buf);
}
buf.put_u16(self.hash_masks.len() as u16);
len += 2;
for item in self.hash_masks.iter() {
len += item.to_compact(buf);
}
buf.put_u8(self.stored_in_database as u8);
len += 1;
len
}
fn from_compact(buf: &[u8], _len: usize) -> (Self, &[u8])
where
Self: Sized,
{
let (key, mut buf) = Vec::from_compact(buf, 0);
let stack_len = buf.get_u16() as usize;
let mut stack = Vec::with_capacity(stack_len);
for _ in 0..stack_len {
let item_len = buf.get_u16() as usize;
stack.push(Vec::from(&buf[..item_len]));
buf.advance(item_len);
}
let (value, mut buf) = HashBuilderValue::from_compact(buf, 0);
let groups_len = buf.get_u16() as usize;
let mut groups = Vec::with_capacity(groups_len);
for _ in 0..groups_len {
let (item, rest) = TrieMask::from_compact(buf, 0);
groups.push(item);
buf = rest;
}
let tree_masks_len = buf.get_u16() as usize;
let mut tree_masks = Vec::with_capacity(tree_masks_len);
for _ in 0..tree_masks_len {
let (item, rest) = TrieMask::from_compact(buf, 0);
tree_masks.push(item);
buf = rest;
}
let hash_masks_len = buf.get_u16() as usize;
let mut hash_masks = Vec::with_capacity(hash_masks_len);
for _ in 0..hash_masks_len {
let (item, rest) = TrieMask::from_compact(buf, 0);
hash_masks.push(item);
buf = rest;
}
let stored_in_database = buf.get_u8() != 0;
(Self { key, stack, value, groups, tree_masks, hash_masks, stored_in_database }, buf)
}
}
/// The current value of the hash builder.
#[derive_arbitrary(compact)]
#[derive(Clone, PartialEq, Serialize, Deserialize)]
pub enum HashBuilderValue {
/// Value of the leaf node.
Hash(H256),
/// Hash of adjacent nodes.
Bytes(Vec<u8>),
}
impl Compact for HashBuilderValue {
fn to_compact<B>(self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
match self {
Self::Hash(hash) => {
buf.put_u8(0);
1 + hash.to_compact(buf)
}
Self::Bytes(bytes) => {
buf.put_u8(1);
1 + bytes.to_compact(buf)
}
}
}
fn from_compact(buf: &[u8], _len: usize) -> (Self, &[u8])
where
Self: Sized,
{
match buf[0] {
0 => {
let (hash, buf) = H256::from_compact(&buf[1..], 32);
(Self::Hash(hash), buf)
}
1 => {
let (bytes, buf) = Vec::from_compact(&buf[1..], 0);
(Self::Bytes(bytes), buf)
}
_ => panic!("Invalid hash builder value"),
}
}
}
impl std::fmt::Debug for HashBuilderValue {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::Bytes(bytes) => write!(f, "Bytes({:?})", hex::encode(bytes)),
Self::Hash(hash) => write!(f, "Hash({:?})", hash),
}
}
}
impl From<Vec<u8>> for HashBuilderValue {
fn from(value: Vec<u8>) -> Self {
Self::Bytes(value)
}
}
impl From<&[u8]> for HashBuilderValue {
fn from(value: &[u8]) -> Self {
Self::Bytes(value.to_vec())
}
}
impl From<H256> for HashBuilderValue {
fn from(value: H256) -> Self {
Self::Hash(value)
}
}
impl Default for HashBuilderValue {
fn default() -> Self {
Self::Bytes(vec![])
}
}
#[cfg(test)]
mod tests {
use super::*;
use proptest::prelude::*;
#[test]
fn hash_builder_state_regression() {
let mut state = HashBuilderState::default();
state.stack.push(vec![]);
let mut buf = vec![];
let len = state.clone().to_compact(&mut buf);
let (decoded, _) = HashBuilderState::from_compact(&buf, len);
assert_eq!(state, decoded);
}
proptest! {
#[test]
fn hash_builder_state_roundtrip(state: HashBuilderState) {
let mut buf = vec![];
let len = state.clone().to_compact(&mut buf);
let (decoded, _) = HashBuilderState::from_compact(&buf, len);
assert_eq!(state, decoded);
}
}
}

View File

@@ -1,5 +1,6 @@
use bytes::Buf;
use derive_more::{BitAnd, BitAndAssign, BitOr, BitOrAssign, Deref, From, Not};
use reth_codecs::Compact;
use reth_codecs::{derive_arbitrary, Compact};
use serde::{Deserialize, Serialize};
/// A struct representing a mask of 16 bits, used for Ethereum trie operations.
@@ -26,6 +27,7 @@ use serde::{Deserialize, Serialize};
BitOrAssign,
Not,
)]
#[derive_arbitrary(compact)]
pub struct TrieMask(u16);
impl TrieMask {
@@ -66,14 +68,15 @@ impl Compact for TrieMask {
where
B: bytes::BufMut + AsMut<[u8]>,
{
buf.put_slice(self.to_be_bytes().as_slice());
buf.put_u16(self.0);
2
}
fn from_compact(buf: &[u8], _len: usize) -> (Self, &[u8])
fn from_compact(mut buf: &[u8], _len: usize) -> (Self, &[u8])
where
Self: Sized,
{
(Self(u16::from_be_bytes(buf[..2].try_into().unwrap())), &buf[2..])
let mask = buf.get_u16();
(Self(mask), buf)
}
}

View File

@@ -1,13 +1,17 @@
//! Collection of trie related types.
mod branch_node;
mod hash_builder;
mod mask;
mod nibbles;
mod storage;
mod subnode;
pub use self::{
branch_node::BranchNodeCompact,
hash_builder::{HashBuilderState, HashBuilderValue},
mask::TrieMask,
nibbles::{StoredNibbles, StoredNibblesSubKey},
storage::StorageTrieEntry,
subnode::StoredSubNode,
};

View File

@@ -0,0 +1,98 @@
use super::BranchNodeCompact;
use bytes::Buf;
use reth_codecs::Compact;
/// Walker sub node for storing intermediate state root calculation state in the database.
/// See [crate::MerkleCheckpoint].
#[derive(Debug, Clone, PartialEq, Default)]
pub struct StoredSubNode {
/// The key of the current node.
pub key: Vec<u8>,
/// The index of the next child to visit.
pub nibble: Option<u8>,
/// The node itself.
pub node: Option<BranchNodeCompact>,
}
impl Compact for StoredSubNode {
fn to_compact<B>(self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let mut len = 0;
buf.put_u16(self.key.len() as u16);
buf.put_slice(&self.key[..]);
len += 2 + self.key.len();
if let Some(nibble) = self.nibble {
buf.put_u8(1);
buf.put_u8(nibble);
len += 2;
} else {
buf.put_u8(0);
len += 1;
}
if let Some(node) = self.node {
buf.put_u8(1);
len += 1;
len += node.to_compact(buf);
} else {
len += 1;
buf.put_u8(0);
}
len
}
fn from_compact(mut buf: &[u8], _len: usize) -> (Self, &[u8])
where
Self: Sized,
{
let key_len = buf.get_u16() as usize;
let key = Vec::from(&buf[..key_len]);
buf.advance(key_len);
let nibbles_exists = buf.get_u8() != 0;
let nibble = if nibbles_exists { Some(buf.get_u8()) } else { None };
let node_exsists = buf.get_u8() != 0;
let node = if node_exsists {
let (node, rest) = BranchNodeCompact::from_compact(buf, 0);
buf = rest;
Some(node)
} else {
None
};
(StoredSubNode { key, nibble, node }, buf)
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::{trie::TrieMask, H256};
#[test]
fn subnode_roundtrip() {
let subnode = StoredSubNode {
key: vec![],
nibble: None,
node: Some(BranchNodeCompact {
state_mask: TrieMask::new(1),
tree_mask: TrieMask::new(0),
hash_mask: TrieMask::new(1),
hashes: vec![H256::zero()],
root_hash: None,
}),
};
let mut encoded = vec![];
subnode.clone().to_compact(&mut encoded);
let (decoded, _) = StoredSubNode::from_compact(&encoded[..], 0);
assert_eq!(subnode, decoded);
}
}

View File

@@ -121,7 +121,7 @@ pub(crate) fn txs_testdata(num_blocks: u64) -> PathBuf {
tx.insert_accounts_and_storages(start_state.clone()).unwrap();
// make first block after genesis have valid state root
let root = StateRoot::new(tx.inner().deref()).root(None).unwrap();
let (root, updates) = StateRoot::new(tx.inner().deref()).root_with_updates().unwrap();
let second_block = blocks.get_mut(1).unwrap();
let cloned_second = second_block.clone();
let mut updated_header = cloned_second.header.unseal();
@@ -131,6 +131,7 @@ pub(crate) fn txs_testdata(num_blocks: u64) -> PathBuf {
let offset = transitions.len() as u64;
tx.insert_transitions(transitions, None).unwrap();
tx.commit(|tx| updates.flush(tx)).unwrap();
let (transitions, final_state) =
random_transition_range(blocks.iter().skip(2), start_state, n_changes, key_range);
@@ -142,7 +143,7 @@ pub(crate) fn txs_testdata(num_blocks: u64) -> PathBuf {
// make last block have valid state root
let root = {
let mut tx_mut = tx.inner();
let root = StateRoot::new(tx.inner().deref()).root(None).unwrap();
let root = StateRoot::new(tx.inner().deref()).root().unwrap();
tx_mut.commit().unwrap();
root
};

View File

@@ -1,9 +1,14 @@
use crate::{ExecInput, ExecOutput, Stage, StageError, StageId, UnwindInput, UnwindOutput};
use reth_db::{database::Database, tables, transaction::DbTxMut};
use reth_codecs::Compact;
use reth_db::{
database::Database,
tables,
transaction::{DbTx, DbTxMut},
};
use reth_interfaces::consensus;
use reth_primitives::{BlockNumber, H256};
use reth_primitives::{hex, BlockNumber, MerkleCheckpoint, H256};
use reth_provider::Transaction;
use reth_trie::StateRoot;
use reth_trie::{IntermediateStateRootState, StateRoot, StateRootProgress};
use std::{fmt::Debug, ops::DerefMut};
use tracing::*;
@@ -82,6 +87,42 @@ impl MerkleStage {
})
}
}
/// Gets the hashing progress
pub fn get_execution_checkpoint<DB: Database>(
&self,
tx: &Transaction<'_, DB>,
) -> Result<Option<MerkleCheckpoint>, StageError> {
let buf =
tx.get::<tables::SyncStageProgress>(MERKLE_EXECUTION.0.into())?.unwrap_or_default();
if buf.is_empty() {
return Ok(None)
}
let (checkpoint, _) = MerkleCheckpoint::from_compact(&buf, buf.len());
Ok(Some(checkpoint))
}
/// Saves the hashing progress
pub fn save_execution_checkpoint<DB: Database>(
&mut self,
tx: &Transaction<'_, DB>,
checkpoint: Option<MerkleCheckpoint>,
) -> Result<(), StageError> {
let mut buf = vec![];
if let Some(checkpoint) = checkpoint {
debug!(
target: "sync::stages::merkle::exec",
last_account_key = ?checkpoint.last_account_key,
last_walker_key = ?hex::encode(&checkpoint.last_walker_key),
"Saving inner merkle checkpoint"
);
checkpoint.to_compact(&mut buf);
}
tx.put::<tables::SyncStageProgress>(MERKLE_EXECUTION.0.into(), buf)?;
Ok(())
}
}
#[async_trait::async_trait]
@@ -121,21 +162,58 @@ impl<DB: Database> Stage<DB> for MerkleStage {
let block_root = tx.get_header(current_blook)?.state_root;
let checkpoint = self.get_execution_checkpoint(tx)?;
let trie_root = if range.is_empty() {
block_root
} else if to_block - from_block > threshold || from_block == 1 {
// if there are more blocks than threshold it is faster to rebuild the trie
debug!(target: "sync::stages::merkle::exec", current = ?current_blook, target = ?to_block, "Rebuilding trie");
tx.clear::<tables::AccountsTrie>()?;
tx.clear::<tables::StoragesTrie>()?;
StateRoot::new(tx.deref_mut()).root(None).map_err(|e| StageError::Fatal(Box::new(e)))?
if let Some(checkpoint) = &checkpoint {
debug!(
target: "sync::stages::merkle::exec",
current = ?current_blook,
target = ?to_block,
last_account_key = ?checkpoint.last_account_key,
last_walker_key = ?hex::encode(&checkpoint.last_walker_key),
"Continuing inner merkle checkpoint"
);
} else {
debug!(
target: "sync::stages::merkle::exec",
current = ?current_blook,
target = ?to_block,
"Rebuilding trie"
);
tx.clear::<tables::AccountsTrie>()?;
tx.clear::<tables::StoragesTrie>()?;
}
let progress = StateRoot::new(tx.deref_mut())
.with_intermediate_state(checkpoint.map(IntermediateStateRootState::from))
.root_with_progress()
.map_err(|e| StageError::Fatal(Box::new(e)))?;
match progress {
StateRootProgress::Progress(state, updates) => {
updates.flush(tx.deref_mut())?;
self.save_execution_checkpoint(tx, Some(state.into()))?;
return Ok(ExecOutput { stage_progress: input.stage_progress(), done: false })
}
StateRootProgress::Complete(root, updates) => {
updates.flush(tx.deref_mut())?;
root
}
}
} else {
debug!(target: "sync::stages::merkle::exec", current = ?current_blook, target =
?to_block, "Updating trie"); // Iterate over
StateRoot::incremental_root(tx.deref_mut(), range, None)
.map_err(|e| StageError::Fatal(Box::new(e)))?
debug!(target: "sync::stages::merkle::exec", current = ?current_blook, target = ?to_block, "Updating trie");
let (root, updates) = StateRoot::incremental_root_with_updates(tx.deref_mut(), range)
.map_err(|e| StageError::Fatal(Box::new(e)))?;
updates.flush(tx.deref_mut())?;
root
};
// Reset the checkpoint
self.save_execution_checkpoint(tx, None)?;
self.validate_state_root(trie_root, block_root, to_block)?;
info!(target: "sync::stages::merkle::exec", "Stage finished");
@@ -162,10 +240,16 @@ impl<DB: Database> Stage<DB> for MerkleStage {
// Unwind trie only if there are transitions
if !range.is_empty() {
let block_root = StateRoot::incremental_root(tx.deref_mut(), range, None)
.map_err(|e| StageError::Fatal(Box::new(e)))?;
let (block_root, updates) =
StateRoot::incremental_root_with_updates(tx.deref_mut(), range)
.map_err(|e| StageError::Fatal(Box::new(e)))?;
// Validate the calulated state root
let target_root = tx.get_header(input.unwind_to)?.state_root;
self.validate_state_root(block_root, target_root, input.unwind_to)?;
// Validation passed, apply unwind changes to the database.
updates.flush(tx.deref_mut())?;
} else {
info!(target: "sync::stages::merkle::unwind", "Nothing to unwind");
}

View File

@@ -78,7 +78,7 @@ macro_rules! impl_uint_compact {
};
}
impl_uint_compact!(u64, u128);
impl_uint_compact!(u8, u64, u128);
impl<T> Compact for Vec<T>
where

View File

@@ -45,8 +45,7 @@ impl_compression_for_compact!(
StoredBlockBodyIndices,
StoredBlockOmmers,
StoredBlockWithdrawals,
Bytecode,
ProofCheckpoint
Bytecode
);
impl_compression_for_compact!(AccountBeforeTx, TransactionSignedNoHash);
impl_compression_for_compact!(CompactU256);

View File

@@ -581,7 +581,8 @@ where
// merkle tree
{
let state_root = StateRoot::incremental_root(self.deref_mut(), range.clone(), None)?;
let (state_root, trie_updates) =
StateRoot::incremental_root_with_updates(self.deref_mut(), range.clone())?;
if state_root != expected_state_root {
return Err(TransactionError::StateTrieRootMismatch {
got: state_root,
@@ -590,6 +591,7 @@ where
block_hash: end_block_hash,
})
}
trie_updates.flush(self.deref_mut())?;
}
Ok(())
}
@@ -983,7 +985,8 @@ where
self.unwind_storage_history_indices(storage_range)?;
// merkle tree
let new_state_root = StateRoot::incremental_root(self.deref(), range.clone(), None)?;
let (new_state_root, trie_updates) =
StateRoot::incremental_root_with_updates(self.deref(), range.clone())?;
let parent_number = range.start().saturating_sub(1);
let parent_state_root = self.get_header(parent_number)?.state_root;
@@ -999,6 +1002,7 @@ where
block_hash: parent_hash,
})
}
trie_updates.flush(self.deref())?;
}
// get blocks
let blocks = self.get_take_block_range::<TAKE>(chain_spec, range.clone())?;

View File

@@ -1,8 +1,6 @@
use super::TrieCursor;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW},
tables, Error,
};
use crate::updates::TrieKey;
use reth_db::{cursor::DbCursorRO, tables, Error};
use reth_primitives::trie::{BranchNodeCompact, StoredNibbles};
/// A cursor over the account trie.
@@ -17,7 +15,7 @@ impl<C> AccountTrieCursor<C> {
impl<'a, C> TrieCursor<StoredNibbles> for AccountTrieCursor<C>
where
C: DbCursorRO<'a, tables::AccountsTrie> + DbCursorRW<'a, tables::AccountsTrie>,
C: DbCursorRO<'a, tables::AccountsTrie>,
{
fn seek_exact(
&mut self,
@@ -30,12 +28,8 @@ where
Ok(self.0.seek(key)?.map(|value| (value.0.inner.to_vec(), value.1)))
}
fn upsert(&mut self, key: StoredNibbles, value: BranchNodeCompact) -> Result<(), Error> {
self.0.upsert(key, value)
}
fn delete_current(&mut self) -> Result<(), Error> {
self.0.delete_current()
fn current(&mut self) -> Result<Option<TrieKey>, Error> {
Ok(self.0.current()?.map(|(k, _)| TrieKey::AccountNode(k)))
}
}

View File

@@ -1,10 +1,11 @@
use super::TrieCursor;
use crate::updates::TrieKey;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO, DbDupCursorRW},
cursor::{DbCursorRO, DbDupCursorRO},
tables, Error,
};
use reth_primitives::{
trie::{BranchNodeCompact, StorageTrieEntry, StoredNibblesSubKey},
trie::{BranchNodeCompact, StoredNibblesSubKey},
H256,
};
@@ -24,10 +25,7 @@ impl<C> StorageTrieCursor<C> {
impl<'a, C> TrieCursor<StoredNibblesSubKey> for StorageTrieCursor<C>
where
C: DbDupCursorRO<'a, tables::StoragesTrie>
+ DbDupCursorRW<'a, tables::StoragesTrie>
+ DbCursorRO<'a, tables::StoragesTrie>
+ DbCursorRW<'a, tables::StoragesTrie>,
C: DbDupCursorRO<'a, tables::StoragesTrie> + DbCursorRO<'a, tables::StoragesTrie>,
{
fn seek_exact(
&mut self,
@@ -50,28 +48,18 @@ where
.map(|value| (value.nibbles.inner.to_vec(), value.node)))
}
fn upsert(&mut self, key: StoredNibblesSubKey, value: BranchNodeCompact) -> Result<(), Error> {
if let Some(entry) = self.cursor.seek_by_key_subkey(self.hashed_address, key.clone())? {
// "seek exact"
if entry.nibbles == key {
self.cursor.delete_current()?;
}
}
self.cursor.upsert(self.hashed_address, StorageTrieEntry { nibbles: key, node: value })?;
Ok(())
}
fn delete_current(&mut self) -> Result<(), Error> {
self.cursor.delete_current()
fn current(&mut self) -> Result<Option<TrieKey>, Error> {
Ok(self.cursor.current()?.map(|(k, v)| TrieKey::StorageNode(k, v.nibbles)))
}
}
#[cfg(test)]
mod tests {
use super::*;
use reth_db::{mdbx::test_utils::create_test_rw_db, tables, transaction::DbTxMut};
use reth_primitives::trie::BranchNodeCompact;
use reth_db::{
cursor::DbCursorRW, mdbx::test_utils::create_test_rw_db, tables, transaction::DbTxMut,
};
use reth_primitives::trie::{BranchNodeCompact, StorageTrieEntry};
use reth_provider::Transaction;
// tests that upsert and seek match on the storagetrie cursor
@@ -79,14 +67,20 @@ mod tests {
fn test_storage_cursor_abstraction() {
let db = create_test_rw_db();
let tx = Transaction::new(db.as_ref()).unwrap();
let cursor = tx.cursor_dup_write::<tables::StoragesTrie>().unwrap();
let mut cursor = StorageTrieCursor::new(cursor, H256::random());
let mut cursor = tx.cursor_dup_write::<tables::StoragesTrie>().unwrap();
let hashed_address = H256::random();
let key = vec![0x2, 0x3];
let value = BranchNodeCompact::new(1, 1, 1, vec![H256::random()], None);
cursor.upsert(key.clone().into(), value.clone()).unwrap();
assert_eq!(cursor.seek(key.into()).unwrap().unwrap().1, value);
cursor
.upsert(
hashed_address,
StorageTrieEntry { nibbles: key.clone().into(), node: value.clone() },
)
.unwrap();
let mut cursor = StorageTrieCursor::new(cursor, hashed_address);
assert_eq!(cursor.seek(key.clone().into()).unwrap().unwrap().1, value);
}
}

View File

@@ -1,5 +1,8 @@
use crate::{nodes::CHILD_INDEX_RANGE, Nibbles};
use reth_primitives::{trie::BranchNodeCompact, H256};
use reth_primitives::{
trie::{BranchNodeCompact, StoredSubNode},
H256,
};
/// Cursor for iterating over a subtrie.
#[derive(Clone)]
@@ -31,6 +34,23 @@ impl std::fmt::Debug for CursorSubNode {
}
}
impl From<StoredSubNode> for CursorSubNode {
fn from(value: StoredSubNode) -> Self {
let nibble = match value.nibble {
Some(n) => n as i8,
None => -1,
};
Self { key: Nibbles::from(value.key), nibble, node: value.node }
}
}
impl From<CursorSubNode> for StoredSubNode {
fn from(value: CursorSubNode) -> Self {
let nibble = if value.nibble >= 0 { Some(value.nibble as u8) } else { None };
Self { key: value.key.hex_data, nibble, node: value.node }
}
}
impl CursorSubNode {
/// Creates a new `CursorSubNode` from a key and an optional node.
pub fn new(key: Nibbles, node: Option<BranchNodeCompact>) -> Self {

View File

@@ -1,3 +1,4 @@
use crate::updates::TrieKey;
use reth_db::{table::Key, Error};
use reth_primitives::trie::BranchNodeCompact;
@@ -9,9 +10,6 @@ pub trait TrieCursor<K: Key> {
/// Move the cursor to the key and return a value matching of greater than the key.
fn seek(&mut self, key: K) -> Result<Option<(Vec<u8>, BranchNodeCompact)>, Error>;
/// Upsert the key/value pair.
fn upsert(&mut self, key: K, value: BranchNodeCompact) -> Result<(), Error>;
/// Delete the key/value pair at the current cursor position.
fn delete_current(&mut self) -> Result<(), Error>;
/// Get the current entry.
fn current(&mut self) -> Result<Option<TrieKey>, Error>;
}

View File

@@ -5,13 +5,10 @@ use crate::{
use reth_primitives::{
keccak256,
proofs::EMPTY_ROOT,
trie::{BranchNodeCompact, TrieMask},
trie::{BranchNodeCompact, HashBuilderState, HashBuilderValue, TrieMask},
H256,
};
use std::{fmt::Debug, sync::mpsc};
mod value;
use value::HashBuilderValue;
use std::{collections::BTreeMap, fmt::Debug, sync::mpsc};
/// A type alias for a sender of branch nodes.
/// Branch nodes are sent by the Hash Builder to be stored in the database.
@@ -40,7 +37,7 @@ pub type BranchNodeSender = mpsc::Sender<(Nibbles, BranchNodeCompact)>;
/// up, combining the hashes of child nodes and ultimately generating the root hash. The root hash
/// can then be used to verify the integrity and authenticity of the trie's data by constructing and
/// verifying Merkle proofs.
#[derive(Clone, Debug, Default)]
#[derive(Debug, Default)]
pub struct HashBuilder {
key: Nibbles,
stack: Vec<Vec<u8>>,
@@ -52,19 +49,66 @@ pub struct HashBuilder {
stored_in_database: bool,
branch_node_sender: Option<BranchNodeSender>,
updated_branch_nodes: Option<BTreeMap<Nibbles, BranchNodeCompact>>,
}
impl From<HashBuilderState> for HashBuilder {
fn from(state: HashBuilderState) -> Self {
Self {
key: Nibbles::from(state.key),
stack: state.stack,
value: state.value,
groups: state.groups,
tree_masks: state.tree_masks,
hash_masks: state.hash_masks,
stored_in_database: state.stored_in_database,
updated_branch_nodes: None,
}
}
}
impl From<HashBuilder> for HashBuilderState {
fn from(state: HashBuilder) -> Self {
Self {
key: state.key.hex_data,
stack: state.stack,
value: state.value,
groups: state.groups,
tree_masks: state.tree_masks,
hash_masks: state.hash_masks,
stored_in_database: state.stored_in_database,
}
}
}
impl HashBuilder {
/// Creates a new instance of the Hash Builder.
pub fn new(store_tx: Option<BranchNodeSender>) -> Self {
Self { branch_node_sender: store_tx, ..Default::default() }
/// Enables the Hash Builder to store updated branch nodes.
///
/// Call [HashBuilder::split] to get the updates to branch nodes.
pub fn with_updates(mut self, retain_updates: bool) -> Self {
self.set_updates(retain_updates);
self
}
/// Set a branch node sender on the Hash Builder instance.
pub fn with_branch_node_sender(mut self, tx: BranchNodeSender) -> Self {
self.branch_node_sender = Some(tx);
self
/// Enables the Hash Builder to store updated branch nodes.
///
/// Call [HashBuilder::split] to get the updates to branch nodes.
pub fn set_updates(&mut self, retain_updates: bool) {
if retain_updates {
self.updated_branch_nodes = Some(BTreeMap::default());
}
}
/// Splits the [HashBuilder] into a [HashBuilder] and hash builder updates.
pub fn split(mut self) -> (Self, BTreeMap<Nibbles, BranchNodeCompact>) {
let updates = self.updated_branch_nodes.take();
(self, updates.unwrap_or_default())
}
/// The number of total updates accrued.
/// Returns `0` if [Self::with_updates] was not called.
pub fn updates_len(&self) -> usize {
self.updated_branch_nodes.as_ref().map(|u| u.len()).unwrap_or(0)
}
/// Print the current stack of the Hash Builder.
@@ -326,8 +370,8 @@ impl HashBuilder {
// other side of the HashBuilder
tracing::debug!(target: "trie::hash_builder", node = ?n, "intermediate node");
let common_prefix = current.slice(0, len);
if let Some(tx) = &self.branch_node_sender {
let _ = tx.send((common_prefix, n));
if let Some(nodes) = self.updated_branch_nodes.as_mut() {
nodes.insert(common_prefix, n);
}
}
}
@@ -429,8 +473,7 @@ mod tests {
#[test]
fn test_generates_branch_node() {
let (sender, recv) = mpsc::channel();
let mut hb = HashBuilder::new(Some(sender));
let mut hb = HashBuilder::default().with_updates(true);
// We have 1 branch node update to be stored at 0x01, indicated by the first nibble.
// That branch root node has 2 branch node children present at 0x1 and 0x2.
@@ -477,11 +520,9 @@ mod tests {
hb.add_leaf(nibbles, val.as_ref());
});
let root = hb.root();
drop(hb);
let updates = recv.iter().collect::<Vec<_>>();
let (_, updates) = hb.split();
let updates = updates.iter().cloned().collect::<BTreeMap<_, _>>();
let update = updates.get(&Nibbles::from(hex!("01").as_slice())).unwrap();
assert_eq!(update.state_mask, TrieMask::new(0b1111)); // 1st nibble: 0, 1, 2, 3
assert_eq!(update.tree_mask, TrieMask::new(0));

View File

@@ -1,40 +0,0 @@
use reth_primitives::H256;
#[derive(Clone)]
pub(crate) enum HashBuilderValue {
Bytes(Vec<u8>),
Hash(H256),
}
impl std::fmt::Debug for HashBuilderValue {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::Bytes(bytes) => write!(f, "Bytes({:?})", hex::encode(bytes)),
Self::Hash(hash) => write!(f, "Hash({:?})", hash),
}
}
}
impl From<Vec<u8>> for HashBuilderValue {
fn from(value: Vec<u8>) -> Self {
Self::Bytes(value)
}
}
impl From<&[u8]> for HashBuilderValue {
fn from(value: &[u8]) -> Self {
Self::Bytes(value.to_vec())
}
}
impl From<H256> for HashBuilderValue {
fn from(value: H256) -> Self {
Self::Hash(value)
}
}
impl Default for HashBuilderValue {
fn default() -> Self {
Self::Bytes(vec![])
}
}

View File

@@ -36,7 +36,14 @@ pub use errors::{StateRootError, StorageRootError};
/// The implementation of the Merkle Patricia Trie.
mod trie;
pub use trie::{BranchNodeUpdate, BranchNodeUpdateSender, StateRoot, StorageRoot};
pub use trie::{StateRoot, StorageRoot};
/// Buffer for trie updates.
pub mod updates;
/// Utilities for state root checkpoint progress.
mod progress;
pub use progress::{IntermediateStateRootState, StateRootProgress};
/// Collection of trie-related test utilities.
#[cfg(any(test, feature = "test-utils"))]

View File

@@ -11,7 +11,7 @@ use reth_db::{
use reth_primitives::{keccak256, BlockNumber, StorageEntry, H256};
use std::{collections::HashMap, ops::RangeInclusive};
/// A wrapper around a database transaction that loads prefix sets within a given transition range.
/// A wrapper around a database transaction that loads prefix sets within a given block range.
#[derive(Deref)]
pub struct PrefixSetLoader<'a, TX>(&'a TX);
@@ -26,7 +26,7 @@ impl<'a, 'b, TX> PrefixSetLoader<'a, TX>
where
TX: DbTx<'b>,
{
/// Load all account and storage changes for the given transition id range.
/// Load all account and storage changes for the given block range.
pub fn load(
self,
range: RangeInclusive<BlockNumber>,

View File

@@ -0,0 +1,47 @@
use crate::{cursor::CursorSubNode, hash_builder::HashBuilder, updates::TrieUpdates, Nibbles};
use reth_primitives::{trie::StoredSubNode, MerkleCheckpoint, H256};
/// The progress of the state root computation.
#[derive(Debug)]
pub enum StateRootProgress {
/// The complete state root computation with updates and computed root.
Complete(H256, TrieUpdates),
/// The intermediate progress of state root computation.
/// Contains the walker stack, the hash builder and the trie updates.
Progress(IntermediateStateRootState, TrieUpdates),
}
/// The intermediate state of the state root computation.
#[derive(Debug)]
pub struct IntermediateStateRootState {
/// Previously constructed hash builder.
pub hash_builder: HashBuilder,
/// Previously recorded walker stack.
pub walker_stack: Vec<CursorSubNode>,
/// The last hashed account key processed.
pub last_account_key: H256,
/// The last walker key processed.
pub last_walker_key: Nibbles,
}
impl From<IntermediateStateRootState> for MerkleCheckpoint {
fn from(value: IntermediateStateRootState) -> Self {
Self {
last_account_key: value.last_account_key,
last_walker_key: value.last_walker_key.hex_data,
walker_stack: value.walker_stack.into_iter().map(StoredSubNode::from).collect(),
state: value.hash_builder.into(),
}
}
}
impl From<MerkleCheckpoint> for IntermediateStateRootState {
fn from(value: MerkleCheckpoint) -> Self {
Self {
hash_builder: HashBuilder::from(value.state),
walker_stack: value.walker_stack.into_iter().map(CursorSubNode::from).collect(),
last_account_key: value.last_account_key,
last_walker_key: Nibbles::from(value.last_walker_key),
}
}
}

File diff suppressed because it is too large Load Diff

151
crates/trie/src/updates.rs Normal file
View File

@@ -0,0 +1,151 @@
use crate::Nibbles;
use derive_more::Deref;
use reth_db::{
cursor::{DbCursorRO, DbCursorRW, DbDupCursorRO, DbDupCursorRW},
tables,
transaction::{DbTx, DbTxMut},
};
use reth_primitives::{
trie::{BranchNodeCompact, StorageTrieEntry, StoredNibbles, StoredNibblesSubKey},
H256,
};
use std::collections::BTreeMap;
/// The key of a trie node.
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)]
pub enum TrieKey {
/// A node in the account trie.
AccountNode(StoredNibbles),
/// A node in the storage trie.
StorageNode(H256, StoredNibblesSubKey),
/// Storage trie of an account.
StorageTrie(H256),
}
/// The operation to perform on the trie.
#[derive(Debug, Clone)]
pub enum TrieOp {
/// Delete the node entry.
Delete,
/// Update the node entry with the provided value.
Update(BranchNodeCompact),
}
impl TrieOp {
/// Returns `true` if the operation is an update.
pub fn is_update(&self) -> bool {
matches!(self, TrieOp::Update(..))
}
}
/// The aggregation of trie updates.
#[derive(Debug, Default, Clone, Deref)]
pub struct TrieUpdates {
trie_operations: BTreeMap<TrieKey, TrieOp>,
}
impl<const N: usize> From<[(TrieKey, TrieOp); N]> for TrieUpdates {
fn from(value: [(TrieKey, TrieOp); N]) -> Self {
Self { trie_operations: BTreeMap::from(value) }
}
}
impl TrieUpdates {
/// Schedule a delete operation on a trie key.
///
/// # Panics
///
/// If the key already exists and the operation is an update.
pub fn schedule_delete(&mut self, key: TrieKey) {
let existing = self.trie_operations.insert(key, TrieOp::Delete);
if let Some(op) = existing {
assert!(!op.is_update(), "Tried to delete a node that was already updated");
}
}
/// Append the updates to the current updates.
pub fn append(&mut self, other: &mut Self) {
self.trie_operations.append(&mut other.trie_operations);
}
/// Extend the updates with trie updates.
pub fn extend(&mut self, updates: impl Iterator<Item = (TrieKey, TrieOp)>) {
self.trie_operations.extend(updates);
}
/// Extend the updates with account trie updates.
pub fn extend_with_account_updates(&mut self, updates: BTreeMap<Nibbles, BranchNodeCompact>) {
self.extend(updates.into_iter().map(|(nibbles, node)| {
(TrieKey::AccountNode(nibbles.hex_data.into()), TrieOp::Update(node))
}));
}
/// Extend the updates with storage trie updates.
pub fn extend_with_storage_updates(
&mut self,
hashed_address: H256,
updates: BTreeMap<Nibbles, BranchNodeCompact>,
) {
self.extend(updates.into_iter().map(|(nibbles, node)| {
(TrieKey::StorageNode(hashed_address, nibbles.hex_data.into()), TrieOp::Update(node))
}));
}
/// Flush updates all aggregated updates to the database.
pub fn flush<'a, 'tx, TX>(self, tx: &'a TX) -> Result<(), reth_db::Error>
where
TX: DbTx<'tx> + DbTxMut<'tx>,
{
if self.trie_operations.is_empty() {
return Ok(())
}
let mut account_trie_cursor = tx.cursor_write::<tables::AccountsTrie>()?;
let mut storage_trie_cursor = tx.cursor_dup_write::<tables::StoragesTrie>()?;
for (key, operation) in self.trie_operations {
match key {
TrieKey::AccountNode(nibbles) => match operation {
TrieOp::Delete => {
if account_trie_cursor.seek_exact(nibbles)?.is_some() {
account_trie_cursor.delete_current()?;
}
}
TrieOp::Update(node) => {
if !nibbles.inner.is_empty() {
account_trie_cursor.upsert(nibbles, node)?;
}
}
},
TrieKey::StorageTrie(hashed_address) => match operation {
TrieOp::Delete => {
if storage_trie_cursor.seek_exact(hashed_address)?.is_some() {
storage_trie_cursor.delete_current_duplicates()?;
}
}
TrieOp::Update(..) => unreachable!("Cannot update full storage trie."),
},
TrieKey::StorageNode(hashed_address, nibbles) => {
if !nibbles.inner.is_empty() {
// Delete the old entry if it exists.
if storage_trie_cursor
.seek_by_key_subkey(hashed_address, nibbles.clone())?
.filter(|e| e.nibbles == nibbles)
.is_some()
{
storage_trie_cursor.delete_current()?;
}
// The operation is an update, insert new entry.
if let TrieOp::Update(node) = operation {
storage_trie_cursor
.upsert(hashed_address, StorageTrieEntry { nibbles, node })?;
}
}
}
};
}
Ok(())
}
}

View File

@@ -1,6 +1,7 @@
use crate::{
cursor::{CursorSubNode, TrieCursor},
prefix_set::PrefixSet,
updates::TrieUpdates,
Nibbles,
};
use reth_db::{table::Key, Error};
@@ -20,6 +21,8 @@ pub struct TrieWalker<'a, K, C> {
pub can_skip_current_node: bool,
/// A `PrefixSet` representing the changes to be applied to the trie.
pub changes: PrefixSet,
/// The trie updates to be applied to the trie.
trie_updates: Option<TrieUpdates>,
__phantom: PhantomData<K>,
}
@@ -30,8 +33,9 @@ impl<'a, K: Key + From<Vec<u8>>, C: TrieCursor<K>> TrieWalker<'a, K, C> {
let mut this = Self {
cursor,
changes,
can_skip_current_node: false,
stack: vec![CursorSubNode::default()],
can_skip_current_node: false,
trie_updates: None,
__phantom: PhantomData::default(),
};
@@ -45,6 +49,39 @@ impl<'a, K: Key + From<Vec<u8>>, C: TrieCursor<K>> TrieWalker<'a, K, C> {
this
}
/// Constructs a new TrieWalker from existing stack and a cursor.
pub fn from_stack(cursor: &'a mut C, stack: Vec<CursorSubNode>, changes: PrefixSet) -> Self {
let mut this = Self {
cursor,
changes,
stack,
can_skip_current_node: false,
trie_updates: None,
__phantom: PhantomData::default(),
};
this.update_skip_node();
this
}
/// Sets the flag whether the trie updates should be stored.
pub fn with_updates(mut self, retain_updates: bool) -> Self {
self.set_updates(retain_updates);
self
}
/// Sets the flag whether the trie updates should be stored.
pub fn set_updates(&mut self, retain_updates: bool) {
if retain_updates {
self.trie_updates = Some(TrieUpdates::default());
}
}
/// Split the walker into stack and trie updates.
pub fn split(mut self) -> (Vec<CursorSubNode>, TrieUpdates) {
let trie_updates = self.trie_updates.take();
(self.stack, trie_updates.unwrap_or_default())
}
/// Prints the current stack of trie nodes.
pub fn print_stack(&self) {
println!("====================== STACK ======================");
@@ -54,6 +91,11 @@ impl<'a, K: Key + From<Vec<u8>>, C: TrieCursor<K>> TrieWalker<'a, K, C> {
println!("====================== END STACK ======================\n");
}
/// The current length of the trie updates.
pub fn updates_len(&self) -> usize {
self.trie_updates.as_ref().map(|u| u.len()).unwrap_or(0)
}
/// Advances the walker to the next trie node and updates the skip node flag.
///
/// # Returns
@@ -121,7 +163,9 @@ impl<'a, K: Key + From<Vec<u8>>, C: TrieCursor<K>> TrieWalker<'a, K, C> {
// Delete the current node if it's included in the prefix set or it doesn't contain the root
// hash.
if !self.can_skip_current_node || nibble != -1 {
self.cursor.delete_current()?;
if let Some((updates, key)) = self.trie_updates.as_mut().zip(self.cursor.current()?) {
updates.schedule_delete(key);
}
}
Ok(())
@@ -209,7 +253,10 @@ impl<'a, K: Key + From<Vec<u8>>, C: TrieCursor<K>> TrieWalker<'a, K, C> {
mod tests {
use super::*;
use crate::cursor::{AccountTrieCursor, StorageTrieCursor};
use reth_db::{mdbx::test_utils::create_test_rw_db, tables, transaction::DbTxMut};
use reth_db::{
cursor::DbCursorRW, mdbx::test_utils::create_test_rw_db, tables, transaction::DbTxMut,
};
use reth_primitives::trie::StorageTrieEntry;
use reth_provider::Transaction;
#[test]
@@ -237,26 +284,32 @@ mod tests {
let db = create_test_rw_db();
let tx = Transaction::new(db.as_ref()).unwrap();
let account_trie =
AccountTrieCursor::new(tx.cursor_write::<tables::AccountsTrie>().unwrap());
test_cursor(account_trie, &inputs, &expected);
let mut account_cursor = tx.cursor_write::<tables::AccountsTrie>().unwrap();
for (k, v) in &inputs {
account_cursor.upsert(k.clone().into(), v.clone()).unwrap();
}
let account_trie = AccountTrieCursor::new(account_cursor);
test_cursor(account_trie, &expected);
let storage_trie = StorageTrieCursor::new(
tx.cursor_dup_write::<tables::StoragesTrie>().unwrap(),
H256::random(),
);
test_cursor(storage_trie, &inputs, &expected);
let hashed_address = H256::random();
let mut storage_cursor = tx.cursor_dup_write::<tables::StoragesTrie>().unwrap();
for (k, v) in &inputs {
storage_cursor
.upsert(
hashed_address,
StorageTrieEntry { nibbles: k.clone().into(), node: v.clone() },
)
.unwrap();
}
let storage_trie = StorageTrieCursor::new(storage_cursor, hashed_address);
test_cursor(storage_trie, &expected);
}
fn test_cursor<K, T>(mut trie: T, inputs: &[(Vec<u8>, BranchNodeCompact)], expected: &[Vec<u8>])
fn test_cursor<K, T>(mut trie: T, expected: &[Vec<u8>])
where
K: Key + From<Vec<u8>>,
T: TrieCursor<K>,
{
for (k, v) in inputs {
trie.upsert(k.clone().into(), v.clone()).unwrap();
}
let mut walker = TrieWalker::new(&mut trie, Default::default());
assert!(walker.key().unwrap().is_empty());
@@ -275,11 +328,7 @@ mod tests {
fn cursor_rootnode_with_changesets() {
let db = create_test_rw_db();
let tx = Transaction::new(db.as_ref()).unwrap();
let mut trie = StorageTrieCursor::new(
tx.cursor_dup_write::<tables::StoragesTrie>().unwrap(),
H256::random(),
);
let mut cursor = tx.cursor_dup_write::<tables::StoragesTrie>().unwrap();
let nodes = vec![
(
@@ -306,10 +355,13 @@ mod tests {
),
];
let hashed_address = H256::random();
for (k, v) in nodes {
trie.upsert(k.into(), v).unwrap();
cursor.upsert(hashed_address, StorageTrieEntry { nibbles: k.into(), node: v }).unwrap();
}
let mut trie = StorageTrieCursor::new(cursor, hashed_address);
// No changes
let mut cursor = TrieWalker::new(&mut trie, Default::default());
assert_eq!(cursor.key(), Some(Nibbles::from(vec![]))); // root