diff --git a/miden-crypto/benches/large-smt.rs b/miden-crypto/benches/large-smt.rs index 6d1daf0ea..ac149d6ef 100644 --- a/miden-crypto/benches/large-smt.rs +++ b/miden-crypto/benches/large-smt.rs @@ -2,8 +2,13 @@ use std::hint; use criterion::{Criterion, criterion_group, criterion_main}; use miden_crypto::{ - Word, - merkle::smt::{LargeSmt, RocksDbConfig, RocksDbStorage}, + Felt, FieldElement, Word, + merkle::{ + EmptySubtreeRoots, NodeIndex, + smt::{ + InnerNode, LargeSmt, MemoryStorage, RocksDbConfig, RocksDbStorage, SMT_DEPTH, Subtree, + }, + }, }; mod common; @@ -14,6 +19,100 @@ use crate::{ config::{DEFAULT_MEASUREMENT_TIME, DEFAULT_SAMPLE_SIZE}, }; +// === Subtree Serialization Benchmarks === + +const ROOT_DEPTH: u8 = 24; +const SUBTREE_DEPTH: u8 = 8; + +// Creates a dense subtree with all inner nodes populated (bottom-up by depth). +fn create_dense_subtree() -> Subtree { + let root_index = NodeIndex::new(ROOT_DEPTH, 0).unwrap(); + let mut subtree = Subtree::new(root_index); + let node_hash: Word = [Felt::ONE; 4].into(); + + for relative_depth in (1..=SUBTREE_DEPTH).rev() { + let depth = ROOT_DEPTH + relative_depth; + let nodes_at_depth = 1u64 << (relative_depth - 1); + let first_value = nodes_at_depth; + + for offset in 0..nodes_at_depth { + let idx = NodeIndex::new(depth, first_value + offset).unwrap(); + let node = InnerNode { left: node_hash, right: node_hash }; + subtree.insert_inner_node(idx, node); + } + } + subtree +} + +// Creates a sparse subtree with a single path from root to leaf (leftmost path). +fn create_sparse_subtree() -> Subtree { + let root_index = NodeIndex::new(ROOT_DEPTH, 0).unwrap(); + let mut subtree = Subtree::new(root_index); + + let mut child_hash: Word = [Felt::ONE; 4].into(); + let mut current_idx = NodeIndex::new(ROOT_DEPTH + SUBTREE_DEPTH, 0).unwrap(); + + for _ in 0..SUBTREE_DEPTH { + let depth = current_idx.depth(); + let empty_hash = *EmptySubtreeRoots::entry(SMT_DEPTH, depth + 1); + let node = InnerNode { left: child_hash, right: empty_hash }; + child_hash = node.hash(); + subtree.insert_inner_node(current_idx, node); + current_idx = current_idx.parent(); + } + subtree +} + +benchmark_with_setup_data! { + subtree_serialize_dense, + DEFAULT_MEASUREMENT_TIME, + DEFAULT_SAMPLE_SIZE, + "serialize_dense", + create_dense_subtree, + |b: &mut criterion::Bencher, subtree: &Subtree| { + b.iter(|| hint::black_box(subtree.to_vec())) + }, +} + +benchmark_with_setup_data! { + subtree_deserialize_dense, + DEFAULT_MEASUREMENT_TIME, + DEFAULT_SAMPLE_SIZE, + "deserialize_dense", + || { + let subtree = create_dense_subtree(); + (subtree.root_index(), subtree.to_vec()) + }, + |b: &mut criterion::Bencher, (root_index, bytes): &(NodeIndex, Vec)| { + b.iter(|| hint::black_box(Subtree::from_vec(*root_index, bytes).unwrap())) + }, +} + +benchmark_with_setup_data! { + subtree_serialize_sparse, + DEFAULT_MEASUREMENT_TIME, + DEFAULT_SAMPLE_SIZE, + "serialize_sparse", + create_sparse_subtree, + |b: &mut criterion::Bencher, subtree: &Subtree| { + b.iter(|| hint::black_box(subtree.to_vec())) + }, +} + +benchmark_with_setup_data! { + subtree_deserialize_sparse, + DEFAULT_MEASUREMENT_TIME, + DEFAULT_SAMPLE_SIZE, + "deserialize_sparse", + || { + let subtree = create_sparse_subtree(); + (subtree.root_index(), subtree.to_vec()) + }, + |b: &mut criterion::Bencher, (root_index, bytes): &(NodeIndex, Vec)| { + b.iter(|| hint::black_box(Subtree::from_vec(*root_index, bytes).unwrap())) + }, +} + benchmark_with_setup_data! { large_smt_open, DEFAULT_MEASUREMENT_TIME, @@ -143,6 +242,91 @@ benchmark_batch! { |size| Some(criterion::Throughput::Elements(size as u64)) } +// === Memory Storage Benchmarks === + +benchmark_with_setup_data! { + memory_smt_open, + DEFAULT_MEASUREMENT_TIME, + DEFAULT_SAMPLE_SIZE, + "open", + || { + let entries = generate_smt_entries_sequential(256); + let keys = generate_test_keys_sequential(10); + let storage = MemoryStorage::new(); + let smt = LargeSmt::with_entries(storage, entries).unwrap(); + (smt, keys) + }, + |b: &mut criterion::Bencher, (smt, keys): &(LargeSmt, Vec)| { + b.iter(|| { + for key in keys { + hint::black_box(smt.open(key)); + } + }) + }, +} + +benchmark_with_setup_data! { + memory_smt_compute_mutations, + DEFAULT_MEASUREMENT_TIME, + DEFAULT_SAMPLE_SIZE, + "compute_mutations", + || { + let entries = generate_smt_entries_sequential(256); + let storage = MemoryStorage::new(); + let smt = LargeSmt::with_entries(storage, entries).unwrap(); + let new_entries = generate_smt_entries_sequential(10_000); + (smt, new_entries) + }, + |b: &mut criterion::Bencher, (smt, new_entries): &(LargeSmt, Vec<(Word, Word)>)| { + b.iter(|| { + hint::black_box(smt.compute_mutations(new_entries.clone()).unwrap()); + }) + }, +} + +benchmark_batch! { + memory_smt_apply_mutations, + &[100, 1_000, 10_000], + |b: &mut criterion::Bencher, entry_count: usize| { + use criterion::BatchSize; + + let base_entries = generate_smt_entries_sequential(256); + + b.iter_batched( + || { + let storage = MemoryStorage::new(); + let smt = LargeSmt::with_entries(storage, base_entries.clone()).unwrap(); + let new_entries = generate_smt_entries_sequential(entry_count); + let mutations = smt.compute_mutations(new_entries).unwrap(); + (smt, mutations) + }, + |(mut smt, mutations)| { + smt.apply_mutations(mutations).unwrap(); + }, + BatchSize::LargeInput, + ) + }, + |size| Some(criterion::Throughput::Elements(size as u64)) +} + +benchmark_batch! { + memory_smt_insert_batch, + &[1, 16, 32, 64, 128], + |b: &mut criterion::Bencher, insert_count: usize| { + let base_entries = generate_smt_entries_sequential(256); + let storage = MemoryStorage::new(); + let mut smt = LargeSmt::with_entries(storage, base_entries).unwrap(); + + b.iter(|| { + for _ in 0..insert_count { + let new_entries = generate_smt_entries_sequential(10_000); + smt.insert_batch(new_entries).unwrap(); + } + }) + }, + |size| Some(criterion::Throughput::Elements(size as u64)) +} + criterion_group!( large_smt_benchmark_group, large_smt_open, @@ -152,4 +336,20 @@ criterion_group!( large_smt_insert_batch, ); -criterion_main!(large_smt_benchmark_group); +criterion_group!( + memory_smt_benchmark_group, + memory_smt_open, + memory_smt_compute_mutations, + memory_smt_apply_mutations, + memory_smt_insert_batch, +); + +criterion_group!( + subtree_benchmark_group, + subtree_serialize_dense, + subtree_deserialize_dense, + subtree_serialize_sparse, + subtree_deserialize_sparse, +); + +criterion_main!(large_smt_benchmark_group, memory_smt_benchmark_group, subtree_benchmark_group); diff --git a/miden-crypto/src/merkle/smt/large/batch_ops.rs b/miden-crypto/src/merkle/smt/large/batch_ops.rs index db48c3fe0..bae948c30 100644 --- a/miden-crypto/src/merkle/smt/large/batch_ops.rs +++ b/miden-crypto/src/merkle/smt/large/batch_ops.rs @@ -11,8 +11,8 @@ use super::{ use crate::{ Word, merkle::smt::{ - EmptySubtreeRoots, LeafIndex, Map, MerkleError, MutationSet, NodeIndex, NodeMutation, - NodeMutations, SmtLeaf, SparseMerkleTree, + EmptySubtreeRoots, InnerNode, LeafIndex, Map, MerkleError, MutationSet, NodeIndex, + NodeMutation, NodeMutations, SmtLeaf, SparseMerkleTree, full::concurrent::{ SUBTREE_DEPTH, SubtreeLeaf, SubtreeLeavesIter, fetch_sibling_pair, process_sorted_pairs_to_leaves, @@ -51,16 +51,15 @@ impl LargeSmt { let subtree_root_index = NodeIndex::new_unchecked(subtree_root_depth, subtree_leaves[0].col >> SUBTREE_DEPTH); - // Load subtree from storage if below in-memory horizon; otherwise use in-memory nodes - let mut subtree_opt = if subtree_root_depth < IN_MEMORY_DEPTH { - None + let is_storage_subtree = subtree_root_depth >= IN_MEMORY_DEPTH; + + // For storage subtrees, load existing subtree to read old nodes during mutation building + let existing_subtree = if is_storage_subtree { + self.storage + .get_subtree(subtree_root_index) + .expect("Storage error getting subtree in insert_batch") } else { - Some( - self.storage - .get_subtree(subtree_root_index) - .expect("Storage error getting subtree in insert_batch") - .unwrap_or_else(|| Subtree::new(subtree_root_index)), - ) + None }; // Build mutations for the subtree @@ -68,36 +67,41 @@ impl LargeSmt { subtree_leaves, SMT_DEPTH, subtree_root_depth, - subtree_opt.as_ref(), + existing_subtree.as_ref(), ); - let (in_memory_mutations, subtree_update) = if subtree_root_depth < IN_MEMORY_DEPTH { + let (in_memory_mutations, subtree_update) = if !is_storage_subtree { // In-memory nodes: return mutations for direct application (mutations, None) } else { - // Storage nodes: apply mutations to loaded subtree and determine storage action + // Storage nodes: merge existing nodes with mutations and build directly let modified = !mutations.is_empty(); - if let Some(subtree) = subtree_opt.as_mut() { + + let update = if !modified { + None + } else { + // Start with existing nodes (if any), then apply mutations + let mut final_nodes: Map = + existing_subtree.map(|s| s.iter_nodes().collect()).unwrap_or_default(); + + // Apply mutations: remove Removals, add/update Additions for (index, mutation) in mutations { match mutation { NodeMutation::Removal => { - subtree.remove_inner_node(index); + final_nodes.remove(&index); }, NodeMutation::Addition(node) => { - subtree.insert_inner_node(index, node); + final_nodes.insert(index, node); }, } } - } - let update = if !modified { - None - } else if let Some(subtree) = subtree_opt - && !subtree.is_empty() - { - Some(SubtreeUpdate::Store { index: subtree_root_index, subtree }) - } else { - Some(SubtreeUpdate::Delete { index: subtree_root_index }) + if final_nodes.is_empty() { + Some(SubtreeUpdate::Delete { index: subtree_root_index }) + } else { + let subtree = Subtree::from_nodes(subtree_root_index, final_nodes); + Some(SubtreeUpdate::Store { index: subtree_root_index, subtree }) + } }; (NodeMutations::default(), update) @@ -547,9 +551,15 @@ impl LargeSmt { // Update the root in memory self.in_memory_nodes[ROOT_MEMORY_INDEX] = new_root; - // Process node mutations + // Process node mutations - group by subtree for efficient batch application. + // Mutations are pre-sorted by subtree root, so consecutive mutations belong + // to the same subtree. + let mut current_subtree_root: Option = None; + let mut current_subtree_mutations: Vec<(NodeIndex, NodeMutation)> = Vec::new(); + for (index, mutation) in sorted_node_mutations { if index.depth() < IN_MEMORY_DEPTH { + // In-memory mutations applied directly match mutation { Removal => { SparseMerkleTree::::remove_inner_node(self, index); @@ -560,23 +570,33 @@ impl LargeSmt { }; } else { let subtree_root_index = Subtree::find_subtree_root(index); - let subtree = loaded_subtrees - .get_mut(&subtree_root_index) - .expect("Subtree map entry must exist") - .as_mut() - .expect("Subtree must exist as it was either fetched or created"); - match mutation { - Removal => { - subtree.remove_inner_node(index); - }, - Addition(node) => { - subtree.insert_inner_node(index, node); - }, - }; + // Check if we've moved to a new subtree + if current_subtree_root != Some(subtree_root_index) { + // Apply accumulated mutations to previous subtree + if let Some(prev_root) = current_subtree_root { + Self::apply_mutations_to_subtree( + &mut loaded_subtrees, + prev_root, + mem::take(&mut current_subtree_mutations), + ); + } + current_subtree_root = Some(subtree_root_index); + } + + current_subtree_mutations.push((index, mutation)); } } + // Apply any remaining mutations to the last subtree + if let Some(prev_root) = current_subtree_root { + Self::apply_mutations_to_subtree( + &mut loaded_subtrees, + prev_root, + current_subtree_mutations, + ); + } + // Go through subtrees, see if any are empty, and if so remove them for (_index, subtree) in loaded_subtrees.iter_mut() { if subtree.as_ref().is_some_and(|s| s.is_empty()) { @@ -640,6 +660,46 @@ impl LargeSmt { Ok(()) } + /// Applies a batch of mutations to a subtree efficiently. + /// + /// Instead of using `insert_inner_node`/`remove_inner_node` calls repeatedly, + /// this collects existing nodes into a Map, applies all mutations, then rebuilds + /// the subtree in one pass using `from_nodes()`. + fn apply_mutations_to_subtree( + loaded_subtrees: &mut Map>, + subtree_root_index: NodeIndex, + mutations: Vec<(NodeIndex, NodeMutation)>, + ) { + let subtree_entry = loaded_subtrees + .get_mut(&subtree_root_index) + .expect("Subtree map entry must exist"); + + let existing_subtree = subtree_entry + .take() + .expect("Subtree must exist as it was either fetched or created"); + + // Collect existing nodes into a Map + let mut final_nodes: Map = existing_subtree.iter_nodes().collect(); + + // Apply all mutations + for (index, mutation) in mutations { + match mutation { + NodeMutation::Removal => { + final_nodes.remove(&index); + }, + NodeMutation::Addition(node) => { + final_nodes.insert(index, node); + }, + } + } + + // Rebuild subtree from final node set (or leave as None if empty) + if !final_nodes.is_empty() { + *subtree_entry = Some(Subtree::from_nodes(subtree_root_index, final_nodes)); + } + // If final_nodes is empty, subtree_entry is already None from the take() + } + /// Computes what changes are necessary to insert the specified key-value pairs into this Merkle /// tree, allowing for validation before applying those changes. /// diff --git a/miden-crypto/src/merkle/smt/large/construction.rs b/miden-crypto/src/merkle/smt/large/construction.rs index fcbdc5fd9..7f11d206c 100644 --- a/miden-crypto/src/merkle/smt/large/construction.rs +++ b/miden-crypto/src/merkle/smt/large/construction.rs @@ -264,10 +264,7 @@ impl LargeSmt { let subtree_root_index = NodeIndex::new(bottom_depth - SUBTREE_DEPTH, subtree_root.col).unwrap(); - let mut subtree = Subtree::new(subtree_root_index); - for (index, node) in nodes { - subtree.insert_inner_node(index, node); - } + let subtree = Subtree::from_nodes(subtree_root_index, nodes); sender.send(subtree).expect("Flume channel disconnected unexpectedly"); subtree_root }) diff --git a/miden-crypto/src/merkle/smt/large/subtree/mod.rs b/miden-crypto/src/merkle/smt/large/subtree/mod.rs index bc7348fb3..114a32484 100644 --- a/miden-crypto/src/merkle/smt/large/subtree/mod.rs +++ b/miden-crypto/src/merkle/smt/large/subtree/mod.rs @@ -1,10 +1,7 @@ use alloc::vec::Vec; use super::{EmptySubtreeRoots, InnerNode, InnerNodeInfo, NodeIndex, SMT_DEPTH}; -use crate::{ - Word, - merkle::smt::{Map, full::concurrent::SUBTREE_DEPTH}, -}; +use crate::{Word, merkle::smt::full::concurrent::SUBTREE_DEPTH}; mod error; pub use error::SubtreeError; @@ -15,28 +12,31 @@ mod tests; /// Represents a complete 8-depth subtree that is serialized into a single RocksDB entry. /// /// ### What is stored -/// - `nodes` tracks only **non-empty inner nodes** of this subtree (i.e., nodes for which at least -/// one child differs from the canonical empty hash). Each entry stores an `InnerNode` (hash -/// pair). +/// - `non_empty_node_bits` is a 256-bit bitmask (4 x u64) where each bit indicates whether the +/// corresponding node exists (i.e., differs from the canonical empty state). +/// - `nodes` stores the left and right child hashes for each existing node, packed in order of set +/// bits. For each set bit at position `i`, two consecutive Words (left, right) are stored. /// /// ### Local index layout (how indices are computed) /// - Indices are **subtree-local** and follow binary-heap (level-order) layout: `root = 0`; /// children of `i` are at `2i+1` and `2i+2`. /// - Equivalently, given a `(depth, value)` from the parent tree, the local index is obtained by -/// taking the node’s depth **relative to the subtree root** and its left-to-right position within +/// taking the node's depth **relative to the subtree root** and its left-to-right position within /// that level (offset by the total number of nodes in all previous levels). /// /// ### Serialization (`to_vec` / `from_vec`) /// - Uses a **512-bit bitmask** (2 bits per node) to mark non-empty left/right children, followed /// by a packed stream of `Word` hashes for each set bit. /// - Children equal to the canonical empty hash are omitted in the byte representation and -/// reconstructed on load using `EmptySubtreeRoots` and the child’s depth in the parent tree. +/// reconstructed on load using `EmptySubtreeRoots` and the child's depth in the parent tree. #[derive(Debug, Clone)] pub struct Subtree { /// Index of this subtree's root in the parent SMT. root_index: NodeIndex, - /// Inner nodes keyed by subtree-local index (binary-heap order). - nodes: Map, + /// Bitmask indicating which nodes exist (256 bits for local indices 0-255). + non_empty_node_bits: [u64; 4], + /// Child hashes for existing nodes, stored as pairs (left, right) in order of set bits. + nodes: Vec, } impl Subtree { @@ -46,7 +46,41 @@ impl Subtree { const BITS_PER_NODE: usize = 2; pub fn new(root_index: NodeIndex) -> Self { - Self { root_index, nodes: Map::new() } + Self { + root_index, + non_empty_node_bits: [0u64; 4], + nodes: Vec::new(), + } + } + + /// Creates a subtree from an iterator of nodes. + /// + /// This is more efficient than calling `insert_inner_node` repeatedly, + /// as it builds the bitmask and node vector in a single pass after sorting. + pub fn from_nodes( + root_index: NodeIndex, + nodes: impl IntoIterator, + ) -> Self { + // Convert to local indices and collect + let mut local_nodes: Vec<(u8, InnerNode)> = nodes + .into_iter() + .map(|(index, node)| (Self::global_to_local(index, root_index), node)) + .collect(); + + // Sort by local index for sequential Vec building + local_nodes.sort_unstable_by_key(|(local_idx, _)| *local_idx); + + // Build bitmask and nodes Vec in one pass + let mut subtree = Self::new(root_index); + subtree.nodes.reserve(local_nodes.len() * 2); + + for (local_index, inner_node) in local_nodes { + subtree.set_node_bit(local_index, true); + subtree.nodes.push(inner_node.left); + subtree.nodes.push(inner_node.right); + } + + subtree } pub fn root_index(&self) -> NodeIndex { @@ -54,7 +88,7 @@ impl Subtree { } pub fn len(&self) -> usize { - self.nodes.len() + self.non_empty_node_bits.iter().map(|&bits| bits.count_ones() as usize).sum() } pub fn insert_inner_node( @@ -63,17 +97,98 @@ impl Subtree { inner_node: InnerNode, ) -> Option { let local_index = Self::global_to_local(index, self.root_index); - self.nodes.insert(local_index, inner_node) + let was_present = self.is_node_present(local_index); + + if was_present { + // Node exists - replace in place + let position = self.node_position(local_index); + let old_left = self.nodes[position * 2]; + let old_right = self.nodes[position * 2 + 1]; + self.nodes[position * 2] = inner_node.left; + self.nodes[position * 2 + 1] = inner_node.right; + Some(InnerNode { left: old_left, right: old_right }) + } else { + // Node doesn't exist - insert at the correct position + let position = self.node_position(local_index); + self.set_node_bit(local_index, true); + self.nodes.insert(position * 2, inner_node.left); + self.nodes.insert(position * 2 + 1, inner_node.right); + None + } } pub fn remove_inner_node(&mut self, index: NodeIndex) -> Option { let local_index = Self::global_to_local(index, self.root_index); - self.nodes.remove(&local_index) + + if !self.is_node_present(local_index) { + return None; + } + + let position = self.node_position(local_index); + let old_left = self.nodes[position * 2]; + let old_right = self.nodes[position * 2 + 1]; + + self.set_node_bit(local_index, false); + self.nodes.remove(position * 2 + 1); + self.nodes.remove(position * 2); + + Some(InnerNode { left: old_left, right: old_right }) } pub fn get_inner_node(&self, index: NodeIndex) -> Option { let local_index = Self::global_to_local(index, self.root_index); - self.nodes.get(&local_index).cloned() + + if !self.is_node_present(local_index) { + return None; + } + + let position = self.node_position(local_index); + Some(InnerNode { + left: self.nodes[position * 2], + right: self.nodes[position * 2 + 1], + }) + } + + /// Returns true if a node exists at the given local index. + #[inline] + fn is_node_present(&self, local_index: u8) -> bool { + let word_idx = (local_index / 64) as usize; + let bit_idx = local_index % 64; + (self.non_empty_node_bits[word_idx] >> bit_idx) & 1 != 0 + } + + /// Sets or clears the bit for the given local index. + #[inline] + fn set_node_bit(&mut self, local_index: u8, present: bool) { + let word_idx = (local_index / 64) as usize; + let bit_idx = local_index % 64; + if present { + self.non_empty_node_bits[word_idx] |= 1u64 << bit_idx; + } else { + self.non_empty_node_bits[word_idx] &= !(1u64 << bit_idx); + } + } + + /// Returns the position in the `nodes` Vec for the given local index. + /// This is the count of set bits before this index in the bitmask. + #[inline] + fn node_position(&self, local_index: u8) -> usize { + let mut count = 0usize; + let full_words = (local_index / 64) as usize; + let remaining_bits = local_index % 64; + + // Count all bits in full words + for i in 0..full_words { + count += self.non_empty_node_bits[i].count_ones() as usize; + } + + // Count bits in the partial word up to (but not including) the target bit + if remaining_bits > 0 { + let mask = (1u64 << remaining_bits) - 1; + count += (self.non_empty_node_bits[full_words] & mask).count_ones() as usize; + } + + count } /// Serializes this subtree into a compact byte representation. @@ -95,20 +210,24 @@ impl Subtree { let mut bitmask = [0u8; Self::BITMASK_SIZE]; for local_index in 0..Self::MAX_NODES { - if let Some(node) = self.nodes.get(&local_index) { + if self.is_node_present(local_index) { + let position = self.node_position(local_index); + let left = self.nodes[position * 2]; + let right = self.nodes[position * 2 + 1]; + let bit_offset = (local_index as usize) * Self::BITS_PER_NODE; let node_depth_in_subtree = Self::local_index_to_depth(local_index); let child_depth = self.root_index.depth() + node_depth_in_subtree + 1; let empty_hash = *EmptySubtreeRoots::entry(SMT_DEPTH, child_depth); - if node.left != empty_hash { + if left != empty_hash { Self::set_bit(&mut bitmask, bit_offset); - data.extend_from_slice(&node.left.as_bytes()); + data.extend_from_slice(&left.as_bytes()); } - if node.right != empty_hash { + if right != empty_hash { Self::set_bit(&mut bitmask, bit_offset + 1); - data.extend_from_slice(&node.right.as_bytes()); + data.extend_from_slice(&right.as_bytes()); } } } @@ -136,7 +255,7 @@ impl Subtree { /// a `Word` hash is read sequentially from the data section. /// /// When a child bit is unset, the corresponding hash is reconstructed from - /// `EmptySubtreeRoots` based on the child’s depth in the full tree. + /// `EmptySubtreeRoots` based on the child's depth in the full tree. /// /// Errors are returned if the byte slice is too short, contains an unexpected /// number of hashes, or leaves unconsumed data at the end. @@ -156,7 +275,8 @@ impl Subtree { }); } - let mut nodes = Map::new(); + let mut non_empty_node_bits = [0u64; 4]; + let mut nodes = Vec::new(); let mut hash_chunks = hash_data.chunks_exact(Self::HASH_SIZE); // Process each potential node position @@ -193,8 +313,14 @@ impl Subtree { empty_hash }; - let inner_node = InnerNode { left: left_hash, right: right_hash }; - nodes.insert(local_index, inner_node); + // Set the bit in the bitmask + let word_idx = (local_index / 64) as usize; + let bit_idx = local_index % 64; + non_empty_node_bits[word_idx] |= 1u64 << bit_idx; + + // Store the child hashes + nodes.push(left_hash); + nodes.push(right_hash); } } @@ -203,7 +329,7 @@ impl Subtree { return Err(SubtreeError::ExtraData); } - Ok(Self { root_index, nodes }) + Ok(Self { root_index, non_empty_node_bits, nodes }) } fn global_to_local(global: NodeIndex, base: NodeIndex) -> u8 { @@ -245,7 +371,37 @@ impl Subtree { } pub fn is_empty(&self) -> bool { - self.len() == 0 + self.non_empty_node_bits.iter().all(|&bits| bits == 0) + } + + /// Returns an iterator over all (NodeIndex, InnerNode) pairs in this subtree. + pub fn iter_nodes(&self) -> impl Iterator + '_ { + (0..Self::MAX_NODES) + .filter(|&local_index| self.is_node_present(local_index)) + .map(|local_index| { + let position = self.node_position(local_index); + let inner_node = InnerNode { + left: self.nodes[position * 2], + right: self.nodes[position * 2 + 1], + }; + let global_index = Self::local_to_global(local_index, self.root_index); + (global_index, inner_node) + }) + } + + /// Converts a local subtree index back to a global NodeIndex. + fn local_to_global(local_index: u8, root_index: NodeIndex) -> NodeIndex { + let local_depth = Self::local_index_to_depth(local_index); + let global_depth = root_index.depth() + local_depth; + + // Calculate position within the level + let level_start = (1u8 << local_depth) - 1; + let position_in_level = local_index - level_start; + + // Global value is root's value shifted left by local_depth, plus position + let global_value = (root_index.value() << local_depth) | (position_in_level as u64); + + NodeIndex::new_unchecked(global_depth, global_value) } /// Convert local index to depth within subtree @@ -256,10 +412,14 @@ impl Subtree { } pub fn iter_inner_node_info(&self) -> impl Iterator + '_ { - self.nodes.values().map(|inner_node_ref| InnerNodeInfo { - value: inner_node_ref.hash(), - left: inner_node_ref.left, - right: inner_node_ref.right, - }) + (0..Self::MAX_NODES) + .filter(|&local_index| self.is_node_present(local_index)) + .map(|local_index| { + let position = self.node_position(local_index); + let left = self.nodes[position * 2]; + let right = self.nodes[position * 2 + 1]; + let inner_node = InnerNode { left, right }; + InnerNodeInfo { value: inner_node.hash(), left, right } + }) } }