diff --git a/core/primitives/src/state.rs b/core/primitives/src/state.rs index 8c7da64bb84..243d40e6a5f 100644 --- a/core/primitives/src/state.rs +++ b/core/primitives/src/state.rs @@ -4,7 +4,7 @@ use near_primitives_core::hash::{hash, CryptoHash}; use near_schema_checker_lib::ProtocolSchema; /// State value reference. Used to charge fees for value length before retrieving the value itself. -#[derive(BorshSerialize, BorshDeserialize, Clone, PartialEq, Eq, Hash, ProtocolSchema)] +#[derive(BorshSerialize, BorshDeserialize, Clone, Copy, PartialEq, Eq, Hash, ProtocolSchema)] pub struct ValueRef { /// Value length in bytes. pub length: u32, @@ -91,7 +91,7 @@ impl FlatStateValue { pub fn to_value_ref(&self) -> ValueRef { match self { - Self::Ref(value_ref) => value_ref.clone(), + Self::Ref(value_ref) => *value_ref, Self::Inlined(value) => ValueRef::new(value), } } diff --git a/core/store/src/flat/storage.rs b/core/store/src/flat/storage.rs index 035ac11cd7c..661862e9512 100644 --- a/core/store/src/flat/storage.rs +++ b/core/store/src/flat/storage.rs @@ -305,7 +305,7 @@ impl FlatStorage { let changes = guard.get_block_changes(block_hash)?; match changes.get(key) { Some(value_ref) => { - return Ok(value_ref.clone().map(|value_ref| FlatStateValue::Ref(value_ref))); + return Ok((*value_ref).map(|value_ref| FlatStateValue::Ref(value_ref))); } None => {} }; diff --git a/core/store/src/trie/insert_delete.rs b/core/store/src/trie/insert_delete.rs index af460c0ce88..751cb04312f 100644 --- a/core/store/src/trie/insert_delete.rs +++ b/core/store/src/trie/insert_delete.rs @@ -9,25 +9,27 @@ use borsh::BorshSerialize; use near_primitives::hash::{hash, CryptoHash}; use near_primitives::state::ValueRef; -pub(crate) struct NodesStorage { +pub(crate) struct NodesStorage<'a> { nodes: Vec>, values: Vec>>, pub(crate) refcount_changes: TrieRefcountDeltaMap, + pub(crate) trie: &'a Trie, } const INVALID_STORAGE_HANDLE: &str = "invalid storage handle"; /// Local mutable storage that owns node objects. -impl NodesStorage { - pub fn new() -> NodesStorage { +impl<'a> NodesStorage<'a> { + pub fn new(trie: &'a Trie) -> NodesStorage<'a> { NodesStorage { nodes: Vec::new(), refcount_changes: TrieRefcountDeltaMap::new(), values: Vec::new(), + trie, } } - fn destroy(&mut self, handle: StorageHandle) -> TrieNodeWithSize { + pub(crate) fn destroy(&mut self, handle: StorageHandle) -> TrieNodeWithSize { self.nodes .get_mut(handle.0) .expect(INVALID_STORAGE_HANDLE) @@ -57,8 +59,9 @@ impl NodesStorage { } pub(crate) fn store_value(&mut self, value: Vec) -> StorageValueHandle { + let value_len = value.len(); self.values.push(Some(value)); - StorageValueHandle(self.values.len() - 1) + StorageValueHandle(self.values.len() - 1, value_len) } pub(crate) fn value_ref(&self, handle: StorageValueHandle) -> &[u8] { @@ -69,7 +72,7 @@ impl NodesStorage { .expect(INVALID_STORAGE_HANDLE) } - fn store_at(&mut self, handle: StorageHandle, node: TrieNodeWithSize) { + pub(crate) fn store_at(&mut self, handle: StorageHandle, node: TrieNodeWithSize) { debug_assert!(self.nodes.get(handle.0).expect(INVALID_STORAGE_HANDLE).is_none()); self.nodes[handle.0] = Some(node); } @@ -432,86 +435,84 @@ impl Trie { } } } - self.fix_nodes(memory, path, key_deleted)?; + + let mut child_memory_usage = 0; + for handle in path.into_iter().rev() { + // First, recompute memory usage, emulating the recursive descent. + let TrieNodeWithSize { node, mut memory_usage } = memory.destroy(handle); + memory_usage += child_memory_usage; + memory.store_at(handle, TrieNodeWithSize { node, memory_usage }); + + // Then, squash node to ensure unique trie structure, changing its + // type if needed. If `key_deleted` is false, trie structure is + // untouched. + if key_deleted { + self.squash_node(memory, handle)?; + } + + child_memory_usage = memory.node_ref(handle).memory_usage; + } + Ok(root_node) } - /// Iterates over nodes in `path`, changing their types where needed, - /// if `key_deleted` is true, so trie structure has to change. - /// If `key_deleted` is false, only recomputes memory usages along the path. - fn fix_nodes( + pub(crate) fn squash_node( &self, memory: &mut NodesStorage, - path: Vec, - key_deleted: bool, + handle: StorageHandle, ) -> Result<(), StorageError> { - let mut child_memory_usage = 0; - for handle in path.into_iter().rev() { - let TrieNodeWithSize { node, memory_usage } = memory.destroy(handle); - let memory_usage = memory_usage + child_memory_usage; - if key_deleted { - match node { - TrieNode::Empty => { - memory.store_at(handle, TrieNodeWithSize::empty()); - } - TrieNode::Leaf(key, value) => { - memory.store_at( - handle, - TrieNodeWithSize::new(TrieNode::Leaf(key, value), memory_usage), - ); - } - TrieNode::Branch(mut children, value) => { - for child in children.0.iter_mut() { - if let Some(NodeHandle::InMemory(h)) = child { - if let TrieNode::Empty = memory.node_ref(*h).node { - *child = None - } - } - } - let num_children = children.iter().count(); - if num_children == 0 { - if let Some(value) = value { - let empty = NibbleSlice::new(&[]).encoded(true).into_vec(); - let leaf_node = TrieNode::Leaf(empty, value); - let memory_usage = leaf_node.memory_usage_direct(memory); - memory.store_at( - handle, - TrieNodeWithSize::new(leaf_node, memory_usage), - ); - } else { - memory.store_at(handle, TrieNodeWithSize::empty()); - } - } else if num_children == 1 && value.is_none() { - // Branch with one child becomes extension - // Extension followed by leaf becomes leaf - // Extension followed by extension becomes extension - let idx = children.iter().next().unwrap().0; - let child = children[idx].take().unwrap(); - let key = NibbleSlice::new(&[(idx << 4) as u8]) - .encoded_leftmost(1, false) - .into_vec(); - self.fix_extension_node(memory, handle, key, child)?; - } else { - let node = TrieNodeWithSize::new( - TrieNode::Branch(children, value), - memory_usage, - ); - memory.store_at(handle, node); + let TrieNodeWithSize { node, memory_usage } = memory.destroy(handle); + match node { + TrieNode::Empty => { + memory.store_at(handle, TrieNodeWithSize::empty()); + } + TrieNode::Leaf(key, value) => { + memory.store_at( + handle, + TrieNodeWithSize::new(TrieNode::Leaf(key, value), memory_usage), + ); + } + TrieNode::Branch(mut children, value) => { + for child in children.0.iter_mut() { + if let Some(NodeHandle::InMemory(h)) = child { + if let TrieNode::Empty = memory.node_ref(*h).node { + *child = None } } - TrieNode::Extension(key, child) => { - self.fix_extension_node(memory, handle, key, child)?; + } + let num_children = children.iter().count(); + if num_children == 0 { + if let Some(value) = value { + let empty = NibbleSlice::new(&[]).encoded(true).into_vec(); + let leaf_node = TrieNode::Leaf(empty, value); + let memory_usage = leaf_node.memory_usage_direct(memory); + memory.store_at(handle, TrieNodeWithSize::new(leaf_node, memory_usage)); + } else { + memory.store_at(handle, TrieNodeWithSize::empty()); } + } else if num_children == 1 && value.is_none() { + // Branch with one child becomes extension + // Extension followed by leaf becomes leaf + // Extension followed by extension becomes extension + let idx = children.iter().next().unwrap().0; + let child = children[idx].take().unwrap(); + let key = + NibbleSlice::new(&[(idx << 4) as u8]).encoded_leftmost(1, false).into_vec(); + self.squash_extension_node(memory, handle, key, child)?; + } else { + let node = + TrieNodeWithSize::new(TrieNode::Branch(children, value), memory_usage); + memory.store_at(handle, node); } - } else { - memory.store_at(handle, TrieNodeWithSize { node, memory_usage }); } - child_memory_usage = memory.node_ref(handle).memory_usage; + TrieNode::Extension(key, child) => { + self.squash_extension_node(memory, handle, key, child)?; + } } Ok(()) } - fn fix_extension_node( + fn squash_extension_node( &self, memory: &mut NodesStorage, handle: StorageHandle, @@ -603,7 +604,7 @@ impl Trie { i += 1; } let new_value = - value.clone().map(|value| Trie::flatten_value(&mut memory, value)); + (*value).map(|value| Trie::flatten_value(&mut memory, value)); RawTrieNode::branch(*new_children, new_value) } FlattenNodesCrumb::Exiting => unreachable!(), @@ -622,7 +623,7 @@ impl Trie { }, TrieNode::Leaf(key, value) => { let key = key.clone(); - let value = value.clone(); + let value = *value; let value = Trie::flatten_value(&mut memory, value); RawTrieNode::Leaf(key, value) } diff --git a/core/store/src/trie/mem/resharding.rs b/core/store/src/trie/mem/resharding.rs index e37c778188b..e29047106be 100644 --- a/core/store/src/trie/mem/resharding.rs +++ b/core/store/src/trie/mem/resharding.rs @@ -1,10 +1,17 @@ -use crate::{NibbleSlice, TrieChanges}; +use crate::trie::insert_delete::NodesStorage; +#[cfg(test)] +use crate::trie::NodeHandle; +use crate::{NibbleSlice, Trie, TrieChanges}; use super::arena::ArenaMemory; -use super::updating::{MemTrieUpdate, OldOrUpdatedNodeId, TrieAccesses, UpdatedMemTrieNode}; +use super::updating::{ + GenericNodeOrIndex, GenericTrieUpdate, GenericUpdatedNodeId, GenericUpdatedTrieNode, + GenericUpdatedTrieNodeWithSize, HasValueLength, MemTrieUpdate, TrieAccesses, +}; use itertools::Itertools; +use near_primitives::errors::StorageError; use near_primitives::trie_key::col::COLUMNS_WITH_ACCOUNT_ID_IN_KEY; -use near_primitives::types::AccountId; +use near_primitives::types::{AccountId, StateRoot}; use std::ops::Range; #[derive(Debug)] @@ -25,6 +32,38 @@ enum RetainDecision { Descend, } +/// By the boundary account and the retain mode, generates the list of ranges +/// to be retained in trie. +fn boundary_account_to_intervals( + boundary_account: &AccountId, + retain_mode: RetainMode, +) -> Vec>> { + let mut intervals = vec![]; + // TODO(#12074): generate correct intervals in nibbles. + for (col, _) in COLUMNS_WITH_ACCOUNT_ID_IN_KEY { + match retain_mode { + RetainMode::Left => { + intervals.push(vec![col]..[&[col], boundary_account.as_bytes()].concat()) + } + RetainMode::Right => { + intervals.push([&[col], boundary_account.as_bytes()].concat()..vec![col + 1]) + } + } + } + intervals +} + +/// Converts the list of ranges in bytes to the list of ranges in nibbles. +fn intervals_to_nibbles(intervals: &[Range>]) -> Vec>> { + intervals + .iter() + .map(|range| { + NibbleSlice::new(&range.start).iter().collect_vec() + ..NibbleSlice::new(&range.end).iter().collect_vec() + }) + .collect_vec() +} + impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> { /// Splits the trie, separating entries by the boundary account. /// Leaves the left or right part of the trie, depending on the retain mode. @@ -37,18 +76,7 @@ impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> { boundary_account: &AccountId, retain_mode: RetainMode, ) -> (TrieChanges, TrieAccesses) { - let mut intervals = vec![]; - // TODO(#12074): generate correct intervals in nibbles. - for (col, _) in COLUMNS_WITH_ACCOUNT_ID_IN_KEY { - match retain_mode { - RetainMode::Left => { - intervals.push(vec![col]..[&[col], boundary_account.as_bytes()].concat()) - } - RetainMode::Right => { - intervals.push([&[col], boundary_account.as_bytes()].concat()..vec![col + 1]) - } - } - } + let intervals = boundary_account_to_intervals(boundary_account, retain_mode); self.retain_multi_range(&intervals) } @@ -59,20 +87,46 @@ impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> { /// retain operation. fn retain_multi_range(mut self, intervals: &[Range>]) -> (TrieChanges, TrieAccesses) { debug_assert!(intervals.iter().all(|range| range.start < range.end)); - let intervals_nibbles = intervals - .iter() - .map(|range| { - NibbleSlice::new(&range.start).iter().collect_vec() - ..NibbleSlice::new(&range.end).iter().collect_vec() - }) - .collect_vec(); + let intervals_nibbles = intervals_to_nibbles(intervals); // TODO(#12074): consider handling the case when no changes are made. // TODO(#12074): restore proof as well. - self.retain_multi_range_recursive(0, vec![], &intervals_nibbles); + self.generic_retain_multi_range_recursive(0, vec![], &intervals_nibbles).unwrap(); self.to_trie_changes() } +} +impl Trie { + pub fn retain_split_shard( + &self, + boundary_account: &AccountId, + retain_mode: RetainMode, + ) -> Result { + let intervals = boundary_account_to_intervals(boundary_account, retain_mode); + self.retain_multi_range(&intervals) + } + + fn retain_multi_range(&self, intervals: &[Range>]) -> Result { + debug_assert!(intervals.iter().all(|range| range.start < range.end)); + let intervals_nibbles = intervals_to_nibbles(intervals); + + let mut memory = NodesStorage::new(&self); + let root_node = self.move_node_to_mutable(&mut memory, &self.root)?; + + memory.generic_retain_multi_range_recursive(0, vec![], &intervals_nibbles).unwrap(); + + #[cfg(test)] + { + self.memory_usage_verify(&memory, NodeHandle::InMemory(root_node)); + } + let result = Trie::flatten_nodes(&self.root, memory, root_node)?; + Ok(result.new_root) + } +} + +trait GenericTrieUpdateRetain<'a, N: std::fmt::Debug, V: std::fmt::Debug + HasValueLength>: + GenericTrieUpdate<'a, N, V> +{ /// Recursive implementation of the algorithm of retaining keys belonging to /// any of the ranges given in `intervals` from the trie. All changes are /// applied in `updated_nodes`. @@ -80,89 +134,123 @@ impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> { /// `node_id` is the root of subtree being explored. /// `key_nibbles` is the key corresponding to `root`. /// `intervals_nibbles` is the list of ranges to be retained. - fn retain_multi_range_recursive( + fn generic_retain_multi_range_recursive( &mut self, - node_id: usize, + node_id: GenericUpdatedNodeId, key_nibbles: Vec, intervals_nibbles: &[Range>], - ) { + ) -> Result<(), StorageError> { let decision = retain_decision(&key_nibbles, intervals_nibbles); match decision { - RetainDecision::RetainAll => return, + RetainDecision::RetainAll => return Ok(()), RetainDecision::DiscardAll => { - let _ = self.take_node(node_id); - self.place_node(node_id, UpdatedMemTrieNode::Empty); - return; + let _ = self.generic_take_node(node_id); + self.generic_place_node(node_id, GenericUpdatedTrieNodeWithSize::empty()); + return Ok(()); } RetainDecision::Descend => { // We need to descend into all children. The logic follows below. } } - let node = self.take_node(node_id); + let GenericUpdatedTrieNodeWithSize { node, memory_usage } = self.generic_take_node(node_id); match node { - UpdatedMemTrieNode::Empty => { + GenericUpdatedTrieNode::Empty => { // Nowhere to descend. - self.place_node(node_id, UpdatedMemTrieNode::Empty); - return; + self.generic_place_node(node_id, GenericUpdatedTrieNodeWithSize::empty()); + return Ok(()); } - UpdatedMemTrieNode::Leaf { extension, value } => { + GenericUpdatedTrieNode::Leaf { extension, value } => { let full_key_nibbles = [key_nibbles, NibbleSlice::from_encoded(&extension).0.iter().collect_vec()] .concat(); if !intervals_nibbles.iter().any(|interval| interval.contains(&full_key_nibbles)) { - self.place_node(node_id, UpdatedMemTrieNode::Empty); + self.generic_place_node(node_id, GenericUpdatedTrieNodeWithSize::empty()); } else { - self.place_node(node_id, UpdatedMemTrieNode::Leaf { extension, value }); + self.generic_place_node( + node_id, + GenericUpdatedTrieNodeWithSize { + node: GenericUpdatedTrieNode::Leaf { extension, value }, + memory_usage, + }, + ); } - return; + return Ok(()); } - UpdatedMemTrieNode::Branch { mut children, mut value } => { + GenericUpdatedTrieNode::Branch { mut children, mut value } => { if !intervals_nibbles.iter().any(|interval| interval.contains(&key_nibbles)) { value = None; } + let mut memory_usage = 0; for (i, child) in children.iter_mut().enumerate() { let Some(old_child_id) = child.take() else { continue; }; - let new_child_id = self.ensure_updated(old_child_id); + let new_child_id = self.generic_ensure_updated(old_child_id)?; let child_key_nibbles = [key_nibbles.clone(), vec![i as u8]].concat(); - self.retain_multi_range_recursive( + self.generic_retain_multi_range_recursive( new_child_id, child_key_nibbles, intervals_nibbles, - ); - if self.updated_nodes[new_child_id] == Some(UpdatedMemTrieNode::Empty) { + )?; + + let GenericUpdatedTrieNodeWithSize { node, memory_usage: child_memory_usage } = + self.generic_get_node(new_child_id); + if matches!(node, GenericUpdatedTrieNode::Empty) { *child = None; } else { - *child = Some(OldOrUpdatedNodeId::Updated(new_child_id)); + *child = Some(GenericNodeOrIndex::Updated(new_child_id)); + memory_usage += child_memory_usage; } } - self.place_node(node_id, UpdatedMemTrieNode::Branch { children, value }); + let node = GenericUpdatedTrieNode::Branch { children, value }; + memory_usage += node.memory_usage_direct(); + self.generic_place_node( + node_id, + GenericUpdatedTrieNodeWithSize { node, memory_usage }, + ); } - UpdatedMemTrieNode::Extension { extension, child } => { - let new_child_id = self.ensure_updated(child); + GenericUpdatedTrieNode::Extension { extension, child } => { + let new_child_id = self.generic_ensure_updated(child)?; let extension_nibbles = NibbleSlice::from_encoded(&extension).0.iter().collect_vec(); let child_key = [key_nibbles, extension_nibbles].concat(); - self.retain_multi_range_recursive(new_child_id, child_key, intervals_nibbles); + self.generic_retain_multi_range_recursive( + new_child_id, + child_key, + intervals_nibbles, + )?; - let node = UpdatedMemTrieNode::Extension { + let node = GenericUpdatedTrieNode::Extension { extension, - child: OldOrUpdatedNodeId::Updated(new_child_id), + child: GenericNodeOrIndex::Updated(new_child_id), }; - self.place_node(node_id, node); + let child_memory_usage = self.generic_get_node(new_child_id).memory_usage; + let memory_usage = node.memory_usage_direct() + child_memory_usage; + self.generic_place_node( + node_id, + GenericUpdatedTrieNodeWithSize { node, memory_usage }, + ); } } // We may need to change node type to keep the trie structure unique. - self.squash_node(node_id); + self.generic_squash_node(node_id) } } +impl< + 'a, + N: std::fmt::Debug, + V: std::fmt::Debug + HasValueLength, + T: GenericTrieUpdate<'a, N, V>, + > GenericTrieUpdateRetain<'a, N, V> for T +{ +} + /// Based on the key and the intervals, makes decision on the subtree exploration. fn retain_decision(key: &[u8], intervals: &[Range>]) -> RetainDecision { let mut should_descend = false; @@ -211,18 +299,14 @@ mod tests { use itertools::Itertools; use near_primitives::{shard_layout::ShardUId, types::StateRoot}; - use crate::{ - test_utils::TestTriesBuilder, - trie::{ - mem::{ - iter::MemTrieIterator, - mem_tries::MemTries, - nibbles_utils::{all_two_nibble_nibbles, hex_to_nibbles, multi_hex_to_nibbles}, - }, - trie_storage::TrieMemoryPartialStorage, - }, - Trie, + use crate::test_utils::TestTriesBuilder; + use crate::trie::mem::iter::MemTrieIterator; + use crate::trie::mem::mem_tries::MemTries; + use crate::trie::mem::nibbles_utils::{ + all_two_nibble_nibbles, hex_to_nibbles, multi_hex_to_nibbles, }; + use crate::trie::trie_storage::TrieMemoryPartialStorage; + use crate::trie::Trie; // Logic for a single test. // Creates trie from initial entries, applies retain multi range to it and @@ -241,13 +325,27 @@ mod tests { .iter() .map(|(key, value)| (key.clone(), Some(value.clone()))) .collect_vec(); - let expected_state_root = crate::test_utils::test_populate_trie( + let expected_naive_state_root = crate::test_utils::test_populate_trie( &shard_tries, &Trie::EMPTY_ROOT, ShardUId::single_shard(), changes, ); + let shard_tries = TestTriesBuilder::new().build(); + let initial_changes = initial_entries + .iter() + .map(|(key, value)| (key.clone(), Some(value.clone()))) + .collect_vec(); + let initial_state_root = crate::test_utils::test_populate_trie( + &shard_tries, + &Trie::EMPTY_ROOT, + ShardUId::single_shard(), + initial_changes, + ); + let trie = shard_tries.get_trie_for_shard(ShardUId::single_shard(), initial_state_root); + let expected_disk_state_root = trie.retain_multi_range(&retain_multi_ranges).unwrap(); + let mut memtries = MemTries::new(ShardUId::single_shard()); let mut update = memtries.update(Trie::EMPTY_ROOT, false).unwrap(); for (key, value) in initial_entries { @@ -259,12 +357,12 @@ mod tests { let update = memtries.update(state_root, true).unwrap(); let (mut trie_changes, _) = update.retain_multi_range(&retain_multi_ranges); let memtrie_changes = trie_changes.mem_trie_changes.take().unwrap(); - let new_state_root = memtries.apply_memtrie_changes(1, &memtrie_changes); + let mem_state_root = memtries.apply_memtrie_changes(1, &memtrie_changes); - let entries = if new_state_root != StateRoot::default() { - let state_root_ptr = memtries.get_root(&new_state_root).unwrap(); + let entries = if mem_state_root != StateRoot::default() { + let state_root_ptr = memtries.get_root(&mem_state_root).unwrap(); let trie = - Trie::new(Arc::new(TrieMemoryPartialStorage::default()), new_state_root, None); + Trie::new(Arc::new(TrieMemoryPartialStorage::default()), mem_state_root, None); MemTrieIterator::new(Some(state_root_ptr), &trie).map(|e| e.unwrap()).collect_vec() } else { vec![] @@ -274,7 +372,10 @@ mod tests { assert_eq!(entries, retain_result_naive); // Check state root, because it must be unique. - assert_eq!(new_state_root, expected_state_root); + assert_eq!(mem_state_root, expected_naive_state_root); + + // Check state root with disk-trie state root. + assert_eq!(mem_state_root, expected_disk_state_root); } #[test] diff --git a/core/store/src/trie/mem/updating.rs b/core/store/src/trie/mem/updating.rs index f6718da6020..b8c3a8a9e63 100644 --- a/core/store/src/trie/mem/updating.rs +++ b/core/store/src/trie/mem/updating.rs @@ -1,47 +1,209 @@ +//! Structures and logic for updating in-memory trie. +//! +//! DISCLAIMER: This is in process of rewriting to generic structures. +//! See #12324. +//! For now we keep the old types together with new ones to change logic +//! incrementally (for example, `GenericNodeOrIndex` and `OldOrUpdatedNodeId`). +//! New methods will be prefixed with `generic_` to distinguish them from the +//! old ones. When the old methods are removed, the prefix will be dropped. + use super::arena::{ArenaMemory, ArenaMut}; use super::flexible_data::children::ChildrenView; use super::metrics::MEM_TRIE_NUM_NODES_CREATED_FROM_UPDATES; use super::node::{InputMemTrieNode, MemTrieNodeId, MemTrieNodeView}; -use crate::trie::{Children, MemTrieChanges, TrieRefcountDeltaMap, TRIE_COSTS}; +use crate::trie::insert_delete::NodesStorage; +use crate::trie::{ + Children, MemTrieChanges, NodeHandle, StorageHandle, TrieNode, TrieNodeWithSize, + TrieRefcountDeltaMap, ValueHandle, TRIE_COSTS, +}; use crate::{NibbleSlice, RawTrieNode, RawTrieNodeWithSize, TrieChanges}; +use near_primitives::errors::StorageError; use near_primitives::hash::{hash, CryptoHash}; use near_primitives::state::FlatStateValue; use std::collections::HashMap; use std::sync::Arc; +/// For updated nodes, the ID is simply the index into the array of updated nodes we keep. +pub type GenericUpdatedNodeId = usize; + +pub type UpdatedMemTrieNodeId = usize; + /// An old node means a node in the current in-memory trie. An updated node means a /// node we're going to store in the in-memory trie but have not constructed there yet. #[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub enum OldOrUpdatedNodeId { - Old(MemTrieNodeId), - Updated(UpdatedMemTrieNodeId), +pub enum GenericNodeOrIndex { + Old(GenericTrieNodePtr), + Updated(GenericUpdatedNodeId), } -/// For updated nodes, the ID is simply the index into the array of updated nodes we keep. -pub type UpdatedMemTrieNodeId = usize; +pub type OldOrUpdatedNodeId = GenericNodeOrIndex; -/// An updated node - a node that will eventually become an in-memory trie node. +/// Trait for trie values to get their length. +pub trait HasValueLength { + fn len(&self) -> u64; +} + +impl HasValueLength for FlatStateValue { + fn len(&self) -> u64 { + self.value_len() as u64 + } +} + +impl HasValueLength for ValueHandle { + fn len(&self) -> u64 { + match self { + ValueHandle::HashAndSize(value) => value.length as u64, + ValueHandle::InMemory(value) => value.1 as u64, + } + } +} + +/// An updated node - a node that will eventually become a trie node. /// It references children that are either old or updated nodes. #[derive(Debug, Clone, PartialEq, Eq)] -pub enum UpdatedMemTrieNode { +pub enum GenericUpdatedTrieNode { /// Used for either an empty root node (indicating an empty trie), or as a temporary /// node to ease implementation. Empty, Leaf { extension: Box<[u8]>, - value: FlatStateValue, + value: GenericValueHandle, }, Extension { extension: Box<[u8]>, - child: OldOrUpdatedNodeId, + child: GenericNodeOrIndex, }, /// Corresponds to either a Branch or BranchWithValue node. Branch { - children: Box<[Option; 16]>, - value: Option, + children: Box<[Option>; 16]>, + value: Option, }, } +pub type UpdatedMemTrieNode = GenericUpdatedTrieNode; + +/// An updated node with its memory usage. +/// Needed to recompute subtree function (memory usage) on the fly. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct GenericUpdatedTrieNodeWithSize { + pub node: GenericUpdatedTrieNode, + pub memory_usage: u64, +} + +impl GenericUpdatedTrieNodeWithSize { + pub fn empty() -> Self { + Self { node: GenericUpdatedTrieNode::Empty, memory_usage: 0 } + } +} + +impl + GenericUpdatedTrieNode +{ + fn memory_usage_value(value_length: u64) -> u64 { + value_length * TRIE_COSTS.byte_of_value + TRIE_COSTS.node_cost + } + + /// Returns the memory usage of the **single** node, in Near's trie cost + /// terms, not in terms of the physical memory usage. + /// TODO(#12324): replace `TrieNode::memory_usage_direct_internal` and + ///`MemTrieNodeView::memory_usage` by this method. + pub fn memory_usage_direct(&self) -> u64 { + match self { + Self::Empty => { + // DEVNOTE: empty nodes don't exist in storage. + // In the in-memory implementation Some(TrieNode::Empty) and None are interchangeable as + // children of branch nodes which means cost has to be 0 + 0 + } + Self::Leaf { extension, value } => { + TRIE_COSTS.node_cost + + (extension.len() as u64) * TRIE_COSTS.byte_of_key + + Self::memory_usage_value(value.len()) + } + Self::Branch { value, .. } => { + TRIE_COSTS.node_cost + + value.as_ref().map_or(0, |value| Self::memory_usage_value(value.len())) + } + Self::Extension { extension, .. } => { + TRIE_COSTS.node_cost + (extension.len() as u64) * TRIE_COSTS.byte_of_key + } + } + } +} + +pub type UpdatedMemTrieNodeWithSize = GenericUpdatedTrieNodeWithSize; + +/// Trait for trie updates to handle updated nodes. +/// +/// So far, this is used to handle key-value insertions, deletions and range +/// retain operation. To be performant, such logic requires keeping track of +/// intermediate updated nodes, together with subtree function (memory usage). +/// +/// `GenericTrieUpdate` abstracts the storage of updated nodes for the original +/// node type `GenericTrieNodePtr`. +/// +/// In this storage, nodes are indexed by `GenericUpdatedNodeId`. +/// Node is stored as `GenericUpdatedTrieNodeWithSize`, which stores children +/// as `GenericNodeOrIndex`. Each child may be either an old node or an updated +/// node. +/// +/// The flow of interaction with this storage is: +/// - In the beginning, call `ensure_updated` for the +/// `GenericNodeOrIndex::Old(root_node)` which returns `GenericUpdatedNodeId`, +/// it should be zero. +/// - For every update (single insert, single delete, recursive range +/// operation...), call corresponding method with `GenericUpdatedNodeId` for +/// the root. +/// - Then, we hold the invariant that on every descent we have +/// `GenericUpdatedNodeId`. +/// - So, first, we call `take_node` to get `GenericUpdatedTrieNodeWithSize` +/// back; +/// - We possibly descend into its children and modify the node; +/// - Then, we call `place_node` to put the node back and return to the +/// node parent. +/// - Finally, we end up with storage of new nodes, which are used to produce +/// new state root. The exact logic depends on trait implementation. +/// +/// TODO(#12324): instead of `GenericValueHandle`, consider always using +/// `FlatStateValue`. +/// +/// Note that it has nothing to do with `TrieUpdate` used for runtime to store +/// temporary state changes (TODO(#12324) - consider renaming it). +pub(crate) trait GenericTrieUpdate<'a, GenericTrieNodePtr, GenericValueHandle> { + /// If the ID was old, converts underlying node to an updated one. + fn generic_ensure_updated( + &mut self, + node: GenericNodeOrIndex, + ) -> Result; + + /// Takes a node from the set of updated nodes, setting it to None. + /// It is expected that place_node is then called to return the node to + /// the same slot. + fn generic_take_node( + &mut self, + node_id: GenericUpdatedNodeId, + ) -> GenericUpdatedTrieNodeWithSize; + + /// Puts a node to the set of updated nodes. + fn generic_place_node( + &mut self, + node_id: GenericUpdatedNodeId, + node: GenericUpdatedTrieNodeWithSize, + ); + + /// Gets a node from the set of updated nodes. + /// TODO(#12324): we actually should get a reference, but type + /// incompatibility don't allow it for now. + fn generic_get_node( + &self, + node_id: GenericUpdatedNodeId, + ) -> GenericUpdatedTrieNodeWithSize; + + /// Squashes a node to ensure uniqueness of the trie structure. + /// TODO(#12324): should be implemented using the methods above. + fn generic_squash_node(&mut self, node_id: GenericUpdatedNodeId) -> Result<(), StorageError>; +} + /// Keeps values and internal nodes accessed on updating memtrie. pub struct TrieAccesses { /// Hashes and encoded trie nodes. @@ -116,6 +278,166 @@ impl UpdatedMemTrieNode { } } +impl<'a, M: ArenaMemory> GenericTrieUpdate<'a, MemTrieNodeId, FlatStateValue> + for MemTrieUpdate<'a, M> +{ + fn generic_ensure_updated( + &mut self, + node: GenericNodeOrIndex, + ) -> Result { + Ok(self.ensure_updated(node)) + } + + fn generic_take_node(&mut self, index: UpdatedMemTrieNodeId) -> UpdatedMemTrieNodeWithSize { + // TODO(#12324): IMPORTANT: now, we don't compute memory usage on the + // fly for memtries. This happens in `compute_hashes_and_serialized_nodes`. + // Memory usages here are zeroed and ignored. + // However, this is fundamentally wrong because the current approach + // needs ALL children of any changed branch in memtrie. In reality, it + // is enough to have only children that are changed. + // So, we need to change `MemTrieUpdate` to store current memory usages + // and retrieve them correctly. + UpdatedMemTrieNodeWithSize { node: self.take_node(index), memory_usage: 0 } + } + + fn generic_place_node( + &mut self, + index: UpdatedMemTrieNodeId, + node: UpdatedMemTrieNodeWithSize, + ) { + self.place_node(index, node.node); + } + + fn generic_get_node(&self, node_id: GenericUpdatedNodeId) -> UpdatedMemTrieNodeWithSize { + UpdatedMemTrieNodeWithSize { + node: self.updated_nodes[node_id].as_ref().unwrap().clone(), + memory_usage: 0, + } + } + + fn generic_squash_node(&mut self, node_id: GenericUpdatedNodeId) -> Result<(), StorageError> { + self.squash_node(node_id); + Ok(()) + } +} + +pub(crate) type TrieStorageNodePtr = CryptoHash; + +pub(crate) type UpdatedTrieStorageNode = GenericUpdatedTrieNode; + +pub(crate) type UpdatedTrieStorageNodeWithSize = + GenericUpdatedTrieNodeWithSize; + +/// Conversion between updated node for trie storage and generic updated node. +/// TODO(#12324): remove once the whole trie storage logic is rewritten in +/// generic terms. +impl UpdatedTrieStorageNode { + pub fn from_trie_node_with_size(node: TrieNodeWithSize) -> Self { + match node.node { + TrieNode::Empty => Self::Empty, + TrieNode::Leaf(extension, value) => { + Self::Leaf { extension: extension.to_vec().into_boxed_slice(), value } + } + TrieNode::Branch(children, value) => Self::Branch { + children: Box::new(children.0.map(|child| { + child.map(|id| match id { + NodeHandle::Hash(id) => GenericNodeOrIndex::Old(id), + NodeHandle::InMemory(id) => GenericNodeOrIndex::Updated(id.0), + }) + })), + value, + }, + TrieNode::Extension(extension, child) => Self::Extension { + extension: extension.to_vec().into_boxed_slice(), + child: match child { + NodeHandle::Hash(id) => GenericNodeOrIndex::Old(id), + NodeHandle::InMemory(id) => GenericNodeOrIndex::Updated(id.0), + }, + }, + } + } + + pub fn into_trie_node_with_size(self, memory_usage: u64) -> TrieNodeWithSize { + match self { + Self::Empty => TrieNodeWithSize { node: TrieNode::Empty, memory_usage }, + Self::Leaf { extension, value } => { + TrieNodeWithSize { node: TrieNode::Leaf(extension.into_vec(), value), memory_usage } + } + Self::Branch { children, value } => TrieNodeWithSize { + node: TrieNode::Branch( + Box::new(Children(children.map(|child| { + child.map(|id| match id { + GenericNodeOrIndex::Old(id) => NodeHandle::Hash(id), + GenericNodeOrIndex::Updated(id) => { + NodeHandle::InMemory(StorageHandle(id)) + } + }) + }))), + value, + ), + memory_usage, + }, + Self::Extension { extension, child } => TrieNodeWithSize { + node: TrieNode::Extension( + extension.into_vec(), + match child { + GenericNodeOrIndex::Old(id) => NodeHandle::Hash(id), + GenericNodeOrIndex::Updated(id) => NodeHandle::InMemory(StorageHandle(id)), + }, + ), + memory_usage, + }, + } + } +} + +impl<'a> GenericTrieUpdate<'a, TrieStorageNodePtr, ValueHandle> for NodesStorage<'a> { + fn generic_ensure_updated( + &mut self, + node: GenericNodeOrIndex, + ) -> Result { + match node { + GenericNodeOrIndex::Old(node_hash) => { + self.trie.move_node_to_mutable(self, &node_hash).map(|handle| handle.0) + } + GenericNodeOrIndex::Updated(node_id) => Ok(node_id), + } + } + + fn generic_take_node(&mut self, index: GenericUpdatedNodeId) -> UpdatedTrieStorageNodeWithSize { + let node = self.destroy(StorageHandle(index)); + let memory_usage = node.memory_usage; + UpdatedTrieStorageNodeWithSize { + node: UpdatedTrieStorageNode::from_trie_node_with_size(node), + memory_usage, + } + } + + fn generic_place_node( + &mut self, + index: GenericUpdatedNodeId, + node: UpdatedTrieStorageNodeWithSize, + ) { + let UpdatedTrieStorageNodeWithSize { node, memory_usage } = node; + let node = node.into_trie_node_with_size(memory_usage); + self.store_at(StorageHandle(index), node); + } + + fn generic_get_node(&self, index: GenericUpdatedNodeId) -> UpdatedTrieStorageNodeWithSize { + let node = self.node_ref(StorageHandle(index)).clone(); + let memory_usage = node.memory_usage; + UpdatedTrieStorageNodeWithSize { + node: UpdatedTrieStorageNode::from_trie_node_with_size(node), + memory_usage, + } + } + + fn generic_squash_node(&mut self, index: GenericUpdatedNodeId) -> Result<(), StorageError> { + let trie = self.trie; + trie.squash_node(self, StorageHandle(index)) + } +} + impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> { pub fn new( root: Option, @@ -190,7 +512,6 @@ impl<'a, M: ArenaMemory> MemTrieUpdate<'a, M> { } } - /// If the ID was old, converts it to an updated one. pub(crate) fn ensure_updated(&mut self, node: OldOrUpdatedNodeId) -> UpdatedMemTrieNodeId { match node { OldOrUpdatedNodeId::Old(node_id) => self.convert_existing_to_updated(Some(node_id)), diff --git a/core/store/src/trie/mod.rs b/core/store/src/trie/mod.rs index ee4a34d2028..23994365748 100644 --- a/core/store/src/trie/mod.rs +++ b/core/store/src/trie/mod.rs @@ -72,8 +72,10 @@ pub struct PartialStorage { #[derive(Clone, Hash, Debug, Copy)] pub(crate) struct StorageHandle(usize); +/// Stores index of value in the array of new values and its length for memory +/// counting. #[derive(Clone, Hash, Debug, Copy)] -pub(crate) struct StorageValueHandle(usize); +pub(crate) struct StorageValueHandle(usize, usize); pub struct TrieCosts { pub byte_of_key: u64, @@ -114,8 +116,8 @@ impl std::fmt::Debug for NodeHandle { } } -#[derive(Clone, Hash)] -enum ValueHandle { +#[derive(Clone, Copy, Hash)] +pub(crate) enum ValueHandle { InMemory(StorageValueHandle), HashAndSize(ValueRef), } @@ -124,7 +126,7 @@ impl std::fmt::Debug for ValueHandle { fn fmt(&self, fmtr: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { Self::HashAndSize(value) => write!(fmtr, "{value:?}"), - Self::InMemory(StorageValueHandle(num)) => write!(fmtr, "@{num}"), + Self::InMemory(StorageValueHandle(num, _)) => write!(fmtr, "@{num}"), } } } @@ -874,7 +876,7 @@ impl Trie { eprintln!("Computed is {}", memory_usage); match handle { NodeHandle::InMemory(h) => { - eprintln!("TRIE!!!!"); + eprintln!("In-memory node:"); eprintln!("{}", memory.node_ref(h).node.deep_to_string(memory)); } NodeHandle::Hash(_h) => { @@ -1226,7 +1228,7 @@ impl Trie { } } - fn move_node_to_mutable( + pub(crate) fn move_node_to_mutable( &self, memory: &mut NodesStorage, hash: &CryptoHash, @@ -1665,7 +1667,7 @@ impl Trie { Ok(trie_changes) } None => { - let mut memory = NodesStorage::new(); + let mut memory = NodesStorage::new(&self); let mut root_node = self.move_node_to_mutable(&mut memory, &self.root)?; for (key, value) in changes { let key = NibbleSlice::new(&key); diff --git a/core/store/src/trie/raw_node.rs b/core/store/src/trie/raw_node.rs index b55f7c270df..7c1b715d13e 100644 --- a/core/store/src/trie/raw_node.rs +++ b/core/store/src/trie/raw_node.rs @@ -134,7 +134,7 @@ fn test_encode_decode() { } let value = ValueRef { length: 3, hash: CryptoHash::hash_bytes(&[123, 245, 255]) }; - let node = RawTrieNode::Leaf(vec![1, 2, 3], value.clone()); + let node = RawTrieNode::Leaf(vec![1, 2, 3], value); #[rustfmt::skip] let encoded = [ /* node type: */ 0,